getIncompatibilities() {
return mIncompatibilities;
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public int hashCode() {
final int prime = 31;
@@ -657,7 +674,9 @@ public int hashCode() {
return result;
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public boolean equals(Object obj) {
if (this == obj) {
@@ -680,7 +699,9 @@ public boolean equals(Object obj) {
return mCompatibilityType == other.mCompatibilityType;
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public String toString() {
return String.format("SchemaCompatibilityResult{compatibility:%s, incompatibilities:%s}", mCompatibilityType,
@@ -737,8 +758,8 @@ public Schema getWriterFragment() {
* Returns a human-readable message with more details about what failed. Syntax
* depends on the SchemaIncompatibilityType.
*
- * @see #getType()
* @return a String with details about the incompatibility.
+ * @see #getType()
*/
public String getMessage() {
return mMessage;
@@ -768,7 +789,9 @@ public String getLocation() {
return s.toString();
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public int hashCode() {
final int prime = 31;
@@ -781,7 +804,9 @@ public int hashCode() {
return result;
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public boolean equals(Object obj) {
if (this == obj) {
@@ -825,7 +850,9 @@ public boolean equals(Object obj) {
}
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public String toString() {
return String.format("Incompatibility{type:%s, location:%s, message:%s, reader:%s, writer:%s}", mType,
@@ -837,21 +864,29 @@ public String toString() {
/**
* Provides information about the compatibility of a single reader and writer
* schema pair.
- *
+ *
* Note: This class represents a one-way relationship from the reader to the
* writer schema.
*/
public static final class SchemaPairCompatibility {
- /** The details of this result. */
+ /**
+ * The details of this result.
+ */
private final SchemaCompatibilityResult mResult;
- /** Validated reader schema. */
+ /**
+ * Validated reader schema.
+ */
private final Schema mReader;
- /** Validated writer schema. */
+ /**
+ * Validated writer schema.
+ */
private final Schema mWriter;
- /** Human readable description of this result. */
+ /**
+ * Human readable description of this result.
+ */
private final String mDescription;
/**
@@ -915,14 +950,18 @@ public String getDescription() {
return mDescription;
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public String toString() {
return String.format("SchemaPairCompatibility{result:%s, readerSchema:%s, writerSchema:%s, description:%s}",
mResult, mReader, mWriter, mDescription);
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public boolean equals(Object other) {
if ((other instanceof SchemaPairCompatibility)) {
@@ -934,14 +973,18 @@ public boolean equals(Object other) {
}
}
- /** {@inheritDoc} */
+ /**
+ * {@inheritDoc}
+ */
@Override
public int hashCode() {
- return Arrays.hashCode(new Object[] { mResult, mReader, mWriter, mDescription });
+ return Arrays.hashCode(new Object[] {mResult, mReader, mWriter, mDescription});
}
}
- /** Borrowed from Guava's Objects.equal(a, b) */
+ /**
+ * Borrowed from Guava's Objects.equal(a, b)
+ */
private static boolean objectsEqual(Object obj1, Object obj2) {
return Objects.equals(obj1, obj2);
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
index 24adb1d161ec8..fcfc8a4f0b9fb 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
@@ -55,8 +55,11 @@ public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema, bo
}
/**
- * Establishes whether {@code prevSchema} is compatible w/ {@code newSchema}, as
- * defined by Avro's {@link AvroSchemaCompatibility}
+ * Establishes whether {@code newSchema} is compatible w/ {@code prevSchema}, as
+ * defined by Avro's {@link AvroSchemaCompatibility}.
+ * From avro's compatability standpoint, prevSchema is writer schema and new schema is reader schema.
+ * {@code newSchema} is considered compatible to {@code prevSchema}, iff data written using {@code prevSchema}
+ * could be read by {@code newSchema}
*
* @param prevSchema previous instance of the schema
* @param newSchema new instance of the schema
@@ -116,9 +119,33 @@ public static String getAvroRecordQualifiedName(String tableName) {
return "hoodie." + sanitizedTableName + "." + sanitizedTableName + "_record";
}
+ /**
+ * Validate whether the {@code targetSchema} is a valid evolution of {@code sourceSchema}.
+ * Basically {@link #isCompatibleProjectionOf(Schema, Schema)} but type promotion in the
+ * opposite direction
+ */
+ public static boolean isValidEvolutionOf(Schema sourceSchema, Schema targetSchema) {
+ return (sourceSchema.getType() == Schema.Type.NULL) || isProjectionOfInternal(sourceSchema, targetSchema,
+ AvroSchemaUtils::isAtomicSchemasCompatibleEvolution);
+ }
+
+ /**
+ * Establishes whether {@code newReaderSchema} is compatible w/ {@code prevWriterSchema}, as
+ * defined by Avro's {@link AvroSchemaCompatibility}.
+ * {@code newReaderSchema} is considered compatible to {@code prevWriterSchema}, iff data written using {@code prevWriterSchema}
+ * could be read by {@code newReaderSchema}
+ * @param newReaderSchema new reader schema instance.
+ * @param prevWriterSchema prev writer schema instance.
+ * @return true if its compatible. else false.
+ */
+ private static boolean isAtomicSchemasCompatibleEvolution(Schema newReaderSchema, Schema prevWriterSchema) {
+ // NOTE: Checking for compatibility of atomic types, we should ignore their
+ // corresponding fully-qualified names (as irrelevant)
+ return isSchemaCompatible(prevWriterSchema, newReaderSchema, false, true);
+ }
+
/**
* Validate whether the {@code targetSchema} is a "compatible" projection of {@code sourceSchema}.
- *
* Only difference of this method from {@link #isStrictProjectionOf(Schema, Schema)} is
* the fact that it allows some legitimate type promotions (like {@code int -> long},
* {@code decimal(3, 2) -> decimal(5, 2)}, etc) that allows projection to have a "wider"
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
index 1d8808a64e55b..54c37b333e742 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
@@ -96,6 +96,8 @@
import java.util.regex.Pattern;
import java.util.stream.Collectors;
+import static org.apache.avro.Schema.Type.ARRAY;
+import static org.apache.avro.Schema.Type.MAP;
import static org.apache.avro.Schema.Type.UNION;
import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
import static org.apache.hudi.avro.AvroSchemaUtils.isNullable;
@@ -227,6 +229,10 @@ public static GenericRecord jsonBytesToAvro(byte[] bytes, Schema schema) throws
return reader.read(null, jsonDecoder);
}
+ public static boolean isTypeNumeric(Schema.Type type) {
+ return type == Schema.Type.INT || type == Schema.Type.LONG || type == Schema.Type.FLOAT || type == Schema.Type.DOUBLE;
+ }
+
public static boolean isMetadataField(String fieldName) {
return HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(fieldName);
}
@@ -400,15 +406,15 @@ public static GenericRecord stitchRecords(GenericRecord left, GenericRecord righ
/**
* Given an Avro record with a given schema, rewrites it into the new schema while setting fields only from the new
* schema.
- *
+ *
* NOTE: This method is rewriting every record's field that is record itself recursively. It's
- * caller's responsibility to make sure that no unnecessary re-writing occurs (by preemptively
- * checking whether the record does require re-writing to adhere to the new schema)
- *
+ * caller's responsibility to make sure that no unnecessary re-writing occurs (by preemptively
+ * checking whether the record does require re-writing to adhere to the new schema)
+ *
* NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields)
- * to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the
- * new schema and the default/existing values are carried over.
- *
+ * to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the
+ * new schema and the default/existing values are carried over.
+ *
* This particular method does the following:
*
* Create a new empty GenericRecord with the new schema.
@@ -416,7 +422,7 @@ public static GenericRecord stitchRecords(GenericRecord left, GenericRecord righ
* fields of this transformed schema
* For SpecificRecord, hoodie_metadata_fields have a special treatment (see below)
*
- *
+ *
* For SpecificRecord we ignore Hudi Metadata fields, because for code generated
* avro classes (HoodieMetadataRecord), the avro record is a SpecificBaseRecord type instead of a GenericRecord.
* SpecificBaseRecord throws null pointer exception for record.get(name) if name is not present in the schema of the
@@ -771,7 +777,7 @@ public static String sanitizeName(String name) {
* Sanitizes Name according to Avro rule for names.
* Removes characters other than the ones mentioned in https://avro.apache.org/docs/current/spec.html#names .
*
- * @param name input name
+ * @param name input name
* @param invalidCharMask replacement for invalid characters.
* @return sanitized name
*/
@@ -832,13 +838,13 @@ public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord,
* 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
+ * @param oldRecord oldRecord to be rewritten
+ * @param newSchema newSchema used to rewrite oldRecord
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
* @return newRecord for new Schema
*/
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema, Map renameCols) {
- Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema, renameCols, new LinkedList<>(),false);
+ Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema, renameCols, new LinkedList<>(), false);
return (GenericData.Record) newRecord;
}
@@ -854,11 +860,11 @@ public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord,
* 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 oldRecord oldRecord to be rewritten
* @param oldAvroSchema old avro schema.
- * @param newSchema newSchema used to rewrite oldRecord
- * @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
- * @param fieldNames track the full name of visited field when we travel new schema.
+ * @param newSchema newSchema used to rewrite oldRecord
+ * @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
+ * @param fieldNames track the full name of visited field when we travel new schema.
* @return newRecord for new Schema
*/
@@ -921,7 +927,7 @@ private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, Schem
case ARRAY:
ValidationUtils.checkArgument(oldRecord instanceof Collection, "cannot rewrite record with different type");
Collection array = (Collection) oldRecord;
- List newArray = new ArrayList();
+ List newArray = new ArrayList(array.size());
fieldNames.push("element");
for (Object element : array) {
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType(), renameCols, fieldNames, validate));
@@ -931,7 +937,7 @@ private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, Schem
case MAP:
ValidationUtils.checkArgument(oldRecord instanceof Map, "cannot rewrite record with different type");
Map map = (Map) oldRecord;
- Map newMap = new HashMap<>();
+ Map newMap = new HashMap<>(map.size(), 1);
fieldNames.push("value");
for (Map.Entry entry : map.entrySet()) {
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType(), renameCols, fieldNames, validate));
@@ -1017,7 +1023,7 @@ private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Sche
break;
case FLOAT:
if ((oldSchema.getType() == Schema.Type.INT)
- || (oldSchema.getType() == Schema.Type.LONG)) {
+ || (oldSchema.getType() == Schema.Type.LONG)) {
return oldSchema.getType() == Schema.Type.INT ? ((Integer) oldValue).floatValue() : ((Long) oldValue).floatValue();
}
break;
@@ -1033,7 +1039,7 @@ private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Sche
break;
case BYTES:
if (oldSchema.getType() == Schema.Type.STRING) {
- return getUTF8Bytes(oldValue.toString());
+ return ByteBuffer.wrap(getUTF8Bytes(oldValue.toString()));
}
break;
case STRING:
@@ -1041,15 +1047,15 @@ private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Sche
return String.valueOf(oldValue);
}
if (oldSchema.getType() == Schema.Type.BYTES) {
- return String.valueOf(((byte[]) oldValue));
+ return String.valueOf(((ByteBuffer) 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) {
+ || 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) {
@@ -1081,9 +1087,72 @@ private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Sche
throw new AvroRuntimeException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema));
}
+ /**
+ * Avro does not support type promotion from numbers to string. This function returns true if
+ * it will be necessary to rewrite the record to support this promotion.
+ * NOTE: this does not determine whether the writerSchema and readerSchema are compatible.
+ * It is just trying to find if the reader expects a number to be promoted to string, as quick as possible.
+ */
+ public static boolean recordNeedsRewriteForExtendedAvroTypePromotion(Schema writerSchema, Schema readerSchema) {
+ if (writerSchema.equals(readerSchema)) {
+ return false;
+ }
+ switch (readerSchema.getType()) {
+ case RECORD:
+ Map writerFields = new HashMap<>();
+ for (Schema.Field field : writerSchema.getFields()) {
+ writerFields.put(field.name(), field);
+ }
+ for (Schema.Field field : readerSchema.getFields()) {
+ if (writerFields.containsKey(field.name())) {
+ if (recordNeedsRewriteForExtendedAvroTypePromotion(writerFields.get(field.name()).schema(), field.schema())) {
+ return true;
+ }
+ }
+ }
+ return false;
+ case ARRAY:
+ if (writerSchema.getType().equals(ARRAY)) {
+ return recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema.getElementType(), readerSchema.getElementType());
+ }
+ return false;
+ case MAP:
+ if (writerSchema.getType().equals(MAP)) {
+ return recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema.getValueType(), readerSchema.getValueType());
+ }
+ return false;
+ case UNION:
+ return recordNeedsRewriteForExtendedAvroTypePromotion(getActualSchemaFromUnion(writerSchema, null), getActualSchemaFromUnion(readerSchema, null));
+ case ENUM:
+ case STRING:
+ case BYTES:
+ return needsRewriteToString(writerSchema);
+ default:
+ return false;
+ }
+ }
+
+ /**
+ * Helper for recordNeedsRewriteForExtendedAvroSchemaEvolution. Returns true if schema type is
+ * int, long, float, double, or bytes because avro doesn't support evolution from those types to
+ * string so some intervention is needed
+ */
+ private static boolean needsRewriteToString(Schema schema) {
+ switch (schema.getType()) {
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ case BYTES:
+ return true;
+ default:
+ return false;
+ }
+ }
+
/**
* convert days to Date
- *
+ *
* NOTE: This method could only be used in tests
*
* @VisibleForTesting
@@ -1097,7 +1166,7 @@ public static java.sql.Date toJavaDate(int days) {
/**
* convert Date to days
- *
+ *
* NOTE: This method could only be used in tests
*
* @VisibleForTesting
@@ -1111,17 +1180,19 @@ public static int fromJavaDate(Date date) {
private static Schema getActualSchemaFromUnion(Schema schema, Object data) {
Schema actualSchema;
- if (!schema.getType().equals(UNION)) {
+ if (schema.getType() != UNION) {
return schema;
}
if (schema.getTypes().size() == 2
- && schema.getTypes().get(0).getType() == Schema.Type.NULL) {
+ && 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) {
+ && 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 if (data == null) {
+ return schema;
} else {
// deal complex union. this should not happen in hoodie,
// since flink/spark do not write this type.
@@ -1158,7 +1229,7 @@ public static HoodieRecord createHoodieRecordFromAvro(
* Given avro records, rewrites them with new schema.
*
* @param oldRecords oldRecords to be rewritten
- * @param newSchema newSchema used to rewrite oldRecord
+ * @param newSchema newSchema used to rewrite oldRecord
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
* @return a iterator of rewritten GenericRecords
*/
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java
index 4743764b00682..ec96faf8d7e0e 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java
@@ -58,10 +58,12 @@ public class HoodieCommonConfig extends HoodieConfig {
.markAdvanced()
.withDocumentation("The query instant for time travel. Without specified this option, we query the latest snapshot.");
+ @Deprecated
public static final ConfigProperty RECONCILE_SCHEMA = ConfigProperty
.key("hoodie.datasource.write.reconcile.schema")
.defaultValue(false)
.markAdvanced()
+ .deprecatedAfter("0.14.1")
.withDocumentation("This config controls how writer's schema will be selected based on the incoming batch's "
+ "schema as well as existing table's one. When schema reconciliation is DISABLED, incoming batch's "
+ "schema will be picked as a writer-schema (therefore updating table's schema). When schema reconciliation "
@@ -79,6 +81,14 @@ public class HoodieCommonConfig extends HoodieConfig {
+ " operation will fail schema compatibility check. Set this option to true will make the newly added "
+ " column nullable to successfully complete the write operation.");
+ public static final ConfigProperty SET_NULL_FOR_MISSING_COLUMNS = ConfigProperty
+ .key("hoodie.write.set.null.for.missing.columns")
+ .defaultValue("false")
+ .markAdvanced()
+ .withDocumentation("When a non-nullable column is missing from incoming batch during a write operation, the write "
+ + " operation will fail schema compatibility check. Set this option to true will make the missing "
+ + " column be filled with null values to successfully complete the write operation.");
+
public static final ConfigProperty SPILLABLE_DISK_MAP_TYPE = ConfigProperty
.key("hoodie.common.spillable.diskmap.type")
.defaultValue(ExternalSpillableMap.DiskMapType.BITCASK)
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java
index e75f3743fce0e..96e00e6b955c9 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java
@@ -139,6 +139,13 @@ public static boolean isDataChange(WriteOperationType operation) {
|| operation == WriteOperationType.BOOTSTRAP;
}
+ public static boolean canUpdateSchema(WriteOperationType operation) {
+ return !(operation == WriteOperationType.CLUSTER
+ || operation == WriteOperationType.COMPACT
+ || operation == WriteOperationType.INDEX
+ || operation == WriteOperationType.LOG_COMPACT);
+ }
+
public static boolean isInsert(WriteOperationType operation) {
return operation == WriteOperationType.INSERT
|| operation == WriteOperationType.INSERT_PREPPED
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java
index e757affe4bd72..9b31a51d92504 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java
@@ -23,6 +23,7 @@
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
@@ -388,7 +389,17 @@ public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws
* @return InternalSchema for this table
*/
public Option getTableInternalSchemaFromCommitMetadata() {
- HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+ HoodieTimeline completedInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+ HoodieTimeline timeline = completedInstants
+ .filter(instant -> { // consider only instants that can update/change schema.
+ try {
+ HoodieCommitMetadata commitMetadata =
+ HoodieCommitMetadata.fromBytes(completedInstants.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
+ return WriteOperationType.canUpdateSchema(commitMetadata.getOperationType());
+ } catch (IOException e) {
+ throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e);
+ }
+ });
return timeline.lastInstant().flatMap(this::getTableInternalSchemaFromCommitMetadata);
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
index 156bebb4495f6..afc03ddee7a25 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
@@ -18,6 +18,7 @@
package org.apache.hudi.common.table.log.block;
+import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.engine.HoodieReaderContext;
import org.apache.hudi.common.fs.SizeAwareDataInputStream;
import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
@@ -61,6 +62,7 @@
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
+import static org.apache.hudi.avro.HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion;
import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
@@ -158,7 +160,7 @@ private static class RecordIterator implements ClosableIterator {
private final SizeAwareDataInputStream dis;
private final GenericDatumReader reader;
private final ThreadLocal decoderCache = new ThreadLocal<>();
-
+ private Option promotedSchema = Option.empty();
private int totalRecords = 0;
private int readRecords = 0;
@@ -173,7 +175,12 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content)
this.totalRecords = this.dis.readInt();
}
- this.reader = new GenericDatumReader<>(writerSchema, readerSchema);
+ if (recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema, readerSchema)) {
+ this.reader = new GenericDatumReader<>(writerSchema, writerSchema);
+ this.promotedSchema = Option.of(readerSchema);
+ } else {
+ this.reader = new GenericDatumReader<>(writerSchema, readerSchema);
+ }
}
public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException {
@@ -206,6 +213,9 @@ public IndexedRecord next() {
IndexedRecord record = this.reader.read(null, decoder);
this.dis.skipBytes(recordLength);
this.readRecords++;
+ if (this.promotedSchema.isPresent()) {
+ return HoodieAvroUtils.rewriteRecordWithNewSchema(record, this.promotedSchema.get());
+ }
return record;
} catch (IOException e) {
throw new HoodieIOException("Unable to convert bytes to record.", e);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
index 2c70239dcdfbe..1f654cf76adb5 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
@@ -20,6 +20,7 @@
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.FileIOUtils;
@@ -359,7 +360,8 @@ public Option> getLastCommitMetadataWi
return Option.fromJavaOptional(
getCommitMetadataStream()
.filter(instantCommitMetadataPair ->
- !StringUtils.isNullOrEmpty(instantCommitMetadataPair.getValue().getMetadata(HoodieCommitMetadata.SCHEMA_KEY)))
+ WriteOperationType.canUpdateSchema(instantCommitMetadataPair.getRight().getOperationType())
+ && !StringUtils.isNullOrEmpty(instantCommitMetadataPair.getValue().getMetadata(HoodieCommitMetadata.SCHEMA_KEY)))
.findFirst()
);
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java
index 7afdf5bf18089..786ac538271a2 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java
@@ -68,6 +68,27 @@ public static Schema convert(InternalSchema internalSchema, String name) {
return buildAvroSchemaFromInternalSchema(internalSchema, name);
}
+ /**
+ * Converting from avro -> internal schema -> avro
+ * causes null to always be first in unions.
+ * if we compare a schema that has not been converted to internal schema
+ * at any stage, the difference in ordering can cause issues. To resolve this,
+ * we order null to be first for any avro schema that enters into hudi.
+ * AvroSchemaUtils.isProjectionOfInternal uses index based comparison for unions.
+ * Spark and flink don't support complex unions so this would not be an issue
+ * but for the metadata table HoodieMetadata.avsc uses a trick where we have a bunch of
+ * different types wrapped in record for col stats.
+ *
+ * @param Schema avro schema.
+ * @return an avro Schema where null is the first.
+ */
+ public static Schema fixNullOrdering(Schema schema) {
+ if (schema.getType() == Schema.Type.NULL) {
+ return schema;
+ }
+ return convert(convert(schema), schema.getFullName());
+ }
+
/**
* Convert RecordType to avro Schema.
*
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java
index 13c1f0e2277ab..2fdd2f4c2db64 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java
@@ -18,10 +18,12 @@
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.action.TableChanges;
+import org.apache.avro.Schema;
+
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@@ -41,18 +43,23 @@ public class AvroSchemaEvolutionUtils {
* 1) incoming data has missing columns that were already defined in the table –> null values will be injected into missing columns
* 2) incoming data contains new columns not defined yet in the table -> columns will be added to the table schema (incoming dataframe?)
* 3) incoming data has missing columns that are already defined in the table and new columns not yet defined in the table ->
- * new columns will be added to the table schema, missing columns will be injected with null values
+ * new columns will be added to the table schema, missing columns will be injected with null values
* 4) support type change
* 5) support nested schema change.
* Notice:
- * the incoming schema should not have delete/rename semantics.
- * for example: incoming schema: int a, int b, int d; oldTableSchema int a, int b, int c, int d
- * we must guarantee the column c is missing semantic, instead of delete semantic.
+ * the incoming schema should not have delete/rename semantics.
+ * for example: incoming schema: int a, int b, int d; oldTableSchema int a, int b, int c, int d
+ * we must guarantee the column c is missing semantic, instead of delete semantic.
+ *
* @param incomingSchema implicitly evolution of avro when hoodie write operation
* @param oldTableSchema old internalSchema
* @return reconcile Schema
*/
public static InternalSchema reconcileSchema(Schema incomingSchema, InternalSchema oldTableSchema) {
+ /* If incoming schema is null, we fall back on table schema. */
+ if (incomingSchema.getType() == Schema.Type.NULL) {
+ return oldTableSchema;
+ }
InternalSchema inComingInternalSchema = convert(incomingSchema);
// check column add/missing
List colNamesFromIncoming = inComingInternalSchema.getAllColsFullName();
@@ -73,7 +80,7 @@ public static InternalSchema reconcileSchema(Schema incomingSchema, InternalSche
// 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 finalAddAction = new TreeMap<>();
- for (int i = 0; i < diffFromEvolutionColumns.size(); i++) {
+ for (int i = 0; i < diffFromEvolutionColumns.size(); i++) {
String name = diffFromEvolutionColumns.get(i);
int splitPoint = name.lastIndexOf(".");
String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : "";
@@ -95,7 +102,7 @@ public static InternalSchema reconcileSchema(Schema incomingSchema, InternalSche
colNamesFromIncoming.stream().filter(c ->
c.lastIndexOf(".") == splitPoint
&& c.startsWith(parentName)
- && inComingInternalSchema.findIdByName(c) > inComingInternalSchema.findIdByName(name)
+ && inComingInternalSchema.findIdByName(c) > inComingInternalSchema.findIdByName(name)
&& oldTableSchema.findIdByName(c) > 0).sorted((s1, s2) -> oldTableSchema.findIdByName(s1) - oldTableSchema.findIdByName(s2)).findFirst();
addChange.addColumns(parentName, rawName, inComingInternalSchema.findType(name), null);
inferPosition.map(i -> addChange.addPositionChange(name, i, "before"));
@@ -111,18 +118,29 @@ public static InternalSchema reconcileSchema(Schema incomingSchema, InternalSche
return SchemaChangeUtils.applyTableChanges2Schema(internalSchemaAfterAddColumns, typeChange);
}
+ public static Schema reconcileSchema(Schema incomingSchema, Schema oldTableSchema) {
+ return convert(reconcileSchema(incomingSchema, convert(oldTableSchema)), oldTableSchema.getFullName());
+ }
+
/**
- * Reconciles nullability requirements b/w {@code source} and {@code target} schemas,
+ * Reconciles nullability and datatype requirements b/w {@code source} and {@code target} schemas,
* by adjusting these of the {@code source} schema to be in-line with the ones of the
- * {@code target} one
+ * {@code target} one. Source is considered to be new incoming schema, while target could refer to prev table schema.
+ * For example,
+ * if colA in source is non-nullable, but is nullable in target, output schema will have colA as nullable.
+ * if "hoodie.datasource.write.new.columns.nullable" is set to true and if colB is not present in source, but
+ * is present in target, output schema will have colB as nullable.
+ * if colC has different data type in source schema compared to target schema and if its promotable, (say source is int,
+ * and target is long and since int can be promoted to long), colC will be long data type in output schema.
+ *
*
* @param sourceSchema source schema that needs reconciliation
* @param targetSchema target schema that source schema will be reconciled against
- * @param opts config options
- * @return schema (based off {@code source} one) that has nullability constraints reconciled
+ * @param opts config options
+ * @return schema (based off {@code source} one) that has nullability constraints and datatypes reconciled
*/
- public static Schema reconcileNullability(Schema sourceSchema, Schema targetSchema, Map opts) {
- if (sourceSchema.getFields().isEmpty() || targetSchema.getFields().isEmpty()) {
+ public static Schema reconcileSchemaRequirements(Schema sourceSchema, Schema targetSchema, Map opts) {
+ if (sourceSchema.getType() == Schema.Type.NULL || sourceSchema.getFields().isEmpty() || targetSchema.getFields().isEmpty()) {
return sourceSchema;
}
@@ -131,20 +149,41 @@ public static Schema reconcileNullability(Schema sourceSchema, Schema targetSche
List colNamesSourceSchema = sourceInternalSchema.getAllColsFullName();
List colNamesTargetSchema = targetInternalSchema.getAllColsFullName();
- List candidateUpdateCols = colNamesSourceSchema.stream()
- .filter(f -> (("true".equals(opts.get(MAKE_NEW_COLUMNS_NULLABLE.key())) && !colNamesTargetSchema.contains(f))
- || colNamesTargetSchema.contains(f) && sourceInternalSchema.findField(f).isOptional() != targetInternalSchema.findField(f).isOptional()
- )
- ).collect(Collectors.toList());
+ boolean makeNewColsNullable = "true".equals(opts.get(MAKE_NEW_COLUMNS_NULLABLE.key()));
+
+ List nullableUpdateColsInSource = new ArrayList<>();
+ List typeUpdateColsInSource = new ArrayList<>();
+ colNamesSourceSchema.forEach(field -> {
+ // handle columns that needs to be made nullable
+ if ((makeNewColsNullable && !colNamesTargetSchema.contains(field))
+ || colNamesTargetSchema.contains(field) && sourceInternalSchema.findField(field).isOptional() != targetInternalSchema.findField(field).isOptional()) {
+ nullableUpdateColsInSource.add(field);
+ }
+ // handle columns that needs type to be updated
+ if (colNamesTargetSchema.contains(field) && SchemaChangeUtils.shouldPromoteType(sourceInternalSchema.findType(field), targetInternalSchema.findType(field))) {
+ typeUpdateColsInSource.add(field);
+ }
+ });
- if (candidateUpdateCols.isEmpty()) {
- return sourceSchema;
+ if (nullableUpdateColsInSource.isEmpty() && typeUpdateColsInSource.isEmpty()) {
+ //standardize order of unions
+ return convert(sourceInternalSchema, sourceSchema.getFullName());
}
+ TableChanges.ColumnUpdateChange schemaChange = TableChanges.ColumnUpdateChange.get(sourceInternalSchema);
+
// Reconcile nullability constraints (by executing phony schema change)
- TableChanges.ColumnUpdateChange schemaChange =
- reduce(candidateUpdateCols, TableChanges.ColumnUpdateChange.get(sourceInternalSchema),
+ if (!nullableUpdateColsInSource.isEmpty()) {
+ schemaChange = reduce(nullableUpdateColsInSource, schemaChange,
(change, field) -> change.updateColumnNullability(field, true));
+ }
+
+ // Reconcile type promotions
+ if (!typeUpdateColsInSource.isEmpty()) {
+ schemaChange = reduce(typeUpdateColsInSource, schemaChange,
+ (change, field) -> change.updateColumnType(field, targetInternalSchema.findType(field)));
+ }
+
return convert(SchemaChangeUtils.applyTableChanges2Schema(sourceInternalSchema, schemaChange), sourceSchema.getFullName());
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java
index ff2ca89e98ebc..b2751cc43e87a 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java
@@ -58,6 +58,17 @@ public static boolean isTypeUpdateAllow(Type src, Type dsr) {
if (src.equals(dsr)) {
return true;
}
+ return isTypeUpdateAllowInternal(src, dsr);
+ }
+
+ public static boolean shouldPromoteType(Type src, Type dsr) {
+ if (src.equals(dsr) || src.isNestedType() || dsr.isNestedType()) {
+ return false;
+ }
+ return isTypeUpdateAllowInternal(src, dsr);
+ }
+
+ private static boolean isTypeUpdateAllowInternal(Type src, Type dsr) {
switch (src.typeId()) {
case INT:
return dsr == Types.LongType.get() || dsr == Types.FloatType.get()
@@ -69,6 +80,7 @@ public static boolean isTypeUpdateAllow(Type src, Type dsr) {
case DOUBLE:
return dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
case DATE:
+ case BINARY:
return dsr == Types.StringType.get();
case DECIMAL:
if (dsr.typeId() == Type.TypeID.DECIMAL) {
@@ -85,7 +97,7 @@ public static boolean isTypeUpdateAllow(Type src, Type dsr) {
}
break;
case STRING:
- return dsr == Types.DateType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
+ return dsr == Types.DateType.get() || dsr.typeId() == Type.TypeID.DECIMAL || dsr == Types.BinaryType.get();
default:
return false;
}
diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java
index 6c5fcb7049c38..0be0a5f89c528 100644
--- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java
@@ -19,6 +19,7 @@
package org.apache.hudi.internal.schema.utils;
import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
import org.apache.hudi.internal.schema.Type;
@@ -207,6 +208,20 @@ public void testRefreshNewId() {
Assertions.assertEquals(newRecord, recordWithNewId);
}
+ @Test
+ public void testFixNullOrdering() {
+ Schema schema = SchemaTestUtil.getSchemaFromResource(TestAvroSchemaEvolutionUtils.class, "/nullWrong.avsc");
+ Schema expectedSchema = SchemaTestUtil.getSchemaFromResource(TestAvroSchemaEvolutionUtils.class, "/nullRight.avsc");
+ Assertions.assertEquals(expectedSchema, AvroInternalSchemaConverter.fixNullOrdering(schema));
+ Assertions.assertEquals(expectedSchema, AvroInternalSchemaConverter.fixNullOrdering(expectedSchema));
+ }
+
+ @Test
+ public void testFixNullOrderingSameSchemaCheck() {
+ Schema schema = SchemaTestUtil.getSchemaFromResource(TestAvroSchemaEvolutionUtils.class, "/source_evolved.avsc");
+ Assertions.assertEquals(schema, AvroInternalSchemaConverter.fixNullOrdering(schema));
+ }
+
public enum Enum {
ENUM1, ENUM2
}
diff --git a/hudi-common/src/test/resources/nullRight.avsc b/hudi-common/src/test/resources/nullRight.avsc
new file mode 100644
index 0000000000000..05e7a7c384017
--- /dev/null
+++ b/hudi-common/src/test/resources/nullRight.avsc
@@ -0,0 +1,213 @@
+/*
+ * 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.
+ */
+{
+ "type": "record",
+ "name": "SchemaName",
+ "namespace": "SchemaNS",
+ "fields": [
+ {
+ "name": "key",
+ "type": "string"
+ },
+ {
+ "name": "version",
+ "type": [
+ "null",
+ "string"
+ ],
+ "doc": "versionComment",
+ "default": null
+ },
+ {
+ "name": "data1",
+ "type": {
+ "type": "record",
+ "name": "data1",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "innerKey",
+ "type": "string",
+ "doc": "innerKeyComment"
+ },
+ {
+ "name": "value",
+ "type": [
+ "null",
+ "long"
+ ],
+ "doc": "valueComment",
+ "default": null
+ }
+ ]
+ }
+ },
+ {
+ "name": "data2",
+ "type": [
+ "null",
+ {
+ "type": "record",
+ "name": "data2",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "innerKey",
+ "type": "string",
+ "doc": "innerKeyComment"
+ },
+ {
+ "name": "value",
+ "type": [
+ "null",
+ "long"
+ ],
+ "doc": "valueComment",
+ "default": null
+ }
+ ]
+ }
+ ],
+ "default": null
+ },
+ {
+ "name": "nullableMap",
+ "type": [
+ "null",
+ {
+ "type": "map",
+ "values": [
+ "null",
+ {
+ "type": "record",
+ "name": "nullableMap",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "mapKey",
+ "type": "string",
+ "doc": "mapKeyComment"
+ },
+ {
+ "name": "mapVal",
+ "type": [
+ "null",
+ "int"
+ ],
+ "default": null
+ }
+ ]
+ }
+ ]
+ }
+ ],
+ "default": null
+ },
+ {
+ "name": "map",
+ "type": {
+ "type": "map",
+ "values": [
+ "null",
+ {
+ "type": "record",
+ "name": "map",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "mapKey",
+ "type": "string",
+ "doc": "mapKeyComment"
+ },
+ {
+ "name": "mapVal",
+ "type": [
+ "null",
+ "int"
+ ],
+ "default": null
+ }
+ ]
+ }
+ ]
+ }
+ },
+ {
+ "name": "nullableArray",
+ "type": [
+ "null",
+ {
+ "type": "array",
+ "items": [
+ "null",
+ {
+ "type": "record",
+ "name": "nullableArray",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "arrayKey",
+ "type": "string"
+ },
+ {
+ "name": "arrayVal",
+ "type": [
+ "null",
+ "int"
+ ],
+ "doc": "arrayValComment",
+ "default": null
+ }
+ ]
+ }
+ ]
+ }
+ ],
+ "default": null
+ },
+ {
+ "name": "array",
+ "type": {
+ "type": "array",
+ "items": [
+ "null",
+ {
+ "type": "record",
+ "name": "array",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "arrayKey",
+ "type": "string"
+ },
+ {
+ "name": "arrayVal",
+ "type": [
+ "null",
+ "int"
+ ],
+ "doc": "arrayValComment",
+ "default": null
+ }
+ ]
+ }
+ ]
+ }
+ }
+ ]
+}
\ No newline at end of file
diff --git a/hudi-common/src/test/resources/nullWrong.avsc b/hudi-common/src/test/resources/nullWrong.avsc
new file mode 100644
index 0000000000000..1ef9ee931da4d
--- /dev/null
+++ b/hudi-common/src/test/resources/nullWrong.avsc
@@ -0,0 +1,203 @@
+/*
+ * 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.
+ */
+{
+ "type": "record",
+ "name": "SchemaName",
+ "namespace": "SchemaNS",
+ "fields": [
+ {
+ "name": "key",
+ "type": "string"
+ },
+ {
+ "name": "version",
+ "type": [
+ "string",
+ "null"
+ ],
+ "doc": "versionComment"
+ },
+ {
+ "name": "data1",
+ "type": {
+ "type": "record",
+ "name": "data1",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "innerKey",
+ "type": "string",
+ "doc": "innerKeyComment"
+ },
+ {
+ "name": "value",
+ "type": [
+ "long",
+ "null"
+ ],
+ "doc": "valueComment"
+ }
+ ]
+ }
+ },
+ {
+ "name": "data2",
+ "type": [
+ "null",
+ {
+ "type": "record",
+ "name": "data2",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "innerKey",
+ "type": "string",
+ "doc": "innerKeyComment"
+ },
+ {
+ "name": "value",
+ "type": [
+ "long",
+ "null"
+ ],
+ "doc": "valueComment"
+ }
+ ]
+ }
+ ]
+ },
+ {
+ "name": "nullableMap",
+ "type": [
+ {
+ "type": "map",
+ "values": [
+ {
+ "type": "record",
+ "name": "nullableMap",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "mapKey",
+ "type": "string",
+ "doc": "mapKeyComment"
+ },
+ {
+ "name": "mapVal",
+ "type": [
+ "int",
+ "null"
+ ]
+ }
+ ]
+ },
+ "null"
+ ]
+ },
+ "null"
+ ]
+ },
+ {
+ "name": "map",
+ "type": {
+ "type": "map",
+ "values": [
+ {
+ "type": "record",
+ "name": "map",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "mapKey",
+ "type": "string",
+ "doc": "mapKeyComment"
+ },
+ {
+ "name": "mapVal",
+ "type": [
+ "int",
+ "null"
+ ]
+ }
+ ]
+ },
+ "null"
+ ]
+ }
+ },
+ {
+ "name": "nullableArray",
+ "type": [
+ {
+ "type": "array",
+ "items": [
+ {
+ "type": "record",
+ "name": "nullableArray",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "arrayKey",
+ "type": "string"
+ },
+ {
+ "name": "arrayVal",
+ "type": [
+ "int",
+ "null"
+ ],
+ "doc": "arrayValComment"
+ }
+ ]
+ },
+ "null"
+ ]
+ },
+ "null"
+ ]
+ },
+ {
+ "name": "array",
+ "type": {
+ "type": "array",
+ "items": [
+ {
+ "type": "record",
+ "name": "array",
+ "namespace": "SchemaNS.SchemaName",
+ "fields": [
+ {
+ "name": "arrayKey",
+ "type": "string"
+ },
+ {
+ "name": "arrayVal",
+ "type": [
+ "int",
+ "null"
+ ],
+ "doc": "arrayValComment"
+ }
+ ]
+ },
+ "null"
+ ]
+ }
+ }
+ ]
+}
diff --git a/hudi-common/src/test/resources/source_evolved.avsc b/hudi-common/src/test/resources/source_evolved.avsc
new file mode 100644
index 0000000000000..9571b4886f83e
--- /dev/null
+++ b/hudi-common/src/test/resources/source_evolved.avsc
@@ -0,0 +1,158 @@
+/*
+ * 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.
+ */
+{
+ "type": "record",
+ "name": "triprec",
+ "fields": [
+ {
+ "name": "timestamp",
+ "type": "long"
+ },
+ {
+ "name": "_row_key",
+ "type": "string"
+ },
+ {
+ "name": "partition_path",
+ "type": "string"
+ },
+ {
+ "name": "trip_type",
+ "type": "string"
+ },
+ {
+ "name": "rider",
+ "type": "string"
+ },
+ {
+ "name": "driver",
+ "type": "string"
+ },
+ {
+ "name": "begin_lat",
+ "type": "double"
+ },
+ {
+ "name": "begin_lon",
+ "type": "double"
+ },
+ {
+ "name": "end_lat",
+ "type": "double"
+ },
+ {
+ "name": "end_lon",
+ "type": "double"
+ },
+ {
+ "name": "distance_in_meters",
+ "type": "int"
+ },
+ {
+ "name": "seconds_since_epoch",
+ "type": "long"
+ },
+ {
+ "name": "weight",
+ "type": "float"
+ },
+ {
+ "name": "nation",
+ "type": "bytes"
+ },
+ {
+ "name": "current_date",
+ "type": {
+ "type": "int",
+ "logicalType": "date"
+ }
+ },
+ {
+ "name": "current_ts",
+ "type": "long"
+ },
+ {
+ "name": "height",
+ "type": {
+ "type": "fixed",
+ "name": "fixed",
+ "namespace": "triprec.height",
+ "size": 5,
+ "logicalType": "decimal",
+ "precision": 10,
+ "scale": 6
+ }
+ },
+ {
+ "name": "city_to_state",
+ "type": {
+ "type": "map",
+ "values": "string"
+ }
+ },
+ {
+ "name": "fare",
+ "type": {
+ "type": "record",
+ "name": "fare",
+ "fields": [
+ {
+ "name": "amount",
+ "type": "double"
+ },
+ {
+ "name": "currency",
+ "type": "string"
+ }
+ ]
+ }
+ },
+ {
+ "name": "tip_history",
+ "type": {
+ "type": "array",
+ "items": {
+ "type": "record",
+ "name": "tip_history",
+ "fields": [
+ {
+ "name": "amount",
+ "type": "double"
+ },
+ {
+ "name": "currency",
+ "type": "string"
+ }
+ ]
+ }
+ }
+ },
+ {
+ "name": "_hoodie_is_deleted",
+ "type": "boolean"
+ },
+ {
+ "name": "evoluted_optional_union_field",
+ "type": [
+ "null",
+ "string"
+ ],
+ "default": null
+ }
+ ]
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java
index d8d5dad329d02..d4665282be12f 100644
--- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java
@@ -21,6 +21,7 @@
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.testutils.InProcessTimeGenerator;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
@@ -28,6 +29,7 @@
import org.apache.hudi.utilities.sources.InputBatch;
import org.apache.hudi.utilities.streamer.StreamSync;
+import org.apache.hadoop.conf.Configuration;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -80,8 +82,12 @@ public JavaRDD compact() throws Exception {
public Pair>> fetchSource() throws Exception {
StreamSync service = getDeltaSync();
service.refreshTimeline();
+ HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+ .setConf(new Configuration(service.getFs().getConf()))
+ .setBasePath(service.getCfg().targetBasePath)
+ .build();
String instantTime = InProcessTimeGenerator.createNewInstantTime();
- InputBatch inputBatch = service.readFromSource(instantTime).getLeft();
+ InputBatch inputBatch = service.readFromSource(instantTime, metaClient).getLeft();
return Pair.of(inputBatch.getSchemaProvider(), Pair.of(inputBatch.getCheckpointForNextBatch(), (JavaRDD) inputBatch.getBatch().get()));
}
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
index e7364316205f8..dc54825ac90de 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
@@ -536,8 +536,11 @@ object DataSourceWriteOptions {
.markAdvanced()
.withDocumentation("Sync tool class name used to sync to metastore. Defaults to Hive.")
+ @Deprecated
val RECONCILE_SCHEMA: ConfigProperty[java.lang.Boolean] = HoodieCommonConfig.RECONCILE_SCHEMA
+ val SET_NULL_FOR_MISSING_COLUMNS: ConfigProperty[String] = HoodieCommonConfig.SET_NULL_FOR_MISSING_COLUMNS
+
val MAKE_NEW_COLUMNS_NULLABLE: ConfigProperty[java.lang.Boolean] = HoodieCommonConfig.MAKE_NEW_COLUMNS_NULLABLE
val SPARK_SQL_INSERT_INTO_OPERATION: ConfigProperty[String] = ConfigProperty
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSchemaUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSchemaUtils.scala
new file mode 100644
index 0000000000000..ed073ce4b1747
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSchemaUtils.scala
@@ -0,0 +1,237 @@
+/*
+ * 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
+
+import org.apache.avro.Schema
+import org.apache.hudi.HoodieSparkSqlWriter.{CANONICALIZE_SCHEMA, SQL_MERGE_INTO_WRITES}
+import org.apache.hudi.avro.AvroSchemaUtils.{isCompatibleProjectionOf, isSchemaCompatible, isValidEvolutionOf}
+import org.apache.hudi.avro.HoodieAvroUtils
+import org.apache.hudi.avro.HoodieAvroUtils.removeMetadataFields
+import org.apache.hudi.common.config.HoodieConfig
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.exception.SchemaCompatibilityException
+import org.apache.hudi.internal.schema.InternalSchema
+import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
+import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils
+import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils.reconcileSchemaRequirements
+import org.slf4j.LoggerFactory
+
+import scala.collection.JavaConversions.{asScalaBuffer, mapAsJavaMap}
+
+/**
+ * Util methods for Schema evolution in Hudi
+ */
+object HoodieSchemaUtils {
+ private val log = LoggerFactory.getLogger(getClass)
+
+ /**
+ * get latest internalSchema from table
+ *
+ * @param config instance of {@link HoodieConfig}
+ * @param tableMetaClient instance of HoodieTableMetaClient
+ * @return Option of InternalSchema. Will always be empty if schema on read is disabled
+ */
+ def getLatestTableInternalSchema(config: HoodieConfig,
+ tableMetaClient: HoodieTableMetaClient): Option[InternalSchema] = {
+ if (!config.getBooleanOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED)) {
+ None
+ } else {
+ try {
+ val tableSchemaResolver = new TableSchemaResolver(tableMetaClient)
+ val internalSchemaOpt = tableSchemaResolver.getTableInternalSchemaFromCommitMetadata
+ if (internalSchemaOpt.isPresent) Some(internalSchemaOpt.get()) else None
+ } catch {
+ case _: Exception => None
+ }
+ }
+ }
+
+ /**
+ * Deduces writer's schema based on
+ *
+ * Source's schema
+ * Target table's schema (including Hudi's [[InternalSchema]] representation)
+ *
+ */
+ def deduceWriterSchema(sourceSchema: Schema,
+ latestTableSchemaOpt: Option[Schema],
+ internalSchemaOpt: Option[InternalSchema],
+ opts: Map[String, String]): Schema = {
+ val setNullForMissingColumns = opts.getOrDefault(DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS.key(),
+ DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS.defaultValue).toBoolean
+ val shouldReconcileSchema = opts(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean
+ val shouldValidateSchemasCompatibility = opts.getOrDefault(HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE.key,
+ HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE.defaultValue).toBoolean
+
+ latestTableSchemaOpt match {
+ // In case table schema is empty we're just going to use the source schema as a
+ // writer's schema.
+ case None => AvroInternalSchemaConverter.fixNullOrdering(sourceSchema)
+ // Otherwise, we need to make sure we reconcile incoming and latest table schemas
+ case Some(latestTableSchemaWithMetaFields) =>
+ // NOTE: Meta-fields will be unconditionally injected by Hudi writing handles, for the sake of
+ // deducing proper writer schema we're stripping them to make sure we can perform proper
+ // analysis
+ //add call to fix null ordering to ensure backwards compatibility
+ val latestTableSchema = AvroInternalSchemaConverter.fixNullOrdering(removeMetadataFields(latestTableSchemaWithMetaFields))
+ // Before validating whether schemas are compatible, we need to "canonicalize" source's schema
+ // relative to the table's one, by doing a (minor) reconciliation of the nullability constraints:
+ // for ex, if in incoming schema column A is designated as non-null, but it's designated as nullable
+ // in the table's one we want to proceed aligning nullability constraints w/ the table's schema
+ // Also, we promote types to the latest table schema if possible.
+ val shouldCanonicalizeSchema = opts.getOrDefault(CANONICALIZE_SCHEMA.key,
+ CANONICALIZE_SCHEMA.defaultValue.toString).toBoolean
+ val mergeIntoWrites = opts.getOrDefault(SQL_MERGE_INTO_WRITES.key(),
+ SQL_MERGE_INTO_WRITES.defaultValue.toString).toBoolean
+
+ val canonicalizedSourceSchema = if (shouldCanonicalizeSchema) {
+ canonicalizeSchema(sourceSchema, latestTableSchema, opts)
+ } else {
+ AvroInternalSchemaConverter.fixNullOrdering(sourceSchema)
+ }
+
+ val allowAutoEvolutionColumnDrop = opts.getOrDefault(HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key,
+ HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.defaultValue).toBoolean
+
+ if (shouldReconcileSchema) {
+ internalSchemaOpt match {
+ case Some(internalSchema) =>
+ // Apply schema evolution, by auto-merging write schema and read schema
+ val mergedInternalSchema = AvroSchemaEvolutionUtils.reconcileSchema(canonicalizedSourceSchema, internalSchema)
+ val evolvedSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, latestTableSchema.getFullName)
+ val shouldRemoveMetaDataFromInternalSchema = sourceSchema.getFields().filter(f => f.name().equalsIgnoreCase(HoodieRecord.RECORD_KEY_METADATA_FIELD)).isEmpty
+ if (shouldRemoveMetaDataFromInternalSchema) HoodieAvroUtils.removeMetadataFields(evolvedSchema) else evolvedSchema
+ case None =>
+ // In case schema reconciliation is enabled we will employ (legacy) reconciliation
+ // strategy to produce target writer's schema (see definition below)
+ val (reconciledSchema, isCompatible) =
+ reconcileSchemasLegacy(latestTableSchema, canonicalizedSourceSchema)
+
+ // NOTE: In some cases we need to relax constraint of incoming dataset's schema to be compatible
+ // w/ the table's one and allow schemas to diverge. This is required in cases where
+ // partial updates will be performed (for ex, `MERGE INTO` Spark SQL statement) and as such
+ // only incoming dataset's projection has to match the table's schema, and not the whole one
+ if (!shouldValidateSchemasCompatibility || isCompatible) {
+ reconciledSchema
+ } else {
+ log.error(
+ s"""Failed to reconcile incoming batch schema with the table's one.
+ |Incoming schema ${sourceSchema.toString(true)}
+ |Incoming schema (canonicalized) ${canonicalizedSourceSchema.toString(true)}
+ |Table's schema ${latestTableSchema.toString(true)}
+ |""".stripMargin)
+ throw new SchemaCompatibilityException("Failed to reconcile incoming schema with the table's one")
+ }
+ }
+ } else {
+ // In case reconciliation is disabled, we have to validate that the source's schema
+ // is compatible w/ the table's latest schema, such that we're able to read existing table's
+ // records using [[sourceSchema]].
+ //
+ // NOTE: In some cases we need to relax constraint of incoming dataset's schema to be compatible
+ // w/ the table's one and allow schemas to diverge. This is required in cases where
+ // partial updates will be performed (for ex, `MERGE INTO` Spark SQL statement) and as such
+ // only incoming dataset's projection has to match the table's schema, and not the whole one
+
+ if (mergeIntoWrites) {
+ // if its merge into writes, do not check for projection nor schema compatibility. Writers down the line will
+ // take care of it.
+ canonicalizedSourceSchema
+ } else {
+ if (!shouldValidateSchemasCompatibility) {
+ // if no validation is enabled, check for col drop
+ if (allowAutoEvolutionColumnDrop) {
+ canonicalizedSourceSchema
+ } else {
+ val reconciledSchema = if (setNullForMissingColumns) {
+ AvroSchemaEvolutionUtils.reconcileSchema(canonicalizedSourceSchema, latestTableSchema)
+ } else {
+ canonicalizedSourceSchema
+ }
+ if (isValidEvolutionOf(reconciledSchema, latestTableSchema)) {
+ reconciledSchema
+ } else {
+ log.error(
+ s"""Incoming batch schema is not compatible with the table's one.
+ |Incoming schema ${sourceSchema.toString(true)}
+ |Incoming schema (canonicalized) ${reconciledSchema.toString(true)}
+ |Table's schema ${latestTableSchema.toString(true)}
+ |""".stripMargin)
+ throw new SchemaCompatibilityException("Incoming batch schema is not compatible with the table's one")
+ }
+ }
+ } else if (isSchemaCompatible(latestTableSchema, canonicalizedSourceSchema, allowAutoEvolutionColumnDrop)) {
+ canonicalizedSourceSchema
+ } else {
+ log.error(
+ s"""Incoming batch schema is not compatible with the table's one.
+ |Incoming schema ${sourceSchema.toString(true)}
+ |Incoming schema (canonicalized) ${canonicalizedSourceSchema.toString(true)}
+ |Table's schema ${latestTableSchema.toString(true)}
+ |""".stripMargin)
+ throw new SchemaCompatibilityException("Incoming batch schema is not compatible with the table's one")
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Canonicalizes [[sourceSchema]] by reconciling it w/ [[latestTableSchema]] in following
+ *
+ *
+ * Nullability: making sure that nullability of the fields in the source schema is matching
+ * that of the latest table's ones
+ *
+ *
+ * TODO support casing reconciliation
+ */
+ private def canonicalizeSchema(sourceSchema: Schema, latestTableSchema: Schema, opts : Map[String, String]): Schema = {
+ reconcileSchemaRequirements(sourceSchema, latestTableSchema, opts)
+ }
+
+
+ private def reconcileSchemasLegacy(tableSchema: Schema, newSchema: Schema): (Schema, Boolean) = {
+ // Legacy reconciliation implements following semantic
+ // - In case new-schema is a "compatible" projection of the existing table's one (projection allowing
+ // permitted type promotions), table's schema would be picked as (reconciled) writer's schema;
+ // - Otherwise, we'd fall back to picking new (batch's) schema as a writer's schema;
+ //
+ // Philosophically, such semantic aims at always choosing a "wider" schema, ie the one containing
+ // the other one (schema A contains schema B, if schema B is a projection of A). This enables us,
+ // to always "extend" the schema during schema evolution and hence never lose the data (when, for ex
+ // existing column is being dropped in a new batch)
+ //
+ // NOTE: By default Hudi doesn't allow automatic schema evolution to drop the columns from the target
+ // table. However, when schema reconciliation is turned on, we would allow columns to be dropped
+ // in the incoming batch (as these would be reconciled in anyway)
+ if (isCompatibleProjectionOf(tableSchema, newSchema)) {
+ // Picking table schema as a writer schema we need to validate that we'd be able to
+ // rewrite incoming batch's data (written in new schema) into it
+ (tableSchema, isSchemaCompatible(newSchema, tableSchema))
+ } else {
+ // Picking new schema as a writer schema we need to validate that we'd be able to
+ // rewrite table's data into it
+ (newSchema, isSchemaCompatible(tableSchema, newSchema))
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
index 93b8c2fcf9182..4504803377e6f 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
@@ -27,9 +27,9 @@ import org.apache.hudi.DataSourceOptionsHelper.fetchMissingWriteConfigsFromTable
import org.apache.hudi.DataSourceUtils.tryOverrideParquetWriteLegacyFormatProperty
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.HoodieConversionUtils.{toProperties, toScalaOption}
-import org.apache.hudi.HoodieSparkSqlWriter.{CANONICALIZE_NULLABLE, SQL_MERGE_INTO_WRITES, StreamingWriteParams}
+import org.apache.hudi.HoodieSparkSqlWriter.{CANONICALIZE_SCHEMA, SQL_MERGE_INTO_WRITES, StreamingWriteParams}
import org.apache.hudi.HoodieWriterUtils._
-import org.apache.hudi.avro.AvroSchemaUtils.{canProject, isCompatibleProjectionOf, isSchemaCompatible, resolveNullableSchema}
+import org.apache.hudi.avro.AvroSchemaUtils.{isCompatibleProjectionOf, isSchemaCompatible, isValidEvolutionOf, resolveNullableSchema}
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.avro.HoodieAvroUtils.removeMetadataFields
import org.apache.hudi.client.common.HoodieSparkEngineContext
@@ -42,7 +42,7 @@ import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
import org.apache.hudi.common.model._
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType
-import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstantTimeGenerator, TimeGenerators}
+import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.ConfigUtils.getAllConfigKeys
import org.apache.hudi.common.util.{CommitUtils, StringUtils, Option => HOption}
@@ -53,7 +53,7 @@ import org.apache.hudi.exception.{HoodieException, HoodieWriteConflictException,
import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool}
import org.apache.hudi.internal.schema.InternalSchema
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
-import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils.reconcileNullability
+import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils.reconcileSchemaRequirements
import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper}
import org.apache.hudi.keygen.constant.KeyGeneratorType
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
@@ -94,8 +94,8 @@ object HoodieSparkSqlWriter {
*
* NOTE: This is an internal config that is not exposed to the public
*/
- val CANONICALIZE_NULLABLE: ConfigProperty[Boolean] =
- ConfigProperty.key("hoodie.internal.write.schema.canonicalize.nullable")
+ val CANONICALIZE_SCHEMA: ConfigProperty[Boolean] =
+ ConfigProperty.key("hoodie.internal.write.schema.canonicalize")
.defaultValue(true)
/**
@@ -142,7 +142,14 @@ object HoodieSparkSqlWriter {
latestTableSchemaOpt: Option[Schema],
internalSchemaOpt: Option[InternalSchema],
opts: Map[String, String]): Schema = {
- new HoodieSparkSqlWriterInternal().deduceWriterSchema(sourceSchema, latestTableSchemaOpt, internalSchemaOpt, opts)
+ HoodieSchemaUtils.deduceWriterSchema(sourceSchema, latestTableSchemaOpt, internalSchemaOpt, opts)
+ }
+
+ def deduceWriterSchema(sourceSchema: Schema,
+ latestTableSchemaOpt: Option[Schema],
+ internalSchemaOpt: Option[InternalSchema],
+ props: TypedProperties): Schema = {
+ deduceWriterSchema(sourceSchema, latestTableSchemaOpt, internalSchemaOpt, props.toMap)
}
def cleanup(): Unit = {
@@ -340,7 +347,7 @@ class HoodieSparkSqlWriterInternal {
.getOrElse(getAvroRecordNameAndNamespace(tblName))
val sourceSchema = convertStructTypeToAvroSchema(df.schema, avroRecordName, avroRecordNamespace)
- val internalSchemaOpt = getLatestTableInternalSchema(hoodieConfig, tableMetaClient).orElse {
+ val internalSchemaOpt = HoodieSchemaUtils.getLatestTableInternalSchema(hoodieConfig, tableMetaClient).orElse {
// In case we need to reconcile the schema and schema evolution is enabled,
// we will force-apply schema evolution to the writer's schema
if (shouldReconcileSchema && hoodieConfig.getBooleanOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED)) {
@@ -374,7 +381,7 @@ class HoodieSparkSqlWriterInternal {
}
// Create a HoodieWriteClient & issue the delete.
- val internalSchemaOpt = getLatestTableInternalSchema(hoodieConfig, tableMetaClient)
+ val internalSchemaOpt = HoodieSchemaUtils.getLatestTableInternalSchema(hoodieConfig, tableMetaClient)
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
null, path, tblName,
mapAsJavaMap(addSchemaEvolutionParameters(parameters, internalSchemaOpt) - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
@@ -428,7 +435,7 @@ class HoodieSparkSqlWriterInternal {
// NOTE: Target writer's schema is deduced based on
// - Source's schema
// - Existing table's schema (including its Hudi's [[InternalSchema]] representation)
- val writerSchema = deduceWriterSchema(sourceSchema, latestTableSchemaOpt, internalSchemaOpt, parameters)
+ val writerSchema = HoodieSchemaUtils.deduceWriterSchema(sourceSchema, latestTableSchemaOpt, internalSchemaOpt, parameters)
validateSchemaForHoodieIsDeleted(writerSchema)
mayBeValidateParamsForAutoGenerationOfRecordKeys(parameters, hoodieConfig)
@@ -554,37 +561,41 @@ class HoodieSparkSqlWriterInternal {
* Target table's schema (including Hudi's [[InternalSchema]] representation)
*
*/
- def deduceWriterSchema(sourceSchema: Schema,
+ /*def deduceWriterSchema(sourceSchema: Schema,
latestTableSchemaOpt: Option[Schema],
internalSchemaOpt: Option[InternalSchema],
opts: Map[String, String]): Schema = {
+ val setNullForMissingColumns = opts.getOrDefault(DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS.key(),
+ DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS.defaultValue).toBoolean
val shouldReconcileSchema = opts(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean
val shouldValidateSchemasCompatibility = opts.getOrDefault(HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE.key,
HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE.defaultValue).toBoolean
latestTableSchemaOpt match {
// In case table schema is empty we're just going to use the source schema as a
- // writer's schema. No additional handling is required
- case None => sourceSchema
+ // writer's schema.
+ case None => AvroInternalSchemaConverter.fixNullOrdering(sourceSchema)
// Otherwise, we need to make sure we reconcile incoming and latest table schemas
case Some(latestTableSchemaWithMetaFields) =>
// NOTE: Meta-fields will be unconditionally injected by Hudi writing handles, for the sake of
// deducing proper writer schema we're stripping them to make sure we can perform proper
// analysis
- val latestTableSchema = removeMetadataFields(latestTableSchemaWithMetaFields)
+ //add call to fix null ordering to ensure backwards compatibility
+ val latestTableSchema = AvroInternalSchemaConverter.fixNullOrdering(removeMetadataFields(latestTableSchemaWithMetaFields))
// Before validating whether schemas are compatible, we need to "canonicalize" source's schema
// relative to the table's one, by doing a (minor) reconciliation of the nullability constraints:
// for ex, if in incoming schema column A is designated as non-null, but it's designated as nullable
// in the table's one we want to proceed aligning nullability constraints w/ the table's schema
- val shouldCanonicalizeNullable = opts.getOrDefault(CANONICALIZE_NULLABLE.key,
- CANONICALIZE_NULLABLE.defaultValue.toString).toBoolean
+ // Also, we promote types to the latest table schema if possible.
+ val shouldCanonicalizeSchema = opts.getOrDefault(CANONICALIZE_SCHEMA.key,
+ CANONICALIZE_SCHEMA.defaultValue.toString).toBoolean
val mergeIntoWrites = opts.getOrDefault(SQL_MERGE_INTO_WRITES.key(),
SQL_MERGE_INTO_WRITES.defaultValue.toString).toBoolean
- val canonicalizedSourceSchema = if (shouldCanonicalizeNullable) {
+ val canonicalizedSourceSchema = if (shouldCanonicalizeSchema) {
canonicalizeSchema(sourceSchema, latestTableSchema, opts)
} else {
- sourceSchema
+ AvroInternalSchemaConverter.fixNullOrdering(sourceSchema)
}
val allowAutoEvolutionColumnDrop = opts.getOrDefault(HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key,
@@ -637,17 +648,25 @@ class HoodieSparkSqlWriterInternal {
} else {
if (!shouldValidateSchemasCompatibility) {
// if no validation is enabled, check for col drop
- // if col drop is allowed, go ahead. if not, check for projection, so that we do not allow dropping cols
- if (allowAutoEvolutionColumnDrop || canProject(latestTableSchema, canonicalizedSourceSchema)) {
+ if (allowAutoEvolutionColumnDrop) {
canonicalizedSourceSchema
} else {
- log.error(
- s"""Incoming batch schema is not compatible with the table's one.
- |Incoming schema ${sourceSchema.toString(true)}
- |Incoming schema (canonicalized) ${canonicalizedSourceSchema.toString(true)}
- |Table's schema ${latestTableSchema.toString(true)}
- |""".stripMargin)
- throw new SchemaCompatibilityException("Incoming batch schema is not compatible with the table's one")
+ val reconciledSchema = if (setNullForMissingColumns) {
+ AvroSchemaEvolutionUtils.reconcileSchema(canonicalizedSourceSchema, latestTableSchema)
+ } else {
+ canonicalizedSourceSchema
+ }
+ if (isValidEvolutionOf(reconciledSchema, latestTableSchema)) {
+ reconciledSchema
+ } else {
+ log.error(
+ s"""Incoming batch schema is not compatible with the table's one.
+ |Incoming schema ${sourceSchema.toString(true)}
+ |Incoming schema (canonicalized) ${reconciledSchema.toString(true)}
+ |Table's schema ${latestTableSchema.toString(true)}
+ |""".stripMargin)
+ throw new SchemaCompatibilityException("Incoming batch schema is not compatible with the table's one")
+ }
}
} else if (isSchemaCompatible(latestTableSchema, canonicalizedSourceSchema, allowAutoEvolutionColumnDrop)) {
canonicalizedSourceSchema
@@ -663,7 +682,7 @@ class HoodieSparkSqlWriterInternal {
}
}
}
- }
+ }*/
/**
* Resolve wildcards in partitions
@@ -737,68 +756,6 @@ class HoodieSparkSqlWriterInternal {
HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE.key() -> schemaValidateEnable)
}
- private def reconcileSchemasLegacy(tableSchema: Schema, newSchema: Schema): (Schema, Boolean) = {
- // Legacy reconciliation implements following semantic
- // - In case new-schema is a "compatible" projection of the existing table's one (projection allowing
- // permitted type promotions), table's schema would be picked as (reconciled) writer's schema;
- // - Otherwise, we'd fall back to picking new (batch's) schema as a writer's schema;
- //
- // Philosophically, such semantic aims at always choosing a "wider" schema, ie the one containing
- // the other one (schema A contains schema B, if schema B is a projection of A). This enables us,
- // to always "extend" the schema during schema evolution and hence never lose the data (when, for ex
- // existing column is being dropped in a new batch)
- //
- // NOTE: By default Hudi doesn't allow automatic schema evolution to drop the columns from the target
- // table. However, when schema reconciliation is turned on, we would allow columns to be dropped
- // in the incoming batch (as these would be reconciled in anyway)
- if (isCompatibleProjectionOf(tableSchema, newSchema)) {
- // Picking table schema as a writer schema we need to validate that we'd be able to
- // rewrite incoming batch's data (written in new schema) into it
- (tableSchema, isSchemaCompatible(newSchema, tableSchema))
- } else {
- // Picking new schema as a writer schema we need to validate that we'd be able to
- // rewrite table's data into it
- (newSchema, isSchemaCompatible(tableSchema, newSchema))
- }
- }
-
- /**
- * Canonicalizes [[sourceSchema]] by reconciling it w/ [[latestTableSchema]] in following
- *
- *
- * Nullability: making sure that nullability of the fields in the source schema is matching
- * that of the latest table's ones
- *
- *
- * TODO support casing reconciliation
- */
- private def canonicalizeSchema(sourceSchema: Schema, latestTableSchema: Schema, opts : Map[String, String]): Schema = {
- reconcileNullability(sourceSchema, latestTableSchema, opts)
- }
-
-
- /**
- * get latest internalSchema from table
- *
- * @param config instance of {@link HoodieConfig}
- * @param tableMetaClient instance of HoodieTableMetaClient
- * @return Pair of(boolean, table schema), where first entry will be true only if schema conversion is required.
- */
- def getLatestTableInternalSchema(config: HoodieConfig,
- tableMetaClient: HoodieTableMetaClient): Option[InternalSchema] = {
- if (!config.getBooleanOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED)) {
- Option.empty[InternalSchema]
- } else {
- try {
- val tableSchemaResolver = new TableSchemaResolver(tableMetaClient)
- val internalSchemaOpt = tableSchemaResolver.getTableInternalSchemaFromCommitMetadata
- if (internalSchemaOpt.isPresent) Some(internalSchemaOpt.get()) else None
- } catch {
- case _: Exception => None
- }
- }
- }
-
private def registerAvroSchemasWithKryo(sparkContext: SparkContext, targetAvroSchemas: Schema*): Unit = {
sparkContext.getConf.registerAvroSchemas(targetAvroSchemas: _*)
}
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
index ce1a719cb94ba..599bbebe4f6c4 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
@@ -30,11 +30,13 @@ object HoodieParquetFileFormatHelper {
val convert = new ParquetToSparkSchemaConverter(hadoopConf)
val fileStruct = convert.convert(parquetFileMetaData.getSchema)
val fileStructMap = fileStruct.fields.map(f => (f.name, f.dataType)).toMap
+ // if there are missing fields or if field's data type needs to be changed while reading, we handle it here.
val sparkRequestStructFields = requiredSchema.map(f => {
val requiredType = f.dataType
if (fileStructMap.contains(f.name) && !isDataTypeEqual(requiredType, fileStructMap(f.name))) {
- implicitTypeChangeInfo.put(new Integer(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, fileStructMap(f.name)))
- StructField(f.name, fileStructMap(f.name), f.nullable)
+ val readerType = addMissingFields(requiredType, fileStructMap(f.name))
+ implicitTypeChangeInfo.put(new Integer(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, readerType))
+ StructField(f.name, readerType, f.nullable)
} else {
f
}
@@ -69,4 +71,19 @@ object HoodieParquetFileFormatHelper {
case _ => false
}
+
+ def addMissingFields(requiredType: DataType, fileType: DataType): DataType = (requiredType, fileType) match {
+ case (requiredType, fileType) if requiredType == fileType => fileType
+ case (ArrayType(rt, _), ArrayType(ft, _)) => ArrayType(addMissingFields(rt, ft))
+ case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => MapType(addMissingFields(requiredKey, fileKey), addMissingFields(requiredValue, fileValue))
+ case (StructType(requiredFields), StructType(fileFields)) =>
+ val fileFieldMap = fileFields.map(f => f.name -> f).toMap
+ StructType(requiredFields.map(f => {
+ fileFieldMap.get(f.name) match {
+ case Some(ff) => StructField(ff.name, addMissingFields(f.dataType, ff.dataType), ff.nullable, ff.metadata)
+ case None => f
+ }
+ }))
+ case _ => fileType
+ }
}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
index 4cbdf3778bd50..dd3c9df3bc881 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi.command
import org.apache.avro.Schema
import org.apache.hudi.AvroConversionUtils.convertStructTypeToAvroSchema
import org.apache.hudi.DataSourceWriteOptions._
-import org.apache.hudi.HoodieSparkSqlWriter.CANONICALIZE_NULLABLE
+import org.apache.hudi.HoodieSparkSqlWriter.CANONICALIZE_SCHEMA
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.common.model.HoodieAvroRecordMerger
import org.apache.hudi.common.util.StringUtils
@@ -726,7 +726,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
// target table, ie partially updating)
AVRO_SCHEMA_VALIDATE_ENABLE.key -> "false",
RECONCILE_SCHEMA.key -> "false",
- CANONICALIZE_NULLABLE.key -> "false",
+ CANONICALIZE_SCHEMA.key -> "false",
SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key -> "true",
HoodieSparkSqlWriter.SQL_MERGE_INTO_WRITES.key -> "true",
HoodieWriteConfig.SPARK_SQL_MERGE_INTO_PREPPED_KEY -> enableOptimizedMerge,
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala
index 16df1f869c6bc..d42e28fb98104 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala
@@ -89,7 +89,9 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"name" : "nullableMap",
"type" : [ "null", {
"type" : "map",
- "values" : [ {
+ "values" : [
+ "null",
+ {
"type" : "record",
"name" : "nullableMap",
"namespace" : "SchemaNS.SchemaName",
@@ -101,14 +103,16 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"type" : [ "null", "int" ],
"default" : null
} ]
- }, "null" ]
+ } ]
} ],
"default" : null
}, {
"name" : "map",
"type" : {
"type" : "map",
- "values" : [ {
+ "values" : [
+ "null",
+ {
"type" : "record",
"name" : "map",
"namespace" : "SchemaNS.SchemaName",
@@ -120,13 +124,15 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"type" : [ "null", "int" ],
"default" : null
} ]
- }, "null" ]
+ } ]
}
}, {
"name" : "nullableArray",
"type" : [ "null", {
"type" : "array",
- "items" : [ {
+ "items" : [
+ "null",
+ {
"type" : "record",
"name" : "nullableArray",
"namespace" : "SchemaNS.SchemaName",
@@ -138,14 +144,16 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"type" : [ "null", "int" ],
"default" : null
} ]
- }, "null" ]
+ } ]
} ],
"default" : null
}, {
"name" : "array",
"type" : {
"type" : "array",
- "items" : [ {
+ "items" : [
+ "null",
+ {
"type" : "record",
"name" : "array",
"namespace" : "SchemaNS.SchemaName",
@@ -157,7 +165,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"type" : [ "null", "int" ],
"default" : null
} ]
- }, "null" ]
+ } ]
}
} ]
}
@@ -257,6 +265,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
{
"type": "map",
"values": [
+ "null",
{
"type": "record",
"name": "nullableMap",
@@ -276,8 +285,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"default": null
}
]
- },
- "null"
+ }
]
}
],
@@ -288,6 +296,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"type": {
"type": "map",
"values": [
+ "null",
{
"type": "record",
"name": "map",
@@ -307,8 +316,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"default": null
}
]
- },
- "null"
+ }
]
}
},
@@ -319,6 +327,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
{
"type": "array",
"items": [
+ "null",
{
"type": "record",
"name": "nullableArray",
@@ -338,8 +347,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"default": null
}
]
- },
- "null"
+ }
]
}
],
@@ -350,6 +358,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"type": {
"type": "array",
"items": [
+ "null",
{
"type": "record",
"name": "array",
@@ -369,8 +378,7 @@ class TestAvroConversionUtils extends FunSuite with Matchers {
"default": null
}
]
- },
- "null"
+ }
]
}
}
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala
index 2b1060e90f0cd..a8f7c3c10ee1f 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala
@@ -727,7 +727,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss
df2.printSchema()
df2.show(false)
// upsert
- upsertData(df2, tempRecordPath, isCow)
+ upsertData(df2, tempRecordPath, isCow, true)
// read out the table
val readDf = spark.read.format("hudi").load(tempRecordPath)
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala
index 51682119d23f9..36ac37cfd6d4b 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala
@@ -20,8 +20,8 @@ package org.apache.hudi
import org.apache.avro.generic.GenericRecord
import org.apache.hudi.testutils.DataSourceTestUtils
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.types.{ArrayType, StructField, StructType}
+import org.apache.spark.sql.{DataFrame, Row, SparkSession}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
@@ -212,3 +212,28 @@ class TestHoodieSparkUtils {
def convertRowListToSeq(inputList: java.util.List[Row]): Seq[Row] =
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
}
+
+object TestHoodieSparkUtils {
+
+
+ def setNullableRec(structType: StructType, columnName: Array[String], index: Int): StructType = {
+ StructType(structType.map {
+ case StructField(name, StructType(fields), nullable, metadata) if name.equals(columnName(index)) =>
+ StructField(name, setNullableRec(StructType(fields), columnName, index + 1), nullable, metadata)
+ case StructField(name, ArrayType(StructType(fields), _), nullable, metadata) if name.equals(columnName(index)) =>
+ StructField(name, ArrayType(setNullableRec(StructType(fields), columnName, index + 1)), nullable, metadata)
+ case StructField(name, dataType, _, metadata) if name.equals(columnName(index)) =>
+ StructField(name, dataType, nullable = false, metadata)
+ case y: StructField => y
+ })
+ }
+
+ def setColumnNotNullable(df: DataFrame, columnName: String): DataFrame = {
+ // get schema
+ val schema = df.schema
+ // modify [[StructField] with name `cn`
+ val newSchema = setNullableRec(schema, columnName.split('.'), 0)
+ // apply new schema
+ df.sqlContext.createDataFrame(df.rdd, newSchema)
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala
index b5d1e61b7aa30..dfb69da29c005 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala
@@ -338,11 +338,16 @@ class TestBasicSchemaEvolution extends HoodieSparkClientTestBase with ScalaAsser
Row("11", "14", "1", 1),
Row("12", "16", "1", 1))
- // NOTE: Expected to fail in both cases, as such transformation is not permitted
- assertThrows(classOf[SchemaCompatibilityException]) {
+ // Now, only fails for reconcile
+ if (shouldReconcileSchema) {
+ assertThrows(classOf[SchemaCompatibilityException]) {
+ appendData(sixthSchema, sixthBatch)
+ }
+ } else {
appendData(sixthSchema, sixthBatch)
}
+
// TODO add test w/ overlapping updates
}
}
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
index 2686ba8689bb2..d6c29d430516b 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
@@ -564,13 +564,9 @@ public static SchemaProvider createRowBasedSchemaProvider(StructType structType,
return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc, null);
}
- public static Option getLatestTableSchema(JavaSparkContext jssc, FileSystem fs, String basePath) {
+ public static Option getLatestTableSchema(JavaSparkContext jssc, FileSystem fs, String basePath, HoodieTableMetaClient tableMetaClient) {
try {
if (FSUtils.isTableExists(basePath, fs)) {
- HoodieTableMetaClient tableMetaClient = HoodieTableMetaClient.builder()
- .setConf(jssc.sc().hadoopConfiguration())
- .setBasePath(basePath)
- .build();
TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(tableMetaClient);
return tableSchemaResolver.getTableAvroSchemaFromLatestCommit(false);
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/LazyCastingIterator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/LazyCastingIterator.java
new file mode 100644
index 0000000000000..eb654a69269c6
--- /dev/null
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/LazyCastingIterator.java
@@ -0,0 +1,42 @@
+/*
+ * 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.utilities.schema;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.Iterator;
+
+public class LazyCastingIterator extends LazyIterableIterator {
+
+ private final Schema targetSchema;
+ public LazyCastingIterator(Iterator in, String serializedTargetSchema) {
+ super(in);
+ this.targetSchema = new Schema.Parser().parse(serializedTargetSchema);
+ }
+
+ @Override
+ protected GenericRecord computeNext() {
+ return HoodieAvroUtils.rewriteRecordDeep(inputItr.next(), targetSchema);
+ }
+}
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java
index 4bb79a146a72d..362bbb72f9e9a 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java
@@ -23,6 +23,7 @@
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.HoodieConversionUtils;
+import org.apache.hudi.HoodieSchemaUtils;
import org.apache.hudi.HoodieSparkSqlWriter;
import org.apache.hudi.HoodieSparkUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
@@ -86,6 +87,7 @@
import org.apache.hudi.utilities.exception.HoodieStreamerWriteException;
import org.apache.hudi.utilities.ingestion.HoodieIngestionMetrics;
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
+import org.apache.hudi.utilities.schema.LazyCastingIterator;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SchemaSet;
import org.apache.hudi.utilities.schema.SimpleSchemaProvider;
@@ -400,7 +402,7 @@ public Pair, JavaRDD> syncOnce() throws IOException
.build();
String instantTime = metaClient.createNewInstantTime();
- Pair inputBatchIsEmptyPair = readFromSource(instantTime);
+ Pair inputBatchIsEmptyPair = readFromSource(instantTime, metaClient);
if (inputBatchIsEmptyPair != null) {
final JavaRDD recordsFromSource;
@@ -476,7 +478,8 @@ private Option getLastPendingCompactionInstant(Option co
* @return Pair Input data read from upstream source, and boolean is true if empty.
* @throws Exception in case of any Exception
*/
- public Pair readFromSource(String instantTime) throws IOException {
+
+ public Pair readFromSource(String instantTime, HoodieTableMetaClient metaClient) throws IOException {
// Retrieve the previous round checkpoints, if any
Option resumeCheckpointStr = Option.empty();
if (commitsTimelineOpt.isPresent()) {
@@ -494,7 +497,7 @@ public Pair readFromSource(String instantTime) throws IOExc
Pair sourceDataToSync = null;
while (curRetryCount++ < maxRetryCount && sourceDataToSync == null) {
try {
- sourceDataToSync = fetchFromSourceAndPrepareRecords(resumeCheckpointStr, instantTime);
+ sourceDataToSync = fetchFromSourceAndPrepareRecords(resumeCheckpointStr, instantTime, metaClient);
} catch (HoodieSourceTimeoutException e) {
if (curRetryCount >= maxRetryCount) {
throw e;
@@ -511,7 +514,8 @@ public Pair readFromSource(String instantTime) throws IOExc
return sourceDataToSync;
}
- private Pair fetchFromSourceAndPrepareRecords(Option resumeCheckpointStr, String instantTime) {
+ private Pair fetchFromSourceAndPrepareRecords(Option resumeCheckpointStr, String instantTime,
+ HoodieTableMetaClient metaClient) {
HoodieRecordType recordType = createRecordMerger(props).getRecordType();
if (recordType == HoodieRecordType.SPARK && HoodieTableType.valueOf(cfg.tableType) == HoodieTableType.MERGE_ON_READ
&& !cfg.operation.equals(WriteOperationType.BULK_INSERT)
@@ -520,7 +524,7 @@ private Pair fetchFromSourceAndPrepareRecords(Option fetchFromSourceAndPrepareRecords(Option resumeCheckpointStr) {
+ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr, HoodieTableMetaClient metaClient) {
Option> avroRDDOptional = null;
String checkpointStr = null;
SchemaProvider schemaProvider = null;
InputBatch inputBatchForWriter = null; // row writer
+ boolean reconcileSchema = props.getBoolean(DataSourceWriteOptions.RECONCILE_SCHEMA().key());
if (transformer.isPresent()) {
// Transformation is needed. Fetch New rows in Row Format, apply transformation and then convert them
// to generic records for writing
@@ -572,7 +577,6 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr)
ErrorEvent.ErrorReason.CUSTOM_TRANSFORMER_FAILURE);
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
- boolean reconcileSchema = props.getBoolean(DataSourceWriteOptions.RECONCILE_SCHEMA().key());
if (this.userProvidedSchemaProvider != null && this.userProvidedSchemaProvider.getTargetSchema() != null) {
if (useRowWriter) {
if (errorTableWriter.isPresent()) {
@@ -581,6 +585,9 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr)
inputBatchForWriter = new InputBatch(transformed, checkpointStr, this.userProvidedSchemaProvider);
} else {
// non row writer path
+ // Let's deduce the schema provider for writer side first!
+ schemaProvider = getDeducedSchemaProvider(this.userProvidedSchemaProvider.getTargetSchema(), this.userProvidedSchemaProvider, metaClient);
+ SchemaProvider finalSchemaProvider = schemaProvider;
// If the target schema is specified through Avro schema,
// pass in the schema for the Row-to-Avro conversion
// to avoid nullability mismatch between Avro schema and Row schema
@@ -593,7 +600,7 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr)
rowDataset -> {
Tuple2, RDD> safeCreateRDDs = HoodieSparkUtils.safeCreateRDD(rowDataset,
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, reconcileSchema,
- Option.of(this.userProvidedSchemaProvider.getTargetSchema()));
+ Option.of(finalSchemaProvider.getTargetSchema()));
errorTableWriter.get().addErrorEvents(safeCreateRDDs._2().toJavaRDD()
.map(evStr -> new ErrorEvent<>(evStr,
ErrorEvent.ErrorReason.AVRO_DESERIALIZATION_FAILURE)));
@@ -601,30 +608,18 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr)
});
} else {
avroRDDOptional = transformed.map(
- rowDataset -> getTransformedRDD(rowDataset, reconcileSchema, this.userProvidedSchemaProvider.getTargetSchema()));
+ rowDataset -> getTransformedRDD(rowDataset, reconcileSchema, finalSchemaProvider.getTargetSchema()));
}
- schemaProvider = this.userProvidedSchemaProvider;
}
} else {
- Option latestTableSchemaOpt = UtilHelpers.getLatestTableSchema(hoodieSparkContext.jsc(), fs, cfg.targetBasePath);
- // Deduce proper target (writer's) schema for the transformed dataset, reconciling its
+ // Deduce proper target (writer's) schema for the input dataset, reconciling its
// schema w/ the table's one
- Option targetSchemaOpt = transformed.map(df -> {
- Schema sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema(),
- latestTableSchemaOpt.map(Schema::getFullName).orElse(getAvroRecordQualifiedName(cfg.targetTableName)));
- // Target (writer's) schema is determined based on the incoming source schema
- // and existing table's one, reconciling the two (if necessary) based on configuration
- return HoodieSparkSqlWriter.deduceWriterSchema(
- sourceSchema,
- HoodieConversionUtils.toScalaOption(latestTableSchemaOpt),
- HoodieConversionUtils.toScalaOption(Option.empty()),
- HoodieConversionUtils.fromProperties(props));
- });
- // Override schema provider with the reconciled target schema
- schemaProvider = targetSchemaOpt.map(targetSchema ->
- (SchemaProvider) new DelegatingSchemaProvider(props, hoodieSparkContext.jsc(), dataAndCheckpoint.getSchemaProvider(),
- new SimpleSchemaProvider(hoodieSparkContext.jsc(), targetSchema, props)))
+ Option incomingSchemaOpt = transformed.map(df ->
+ AvroConversionUtils.convertStructTypeToAvroSchema(df.schema(), getAvroRecordQualifiedName(cfg.targetTableName)));
+
+ schemaProvider = incomingSchemaOpt.map(incomingSchema -> getDeducedSchemaProvider(incomingSchema, dataAndCheckpoint.getSchemaProvider(), metaClient))
.orElse(dataAndCheckpoint.getSchemaProvider());
+
if (useRowWriter) {
inputBatchForWriter = new InputBatch(transformed, checkpointStr, schemaProvider);
} else {
@@ -638,14 +633,15 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr)
inputBatchForWriter = formatAdapter.fetchNewDataInRowFormat(resumeCheckpointStr, cfg.sourceLimit);
} else {
// Pull the data from the source & prepare the write
- InputBatch> dataAndCheckpoint =
- formatAdapter.fetchNewDataInAvroFormat(resumeCheckpointStr, cfg.sourceLimit);
- avroRDDOptional = dataAndCheckpoint.getBatch();
+ InputBatch> dataAndCheckpoint = formatAdapter.fetchNewDataInAvroFormat(resumeCheckpointStr, cfg.sourceLimit);
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
- schemaProvider = dataAndCheckpoint.getSchemaProvider();
+ // Rewrite transformed records into the expected target schema
+ schemaProvider = getDeducedSchemaProvider(dataAndCheckpoint.getSchemaProvider().getTargetSchema(), dataAndCheckpoint.getSchemaProvider(), metaClient);
+ String serializedTargetSchema = schemaProvider.getTargetSchema().toString();
+ avroRDDOptional = dataAndCheckpoint.getBatch().map(t -> t.mapPartitions(iterator ->
+ new LazyCastingIterator(iterator, serializedTargetSchema)));
}
}
-
if (useRowWriter) {
return inputBatchForWriter;
} else {
@@ -680,6 +676,30 @@ private Pair handleEmptyBatch(boolean useRowWriter, InputBa
return Pair.of(inputBatch, false);
}
+ /**
+ * Apply schema reconcile and schema evolution rules(schema on read) and generate new target schema provider.
+ *
+ * @param incomingSchema schema of the source data
+ * @param sourceSchemaProvider Source schema provider.
+ * @return the SchemaProvider that can be used as writer schema.
+ */
+ private SchemaProvider getDeducedSchemaProvider(Schema incomingSchema, SchemaProvider sourceSchemaProvider, HoodieTableMetaClient metaClient) {
+ Option latestTableSchemaOpt = UtilHelpers.getLatestTableSchema(hoodieSparkContext.jsc(), fs, cfg.targetBasePath, metaClient);
+ Option internalSchemaOpt = HoodieConversionUtils.toJavaOption(
+ HoodieSchemaUtils.getLatestTableInternalSchema(
+ new HoodieConfig(HoodieStreamer.Config.getProps(fs, cfg)), metaClient));
+ // Deduce proper target (writer's) schema for the input dataset, reconciling its
+ // schema w/ the table's one
+ Schema targetSchema = HoodieSparkSqlWriter.deduceWriterSchema(
+ incomingSchema,
+ HoodieConversionUtils.toScalaOption(latestTableSchemaOpt),
+ HoodieConversionUtils.toScalaOption(internalSchemaOpt), props);
+
+ // Override schema provider with the reconciled target schema
+ return new DelegatingSchemaProvider(props, hoodieSparkContext.jsc(), sourceSchemaProvider,
+ new SimpleSchemaProvider(hoodieSparkContext.jsc(), targetSchema, props));
+ }
+
private JavaRDD getTransformedRDD(Dataset rowDataset, boolean reconcileSchema, Schema readerSchema) {
return HoodieSparkUtils.createRdd(rowDataset, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, reconcileSchema,
Option.ofNullable(readerSchema)).toJavaRDD();
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java
index 095a34cf75c98..594fe3d102899 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java
@@ -31,6 +31,8 @@
import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
+import org.apache.hudi.utilities.config.HoodieStreamerConfig;
+import org.apache.hudi.utilities.config.KafkaSourceConfig;
import org.apache.hudi.utilities.config.SourceTestConfig;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.sources.HoodieIncrSource;
@@ -41,6 +43,8 @@
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.streaming.kafka010.KafkaTestUtils;
@@ -57,6 +61,7 @@
import java.util.List;
import java.util.Map;
import java.util.Random;
+import java.util.UUID;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@@ -70,6 +75,7 @@
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
+import static org.apache.hudi.utilities.config.KafkaSourceConfig.KAFKA_AVRO_VALUE_DESERIALIZER_CLASS;
import static org.apache.hudi.utilities.streamer.HoodieStreamer.CHECKPOINT_KEY;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -91,6 +97,7 @@ public class HoodieDeltaStreamerTestBase extends UtilitiesTestBase {
static final String PROPS_FILENAME_TEST_PARQUET = "test-parquet-dfs-source.properties";
static final String PROPS_FILENAME_TEST_ORC = "test-orc-dfs-source.properties";
static final String PROPS_FILENAME_TEST_JSON_KAFKA = "test-json-kafka-dfs-source.properties";
+ static final String PROPS_FILENAME_TEST_AVRO_KAFKA = "test-avro-kafka-dfs-source.properties";
static final String PROPS_FILENAME_TEST_SQL_SOURCE = "test-sql-source-source.properties";
static final String PROPS_FILENAME_TEST_MULTI_WRITER = "test-multi-writer.properties";
static final String FIRST_PARQUET_FILE_NAME = "1.parquet";
@@ -377,6 +384,26 @@ protected void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTra
UtilitiesTestBase.Helpers.savePropsToDFS(parquetProps, fs, basePath + "/" + propsFileName);
}
+ protected void prepareAvroKafkaDFSSource(String propsFileName, Long maxEventsToReadFromKafkaSource, String topicName, String partitionPath, TypedProperties extraProps) throws IOException {
+ TypedProperties props = new TypedProperties(extraProps);
+ props.setProperty("bootstrap.servers", testUtils.brokerAddress());
+ props.put(HoodieStreamerConfig.KAFKA_APPEND_OFFSETS.key(), "false");
+ props.setProperty("auto.offset.reset", "earliest");
+ props.setProperty("include", "base.properties");
+ props.setProperty("hoodie.embed.timeline.server", "false");
+ props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
+ props.setProperty("hoodie.datasource.write.partitionpath.field", partitionPath);
+ props.setProperty("hoodie.deltastreamer.source.kafka.topic", topicName);
+ props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", String.valueOf(5000));
+ props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
+ props.setProperty(KAFKA_AVRO_VALUE_DESERIALIZER_CLASS.key(), ByteArrayDeserializer.class.getName());
+ props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents",
+ maxEventsToReadFromKafkaSource != null ? String.valueOf(maxEventsToReadFromKafkaSource) :
+ String.valueOf(KafkaSourceConfig.MAX_EVENTS_FROM_KAFKA_SOURCE.defaultValue()));
+ props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+ UtilitiesTestBase.Helpers.savePropsToDFS(props, fs, basePath + "/" + propsFileName);
+ }
+
protected static void prepareORCDFSFiles(int numRecords) throws IOException {
prepareORCDFSFiles(numRecords, ORC_SOURCE_ROOT);
}
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java
index 2b042abf45e6d..967ad45218263 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java
@@ -1460,8 +1460,8 @@ private void testBulkInsertRowWriterContinuousMode(Boolean useSchemaProvider, Li
@ParameterizedTest
@EnumSource(value = HoodieRecordType.class, names = {"AVRO","SPARK"})
public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline(HoodieRecordType recordType) throws Exception {
- String tableBasePath = basePath + "/test_table2";
- String downstreamTableBasePath = basePath + "/test_downstream_table2";
+ String tableBasePath = basePath + "/" + recordType.toString() + "/test_table2";
+ String downstreamTableBasePath = basePath + "/" + recordType.toString() + "/test_downstream_table2";
// Initial bulk insert to ingest to first hudi table
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT,
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java
new file mode 100644
index 0000000000000..87dc5b89da068
--- /dev/null
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java
@@ -0,0 +1,296 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieClusteringConfig;
+import org.apache.hudi.config.HoodieCompactionConfig;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.hudi.utilities.sources.AvroKafkaSource;
+import org.apache.hudi.utilities.sources.ParquetDFSSource;
+import org.apache.hudi.utilities.streamer.HoodieStreamer;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
+import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Add test cases for out of the box schema evolution for deltastreamer:
+ * https://hudi.apache.org/docs/schema_evolution#out-of-the-box-schema-evolution
+ */
+public class TestHoodieDeltaStreamerSchemaEvolutionBase extends HoodieDeltaStreamerTestBase {
+
+ protected static Set createdTopicNames = new HashSet<>();
+
+ protected String tableType;
+ protected String tableBasePath;
+ protected Boolean shouldCluster;
+ protected Boolean shouldCompact;
+ protected Boolean rowWriterEnable;
+ protected Boolean addFilegroups;
+ protected Boolean multiLogFiles;
+ protected Boolean useSchemaProvider;
+ protected Boolean hasTransformer;
+ protected String sourceSchemaFile;
+ protected String targetSchemaFile;
+ protected boolean useKafkaSource;
+ protected boolean useTransformer;
+ protected boolean userProvidedSchema;
+
+ @BeforeAll
+ public static void initKafka() {
+ defaultSchemaProviderClassName = TestSchemaProvider.class.getName();
+ }
+
+ @BeforeEach
+ public void setupTest() {
+ super.setupTest();
+ useSchemaProvider = false;
+ hasTransformer = false;
+ sourceSchemaFile = "";
+ targetSchemaFile = "";
+ topicName = "topic" + testNum;
+ }
+
+ @AfterEach
+ public void teardown() throws Exception {
+ super.teardown();
+ TestSchemaProvider.resetTargetSchema();
+ }
+
+ @AfterAll
+ static void teardownAll() {
+ defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName();
+ HoodieDeltaStreamerTestBase.cleanupKafkaTestUtils();
+ }
+
+ protected HoodieStreamer deltaStreamer;
+
+ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig() throws IOException {
+ return getDeltaStreamerConfig(true);
+ }
+
+ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(boolean nullForDeletedCols) throws IOException {
+ String[] transformerClasses = useTransformer ? new String[] {TestHoodieDeltaStreamer.TestIdentityTransformer.class.getName()}
+ : new String[0];
+ return getDeltaStreamerConfig(transformerClasses, nullForDeletedCols);
+ }
+
+ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformerClasses, boolean nullForDeletedCols) throws IOException {
+ return getDeltaStreamerConfig(transformerClasses, nullForDeletedCols, new TypedProperties());
+ }
+
+ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformerClasses, boolean nullForDeletedCols,
+ TypedProperties extraProps) throws IOException {
+ extraProps.setProperty("hoodie.datasource.write.table.type", tableType);
+ extraProps.setProperty("hoodie.datasource.write.row.writer.enable", rowWriterEnable.toString());
+ extraProps.setProperty(DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS().key(), Boolean.toString(nullForDeletedCols));
+
+ //we set to 0 so that we create new base files on insert instead of adding inserts to existing filegroups via small file handling
+ extraProps.setProperty("hoodie.parquet.small.file.limit", "0");
+
+ //We only want compaction/clustering to kick in after the final commit. This is because after compaction/clustering we have base files again
+ //and adding to base files is already covered by the tests. This is important especially for mor, because we want to see how compaction/clustering
+ //behaves when schema evolution is happening in the log files
+ int maxCommits = 2;
+ if (addFilegroups) {
+ maxCommits++;
+ }
+ if (multiLogFiles) {
+ maxCommits++;
+ }
+
+ extraProps.setProperty(HoodieCompactionConfig.INLINE_COMPACT.key(), shouldCompact.toString());
+ if (shouldCompact) {
+ extraProps.setProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), Integer.toString(maxCommits));
+ }
+
+ if (shouldCluster) {
+ extraProps.setProperty(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true");
+ extraProps.setProperty(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key(), Integer.toString(maxCommits));
+ extraProps.setProperty(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key(), "_row_key");
+ }
+
+ List transformerClassNames = new ArrayList<>();
+ Collections.addAll(transformerClassNames, transformerClasses);
+
+ HoodieDeltaStreamer.Config cfg;
+ if (useKafkaSource) {
+ prepareAvroKafkaDFSSource(PROPS_FILENAME_TEST_AVRO_KAFKA, null, topicName,"partition_path", extraProps);
+ cfg = TestHoodieDeltaStreamer.TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, AvroKafkaSource.class.getName(),
+ transformerClassNames, PROPS_FILENAME_TEST_AVRO_KAFKA, false, useSchemaProvider, 100000, false, null, tableType, "timestamp", null);
+ } else {
+ prepareParquetDFSSource(false, hasTransformer, sourceSchemaFile, targetSchemaFile, PROPS_FILENAME_TEST_PARQUET,
+ PARQUET_SOURCE_ROOT, false, "partition_path", "", extraProps);
+ cfg = TestHoodieDeltaStreamer.TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, ParquetDFSSource.class.getName(),
+ transformerClassNames, PROPS_FILENAME_TEST_PARQUET, false,
+ useSchemaProvider, 100000, false, null, tableType, "timestamp", null);
+ }
+ cfg.forceDisableCompaction = !shouldCompact;
+ return cfg;
+ }
+
+ protected void addData(Dataset df, Boolean isFirst) {
+ if (useSchemaProvider) {
+ TestSchemaProvider.sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema(), HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE);
+ }
+ if (useKafkaSource) {
+ addKafkaData(df, isFirst);
+ } else {
+ addParquetData(df, isFirst);
+ }
+ }
+
+ protected void addParquetData(Dataset df, Boolean isFirst) {
+ df.write().format("parquet").mode(isFirst ? SaveMode.Overwrite : SaveMode.Append).save(PARQUET_SOURCE_ROOT);
+ }
+
+ protected void addKafkaData(Dataset df, Boolean isFirst) {
+ if (isFirst && !createdTopicNames.contains(topicName)) {
+ testUtils.createTopic(topicName);
+ createdTopicNames.add(topicName);
+ }
+ List records = HoodieSparkUtils.createRdd(df, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, false, Option.empty()).toJavaRDD().collect();
+ try (Producer producer = new KafkaProducer<>(getProducerProperties())) {
+ for (GenericRecord record : records) {
+ producer.send(new ProducerRecord<>(topicName, 0, "key", HoodieAvroUtils.avroToBytes(record)));
+ }
+ }
+ }
+
+ protected Properties getProducerProperties() {
+ Properties props = new Properties();
+ props.put("bootstrap.servers", testUtils.brokerAddress());
+ props.put("value.serializer", ByteArraySerializer.class.getName());
+ props.put("value.deserializer", ByteArraySerializer.class.getName());
+ // Key serializer is required.
+ props.put("key.serializer", StringSerializer.class.getName());
+ props.put("auto.register.schemas", "false");
+ // wait for all in-sync replicas to ack sends
+ props.put("acks", "all");
+ return props;
+ }
+
+ /**
+ * see how many files are read from in the latest commit. This verification is for making sure the test scenarios
+ * are setup as expected, rather than testing schema evolution functionality
+ */
+ protected void assertFileNumber(int expected, boolean isCow) {
+ if (isCow) {
+ assertBaseFileOnlyNumber(expected);
+ } else {
+ //we can't differentiate between _hoodie_file_name for log files, so we use commit time as the differentiator between them
+ assertEquals(expected, sparkSession.read().format("hudi").load(tableBasePath).select("_hoodie_commit_time", "_hoodie_file_name").distinct().count());
+ }
+ }
+
+ /**
+ * Base files might have multiple different commit times in the same file. To ensure this is only used when there are only base files
+ * there is a check that every file ends with .parquet, as log files don't in _hoodie_file_name
+ */
+ protected void assertBaseFileOnlyNumber(int expected) {
+ Dataset df = sparkSession.read().format("hudi").load(tableBasePath).select("_hoodie_file_name");
+ df.createOrReplaceTempView("assertFileNumberPostCompactCluster");
+ assertEquals(df.count(), sparkSession.sql("select * from assertFileNumberPostCompactCluster where _hoodie_file_name like '%.parquet'").count());
+ assertEquals(expected, df.distinct().count());
+ }
+
+ protected void assertRecordCount(int expected) {
+ sqlContext.clearCache();
+ long recordCount = sqlContext.read().format("org.apache.hudi").load(tableBasePath).count();
+ assertEquals(expected, recordCount);
+ }
+
+ protected StructType createFareStruct(DataType amountType) {
+ return createFareStruct(amountType, false);
+ }
+
+ protected StructType createFareStruct(DataType amountType, Boolean dropCols) {
+ if (dropCols) {
+ return DataTypes.createStructType(new StructField[]{new StructField("amount", amountType, true, Metadata.empty())});
+ }
+ return DataTypes.createStructType(new StructField[]{new StructField("amount", amountType, true, Metadata.empty()),
+ new StructField("currency", DataTypes.StringType, true, Metadata.empty())});
+ }
+
+ public static class TestSchemaProvider extends SchemaProvider {
+
+ public static Schema sourceSchema;
+ public static Schema targetSchema = null;
+
+ public TestSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
+ super(props, jssc);
+ }
+
+ @Override
+ public Schema getSourceSchema() {
+ return sourceSchema;
+ }
+
+ @Override
+ public Schema getTargetSchema() {
+ return targetSchema != null ? targetSchema : sourceSchema;
+ }
+
+ public static void setTargetSchema(Schema targetSchema) {
+ TestSchemaProvider.targetSchema = targetSchema;
+ }
+
+ public static void resetTargetSchema() {
+ TestSchemaProvider.targetSchema = null;
+ }
+ }
+}
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionExtensive.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionExtensive.java
new file mode 100644
index 0000000000000..723971f6fa1fb
--- /dev/null
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionExtensive.java
@@ -0,0 +1,500 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.hudi.TestHoodieSparkUtils;
+
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+
+/**
+ * Takes hours to run. Use to debug schema evolution. Don't enable for ci
+ */
+@Disabled
+public class TestHoodieDeltaStreamerSchemaEvolutionExtensive extends TestHoodieDeltaStreamerSchemaEvolutionBase {
+
+ protected void testBase(String updateFile, String updateColumn, String condition, int count) throws Exception {
+ Map conditions = new HashMap<>();
+ conditions.put(condition, count);
+ testBase(updateFile, updateColumn, conditions, true);
+
+ //adding non-nullable cols should fail, but instead it is adding nullable cols
+ //assertThrows(Exception.class, () -> testBase(tableType, shouldCluster, shouldCompact, reconcileSchema, rowWriterEnable, updateFile, updateColumn, condition, count, false));
+ }
+
+ protected void testBase(String updateFile, String updateColumn, Map conditions) throws Exception {
+ testBase(updateFile, updateColumn, conditions, true);
+ }
+
+ protected void doFirstDeltaWrite() throws Exception {
+ doDeltaWriteBase("start.json", true, false,null);
+ }
+
+ protected void doFirstDeltaWriteTypePromo(String colName, DataType colType) throws Exception {
+ doDeltaWriteBase("startTypePromotion.json", true, false, true, colName, colType);
+ }
+
+ protected void doDeltaWriteTypePromo(String resourceString, String colName, DataType colType) throws Exception {
+ doDeltaWriteBase(resourceString, false, false, true, colName, colType);
+
+ }
+
+ protected void doNonNullableDeltaWrite(String resourceString, String colName) throws Exception {
+ doDeltaWriteBase(resourceString, false, true, colName);
+ }
+
+ protected void doDeltaWrite(String resourceString) throws Exception {
+ doDeltaWriteBase(resourceString, false, false,null);
+ }
+
+ protected void doDeltaWriteBase(String resourceString, Boolean isFirst, Boolean nonNullable, String colName) throws Exception {
+ doDeltaWriteBase(resourceString, isFirst, nonNullable, false, colName, null);
+ }
+
+ protected void doDeltaWriteBase(String resourceString, Boolean isFirst, Boolean nonNullable, Boolean castColumn, String colName, DataType colType) throws Exception {
+ String datapath = String.class.getResource("/data/schema-evolution/" + resourceString).getPath();
+ Dataset df = sparkSession.read().json(datapath);
+ if (nonNullable) {
+ df = TestHoodieSparkUtils.setColumnNotNullable(df, colName);
+ }
+ if (castColumn) {
+ Column col = df.col(colName);
+ df = df.withColumn(colName, col.cast(colType));
+ }
+
+ addData(df, isFirst);
+ deltaStreamer.sync();
+ }
+
+ /**
+ * Main testing logic for non-type promotion tests
+ */
+ protected void testBase(String updateFile, String updateColumn, Map conditions, Boolean nullable) throws Exception {
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + testNum++;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+ this.deltaStreamer = new HoodieDeltaStreamer(getDeltaStreamerConfig(), jsc);
+
+ //first write
+ doFirstDeltaWrite();
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+
+
+ //add extra log files
+ if (multiLogFiles) {
+ doDeltaWrite("extraLogFiles.json");
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ //make other filegroups
+ if (addFilegroups) {
+ doDeltaWrite("newFileGroups.json");
+ numRecords += 3;
+ numFiles += 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+ }
+
+ //write updates
+ if (!nullable) {
+ doNonNullableDeltaWrite(updateFile, updateColumn);
+ } else {
+ doDeltaWrite(updateFile);
+ }
+ if (shouldCluster) {
+ //everything combines into 1 file per partition
+ assertBaseFileOnlyNumber(3);
+ } else if (shouldCompact || isCow) {
+ assertBaseFileOnlyNumber(numFiles);
+ } else {
+ numFiles += 2;
+ if (updateFile.equals("testAddColChangeOrderAllFiles.json")) {
+ //this test updates all 3 partitions instead of 2 like the rest of the tests
+ numFiles++;
+ }
+ assertFileNumber(numFiles, false);
+ }
+ assertRecordCount(numRecords);
+
+ Dataset df = sparkSession.read().format("hudi").load(tableBasePath);
+ df.show(9,false);
+ df.select(updateColumn).show(9);
+ for (String condition : conditions.keySet()) {
+ assertEquals(conditions.get(condition).intValue(), df.filter(condition).count());
+ }
+
+ }
+
+ protected static Stream testArgs() {
+ Stream.Builder b = Stream.builder();
+ //only testing row-writer enabled for now
+ for (Boolean rowWriterEnable : new Boolean[]{true}) {
+ for (Boolean addFilegroups : new Boolean[]{false, true}) {
+ for (Boolean multiLogFiles : new Boolean[]{false, true}) {
+ for (Boolean shouldCluster : new Boolean[]{false, true}) {
+ for (String tableType : new String[]{"COPY_ON_WRITE", "MERGE_ON_READ"}) {
+ if (!multiLogFiles || tableType.equals("MERGE_ON_READ")) {
+ b.add(Arguments.of(tableType, shouldCluster, false, rowWriterEnable, addFilegroups, multiLogFiles));
+ }
+ }
+ }
+ b.add(Arguments.of("MERGE_ON_READ", false, true, rowWriterEnable, addFilegroups, multiLogFiles));
+ }
+ }
+ }
+ return b.build();
+ }
+
+ /**
+ * Add a new column at root level at the end
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testAddColRoot(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testAddColRoot.json", "zextra_col", "zextra_col = 'yes'", 2);
+ }
+
+ /**
+ * Drop a root column
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testDropColRoot(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testDropColRoot.json", "trip_type", "trip_type is NULL", 2);
+ }
+
+ /**
+ * Add a custom Hudi meta column
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testAddMetaCol(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testAddMetaCol.json", "_extra_col", "_extra_col = 'yes'", 2);
+ }
+
+ /**
+ * Add a new column to inner struct (at the end)
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testAddColStruct(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testAddColStruct.json", "tip_history.zextra_col", "tip_history[0].zextra_col = 'yes'", 2);
+ }
+
+ /**
+ * Drop a root column
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testDropColStruct(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testDropColStruct.json", "tip_history.currency", "tip_history[0].currency is NULL", 2);
+ }
+
+ /**
+ * Add a new complex type field with default (array)
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testAddComplexField(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testAddComplexField.json", "zcomplex_array", "size(zcomplex_array) > 0", 2);
+ }
+
+ /**
+ * Add a new column and change the ordering of fields
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testAddColChangeOrder(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ testBase("testAddColChangeOrderAllFiles.json", "extra_col", "extra_col = 'yes'", 2);
+ //according to the docs, this should fail. But it doesn't
+ //assertThrows(Exception.class, () -> testBase("testAddColChangeOrderSomeFiles.json", "extra_col", "extra_col = 'yes'", 1));
+ }
+
+ /**
+ * Add and drop cols in the same write
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testAddAndDropCols(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ Map conditions = new HashMap<>();
+ conditions.put("distance_in_meters is NULL", 2);
+ conditions.put("tip_history[0].currency is NULL", 2);
+ conditions.put("tip_history[0].zextra_col_nest = 'yes'", 2);
+ conditions.put("zextra_col = 'yes'", 2);
+ testBase("testAddAndDropCols.json", "tip_history", conditions);
+ }
+
+ protected String typePromoUpdates;
+
+ protected void assertDataType(String colName, DataType expectedType) {
+ assertEquals(expectedType, sparkSession.read().format("hudi").load(tableBasePath).select(colName).schema().fields()[0].dataType());
+ }
+
+ protected void testTypePromotionBase(String colName, DataType startType, DataType updateType) throws Exception {
+ testTypePromotionBase(colName, startType, updateType, updateType);
+ }
+
+ protected void testTypeDemotionBase(String colName, DataType startType, DataType updateType) throws Exception {
+ testTypePromotionBase(colName, startType, updateType, startType);
+ }
+
+ protected void testTypePromotionBase(String colName, DataType startType, DataType updateType, DataType endType) throws Exception {
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + testNum++;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+ this.deltaStreamer = new HoodieDeltaStreamer(getDeltaStreamerConfig(), jsc);
+
+ //first write
+ doFirstDeltaWriteTypePromo(colName, startType);
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+ assertDataType(colName, startType);
+
+ //add extra log files
+ if (multiLogFiles) {
+ doDeltaWriteTypePromo("extraLogFilesTypePromo.json", colName, startType);
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ //make other filegroups
+ if (addFilegroups) {
+ doDeltaWriteTypePromo("newFileGroupsTypePromo.json", colName, startType);
+ numRecords += 3;
+ numFiles += 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+ }
+
+ //write updates
+ doDeltaWriteTypePromo(typePromoUpdates, colName, updateType);
+ if (shouldCluster) {
+ //everything combines into 1 file per partition
+ assertBaseFileOnlyNumber(3);
+ } else if (shouldCompact || isCow) {
+ assertBaseFileOnlyNumber(numFiles);
+ } else {
+ numFiles += 2;
+ assertFileNumber(numFiles, false);
+ }
+ assertRecordCount(numRecords);
+ sparkSession.read().format("hudi").load(tableBasePath).select(colName).show(9);
+ assertDataType(colName, endType);
+ }
+
+ /**
+ * Test type promotion for fields
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testTypePromotion(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ testTypePromotion(tableType, shouldCluster, shouldCompact, rowWriterEnable, addFilegroups, multiLogFiles, false);
+ }
+
+
+ /**
+ * Test type promotion for fields
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testTypePromotionDropCols(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles) throws Exception {
+ testTypePromotion(tableType, shouldCluster, shouldCompact, rowWriterEnable, addFilegroups, multiLogFiles, true);
+ }
+
+ public void testTypePromotion(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles,
+ Boolean dropCols) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ if (dropCols) {
+ this.typePromoUpdates = "endTypePromotionDropCols.json";
+ } else {
+ this.typePromoUpdates = "endTypePromotion.json";
+ }
+
+
+ //root data type promotions
+ testTypePromotionBase("distance_in_meters", DataTypes.IntegerType, DataTypes.LongType);
+ testTypePromotionBase("distance_in_meters", DataTypes.IntegerType, DataTypes.FloatType);
+ testTypePromotionBase("distance_in_meters", DataTypes.IntegerType, DataTypes.DoubleType);
+ testTypePromotionBase("distance_in_meters", DataTypes.IntegerType, DataTypes.StringType);
+ testTypePromotionBase("distance_in_meters", DataTypes.LongType, DataTypes.FloatType);
+ testTypePromotionBase("distance_in_meters", DataTypes.LongType, DataTypes.DoubleType);
+ testTypePromotionBase("distance_in_meters", DataTypes.LongType, DataTypes.StringType);
+ testTypePromotionBase("begin_lat", DataTypes.FloatType, DataTypes.DoubleType);
+ testTypePromotionBase("begin_lat", DataTypes.FloatType, DataTypes.StringType);
+ testTypePromotionBase("begin_lat", DataTypes.DoubleType, DataTypes.StringType);
+ //should stay with the original
+ testTypeDemotionBase("rider", DataTypes.StringType, DataTypes.BinaryType);
+ testTypeDemotionBase("rider", DataTypes.BinaryType, DataTypes.StringType);
+
+ //nested data type promotions
+ testTypePromotionBase("fare", createFareStruct(DataTypes.FloatType), createFareStruct(DataTypes.DoubleType, dropCols), createFareStruct(DataTypes.DoubleType));
+ testTypePromotionBase("fare", createFareStruct(DataTypes.FloatType), createFareStruct(DataTypes.StringType, dropCols), createFareStruct(DataTypes.StringType));
+
+ //complex data type promotion
+ testTypePromotionBase("tip_history", DataTypes.createArrayType(DataTypes.IntegerType), DataTypes.createArrayType(DataTypes.LongType));
+ testTypePromotionBase("tip_history", DataTypes.createArrayType(DataTypes.IntegerType), DataTypes.createArrayType(DataTypes.DoubleType));
+ testTypePromotionBase("tip_history", DataTypes.createArrayType(DataTypes.IntegerType), DataTypes.createArrayType(DataTypes.StringType));
+
+ //test type demotions
+ //root data type demotion
+ testTypeDemotionBase("distance_in_meters", DataTypes.LongType, DataTypes.IntegerType);
+ testTypeDemotionBase("distance_in_meters", DataTypes.StringType, DataTypes.LongType);
+ //nested data type demotion
+ testTypePromotionBase("fare", createFareStruct(DataTypes.DoubleType), createFareStruct(DataTypes.FloatType, dropCols), createFareStruct(DataTypes.DoubleType));
+ testTypePromotionBase("fare", createFareStruct(DataTypes.StringType), createFareStruct(DataTypes.DoubleType, dropCols), createFareStruct(DataTypes.StringType));
+ //complex data type demotion
+ testTypeDemotionBase("tip_history", DataTypes.createArrayType(DataTypes.LongType), DataTypes.createArrayType(DataTypes.IntegerType));
+ testTypeDemotionBase("tip_history", DataTypes.createArrayType(DataTypes.StringType), DataTypes.createArrayType(DataTypes.LongType));
+ }
+}
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java
new file mode 100644
index 0000000000000..de21b33fff4e6
--- /dev/null
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java
@@ -0,0 +1,596 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.SchemaCompatibilityException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.streamer.HoodieStreamer;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieDeltaStreamerSchemaEvolutionQuick extends TestHoodieDeltaStreamerSchemaEvolutionBase {
+
+ @AfterEach
+ public void teardown() throws Exception {
+ super.teardown();
+ TestSchemaProvider.resetTargetSchema();
+ }
+
+ protected static Stream testArgs() {
+ Stream.Builder b = Stream.builder();
+ //only testing row-writer enabled for now
+ for (Boolean rowWriterEnable : new Boolean[] {true}) {
+ for (Boolean nullForDeletedCols : new Boolean[] {false, true}) {
+ for (Boolean useKafkaSource : new Boolean[] {false, true}) {
+ for (Boolean addFilegroups : new Boolean[] {false, true}) {
+ for (Boolean multiLogFiles : new Boolean[] {false, true}) {
+ for (Boolean shouldCluster : new Boolean[] {false, true}) {
+ for (String tableType : new String[] {"COPY_ON_WRITE", "MERGE_ON_READ"}) {
+ if (!multiLogFiles || tableType.equals("MERGE_ON_READ")) {
+ b.add(Arguments.of(tableType, shouldCluster, false, rowWriterEnable, addFilegroups, multiLogFiles, useKafkaSource, nullForDeletedCols));
+ }
+ }
+ }
+ b.add(Arguments.of("MERGE_ON_READ", false, true, rowWriterEnable, addFilegroups, multiLogFiles, useKafkaSource, nullForDeletedCols));
+ }
+ }
+ }
+ }
+ }
+ return b.build();
+ }
+
+ protected static Stream testReorderedColumn() {
+ Stream.Builder b = Stream.builder();
+ for (Boolean rowWriterEnable : new Boolean[] {true}) {
+ for (Boolean nullForDeletedCols : new Boolean[] {false, true}) {
+ for (Boolean useKafkaSource : new Boolean[] {false, true}) {
+ for (String tableType : new String[] {"COPY_ON_WRITE", "MERGE_ON_READ"}) {
+ b.add(Arguments.of(tableType, rowWriterEnable, useKafkaSource, nullForDeletedCols));
+ }
+ }
+ }
+ }
+ return b.build();
+ }
+
+ protected static Stream testParamsWithSchemaTransformer() {
+ Stream.Builder b = Stream.builder();
+ for (Boolean useTransformer : new Boolean[] {false, true}) {
+ for (Boolean setSchema : new Boolean[] {false, true}) {
+ for (Boolean rowWriterEnable : new Boolean[] {true}) {
+ for (Boolean nullForDeletedCols : new Boolean[] {false, true}) {
+ for (Boolean useKafkaSource : new Boolean[] {false, true}) {
+ for (String tableType : new String[] {"COPY_ON_WRITE", "MERGE_ON_READ"}) {
+ b.add(Arguments.of(tableType, rowWriterEnable, useKafkaSource, nullForDeletedCols, useTransformer, setSchema));
+ }
+ }
+ }
+ }
+ }
+ }
+ return b.build();
+ }
+
+ /**
+ * Main testing logic for non-type promotion tests
+ */
+ @ParameterizedTest
+ @MethodSource("testArgs")
+ public void testBase(String tableType,
+ Boolean shouldCluster,
+ Boolean shouldCompact,
+ Boolean rowWriterEnable,
+ Boolean addFilegroups,
+ Boolean multiLogFiles,
+ Boolean useKafkaSource,
+ Boolean allowNullForDeletedCols) throws Exception {
+ this.tableType = tableType;
+ this.shouldCluster = shouldCluster;
+ this.shouldCompact = shouldCompact;
+ this.rowWriterEnable = rowWriterEnable;
+ this.addFilegroups = addFilegroups;
+ this.multiLogFiles = multiLogFiles;
+ this.useKafkaSource = useKafkaSource;
+ if (useKafkaSource) {
+ this.useSchemaProvider = true;
+ }
+ this.useTransformer = true;
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + ++testNum;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+ this.deltaStreamer = new HoodieDeltaStreamer(getDeltaStreamerConfig(allowNullForDeletedCols), jsc);
+
+ //first write
+ String datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ Dataset df = sparkSession.read().json(datapath);
+ addData(df, true);
+ deltaStreamer.sync();
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+
+ //add extra log files
+ if (multiLogFiles) {
+ datapath = String.class.getResource("/data/schema-evolution/extraLogFilesTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ addData(df, false);
+ deltaStreamer.sync();
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ //make other filegroups
+ if (addFilegroups) {
+ datapath = String.class.getResource("/data/schema-evolution/newFileGroupsTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ addData(df, false);
+ deltaStreamer.sync();
+ numRecords += 3;
+ numFiles += 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+ }
+
+ //write updates
+ datapath = String.class.getResource("/data/schema-evolution/endTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ //do casting
+ Column col = df.col("tip_history");
+ df = df.withColumn("tip_history", col.cast(DataTypes.createArrayType(DataTypes.LongType)));
+ col = df.col("fare");
+ df = df.withColumn("fare", col.cast(DataTypes.createStructType(new StructField[]{
+ new StructField("amount", DataTypes.StringType, true, Metadata.empty()),
+ new StructField("currency", DataTypes.StringType, true, Metadata.empty()),
+ new StructField("zextra_col_nested", DataTypes.StringType, true, Metadata.empty())
+ })));
+ col = df.col("begin_lat");
+ df = df.withColumn("begin_lat", col.cast(DataTypes.DoubleType));
+ col = df.col("end_lat");
+ df = df.withColumn("end_lat", col.cast(DataTypes.StringType));
+ col = df.col("distance_in_meters");
+ df = df.withColumn("distance_in_meters", col.cast(DataTypes.FloatType));
+ col = df.col("seconds_since_epoch");
+ df = df.withColumn("seconds_since_epoch", col.cast(DataTypes.StringType));
+
+ try {
+ addData(df, false);
+ deltaStreamer.sync();
+ assertTrue(allowNullForDeletedCols);
+ } catch (SchemaCompatibilityException e) {
+ assertTrue(e.getMessage().contains("Incoming batch schema is not compatible with the table's one"));
+ assertFalse(allowNullForDeletedCols);
+ return;
+ }
+
+ if (shouldCluster) {
+ //everything combines into 1 file per partition
+ assertBaseFileOnlyNumber(3);
+ } else if (shouldCompact || isCow) {
+ assertBaseFileOnlyNumber(numFiles);
+ } else {
+ numFiles += 2;
+ assertFileNumber(numFiles, false);
+ }
+ assertRecordCount(numRecords);
+
+ df = sparkSession.read().format("hudi").load(tableBasePath);
+ df.show(100,false);
+ df.cache();
+ assertDataType(df, "tip_history", DataTypes.createArrayType(DataTypes.LongType));
+ assertDataType(df, "fare", DataTypes.createStructType(new StructField[]{
+ new StructField("amount", DataTypes.StringType, true, Metadata.empty()),
+ new StructField("currency", DataTypes.StringType, true, Metadata.empty()),
+ new StructField("extra_col_struct", DataTypes.LongType, true, Metadata.empty()),
+ new StructField("zextra_col_nested", DataTypes.StringType, true, Metadata.empty())
+ }));
+ assertDataType(df, "begin_lat", DataTypes.DoubleType);
+ assertDataType(df, "end_lat", DataTypes.StringType);
+ assertDataType(df, "distance_in_meters", DataTypes.FloatType);
+ assertDataType(df, "seconds_since_epoch", DataTypes.StringType);
+ assertCondition(df, "zextra_col = 'yes'", 2);
+ assertCondition(df, "_extra_col = 'yes'", 2);
+ assertCondition(df, "fare.zextra_col_nested = 'yes'", 2);
+ assertCondition(df, "size(zcomplex_array) > 0", 2);
+ assertCondition(df, "extra_col_regular is NULL", 2);
+ assertCondition(df, "fare.extra_col_struct is NULL", 2);
+ }
+
+
+ /**
+ * Main testing logic for non-type promotion tests
+ */
+ @ParameterizedTest
+ @MethodSource("testReorderedColumn")
+ public void testReorderingColumn(String tableType,
+ Boolean rowWriterEnable,
+ Boolean useKafkaSource,
+ Boolean allowNullForDeletedCols) throws Exception {
+ this.tableType = tableType;
+ this.rowWriterEnable = rowWriterEnable;
+ this.useKafkaSource = useKafkaSource;
+ this.shouldCluster = false;
+ this.shouldCompact = false;
+ this.addFilegroups = false;
+ this.multiLogFiles = false;
+ this.useTransformer = true;
+ if (useKafkaSource) {
+ this.useSchemaProvider = true;
+ }
+
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + ++testNum;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+
+ //first write
+ String datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ Dataset df = sparkSession.read().json(datapath);
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+ addData(df, true);
+ deltaStreamer.sync();
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+
+ //add extra log files
+ if (tableType.equals("MERGE_ON_READ")) {
+ datapath = String.class.getResource("/data/schema-evolution/extraLogFilesTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ addData(df, false);
+ deltaStreamer.sync();
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ assertRecordCount(numRecords);
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+
+ HoodieStreamer.Config dsConfig = deltaStreamer.getConfig();
+ HoodieTableMetaClient metaClient = getMetaClient(dsConfig);
+ HoodieInstant lastInstant = metaClient.getActiveTimeline().lastInstant().get();
+
+ //test reordering column
+ datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ df = df.drop("rider").withColumn("rider", functions.lit("rider-003"));
+
+ addData(df, false);
+ deltaStreamer.sync();
+
+ metaClient.reloadActiveTimeline();
+ Option latestTableSchemaOpt = UtilHelpers.getLatestTableSchema(jsc, fs, dsConfig.targetBasePath, metaClient);
+ assertTrue(latestTableSchemaOpt.get().getField("rider").schema().getTypes()
+ .stream().anyMatch(t -> t.getType().equals(Schema.Type.STRING)));
+ assertTrue(metaClient.reloadActiveTimeline().lastInstant().get().compareTo(lastInstant) > 0);
+ }
+
+ @ParameterizedTest
+ @MethodSource("testParamsWithSchemaTransformer")
+ public void testDroppedColumn(String tableType,
+ Boolean rowWriterEnable,
+ Boolean useKafkaSource,
+ Boolean allowNullForDeletedCols,
+ Boolean useTransformer,
+ Boolean targetSchemaSameAsTableSchema) throws Exception {
+ this.tableType = tableType;
+ this.rowWriterEnable = rowWriterEnable;
+ this.useKafkaSource = useKafkaSource;
+ this.shouldCluster = false;
+ this.shouldCompact = false;
+ this.addFilegroups = false;
+ this.multiLogFiles = false;
+ this.useTransformer = useTransformer;
+ if (useKafkaSource || targetSchemaSameAsTableSchema) {
+ this.useSchemaProvider = true;
+ }
+
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + ++testNum;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+
+ //first write
+ String datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ Dataset df = sparkSession.read().json(datapath);
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+ addData(df, true);
+ deltaStreamer.sync();
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+
+ //add extra log files
+ if (tableType.equals("MERGE_ON_READ")) {
+ datapath = String.class.getResource("/data/schema-evolution/extraLogFilesTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ addData(df, false);
+ deltaStreamer.sync();
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ if (targetSchemaSameAsTableSchema) {
+ TestSchemaProvider.setTargetSchema(TestSchemaProvider.sourceSchema);
+ }
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+
+ HoodieStreamer.Config dsConfig = deltaStreamer.getConfig();
+ HoodieTableMetaClient metaClient = getMetaClient(dsConfig);
+ HoodieInstant lastInstant = metaClient.getActiveTimeline().lastInstant().get();
+
+ // drop column
+ datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ Dataset droppedColumnDf = df.drop("rider");
+ try {
+ addData(droppedColumnDf, true);
+ deltaStreamer.sync();
+ assertTrue(allowNullForDeletedCols || targetSchemaSameAsTableSchema);
+
+ metaClient.reloadActiveTimeline();
+ Option latestTableSchemaOpt = UtilHelpers.getLatestTableSchema(jsc, fs, dsConfig.targetBasePath, metaClient);
+ assertTrue(latestTableSchemaOpt.get().getField("rider").schema().getTypes()
+ .stream().anyMatch(t -> t.getType().equals(Schema.Type.STRING)));
+ assertTrue(metaClient.reloadActiveTimeline().lastInstant().get().compareTo(lastInstant) > 0);
+ } catch (SchemaCompatibilityException e) {
+ assertFalse(allowNullForDeletedCols || targetSchemaSameAsTableSchema);
+ assertTrue(e.getMessage().contains("Incoming batch schema is not compatible with the table's one"));
+ assertFalse(allowNullForDeletedCols);
+ }
+ }
+
+ @ParameterizedTest
+ @MethodSource("testParamsWithSchemaTransformer")
+ public void testTypePromotion(String tableType,
+ Boolean rowWriterEnable,
+ Boolean useKafkaSource,
+ Boolean allowNullForDeletedCols,
+ Boolean useTransformer,
+ Boolean targetSchemaSameAsTableSchema) throws Exception {
+ this.tableType = tableType;
+ this.rowWriterEnable = rowWriterEnable;
+ this.useKafkaSource = useKafkaSource;
+ this.shouldCluster = false;
+ this.shouldCompact = false;
+ this.addFilegroups = false;
+ this.multiLogFiles = false;
+ this.useTransformer = useTransformer;
+ if (useKafkaSource || targetSchemaSameAsTableSchema) {
+ this.useSchemaProvider = true;
+ }
+
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + ++testNum;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+
+ //first write
+ String datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ Dataset df = sparkSession.read().json(datapath);
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+ addData(df, true);
+ deltaStreamer.sync();
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+
+ //add extra log files
+ if (tableType.equals("MERGE_ON_READ")) {
+ datapath = String.class.getResource("/data/schema-evolution/extraLogFilesTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ addData(df, false);
+ deltaStreamer.sync();
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ if (targetSchemaSameAsTableSchema) {
+ TestSchemaProvider.setTargetSchema(TestSchemaProvider.sourceSchema);
+ }
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+
+ HoodieStreamer.Config dsConfig = deltaStreamer.getConfig();
+ HoodieTableMetaClient metaClient = getMetaClient(dsConfig);
+ HoodieInstant lastInstant = metaClient.getActiveTimeline().lastInstant().get();
+
+ // type promotion for dataset (int -> long)
+ datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ Column col = df.col("distance_in_meters");
+ Dataset typePromotionDf = df.withColumn("distance_in_meters", col.cast(DataTypes.DoubleType));
+ try {
+ addData(typePromotionDf, true);
+ deltaStreamer.sync();
+ assertFalse(targetSchemaSameAsTableSchema);
+
+ metaClient.reloadActiveTimeline();
+ Option latestTableSchemaOpt = UtilHelpers.getLatestTableSchema(jsc, fs, dsConfig.targetBasePath, metaClient);
+ assertTrue(latestTableSchemaOpt.get().getField("distance_in_meters").schema().getTypes()
+ .stream().anyMatch(t -> t.getType().equals(Schema.Type.DOUBLE)), latestTableSchemaOpt.get().getField("distance_in_meters").schema().toString());
+ assertTrue(metaClient.reloadActiveTimeline().lastInstant().get().compareTo(lastInstant) > 0);
+ } catch (Exception e) {
+ assertTrue(targetSchemaSameAsTableSchema);
+ if (!useKafkaSource) {
+ assertTrue(containsErrorMessage(e, "Incoming batch schema is not compatible with the table's one",
+ "org.apache.spark.sql.catalyst.expressions.MutableDouble cannot be cast to org.apache.spark.sql.catalyst.expressions.MutableLong",
+ "cannot support rewrite value for schema type: \"long\" since the old schema type is: \"double\""),
+ e.getMessage());
+ } else {
+ assertTrue(containsErrorMessage(e, "Incoming batch schema is not compatible with the table's one",
+ "cannot support rewrite value for schema type: \"long\" since the old schema type is: \"double\""),
+ e.getMessage());
+ }
+ }
+ }
+
+ @ParameterizedTest
+ @MethodSource("testParamsWithSchemaTransformer")
+ public void testTypeDemotion(String tableType,
+ Boolean rowWriterEnable,
+ Boolean useKafkaSource,
+ Boolean allowNullForDeletedCols,
+ Boolean useTransformer,
+ Boolean targetSchemaSameAsTableSchema) throws Exception {
+ this.tableType = tableType;
+ this.rowWriterEnable = rowWriterEnable;
+ this.useKafkaSource = useKafkaSource;
+ this.shouldCluster = false;
+ this.shouldCompact = false;
+ this.addFilegroups = false;
+ this.multiLogFiles = false;
+ this.useTransformer = useTransformer;
+ if (useKafkaSource || targetSchemaSameAsTableSchema) {
+ this.useSchemaProvider = true;
+ }
+
+ boolean isCow = tableType.equals("COPY_ON_WRITE");
+ PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + ++testNum;
+ tableBasePath = basePath + "test_parquet_table" + testNum;
+
+ //first write
+ String datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ Dataset df = sparkSession.read().json(datapath);
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+ addData(df, true);
+ deltaStreamer.sync();
+ int numRecords = 6;
+ int numFiles = 3;
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, isCow);
+
+ //add extra log files
+ if (tableType.equals("MERGE_ON_READ")) {
+ datapath = String.class.getResource("/data/schema-evolution/extraLogFilesTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ addData(df, false);
+ deltaStreamer.sync();
+ //this write contains updates for the 6 records from the first write, so
+ //although we have 2 files for each filegroup, we only see the log files
+ //represented in the read. So that is why numFiles is 3, not 6
+ assertRecordCount(numRecords);
+ assertFileNumber(numFiles, false);
+ }
+
+ if (targetSchemaSameAsTableSchema) {
+ TestSchemaProvider.setTargetSchema(TestSchemaProvider.sourceSchema);
+ }
+ resetTopicAndDeltaStreamer(allowNullForDeletedCols);
+
+ HoodieStreamer.Config dsConfig = deltaStreamer.getConfig();
+ HoodieTableMetaClient metaClient = getMetaClient(dsConfig);
+ HoodieInstant lastInstant = metaClient.getActiveTimeline().lastInstant().get();
+
+ // type demotion
+ datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath();
+ df = sparkSession.read().json(datapath);
+ Column col = df.col("current_ts");
+ Dataset typeDemotionDf = df.withColumn("current_ts", col.cast(DataTypes.IntegerType));
+ addData(typeDemotionDf, true);
+ deltaStreamer.sync();
+
+ metaClient.reloadActiveTimeline();
+ Option latestTableSchemaOpt = UtilHelpers.getLatestTableSchema(jsc, fs, dsConfig.targetBasePath, metaClient);
+ assertTrue(latestTableSchemaOpt.get().getField("current_ts").schema().getTypes()
+ .stream().anyMatch(t -> t.getType().equals(Schema.Type.LONG)));
+ assertTrue(metaClient.reloadActiveTimeline().lastInstant().get().compareTo(lastInstant) > 0);
+ }
+
+ private static HoodieTableMetaClient getMetaClient(HoodieStreamer.Config dsConfig) {
+ return HoodieTableMetaClient.builder()
+ .setConf(new Configuration(fs.getConf()))
+ .setBasePath(dsConfig.targetBasePath)
+ .setPayloadClassName(dsConfig.payloadClassName)
+ .build();
+ }
+
+ private void resetTopicAndDeltaStreamer(Boolean allowNullForDeletedCols) throws IOException {
+ topicName = "topic" + ++testNum;
+ if (this.deltaStreamer != null) {
+ this.deltaStreamer.shutdownGracefully();
+ }
+ String[] transformerClassNames = useTransformer ? new String[] {TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName()}
+ : new String[0];
+ TypedProperties extraProps = new TypedProperties();
+ extraProps.setProperty("hoodie.streamer.checkpoint.force.skip", "true");
+ HoodieDeltaStreamer.Config deltaStreamerConfig = getDeltaStreamerConfig(transformerClassNames, allowNullForDeletedCols, extraProps);
+ deltaStreamerConfig.checkpoint = "0";
+ this.deltaStreamer = new HoodieDeltaStreamer(deltaStreamerConfig, jsc);
+ }
+
+ private boolean containsErrorMessage(Throwable e, String... messages) {
+ while (e != null) {
+ for (String msg : messages) {
+ if (e.getMessage().contains(msg)) {
+ return true;
+ }
+ }
+ e = e.getCause();
+ }
+
+ return false;
+ }
+
+ protected void assertDataType(Dataset df, String colName, DataType expectedType) {
+ assertEquals(expectedType, df.select(colName).schema().fields()[0].dataType());
+ }
+
+ protected void assertCondition(Dataset df, String condition, int count) {
+ assertEquals(count, df.filter(condition).count());
+ }
+
+}
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestTransformer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestTransformer.java
index 888f5ebc2de17..494149cc5ef84 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestTransformer.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestTransformer.java
@@ -80,6 +80,7 @@ public void testMultipleTransformersWithIdentifiers() throws Exception {
assertRecordCount(parquetRecordsCount, tableBasePath, sqlContext);
assertEquals(0, sqlContext.read().format("org.apache.hudi").load(tableBasePath).where("timestamp != 110").count());
+ testNum++;
}
/**
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/schema/TestLazyCastingIterator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/schema/TestLazyCastingIterator.java
new file mode 100644
index 0000000000000..397c275383b9c
--- /dev/null
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/schema/TestLazyCastingIterator.java
@@ -0,0 +1,196 @@
+/*
+ * 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.utilities.schema;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestLazyCastingIterator {
+
+ private static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": ["
+ + "{\"name\": \"prop1\",\"type\": [\"null\", \"string\"]},{\"name\": \"prop2\", \"type\": \"long\"}]}";
+
+ private static final String EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ + "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ + "{\"name\": \"int_col\", \"type\": [\"null\", \"int\"], \"default\": null },"
+ + "{\"name\": \"long_col\", \"type\": [\"null\", \"long\"], \"default\": null },"
+ + "{\"name\": \"nested_col\",\"type\": [\"null\", " + NESTED_COL_SCHEMA + "]}"
+ + "]}";
+
+ private static final String EXAMPLE_SCHEMA_WITHOUT_NESTED_COL = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ + "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ + "{\"name\": \"int_col\", \"type\": [\"null\", \"int\"], \"default\": null },"
+ + "{\"name\": \"long_col\", \"type\": [\"null\", \"long\"], \"default\": null }"
+ + "]}";
+
+ private static final String EXAMPLE_SCHEMA_INT_COL_AS_LONG = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ + "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ + "{\"name\": \"int_col\", \"type\": [\"null\", \"long\"], \"default\": null },"
+ + "{\"name\": \"long_col\", \"type\": [\"null\", \"long\"], \"default\": null },"
+ + "{\"name\": \"nested_col\",\"type\": [\"null\", " + NESTED_COL_SCHEMA + "]}"
+ + "]}";
+
+ private static final String EXAMPLE_SCHEMA_LONG_COL_AS_INT = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ + "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ + "{\"name\": \"int_col\", \"type\": [\"null\", \"int\"], \"default\": null },"
+ + "{\"name\": \"long_col\", \"type\": [\"null\", \"int\"], \"default\": null },"
+ + "{\"name\": \"nested_col\",\"type\": [\"null\", " + NESTED_COL_SCHEMA + "]}"
+ + "]}";
+
+ private static final GenericRecord GEN_RECORD_EXAMPLE_WITH_NESTED = getRecordWithExampleSchema();
+ private static final GenericRecord GEN_RECORD_EXAMPLE_WITH_NULL_NESTED = getRecordWithExampleSchemaNullNestedCol();
+ private static final GenericRecord GEN_RECORD_EXAMPLE_WITHOUT_NESTED = getRecordWithExampleSchemaWithoutNestedCol();
+ private static final GenericRecord GEN_RECORD_EXAMPLE_INT_COL_AS_LONG = getRecordWithExampleSchemaIntColAsLong();
+ private static final GenericRecord GEN_RECORD_EXAMPLE_LONG_COL_AS_INT = getRecordWithExampleSchemaLongColAsInt();
+
+ @Test
+ // no changes to record
+ public void testHappyPath() {
+ List genericRecords = Collections.singletonList(GEN_RECORD_EXAMPLE_WITH_NESTED);
+ LazyCastingIterator itr = new LazyCastingIterator(genericRecords.iterator(), EXAMPLE_SCHEMA);
+ GenericRecord outGenRec = itr.next();
+ assertEquals(genericRecords.get(0), outGenRec);
+ }
+
+ @Test
+ // data has 1 additional col compared to schema
+ public void testDataWithAdditionalCol() {
+ List genericRecords = Collections.singletonList(GEN_RECORD_EXAMPLE_WITH_NESTED);
+ LazyCastingIterator itr = new LazyCastingIterator(genericRecords.iterator(), EXAMPLE_SCHEMA_WITHOUT_NESTED_COL);
+ GenericRecord outGenRec = itr.next();
+ // data will be equivalent to not having the additional col.
+ assertEquals(GEN_RECORD_EXAMPLE_WITHOUT_NESTED, outGenRec);
+ }
+
+ @Test
+ // data has 1 col missing compared to schema
+ public void testDataWithMissingCol() {
+ List genericRecords = Collections.singletonList(GEN_RECORD_EXAMPLE_WITHOUT_NESTED);
+ LazyCastingIterator itr = new LazyCastingIterator(genericRecords.iterator(), EXAMPLE_SCHEMA);
+ GenericRecord outGenRec = itr.next();
+ assertEquals(GEN_RECORD_EXAMPLE_WITH_NULL_NESTED, outGenRec);
+ }
+
+ @Test
+ // data has 1 col as int which is long in target schema. should cast w/o issues.
+ public void testDataForIntToLongPromotion() {
+ List genericRecords = Collections.singletonList(GEN_RECORD_EXAMPLE_LONG_COL_AS_INT);
+ LazyCastingIterator itr = new LazyCastingIterator(genericRecords.iterator(), EXAMPLE_SCHEMA);
+ GenericRecord outGenRec = itr.next();
+ assertEquals(GEN_RECORD_EXAMPLE_WITH_NESTED, outGenRec);
+ }
+
+ @Test
+ // data has 1 col as long which is int in target schema. casting directly should throw exception
+ public void testDataForLongToIntPromotion() {
+ List genericRecords = Collections.singletonList(GEN_RECORD_EXAMPLE_INT_COL_AS_LONG);
+ LazyCastingIterator itr = new LazyCastingIterator(genericRecords.iterator(), EXAMPLE_SCHEMA);
+ Exception e = assertThrows(RuntimeException.class, () -> {
+ itr.next();
+ }, "Should error out since long cannot be promoted to int");
+ assertTrue(e.getMessage().contains("cannot support rewrite value for schema type: \"int\" since the old schema type is: \"long\""));
+ }
+
+ public static GenericRecord getRecordWithExampleSchema() {
+ return getRecordWithExampleSchema(getNestedColRecord("val1", 10L));
+ }
+
+ public static GenericRecord getRecordWithExampleSchemaIntColAsLong() {
+ return getRecordWithExampleSchemaIntColAsLong(getNestedColRecord("val1", 10L));
+ }
+
+ public static GenericRecord getRecordWithExampleSchemaLongColAsInt() {
+ return getRecordWithExampleSchemaLongColAsInt(getNestedColRecord("val1", 10L));
+ }
+
+ public static GenericRecord getRecordWithExampleSchemaNullNestedCol() {
+ return getRecordWithExampleSchema(null);
+ }
+
+ public static GenericRecord getNestedColRecord(String prop1Value, Long prop2Value) {
+ GenericRecord nestedColRecord = new GenericData.Record(new Schema.Parser().parse(NESTED_COL_SCHEMA));
+ nestedColRecord.put("prop1", prop1Value);
+ nestedColRecord.put("prop2", prop2Value);
+ return nestedColRecord;
+ }
+
+ public static GenericRecord getRecordWithExampleSchema(GenericRecord nestedColRecord) {
+ GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA));
+ record.put("timestamp", 4357686L);
+ record.put("_row_key", "key1");
+ record.put("ts_ms", "2020-03-21");
+ record.put("int_col", 10);
+ record.put("long_col", 100L);
+ if (nestedColRecord != null) {
+ record.put("nested_col", nestedColRecord);
+ }
+ return record;
+ }
+
+ public static GenericRecord getRecordWithExampleSchemaIntColAsLong(GenericRecord nestedColRecord) {
+ GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA_INT_COL_AS_LONG));
+ record.put("timestamp", 4357686L);
+ record.put("_row_key", "key1");
+ record.put("ts_ms", "2020-03-21");
+ record.put("int_col", 10L);
+ record.put("long_col", 100L);
+ if (nestedColRecord != null) {
+ record.put("nested_col", nestedColRecord);
+ }
+ return record;
+ }
+
+ public static GenericRecord getRecordWithExampleSchemaLongColAsInt(GenericRecord nestedColRecord) {
+ GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA_LONG_COL_AS_INT));
+ record.put("timestamp", 4357686L);
+ record.put("_row_key", "key1");
+ record.put("ts_ms", "2020-03-21");
+ record.put("int_col", 10);
+ record.put("long_col", 100);
+ if (nestedColRecord != null) {
+ record.put("nested_col", nestedColRecord);
+ }
+ return record;
+ }
+
+ public static GenericRecord getRecordWithExampleSchemaWithoutNestedCol() {
+ GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA_WITHOUT_NESTED_COL));
+ record.put("timestamp", 4357686L);
+ record.put("_row_key", "key1");
+ record.put("ts_ms", "2020-03-21");
+ record.put("int_col", 10);
+ record.put("long_col", 100L);
+ return record;
+ }
+
+}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/endTestEverything.json b/hudi-utilities/src/test/resources/data/schema-evolution/endTestEverything.json
new file mode 100644
index 0000000000000..d7845996f294e
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/endTestEverything.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567235019,"begin_lon":0.5594020723452937,"end_lat":0.7161653985102948,"end_lon":0.4971679897910298,"distance_in_meters":9361439213,"seconds_since_epoch":3794145268659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":16.671341480371346,"currency":"USD","zextra_col_nested":"yes"},"tip_history":[951],"_hoodie_is_deleted":false,"zextra_col":"yes","zcomplex_array":["a","b","c"],"_extra_col":"yes"}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.74714076296948563,"begin_lon":0.8776437421094859,"end_lat":0.9648524370765467,"end_lon":0.3911456321548304,"distance_in_meters":1137123412,"seconds_since_epoch":5028479681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":75.97606478430822,"currency":"USD","zextra_col_nested":"yes"},"tip_history":[138],"_hoodie_is_deleted":false,"zextra_col":"yes","zcomplex_array":["d"],"_extra_col":"yes"}
\ No newline at end of file
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/endTypePromotion.json b/hudi-utilities/src/test/resources/data/schema-evolution/endTypePromotion.json
new file mode 100644
index 0000000000000..68ea9cf6fde2c
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/endTypePromotion.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567235019,"begin_lon":0.5594020723452937,"end_lat":0.7161653985102948,"end_lon":0.4971679897910298,"distance_in_meters":9361439213,"seconds_since_epoch":3794145268659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":16.671341480371346,"currency":"USD"},"tip_history":[951],"_hoodie_is_deleted":false}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.74714076296948563,"begin_lon":0.8776437421094859,"end_lat":0.9648524370765467,"end_lon":0.3911456321548304,"distance_in_meters":1137123412,"seconds_since_epoch":5028479681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":75.97606478430822,"currency":"USD"},"tip_history":[138],"_hoodie_is_deleted":false}
\ No newline at end of file
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/endTypePromotionDropCols.json b/hudi-utilities/src/test/resources/data/schema-evolution/endTypePromotionDropCols.json
new file mode 100644
index 0000000000000..3694b22b4bead
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/endTypePromotionDropCols.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567235019,"begin_lon":0.5594020723452937,"end_lat":0.7161653985102948,"end_lon":0.4971679897910298,"distance_in_meters":9361439213,"seconds_since_epoch":3794145268659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":16.671341480371346},"tip_history":[951],"_hoodie_is_deleted":false}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","rider":"rider-003","driver":"driver-003","begin_lat":0.74714076296948563,"begin_lon":0.8776437421094859,"end_lat":0.9648524370765467,"end_lon":0.3911456321548304,"distance_in_meters":1137123412,"seconds_since_epoch":5028479681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":75.97606478430822},"tip_history":[138],"_hoodie_is_deleted":false}
\ No newline at end of file
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFiles.json b/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFiles.json
new file mode 100644
index 0000000000000..cf2d787644cc6
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFiles.json
@@ -0,0 +1,6 @@
+{"timestamp":1,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"one","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"one","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-001","driver":"driver-001","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"one","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD"},"tip_history":[{"amount":91.54707889420283,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"b7000dbd-d80f-4024-905d-532977ae43f9","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-001","driver":"driver-001","begin_lat":0.5931504793109675,"begin_lon":0.9886471058049089,"end_lat":0.006118306492296055,"end_lon":0.19266950151149498,"distance_in_meters":-1686525516,"seconds_since_epoch":4166715486945369394,"weight":0.8310657,"nation":"one","current_date":"1970-01-13","current_ts":1105887562,"height":0.557941,"city_to_state":{"LA":"CA"},"fare":{"amount":63.60969374104979,"currency":"USD"},"tip_history":[{"amount":87.00454921048154,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"07076280-5bab-4b0d-8930-94a1de5991cd","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.04245323335756779,"begin_lon":0.9152007089994821,"end_lat":0.6511125556291417,"end_lon":0.28444356863277487,"distance_in_meters":-480499072,"seconds_since_epoch":-4541489022232815692,"weight":0.8729432,"nation":"one","current_date":"1970-01-14","current_ts":1180252692,"height":0.321330,"city_to_state":{"LA":"CA"},"fare":{"amount":56.86865265269785,"currency":"USD"},"tip_history":[{"amount":30.2448146817467,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"d41c5703-6c86-4f4c-ab2c-51253b02deaf","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.5331332869796412,"begin_lon":0.11236032208831404,"end_lat":0.7610323238172235,"end_lon":0.6414706864249624,"distance_in_meters":1212983241,"seconds_since_epoch":7090335803227873266,"weight":0.40637594,"nation":"one","current_date":"1970-01-14","current_ts":1172551761,"height":0.183033,"city_to_state":{"LA":"CA"},"fare":{"amount":87.58991293970846,"currency":"USD"},"tip_history":[{"amount":11.69405524258501,"currency":"USD"}],"_hoodie_is_deleted":false}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFilesTestEverything.json b/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFilesTestEverything.json
new file mode 100644
index 0000000000000..85abab65788b0
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFilesTestEverything.json
@@ -0,0 +1,7 @@
+{"timestamp":1,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"zero","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD","extra_col_struct":1},"tip_history":[90],"extra_col_regular":1.5,"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"zero","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD","extra_col_struct":2},"tip_history":[13],"extra_col_regular":2.5,"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-001","driver":"driver-001","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"zero","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD","extra_col_struct":3},"tip_history":[91],"extra_col_regular":3.5,"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"b7000dbd-d80f-4024-905d-532977ae43f9","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-001","driver":"driver-001","begin_lat":0.5931504793109675,"begin_lon":0.9886471058049089,"end_lat":0.006118306492296055,"end_lon":0.19266950151149498,"distance_in_meters":-1686525516,"seconds_since_epoch":4166715486945369394,"weight":0.8310657,"nation":"zero","current_date":"1970-01-13","current_ts":1105887562,"height":0.557941,"city_to_state":{"LA":"CA"},"fare":{"amount":63.60969374104979,"currency":"USD","extra_col_struct":4},"tip_history":[87],"extra_col_regular":4.5,"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"07076280-5bab-4b0d-8930-94a1de5991cd","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.04245323335756779,"begin_lon":0.9152007089994821,"end_lat":0.6511125556291417,"end_lon":0.28444356863277487,"distance_in_meters":-480499072,"seconds_since_epoch":-4541489022232815692,"weight":0.8729432,"nation":"zero","current_date":"1970-01-14","current_ts":1180252692,"height":0.321330,"city_to_state":{"LA":"CA"},"fare":{"amount":56.86865265269785,"currency":"USD","extra_col_struct":5},"tip_history":[30],"extra_col_regular":5.5,"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"d41c5703-6c86-4f4c-ab2c-51253b02deaf","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.5331332869796412,"begin_lon":0.11236032208831404,"end_lat":0.7610323238172235,"end_lon":0.6414706864249624,"distance_in_meters":1212983241,"seconds_since_epoch":7090335803227873266,"weight":0.40637594,"nation":"zero","current_date":"1970-01-14","current_ts":1172551761,"height":0.183033,"city_to_state":{"LA":"CA"},"fare":{"amount":87.58991293970846,"currency":"USD","extra_col_struct":6},"tip_history":[11],"extra_col_regular":6.5,"_hoodie_is_deleted":false}
+
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFilesTypePromo.json b/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFilesTypePromo.json
new file mode 100644
index 0000000000000..09ab080ef75ef
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/extraLogFilesTypePromo.json
@@ -0,0 +1,7 @@
+{"timestamp":1,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"one","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[90],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"one","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[13],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-001","driver":"driver-001","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"one","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD"},"tip_history":[91],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"b7000dbd-d80f-4024-905d-532977ae43f9","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-001","driver":"driver-001","begin_lat":0.5931504793109675,"begin_lon":0.9886471058049089,"end_lat":0.006118306492296055,"end_lon":0.19266950151149498,"distance_in_meters":-1686525516,"seconds_since_epoch":4166715486945369394,"weight":0.8310657,"nation":"one","current_date":"1970-01-13","current_ts":1105887562,"height":0.557941,"city_to_state":{"LA":"CA"},"fare":{"amount":63.60969374104979,"currency":"USD"},"tip_history":[87],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"07076280-5bab-4b0d-8930-94a1de5991cd","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.04245323335756779,"begin_lon":0.9152007089994821,"end_lat":0.6511125556291417,"end_lon":0.28444356863277487,"distance_in_meters":-480499072,"seconds_since_epoch":-4541489022232815692,"weight":0.8729432,"nation":"one","current_date":"1970-01-14","current_ts":1180252692,"height":0.321330,"city_to_state":{"LA":"CA"},"fare":{"amount":56.86865265269785,"currency":"USD"},"tip_history":[30],"_hoodie_is_deleted":false}
+{"timestamp":1,"_row_key":"d41c5703-6c86-4f4c-ab2c-51253b02deaf","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-001","driver":"driver-001","begin_lat":0.5331332869796412,"begin_lon":0.11236032208831404,"end_lat":0.7610323238172235,"end_lon":0.6414706864249624,"distance_in_meters":1212983241,"seconds_since_epoch":7090335803227873266,"weight":0.40637594,"nation":"one","current_date":"1970-01-14","current_ts":1172551761,"height":0.183033,"city_to_state":{"LA":"CA"},"fare":{"amount":87.58991293970846,"currency":"USD"},"tip_history":[11],"_hoodie_is_deleted":false}
+
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroups.json b/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroups.json
new file mode 100644
index 0000000000000..76d31b785ce83
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroups.json
@@ -0,0 +1,3 @@
+{"timestamp":2,"_row_key":"bcea510f-aaf6-42f5-a490-c61b42f59784","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-002","driver":"driver-002","begin_lat":0.7362562672182036,"begin_lon":0.4745041047602002,"end_lat":0.22777332842138953,"end_lon":0.10094789978439622,"distance_in_meters":60306142,"seconds_since_epoch":5390769490275546019,"weight":0.9655821,"nation":"two","current_date":"1970-01-12","current_ts":982643754,"height":0.982110,"city_to_state":{"LA":"CA"},"fare":{"amount":70.10088696225361,"currency":"USD"},"tip_history":[{"amount":96.79449667264703,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":2,"_row_key":"ad5ab2be-769a-4c7b-98af-e2780d016a9c","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-002","driver":"driver-002","begin_lat":0.5390219572718705,"begin_lon":0.08683108180272892,"end_lat":0.7835345528085245,"end_lon":0.695364227220298,"distance_in_meters":1746406037,"seconds_since_epoch":-1859359059343187038,"weight":0.7024137,"nation":"two","current_date":"1970-01-16","current_ts":1356858937,"height":0.189173,"city_to_state":{"LA":"CA"},"fare":{"amount":29.865323585321068,"currency":"USD"},"tip_history":[{"amount":19.760372723830354,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":2,"_row_key":"6c8b77e5-7806-43f1-9ecc-706a999d49fe","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-002","driver":"driver-002","begin_lat":0.5347242863334416,"begin_lon":0.03138005638340591,"end_lat":0.6037366738340498,"end_lon":0.49273899834224566,"distance_in_meters":-1370828602,"seconds_since_epoch":-4712777615466527378,"weight":0.580827,"nation":"two","current_date":"1970-01-12","current_ts":1009523468,"height":0.624823,"city_to_state":{"LA":"CA"},"fare":{"amount":71.77332900090153,"currency":"USD"},"tip_history":[{"amount":7.720702671399637,"currency":"USD"}],"_hoodie_is_deleted":false}
\ No newline at end of file
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroupsTestEverything.json b/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroupsTestEverything.json
new file mode 100644
index 0000000000000..61fb77f47888c
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroupsTestEverything.json
@@ -0,0 +1,3 @@
+{"timestamp":2,"_row_key":"bcea510f-aaf6-42f5-a490-c61b42f59784","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-002","driver":"driver-002","begin_lat":0.7362562672182036,"begin_lon":0.4745041047602002,"end_lat":0.22777332842138953,"end_lon":0.10094789978439622,"distance_in_meters":60306142,"seconds_since_epoch":5390769490275546019,"weight":0.9655821,"nation":"two","current_date":"1970-01-12","current_ts":982643754,"height":0.982110,"city_to_state":{"LA":"CA"},"fare":{"amount":70.10088696225361,"currency":"USD","extra_col_struct":7},"tip_history":[96],"extra_col_regular":7.5,"_hoodie_is_deleted":false}
+{"timestamp":2,"_row_key":"ad5ab2be-769a-4c7b-98af-e2780d016a9c","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-002","driver":"driver-002","begin_lat":0.5390219572718705,"begin_lon":0.08683108180272892,"end_lat":0.7835345528085245,"end_lon":0.695364227220298,"distance_in_meters":1746406037,"seconds_since_epoch":-1859359059343187038,"weight":0.7024137,"nation":"two","current_date":"1970-01-16","current_ts":1356858937,"height":0.189173,"city_to_state":{"LA":"CA"},"fare":{"amount":29.865323585321068,"currency":"USD","extra_col_struct":8},"tip_history":[19],"extra_col_regular":8.5,"_hoodie_is_deleted":false}
+{"timestamp":2,"_row_key":"6c8b77e5-7806-43f1-9ecc-706a999d49fe","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-002","driver":"driver-002","begin_lat":0.5347242863334416,"begin_lon":0.03138005638340591,"end_lat":0.6037366738340498,"end_lon":0.49273899834224566,"distance_in_meters":-1370828602,"seconds_since_epoch":-4712777615466527378,"weight":0.580827,"nation":"two","current_date":"1970-01-12","current_ts":1009523468,"height":0.624823,"city_to_state":{"LA":"CA"},"fare":{"amount":71.77332900090153,"currency":"USD","extra_col_struct":9},"tip_history":[7],"extra_col_regular":9.5,"_hoodie_is_deleted":false}
\ No newline at end of file
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroupsTypePromo.json b/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroupsTypePromo.json
new file mode 100644
index 0000000000000..d0f4ef1657ceb
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/newFileGroupsTypePromo.json
@@ -0,0 +1,3 @@
+{"timestamp":2,"_row_key":"bcea510f-aaf6-42f5-a490-c61b42f59784","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-002","driver":"driver-002","begin_lat":0.7362562672182036,"begin_lon":0.4745041047602002,"end_lat":0.22777332842138953,"end_lon":0.10094789978439622,"distance_in_meters":60306142,"seconds_since_epoch":5390769490275546019,"weight":0.9655821,"nation":"two","current_date":"1970-01-12","current_ts":982643754,"height":0.982110,"city_to_state":{"LA":"CA"},"fare":{"amount":70.10088696225361,"currency":"USD"},"tip_history":[96],"_hoodie_is_deleted":false}
+{"timestamp":2,"_row_key":"ad5ab2be-769a-4c7b-98af-e2780d016a9c","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-002","driver":"driver-002","begin_lat":0.5390219572718705,"begin_lon":0.08683108180272892,"end_lat":0.7835345528085245,"end_lon":0.695364227220298,"distance_in_meters":1746406037,"seconds_since_epoch":-1859359059343187038,"weight":0.7024137,"nation":"two","current_date":"1970-01-16","current_ts":1356858937,"height":0.189173,"city_to_state":{"LA":"CA"},"fare":{"amount":29.865323585321068,"currency":"USD"},"tip_history":[19],"_hoodie_is_deleted":false}
+{"timestamp":2,"_row_key":"6c8b77e5-7806-43f1-9ecc-706a999d49fe","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-002","driver":"driver-002","begin_lat":0.5347242863334416,"begin_lon":0.03138005638340591,"end_lat":0.6037366738340498,"end_lon":0.49273899834224566,"distance_in_meters":-1370828602,"seconds_since_epoch":-4712777615466527378,"weight":0.580827,"nation":"two","current_date":"1970-01-12","current_ts":1009523468,"height":0.624823,"city_to_state":{"LA":"CA"},"fare":{"amount":71.77332900090153,"currency":"USD"},"tip_history":[7],"_hoodie_is_deleted":false}
\ No newline at end of file
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/plain.json b/hudi-utilities/src/test/resources/data/schema-evolution/plain.json
new file mode 100644
index 0000000000000..5a1f85f9ea36d
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/plain.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/start.json b/hudi-utilities/src/test/resources/data/schema-evolution/start.json
new file mode 100644
index 0000000000000..bad4edbb6a1c3
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/start.json
@@ -0,0 +1,6 @@
+{"timestamp":0,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"zero","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"zero","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-000","driver":"driver-000","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"zero","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD"},"tip_history":[{"amount":91.54707889420283,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"b7000dbd-d80f-4024-905d-532977ae43f9","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-000","driver":"driver-000","begin_lat":0.5931504793109675,"begin_lon":0.9886471058049089,"end_lat":0.006118306492296055,"end_lon":0.19266950151149498,"distance_in_meters":-1686525516,"seconds_since_epoch":4166715486945369394,"weight":0.8310657,"nation":"zero","current_date":"1970-01-13","current_ts":1105887562,"height":0.557941,"city_to_state":{"LA":"CA"},"fare":{"amount":63.60969374104979,"currency":"USD"},"tip_history":[{"amount":87.00454921048154,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"07076280-5bab-4b0d-8930-94a1de5991cd","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.04245323335756779,"begin_lon":0.9152007089994821,"end_lat":0.6511125556291417,"end_lon":0.28444356863277487,"distance_in_meters":-480499072,"seconds_since_epoch":-4541489022232815692,"weight":0.8729432,"nation":"zero","current_date":"1970-01-14","current_ts":1180252692,"height":0.321330,"city_to_state":{"LA":"CA"},"fare":{"amount":56.86865265269785,"currency":"USD"},"tip_history":[{"amount":30.2448146817467,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"d41c5703-6c86-4f4c-ab2c-51253b02deaf","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.5331332869796412,"begin_lon":0.11236032208831404,"end_lat":0.7610323238172235,"end_lon":0.6414706864249624,"distance_in_meters":1212983241,"seconds_since_epoch":7090335803227873266,"weight":0.40637594,"nation":"zero","current_date":"1970-01-14","current_ts":1172551761,"height":0.183033,"city_to_state":{"LA":"CA"},"fare":{"amount":87.58991293970846,"currency":"USD"},"tip_history":[{"amount":11.69405524258501,"currency":"USD"}],"_hoodie_is_deleted":false}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/startTestEverything.json b/hudi-utilities/src/test/resources/data/schema-evolution/startTestEverything.json
new file mode 100644
index 0000000000000..ac1486b9783e1
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/startTestEverything.json
@@ -0,0 +1,7 @@
+{"timestamp":0,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"zero","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD","extra_col_struct":1},"tip_history":[90],"extra_col_regular":1.5,"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"zero","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD","extra_col_struct":2},"tip_history":[13],"extra_col_regular":2.5,"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-000","driver":"driver-000","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"zero","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD","extra_col_struct":3},"tip_history":[91],"extra_col_regular":3.5,"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"b7000dbd-d80f-4024-905d-532977ae43f9","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-000","driver":"driver-000","begin_lat":0.5931504793109675,"begin_lon":0.9886471058049089,"end_lat":0.006118306492296055,"end_lon":0.19266950151149498,"distance_in_meters":-1686525516,"seconds_since_epoch":4166715486945369394,"weight":0.8310657,"nation":"zero","current_date":"1970-01-13","current_ts":1105887562,"height":0.557941,"city_to_state":{"LA":"CA"},"fare":{"amount":63.60969374104979,"currency":"USD","extra_col_struct":4},"tip_history":[87],"extra_col_regular":4.5,"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"07076280-5bab-4b0d-8930-94a1de5991cd","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.04245323335756779,"begin_lon":0.9152007089994821,"end_lat":0.6511125556291417,"end_lon":0.28444356863277487,"distance_in_meters":-480499072,"seconds_since_epoch":-4541489022232815692,"weight":0.8729432,"nation":"zero","current_date":"1970-01-14","current_ts":1180252692,"height":0.321330,"city_to_state":{"LA":"CA"},"fare":{"amount":56.86865265269785,"currency":"USD","extra_col_struct":5},"tip_history":[30],"extra_col_regular":5.5,"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"d41c5703-6c86-4f4c-ab2c-51253b02deaf","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.5331332869796412,"begin_lon":0.11236032208831404,"end_lat":0.7610323238172235,"end_lon":0.6414706864249624,"distance_in_meters":1212983241,"seconds_since_epoch":7090335803227873266,"weight":0.40637594,"nation":"zero","current_date":"1970-01-14","current_ts":1172551761,"height":0.183033,"city_to_state":{"LA":"CA"},"fare":{"amount":87.58991293970846,"currency":"USD","extra_col_struct":6},"tip_history":[11],"extra_col_regular":6.5,"_hoodie_is_deleted":false}
+
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/startTypePromotion.json b/hudi-utilities/src/test/resources/data/schema-evolution/startTypePromotion.json
new file mode 100644
index 0000000000000..d4fddb55282fa
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/startTypePromotion.json
@@ -0,0 +1,7 @@
+{"timestamp":0,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"zero","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[90],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"zero","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[13],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-000","driver":"driver-000","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"zero","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD"},"tip_history":[91],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"b7000dbd-d80f-4024-905d-532977ae43f9","partition_path":"2016/03/15","trip_type":"UBERX","rider":"rider-000","driver":"driver-000","begin_lat":0.5931504793109675,"begin_lon":0.9886471058049089,"end_lat":0.006118306492296055,"end_lon":0.19266950151149498,"distance_in_meters":-1686525516,"seconds_since_epoch":4166715486945369394,"weight":0.8310657,"nation":"zero","current_date":"1970-01-13","current_ts":1105887562,"height":0.557941,"city_to_state":{"LA":"CA"},"fare":{"amount":63.60969374104979,"currency":"USD"},"tip_history":[87],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"07076280-5bab-4b0d-8930-94a1de5991cd","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.04245323335756779,"begin_lon":0.9152007089994821,"end_lat":0.6511125556291417,"end_lon":0.28444356863277487,"distance_in_meters":-480499072,"seconds_since_epoch":-4541489022232815692,"weight":0.8729432,"nation":"zero","current_date":"1970-01-14","current_ts":1180252692,"height":0.321330,"city_to_state":{"LA":"CA"},"fare":{"amount":56.86865265269785,"currency":"USD"},"tip_history":[30],"_hoodie_is_deleted":false}
+{"timestamp":0,"_row_key":"d41c5703-6c86-4f4c-ab2c-51253b02deaf","partition_path":"2015/03/17","trip_type":"BLACK","rider":"rider-000","driver":"driver-000","begin_lat":0.5331332869796412,"begin_lon":0.11236032208831404,"end_lat":0.7610323238172235,"end_lon":0.6414706864249624,"distance_in_meters":1212983241,"seconds_since_epoch":7090335803227873266,"weight":0.40637594,"nation":"zero","current_date":"1970-01-14","current_ts":1172551761,"height":0.183033,"city_to_state":{"LA":"CA"},"fare":{"amount":87.58991293970846,"currency":"USD"},"tip_history":[11],"_hoodie_is_deleted":false}
+
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddAndDropCols.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddAndDropCols.json
new file mode 100644
index 0000000000000..d966adf2b6e97
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddAndDropCols.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"zextra_col_nest":"yes"}],"_hoodie_is_deleted":false,"zextra_col":"yes"}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"zextra_col_nest":"yes"}],"_hoodie_is_deleted":false,"zextra_col":"yes"}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddColChangeOrderAllFiles.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColChangeOrderAllFiles.json
new file mode 100644
index 0000000000000..8a92bb8198826
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColChangeOrderAllFiles.json
@@ -0,0 +1,3 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false,"extra_col":"yes"}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false,"extra_col":"yes"}
+{"timestamp":3,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-003","driver":"driver-003","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"three","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD"},"tip_history":[{"amount":91.54707889420283,"currency":"USD"}],"_hoodie_is_deleted":false,"extra_col":"no"}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddColChangeOrderSomeFiles.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColChangeOrderSomeFiles.json
new file mode 100644
index 0000000000000..612f6018c5ce4
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColChangeOrderSomeFiles.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false,"extra_col":"yes"}
+{"timestamp":3,"_row_key":"1f7f4473-8889-488a-86f8-aaa63319b4b4","partition_path":"2015/03/17","trip_type":"UBERX","rider":"rider-003","driver":"driver-003","begin_lat":0.09283534365767165,"begin_lon":0.7406047279761032,"end_lat":0.259529402287365,"end_lon":0.3793829234810173,"distance_in_meters":-1289053159,"seconds_since_epoch":6540247735540261975,"weight":0.74709326,"nation":"three","current_date":"1970-01-16","current_ts":1338290882,"height":0.474291,"city_to_state":{"LA":"CA"},"fare":{"amount":41.8217733941428,"currency":"USD"},"tip_history":[{"amount":91.54707889420283,"currency":"USD"}],"_hoodie_is_deleted":false,"extra_col":"no"}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddColRoot.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColRoot.json
new file mode 100644
index 0000000000000..e17e47eb302ec
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColRoot.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false,"zextra_col":"yes"}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false,"zextra_col":"yes"}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddColStruct.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColStruct.json
new file mode 100644
index 0000000000000..8def81033d1d2
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddColStruct.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD","zextra_col":"yes"}],"_hoodie_is_deleted":false}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD","zextra_col":"yes"}],"_hoodie_is_deleted":false}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddComplexField.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddComplexField.json
new file mode 100644
index 0000000000000..44ded6f8f0c87
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddComplexField.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false,"zcomplex_array":["a","b","c"]}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false,"zcomplex_array":["d"]}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testAddMetaCol.json b/hudi-utilities/src/test/resources/data/schema-evolution/testAddMetaCol.json
new file mode 100644
index 0000000000000..b005e6c8f3bf8
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testAddMetaCol.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false,"_extra_col":"yes"}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false,"_extra_col":"yes"}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testDropColRoot.json b/hudi-utilities/src/test/resources/data/schema-evolution/testDropColRoot.json
new file mode 100644
index 0000000000000..6d3d8f011ecea
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testDropColRoot.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568,"currency":"USD"}],"_hoodie_is_deleted":false}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558,"currency":"USD"}],"_hoodie_is_deleted":false}
diff --git a/hudi-utilities/src/test/resources/data/schema-evolution/testDropColStruct.json b/hudi-utilities/src/test/resources/data/schema-evolution/testDropColStruct.json
new file mode 100644
index 0000000000000..bcfee99ed7804
--- /dev/null
+++ b/hudi-utilities/src/test/resources/data/schema-evolution/testDropColStruct.json
@@ -0,0 +1,2 @@
+{"timestamp":3,"_row_key":"154fee81-6e2a-4c32-94f5-be5c456fdd0a","partition_path":"2016/03/15","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.21927838567558522,"begin_lon":0.5594020723099724,"end_lat":0.7161653985926594,"end_lon":0.49716798979953447,"distance_in_meters":936143957,"seconds_since_epoch":3794105168659998336,"weight":0.18520206,"nation":"three","current_date":"1970-01-15","current_ts":1244853103,"height":0.272661,"city_to_state":{"LA":"CA"},"fare":{"amount":12.671341480371346,"currency":"USD"},"tip_history":[{"amount":90.26735894145568}],"_hoodie_is_deleted":false}
+{"timestamp":3,"_row_key":"c8c1bd1a-d58b-46c6-a38b-79a2a610c956","partition_path":"2015/03/16","trip_type":"BLACK","rider":"rider-003","driver":"driver-003","begin_lat":0.7471407629318884,"begin_lon":0.8776437421395643,"end_lat":0.9648524370990681,"end_lon":0.3911456751705831,"distance_in_meters":1137109733,"seconds_since_epoch":5028439681953251637,"weight":0.023411155,"nation":"three","current_date":"1970-01-12","current_ts":986645693,"height":0.898042,"city_to_state":{"LA":"CA"},"fare":{"amount":85.97606478430822,"currency":"USD"},"tip_history":[{"amount":13.7534224373558}],"_hoodie_is_deleted":false}
diff --git a/hudi-utilities/src/test/resources/streamer-config/source_evolved.avsc b/hudi-utilities/src/test/resources/streamer-config/source_evolved.avsc
index dba040d352557..9571b4886f83e 100644
--- a/hudi-utilities/src/test/resources/streamer-config/source_evolved.avsc
+++ b/hudi-utilities/src/test/resources/streamer-config/source_evolved.avsc
@@ -90,7 +90,8 @@
"name": "height",
"type": {
"type": "fixed",
- "name": "abc",
+ "name": "fixed",
+ "namespace": "triprec.height",
"size": 5,
"logicalType": "decimal",
"precision": 10,
@@ -143,8 +144,7 @@
},
{
"name": "_hoodie_is_deleted",
- "type": "boolean",
- "default": false
+ "type": "boolean"
},
{
"name": "evoluted_optional_union_field",