1
0

[RFC-33] [HUDI-2429][Stacked on HUDI-2560] Support full Schema evolution for Spark (#4910)

* [HUDI-2560] introduce id_based schema to support full schema evolution.

* add test for FileBasedInternalSchemaStorageManger and rebase code

* add support for change column type and fix some test case

* fix some bugs encountered in the production env and delete useless code

* fix test error

* rebase code

* fixed some nested schema change bugs

* [HUDI-2429][Stacked On HUDI-2560]Support full schema evolution for spark

* [use dummyInternalSchema instead of null]

* add support for spark3.1.x

* remove support for spark3.1.x , sicne some compile fail

* support spark3.1.x

* rebase and prepare solve all comments

* address all comments

* rebase code

* fixed the count(*) bug

* try to get internalSchema by parser commit file/history file directly, not use metaclient which is time cost
address some comments

* fixed all comments

* fix new comments

* rebase code,fix UT failed

* fixed mistake

* rebase code ,fixed new comments

* rebase code , and prepare for address new comments

* address commits

* address new comments

* fix new issues

* control fallback original write logical
This commit is contained in:
xiarixiaoyao
2022-04-02 04:20:24 +08:00
committed by GitHub
parent 9275b8fc7e
commit 444ff496a4
89 changed files with 10352 additions and 106 deletions

View File

@@ -19,6 +19,8 @@
package org.apache.hudi.avro;
import org.apache.avro.AvroRuntimeException;
import org.apache.avro.SchemaCompatibility;
import org.apache.avro.Conversions;
import org.apache.avro.Conversions.DecimalConversion;
import org.apache.avro.JsonProperties;
import org.apache.avro.LogicalTypes;
@@ -55,16 +57,26 @@ import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.sql.Timestamp;
import java.time.LocalDate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import java.util.TimeZone;
import java.util.Iterator;
import java.util.stream.Collectors;
import static org.apache.avro.Schema.Type.UNION;
/**
* Helper class to do common stuff across Avro.
*/
@@ -74,6 +86,11 @@ public class HoodieAvroUtils {
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
private static final long MILLIS_PER_DAY = 86400000L;
//Export for test
public static final Conversions.DecimalConversion DECIMAL_CONVERSION = new Conversions.DecimalConversion();
// As per https://avro.apache.org/docs/current/spec.html#names
private static String INVALID_AVRO_CHARS_IN_NAMES = "[^A-Za-z0-9_]";
private static String INVALID_AVRO_FIRST_CHAR_IN_NAMES = "[^A-Za-z_]";
@@ -655,4 +672,271 @@ public class HoodieAvroUtils {
return nonNullType;
}
/**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new schema.
* support deep rewrite for nested record.
* This particular method does the following things :
* a) Create a new empty GenericRecord with the new schema.
* b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this transformed schema
*
* @param oldRecord oldRecord to be rewritten
* @param newSchema newSchema used to rewrite oldRecord
* @return newRecord for new Schema
*/
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema) {
Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema);
return (GenericData.Record) newRecord;
}
private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema) {
if (oldRecord == null) {
return null;
}
switch (newSchema.getType()) {
case RECORD:
if (!(oldRecord instanceof IndexedRecord)) {
throw new IllegalArgumentException("cannot rewrite record with different type");
}
IndexedRecord indexedRecord = (IndexedRecord) oldRecord;
List<Schema.Field> fields = newSchema.getFields();
Map<Integer, Object> helper = new HashMap<>();
for (int i = 0; i < fields.size(); i++) {
Schema.Field field = fields.get(i);
if (oldSchema.getField(field.name()) != null) {
Schema.Field oldField = oldSchema.getField(field.name());
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema()));
}
}
GenericData.Record newRecord = new GenericData.Record(newSchema);
for (int i = 0; i < fields.size(); i++) {
if (helper.containsKey(i)) {
newRecord.put(i, helper.get(i));
} else {
if (fields.get(i).defaultVal() instanceof JsonProperties.Null) {
newRecord.put(i, null);
} else {
newRecord.put(i, fields.get(i).defaultVal());
}
}
}
return newRecord;
case ARRAY:
if (!(oldRecord instanceof Collection)) {
throw new IllegalArgumentException("cannot rewrite record with different type");
}
Collection array = (Collection)oldRecord;
List<Object> newArray = new ArrayList();
for (Object element : array) {
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType()));
}
return newArray;
case MAP:
if (!(oldRecord instanceof Map)) {
throw new IllegalArgumentException("cannot rewrite record with different type");
}
Map<Object, Object> map = (Map<Object, Object>) oldRecord;
Map<Object, Object> newMap = new HashMap<>();
for (Map.Entry<Object, Object> entry : map.entrySet()) {
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType()));
}
return newMap;
case UNION:
return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord));
default:
return rewritePrimaryType(oldRecord, oldSchema, newSchema);
}
}
private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schema newSchema) {
Schema realOldSchema = oldSchema;
if (realOldSchema.getType() == UNION) {
realOldSchema = getActualSchemaFromUnion(oldSchema, oldValue);
}
if (realOldSchema.getType() == newSchema.getType()) {
switch (realOldSchema.getType()) {
case NULL:
case BOOLEAN:
case INT:
case LONG:
case FLOAT:
case DOUBLE:
case BYTES:
case STRING:
return oldValue;
case FIXED:
// fixed size and name must match:
if (!SchemaCompatibility.schemaNameEquals(realOldSchema, newSchema) || realOldSchema.getFixedSize() != newSchema.getFixedSize()) {
// deal with the precision change for decimalType
if (realOldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
final byte[] bytes;
bytes = ((GenericFixed) oldValue).bytes();
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) realOldSchema.getLogicalType();
BigDecimal bd = new BigDecimal(new BigInteger(bytes), decimal.getScale()).setScale(((LogicalTypes.Decimal) newSchema.getLogicalType()).getScale());
return DECIMAL_CONVERSION.toFixed(bd, newSchema, newSchema.getLogicalType());
}
} else {
return oldValue;
}
return oldValue;
default:
throw new AvroRuntimeException("Unknown schema type: " + newSchema.getType());
}
} else {
return rewritePrimaryTypeWithDiffSchemaType(oldValue, realOldSchema, newSchema);
}
}
private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Schema oldSchema, Schema newSchema) {
switch (newSchema.getType()) {
case NULL:
case BOOLEAN:
break;
case INT:
if (newSchema.getLogicalType() == LogicalTypes.date() && oldSchema.getType() == Schema.Type.STRING) {
return fromJavaDate(java.sql.Date.valueOf(oldValue.toString()));
}
break;
case LONG:
if (oldSchema.getType() == Schema.Type.INT) {
return ((Integer) oldValue).longValue();
}
break;
case FLOAT:
if ((oldSchema.getType() == Schema.Type.INT)
|| (oldSchema.getType() == Schema.Type.LONG)) {
return oldSchema.getType() == Schema.Type.INT ? ((Integer) oldValue).floatValue() : ((Long) oldValue).floatValue();
}
break;
case DOUBLE:
if (oldSchema.getType() == Schema.Type.FLOAT) {
// java float cannot convert to double directly, deal with float precision change
return Double.valueOf(oldValue + "");
} else if (oldSchema.getType() == Schema.Type.INT) {
return ((Integer) oldValue).doubleValue();
} else if (oldSchema.getType() == Schema.Type.LONG) {
return ((Long) oldValue).doubleValue();
}
break;
case BYTES:
if (oldSchema.getType() == Schema.Type.STRING) {
return (oldValue.toString()).getBytes(StandardCharsets.UTF_8);
}
break;
case STRING:
if (oldSchema.getType() == Schema.Type.BYTES) {
return String.valueOf(((byte[]) oldValue));
}
if (oldSchema.getLogicalType() == LogicalTypes.date()) {
return toJavaDate((Integer) oldValue).toString();
}
if (oldSchema.getType() == Schema.Type.INT
|| oldSchema.getType() == Schema.Type.LONG
|| oldSchema.getType() == Schema.Type.FLOAT
|| oldSchema.getType() == Schema.Type.DOUBLE) {
return oldValue.toString();
}
if (oldSchema.getType() == Schema.Type.FIXED && oldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
final byte[] bytes;
bytes = ((GenericFixed) oldValue).bytes();
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) oldSchema.getLogicalType();
BigDecimal bd = new BigDecimal(new BigInteger(bytes), decimal.getScale());
return bd.toString();
}
break;
case FIXED:
// deal with decimal Type
if (newSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
// TODO: support more types
if (oldSchema.getType() == Schema.Type.STRING
|| oldSchema.getType() == Schema.Type.DOUBLE
|| oldSchema.getType() == Schema.Type.INT
|| oldSchema.getType() == Schema.Type.LONG
|| oldSchema.getType() == Schema.Type.FLOAT) {
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) newSchema.getLogicalType();
BigDecimal bigDecimal = null;
if (oldSchema.getType() == Schema.Type.STRING) {
bigDecimal = new java.math.BigDecimal(oldValue.toString())
.setScale(decimal.getScale());
} else {
// Due to Java, there will be precision problems in direct conversion, we should use string instead of use double
bigDecimal = new java.math.BigDecimal(oldValue.toString())
.setScale(decimal.getScale());
}
return DECIMAL_CONVERSION.toFixed(bigDecimal, newSchema, newSchema.getLogicalType());
}
}
break;
default:
}
throw new AvroRuntimeException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema));
}
// convert days to Date
private static java.sql.Date toJavaDate(int days) {
long localMillis = Math.multiplyExact(days, MILLIS_PER_DAY);
int timeZoneOffset;
TimeZone defaultTimeZone = TimeZone.getDefault();
if (defaultTimeZone instanceof sun.util.calendar.ZoneInfo) {
timeZoneOffset = ((sun.util.calendar.ZoneInfo) defaultTimeZone).getOffsetsByWall(localMillis, null);
} else {
timeZoneOffset = defaultTimeZone.getOffset(localMillis - defaultTimeZone.getRawOffset());
}
return new java.sql.Date(localMillis - timeZoneOffset);
}
// convert Date to days
private static int fromJavaDate(Date date) {
long millisUtc = date.getTime();
long millisLocal = millisUtc + TimeZone.getDefault().getOffset(millisUtc);
int julianDays = Math.toIntExact(Math.floorDiv(millisLocal, MILLIS_PER_DAY));
return julianDays;
}
private static Schema getActualSchemaFromUnion(Schema schema, Object data) {
Schema actualSchema;
if (!schema.getType().equals(UNION)) {
return schema;
}
if (schema.getTypes().size() == 2
&& schema.getTypes().get(0).getType() == Schema.Type.NULL) {
actualSchema = schema.getTypes().get(1);
} else if (schema.getTypes().size() == 2
&& schema.getTypes().get(1).getType() == Schema.Type.NULL) {
actualSchema = schema.getTypes().get(0);
} else if (schema.getTypes().size() == 1) {
actualSchema = schema.getTypes().get(0);
} else {
// deal complex union. this should not happened in hoodie,
// since flink/spark do not write this type.
int i = GenericData.get().resolveUnion(schema, data);
actualSchema = schema.getTypes().get(i);
}
return actualSchema;
}
/**
* Given avro records, rewrites them with new schema.
*
* @param oldRecords oldRecords to be rewrite
* @param newSchema newSchema used to rewrite oldRecord
* @return a iterator of rewrote GeneriRcords
*/
public static Iterator<GenericRecord> rewriteRecordWithNewSchema(Iterator<GenericRecord> oldRecords, Schema newSchema) {
if (oldRecords == null || newSchema == null) {
return Collections.emptyIterator();
}
return new Iterator<GenericRecord>() {
@Override
public boolean hasNext() {
return oldRecords.hasNext();
}
@Override
public GenericRecord next() {
return rewriteRecordWithNewSchema(oldRecords.next(), newSchema);
}
};
}
}

View File

@@ -50,6 +50,10 @@ public enum WriteOperationType {
COMPACT("compact"),
INDEX("index"),
// alter schema
ALTER_SCHEMA("alter_schema"),
// used for old version
UNKNOWN("unknown");
@@ -90,6 +94,8 @@ public enum WriteOperationType {
return COMPACT;
case "index":
return INDEX;
case "alter_schema":
return ALTER_SCHEMA;
case "unknown":
return UNKNOWN;
default:

View File

@@ -90,6 +90,8 @@ public class HoodieTableMetaClient implements Serializable {
public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR
+ ".fileids";
public static final String SCHEMA_FOLDER_NAME = ".schema";
public static final String MARKER_EXTN = ".marker";
private String basePath;
@@ -192,6 +194,13 @@ public class HoodieTableMetaClient implements Serializable {
return new Path(metaPath, COLUMN_STATISTICS_INDEX_NAME).toString();
}
/**
* @return schema folder path
*/
public String getSchemaFolderName() {
return new Path(metaPath, SCHEMA_FOLDER_NAME).toString();
}
/**
* @return Temp Folder path
*/
@@ -392,6 +401,11 @@ public class HoodieTableMetaClient implements Serializable {
if (!fs.exists(metaPathDir)) {
fs.mkdirs(metaPathDir);
}
// create schema folder
Path schemaPathDir = new Path(metaPathDir, SCHEMA_FOLDER_NAME);
if (!fs.exists(schemaPathDir)) {
fs.mkdirs(schemaPathDir);
}
// if anything other than default archive log folder is specified, create that too
String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER);

View File

@@ -18,13 +18,6 @@
package org.apache.hudi.common.table;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.SchemaCompatibility;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
@@ -44,8 +37,18 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hudi.io.storage.HoodieHFileReader;
import org.apache.hudi.io.storage.HoodieOrcReader;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
import org.apache.hudi.internal.schema.utils.SerDeHelper;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.SchemaCompatibility;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroSchemaConverter;
@@ -534,4 +537,51 @@ public class TableSchemaResolver {
return false;
}
}
/**
* Gets the InternalSchema for a hoodie table from the HoodieCommitMetadata of the instant.
*
* @return InternalSchema for this table
*/
public Option<InternalSchema> getTableInternalSchemaFromCommitMetadata() {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
if (timeline.lastInstant().isPresent()) {
return getTableInternalSchemaFromCommitMetadata(timeline.lastInstant().get());
} else {
return Option.empty();
}
}
/**
* Gets the InternalSchema for a hoodie table from the HoodieCommitMetadata of the instant.
*
* @return InternalSchema for this table
*/
private Option<InternalSchema> getTableInternalSchemaFromCommitMetadata(HoodieInstant instant) {
try {
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedInstants();
byte[] data = timeline.getInstantDetails(instant).get();
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
if (latestInternalSchemaStr != null) {
return SerDeHelper.fromJson(latestInternalSchemaStr);
} else {
return Option.empty();
}
} catch (Exception e) {
throw new HoodieException("Failed to read schema from commit metadata", e);
}
}
/**
* Gets the history schemas as String for a hoodie table from the HoodieCommitMetadata of the instant.
*
* @return history schemas string for this table
*/
public Option<String> getTableHistorySchemaStrFromCommitMetadata() {
// now we only support FileBaseInternalSchemaManager
FileBasedInternalSchemaStorageManager manager = new FileBasedInternalSchemaStorageManager(metaClient);
String result = manager.getHistorySchemaStr();
return result.isEmpty() ? Option.empty() : Option.of(result);
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.table.log;
import org.apache.hudi.common.model.DeleteRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
@@ -36,6 +37,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.InternalSchemaCache;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
@@ -46,6 +48,9 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.action.InternalSchemaMerger;
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -109,6 +114,10 @@ public abstract class AbstractHoodieLogRecordReader {
private final FileSystem fs;
// Total log files read - for metrics
private AtomicLong totalLogFiles = new AtomicLong(0);
// Internal schema, used to support full schema evolution.
private InternalSchema internalSchema;
// Hoodie table path.
private final String path;
// Total log blocks read - for metrics
private AtomicLong totalLogBlocks = new AtomicLong(0);
// Total log records read - for metrics
@@ -135,14 +144,14 @@ public abstract class AbstractHoodieLogRecordReader {
int bufferSize, Option<InstantRange> instantRange,
boolean withOperationField) {
this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
instantRange, withOperationField, true, Option.empty());
instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema());
}
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
boolean withOperationField, boolean enableFullScan,
Option<String> partitionName) {
Option<String> partitionName, InternalSchema internalSchema) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
@@ -159,6 +168,8 @@ public abstract class AbstractHoodieLogRecordReader {
this.instantRange = instantRange;
this.withOperationField = withOperationField;
this.enableFullScan = enableFullScan;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
this.path = basePath;
// Key fields when populate meta fields is disabled (that is, virtual keys enabled)
if (!tableConfig.populateMetaFields()) {
@@ -202,7 +213,7 @@ public abstract class AbstractHoodieLogRecordReader {
// Iterate over the paths
logFormatReaderWrapper = new HoodieLogFormatReader(fs,
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()),
readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField);
readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField, internalSchema);
Set<HoodieLogFile> scannedLogFiles = new HashSet<>();
while (logFormatReaderWrapper.hasNext()) {
HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
@@ -361,8 +372,10 @@ public abstract class AbstractHoodieLogRecordReader {
*/
private void processDataBlock(HoodieDataBlock dataBlock, Option<List<String>> keys) throws Exception {
try (ClosableIterator<IndexedRecord> recordItr = dataBlock.getRecordItr(keys.orElse(Collections.emptyList()))) {
Option<Schema> schemaOption = getMergedSchema(dataBlock);
while (recordItr.hasNext()) {
IndexedRecord record = recordItr.next();
IndexedRecord currentRecord = recordItr.next();
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get()) : currentRecord;
processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
totalLogRecords.incrementAndGet();
@@ -370,6 +383,28 @@ public abstract class AbstractHoodieLogRecordReader {
}
}
/**
* Get final Read Schema for support evolution.
* step1: find the fileSchema for current dataBlock.
* step2: determine whether fileSchema is compatible with the final read internalSchema.
* step3: merge fileSchema and read internalSchema to produce final read schema.
*
* @param dataBlock current processed block
* @return final read schema.
*/
private Option<Schema> getMergedSchema(HoodieDataBlock dataBlock) {
Option<Schema> result = Option.empty();
if (!internalSchema.isEmptySchema()) {
Long currentInstantTime = Long.parseLong(dataBlock.getLogBlockHeader().get(INSTANT_TIME));
InternalSchema fileSchema = InternalSchemaCache
.searchSchemaAndCache(currentInstantTime, hoodieTableMetaClient, false);
Schema mergeSchema = AvroInternalSchemaConverter
.convert(new InternalSchemaMerger(fileSchema, internalSchema, true, false).mergeSchema(), readerSchema.getName());
result = Option.of(mergeSchema);
}
return result;
}
/**
* Create @{@link HoodieRecord} from the @{@link IndexedRecord}.
*

View File

@@ -36,6 +36,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.CorruptedLogFileException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
@@ -44,6 +45,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -74,6 +76,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
private final HoodieLogFile logFile;
private final byte[] magicBuffer = new byte[6];
private final Schema readerSchema;
private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
private final String keyField;
private boolean readBlockLazily;
private long reverseLogFilePosition;
@@ -97,6 +100,12 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
String keyField) throws IOException {
this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, enableRecordLookups, keyField, InternalSchema.getEmptyInternalSchema());
}
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
String keyField, InternalSchema internalSchema) throws IOException {
this.hadoopConf = fs.getConf();
// NOTE: We repackage {@code HoodieLogFile} here to make sure that the provided path
// is prefixed with an appropriate scheme given that we're not propagating the FS
@@ -108,6 +117,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
this.reverseReader = reverseReader;
this.enableRecordLookups = enableRecordLookups;
this.keyField = keyField;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
if (this.reverseReader) {
this.reverseLogFilePosition = this.lastReverseLogFilePosition = this.logFile.getFileSize();
}
@@ -197,10 +207,10 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
switch (Objects.requireNonNull(blockType)) {
case AVRO_DATA_BLOCK:
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
return HoodieAvroDataBlock.getBlock(content.get(), readerSchema);
return HoodieAvroDataBlock.getBlock(content.get(), readerSchema, internalSchema);
} else {
return new HoodieAvroDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
Option.ofNullable(readerSchema), header, footer, keyField);
Option.ofNullable(readerSchema), header, footer, keyField, internalSchema);
}
case HFILE_DATA_BLOCK:

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -42,6 +43,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private HoodieLogFileReader currentReader;
private final FileSystem fs;
private final Schema readerSchema;
private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
private final boolean readBlocksLazily;
private final boolean reverseLogReader;
private final String recordKeyField;
@@ -53,6 +55,12 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles, Schema readerSchema, boolean readBlocksLazily,
boolean reverseLogReader, int bufferSize, boolean enableInlineReading,
String recordKeyField) throws IOException {
this(fs, logFiles, readerSchema, readBlocksLazily, reverseLogReader, bufferSize, enableInlineReading, recordKeyField, InternalSchema.getEmptyInternalSchema());
}
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles, Schema readerSchema, boolean readBlocksLazily,
boolean reverseLogReader, int bufferSize, boolean enableInlineReading,
String recordKeyField, InternalSchema internalSchema) throws IOException {
this.logFiles = logFiles;
this.fs = fs;
this.readerSchema = readerSchema;
@@ -62,10 +70,11 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
this.prevReadersInOpenState = new ArrayList<>();
this.recordKeyField = recordKeyField;
this.enableInlineReading = enableInlineReading;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
if (logFiles.size() > 0) {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
enableInlineReading, recordKeyField);
enableInlineReading, recordKeyField, internalSchema);
}
}
@@ -105,7 +114,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
this.prevReadersInOpenState.add(currentReader);
}
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
enableInlineReading, recordKeyField);
enableInlineReading, recordKeyField, internalSchema);
} catch (IOException io) {
throw new HoodieIOException("unable to initialize read with log file ", io);
}

View File

@@ -36,6 +36,7 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -79,10 +80,10 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
ExternalSpillableMap.DiskMapType diskMapType,
boolean isBitCaskDiskMapCompressionEnabled,
boolean withOperationField, boolean enableFullScan,
Option<String> partitionName) {
Option<String> partitionName, InternalSchema internalSchema) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
instantRange, withOperationField,
enableFullScan, partitionName);
enableFullScan, partitionName, internalSchema);
try {
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
@@ -197,6 +198,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
protected String basePath;
protected List<String> logFilePaths;
protected Schema readerSchema;
private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
protected String latestInstantTime;
protected boolean readBlocksLazily;
protected boolean reverseReader;
@@ -293,6 +295,11 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
return this;
}
public Builder withInternalSchema(InternalSchema internalSchema) {
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
return this;
}
public Builder withOperationField(boolean withOperationField) {
this.withOperationField = withOperationField;
return this;
@@ -310,7 +317,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
bufferSize, spillableMapBasePath, instantRange, autoScan,
diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true,
Option.ofNullable(partitionName));
Option.ofNullable(partitionName), internalSchema);
}
}
}

View File

@@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.internal.schema.InternalSchema;
import javax.annotation.Nonnull;
import java.io.ByteArrayInputStream;
@@ -62,6 +63,17 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
private final ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
public HoodieAvroDataBlock(FSDataInputStream inputStream,
Option<byte[]> content,
boolean readBlockLazily,
HoodieLogBlockContentLocation logBlockContentLocation,
Option<Schema> readerSchema,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer,
String keyField, InternalSchema internalSchema) {
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false, internalSchema);
}
public HoodieAvroDataBlock(FSDataInputStream inputStream,
Option<byte[]> content,
boolean readBlockLazily,
@@ -126,7 +138,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
@Override
protected ClosableIterator<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
checkState(this.readerSchema != null, "Reader's schema has to be non-null");
return RecordIterator.getInstance(this, content);
return RecordIterator.getInstance(this, content, internalSchema);
}
private static class RecordIterator implements ClosableIterator<IndexedRecord> {
@@ -138,7 +150,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
private int totalRecords = 0;
private int readRecords = 0;
private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) throws IOException {
private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, InternalSchema internalSchema) throws IOException {
this.content = content;
this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content)));
@@ -147,17 +159,26 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
int version = this.dis.readInt();
HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version);
this.reader = new GenericDatumReader<>(writerSchema, readerSchema);
Schema finalReadSchema = readerSchema;
if (!internalSchema.isEmptySchema()) {
// we should use write schema to read log file,
// since when we have done some DDL operation, the readerSchema maybe different from writeSchema, avro reader will throw exception.
// eg: origin writeSchema is: "a String, b double" then we add a new column now the readerSchema will be: "a string, c int, b double". it's wrong to use readerSchema to read old log file.
// after we read those record by writeSchema, we rewrite those record with readerSchema in AbstractHoodieLogRecordReader
finalReadSchema = writerSchema;
}
this.reader = new GenericDatumReader<>(writerSchema, finalReadSchema);
if (logBlockVersion.hasRecordCount()) {
this.totalRecords = this.dis.readInt();
}
}
public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException {
public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, InternalSchema internalSchema) throws IOException {
// Get schema from the header
Schema writerSchema = new Schema.Parser().parse(dataBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
return new RecordIterator(dataBlock.readerSchema, writerSchema, content);
return new RecordIterator(dataBlock.readerSchema, writerSchema, content, internalSchema);
}
@Override
@@ -209,12 +230,16 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
super(records, Collections.singletonMap(HeaderMetadataType.SCHEMA, schema.toString()), new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
}
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
return getBlock(content, readerSchema, InternalSchema.getEmptyInternalSchema());
}
/**
* This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using
* HoodieLogFormat V1.
*/
@Deprecated
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema, InternalSchema internalSchema) throws IOException {
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
@@ -228,6 +253,10 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
readerSchema = writerSchema;
}
if (!internalSchema.isEmptySchema()) {
readerSchema = writerSchema;
}
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
// 2. Get the total records
int totalRecords = dis.readInt();

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.internal.schema.InternalSchema;
import java.io.IOException;
import java.util.HashSet;
@@ -60,6 +61,8 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
protected final Schema readerSchema;
protected InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
/**
* NOTE: This ctor is used on the write-path (ie when records ought to be written into the log)
*/
@@ -95,6 +98,25 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
this.enablePointLookups = enablePointLookups;
}
protected HoodieDataBlock(Option<byte[]> content,
FSDataInputStream inputStream,
boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation,
Option<Schema> readerSchema,
Map<HeaderMetadataType, String> headers,
Map<HeaderMetadataType, String> footer,
String keyFieldName,
boolean enablePointLookups,
InternalSchema internalSchema) {
super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.records = Option.empty();
this.keyFieldName = keyFieldName;
// If no reader-schema has been provided assume writer-schema as one
this.readerSchema = readerSchema.orElseGet(() -> getWriterSchema(super.getLogBlockHeader()));
this.enablePointLookups = enablePointLookups;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
}
@Override
public byte[] getContentBytes() throws IOException {
// In case this method is called before realizing records from content

View File

@@ -74,7 +74,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION,
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION,
REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION));
REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION,
REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION));
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
@@ -227,7 +228,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public void deleteInstantFileIfExists(HoodieInstant instant) {
LOG.info("Deleting instant " + instant);
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName());
try {
if (metaClient.getFs().exists(inFlightCommitFilePath)) {
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
@@ -246,7 +247,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private void deleteInstantFile(HoodieInstant instant) {
LOG.info("Deleting instant " + instant);
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName());
try {
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
if (result) {
@@ -261,7 +262,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
@Override
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
Path detailPath = getInstantFileNamePath(instant.getFileName());
return readDataFromPath(detailPath);
}
@@ -307,12 +308,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public Option<byte[]> readCleanerInfoAsBytes(HoodieInstant instant) {
// Cleaner metadata are always stored only in timeline .hoodie
return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName()));
return readDataFromPath(getInstantFileNamePath(instant.getFileName()));
}
public Option<byte[]> readRollbackInfoAsBytes(HoodieInstant instant) {
// Rollback metadata are always stored only in timeline .hoodie
return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName()));
return readDataFromPath(getInstantFileNamePath(instant.getFileName()));
}
public Option<byte[]> readRestoreInfoAsBytes(HoodieInstant instant) {
@@ -542,24 +543,23 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
// Re-create the .inflight file by opening a new file and write the commit metadata in
createFileInMetaPath(fromInstant.getFileName(), data, allowRedundantTransitions);
Path fromInstantPath = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
Path toInstantPath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
Path fromInstantPath = getInstantFileNamePath(fromInstant.getFileName());
Path toInstantPath = getInstantFileNamePath(toInstant.getFileName());
boolean success = metaClient.getFs().rename(fromInstantPath, toInstantPath);
if (!success) {
throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath);
}
} else {
// Ensures old state exists in timeline
LOG.info("Checking for file exists ?" + new Path(metaClient.getMetaPath(), fromInstant.getFileName()));
ValidationUtils.checkArgument(metaClient.getFs().exists(new Path(metaClient.getMetaPath(),
fromInstant.getFileName())));
LOG.info("Checking for file exists ?" + getInstantFileNamePath(fromInstant.getFileName()));
ValidationUtils.checkArgument(metaClient.getFs().exists(getInstantFileNamePath(fromInstant.getFileName())));
// Use Write Once to create Target File
if (allowRedundantTransitions) {
FileIOUtils.createFileInPath(metaClient.getFs(), new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
FileIOUtils.createFileInPath(metaClient.getFs(), getInstantFileNamePath(toInstant.getFileName()), data);
} else {
createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
createImmutableFileInPath(getInstantFileNamePath(toInstant.getFileName()), data);
}
LOG.info("Create new file for toInstant ?" + new Path(metaClient.getMetaPath(), toInstant.getFileName()));
LOG.info("Create new file for toInstant ?" + getInstantFileNamePath(toInstant.getFileName()));
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + fromInstant, e);
@@ -568,8 +568,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
ValidationUtils.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
Path inFlightCommitFilePath = getInstantFileNamePath(inflight.getFileName());
Path commitFilePath = getInstantFileNamePath(completed.getFileName());
try {
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
@@ -580,8 +580,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
} else {
Path requestedInstantFilePath = new Path(metaClient.getMetaPath(),
new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()).getFileName());
Path requestedInstantFilePath = getInstantFileNamePath(new HoodieInstant(State.REQUESTED,
inflight.getAction(), inflight.getTimestamp()).getFileName());
// If inflight and requested files do not exist, create one
if (!metaClient.getFs().exists(requestedInstantFilePath)) {
@@ -600,6 +600,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
private Path getInstantFileNamePath(String fileName) {
return new Path(fileName.contains(SCHEMA_COMMIT_ACTION) ? metaClient.getSchemaFolderName() : metaClient.getMetaPath(), fileName);
}
public void transitionRequestedToInflight(String commitType, String inFlightInstant) {
HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant);
transitionRequestedToInflight(requested, Option.empty(), false);
@@ -716,7 +720,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
private void createFileInMetaPath(String filename, Option<byte[]> content, boolean allowOverwrite) {
Path fullPath = new Path(metaClient.getMetaPath(), filename);
Path fullPath = getInstantFileNamePath(filename);
if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) {
FileIOUtils.createFileInPath(metaClient.getFs(), fullPath, content);
} else {

View File

@@ -176,6 +176,10 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
return isInflight() ? HoodieTimeline.makeInflightIndexFileName(timestamp)
: isRequested() ? HoodieTimeline.makeRequestedIndexFileName(timestamp)
: HoodieTimeline.makeIndexCommitFileName(timestamp);
} else if (HoodieTimeline.SCHEMA_COMMIT_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightSchemaFileName(timestamp)
: isRequested() ? HoodieTimeline.makeRequestSchemaFileName(timestamp)
: HoodieTimeline.makeSchemaFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}

View File

@@ -56,6 +56,8 @@ public interface HoodieTimeline extends Serializable {
String REQUESTED_EXTENSION = ".requested";
String RESTORE_ACTION = "restore";
String INDEXING_ACTION = "indexing";
// only for schema save
String SCHEMA_COMMIT_ACTION = "schemacommit";
String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION,
CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION,
@@ -88,6 +90,9 @@ public interface HoodieTimeline extends Serializable {
String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + INFLIGHT_EXTENSION;
String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + REQUESTED_EXTENSION;
String INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION;
String SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION;
String INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + INFLIGHT_EXTENSION;
String REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + REQUESTED_EXTENSION;
String INVALID_INSTANT_TS = "0";
@@ -497,4 +502,16 @@ public interface HoodieTimeline extends Serializable {
static String makeRequestedIndexFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION);
}
static String makeSchemaFileName(String instantTime) {
return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION);
}
static String makeInflightSchemaFileName(String instantTime) {
return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION);
}
static String makeRequestSchemaFileName(String instantTime) {
return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION);
}
}

View File

@@ -0,0 +1,212 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
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.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
import org.apache.hudi.internal.schema.utils.SerDeHelper;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.TreeMap;
import java.util.stream.Collectors;
public class InternalSchemaCache {
private static final Logger LOG = LogManager.getLogger(InternalSchemaCache.class);
// Use segment lock to reduce competition.
// the lock size should be powers of 2 for better hash.
private static Object[] lockList = new Object[16];
static {
for (int i = 0; i < lockList.length; i++) {
lockList[i] = new Object();
}
}
// historySchemas cache maintain a map about (tablePath, HistorySchemas).
// this is a Global cache, all threads in one container/executor share the same cache.
private static final Cache<String, TreeMap<Long, InternalSchema>>
HISTORICAL_SCHEMA_CACHE = Caffeine.newBuilder().maximumSize(1000).weakValues().build();
/**
* Search internalSchema based on versionID.
* first step: try to get internalSchema from hoodie commit files, we no need to add lock.
* if we cannot get internalSchema by first step, then we try to get internalSchema from cache.
*
* @param versionID schema version_id need to search
* @param metaClient current hoodie metaClient
* @return internalSchema
*/
public static InternalSchema searchSchemaAndCache(long versionID, HoodieTableMetaClient metaClient, boolean cacheEnable) {
Option<InternalSchema> candidateSchema = getSchemaByReadingCommitFile(versionID, metaClient);
if (candidateSchema.isPresent()) {
return candidateSchema.get();
}
if (!cacheEnable) {
// parse history schema and return directly
return InternalSchemaUtils.searchSchema(versionID, getHistoricalSchemas(metaClient));
}
String tablePath = metaClient.getBasePath();
// use segment lock to reduce competition.
synchronized (lockList[tablePath.hashCode() & (lockList.length - 1)]) {
TreeMap<Long, InternalSchema> historicalSchemas = HISTORICAL_SCHEMA_CACHE.getIfPresent(tablePath);
if (historicalSchemas == null || InternalSchemaUtils.searchSchema(versionID, historicalSchemas) == null) {
historicalSchemas = getHistoricalSchemas(metaClient);
HISTORICAL_SCHEMA_CACHE.put(tablePath, historicalSchemas);
} else {
long maxVersionId = historicalSchemas.keySet().stream().max(Long::compareTo).get();
if (versionID > maxVersionId) {
historicalSchemas = getHistoricalSchemas(metaClient);
HISTORICAL_SCHEMA_CACHE.put(tablePath, historicalSchemas);
}
}
return InternalSchemaUtils.searchSchema(versionID, historicalSchemas);
}
}
private static TreeMap<Long, InternalSchema> getHistoricalSchemas(HoodieTableMetaClient metaClient) {
TreeMap<Long, InternalSchema> result = new TreeMap<>();
FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient);
String historySchemaStr = schemasManager.getHistorySchemaStr();
if (!StringUtils.isNullOrEmpty(historySchemaStr)) {
result = SerDeHelper.parseSchemas(historySchemaStr);
}
return result;
}
private static Option<InternalSchema> getSchemaByReadingCommitFile(long versionID, HoodieTableMetaClient metaClient) {
try {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> instants = timeline.getInstants().filter(f -> f.getTimestamp().equals(String.valueOf(versionID))).collect(Collectors.toList());
if (instants.isEmpty()) {
return Option.empty();
}
byte[] data = timeline.getInstantDetails(instants.get(0)).get();
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
return SerDeHelper.fromJson(latestInternalSchemaStr);
} catch (Exception e) {
throw new HoodieException("Failed to read schema from commit metadata", e);
}
}
/**
* Get internalSchema and avroSchema for compaction/cluster operation.
*
* @param metaClient current hoodie metaClient
* @param compactionAndClusteringInstant first instant before current compaction/cluster instant
* @return (internalSchemaStrOpt, avroSchemaStrOpt) a pair of InternalSchema/avroSchema
*/
public static Pair<Option<String>, Option<String>> getInternalSchemaAndAvroSchemaForClusteringAndCompaction(HoodieTableMetaClient metaClient, String compactionAndClusteringInstant) {
// try to load internalSchema to support Schema Evolution
HoodieTimeline timelineBeforeCurrentCompaction = metaClient.getCommitsAndCompactionTimeline().findInstantsBefore(compactionAndClusteringInstant).filterCompletedInstants();
Option<HoodieInstant> lastInstantBeforeCurrentCompaction = timelineBeforeCurrentCompaction.lastInstant();
if (lastInstantBeforeCurrentCompaction.isPresent()) {
// try to find internalSchema
byte[] data = timelineBeforeCurrentCompaction.getInstantDetails(lastInstantBeforeCurrentCompaction.get()).get();
HoodieCommitMetadata metadata;
try {
metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
} catch (Exception e) {
throw new HoodieException(String.format("cannot read metadata from commit: %s", lastInstantBeforeCurrentCompaction.get()), e);
}
String internalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
if (internalSchemaStr != null) {
String existingSchemaStr = metadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY);
return Pair.of(Option.of(internalSchemaStr), Option.of(existingSchemaStr));
}
}
return Pair.of(Option.empty(), Option.empty());
}
/**
* Give a schema versionId return its internalSchema.
* This method will be called by spark tasks, we should minimize time cost.
* We try our best to not use metaClient since the initialization of metaClient is time cost
* step1
* try to parser internalSchema from HoodieInstant directly
* step2
* if we cannot parser internalSchema in step1
* try to find internalSchema in historySchema.
*
* @param versionId the internalSchema version to be search.
* @param tablePath table path
* @param hadoopConf conf
* @param validCommits current validate commits, use to make up the commit file path/verify the validity of the history schema files
* @return a internalSchema.
*/
public static InternalSchema getInternalSchemaByVersionId(long versionId, String tablePath, Configuration hadoopConf, String validCommits) {
Set<String> commitSet = Arrays.stream(validCommits.split(",")).collect(Collectors.toSet());
List<String> validateCommitList = commitSet.stream().map(fileName -> {
String fileExtension = HoodieInstant.getTimelineFileExtension(fileName);
return fileName.replace(fileExtension, "");
}).collect(Collectors.toList());
FileSystem fs = FSUtils.getFs(tablePath, hadoopConf);
Path hoodieMetaPath = new Path(tablePath, HoodieTableMetaClient.METAFOLDER_NAME);
//step1:
Path candidateCommitFile = commitSet.stream().filter(fileName -> {
String fileExtension = HoodieInstant.getTimelineFileExtension(fileName);
return fileName.replace(fileExtension, "").equals(versionId + "");
}).findFirst().map(f -> new Path(hoodieMetaPath, f)).orElse(null);
if (candidateCommitFile != null) {
try {
byte[] data;
try (FSDataInputStream is = fs.open(candidateCommitFile)) {
data = FileIOUtils.readAsByteArray(is);
} catch (IOException e) {
throw e;
}
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
if (latestInternalSchemaStr != null) {
return SerDeHelper.fromJson(latestInternalSchemaStr).orElse(null);
}
} catch (Exception e1) {
// swallow this exception.
LOG.warn(String.format("Cannot find internal schema from commit file %s. Falling back to parsing historical internal schema", candidateCommitFile.toString()));
}
}
// step2:
FileBasedInternalSchemaStorageManager fileBasedInternalSchemaStorageManager = new FileBasedInternalSchemaStorageManager(hadoopConf, new Path(tablePath));
String lastestHistorySchema = fileBasedInternalSchemaStorageManager.getHistorySchemaStrByGivenValidCommits(validateCommitList);
return InternalSchemaUtils.searchSchema(versionId, SerDeHelper.parseSchemas(lastestHistorySchema));
}
}

View File

@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema;
import org.apache.hudi.exception.HoodieException;
/**
* Exception thrown for Hoodie schema convert failures. The root of the exception hierarchy.
* Hoodie Write/Read clients will throw this exception if any of its operations fail. This is a runtime (unchecked)
* exception.
*/
public class HoodieSchemaException extends HoodieException {
public HoodieSchemaException() {
super();
}
public HoodieSchemaException(String message) {
super(message);
}
public HoodieSchemaException(String message, Throwable t) {
super(message, t);
}
public HoodieSchemaException(Throwable t) {
super(t);
}
}

View File

@@ -0,0 +1,291 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.internal.schema.Types.Field;
import org.apache.hudi.internal.schema.Types.RecordType;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Internal schema for hudi table.
* used to support schema evolution.
*/
public class InternalSchema implements Serializable {
private static final long DEFAULT_VERSION_ID = 0;
private final RecordType record;
private int maxColumnId;
private long versionId;
private transient Map<Integer, Field> idToField = null;
private transient Map<String, Integer> nameToId = null;
private transient Map<Integer, String> idToName = null;
public static InternalSchema getEmptyInternalSchema() {
return new InternalSchema(-1L, new ArrayList<>());
}
public boolean isEmptySchema() {
return versionId < 0;
}
public InternalSchema(List<Field> columns) {
this(DEFAULT_VERSION_ID, columns);
}
public InternalSchema(Field... columns) {
this(DEFAULT_VERSION_ID, Arrays.asList(columns));
}
public InternalSchema(long versionId, List<Field> cols) {
this.versionId = versionId;
this.record = RecordType.get(cols);
idToName = cols.isEmpty() ? new HashMap<>() : InternalSchemaBuilder.getBuilder().buildIdToName(record);
nameToId = cols.isEmpty() ? new HashMap<>() : idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
maxColumnId = idToName.isEmpty() ? -1 : idToName.keySet().stream().max(Comparator.comparing(Integer::valueOf)).get();
}
public InternalSchema(long versionId, int maxColumnId, List<Field> cols) {
this.maxColumnId = maxColumnId;
this.versionId = versionId;
this.record = RecordType.get(cols);
buildIdToName();
}
public InternalSchema(long versionId, int maxColumnId, Field... cols) {
this(versionId, maxColumnId, Arrays.asList(cols));
}
public RecordType getRecord() {
return record;
}
private Map<Integer, String> buildIdToName() {
if (idToName == null) {
idToName = InternalSchemaBuilder.getBuilder().buildIdToName(record);
}
return idToName;
}
private Map<String, Integer> buildNameToId() {
if (nameToId == null) {
if (idToName != null && !idToName.isEmpty()) {
nameToId = idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
return nameToId;
}
nameToId = InternalSchemaBuilder.getBuilder().buildNameToId(record);
}
return nameToId;
}
private Map<Integer, Field> buildIdToField() {
if (idToField == null) {
idToField = InternalSchemaBuilder.getBuilder().buildIdToField(record);
}
return idToField;
}
/**
* Get all columns full name.
*/
public List<String> getAllColsFullName() {
if (nameToId == null) {
nameToId = InternalSchemaBuilder.getBuilder().buildNameToId(record);
}
return Arrays.asList(nameToId.keySet().toArray(new String[0]));
}
/**
* Set the version ID for this schema.
*/
public InternalSchema setSchemaId(long versionId) {
this.versionId = versionId;
return this;
}
/**
* Returns the version ID for this schema.
*/
public long schemaId() {
return this.versionId;
}
/**
* Set the version ID for this schema.
*/
public void setMaxColumnId(int maxColumnId) {
this.maxColumnId = maxColumnId;
}
/**
* Returns the max column id for this schema.
*/
public int getMaxColumnId() {
return this.maxColumnId;
}
/**
* Returns a List of the {@link Field columns} in this Schema.
*/
public List<Field> columns() {
return record.fields();
}
/**
* Returns the {@link Type} of a sub-field identified by the field name.
*
* @param id a field id
* @return fullName of field of
*/
public String findfullName(int id) {
if (idToName == null) {
buildIdToName();
}
String result = idToName.get(id);
return result == null ? "" : result;
}
/**
* Returns the {@link Type} of a sub-field identified by the field name.
*
* @param name a field name
* @return a Type for the sub-field or null if it is not found
*/
public Type findType(String name) {
if (name == null || name.isEmpty()) {
return null;
}
Integer id = buildNameToId().get(name);
if (id != null) { // name is found
return findType(id);
}
return null;
}
/**
* Returns the {@link Type} of a sub-field identified by the field id.
*
* @param id a field id
* @return a Type for the sub-field or null if it is not found
*/
public Type findType(int id) {
Field field = buildIdToField().get(id);
if (field != null) {
return field.type();
}
return null;
}
/**
* Returns all field ids
*/
public Set<Integer> getAllIds() {
if (idToName == null) {
buildIdToName();
}
return idToName.keySet();
}
/**
* Returns the sub-field identified by the field id.
*
* @param id a field id
* @return the sub-field or null if it is not found
*/
public Field findField(int id) {
return buildIdToField().get(id);
}
/**
* Returns a sub-field by name as a {@link Field}.
* The result may be a top-level or a nested field.
*
* @param name a String name
* @return a Type for the sub-field or null if it is not found
*/
public Field findField(String name) {
if (name == null || name.isEmpty()) {
return null;
}
Integer id = buildNameToId().get(name);
if (id != null) {
return buildIdToField().get(id);
}
return null;
}
/**
* Whether colName exists in current Schema.
* Case insensitive.
*
* @param colName a colName
* @return Whether colName exists in current Schema
*/
public boolean findDuplicateCol(String colName) {
return idToName.entrySet().stream().map(e -> e.getValue().toLowerCase(Locale.ROOT))
.collect(Collectors.toSet()).contains(colName);
}
public int findIdByName(String name) {
if (name == null || name.isEmpty()) {
return -1;
}
return buildNameToId().getOrDefault(name, -1);
}
@Override
public String toString() {
return String.format("table {\n%s\n}",
StringUtils.join(record.fields().stream()
.map(f -> " " + f)
.collect(Collectors.toList()).toArray(new String[0]), "\n"));
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof InternalSchema)) {
return false;
}
InternalSchema that = (InternalSchema) o;
if (versionId != that.schemaId()) {
return false;
}
return record.equals(that.record);
}
@Override
public int hashCode() {
return record.hashCode();
}
}

View File

@@ -0,0 +1,272 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema;
import org.apache.hudi.internal.schema.visitor.InternalSchemaVisitor;
import org.apache.hudi.internal.schema.visitor.NameToIDVisitor;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Deque;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
/**
* A build class to help build fields for InternalSchema
*/
public class InternalSchemaBuilder implements Serializable {
private static final InternalSchemaBuilder INSTANCE = new InternalSchemaBuilder();
public static InternalSchemaBuilder getBuilder() {
return INSTANCE;
}
private InternalSchemaBuilder() {
}
/**
* Build a mapping from id to full field name for a internal Type.
* if a field y belong to a struct filed x, then the full name of y is x.y
*
* @param type hoodie internal type
* @return a mapping from id to full field name
*/
public Map<Integer, String> buildIdToName(Type type) {
Map<Integer, String> result = new HashMap<>();
buildNameToId(type).forEach((k, v) -> result.put(v, k));
return result;
}
/**
* Build a mapping from full field name to id for a internal Type.
* if a field y belong to a struct filed x, then the full name of y is x.y
*
* @param type hoodie internal type
* @return a mapping from full field name to id
*/
public Map<String, Integer> buildNameToId(Type type) {
return visit(type, new NameToIDVisitor());
}
/**
* Use to traverse all types in internalSchema with visitor.
*
* @param schema hoodie internal schema
* @return vistor expected result.
*/
public <T> T visit(InternalSchema schema, InternalSchemaVisitor<T> visitor) {
return visitor.schema(schema, visit(schema.getRecord(), visitor));
}
public <T> T visit(Type type, InternalSchemaVisitor<T> visitor) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<T> results = new ArrayList<>();
for (Types.Field f : record.fields()) {
visitor.beforeField(f);
T result;
try {
result = visit(f.type(), visitor);
} finally {
visitor.afterField(f);
}
results.add(visitor.field(f, result));
}
return visitor.record(record, results);
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
T elementResult;
Types.Field elementField = array.field(array.elementId());
visitor.beforeArrayElement(elementField);
try {
elementResult = visit(elementField.type(), visitor);
} finally {
visitor.afterArrayElement(elementField);
}
return visitor.array(array, elementResult);
case MAP:
Types.MapType map = (Types.MapType) type;
T keyResult;
T valueResult;
Types.Field keyField = map.field(map.keyId());
visitor.beforeMapKey(keyField);
try {
keyResult = visit(map.keyType(), visitor);
} finally {
visitor.afterMapKey(keyField);
}
Types.Field valueField = map.field(map.valueId());
visitor.beforeMapValue(valueField);
try {
valueResult = visit(map.valueType(), visitor);
} finally {
visitor.afterMapValue(valueField);
}
return visitor.map(map, keyResult, valueResult);
default:
return visitor.primitive((Type.PrimitiveType)type);
}
}
/**
* Build a mapping from id to field for a internal Type.
*
* @param type hoodie internal type
* @return a mapping from id to field
*/
public Map<Integer, Types.Field> buildIdToField(Type type) {
Map<Integer, Types.Field> idToField = new HashMap<>();
visitIdToField(type, idToField);
return idToField;
}
private void visitIdToField(Type type, Map<Integer, Types.Field> index) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
for (Types.Field field : record.fields()) {
visitIdToField(field.type(), index);
index.put(field.fieldId(), field);
}
return;
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
visitIdToField(array.elementType(), index);
for (Types.Field field : array.fields()) {
index.put(field.fieldId(), field);
}
return;
case MAP:
Types.MapType map = (Types.MapType) type;
visitIdToField(map.keyType(), index);
visitIdToField(map.valueType(), index);
for (Types.Field field : map.fields()) {
index.put(field.fieldId(), field);
}
return;
default:
return;
}
}
/**
* Build a mapping which maintain the relation between child field id and it's parent field id.
* if a child field y(which id is 9) belong to a nest field x(which id is 6), then (9 -> 6) will be added to the result map.
* if a field has no parent field, nothings will be added.
*
* @param record hoodie record type.
* @return a mapping from id to parentId for a record Type
*/
public Map<Integer, Integer> index2Parents(Types.RecordType record) {
Map<Integer, Integer> result = new HashMap<>();
Deque<Integer> parentIds = new LinkedList<>();
index2Parents(record, parentIds, result);
return result;
}
private void index2Parents(Type type, Deque<Integer> pids, Map<Integer, Integer> id2p) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType)type;
for (Types.Field f : record.fields()) {
pids.push(f.fieldId());
index2Parents(f.type(), pids, id2p);
pids.pop();
}
for (Types.Field f : record.fields()) {
// root record has no parent id.
if (!pids.isEmpty()) {
Integer pid = pids.peek();
id2p.put(f.fieldId(), pid);
}
}
return;
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Types.Field elementField = array.field(array.elementId());
pids.push(elementField.fieldId());
index2Parents(elementField.type(), pids, id2p);
pids.pop();
id2p.put(array.elementId(), pids.peek());
return;
case MAP:
Types.MapType map = (Types.MapType) type;
Types.Field keyField = map.field(map.keyId());
Types.Field valueField = map.field(map.valueId());
// visit key
pids.push(map.keyId());
index2Parents(keyField.type(), pids, id2p);
pids.pop();
// visit value
pids.push(map.valueId());
index2Parents(valueField.type(), pids, id2p);
pids.pop();
id2p.put(map.keyId(), pids.peek());
id2p.put(map.valueId(), pids.peek());
return;
default:
}
}
/**
* Assigns new ids for all fields in a Type, based on initial id.
*
* @param type a type.
* @param nextId initial id which used to fresh ids for all fields in a type
* @return a new type with new ids
*/
public Type refreshNewId(Type type, AtomicInteger nextId) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Types.Field> oldFields = record.fields();
int currentId = nextId.get();
nextId.set(currentId + record.fields().size());
List<Types.Field> internalFields = new ArrayList<>();
for (int i = 0; i < oldFields.size(); i++) {
Types.Field oldField = oldFields.get(i);
Type fieldType = refreshNewId(oldField.type(), nextId);
internalFields.add(Types.Field.get(currentId++, oldField.isOptional(), oldField.name(), fieldType, oldField.doc()));
}
return Types.RecordType.get(internalFields);
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
int elementId = nextId.get();
nextId.set(elementId + 1);
Type elementType = refreshNewId(array.elementType(), nextId);
return Types.ArrayType.get(elementId, array.isElementOptional(), elementType);
case MAP:
Types.MapType map = (Types.MapType) type;
int keyId = nextId.get();
int valueId = keyId + 1;
nextId.set(keyId + 2);
Type keyType = refreshNewId(map.keyType(), nextId);
Type valueType = refreshNewId(map.valueType(), nextId);
return Types.MapType.get(keyId, valueId, keyType, valueType, map.isValueOptional());
default:
return type;
}
}
}

View File

@@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema;
import java.io.Serializable;
import java.util.List;
import java.util.Locale;
/**
* The type of a schema, reference avro schema.
* now avro version used by hoodie, not support localTime.
* to do add support for localTime if avro version is updated
*/
public interface Type extends Serializable {
enum TypeID {
RECORD, ARRAY, MAP, FIXED, STRING, BINARY,
INT, LONG, FLOAT, DOUBLE, DATE, BOOLEAN, TIME, TIMESTAMP, DECIMAL, UUID;
private String name;
TypeID() {
this.name = this.name().toLowerCase(Locale.ROOT);
}
public String getName() {
return name;
}
}
static TypeID fromValue(String value) {
try {
return TypeID.valueOf(value.toUpperCase(Locale.ROOT));
} catch (IllegalArgumentException e) {
throw new IllegalArgumentException(String.format("Invalid value of Type: %s", value));
}
}
TypeID typeId();
default boolean isNestedType() {
return false;
}
abstract class PrimitiveType implements Type {
@Override
public boolean isNestedType() {
return false;
}
}
abstract class NestedType implements Type {
@Override
public boolean isNestedType() {
return true;
}
public abstract List<Types.Field> fields();
public abstract Type fieldType(String name);
public abstract Types.Field field(int id);
}
}

View File

@@ -0,0 +1,716 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema;
import org.apache.hudi.internal.schema.Type.PrimitiveType;
import org.apache.hudi.internal.schema.Type.NestedType;
import java.io.Serializable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
public class Types {
private Types() {
}
public static class BooleanType extends PrimitiveType {
private static final BooleanType INSTANCE = new BooleanType();
public static BooleanType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return Type.TypeID.BOOLEAN;
}
@Override
public String toString() {
return "boolean";
}
}
public static class IntType extends PrimitiveType {
private static final IntType INSTANCE = new IntType();
public static IntType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.INT;
}
@Override
public String toString() {
return "int";
}
}
public static class LongType extends PrimitiveType {
private static final LongType INSTANCE = new LongType();
public static LongType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.LONG;
}
@Override
public String toString() {
return "long";
}
}
public static class FloatType extends PrimitiveType {
private static final FloatType INSTANCE = new FloatType();
public static FloatType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.FLOAT;
}
@Override
public String toString() {
return "float";
}
}
public static class DoubleType extends PrimitiveType {
private static final DoubleType INSTANCE = new DoubleType();
public static DoubleType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.DOUBLE;
}
@Override
public String toString() {
return "double";
}
}
public static class DateType extends PrimitiveType {
private static final DateType INSTANCE = new DateType();
public static DateType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.DATE;
}
@Override
public String toString() {
return "date";
}
}
public static class TimeType extends PrimitiveType {
private static final TimeType INSTANCE = new TimeType();
public static TimeType get() {
return INSTANCE;
}
private TimeType() {
}
@Override
public TypeID typeId() {
return TypeID.TIME;
}
@Override
public String toString() {
return "time";
}
}
public static class TimestampType extends PrimitiveType {
private static final TimestampType INSTANCE = new TimestampType();
public static TimestampType get() {
return INSTANCE;
}
private TimestampType() {
}
@Override
public TypeID typeId() {
return TypeID.TIMESTAMP;
}
@Override
public String toString() {
return "timestamp";
}
}
public static class StringType extends PrimitiveType {
private static final StringType INSTANCE = new StringType();
public static StringType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.STRING;
}
@Override
public String toString() {
return "string";
}
}
public static class BinaryType extends PrimitiveType {
private static final BinaryType INSTANCE = new BinaryType();
public static BinaryType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.BINARY;
}
@Override
public String toString() {
return "binary";
}
}
public static class FixedType extends PrimitiveType {
public static FixedType getFixed(int size) {
return new FixedType(size);
}
private final int size;
private FixedType(int length) {
this.size = length;
}
public int getFixedSize() {
return size;
}
@Override
public TypeID typeId() {
return TypeID.FIXED;
}
@Override
public String toString() {
return String.format("fixed[%d]", size);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof FixedType)) {
return false;
}
FixedType fixedType = (FixedType) o;
return size == fixedType.size;
}
@Override
public int hashCode() {
return Objects.hash(FixedType.class, size);
}
}
public static class DecimalType extends PrimitiveType {
public static DecimalType get(int precision, int scale) {
return new DecimalType(precision, scale);
}
private final int scale;
private final int precision;
private DecimalType(int precision, int scale) {
this.scale = scale;
this.precision = precision;
}
/**
* Returns whether this DecimalType is wider than `other`. If yes, it means `other`
* can be casted into `this` safely without losing any precision or range.
*/
public boolean isWiderThan(PrimitiveType other) {
if (other instanceof DecimalType) {
DecimalType dt = (DecimalType) other;
return (precision - scale) >= (dt.precision - dt.scale) && scale > dt.scale;
}
if (other instanceof IntType) {
return isWiderThan(get(10, 0));
}
return false;
}
/**
* Returns whether this DecimalType is tighter than `other`. If yes, it means `this`
* can be casted into `other` safely without losing any precision or range.
*/
public boolean isTighterThan(PrimitiveType other) {
if (other instanceof DecimalType) {
DecimalType dt = (DecimalType) other;
return (precision - scale) <= (dt.precision - dt.scale) && scale <= dt.scale;
}
if (other instanceof IntType) {
return isTighterThan(get(10, 0));
}
return false;
}
public int scale() {
return scale;
}
public int precision() {
return precision;
}
@Override
public TypeID typeId() {
return TypeID.DECIMAL;
}
@Override
public String toString() {
return String.format("decimal(%d, %d)", precision, scale);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof DecimalType)) {
return false;
}
DecimalType that = (DecimalType) o;
if (scale != that.scale) {
return false;
}
return precision == that.precision;
}
@Override
public int hashCode() {
return Objects.hash(DecimalType.class, scale, precision);
}
}
public static class UUIDType extends PrimitiveType {
private static final UUIDType INSTANCE = new UUIDType();
public static UUIDType get() {
return INSTANCE;
}
@Override
public TypeID typeId() {
return TypeID.UUID;
}
@Override
public String toString() {
return "uuid";
}
}
/** A field within a record. */
public static class Field implements Serializable {
// Experimental method to support defaultValue
public static Field get(int id, boolean isOptional, String name, Type type, String doc, Object defaultValue) {
return new Field(isOptional, id, name, type, doc, defaultValue);
}
public static Field get(int id, boolean isOptional, String name, Type type, String doc) {
return new Field(isOptional, id, name, type, doc, null);
}
public static Field get(int id, boolean isOptional, String name, Type type) {
return new Field(isOptional, id, name, type, null, null);
}
public static Field get(int id, String name, Type type) {
return new Field(true, id, name, type, null, null);
}
private final boolean isOptional;
private final int id;
private final String name;
private final Type type;
private final String doc;
// Experimental properties
private final Object defaultValue;
private Field(boolean isOptional, int id, String name, Type type, String doc, Object defaultValue) {
this.isOptional = isOptional;
this.id = id;
this.name = name;
this.type = type;
this.doc = doc;
this.defaultValue = defaultValue;
}
public Object getDefaultValue() {
return defaultValue;
}
public boolean isOptional() {
return isOptional;
}
public int fieldId() {
return id;
}
public String name() {
return name;
}
public Type type() {
return type;
}
public String doc() {
return doc;
}
@Override
public String toString() {
return String.format("%d: %s: %s %s",
id, name, isOptional ? "optional" : "required", type) + (doc != null ? " (" + doc + ")" : "");
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof Field)) {
return false;
}
Field that = (Field) o;
if (isOptional != that.isOptional) {
return false;
} else if (id != that.id) {
return false;
} else if (!name.equals(that.name)) {
return false;
} else if (!Objects.equals(doc, that.doc)) {
return false;
}
return type.equals(that.type);
}
@Override
public int hashCode() {
return Objects.hash(Field.class, id, isOptional, name, type);
}
}
public static class RecordType extends NestedType {
public static RecordType get(List<Field> fields) {
return new RecordType(fields);
}
public static RecordType get(Field... fields) {
return new RecordType(Arrays.asList(fields));
}
private final Field[] fields;
private transient Map<String, Field> nameToFields = null;
private transient Map<Integer, Field> idToFields = null;
private RecordType(List<Field> fields) {
this.fields = new Field[fields.size()];
for (int i = 0; i < this.fields.length; i += 1) {
this.fields[i] = fields.get(i);
}
}
@Override
public List<Field> fields() {
return Arrays.asList(fields);
}
public Field field(String name) {
if (nameToFields == null) {
nameToFields = new HashMap<>();
for (Field field : fields) {
nameToFields.put(field.name().toLowerCase(Locale.ROOT), field);
}
}
return nameToFields.get(name.toLowerCase(Locale.ROOT));
}
@Override
public Field field(int id) {
if (idToFields == null) {
idToFields = new HashMap<>();
for (Field field : fields) {
idToFields.put(field.fieldId(), field);
}
}
return idToFields.get(id);
}
@Override
public Type fieldType(String name) {
Field field = field(name);
if (field != null) {
return field.type();
}
return null;
}
@Override
public TypeID typeId() {
return TypeID.RECORD;
}
@Override
public String toString() {
return String.format("Record<%s>", Arrays.stream(fields).map(f -> f.toString()).collect(Collectors.joining("-")));
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof RecordType)) {
return false;
}
RecordType that = (RecordType) o;
return Arrays.equals(fields, that.fields);
}
@Override
public int hashCode() {
return Objects.hash(Field.class, Arrays.hashCode(fields));
}
}
public static class ArrayType extends NestedType {
public static ArrayType get(int elementId, boolean isOptional, Type elementType) {
return new ArrayType(Field.get(elementId, isOptional,"element", elementType));
}
private final Field elementField;
private ArrayType(Field elementField) {
this.elementField = elementField;
}
public Type elementType() {
return elementField.type();
}
@Override
public Type fieldType(String name) {
if ("element".equals(name)) {
return elementType();
}
return null;
}
@Override
public Field field(int id) {
if (elementField.fieldId() == id) {
return elementField;
}
return null;
}
@Override
public List<Field> fields() {
return Arrays.asList(elementField);
}
public int elementId() {
return elementField.fieldId();
}
public boolean isElementOptional() {
return elementField.isOptional;
}
@Override
public TypeID typeId() {
return TypeID.ARRAY;
}
@Override
public String toString() {
return String.format("list<%s>", elementField.type());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof ArrayType)) {
return false;
}
ArrayType listType = (ArrayType) o;
return elementField.equals(listType.elementField);
}
@Override
public int hashCode() {
return Objects.hash(ArrayType.class, elementField);
}
}
public static class MapType extends NestedType {
public static MapType get(int keyId, int valueId, Type keyType, Type valueType) {
return new MapType(
Field.get(keyId, "key", keyType),
Field.get(valueId, "value", valueType));
}
public static MapType get(int keyId, int valueId, Type keyType, Type valueType, boolean isOptional) {
return new MapType(
Field.get(keyId, isOptional, "key", keyType),
Field.get(valueId, isOptional, "value", valueType));
}
private final Field keyField;
private final Field valueField;
private transient List<Field> fields = null;
private MapType(Field keyField, Field valueField) {
this.keyField = keyField;
this.valueField = valueField;
}
public Type keyType() {
return keyField.type();
}
public Type valueType() {
return valueField.type();
}
@Override
public Type fieldType(String name) {
if ("key".equals(name)) {
return keyField.type();
} else if ("value".equals(name)) {
return valueField.type();
}
return null;
}
@Override
public Field field(int id) {
if (keyField.fieldId() == id) {
return keyField;
} else if (valueField.fieldId() == id) {
return valueField;
}
return null;
}
@Override
public List<Field> fields() {
if (fields == null) {
fields = Arrays.asList(keyField, valueField);
}
return fields;
}
public int keyId() {
return keyField.fieldId();
}
public int valueId() {
return valueField.fieldId();
}
public boolean isValueOptional() {
return valueField.isOptional;
}
@Override
public TypeID typeId() {
return TypeID.MAP;
}
@Override
public String toString() {
return String.format("map<%s, %s>", keyField.type(), valueField.type());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (!(o instanceof MapType)) {
return false;
}
MapType mapType = (MapType) o;
if (!keyField.equals(mapType.keyField)) {
return false;
}
return valueField.equals(mapType.valueField);
}
@Override
public int hashCode() {
return Objects.hash(MapType.class, keyField, valueField);
}
}
}

View File

@@ -0,0 +1,164 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.action;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.utils.SchemaChangeUtils;
import java.util.Arrays;
/**
* Manage schema change for HoodieWriteClient.
*/
public class InternalSchemaChangeApplier {
private InternalSchema latestSchema;
public InternalSchemaChangeApplier(InternalSchema latestSchema) {
this.latestSchema = latestSchema;
}
/**
* Add columns to table.
*
* @param colName col name to be added. if we want to add col to a nested filed, the fullName should be specify
* @param colType col type to be added.
* @param doc col doc to be added.
* @param position col position to be added
* @param positionType col position change type. now support three change types: first/after/before
*/
public InternalSchema applyAddChange(
String colName,
Type colType,
String doc,
String position,
TableChange.ColumnPositionChange.ColumnPositionType positionType) {
TableChanges.ColumnAddChange add = TableChanges.ColumnAddChange.get(latestSchema);
String parentName = TableChangesHelper.getParentName(colName);
add.addColumns(parentName, colName, colType, doc);
if (positionType != null) {
switch (positionType) {
case NO_OPERATION:
break;
case FIRST:
add.addPositionChange(colName, "", positionType);
break;
case AFTER:
case BEFORE:
if (position == null || position.isEmpty()) {
throw new IllegalArgumentException("position should not be null/empty_string when specify positionChangeType as after/before");
}
String referParentName = TableChangesHelper.getParentName(position);
if (!parentName.equals(referParentName)) {
throw new IllegalArgumentException("cannot reorder two columns which has different parent");
}
add.addPositionChange(colName, position, positionType);
break;
default:
throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", positionType));
}
} else {
throw new IllegalArgumentException(String.format("positionType should be specified"));
}
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, add);
}
/**
* Delete columns to table.
*
* @param colNames col name to be deleted. if we want to delete col from a nested filed, the fullName should be specify
*/
public InternalSchema applyDeleteChange(String... colNames) {
TableChanges.ColumnDeleteChange delete = TableChanges.ColumnDeleteChange.get(latestSchema);
Arrays.stream(colNames).forEach(colName -> delete.deleteColumn(colName));
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, delete);
}
/**
* Rename col name for hudi table.
*
* @param colName col name to be renamed. if we want to rename col from a nested filed, the fullName should be specify
* @param newName new name for current col. no need to specify fullName.
*/
public InternalSchema applyRenameChange(String colName, String newName) {
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
updateChange.renameColumn(colName, newName);
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
}
/**
* Update col nullability for hudi table.
*
* @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify
* @param nullable .
*/
public InternalSchema applyColumnNullabilityChange(String colName, boolean nullable) {
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
updateChange.updateColumnNullability(colName, nullable);
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
}
/**
* Update col type for hudi table.
*
* @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify
* @param newType .
*/
public InternalSchema applyColumnTypeChange(String colName, Type newType) {
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
updateChange.updateColumnType(colName, newType);
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
}
/**
* Update col comment for hudi table.
*
* @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify
* @param doc .
*/
public InternalSchema applyColumnCommentChange(String colName, String doc) {
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
updateChange.updateColumnComment(colName, doc);
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
}
/**
* Reorder the position of col.
*
* @param colName column which need to be reordered. if we want to change col from a nested filed, the fullName should be specify.
* @param referColName reference position.
* @param positionType col position change type. now support three change types: first/after/before
*/
public InternalSchema applyReOrderColPositionChange(
String colName,
String referColName,
TableChange.ColumnPositionChange.ColumnPositionType positionType) {
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
String parentName = TableChangesHelper.getParentName(colName);
String referParentName = TableChangesHelper.getParentName(referColName);
if (positionType.equals(TableChange.ColumnPositionChange.ColumnPositionType.FIRST)) {
updateChange.addPositionChange(colName, "", positionType);
} else if (parentName.equals(referParentName)) {
updateChange.addPositionChange(colName, referColName, positionType);
} else {
throw new IllegalArgumentException("cannot reorder two columns which has different parent");
}
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
}
}

View File

@@ -0,0 +1,197 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.action;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import java.util.ArrayList;
import java.util.List;
/**
* Auxiliary class.
* help to merge file schema and query schema to produce final read schema for avro/parquet file
*/
public class InternalSchemaMerger {
private final InternalSchema fileSchema;
private final InternalSchema querySchema;
// now there exist some bugs when we use spark update/merge api,
// those operation will change col nullability from optional to required which is wrong.
// Before that bug is fixed, we need to do adapt.
// if mergeRequiredFiledForce is true, we will ignore the col's required attribute.
private final boolean ignoreRequiredAttribute;
// Whether to use column Type from file schema to read files when we find some column type has changed.
// spark parquetReader need the original column type to read data, otherwise the parquetReader will failed.
// eg: current column type is StringType, now we changed it to decimalType,
// we should not pass decimalType to parquetReader, we must pass StringType to it; when we read out the data, we convert data from String to Decimal, everything is ok.
// for log reader
// since our reWriteRecordWithNewSchema function support rewrite directly, so we no need this parameter
// eg: current column type is StringType, now we changed it to decimalType,
// we can pass decimalType to reWriteRecordWithNewSchema directly, everything is ok.
private boolean useColumnTypeFromFileSchema = true;
public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) {
this.fileSchema = fileSchema;
this.querySchema = querySchema;
this.ignoreRequiredAttribute = ignoreRequiredAttribute;
this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema;
}
/**
* Create final read schema to read avro/parquet file.
*
* @return read schema to read avro/parquet file.
*/
public InternalSchema mergeSchema() {
Types.RecordType record = (Types.RecordType) mergeType(querySchema.getRecord(), 0);
return new InternalSchema(record.fields());
}
/**
* Create final read schema to read avro/parquet file.
* this is auxiliary function used by mergeSchema.
*/
private Type mergeType(Type type, int currentTypeId) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Type> newTypes = new ArrayList<>();
for (Types.Field f : record.fields()) {
Type newType = mergeType(f.type(), f.fieldId());
newTypes.add(newType);
}
return Types.RecordType.get(buildRecordType(record.fields(), newTypes));
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Type newElementType;
Types.Field elementField = array.fields().get(0);
newElementType = mergeType(elementField.type(), elementField.fieldId());
return buildArrayType(array, newElementType);
case MAP:
Types.MapType map = (Types.MapType) type;
Type newValueType = mergeType(map.valueType(), map.valueId());
return buildMapType(map, newValueType);
default:
return buildPrimitiveType((Type.PrimitiveType) type, currentTypeId);
}
}
private List<Types.Field> buildRecordType(List<Types.Field> oldFields, List<Type> newTypes) {
List<Types.Field> newFields = new ArrayList<>();
for (int i = 0; i < newTypes.size(); i++) {
Type newType = newTypes.get(i);
Types.Field oldField = oldFields.get(i);
int fieldId = oldField.fieldId();
String fullName = querySchema.findfullName(fieldId);
if (fileSchema.findField(fieldId) != null) {
if (fileSchema.findfullName(fieldId).equals(fullName)) {
// maybe col type changed, deal with it.
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
} else {
// find rename, deal with it.
newFields.add(dealWithRename(fieldId, newType, oldField));
}
} else {
// buildFullName
fullName = normalizeFullName(fullName);
if (fileSchema.findField(fullName) != null) {
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name() + "suffix", oldField.type(), oldField.doc()));
} else {
// find add column
// now there exist some bugs when we use spark update/merge api, those operation will change col optional to required.
if (ignoreRequiredAttribute) {
newFields.add(Types.Field.get(oldField.fieldId(), true, oldField.name(), newType, oldField.doc()));
} else {
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
}
}
}
}
return newFields;
}
private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldField) {
Types.Field fieldFromFileSchema = fileSchema.findField(fieldId);
String nameFromFileSchema = fieldFromFileSchema.name();
Type typeFromFileSchema = fieldFromFileSchema.type();
// Current design mechanism guarantees nestedType change is not allowed, so no need to consider.
if (newType.isNestedType()) {
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, newType, oldField.doc());
} else {
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc());
}
}
private String normalizeFullName(String fullName) {
// find parent rename, and normalize fullName
// eg: we renamed a nest field struct(c, d) to aa, the we delete a.d and add it back later.
String[] nameParts = fullName.split("\\.");
String[] normalizedNameParts = new String[nameParts.length];
System.arraycopy(nameParts, 0, normalizedNameParts, 0, nameParts.length);
for (int j = 0; j < nameParts.length - 1; j++) {
StringBuilder sb = new StringBuilder();
for (int k = 0; k <= j; k++) {
sb.append(nameParts[k]);
}
String parentName = sb.toString();
int parentFieldIdFromQuerySchema = querySchema.findIdByName(parentName);
String parentNameFromFileSchema = fileSchema.findfullName(parentFieldIdFromQuerySchema);
if (parentNameFromFileSchema.isEmpty()) {
break;
}
if (!parentNameFromFileSchema.equalsIgnoreCase(parentName)) {
// find parent rename, update nameParts
String[] parentNameParts = parentNameFromFileSchema.split("\\.");
System.arraycopy(parentNameParts, 0, normalizedNameParts, 0, parentNameParts.length);
}
}
return StringUtils.join(normalizedNameParts, ".");
}
private Type buildArrayType(Types.ArrayType array, Type newType) {
Types.Field elementField = array.fields().get(0);
int elementId = elementField.fieldId();
if (elementField.type() == newType) {
return array;
} else {
return Types.ArrayType.get(elementId, elementField.isOptional(), newType);
}
}
private Type buildMapType(Types.MapType map, Type newValue) {
Types.Field valueFiled = map.fields().get(1);
if (valueFiled.type() == newValue) {
return map;
} else {
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValue, map.isValueOptional());
}
}
private Type buildPrimitiveType(Type.PrimitiveType typeFromQuerySchema, int currentPrimitiveTypeId) {
Type typeFromFileSchema = fileSchema.findType(currentPrimitiveTypeId);
if (typeFromFileSchema == null) {
return typeFromQuerySchema;
} else {
return useColumnTypeFromFileSchema ? typeFromFileSchema : typeFromQuerySchema;
}
}
}

View File

@@ -0,0 +1,252 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.action;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.internal.schema.HoodieSchemaException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
import org.apache.hudi.internal.schema.Types;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
/**
* TableChange subclasses represent requested changes to a table.
* now only column changes support.
* to do support partition changes
*/
public interface TableChange {
/* The action Type of schema change. */
enum ColumnChangeID {
ADD, UPDATE, DELETE, PROPERTY_CHANGE, REPLACE;
private String name;
private ColumnChangeID() {
this.name = this.name().toLowerCase(Locale.ROOT);
}
public String getName() {
return name;
}
}
static ColumnChangeID fromValue(String value) {
switch (value.toLowerCase(Locale.ROOT)) {
case "add":
return ColumnChangeID.ADD;
case "change":
return ColumnChangeID.UPDATE;
case "delete":
return ColumnChangeID.DELETE;
case "property":
return ColumnChangeID.PROPERTY_CHANGE;
case "replace":
return ColumnChangeID.REPLACE;
default:
throw new IllegalArgumentException("Invalid value of Type.");
}
}
ColumnChangeID columnChangeId();
default boolean withPositionChange() {
return false;
}
abstract class BaseColumnChange implements TableChange {
protected final InternalSchema internalSchema;
protected final Map<Integer, Integer> id2parent;
protected final Map<Integer, ArrayList<ColumnPositionChange>> positionChangeMap = new HashMap<>();
BaseColumnChange(InternalSchema schema) {
this.internalSchema = schema;
this.id2parent = InternalSchemaBuilder.getBuilder().index2Parents(schema.getRecord());
}
/**
* Add position change.
*
* @param srcName column which need to be reordered
* @param dsrName reference position
* @param orderType change types
* @return this
*/
public BaseColumnChange addPositionChange(String srcName, String dsrName, ColumnPositionChange.ColumnPositionType orderType) {
Integer srcId = findIdByFullName(srcName);
Option<Integer> dsrIdOpt = dsrName.isEmpty() ? Option.empty() : Option.of(findIdByFullName(dsrName));
Integer srcParentId = id2parent.get(srcId);
Option<Integer> dsrParentIdOpt = dsrIdOpt.map(id2parent::get);
// forbid adjust hoodie metadata columns.
switch (orderType) {
case BEFORE:
checkColModifyIsLegal(dsrName);
break;
case FIRST:
if (srcId == null || srcId == -1 || srcParentId == null || srcParentId == -1) {
throw new HoodieSchemaException("forbid adjust top-level columns position by using through first syntax");
}
break;
case AFTER:
List<String> checkColumns = HoodieRecord.HOODIE_META_COLUMNS.subList(0, HoodieRecord.HOODIE_META_COLUMNS.size() - 2);
if (checkColumns.stream().anyMatch(f -> f.equalsIgnoreCase(dsrName))) {
throw new HoodieSchemaException("forbid adjust the position of ordinary columns between meta columns");
}
break;
case NO_OPERATION:
default:
break;
}
int parentId;
if (srcParentId != null && dsrParentIdOpt.isPresent() && srcParentId.equals(dsrParentIdOpt.get())) {
Types.Field parentField = internalSchema.findField(srcParentId);
if (!(parentField.type() instanceof Types.RecordType)) {
throw new HoodieSchemaException(String.format("only support reorder fields in struct type, but find: %s", parentField.type()));
}
parentId = parentField.fieldId();
} else if (srcParentId == null && !dsrParentIdOpt.isPresent()) {
parentId = -1;
} else if (srcParentId != null && !dsrParentIdOpt.isPresent() && orderType.equals(ColumnPositionChange.ColumnPositionType.FIRST)) {
parentId = srcParentId;
} else {
throw new HoodieSchemaException("cannot order position from different parent");
}
ArrayList<ColumnPositionChange> changes = positionChangeMap.getOrDefault(parentId, new ArrayList<>());
changes.add(ColumnPositionChange.get(srcId, dsrIdOpt.orElse(-1), orderType));
positionChangeMap.put(parentId, changes);
return this;
}
public BaseColumnChange addPositionChange(String srcName, String dsrName, String orderType) {
return addPositionChange(srcName, dsrName, ColumnPositionChange.fromTypeValue(orderType));
}
/**
* Abstract method.
* give a column fullName and return the field id
*
* @param fullName column fullName
* @return field id of current column
*/
protected abstract Integer findIdByFullName(String fullName);
// Modify hudi meta columns is prohibited
protected void checkColModifyIsLegal(String colNeedToModfiy) {
if (HoodieRecord.HOODIE_META_COLUMNS.stream().anyMatch(f -> f.equalsIgnoreCase(colNeedToModfiy))) {
throw new IllegalArgumentException(String.format("cannot modify hudi meta col: %s", colNeedToModfiy));
}
}
@Override
public boolean withPositionChange() {
return false;
}
}
/**
* Column position change.
* now support three change types: FIRST/AFTER/BEFORE
* FIRST means the specified column should be the first column.
* AFTER means the specified column should be put after the given column.
* BEFORE means the specified column should be put before the given column.
* Note that, the specified column may be a nested field:
* AFTER/BEFORE means the given columns should in the same struct;
* FIRST means this field should be the first one within the struct.
*/
class ColumnPositionChange {
public enum ColumnPositionType {
FIRST,
BEFORE,
AFTER,
// only expose to internal use.
NO_OPERATION
}
static ColumnPositionType fromTypeValue(String value) {
switch (value.toLowerCase(Locale.ROOT)) {
case "first":
return ColumnPositionType.FIRST;
case "before":
return ColumnPositionType.BEFORE;
case "after":
return ColumnPositionType.AFTER;
case "no_operation":
return ColumnPositionType.NO_OPERATION;
default:
throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", value));
}
}
private final int srcId;
private final int dsrId;
private final ColumnPositionType type;
static ColumnPositionChange first(int srcId) {
return new ColumnPositionChange(srcId, -1, ColumnPositionType.FIRST);
}
static ColumnPositionChange before(int srcId, int dsrId) {
return new ColumnPositionChange(srcId, dsrId, ColumnPositionType.BEFORE);
}
static ColumnPositionChange after(int srcId, int dsrId) {
return new ColumnPositionChange(srcId, dsrId, ColumnPositionType.AFTER);
}
static ColumnPositionChange get(int srcId, int dsrId, String type) {
return get(srcId, dsrId, fromTypeValue(type));
}
static ColumnPositionChange get(int srcId, int dsrId, ColumnPositionType type) {
switch (type) {
case FIRST:
return ColumnPositionChange.first(srcId);
case BEFORE:
return ColumnPositionChange.before(srcId, dsrId);
case AFTER:
return ColumnPositionChange.after(srcId, dsrId);
default:
throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", type));
}
}
private ColumnPositionChange(int srcId, int dsrId, ColumnPositionType type) {
this.srcId = srcId;
this.dsrId = dsrId;
this.type = type;
}
public int getSrcId() {
return srcId;
}
public int getDsrId() {
return dsrId;
}
public ColumnPositionType type() {
return type;
}
}
}

View File

@@ -0,0 +1,398 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.action;
import org.apache.hudi.internal.schema.HoodieSchemaException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import org.apache.hudi.internal.schema.utils.SchemaChangeUtils;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
public class TableChanges {
/** Deal with update columns changes for table. */
public static class ColumnUpdateChange extends TableChange.BaseColumnChange {
private final Map<Integer, Types.Field> updates = new HashMap<>();
public static ColumnUpdateChange get(InternalSchema schema) {
return new ColumnUpdateChange(schema);
}
private ColumnUpdateChange(InternalSchema schema) {
super(schema);
}
@Override
public boolean withPositionChange() {
return true;
}
public Type applyUpdates(Types.Field oldField, Type type) {
Types.Field update = updates.get(oldField.fieldId());
if (update != null && update.type() != oldField.type()) {
return update.type();
}
//
ArrayList<ColumnPositionChange> pchanges = positionChangeMap.getOrDefault(oldField.fieldId(), new ArrayList<>());
if (!pchanges.isEmpty()) {
// when we build ColumnAddChangewe have already done some check, so it's safe to convert newType to RecordType
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(((Types.RecordType) type).fields(), new ArrayList<>(), pchanges);
return Types.RecordType.get(newFields);
}
return type;
}
public Map<Integer, Types.Field> getUpdates() {
return updates;
}
/**
* Update a column in the schema to a new type.
* only support update primitive type.
* Only updates that widen types are allowed.
*
* @param name name of the column to update
* @param newType new type for the column
* @return this
* @throws IllegalArgumentException
*/
public ColumnUpdateChange updateColumnType(String name, Type newType) {
checkColModifyIsLegal(name);
if (newType.isNestedType()) {
throw new IllegalArgumentException(String.format("only support update primitive type but find nest column: %s", name));
}
Types.Field field = internalSchema.findField(name);
if (field == null) {
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
}
if (!SchemaChangeUtils.isTypeUpdateAllow(field.type(), newType)) {
throw new IllegalArgumentException(String.format("cannot update origin type: %s to a incompatibility type: %s", field.type(), newType));
}
if (field.type().equals(newType)) {
// do nothings
return this;
}
// save update info
Types.Field update = updates.get(field.fieldId());
if (update == null) {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), field.name(), newType, field.doc()));
} else {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), update.name(), newType, update.doc()));
}
return this;
}
/**
* Update a column doc in the schema to a new primitive type.
*
* @param name name of the column to update
* @param newDoc new documentation for the column
* @return this
* @throws IllegalArgumentException
*/
public ColumnUpdateChange updateColumnComment(String name, String newDoc) {
checkColModifyIsLegal(name);
Types.Field field = internalSchema.findField(name);
if (field == null) {
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
}
// consider null
if (Objects.equals(field.doc(), newDoc)) {
// do nothings
return this;
}
// save update info
Types.Field update = updates.get(field.fieldId());
if (update == null) {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), field.name(), field.type(), newDoc));
} else {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), update.name(), update.type(), newDoc));
}
return this;
}
/**
* Rename a column in the schema.
*
* @param name name of the column to rename
* @param newName new name for the column
* @return this
* @throws IllegalArgumentException
*/
public ColumnUpdateChange renameColumn(String name, String newName) {
checkColModifyIsLegal(name);
Types.Field field = internalSchema.findField(name);
if (field == null) {
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
}
if (newName == null || newName.isEmpty()) {
throw new IllegalArgumentException(String.format("cannot rename column: %s to empty", name));
}
// keep consisitent with hive. column names insensitive, so we check 'newName.toLowerCase(Locale.ROOT)'
if (internalSchema.findDuplicateCol(newName.toLowerCase(Locale.ROOT))) {
throw new IllegalArgumentException(String.format("cannot rename column: %s to a existing name", name));
}
// save update info
Types.Field update = updates.get(field.fieldId());
if (update == null) {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), newName, field.type(), field.doc()));
} else {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), newName, update.type(), update.doc()));
}
return this;
}
/**
* Update nullable for column.
* only support required type -> optional type
*
* @param name name of the column to update
* @param nullable nullable for updated name
* @return this
* @throws IllegalArgumentException
*/
public ColumnUpdateChange updateColumnNullability(String name, boolean nullable) {
return updateColumnNullability(name, nullable, false);
}
public ColumnUpdateChange updateColumnNullability(String name, boolean nullable, boolean force) {
checkColModifyIsLegal(name);
Types.Field field = internalSchema.findField(name);
if (field == null) {
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
}
if (field.isOptional() == nullable) {
// do nothings
return this;
}
if (field.isOptional() && !nullable && !force) {
throw new IllegalArgumentException("cannot update column Nullability: optional to required");
}
// save update info
Types.Field update = updates.get(field.fieldId());
if (update == null) {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), nullable, field.name(), field.type(), field.doc()));
} else {
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), nullable, update.name(), update.type(), update.doc()));
}
return this;
}
public Map<Integer, ArrayList<ColumnPositionChange>> getPositionChangeMap() {
return positionChangeMap;
}
@Override
public ColumnChangeID columnChangeId() {
return ColumnChangeID.UPDATE;
}
@Override
protected Integer findIdByFullName(String fullName) {
Types.Field field = internalSchema.findField(fullName);
if (field != null) {
return field.fieldId();
} else {
throw new IllegalArgumentException(String.format("cannot find col id for given column fullName: %s", fullName));
}
}
}
/** Deal with delete columns changes for table. */
public static class ColumnDeleteChange extends TableChange.BaseColumnChange {
private final Set deletes = new HashSet<>();
@Override
public ColumnChangeID columnChangeId() {
return ColumnChangeID.DELETE;
}
public static ColumnDeleteChange get(InternalSchema schema) {
return new ColumnDeleteChange(schema);
}
private ColumnDeleteChange(InternalSchema schema) {
super(schema);
}
@Override
public boolean withPositionChange() {
return false;
}
@Override
public BaseColumnChange addPositionChange(String srcId, String dsrId, String orderType) {
throw new UnsupportedOperationException("no support add position change for ColumnDeleteChange");
}
public ColumnDeleteChange deleteColumn(String name) {
checkColModifyIsLegal(name);
Types.Field field = internalSchema.findField(name);
if (field == null) {
throw new IllegalArgumentException(String.format("cannot delete missing columns: %s", name));
}
deletes.add(field.fieldId());
return this;
}
public Type applyDelete(int id, Type type) {
if (deletes.contains(id)) {
return null;
}
return type;
}
public Set<Integer> getDeletes() {
return deletes;
}
@Override
protected Integer findIdByFullName(String fullName) {
throw new UnsupportedOperationException("delete change cannot support this method");
}
}
/** Deal with add columns changes for table. */
public static class ColumnAddChange extends TableChange.BaseColumnChange {
private final Map<String, Integer> fullColName2Id = new HashMap<>();
private final Map<Integer, ArrayList<Types.Field>> parentId2AddCols = new HashMap<>();
private int nextId;
public static ColumnAddChange get(InternalSchema internalSchema) {
return new ColumnAddChange(internalSchema);
}
public Type applyAdd(Types.Field orignalField, Type type) {
int fieldId = orignalField.fieldId();
ArrayList<Types.Field> addFields = parentId2AddCols.getOrDefault(fieldId, new ArrayList<>());
ArrayList<ColumnPositionChange> pchanges = positionChangeMap.getOrDefault(fieldId, new ArrayList<>());
if (!addFields.isEmpty() || !pchanges.isEmpty()) {
// when we build ColumnAddChangewe have already done some check, so it's safe to convert newType to RecordType
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(((Types.RecordType) type).fields(), addFields, pchanges);
return Types.RecordType.get(newFields);
}
return type;
}
public ColumnAddChange addColumns(String name, Type type, String doc) {
checkColModifyIsLegal(name);
return addColumns("", name, type, doc);
}
public ColumnAddChange addColumns(String parent, String name, Type type, String doc) {
checkColModifyIsLegal(name);
addColumnsInternal(parent, name, type, doc);
return this;
}
private void addColumnsInternal(String parent, String name, Type type, String doc) {
// root record has no parent, so set parentId to -1 as default
int parentId = -1;
// do check
String fullName = name;
if (!parent.isEmpty()) {
Types.Field parentField = internalSchema.findField(parent);
if (parentField == null) {
throw new HoodieSchemaException(String.format("cannot add column: %s which parent: %s is not exist", name, parent));
}
Type parentType = parentField.type();
if (!(parentField.type() instanceof Types.RecordType)) {
throw new HoodieSchemaException("only support add nested columns to struct column");
}
parentId = parentField.fieldId();
Types.Field newParentField = internalSchema.findField(parent + "." + name);
if (newParentField != null) {
throw new HoodieSchemaException(String.format("cannot add column: %s which already exist", name));
}
fullName = parent + "." + name;
} else {
// keep consistent with hive, column name case insensitive
if (internalSchema.findDuplicateCol(name.toLowerCase(Locale.ROOT))) {
throw new HoodieSchemaException(String.format("cannot add column: %s which already exist", name));
}
}
if (fullColName2Id.containsKey(fullName)) {
throw new HoodieSchemaException(String.format("cannot repeat add column: %s", name));
}
fullColName2Id.put(fullName, nextId);
if (parentId != -1) {
id2parent.put(nextId, parentId);
}
AtomicInteger assignNextId = new AtomicInteger(nextId + 1);
Type typeWithNewId = InternalSchemaBuilder.getBuilder().refreshNewId(type, assignNextId);
// only allow add optional columns.
ArrayList<Types.Field> adds = parentId2AddCols.getOrDefault(parentId, new ArrayList<>());
adds.add(Types.Field.get(nextId, true, name, typeWithNewId, doc));
parentId2AddCols.put(parentId, adds);
nextId = assignNextId.get();
}
private ColumnAddChange(InternalSchema internalSchema) {
super(internalSchema);
this.nextId = internalSchema.getMaxColumnId() + 1;
}
public Map<Integer, ArrayList<Types.Field>> getParentId2AddCols() {
return parentId2AddCols;
}
public Map<Integer, ArrayList<ColumnPositionChange>> getPositionChangeMap() {
return positionChangeMap;
}
// expose to test
public Map<String, Integer> getFullColName2Id() {
return fullColName2Id;
}
protected Integer findIdByFullName(String fullName) {
Types.Field field = internalSchema.findField(fullName);
if (field != null) {
return field.fieldId();
}
return fullColName2Id.getOrDefault(fullName, -1);
}
@Override
public ColumnChangeID columnChangeId() {
return ColumnChangeID.ADD;
}
@Override
public boolean withPositionChange() {
return true;
}
}
}

View File

@@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.action;
import org.apache.hudi.internal.schema.Types;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
/**
* Helper class to support Table schema changes.
*/
public class TableChangesHelper {
/**
* Apply add operation and column position change operation.
*
* @param fields origin column fields.
* @param adds column fields to be added.
* @param pchanges a wrapper class hold all the position change operations.
* @return column fields after adjusting the position.
*/
public static List<Types.Field> applyAddChange2Fields(List<Types.Field> fields, ArrayList<Types.Field> adds, ArrayList<TableChange.ColumnPositionChange> pchanges) {
if (adds == null && pchanges == null) {
return fields;
}
LinkedList<Types.Field> result = new LinkedList<>(fields);
// apply add columns
if (adds != null && !adds.isEmpty()) {
result.addAll(adds);
}
// apply position change
if (pchanges != null && !pchanges.isEmpty()) {
for (TableChange.ColumnPositionChange pchange : pchanges) {
Types.Field srcField = result.stream().filter(f -> f.fieldId() == pchange.getSrcId()).findFirst().get();
Types.Field dsrField = result.stream().filter(f -> f.fieldId() == pchange.getDsrId()).findFirst().orElse(null);
// we remove srcField first
result.remove(srcField);
switch (pchange.type()) {
case AFTER:
// add srcField after dsrField
result.add(result.indexOf(dsrField) + 1, srcField);
break;
case BEFORE:
// add srcField before dsrField
result.add(result.indexOf(dsrField), srcField);
break;
case FIRST:
result.addFirst(srcField);
break;
default:
// should not reach here
}
}
}
return result;
}
public static String getParentName(String fullColName) {
int offset = fullColName.lastIndexOf(".");
return offset > 0 ? fullColName.substring(0, offset) : "";
}
}

View File

@@ -0,0 +1,436 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.convert;
import org.apache.avro.JsonProperties;
import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.hudi.internal.schema.HoodieSchemaException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import java.util.ArrayList;
import java.util.Deque;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.avro.Schema.Type.UNION;
/**
* Auxiliary class.
* Converts an avro schema into InternalSchema, or convert InternalSchema to an avro schema
*/
public class AvroInternalSchemaConverter {
/**
* Convert internalSchema to avro Schema.
*
* @param internalSchema internal schema.
* @param tableName the record name.
* @return an avro Schema.
*/
public static Schema convert(InternalSchema internalSchema, String tableName) {
return buildAvroSchemaFromInternalSchema(internalSchema, tableName);
}
/**
* Convert RecordType to avro Schema.
*
* @param type internal schema.
* @param name the record name.
* @return an avro Schema.
*/
public static Schema convert(Types.RecordType type, String name) {
return buildAvroSchemaFromType(type, name);
}
/**
* Convert internal type to avro Schema.
*
* @param type internal type.
* @param name the record name.
* @return an avro Schema.
*/
public static Schema convert(Type type, String name) {
return buildAvroSchemaFromType(type, name);
}
/** Convert an avro schema into internal type. */
public static Type convertToField(Schema schema) {
return buildTypeFromAvroSchema(schema);
}
/** Convert an avro schema into internalSchema. */
public static InternalSchema convert(Schema schema) {
List<Types.Field> fields = ((Types.RecordType) convertToField(schema)).fields();
return new InternalSchema(fields);
}
/** Check whether current avro schema is optional?. */
public static boolean isOptional(Schema schema) {
if (schema.getType() == UNION && schema.getTypes().size() == 2) {
return schema.getTypes().get(0).getType() == Schema.Type.NULL || schema.getTypes().get(1).getType() == Schema.Type.NULL;
}
return false;
}
/** Returns schema with nullable true. */
public static Schema nullableSchema(Schema schema) {
if (schema.getType() == UNION) {
if (!isOptional(schema)) {
throw new HoodieSchemaException(String.format("Union schemas are not supported: %s", schema));
}
return schema;
} else {
return Schema.createUnion(Schema.create(Schema.Type.NULL), schema);
}
}
/**
* Build hudi type from avro schema.
*
* @param schema a avro schema.
* @return a hudi type.
*/
public static Type buildTypeFromAvroSchema(Schema schema) {
// set flag to check this has not been visited.
Deque<String> visited = new LinkedList();
AtomicInteger nextId = new AtomicInteger(1);
return visitAvroSchemaToBuildType(schema, visited, true, nextId);
}
/**
* Converts an avro schema into hudi type.
*
* @param schema a avro schema.
* @param visited track the visit node when do traversal for avro schema; used to check if the name of avro record schema is correct.
* @param firstVisitRoot track whether the current visited schema node is a root node.
* @param nextId a initial id which used to create id for all fields.
* @return a hudi type match avro schema.
*/
private static Type visitAvroSchemaToBuildType(Schema schema, Deque<String> visited, Boolean firstVisitRoot, AtomicInteger nextId) {
switch (schema.getType()) {
case RECORD:
String name = schema.getFullName();
if (visited.contains(name)) {
throw new HoodieSchemaException(String.format("cannot convert recursive avro record %s", name));
}
visited.push(name);
List<Schema.Field> fields = schema.getFields();
List<Type> fieldTypes = new ArrayList<>(fields.size());
int nextAssignId = nextId.get();
// when first visit root record, set nextAssignId = 0;
if (firstVisitRoot) {
nextAssignId = 0;
}
nextId.set(nextAssignId + fields.size());
fields.stream().forEach(field -> {
fieldTypes.add(visitAvroSchemaToBuildType(field.schema(), visited, false, nextId));
});
visited.pop();
List<Types.Field> internalFields = new ArrayList<>(fields.size());
for (int i = 0; i < fields.size(); i++) {
Schema.Field field = fields.get(i);
Type fieldType = fieldTypes.get(i);
internalFields.add(Types.Field.get(nextAssignId, AvroInternalSchemaConverter.isOptional(field.schema()), field.name(), fieldType, field.doc()));
nextAssignId += 1;
}
return Types.RecordType.get(internalFields);
case UNION:
List<Type> fTypes = new ArrayList<>();
schema.getTypes().stream().forEach(t -> {
fTypes.add(visitAvroSchemaToBuildType(t, visited, false, nextId));
});
return fTypes.get(0) == null ? fTypes.get(1) : fTypes.get(0);
case ARRAY:
Schema elementSchema = schema.getElementType();
int elementId = nextId.get();
nextId.set(elementId + 1);
Type elementType = visitAvroSchemaToBuildType(elementSchema, visited, false, nextId);
return Types.ArrayType.get(elementId, AvroInternalSchemaConverter.isOptional(schema.getElementType()), elementType);
case MAP:
int keyId = nextId.get();
int valueId = keyId + 1;
nextId.set(valueId + 1);
Type valueType = visitAvroSchemaToBuildType(schema.getValueType(), visited, false, nextId);
return Types.MapType.get(keyId, valueId, Types.StringType.get(), valueType, AvroInternalSchemaConverter.isOptional(schema.getValueType()));
default:
return visitAvroPrimitiveToBuildInternalType(schema);
}
}
private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) {
LogicalType logical = primitive.getLogicalType();
if (logical != null) {
String name = logical.getName();
if (logical instanceof LogicalTypes.Decimal) {
return Types.DecimalType.get(
((LogicalTypes.Decimal) logical).getPrecision(),
((LogicalTypes.Decimal) logical).getScale());
} else if (logical instanceof LogicalTypes.Date) {
return Types.DateType.get();
} else if (
logical instanceof LogicalTypes.TimeMillis
|| logical instanceof LogicalTypes.TimeMicros) {
return Types.TimeType.get();
} else if (
logical instanceof LogicalTypes.TimestampMillis
|| logical instanceof LogicalTypes.TimestampMicros) {
return Types.TimestampType.get();
} else if (LogicalTypes.uuid().getName().equals(name)) {
return Types.UUIDType.get();
}
}
switch (primitive.getType()) {
case BOOLEAN:
return Types.BooleanType.get();
case INT:
return Types.IntType.get();
case LONG:
return Types.LongType.get();
case FLOAT:
return Types.FloatType.get();
case DOUBLE:
return Types.DoubleType.get();
case STRING:
case ENUM:
return Types.StringType.get();
case FIXED:
return Types.FixedType.getFixed(primitive.getFixedSize());
case BYTES:
return Types.BinaryType.get();
case NULL:
return null;
default:
throw new UnsupportedOperationException("Unsupported primitive type: " + primitive);
}
}
/**
* Converts hudi type into an Avro Schema.
*
* @param type a hudi type.
* @param recordName the record name
* @return a Avro schema match this type
*/
public static Schema buildAvroSchemaFromType(Type type, String recordName) {
Map<Type, Schema> cache = new HashMap<>();
return visitInternalSchemaToBuildAvroSchema(type, cache, recordName);
}
/**
* Converts hudi internal Schema into an Avro Schema.
*
* @param schema a hudi internal Schema.
* @param recordName the record name
* @return a Avro schema match hudi internal schema.
*/
public static Schema buildAvroSchemaFromInternalSchema(InternalSchema schema, String recordName) {
Map<Type, Schema> cache = new HashMap<>();
return visitInternalSchemaToBuildAvroSchema(schema.getRecord(), cache, recordName);
}
/**
* Converts hudi type into an Avro Schema.
*
* @param type a hudi type.
* @param cache use to cache intermediate convert result to save cost.
* @param recordName the record name
* @return a Avro schema match this type
*/
private static Schema visitInternalSchemaToBuildAvroSchema(Type type, Map<Type, Schema> cache, String recordName) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Schema> schemas = new ArrayList<>();
record.fields().forEach(f -> {
Schema tempSchema = visitInternalSchemaToBuildAvroSchema(f.type(), cache, recordName + "_" + f.name());
// convert tempSchema
Schema result = f.isOptional() ? AvroInternalSchemaConverter.nullableSchema(tempSchema) : tempSchema;
schemas.add(result);
});
// check visited
Schema recordSchema;
recordSchema = cache.get(record);
if (recordSchema != null) {
return recordSchema;
}
recordSchema = visitInternalRecordToBuildAvroRecord(record, schemas, recordName);
cache.put(record, recordSchema);
return recordSchema;
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Schema elementSchema;
elementSchema = visitInternalSchemaToBuildAvroSchema(array.elementType(), cache, recordName);
Schema arraySchema;
arraySchema = cache.get(array);
if (arraySchema != null) {
return arraySchema;
}
arraySchema = visitInternalArrayToBuildAvroArray(array, elementSchema);
cache.put(array, arraySchema);
return arraySchema;
case MAP:
Types.MapType map = (Types.MapType) type;
Schema keySchema;
Schema valueSchema;
keySchema = visitInternalSchemaToBuildAvroSchema(map.keyType(), cache, recordName);
valueSchema = visitInternalSchemaToBuildAvroSchema(map.valueType(), cache, recordName);
Schema mapSchema;
mapSchema = cache.get(map);
if (mapSchema != null) {
return mapSchema;
}
mapSchema = visitInternalMapToBuildAvroMap(map, keySchema, valueSchema);
cache.put(map, mapSchema);
return mapSchema;
default:
Schema primitiveSchema = visitInternalPrimitiveToBuildAvroPrimitiveType((Type.PrimitiveType) type);
cache.put(type, primitiveSchema);
return primitiveSchema;
}
}
/**
* Converts hudi RecordType to Avro RecordType.
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
*/
private static Schema visitInternalRecordToBuildAvroRecord(Types.RecordType record, List<Schema> fieldSchemas, String recordName) {
List<Types.Field> fields = record.fields();
List<Schema.Field> avroFields = new ArrayList<>();
for (int i = 0; i < fields.size(); i++) {
Types.Field f = fields.get(i);
Schema.Field field = new Schema.Field(f.name(), fieldSchemas.get(i), f.doc(), f.isOptional() ? JsonProperties.NULL_VALUE : null);
avroFields.add(field);
}
return Schema.createRecord(recordName, null, null, false, avroFields);
}
/**
* Converts hudi ArrayType to Avro ArrayType.
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
*/
private static Schema visitInternalArrayToBuildAvroArray(Types.ArrayType array, Schema elementSchema) {
Schema result;
if (array.isElementOptional()) {
result = Schema.createArray(AvroInternalSchemaConverter.nullableSchema(elementSchema));
} else {
result = Schema.createArray(elementSchema);
}
return result;
}
/**
* Converts hudi MapType to Avro MapType.
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
*/
private static Schema visitInternalMapToBuildAvroMap(Types.MapType map, Schema keySchema, Schema valueSchema) {
Schema mapSchema;
if (keySchema.getType() == Schema.Type.STRING) {
mapSchema = Schema.createMap(map.isValueOptional() ? AvroInternalSchemaConverter.nullableSchema(valueSchema) : valueSchema);
} else {
throw new HoodieSchemaException("only support StringType key for avro MapType");
}
return mapSchema;
}
/**
* Converts hudi PrimitiveType to Avro PrimitiveType.
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
*/
private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.PrimitiveType primitive) {
Schema primitiveSchema;
switch (primitive.typeId()) {
case BOOLEAN:
primitiveSchema = Schema.create(Schema.Type.BOOLEAN);
break;
case INT:
primitiveSchema = Schema.create(Schema.Type.INT);
break;
case LONG:
primitiveSchema = Schema.create(Schema.Type.LONG);
break;
case FLOAT:
primitiveSchema = Schema.create(Schema.Type.FLOAT);
break;
case DOUBLE:
primitiveSchema = Schema.create(Schema.Type.DOUBLE);
break;
case DATE:
primitiveSchema = LogicalTypes.date()
.addToSchema(Schema.create(Schema.Type.INT));
break;
case TIME:
primitiveSchema = LogicalTypes.timeMicros()
.addToSchema(Schema.create(Schema.Type.LONG));
break;
case TIMESTAMP:
primitiveSchema = LogicalTypes.timestampMicros()
.addToSchema(Schema.create(Schema.Type.LONG));
break;
case STRING:
primitiveSchema = Schema.create(Schema.Type.STRING);
break;
case UUID:
primitiveSchema = LogicalTypes.uuid()
.addToSchema(Schema.createFixed("uuid_fixed", null, null, 16));
break;
case FIXED:
Types.FixedType fixed = (Types.FixedType) primitive;
primitiveSchema = Schema.createFixed("fixed_" + fixed.getFixedSize(), null, null, fixed.getFixedSize());
break;
case BINARY:
primitiveSchema = Schema.create(Schema.Type.BYTES);
break;
case DECIMAL:
Types.DecimalType decimal = (Types.DecimalType) primitive;
primitiveSchema = LogicalTypes.decimal(decimal.precision(), decimal.scale())
.addToSchema(Schema.createFixed(
"decimal_" + decimal.precision() + "_" + decimal.scale(),
null, null, computeMinBytesForPrecision(decimal.precision())));
break;
default:
throw new UnsupportedOperationException(
"Unsupported type ID: " + primitive.typeId());
}
return primitiveSchema;
}
/**
* Return the minimum number of bytes needed to store a decimal with a give 'precision'.
* reference from Spark release 3.1 .
*/
private static int computeMinBytesForPrecision(int precision) {
int numBytes = 1;
while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) {
numBytes += 1;
}
return numBytes;
}
}

View File

@@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.io;
import org.apache.hudi.common.util.Option;
import java.util.List;
abstract class AbstractInternalSchemaStorageManager {
/**
* Persist history schema str.
*/
public abstract void persistHistorySchemaStr(String instantTime, String historySchemaStr);
/**
* Get latest history schema string.
*/
public abstract String getHistorySchemaStr();
/**
* Get latest history schema string.
* Using give validCommits to validate all legal histroy Schema files, and return the latest one.
* If the passed valid commits is null or empty, valid instants will be fetched from the file-system and used.
*/
public abstract String getHistorySchemaStrByGivenValidCommits(List<String> validCommits);
/**
* Get internalSchema by using given versionId
*
* @param versionId schema version_id need to search
* @return internalSchema
*/
public abstract Option getSchemaByKey(String versionId);
}

View File

@@ -0,0 +1,184 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.io;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
import org.apache.hudi.internal.schema.utils.SerDeHelper;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.TreeMap;
import java.util.stream.Collectors;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.SCHEMA_COMMIT_ACTION;
public class FileBasedInternalSchemaStorageManager extends AbstractInternalSchemaStorageManager {
private static final Logger LOG = LogManager.getLogger(FileBasedInternalSchemaStorageManager.class);
public static final String SCHEMA_NAME = ".schema";
private final Path baseSchemaPath;
private final Configuration conf;
private HoodieTableMetaClient metaClient;
public FileBasedInternalSchemaStorageManager(Configuration conf, Path baseTablePath) {
Path metaPath = new Path(baseTablePath, ".hoodie");
this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
this.conf = conf;
}
public FileBasedInternalSchemaStorageManager(HoodieTableMetaClient metaClient) {
Path metaPath = new Path(metaClient.getBasePath(), ".hoodie");
this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
this.conf = metaClient.getHadoopConf();
this.metaClient = metaClient;
}
// make metaClient build lazy
private HoodieTableMetaClient getMetaClient() {
if (metaClient == null) {
metaClient = HoodieTableMetaClient.builder().setBasePath(baseSchemaPath.getParent().getParent().toString()).setConf(conf).build();
}
return metaClient;
}
@Override
public void persistHistorySchemaStr(String instantTime, String historySchemaStr) {
cleanResidualFiles();
HoodieActiveTimeline timeline = getMetaClient().getActiveTimeline();
HoodieInstant hoodieInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, SCHEMA_COMMIT_ACTION, instantTime);
timeline.createNewInstant(hoodieInstant);
byte[] writeContent = historySchemaStr.getBytes(StandardCharsets.UTF_8);
timeline.transitionRequestedToInflight(hoodieInstant, Option.empty());
timeline.saveAsComplete(new HoodieInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.getTimestamp()), Option.of(writeContent));
LOG.info(String.format("persist history schema success on commit time: %s", instantTime));
}
private void cleanResidualFiles() {
List<String> validateCommits = getValidInstants();
try {
FileSystem fs = baseSchemaPath.getFileSystem(conf);
if (fs.exists(baseSchemaPath)) {
List<String> candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile())
.map(file -> file.getPath().getName()).collect(Collectors.toList());
List<String> residualSchemaFiles = candidateSchemaFiles.stream().filter(f -> !validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList());
// clean residual files
residualSchemaFiles.forEach(f -> {
try {
fs.delete(new Path(getMetaClient().getSchemaFolderName(), f));
} catch (IOException o) {
throw new HoodieException(o);
}
});
}
} catch (IOException e) {
throw new HoodieException(e);
}
}
public void cleanOldFiles(List<String> validateCommits) {
try {
FileSystem fs = baseSchemaPath.getFileSystem(conf);
if (fs.exists(baseSchemaPath)) {
List<String> candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile())
.map(file -> file.getPath().getName()).collect(Collectors.toList());
List<String> validateSchemaFiles = candidateSchemaFiles.stream().filter(f -> validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList());
for (int i = 0; i < validateSchemaFiles.size(); i++) {
fs.delete(new Path(validateSchemaFiles.get(i)));
}
}
} catch (IOException e) {
throw new HoodieException(e);
}
}
private List<String> getValidInstants() {
return getMetaClient().getCommitsTimeline()
.filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList());
}
@Override
public String getHistorySchemaStr() {
return getHistorySchemaStrByGivenValidCommits(Collections.EMPTY_LIST);
}
@Override
public String getHistorySchemaStrByGivenValidCommits(List<String> validCommits) {
List<String> commitList = validCommits == null || validCommits.isEmpty() ? getValidInstants() : validCommits;
try {
FileSystem fs = FSUtils.getFs(baseSchemaPath.toString(), conf);
if (fs.exists(baseSchemaPath)) {
List<String> validaSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath))
.filter(f -> f.isFile() && f.getPath().getName().endsWith(SCHEMA_COMMIT_ACTION))
.map(file -> file.getPath().getName()).filter(f -> commitList.contains(f.split("\\.")[0])).sorted().collect(Collectors.toList());
if (!validaSchemaFiles.isEmpty()) {
Path latestFilePath = new Path(baseSchemaPath, validaSchemaFiles.get(validaSchemaFiles.size() - 1));
byte[] content;
try (FSDataInputStream is = fs.open(latestFilePath)) {
content = FileIOUtils.readAsByteArray(is);
LOG.info(String.format("read history schema success from file : %s", latestFilePath));
return new String(content, StandardCharsets.UTF_8);
} catch (IOException e) {
throw new HoodieIOException("Could not read history schema from " + latestFilePath, e);
}
}
}
} catch (IOException io) {
throw new HoodieException(io);
}
LOG.info("failed to read history schema");
return "";
}
@Override
public Option<InternalSchema> getSchemaByKey(String versionId) {
String historySchemaStr = getHistorySchemaStr();
TreeMap<Long, InternalSchema> treeMap;
if (historySchemaStr.isEmpty()) {
return Option.empty();
} else {
treeMap = SerDeHelper.parseSchemas(historySchemaStr);
InternalSchema result = InternalSchemaUtils.searchSchema(Long.valueOf(versionId), treeMap);
if (result == null) {
return Option.empty();
}
return Option.of(result);
}
}
}

View File

@@ -0,0 +1,142 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.utils;
import org.apache.avro.Schema;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Types;
import org.apache.hudi.internal.schema.action.TableChanges;
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
import java.util.ArrayList;
import java.util.List;
import java.util.TreeMap;
import java.util.stream.Collectors;
/**
* Utility methods to support evolve old avro schema based on a given schema.
*/
public class AvroSchemaEvolutionUtils {
/**
* Support evolution from a new avroSchema.
* Now hoodie support implicitly add columns when hoodie write operation,
* This ability needs to be preserved, so implicitly evolution for internalSchema should supported.
*
* @param evolvedSchema implicitly evolution of avro when hoodie write operation
* @param oldSchema old internalSchema
* @param supportPositionReorder support position reorder
* @return evolution Schema
*/
public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema, Boolean supportPositionReorder) {
InternalSchema evolvedInternalSchema = AvroInternalSchemaConverter.convert(evolvedSchema);
// do check, only support add column evolution
List<String> colNamesFromEvolved = evolvedInternalSchema.getAllColsFullName();
List<String> colNamesFromOldSchema = oldSchema.getAllColsFullName();
List<String> diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromEvolved.contains(f)).collect(Collectors.toList());
List<Types.Field> newFields = new ArrayList<>();
if (colNamesFromEvolved.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) {
// no changes happen
if (supportPositionReorder) {
evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name())));
return new InternalSchema(newFields);
}
return oldSchema;
}
// try to find all added columns
if (diffFromOldSchema.size() != 0) {
throw new UnsupportedOperationException("Cannot evolve schema implicitly, find delete/rename operation");
}
List<String> diffFromEvolutionSchema = colNamesFromEvolved.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList());
// Remove redundancy from diffFromEvolutionSchema.
// for example, now we add a struct col in evolvedSchema, the struct col is " user struct<name:string, age:int> "
// when we do diff operation: user, user.name, user.age will appeared in the resultSet which is redundancy, user.name and user.age should be excluded.
// deal with add operation
TreeMap<Integer, String> finalAddAction = new TreeMap<>();
for (int i = 0; i < diffFromEvolutionSchema.size(); i++) {
String name = diffFromEvolutionSchema.get(i);
int splitPoint = name.lastIndexOf(".");
String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : "";
if (!parentName.isEmpty() && diffFromEvolutionSchema.contains(parentName)) {
// find redundancy, skip it
continue;
}
finalAddAction.put(evolvedInternalSchema.findIdByName(name), name);
}
TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema);
finalAddAction.entrySet().stream().forEach(f -> {
String name = f.getValue();
int splitPoint = name.lastIndexOf(".");
String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : "";
String rawName = splitPoint > 0 ? name.substring(splitPoint + 1) : name;
addChange.addColumns(parentName, rawName, evolvedInternalSchema.findType(name), null);
});
InternalSchema res = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange);
if (supportPositionReorder) {
evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name())));
return new InternalSchema(newFields);
} else {
return res;
}
}
public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema) {
return evolveSchemaFromNewAvroSchema(evolvedSchema, oldSchema, false);
}
/**
* Canonical the nullability.
* Do not allow change cols Nullability field from optional to required.
* If above problem occurs, try to correct it.
*
* @param writeSchema writeSchema hoodie used to write data.
* @param readSchema read schema
* @return canonical Schema
*/
public static Schema canonicalizeColumnNullability(Schema writeSchema, Schema readSchema) {
if (writeSchema.getFields().isEmpty() || readSchema.getFields().isEmpty()) {
return writeSchema;
}
InternalSchema writeInternalSchema = AvroInternalSchemaConverter.convert(writeSchema);
InternalSchema readInternalSchema = AvroInternalSchemaConverter.convert(readSchema);
List<String> colNamesWriteSchema = writeInternalSchema.getAllColsFullName();
List<String> colNamesFromReadSchema = readInternalSchema.getAllColsFullName();
// try to deal with optional change. now when we use sparksql to update hudi table,
// sparksql Will change the col type from optional to required, this is a bug.
List<String> candidateUpdateCols = colNamesWriteSchema.stream().filter(f -> {
boolean exist = colNamesFromReadSchema.contains(f);
if (exist && (writeInternalSchema.findField(f).isOptional() != readInternalSchema.findField(f).isOptional())) {
return true;
} else {
return false;
}
}).collect(Collectors.toList());
if (candidateUpdateCols.isEmpty()) {
return writeSchema;
}
// try to correct all changes
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(writeInternalSchema);
candidateUpdateCols.stream().forEach(f -> updateChange.updateColumnNullability(f, true));
Schema result = AvroInternalSchemaConverter.convert(SchemaChangeUtils.applyTableChanges2Schema(writeInternalSchema, updateChange), writeSchema.getName());
return result;
}
}

View File

@@ -0,0 +1,270 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.utils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.internal.schema.HoodieSchemaException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import org.apache.hudi.internal.schema.Types.Field;
import java.util.ArrayList;
import java.util.Deque;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.SortedMap;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Util methods to help us do some operations on InternalSchema.
* eg: column prune, filter rebuild for query engine...
*/
public class InternalSchemaUtils {
private InternalSchemaUtils() {
}
/**
* Create project internalSchema, based on the project names which produced by query engine.
* support nested project.
*
* @param schema a internal schema.
* @param names project names produced by query engine.
* @return a project internalSchema.
*/
public static InternalSchema pruneInternalSchema(InternalSchema schema, List<String> names) {
// do check
List<Integer> prunedIds = names.stream().map(name -> {
int id = schema.findIdByName(name);
if (id == -1) {
throw new IllegalArgumentException(String.format("cannot prune col: %s which not exisit in hudi table", name));
}
return id;
}).collect(Collectors.toList());
// find top parent field ID. eg: a.b.c, f.g.h, only collect id of a and f ignore all child field.
List<Integer> topParentFieldIds = new ArrayList<>();
names.stream().forEach(f -> {
int id = schema.findIdByName(f.split("\\.")[0]);
if (!topParentFieldIds.contains(id)) {
topParentFieldIds.add(id);
}
});
return pruneInternalSchemaByID(schema, prunedIds, topParentFieldIds);
}
/**
* Create project internalSchema.
* support nested project.
*
* @param schema a internal schema.
* @param fieldIds project col field_ids.
* @return a project internalSchema.
*/
public static InternalSchema pruneInternalSchemaByID(InternalSchema schema, List<Integer> fieldIds, List<Integer> topParentFieldIds) {
Types.RecordType recordType = (Types.RecordType)pruneType(schema.getRecord(), fieldIds);
// reorder top parent fields, since the recordType.fields() produced by pruneType maybe out of order.
List<Types.Field> newFields = new ArrayList<>();
if (topParentFieldIds != null && !topParentFieldIds.isEmpty()) {
for (int id : topParentFieldIds) {
Types.Field f = recordType.field(id);
if (f != null) {
newFields.add(f);
} else {
throw new HoodieSchemaException(String.format("cannot find pruned id %s in currentSchema %s", id, schema.toString()));
}
}
}
return new InternalSchema(newFields.isEmpty() ? recordType.fields() : newFields);
}
/**
* Project hudi type by projected cols field_ids
* this is auxiliary function used by pruneInternalSchema.
*/
private static Type pruneType(Type type, List<Integer> fieldIds) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Types.Field> fields = record.fields();
List<Type> newTypes = new ArrayList<>();
for (Types.Field f : fields) {
Type newType = pruneType(f.type(), fieldIds);
if (fieldIds.contains(f.fieldId())) {
newTypes.add(f.type());
} else if (newType != null) {
newTypes.add(newType);
} else {
newTypes.add(null);
}
}
boolean changed = false;
List<Field> newFields = new ArrayList<>();
for (int i = 0; i < fields.size(); i++) {
Types.Field oldField = fields.get(i);
Type newType = newTypes.get(i);
if (oldField.type() == newType) {
newFields.add(oldField);
} else if (newType != null) {
changed = true;
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
}
}
if (newFields.isEmpty()) {
return null;
}
if (newFields.size() == fields.size() && !changed) {
return record;
} else {
return Types.RecordType.get(newFields);
}
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Type newElementType = pruneType(array.elementType(), fieldIds);
if (fieldIds.contains(array.elementId())) {
return array;
} else if (newElementType != null) {
if (array.elementType() == newElementType) {
return array;
}
return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType);
}
return null;
case MAP:
Types.MapType map = (Types.MapType) type;
Type newValueType = pruneType(map.valueType(), fieldIds);
if (fieldIds.contains(map.valueId())) {
return map;
} else if (newValueType != null) {
if (map.valueType() == newValueType) {
return map;
}
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional());
}
return null;
default:
return null;
}
}
/**
* A helper function to help correct the colName of pushed filters.
*
* @param name origin col name from pushed filters.
* @param fileSchema the real schema of avro/parquet file.
* @param querySchema the query schema which query engine produced.
* @return a corrected name.
*/
public static String reBuildFilterName(String name, InternalSchema fileSchema, InternalSchema querySchema) {
int nameId = querySchema.findIdByName(name);
if (nameId == -1) {
throw new IllegalArgumentException(String.format("cannot found filter col name%s from querySchema: %s", name, querySchema));
}
if (fileSchema.findField(nameId) == null) {
// added operation found
// the read file does not contain current col, so current colFilter is invalid
return "";
} else {
if (name.equals(fileSchema.findfullName(nameId))) {
// no change happened on current col
return name;
} else {
// find rename operation on current col
// return the name from fileSchema
return fileSchema.findfullName(nameId);
}
}
}
/**
* Collect all type changed cols to build a colPosition -> (newColType, oldColType) map.
* only collect top level col changed. eg: a is a nest field(record(b int, d long), now a.b is changed from int to long,
* only a will be collected, a.b will excluded.
*
* @param schema a type changed internalSchema
* @param oldSchema an old internalSchema.
* @return a map.
*/
public static Map<Integer, Pair<Type, Type>> collectTypeChangedCols(InternalSchema schema, InternalSchema oldSchema) {
Set<Integer> ids = schema.getAllIds();
Set<Integer> otherIds = oldSchema.getAllIds();
Map<Integer, Pair<Type, Type>> result = new HashMap<>();
ids.stream().filter(f -> otherIds.contains(f)).forEach(f -> {
if (!schema.findType(f).equals(oldSchema.findType(f))) {
String[] fieldNameParts = schema.findfullName(f).split("\\.");
String[] otherFieldNameParts = oldSchema.findfullName(f).split("\\.");
String parentName = fieldNameParts[0];
String otherParentName = otherFieldNameParts[0];
if (fieldNameParts.length == otherFieldNameParts.length && schema.findIdByName(parentName) == oldSchema.findIdByName(otherParentName)) {
int index = schema.findIdByName(parentName);
int position = schema.getRecord().fields().stream().map(s -> s.fieldId()).collect(Collectors.toList()).indexOf(index);
if (!result.containsKey(position)) {
result.put(position, Pair.of(schema.findType(parentName), oldSchema.findType(otherParentName)));
}
}
}
});
return result;
}
/**
* Search target internalSchema by version number.
*
* @param versionId the internalSchema version to be search.
* @param internalSchemas internalSchemas to be searched.
* @return a internalSchema.
*/
public static InternalSchema searchSchema(long versionId, List<InternalSchema> internalSchemas) {
TreeMap<Long, InternalSchema> treeMap = new TreeMap<>();
internalSchemas.forEach(s -> treeMap.put(s.schemaId(), s));
return searchSchema(versionId, treeMap);
}
/**
* Search target internalSchema by version number.
*
* @param versionId the internalSchema version to be search.
* @param treeMap internalSchemas collections to be searched.
* @return a internalSchema.
*/
public static InternalSchema searchSchema(long versionId, TreeMap<Long, InternalSchema> treeMap) {
if (treeMap.containsKey(versionId)) {
return treeMap.get(versionId);
} else {
SortedMap<Long, InternalSchema> headMap = treeMap.headMap(versionId);
if (!headMap.isEmpty()) {
return headMap.get(headMap.lastKey());
}
}
return InternalSchema.getEmptyInternalSchema();
}
public static String createFullName(String name, Deque<String> fieldNames) {
String result = name;
if (!fieldNames.isEmpty()) {
List<String> parentNames = new ArrayList<>();
fieldNames.descendingIterator().forEachRemaining(parentNames::add);
result = parentNames.stream().collect(Collectors.joining(".")) + "." + result;
}
return result;
}
}

View File

@@ -0,0 +1,305 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.utils;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import org.apache.hudi.internal.schema.action.TableChanges;
import org.apache.hudi.internal.schema.action.TableChangesHelper;
import java.util.ArrayList;
import java.util.List;
/**
* Helper methods for schema Change.
*/
public class SchemaChangeUtils {
private SchemaChangeUtils() {
}
/**
* Whether to allow the column type to be updated.
* now only support:
* int => long/float/double/string
* long => float/double/string
* float => double/String
* double => String/Decimal
* Decimal => Decimal/String
* String => date/decimal
* date => String
* TODO: support more type update.
*
* @param src origin column type.
* @param dsr new column type.
* @return whether to allow the column type to be updated.
*/
public static boolean isTypeUpdateAllow(Type src, Type dsr) {
if (src.isNestedType() || dsr.isNestedType()) {
throw new IllegalArgumentException("only support update primitive type");
}
if (src.equals(dsr)) {
return true;
}
switch (src.typeId()) {
case INT:
return dsr == Types.LongType.get() || dsr == Types.FloatType.get()
|| dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
case LONG:
return dsr == Types.FloatType.get() || dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
case FLOAT:
return dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
case DOUBLE:
return dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
case DATE:
return dsr == Types.StringType.get();
case DECIMAL:
if (dsr.typeId() == Type.TypeID.DECIMAL) {
Types.DecimalType decimalSrc = (Types.DecimalType)src;
Types.DecimalType decimalDsr = (Types.DecimalType)dsr;
if (decimalDsr.isWiderThan(decimalSrc)) {
return true;
}
} else if (dsr.typeId() == Type.TypeID.STRING) {
return true;
}
break;
case STRING:
return dsr == Types.DateType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
default:
return false;
}
return false;
}
/**
* Apply all the DDL add operations to internalSchema to produce a new internalSchema.
*
* @param internalSchema origin internalSchema.
* @param adds a wrapper class for all the DDL add operations.
* @return a new internalSchema.
*/
public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnAddChange adds) {
Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), adds);
// deal with root level changes
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(),
adds.getParentId2AddCols().get(-1), adds.getPositionChangeMap().get(-1));
return new InternalSchema(newFields);
}
/**
* Apply all the DDL add operations to Type to produce a new internalSchema.
* do not call this method directly. expose this method only for UT.
*
* @param type origin hudi Type.
* @param adds a wrapper class for all the DDL add operations.
* @return a new internalSchema.
*/
public static Type applyTableChange2Type(Type type, TableChanges.ColumnAddChange adds) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Type> newTypes = new ArrayList<>();
for (Types.Field f : record.fields()) {
Type newType = applyTableChange2Type(f.type(), adds);
// try to apply add
newTypes.add(newType.isNestedType() ? adds.applyAdd(f, newType) : newType);
}
List<Types.Field> newFields = new ArrayList<>();
boolean hasChanged = false;
for (int i = 0; i < newTypes.size(); i++) {
Type newType = newTypes.get(i);
Types.Field oldfield = record.fields().get(i);
if (oldfield.type() == newType) {
newFields.add(oldfield);
} else {
hasChanged = true;
newFields.add(Types.Field.get(oldfield.fieldId(), oldfield.isOptional(), oldfield.name(), newType, oldfield.doc()));
}
}
return hasChanged ? Types.RecordType.get(newFields) : record;
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Type newElementType;
Types.Field elementField = array.field(array.elementId());
newElementType = applyTableChange2Type(array.elementType(), adds);
// try to apply add
newElementType = adds.applyAdd(elementField, newElementType);
if (newElementType == array.elementType()) {
return array;
}
return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType);
case MAP:
Types.MapType map = (Types.MapType) type;
Type newValueType;
Types.Field valueField = map.field(map.valueId());
if (adds.getParentId2AddCols().containsKey(map.keyId())) {
throw new IllegalArgumentException("Cannot add fields to map keys: " + map);
}
newValueType = applyTableChange2Type(map.valueType(), adds);
// try to apply add
newValueType = adds.applyAdd(valueField, newValueType);
if (newValueType == map.valueType()) {
return map;
}
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional());
default:
return type;
}
}
/**
* Apply all the DDL delete operations to internalSchema to produce a new internalSchema.
*
* @param internalSchema origin internalSchema.
* @param deletes a wrapper class for all the DDL delete operations.
* @return a new internalSchema.
*/
public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnDeleteChange deletes) {
Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), deletes);
return new InternalSchema(newType.fields());
}
/**
* Apply all the DDL delete operations to Type to produce a new internalSchema.
* do not call this method directly. expose this method only for UT.
*
* @param type origin type.
* @param deletes a wrapper class for all the DDL delete operations.
* @return a new internalSchema.
*/
private static Type applyTableChange2Type(Type type, TableChanges.ColumnDeleteChange deletes) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Types.Field> fields = new ArrayList<>();
for (Types.Field f : record.fields()) {
Type newType = applyTableChange2Type(f.type(), deletes);
// apply delete
newType = deletes.applyDelete(f.fieldId(), newType);
if (newType != null) {
fields.add(Types.Field.get(f.fieldId(), f.isOptional(), f.name(), newType, f.doc()));
}
}
if (fields.isEmpty()) {
throw new UnsupportedOperationException("cannot support delete all columns from Struct");
}
return Types.RecordType.get(fields);
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Type newElementType = applyTableChange2Type(array.elementType(), deletes);
newElementType = deletes.applyDelete(array.elementId(), newElementType);
if (newElementType == null) {
throw new IllegalArgumentException(String.format("cannot delete element from arrayType: %s", array));
}
return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType);
case MAP:
Types.MapType map = (Types.MapType) type;
int keyId = map.fields().get(0).fieldId();
if (deletes.getDeletes().contains(keyId)) {
throw new IllegalArgumentException(String.format("cannot delete key from mapType: %s", map));
}
Type newValueType = applyTableChange2Type(map.valueType(), deletes);
newValueType = deletes.applyDelete(map.valueId(), newValueType);
if (newValueType == null) {
throw new IllegalArgumentException(String.format("cannot delete value from mapType: %s", map));
}
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional());
default:
return type;
}
}
/**
* Apply all the DDL update operations to internalSchema to produce a new internalSchema.
*
* @param internalSchema origin internalSchema.
* @param updates a wrapper class for all the DDL update operations.
* @return a new internalSchema.
*/
public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnUpdateChange updates) {
Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), updates);
// deal with root level changes
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(),
new ArrayList<>(), updates.getPositionChangeMap().get(-1));
return new InternalSchema(newFields);
}
/**
* Apply all the DDL update operations to type to produce a new internalSchema.
* do not call this method directly. expose this method only for UT.
*
* @param type origin internalSchema.
* @param updates a wrapper class for all the DDL update operations.
* @return a new internalSchema.
*/
private static Type applyTableChange2Type(Type type, TableChanges.ColumnUpdateChange updates) {
switch (type.typeId()) {
case RECORD:
Types.RecordType record = (Types.RecordType) type;
List<Type> newTypes = new ArrayList<>();
for (Types.Field f : record.fields()) {
Type newType = applyTableChange2Type(f.type(), updates);
newTypes.add(updates.applyUpdates(f, newType));
}
List<Types.Field> newFields = new ArrayList<>();
for (int i = 0; i < newTypes.size(); i++) {
Type newType = newTypes.get(i);
Types.Field oldField = record.fields().get(i);
Types.Field updateField = updates.getUpdates().get(oldField.fieldId());
if (updateField != null) {
newFields.add(Types.Field.get(oldField.fieldId(), updateField.isOptional(), updateField.name(), newType, updateField.doc()));
} else if (!oldField.type().equals(newType)) {
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
} else {
newFields.add(oldField);
}
}
return Types.RecordType.get(newFields);
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
Type newElementType;
Types.Field elementField = array.fields().get(0);
newElementType = applyTableChange2Type(array.elementType(), updates);
newElementType = updates.applyUpdates(elementField, newElementType);
Types.Field elementUpdate = updates.getUpdates().get(elementField.fieldId());
boolean optional = elementUpdate == null ? array.isElementOptional() : elementUpdate.isOptional();
if (optional == elementField.isOptional() && array.elementType() == newElementType) {
return array;
}
return Types.ArrayType.get(array.elementId(), optional, newElementType);
case MAP:
Types.MapType map = (Types.MapType) type;
Types.Field valueFiled = map.fields().get(1);
Type newValueType;
newValueType = applyTableChange2Type(map.valueType(), updates);
newValueType = updates.applyUpdates(valueFiled, newValueType);
Types.Field valueUpdate = updates.getUpdates().get(valueFiled.fieldId());
boolean valueOptional = valueUpdate == null ? map.isValueOptional() : valueUpdate.isOptional();
if (valueOptional == map.isValueOptional() && map.valueType() == newValueType) {
return map;
}
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, valueOptional);
default:
return type;
}
}
}

View File

@@ -0,0 +1,351 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.utils;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
public class SerDeHelper {
private SerDeHelper() {
}
public static final String LATEST_SCHEMA = "latest_schema";
public static final String SCHEMAS = "schemas";
private static final String MAX_COLUMN_ID = "max_column_id";
private static final String VERSION_ID = "version_id";
private static final String TYPE = "type";
private static final String RECORD = "record";
private static final String ARRAY = "array";
private static final String MAP = "map";
private static final String FIELDS = "fields";
private static final String ELEMENT = "element";
private static final String KEY = "key";
private static final String VALUE = "value";
private static final String DOC = "doc";
private static final String NAME = "name";
private static final String ID = "id";
private static final String ELEMENT_ID = "element_id";
private static final String KEY_ID = "key_id";
private static final String VALUE_ID = "value_id";
private static final String OPTIONAL = "optional";
private static final String ELEMENT_OPTIONAL = "element_optional";
private static final String VALUE_OPTIONAL = "value_optional";
private static final Pattern FIXED = Pattern.compile("fixed\\[(\\d+)\\]");
private static final Pattern DECIMAL = Pattern.compile("decimal\\((\\d+),\\s+(\\d+)\\)");
/**
* Convert history internalSchemas to json.
* this is used when save history schemas into hudi.
*
* @param internalSchemas history internal schemas
* @return a string
*/
public static String toJson(List<InternalSchema> internalSchemas) {
try {
StringWriter writer = new StringWriter();
JsonGenerator generator = (new JsonFactory()).createGenerator(writer);
generator.writeStartObject();
generator.writeArrayFieldStart(SCHEMAS);
for (InternalSchema schema : internalSchemas) {
toJson(schema, generator);
}
generator.writeEndArray();
generator.writeEndObject();
generator.flush();
return writer.toString();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
/**
* Convert internalSchemas to json.
*
* @param internalSchema a internal schema
* @return a string
*/
public static String toJson(InternalSchema internalSchema) {
if (internalSchema == null || internalSchema.isEmptySchema()) {
return "";
}
try {
StringWriter writer = new StringWriter();
JsonGenerator generator = (new JsonFactory()).createGenerator(writer);
toJson(internalSchema, generator);
generator.flush();
return writer.toString();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
private static void toJson(InternalSchema internalSchema, JsonGenerator generator) throws IOException {
toJson(internalSchema.getRecord(), internalSchema.getMaxColumnId(), internalSchema.schemaId(), generator);
}
private static void toJson(Types.RecordType record, Integer maxColumnId, Long versionId, JsonGenerator generator) throws IOException {
generator.writeStartObject();
if (maxColumnId != null) {
generator.writeNumberField(MAX_COLUMN_ID, maxColumnId);
}
if (versionId != null) {
generator.writeNumberField(VERSION_ID, versionId);
}
generator.writeStringField(TYPE, RECORD);
generator.writeArrayFieldStart(FIELDS);
for (Types.Field field : record.fields()) {
generator.writeStartObject();
generator.writeNumberField(ID, field.fieldId());
generator.writeStringField(NAME, field.name());
generator.writeBooleanField(OPTIONAL, field.isOptional());
generator.writeFieldName(TYPE);
toJson(field.type(), generator);
if (field.doc() != null) {
generator.writeStringField(DOC, field.doc());
}
generator.writeEndObject();
}
generator.writeEndArray();
generator.writeEndObject();
}
private static void toJson(Type type, JsonGenerator generator) throws IOException {
switch (type.typeId()) {
case RECORD:
toJson((Types.RecordType) type, null, null, generator);
break;
case ARRAY:
Types.ArrayType array = (Types.ArrayType) type;
generator.writeStartObject();
generator.writeStringField(TYPE, ARRAY);
generator.writeNumberField(ELEMENT_ID, array.elementId());
generator.writeFieldName(ELEMENT);
toJson(array.elementType(), generator);
generator.writeBooleanField(ELEMENT_OPTIONAL, array.isElementOptional());
generator.writeEndObject();
break;
case MAP:
Types.MapType map = (Types.MapType) type;
generator.writeStartObject();
generator.writeStringField(TYPE, MAP);
generator.writeNumberField(KEY_ID, map.keyId());
generator.writeFieldName(KEY);
toJson(map.keyType(), generator);
generator.writeNumberField(VALUE_ID, map.valueId());
generator.writeFieldName(VALUE);
toJson(map.valueType(), generator);
generator.writeBooleanField(VALUE_OPTIONAL, map.isValueOptional());
generator.writeEndObject();
break;
default:
if (!type.isNestedType()) {
generator.writeString(type.toString());
} else {
throw new IllegalArgumentIOException(String.format("cannot write unknown types: %s", type));
}
}
}
private static Type parserTypeFromJson(JsonNode jsonNode) {
if (jsonNode.isTextual()) {
String type = jsonNode.asText().toLowerCase(Locale.ROOT);
// deal with fixed and decimal
Matcher fixed = FIXED.matcher(type);
if (fixed.matches()) {
return Types.FixedType.getFixed(Integer.parseInt(fixed.group(1)));
}
Matcher decimal = DECIMAL.matcher(type);
if (decimal.matches()) {
return Types.DecimalType.get(
Integer.parseInt(decimal.group(1)),
Integer.parseInt(decimal.group(2)));
}
// deal with other type
switch (Type.fromValue(type)) {
case BOOLEAN:
return Types.BooleanType.get();
case INT:
return Types.IntType.get();
case LONG:
return Types.LongType.get();
case FLOAT:
return Types.FloatType.get();
case DOUBLE:
return Types.DoubleType.get();
case DATE:
return Types.DateType.get();
case TIME:
return Types.TimeType.get();
case TIMESTAMP:
return Types.TimestampType.get();
case STRING:
return Types.StringType.get();
case UUID:
return Types.UUIDType.get();
case BINARY:
return Types.BinaryType.get();
default:
throw new IllegalArgumentException("cannot parser types from jsonNode");
}
} else if (jsonNode.isObject()) {
String typeStr = jsonNode.get(TYPE).asText();
if (RECORD.equals(typeStr)) {
JsonNode fieldNodes = jsonNode.get(FIELDS);
Iterator<JsonNode> iter = fieldNodes.elements();
List<Types.Field> fields = new ArrayList<>();
while (iter.hasNext()) {
JsonNode field = iter.next();
// extract
int id = field.get(ID).asInt();
String name = field.get(NAME).asText();
Type type = parserTypeFromJson(field.get(TYPE));
String doc = field.has(DOC) ? field.get(DOC).asText() : null;
boolean optional = field.get(OPTIONAL).asBoolean();
// build fields
fields.add(Types.Field.get(id, optional, name, type, doc));
}
return Types.RecordType.get(fields);
} else if (ARRAY.equals(typeStr)) {
int elementId = jsonNode.get(ELEMENT_ID).asInt();
Type elementType = parserTypeFromJson(jsonNode.get(ELEMENT));
boolean optional = jsonNode.get(ELEMENT_OPTIONAL).asBoolean();
return Types.ArrayType.get(elementId, optional, elementType);
} else if (MAP.equals(typeStr)) {
int keyId = jsonNode.get(KEY_ID).asInt();
Type keyType = parserTypeFromJson(jsonNode.get(KEY));
int valueId = jsonNode.get(VALUE_ID).asInt();
Type valueType = parserTypeFromJson(jsonNode.get(VALUE));
boolean optional = jsonNode.get(VALUE_OPTIONAL).asBoolean();
return Types.MapType.get(keyId, valueId, keyType, valueType, optional);
}
}
throw new IllegalArgumentException(String.format("cannot parse type from jsonNode: %s", jsonNode));
}
/**
* Convert jsonNode to internalSchema.
*
* @param jsonNode a jsonNode.
* @return a internalSchema.
*/
public static InternalSchema fromJson(JsonNode jsonNode) {
Integer maxColumnId = !jsonNode.has(MAX_COLUMN_ID) ? null : jsonNode.get(MAX_COLUMN_ID).asInt();
Long versionId = !jsonNode.has(VERSION_ID) ? null : jsonNode.get(VERSION_ID).asLong();
Types.RecordType type = (Types.RecordType)parserTypeFromJson(jsonNode);
if (versionId == null) {
return new InternalSchema(type.fields());
} else {
if (maxColumnId != null) {
return new InternalSchema(versionId, maxColumnId, type.fields());
} else {
return new InternalSchema(versionId, type.fields());
}
}
}
/**
* Convert string to internalSchema.
*
* @param json a json string.
* @return a internalSchema.
*/
public static Option<InternalSchema> fromJson(String json) {
if (json == null || json.isEmpty()) {
return Option.empty();
}
try {
return Option.of(fromJson((new ObjectMapper(new JsonFactory())).readValue(json, JsonNode.class)));
} catch (IOException e) {
throw new RuntimeException(e);
}
}
/**
* Convert json string to history internalSchemas.
* TreeMap is used to hold history internalSchemas.
*
* @param json a json string
* @return a TreeMap
*/
public static TreeMap<Long, InternalSchema> parseSchemas(String json) {
TreeMap<Long, InternalSchema> result = new TreeMap<>();
try {
JsonNode jsonNode = (new ObjectMapper(new JsonFactory())).readValue(json, JsonNode.class);
if (!jsonNode.has(SCHEMAS)) {
throw new IllegalArgumentException(String.format("cannot parser schemas from current json string, missing key name: %s", SCHEMAS));
}
JsonNode schemas = jsonNode.get(SCHEMAS);
Iterator<JsonNode> iter = schemas.elements();
while (iter.hasNext()) {
JsonNode schema = iter.next();
InternalSchema current = fromJson(schema);
result.put(current.schemaId(), current);
}
} catch (IOException e) {
throw new HoodieException(e);
}
return result;
}
/**
* Add the new schema to the historical schemas.
* use string operations to reduce overhead.
*
* @param newSchema a new internalSchema
* @param oldSchemas historical schemas string.
* @return a string.
*/
public static String inheritSchemas(InternalSchema newSchema, String oldSchemas) {
if (newSchema == null) {
return "";
}
if (oldSchemas == null || oldSchemas.isEmpty()) {
return toJson(Arrays.asList(newSchema));
}
String checkedString = "{\"schemas\":[";
if (!oldSchemas.startsWith("{\"schemas\":")) {
return "";
}
String oldSchemasSuffix = oldSchemas.substring(checkedString.length());
return checkedString + toJson(newSchema) + "," + oldSchemasSuffix;
}
}

View File

@@ -0,0 +1,86 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.visitor;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import java.util.List;
/**
* Base class of schema visitor.
*/
public abstract class InternalSchemaVisitor<T> {
public void beforeField(Types.Field field) {
}
public void afterField(Types.Field field) {
}
public void beforeArrayElement(Types.Field elementField) {
beforeField(elementField);
}
public void afterArrayElement(Types.Field elementField) {
afterField(elementField);
}
public void beforeMapKey(Types.Field keyField) {
beforeField(keyField);
}
public void afterMapKey(Types.Field keyField) {
afterField(keyField);
}
public void beforeMapValue(Types.Field valueField) {
beforeField(valueField);
}
public void afterMapValue(Types.Field valueField) {
afterField(valueField);
}
public T schema(InternalSchema schema, T recordResult) {
return null;
}
public T record(Types.RecordType record, List<T> fieldResults) {
return null;
}
public T field(Types.Field field, T fieldResult) {
return null;
}
public T array(Types.ArrayType array, T elementResult) {
return null;
}
public T map(Types.MapType map, T keyResult, T valueResult) {
return null;
}
public T primitive(Type.PrimitiveType primitive) {
return null;
}
}

View File

@@ -0,0 +1,113 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal.schema.visitor;
import static org.apache.hudi.internal.schema.utils.InternalSchemaUtils.createFullName;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
import java.util.Deque;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
/**
* Schema visitor to produce name -> id map for internalSchema.
*/
public class NameToIDVisitor extends InternalSchemaVisitor<Map<String, Integer>> {
private final Deque fieldNames = new LinkedList<>();
private final Map<String, Integer> nameToId = new HashMap<>();
@Override
public void beforeField(Types.Field field) {
fieldNames.push(field.name());
}
@Override
public void afterField(Types.Field field) {
fieldNames.pop();
}
@Override
public void beforeArrayElement(Types.Field elementField) {
fieldNames.push(elementField.name());
}
@Override
public void afterArrayElement(Types.Field elementField) {
fieldNames.pop();
}
@Override
public void beforeMapKey(Types.Field keyField) {
fieldNames.push(keyField.name());
}
@Override
public void afterMapKey(Types.Field keyField) {
fieldNames.pop();
}
@Override
public void beforeMapValue(Types.Field valueField) {
fieldNames.push(valueField.name());
}
@Override
public void afterMapValue(Types.Field valueField) {
fieldNames.pop();
}
@Override
public Map<String, Integer> schema(InternalSchema schema, Map<String, Integer> recordResult) {
return nameToId;
}
@Override
public Map<String, Integer> record(Types.RecordType record, List<Map<String, Integer>> fieldResults) {
return nameToId;
}
@Override
public Map<String, Integer> field(Types.Field field, Map<String, Integer> fieldResult) {
nameToId.put(createFullName(field.name(), fieldNames), field.fieldId());
return nameToId;
}
@Override
public Map<String, Integer> array(Types.ArrayType array, Map<String, Integer> elementResult) {
nameToId.put(createFullName("element", fieldNames), array.elementId());
return nameToId;
}
@Override
public Map<String, Integer> map(Types.MapType map, Map<String, Integer> keyResult, Map<String, Integer> valueResult) {
nameToId.put(createFullName("key", fieldNames), map.keyId());
nameToId.put(createFullName("value", fieldNames), map.valueId());
return nameToId;
}
@Override
public Map<String, Integer> primitive(Type.PrimitiveType primitive) {
return nameToId;
}
}

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
@@ -65,7 +66,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
Option<InstantRange> instantRange, boolean enableFullScan) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize,
spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false,
enableFullScan, Option.of(partitionName));
enableFullScan, Option.of(partitionName), InternalSchema.getEmptyInternalSchema());
this.mergeKeyFilter = mergeKeyFilter;
if (enableFullScan) {
performScan();