diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 9bad2e3486e7f..9358d14ebeb15 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -48,6 +48,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; @@ -68,6 +69,13 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.action.TableChange; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; +import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager; +import org.apache.hudi.internal.schema.utils.SchemaChangePersistHelper; +import org.apache.hudi.internal.schema.utils.SerDeHelper; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.BulkInsertPartitioner; @@ -78,6 +86,8 @@ import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; import org.apache.hudi.table.upgrade.UpgradeDowngrade; + +import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -1409,4 +1419,138 @@ private void tryUpgrade(HoodieTableMetaClient metaClient, Option instant metaClient.reloadActiveTimeline(); } } + + /** + * add columns to table. + * + * @param colName col name to be added. if we want to add col to a nested filed, the fullName should be specify + * @param schema col type to be added. + * @param doc col doc to be added. + * @param position col position to be added + * @param positionType col position change type. now support three change types: first/after/before + */ + public void addColumn(String colName, Schema schema, String doc, String position, TableChange.ColumnPositionChange.ColumnPositionType positionType) { + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper + .applyAddChange(pair.getLeft(), colName, AvroInternalSchemaConverter.convertToField(schema), doc, position, positionType); + commitTableChange(newSchema, pair.getRight()); + } + + public void addColumn(String colName, Schema schema) { + addColumn(colName, schema, null, "", TableChange.ColumnPositionChange.ColumnPositionType.NO_OPERATION); + } + + /** + * delete columns to table. + * + * @param colNames col name to be deleted. if we want to delete col from a nested filed, the fullName should be specify + */ + public void deleteColumns(String... colNames) { + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper.applyDeleteChange(pair.getLeft(), colNames); + commitTableChange(newSchema, pair.getRight()); + } + + /** + * rename col name for hudi table. + * + * @param colName col name to be renamed. if we want to rename col from a nested filed, the fullName should be specify + * @param newName new name for current col. no need to specify fullName. + */ + public void renameColumn(String colName, String newName) { + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper.applyRenameChange(pair.getLeft(), colName, newName); + commitTableChange(newSchema, pair.getRight()); + } + + /** + * update col nullable attribute for hudi table. + * + * @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify + * @param nullable . + */ + public void updateColumnNullability(String colName, boolean nullable) { + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper.applyColumnNullabilityChange(pair.getLeft(), colName, nullable); + commitTableChange(newSchema, pair.getRight()); + } + + /** + * update col Type for hudi table. + * only support update primitive type to primitive type. + * cannot update nest type to nest type or primitive type eg: RecordType -> MapType, MapType -> LongType. + * + * @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify + * @param newType . + */ + public void updateColumnType(String colName, Type newType) { + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper.applyColumnTypeChange(pair.getLeft(), colName, newType); + commitTableChange(newSchema, pair.getRight()); + } + + /** + * update col comment for hudi table. + * + * @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify + * @param doc . + */ + public void updateColumnComment(String colName, String doc) { + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper.applyColumnCommentChange(pair.getLeft(), colName, doc); + commitTableChange(newSchema, pair.getRight()); + } + + /** + * reorder the position of col. + * + * @param colName column which need to be reordered. if we want to change col from a nested filed, the fullName should be specify. + * @param referColName reference position. + * @param orderType col position change type. now support three change types: first/after/before + */ + public void reOrderColPosition(String colName, String referColName, TableChange.ColumnPositionChange.ColumnPositionType orderType) { + if (colName == null || orderType == null || referColName == null) { + return; + } + //get internalSchema + Pair pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = SchemaChangePersistHelper + .applyReOrderColPositionChange(pair.getLeft(), colName, referColName, orderType); + commitTableChange(newSchema, pair.getRight()); + } + + private Pair getInternalSchemaAndMetaClient() { + HoodieTableMetaClient metaClient = createMetaClient(true); + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + Option internalSchemaOption = schemaUtil.getTableInternalSchemaFromCommitMetadata(); + if (!internalSchemaOption.isPresent()) { + throw new HoodieException(String.format("cannot find schema for current table: %s", config.getBasePath())); + } + return Pair.of(internalSchemaOption.get(), metaClient); + } + + private void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient metaClient) { + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + String historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata().orElse(""); + Schema schema = AvroInternalSchemaConverter.convert(newSchema, config.getTableName()); + String commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType()); + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + startCommitWithTime(instantTime, commitActionType, metaClient); + config.setSchema(schema.toString()); + HoodieActiveTimeline timeLine = metaClient.getActiveTimeline(); + HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + metadata.setOperationType(WriteOperationType.ALTER_SCHEMA); + try { + timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException io) { + throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); + } + Map extraMeta = new HashMap<>(); + extraMeta.put(SerDeHelper.LATESTSCHEMA, SerDeHelper.toJson(newSchema.setSchemaId(Long.getLong(instantTime)))); + // try to save history schemas + FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient); + schemasManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(newSchema, historySchemaStr)); + commitStats(instantTime, Collections.EMPTY_LIST, Option.of(extraMeta), commitActionType); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 15401c0292e14..7d522371572c7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -50,6 +50,7 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkers; @@ -550,6 +551,9 @@ public void archive(HoodieEngineContext context, List instants) t } } writeToFile(wrapperSchema, records); + // try to clean old history schema. + FileBasedInternalSchemaStorageManager fss = new FileBasedInternalSchemaStorageManager(metaClient); + fss.cleanOldFiles(instants.stream().map(is -> is.getTimestamp()).collect(Collectors.toList())); } catch (Exception e) { throw new HoodieCommitException("Failed to archive commits", e); } 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 b5a3cc002366e..bdb3ae1194f65 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 @@ -48,6 +48,8 @@ public enum WriteOperationType { INSERT_OVERWRITE_TABLE("insert_overwrite_table"), // compact COMPACT("compact"), + // alter schema + ALTER_SCHEMA("alter_schema"), // used for old version UNKNOWN("unknown"); @@ -86,6 +88,8 @@ public static WriteOperationType fromValue(String value) { return CLUSTER; case "compact": return COMPACT; + case "alter_schema": + return ALTER_SCHEMA; case "unknown": return UNKNOWN; default: diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 3de93005da66d..f368c0cb89daa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -90,6 +90,8 @@ public class HoodieTableMetaClient implements Serializable { public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR + ".fileids"; + public static final String SCHEMA_FOLDER_NAME = ".schema"; + public static final String MARKER_EXTN = ".marker"; private String basePath; @@ -192,6 +194,13 @@ public String getColumnStatsIndexPath() { return new Path(metaPath, COLUMN_STATISTICS_INDEX_NAME).toString(); } + /** + * @return schema folder path + */ + public String getSchemaFolderName() { + return new Path(metaPath, SCHEMA_FOLDER_NAME).toString(); + } + /** * @return Temp Folder path */ 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 a0771d1249c75..dbea4318b5db2 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 @@ -18,13 +18,6 @@ package org.apache.hudi.common.table; -import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; -import org.apache.avro.SchemaCompatibility; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; @@ -44,8 +37,18 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.InvalidTableException; import org.apache.hudi.io.storage.HoodieHFileReader; - import org.apache.hudi.io.storage.HoodieOrcReader; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager; +import org.apache.hudi.internal.schema.utils.SerDeHelper; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.SchemaCompatibility; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroSchemaConverter; @@ -537,4 +540,51 @@ private boolean hasOperationField() { return false; } } + + /** + * Gets the InternalSchema for a hoodie table from the HoodieCommitMetadata of the instant. + * + * @return InternalSchema for this table + */ + public Option getTableInternalSchemaFromCommitMetadata() { + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + if (timeline.lastInstant().isPresent()) { + return getTableInternalSchemaFromCommitMetadata(timeline.lastInstant().get()); + } else { + return Option.empty(); + } + } + + /** + * Gets the InternalSchema for a hoodie table from the HoodieCommitMetadata of the instant. + * + * @return InternalSchema for this table + */ + private Option getTableInternalSchemaFromCommitMetadata(HoodieInstant instant) { + try { + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + byte[] data = timeline.getInstantDetails(instant).get(); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATESTSCHEMA); + if (latestInternalSchemaStr != null) { + return SerDeHelper.fromJson(latestInternalSchemaStr); + } else { + return Option.empty(); + } + } catch (Exception e) { + throw new HoodieException("Failed to read schema from commit metadata", e); + } + } + + /** + * Gets the history schemas as String for a hoodie table from the HoodieCommitMetadata of the instant. + * + * @return history schemas string for this table + */ + public Option getTableHistorySchemaStrFromCommitMetadata() { + // now we only support FileBaseInternalSchemaManager + FileBasedInternalSchemaStorageManager manager = new FileBasedInternalSchemaStorageManager(metaClient); + String result = manager.getHistorySchemaStr(); + return result.isEmpty() ? Option.empty() : Option.of(result); + } } 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 36dd5368d4a63..27fa630922b35 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 @@ -73,7 +73,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, - REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION)); + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, + REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION)); private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class); protected HoodieTableMetaClient metaClient; @@ -222,7 +223,7 @@ public void deleteCompactionRequested(HoodieInstant instant) { private void deleteInstantFileIfExists(HoodieInstant instant) { LOG.info("Deleting instant " + instant); - Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName()); + Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName()); try { if (metaClient.getFs().exists(inFlightCommitFilePath)) { boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false); @@ -241,7 +242,7 @@ private void deleteInstantFileIfExists(HoodieInstant instant) { private void deleteInstantFile(HoodieInstant instant) { LOG.info("Deleting instant " + instant); - Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName()); + Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName()); try { boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false); if (result) { @@ -256,7 +257,7 @@ private void deleteInstantFile(HoodieInstant instant) { @Override public Option getInstantDetails(HoodieInstant instant) { - Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName()); + Path detailPath = getInstantFileNamePath(instant.getFileName()); return readDataFromPath(detailPath); } @@ -302,12 +303,12 @@ public Option> getLastCommitMetadataWi public Option readCleanerInfoAsBytes(HoodieInstant instant) { // Cleaner metadata are always stored only in timeline .hoodie - return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName())); + return readDataFromPath(getInstantFileNamePath(instant.getFileName())); } public Option readRollbackInfoAsBytes(HoodieInstant instant) { // Rollback metadata are always stored only in timeline .hoodie - return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName())); + return readDataFromPath(getInstantFileNamePath(instant.getFileName())); } public Option readRestoreInfoAsBytes(HoodieInstant instant) { @@ -533,24 +534,23 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, if (metaClient.getTimelineLayoutVersion().isNullVersion()) { // Re-create the .inflight file by opening a new file and write the commit metadata in createFileInMetaPath(fromInstant.getFileName(), data, allowRedundantTransitions); - Path fromInstantPath = new Path(metaClient.getMetaPath(), fromInstant.getFileName()); - Path toInstantPath = new Path(metaClient.getMetaPath(), toInstant.getFileName()); + Path fromInstantPath = getInstantFileNamePath(fromInstant.getFileName()); + Path toInstantPath = getInstantFileNamePath(toInstant.getFileName()); boolean success = metaClient.getFs().rename(fromInstantPath, toInstantPath); if (!success) { throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath); } } else { // Ensures old state exists in timeline - LOG.info("Checking for file exists ?" + new Path(metaClient.getMetaPath(), fromInstant.getFileName())); - ValidationUtils.checkArgument(metaClient.getFs().exists(new Path(metaClient.getMetaPath(), - fromInstant.getFileName()))); + LOG.info("Checking for file exists ?" + getInstantFileNamePath(fromInstant.getFileName())); + ValidationUtils.checkArgument(metaClient.getFs().exists(getInstantFileNamePath(fromInstant.getFileName()))); // Use Write Once to create Target File if (allowRedundantTransitions) { - FileIOUtils.createFileInPath(metaClient.getFs(), new Path(metaClient.getMetaPath(), toInstant.getFileName()), data); + FileIOUtils.createFileInPath(metaClient.getFs(), getInstantFileNamePath(toInstant.getFileName()), data); } else { - createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data); + createImmutableFileInPath(getInstantFileNamePath(toInstant.getFileName()), data); } - LOG.info("Create new file for toInstant ?" + new Path(metaClient.getMetaPath(), toInstant.getFileName())); + LOG.info("Create new file for toInstant ?" + getInstantFileNamePath(toInstant.getFileName())); } } catch (IOException e) { throw new HoodieIOException("Could not complete " + fromInstant, e); @@ -559,8 +559,8 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { ValidationUtils.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp())); - Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName()); - Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); + Path inFlightCommitFilePath = getInstantFileNamePath(inflight.getFileName()); + Path commitFilePath = getInstantFileNamePath(completed.getFileName()); try { if (metaClient.getTimelineLayoutVersion().isNullVersion()) { if (!metaClient.getFs().exists(inFlightCommitFilePath)) { @@ -571,8 +571,8 @@ private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inf } } } else { - Path requestedInstantFilePath = new Path(metaClient.getMetaPath(), - new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()).getFileName()); + Path requestedInstantFilePath = getInstantFileNamePath(new HoodieInstant(State.REQUESTED, + inflight.getAction(), inflight.getTimestamp()).getFileName()); // If inflight and requested files do not exist, create one if (!metaClient.getFs().exists(requestedInstantFilePath)) { @@ -591,6 +591,10 @@ private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inf } } + private Path getInstantFileNamePath(String fileName) { + return new Path(fileName.contains(SAVE_SCHEMA_ACTION) ? metaClient.getSchemaFolderName() : metaClient.getMetaPath(), fileName); + } + public void transitionRequestedToInflight(String commitType, String inFlightInstant) { HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant); transitionRequestedToInflight(requested, Option.empty(), false); @@ -648,7 +652,7 @@ public void saveToRestoreRequested(HoodieInstant instant, Option content } private void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { - Path fullPath = new Path(metaClient.getMetaPath(), filename); + Path fullPath = getInstantFileNamePath(filename); if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { FileIOUtils.createFileInPath(metaClient.getFs(), fullPath, content); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index 9cd0883126495..ea434b32bf2a7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -172,6 +172,10 @@ public String getFileName() { return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp) : isRequested() ? HoodieTimeline.makeRequestedReplaceFileName(timestamp) : HoodieTimeline.makeReplaceFileName(timestamp); + } else if (HoodieTimeline.SAVE_SCHEMA_ACTION.equals(action)) { + return isInflight() ? HoodieTimeline.makeInflightSchemaFileName(timestamp) + : isRequested() ? HoodieTimeline.makeRequestSchemaFileName(timestamp) + : HoodieTimeline.makeSchemaFileName(timestamp); } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index 25b9c2ec6f2e4..37a2c0967e025 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -55,6 +55,8 @@ public interface HoodieTimeline extends Serializable { String COMPACTION_ACTION = "compaction"; String REQUESTED_EXTENSION = ".requested"; String RESTORE_ACTION = "restore"; + // only for schema save + String SAVE_SCHEMA_ACTION = "schemacommit"; String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION, @@ -84,6 +86,10 @@ public interface HoodieTimeline extends Serializable { String INFLIGHT_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + INFLIGHT_EXTENSION; String REQUESTED_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + REQUESTED_EXTENSION; String REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION; + String SAVE_SCHEMA_ACTION_EXTENSION = "." + SAVE_SCHEMA_ACTION; + String INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION = "." + SAVE_SCHEMA_ACTION + INFLIGHT_EXTENSION; + String REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION = "." + SAVE_SCHEMA_ACTION + REQUESTED_EXTENSION; + String INVALID_INSTANT_TS = "0"; @@ -454,4 +460,16 @@ static String makeFileNameAsComplete(String fileName) { static String makeFileNameAsInflight(String fileName) { return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION); } + + static String makeSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION); + } + + static String makeInflightSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION); + } + + static String makeRequestSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/HoodieSchemaException.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/HoodieSchemaException.java new file mode 100644 index 0000000000000..7fdafc5238d10 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/HoodieSchemaException.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema; + +import org.apache.hudi.exception.HoodieException; + +/** + * Exception thrown for Hoodie schema convert failures. The root of the exception hierarchy. + * Hoodie Write/Read clients will throw this exception if any of its operations fail. This is a runtime (unchecked) + * exception. + */ +public class HoodieSchemaException extends HoodieException { + public HoodieSchemaException() { + super(); + } + + public HoodieSchemaException(String message) { + super(message); + } + + public HoodieSchemaException(String message, Throwable t) { + super(message, t); + } + + public HoodieSchemaException(Throwable t) { + super(t); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java new file mode 100644 index 0000000000000..bdcc82b0658a5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.internal.schema.Types.RecordType; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Internal schema for hudi table. + * used to support schema evolution. + */ +public class InternalSchema implements Serializable { + + private static final long DEFAULT_VERSION_ID = 0; + + private final RecordType record; + + private int maxColumnId; + private long versionId; + + private transient Map idToField = null; + private transient Map nameToId = null; + private transient Map idToName = null; + + public InternalSchema(List columns) { + this(DEFAULT_VERSION_ID, columns); + } + + public InternalSchema(Field... columns) { + this(DEFAULT_VERSION_ID, Arrays.asList(columns)); + } + + public InternalSchema(long versionId, List cols) { + this.versionId = versionId; + this.record = RecordType.get(cols); + buildIdToName(); + maxColumnId = idToName.keySet().stream().max(Comparator.comparing(Integer::valueOf)).get(); + } + + public InternalSchema(long versionId, int maxColumnId, List cols) { + this.maxColumnId = maxColumnId; + this.versionId = versionId; + this.record = RecordType.get(cols); + buildIdToName(); + } + + public InternalSchema(long versionId, int maxColumnId, Field... cols) { + this(versionId, maxColumnId, Arrays.asList(cols)); + } + + public RecordType getRecord() { + return record; + } + + private Map buildIdToName() { + if (idToName == null) { + idToName = InternalSchemaUtils.buildIdToName(record); + } + return idToName; + } + + private Map buildNameToId() { + if (nameToId == null) { + if (idToName != null && !idToName.isEmpty()) { + nameToId = idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + return nameToId; + } + nameToId = InternalSchemaUtils.buildNameToId(record); + } + return nameToId; + } + + private Map buildIdToField() { + if (idToField == null) { + idToField = InternalSchemaUtils.buildIdToField(record); + } + return idToField; + } + + /** + * get all columns full name. + */ + public List getAllColsFullName() { + if (nameToId == null) { + nameToId = InternalSchemaUtils.buildNameToId(record); + } + return Arrays.asList(nameToId.keySet().toArray(new String[0])); + } + + /** + * set the version ID for this schema. + */ + public InternalSchema setSchemaId(long versionId) { + this.versionId = versionId; + return this; + } + + /** + * Returns the version ID for this schema. + */ + public long schemaId() { + return this.versionId; + } + + /** + * set the version ID for this schema. + */ + public void setMaxColumnId(int maxColumnId) { + this.maxColumnId = maxColumnId; + } + + /** + * Returns the max column id for this schema. + */ + public int getMaxColumnId() { + return this.maxColumnId; + } + + /** + * Returns a List of the {@link Field columns} in this Schema. + */ + public List columns() { + return record.fields(); + } + + /** + * Returns the {@link Type} of a sub-field identified by the field name. + * + * @param id a field id + * @return fullName of field of + */ + public String findfullName(int id) { + if (idToName == null) { + buildIdToName(); + } + String result = idToName.get(id); + return result == null ? "" : result; + } + + /** + * Returns the {@link Type} of a sub-field identified by the field name. + * + * @param name a field name + * @return a Type for the sub-field or null if it is not found + */ + public Type findType(String name) { + if (name == null || name.isEmpty()) { + return null; + } + Integer id = buildNameToId().get(name); + if (id != null) { // name is found + return findType(id); + } + return null; + } + + /** + * Returns the {@link Type} of a sub-field identified by the field id. + * + * @param id a field id + * @return a Type for the sub-field or null if it is not found + */ + public Type findType(int id) { + Field field = buildIdToField().get(id); + if (field != null) { + return field.type(); + } + return null; + } + + /** + * Returns all field ids + */ + public Set getAllIds() { + if (idToName == null) { + buildIdToName(); + } + return idToName.keySet(); + } + + /** + * Returns the sub-field identified by the field id. + * + * @param id a field id + * @return the sub-field or null if it is not found + */ + public Field findField(int id) { + return buildIdToField().get(id); + } + + /** + * Returns a sub-field by name as a {@link Field}. + * The result may be a top-level or a nested field. + * + * @param name a String name + * @return a Type for the sub-field or null if it is not found + */ + public Field findField(String name) { + if (name == null || name.isEmpty()) { + return null; + } + Integer id = buildNameToId().get(name); + if (id != null) { + return buildIdToField().get(id); + } + return null; + } + + /** + * Whether colName exists in current Schema. + * Case insensitive. + * + * @param colName a colName + * @return Whether colName exists in current Schema + */ + public boolean findDuplicateCol(String colName) { + return idToName.entrySet().stream().map(e -> e.getValue().toLowerCase(Locale.ROOT)) + .collect(Collectors.toSet()).contains(colName); + } + + public int findIdByName(String name) { + if (name == null || name.isEmpty()) { + return -1; + } + return buildNameToId().getOrDefault(name, -1); + } + + @Override + public String toString() { + return String.format("table {\n%s\n}", + StringUtils.join(record.fields().stream() + .map(f -> " " + f) + .collect(Collectors.toList()).toArray(new String[0]), "\n")); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof InternalSchema)) { + return false; + } + InternalSchema that = (InternalSchema) o; + if (versionId != that.schemaId()) { + return false; + } + return record.equals(that.record); + } + + @Override + public int hashCode() { + return record.hashCode(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java new file mode 100644 index 0000000000000..b89cceb21319d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema; + +import java.io.Serializable; +import java.util.List; +import java.util.Locale; + +/** + * The type of a schema, reference avro schema. + * now avro version used by hoodie, not support localTime. + * to do add support for localTime if avro version is updated + */ +public interface Type extends Serializable { + enum TypeID { + RECORD, ARRAY, MAP, FIXED, STRING, BINARY, + INT, LONG, FLOAT, DOUBLE, DATE, BOOLEAN, TIME, TIMESTAMP, DECIMAL, UUID; + private String name; + TypeID() { + this.name = this.name().toLowerCase(Locale.ROOT); + } + + public String getName() { + return name; + } + } + + static TypeID fromValue(String value) { + try { + return TypeID.valueOf(value.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException(String.format("Invalid value of Type: %s", value)); + } + } + + TypeID typeId(); + + default boolean isNestedType() { + return false; + } + + abstract class PrimitiveType implements Type { + @Override + public boolean isNestedType() { + return false; + } + } + + abstract class NestedType implements Type { + + @Override + public boolean isNestedType() { + return true; + } + + public abstract List fields(); + + public abstract Type fieldType(String name); + + public abstract Types.Field field(int id); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java new file mode 100644 index 0000000000000..fff10a700f618 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -0,0 +1,716 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema; + +import org.apache.hudi.internal.schema.Type.PrimitiveType; +import org.apache.hudi.internal.schema.Type.NestedType; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class Types { + private Types() { + } + + public static class BooleanType extends PrimitiveType { + private static final BooleanType INSTANCE = new BooleanType(); + + public static BooleanType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return Type.TypeID.BOOLEAN; + } + + @Override + public String toString() { + return "boolean"; + } + } + + public static class IntType extends PrimitiveType { + private static final IntType INSTANCE = new IntType(); + + public static IntType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.INT; + } + + @Override + public String toString() { + return "int"; + } + } + + public static class LongType extends PrimitiveType { + private static final LongType INSTANCE = new LongType(); + + public static LongType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.LONG; + } + + @Override + public String toString() { + return "long"; + } + } + + public static class FloatType extends PrimitiveType { + private static final FloatType INSTANCE = new FloatType(); + + public static FloatType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.FLOAT; + } + + @Override + public String toString() { + return "float"; + } + } + + public static class DoubleType extends PrimitiveType { + private static final DoubleType INSTANCE = new DoubleType(); + + public static DoubleType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.DOUBLE; + } + + @Override + public String toString() { + return "double"; + } + } + + public static class DateType extends PrimitiveType { + private static final DateType INSTANCE = new DateType(); + + public static DateType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.DATE; + } + + @Override + public String toString() { + return "date"; + } + } + + public static class TimeType extends PrimitiveType { + private static final TimeType INSTANCE = new TimeType(); + + public static TimeType get() { + return INSTANCE; + } + + private TimeType() { + } + + @Override + public TypeID typeId() { + return TypeID.TIME; + } + + @Override + public String toString() { + return "time"; + } + } + + public static class TimestampType extends PrimitiveType { + private static final TimestampType INSTANCE = new TimestampType(); + + public static TimestampType get() { + return INSTANCE; + } + + private TimestampType() { + } + + @Override + public TypeID typeId() { + return TypeID.TIMESTAMP; + } + + @Override + public String toString() { + return "timestamp"; + } + } + + public static class StringType extends PrimitiveType { + private static final StringType INSTANCE = new StringType(); + + public static StringType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.STRING; + } + + @Override + public String toString() { + return "string"; + } + } + + public static class BinaryType extends PrimitiveType { + private static final BinaryType INSTANCE = new BinaryType(); + + public static BinaryType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.BINARY; + } + + @Override + public String toString() { + return "binary"; + } + } + + public static class FixedType extends PrimitiveType { + public static FixedType getFixed(int size) { + return new FixedType(size); + } + + private final int size; + + private FixedType(int length) { + this.size = length; + } + + public int getFixedSize() { + return size; + } + + @Override + public TypeID typeId() { + return TypeID.FIXED; + } + + @Override + public String toString() { + return String.format("fixed[%d]", size); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof FixedType)) { + return false; + } + + FixedType fixedType = (FixedType) o; + return size == fixedType.size; + } + + @Override + public int hashCode() { + return Objects.hash(FixedType.class, size); + } + } + + public static class DecimalType extends PrimitiveType { + public static DecimalType get(int precision, int scale) { + return new DecimalType(precision, scale); + } + + private final int scale; + private final int precision; + + private DecimalType(int precision, int scale) { + this.scale = scale; + this.precision = precision; + } + + /** + * Returns whether this DecimalType is wider than `other`. If yes, it means `other` + * can be casted into `this` safely without losing any precision or range. + */ + public boolean isWiderThan(PrimitiveType other) { + if (other instanceof DecimalType) { + DecimalType dt = (DecimalType) other; + return (precision - scale) >= (dt.precision - dt.scale) && scale > dt.scale; + } + if (other instanceof IntType) { + return isWiderThan(get(10, 0)); + } + return false; + } + + /** + * Returns whether this DecimalType is tighter than `other`. If yes, it means `this` + * can be casted into `other` safely without losing any precision or range. + */ + public boolean isTighterThan(PrimitiveType other) { + if (other instanceof DecimalType) { + DecimalType dt = (DecimalType) other; + return (precision - scale) <= (dt.precision - dt.scale) && scale <= dt.scale; + } + if (other instanceof IntType) { + return isTighterThan(get(10, 0)); + } + return false; + } + + public int scale() { + return scale; + } + + public int precision() { + return precision; + } + + @Override + public TypeID typeId() { + return TypeID.DECIMAL; + } + + @Override + public String toString() { + return String.format("decimal(%d, %d)", precision, scale); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof DecimalType)) { + return false; + } + + DecimalType that = (DecimalType) o; + if (scale != that.scale) { + return false; + } + return precision == that.precision; + } + + @Override + public int hashCode() { + return Objects.hash(DecimalType.class, scale, precision); + } + } + + public static class UUIDType extends PrimitiveType { + private static final UUIDType INSTANCE = new UUIDType(); + + public static UUIDType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.UUID; + } + + @Override + public String toString() { + return "uuid"; + } + } + + /** A field within a record. */ + public static class Field implements Serializable { + // Experimental method to support defaultValue + public static Field get(int id, boolean isOptional, String name, Type type, String doc, Object defaultValue) { + return new Field(isOptional, id, name, type, doc, defaultValue); + } + + public static Field get(int id, boolean isOptional, String name, Type type, String doc) { + return new Field(isOptional, id, name, type, doc, null); + } + + public static Field get(int id, boolean isOptional, String name, Type type) { + return new Field(isOptional, id, name, type, null, null); + } + + public static Field get(int id, String name, Type type) { + return new Field(true, id, name, type, null, null); + } + + private final boolean isOptional; + private final int id; + private final String name; + private final Type type; + private final String doc; + // Experimental properties + private final Object defaultValue; + + private Field(boolean isOptional, int id, String name, Type type, String doc, Object defaultValue) { + this.isOptional = isOptional; + this.id = id; + this.name = name; + this.type = type; + this.doc = doc; + this.defaultValue = defaultValue; + } + + public Object getDefaultValue() { + return defaultValue; + } + + public boolean isOptional() { + return isOptional; + } + + public int fieldId() { + return id; + } + + public String name() { + return name; + } + + public Type type() { + return type; + } + + public String doc() { + return doc; + } + + @Override + public String toString() { + return String.format("%d: %s: %s %s", + id, name, isOptional ? "optional" : "required", type) + (doc != null ? " (" + doc + ")" : ""); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof Field)) { + return false; + } + + Field that = (Field) o; + if (isOptional != that.isOptional) { + return false; + } else if (id != that.id) { + return false; + } else if (!name.equals(that.name)) { + return false; + } else if (!Objects.equals(doc, that.doc)) { + return false; + } + return type.equals(that.type); + } + + @Override + public int hashCode() { + return Objects.hash(Field.class, id, isOptional, name, type); + } + } + + public static class RecordType extends NestedType { + + public static RecordType get(List fields) { + return new RecordType(fields); + } + + public static RecordType get(Field... fields) { + return new RecordType(Arrays.asList(fields)); + } + + private final Field[] fields; + + private transient Map nameToFields = null; + private transient Map idToFields = null; + + private RecordType(List fields) { + this.fields = new Field[fields.size()]; + for (int i = 0; i < this.fields.length; i += 1) { + this.fields[i] = fields.get(i); + } + } + + @Override + public List fields() { + return Arrays.asList(fields); + } + + public Field field(String name) { + if (nameToFields == null) { + nameToFields = new HashMap<>(); + for (Field field : fields) { + nameToFields.put(field.name().toLowerCase(Locale.ROOT), field); + } + } + return nameToFields.get(name.toLowerCase(Locale.ROOT)); + } + + @Override + public Field field(int id) { + if (idToFields == null) { + idToFields = new HashMap<>(); + for (Field field : fields) { + idToFields.put(field.fieldId(), field); + } + } + return idToFields.get(id); + } + + @Override + public Type fieldType(String name) { + Field field = field(name); + if (field != null) { + return field.type(); + } + return null; + } + + @Override + public TypeID typeId() { + return TypeID.RECORD; + } + + @Override + public String toString() { + return String.format("Record<%s>", Arrays.stream(fields).map(f -> f.toString()).collect(Collectors.joining("-"))); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof RecordType)) { + return false; + } + + RecordType that = (RecordType) o; + return Arrays.equals(fields, that.fields); + } + + @Override + public int hashCode() { + return Objects.hash(Field.class, Arrays.hashCode(fields)); + } + } + + public static class ArrayType extends NestedType { + public static ArrayType get(int elementId, boolean isOptional, Type elementType) { + return new ArrayType(Field.get(elementId, isOptional,"element", elementType)); + } + + private final Field elementField; + + private ArrayType(Field elementField) { + this.elementField = elementField; + } + + public Type elementType() { + return elementField.type(); + } + + @Override + public Type fieldType(String name) { + if ("element".equals(name)) { + return elementType(); + } + return null; + } + + @Override + public Field field(int id) { + if (elementField.fieldId() == id) { + return elementField; + } + return null; + } + + @Override + public List fields() { + return Arrays.asList(elementField); + } + + public int elementId() { + return elementField.fieldId(); + } + + public boolean isElementOptional() { + return elementField.isOptional; + } + + @Override + public TypeID typeId() { + return TypeID.ARRAY; + } + + @Override + public String toString() { + return String.format("list<%s>", elementField.type()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof ArrayType)) { + return false; + } + ArrayType listType = (ArrayType) o; + return elementField.equals(listType.elementField); + } + + @Override + public int hashCode() { + return Objects.hash(ArrayType.class, elementField); + } + } + + public static class MapType extends NestedType { + + public static MapType get(int keyId, int valueId, Type keyType, Type valueType) { + return new MapType( + Field.get(keyId, "key", keyType), + Field.get(valueId, "value", valueType)); + } + + public static MapType get(int keyId, int valueId, Type keyType, Type valueType, boolean isOptional) { + return new MapType( + Field.get(keyId, isOptional, "key", keyType), + Field.get(valueId, isOptional, "value", valueType)); + } + + private final Field keyField; + private final Field valueField; + private transient List fields = null; + + private MapType(Field keyField, Field valueField) { + this.keyField = keyField; + this.valueField = valueField; + } + + public Type keyType() { + return keyField.type(); + } + + public Type valueType() { + return valueField.type(); + } + + @Override + public Type fieldType(String name) { + if ("key".equals(name)) { + return keyField.type(); + } else if ("value".equals(name)) { + return valueField.type(); + } + return null; + } + + @Override + public Field field(int id) { + if (keyField.fieldId() == id) { + return keyField; + } else if (valueField.fieldId() == id) { + return valueField; + } + return null; + } + + @Override + public List fields() { + if (fields == null) { + fields = Arrays.asList(keyField, valueField); + } + return fields; + } + + public int keyId() { + return keyField.fieldId(); + } + + public int valueId() { + return valueField.fieldId(); + } + + public boolean isValueOptional() { + return valueField.isOptional; + } + + @Override + public TypeID typeId() { + return TypeID.MAP; + } + + @Override + public String toString() { + return String.format("map<%s, %s>", keyField.type(), valueField.type()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof MapType)) { + return false; + } + + MapType mapType = (MapType) o; + if (!keyField.equals(mapType.keyField)) { + return false; + } + return valueField.equals(mapType.valueField); + } + + @Override + public int hashCode() { + return Objects.hash(MapType.class, keyField, valueField); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java new file mode 100644 index 0000000000000..73c62f2715367 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.action; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; + +import java.util.ArrayList; +import java.util.List; + +/** + * auxiliary class. + * help to merge file schema and query schema to produce final read schema for avro/parquet file + */ +public class InternalSchemaMerger { + private final InternalSchema fileSchema; + private final InternalSchema querySchema; + // Now there exist some bugs when we use spark update/merge api, + // https://issues.apache.org/jira/browse/HUDI-3646 + // These operations will change col nullability from optional to required which is wrong. + // Before that bug is fixed, we need to do adapt. + // If mergeRequiredFiledForce is true, we will ignore the col's required attribute. + private final boolean ignoreRequiredAttribute; + // Whether to use column Type from file schema to read files when we find some column type has changed. + private boolean useColumnTypeFromFileSchema = true; + + public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) { + this.fileSchema = fileSchema; + this.querySchema = querySchema; + this.ignoreRequiredAttribute = ignoreRequiredAttribute; + this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema; + } + + public List buildRecordType(List oldFields, List newTypes) { + List newFields = new ArrayList<>(); + for (int i = 0; i < newTypes.size(); i++) { + Type newType = newTypes.get(i); + Types.Field oldField = oldFields.get(i); + int fieldId = oldField.fieldId(); + String fullName = querySchema.findfullName(fieldId); + if (fileSchema.findField(fieldId) != null) { + if (fileSchema.findfullName(fieldId).equals(fullName)) { + // maybe col type changed, deal with it. + newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc())); + } else { + // find rename, deal with it. + newFields.add(dealWithRename(fieldId, newType, oldField)); + } + } else { + // buildFullName + fullName = normalizeFullName(fullName); + if (fileSchema.findField(fullName) != null) { + newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name() + "suffix", oldField.type(), oldField.doc())); + } else { + // find add column + // now there exist some bugs when we use spark update/merge api, those operation will change col optional to required. + if (ignoreRequiredAttribute) { + newFields.add(Types.Field.get(oldField.fieldId(), true, oldField.name(), newType, oldField.doc())); + } else { + newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc())); + } + } + } + } + return newFields; + } + + private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldField) { + Types.Field fieldFromFileSchema = fileSchema.findField(fieldId); + String nameFromFileSchema = fieldFromFileSchema.name(); + Type typeFromFileSchema = fieldFromFileSchema.type(); + // Current design mechanism guarantees nestedType change is not allowed, so no need to consider. + if (newType.isNestedType()) { + return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, newType, oldField.doc()); + } else { + return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc()); + } + } + + private String normalizeFullName(String fullName) { + // find parent rename, and normalize fullName + // eg: we renamed a nest field struct(c, d) to aa, the we delete a.d and add it back later. + String[] nameParts = fullName.split("\\."); + String[] normalizedNameParts = new String[nameParts.length]; + System.arraycopy(nameParts, 0, normalizedNameParts, 0, nameParts.length); + for (int j = 0; j < nameParts.length - 1; j++) { + StringBuilder sb = new StringBuilder(); + for (int k = 0; k <= j; k++) { + sb.append(nameParts[k]); + } + String parentName = sb.toString(); + int parentFieldIdFromQuerySchema = querySchema.findIdByName(parentName); + String parentNameFromFileSchema = fileSchema.findfullName(parentFieldIdFromQuerySchema); + if (parentNameFromFileSchema.isEmpty()) { + break; + } + if (!parentNameFromFileSchema.equalsIgnoreCase(parentName)) { + // find parent rename, update nameParts + String[] parentNameParts = parentNameFromFileSchema.split("\\."); + System.arraycopy(parentNameParts, 0, normalizedNameParts, 0, parentNameParts.length); + } + } + return StringUtils.join(normalizedNameParts, "."); + } + + public Type buildArrayType(Types.ArrayType array, Type newType) { + Types.Field elementField = array.fields().get(0); + int elementId = elementField.fieldId(); + if (elementField.type() == newType) { + return array; + } else { + return Types.ArrayType.get(elementId, elementField.isOptional(), newType); + } + } + + public Type buildMapType(Types.MapType map, Type newValue) { + Types.Field valueFiled = map.fields().get(1); + if (valueFiled.type() == newValue) { + return map; + } else { + return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValue, map.isValueOptional()); + } + } + + public Type buildPrimitiveType(Type.PrimitiveType typeFromQuerySchema, int currentPrimitiveTypeId) { + Type typeFromFileSchema = fileSchema.findType(currentPrimitiveTypeId); + if (typeFromFileSchema == null) { + return typeFromQuerySchema; + } else { + return useColumnTypeFromFileSchema ? typeFromFileSchema : typeFromQuerySchema; + } + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChange.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChange.java new file mode 100644 index 0000000000000..7061b727ee0d8 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChange.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.action; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.HoodieSchemaException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; +import org.apache.hudi.internal.schema.Types; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +/** + * TableChange subclasses represent requested changes to a table. + * now only column changes support. + * to do support partition changes + */ +public interface TableChange { + /* The action Type of schema change. */ + enum ColumnChangeID { + ADD, UPDATE, DELETE, PROPERTY_CHANGE; + private String name; + + private ColumnChangeID() { + this.name = this.name().toLowerCase(Locale.ROOT); + } + + public String getName() { + return name; + } + } + + static ColumnChangeID fromValue(String value) { + switch (value.toLowerCase(Locale.ROOT)) { + case "add": + return ColumnChangeID.ADD; + case "change": + return ColumnChangeID.UPDATE; + case "delete": + return ColumnChangeID.DELETE; + case "property": + return ColumnChangeID.PROPERTY_CHANGE; + default: + throw new IllegalArgumentException("Invalid value of Type."); + } + } + + ColumnChangeID columnChangeId(); + + default boolean withPositionChange() { + return false; + } + + abstract class BaseColumnChange implements TableChange { + protected final InternalSchema internalSchema; + protected final Map id2parent; + protected final Map> positionChangeMap = new HashMap<>(); + + BaseColumnChange(InternalSchema schema) { + this.internalSchema = schema; + this.id2parent = InternalSchemaUtils.index2Parents(schema.getRecord()); + } + + /** + * add position change. + * + * @param srcName column which need to be reordered + * @param dsrName reference position + * @param orderType change types + * @return this + */ + public BaseColumnChange addPositionChange(String srcName, String dsrName, ColumnPositionChange.ColumnPositionType orderType) { + Integer srcId = findIdByFullName(srcName); + Option dsrIdOpt = dsrName.isEmpty() ? Option.empty() : Option.of(findIdByFullName(dsrName)); + Integer srcParentId = id2parent.get(srcId); + Option dsrParentIdOpt = dsrIdOpt.map(id2parent::get); + // forbid adjust hoodie metadata columns. + switch (orderType) { + case BEFORE: + checkColModifyIsLegal(dsrName); + break; + case FIRST: + if (srcId == null || srcId == -1 || srcParentId == null || srcParentId == -1) { + throw new HoodieSchemaException("forbid adjust top-level columns position by using through first syntax"); + } + break; + case AFTER: + List checkColumns = HoodieRecord.HOODIE_META_COLUMNS.subList(0, HoodieRecord.HOODIE_META_COLUMNS.size() - 2); + if (checkColumns.stream().anyMatch(f -> f.equalsIgnoreCase(dsrName))) { + throw new HoodieSchemaException("forbid adjust the position of ordinary columns between meta columns"); + } + break; + case NO_OPERATION: + default: + break; + } + int parentId; + if (srcParentId != null && dsrParentIdOpt.isPresent() && srcParentId.equals(dsrParentIdOpt.get())) { + Types.Field parentField = internalSchema.findField(srcParentId); + if (!(parentField.type() instanceof Types.RecordType)) { + throw new HoodieSchemaException(String.format("only support reorder fields in struct type, but find: %s", parentField.type())); + } + parentId = parentField.fieldId(); + } else if (srcParentId == null && !dsrParentIdOpt.isPresent()) { + parentId = -1; + } else if (srcParentId != null && !dsrParentIdOpt.isPresent() && orderType.equals(ColumnPositionChange.ColumnPositionType.FIRST)) { + parentId = srcParentId; + } else { + throw new HoodieSchemaException("cannot order position from different parent"); + } + + ArrayList changes = positionChangeMap.getOrDefault(parentId, new ArrayList<>()); + changes.add(ColumnPositionChange.get(srcId, dsrIdOpt.orElse(-1), orderType)); + positionChangeMap.put(parentId, changes); + return this; + } + + public BaseColumnChange addPositionChange(String srcName, String dsrName, String orderType) { + return addPositionChange(srcName, dsrName, ColumnPositionChange.fromTypeValue(orderType)); + } + + /** + * abstract method. + * give a column fullName and return the field id + * + * @param fullName column fullName + * @return field id of current column + */ + protected abstract Integer findIdByFullName(String fullName); + + // Modify hudi meta columns is prohibited + protected void checkColModifyIsLegal(String colNeedToModfiy) { + if (HoodieRecord.HOODIE_META_COLUMNS.stream().anyMatch(f -> f.equalsIgnoreCase(colNeedToModfiy))) { + throw new IllegalArgumentException(String.format("cannot modify hudi meta col: %s", colNeedToModfiy)); + } + } + + @Override + public boolean withPositionChange() { + return false; + } + } + + /** + * Column position change. + * now support three change types: FIRST/AFTER/BEFORE + * FIRST means the specified column should be the first column. + * AFTER means the specified column should be put after the given column. + * BEFORE means the specified column should be put before the given column. + * Note that, the specified column may be a nested field: + * AFTER/BEFORE means the given columns should in the same struct; + * FIRST means this field should be the first one within the struct. + */ + class ColumnPositionChange { + public enum ColumnPositionType { + FIRST, + BEFORE, + AFTER, + // only expose to internal use. + NO_OPERATION + } + + static ColumnPositionType fromTypeValue(String value) { + switch (value.toLowerCase(Locale.ROOT)) { + case "first": + return ColumnPositionType.FIRST; + case "before": + return ColumnPositionType.BEFORE; + case "after": + return ColumnPositionType.AFTER; + case "no_operation": + return ColumnPositionType.NO_OPERATION; + default: + throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", value)); + } + } + + private final int srcId; + private final int dsrId; + private final ColumnPositionType type; + + static ColumnPositionChange first(int srcId) { + return new ColumnPositionChange(srcId, -1, ColumnPositionType.FIRST); + } + + static ColumnPositionChange before(int srcId, int dsrId) { + return new ColumnPositionChange(srcId, dsrId, ColumnPositionType.BEFORE); + } + + static ColumnPositionChange after(int srcId, int dsrId) { + return new ColumnPositionChange(srcId, dsrId, ColumnPositionType.AFTER); + } + + static ColumnPositionChange get(int srcId, int dsrId, String type) { + return get(srcId, dsrId, fromTypeValue(type)); + } + + static ColumnPositionChange get(int srcId, int dsrId, ColumnPositionType type) { + switch (type) { + case FIRST: + return ColumnPositionChange.first(srcId); + case BEFORE: + return ColumnPositionChange.before(srcId, dsrId); + case AFTER: + return ColumnPositionChange.after(srcId, dsrId); + default: + throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", type)); + } + } + + private ColumnPositionChange(int srcId, int dsrId, ColumnPositionType type) { + this.srcId = srcId; + this.dsrId = dsrId; + this.type = type; + } + + public int getSrcId() { + return srcId; + } + + public int getDsrId() { + return dsrId; + } + + public ColumnPositionType type() { + return type; + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChanges.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChanges.java new file mode 100644 index 0000000000000..b95b6d5e9daf9 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChanges.java @@ -0,0 +1,398 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.action; + +import org.apache.hudi.internal.schema.HoodieSchemaException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.utils.SchemaChangeUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +public class TableChanges { + + /** deal with update columns changes for table. */ + public static class ColumnUpdateChange extends TableChange.BaseColumnChange { + private final Map updates = new HashMap<>(); + + public static ColumnUpdateChange get(InternalSchema schema) { + return new ColumnUpdateChange(schema); + } + + private ColumnUpdateChange(InternalSchema schema) { + super(schema); + } + + @Override + public boolean withPositionChange() { + return true; + } + + public Type applyUpdates(Types.Field oldField, Type type) { + Types.Field update = updates.get(oldField.fieldId()); + if (update != null && update.type() != oldField.type()) { + return update.type(); + } + // + ArrayList pchanges = positionChangeMap.getOrDefault(oldField.fieldId(), new ArrayList<>()); + if (!pchanges.isEmpty()) { + // when we build ColumnAddChange,we have already done some check, so it's safe to convert newType to RecordType + List newFields = TableChangesHelper.applyAddChange2Fields(((Types.RecordType) type).fields(), new ArrayList<>(), pchanges); + return Types.RecordType.get(newFields); + } + return type; + } + + public Map getUpdates() { + return updates; + } + + /** + * Update a column in the schema to a new type. + * only support update primitive type. + * Only updates that widen types are allowed. + * + * @param name name of the column to update + * @param newType new type for the column + * @return this + * @throws IllegalArgumentException + */ + public ColumnUpdateChange updateColumnType(String name, Type newType) { + checkColModifyIsLegal(name); + if (newType.isNestedType()) { + throw new IllegalArgumentException(String.format("only support update primitive type but find nest column: %s", name)); + } + Types.Field field = internalSchema.findField(name); + if (field == null) { + throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name)); + } + + if (!SchemaChangeUtils.isTypeUpdateAllow(field.type(), newType)) { + throw new IllegalArgumentException(String.format("cannot update origin type: %s to a incompatibility type: %s", field.type(), newType)); + } + + if (field.type().equals(newType)) { + // do nothings + return this; + } + // save update info + Types.Field update = updates.get(field.fieldId()); + if (update == null) { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), field.name(), newType, field.doc())); + } else { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), update.name(), newType, update.doc())); + } + return this; + } + + /** + * Update a column doc in the schema to a new primitive type. + * + * @param name name of the column to update + * @param newDoc new documentation for the column + * @return this + * @throws IllegalArgumentException + */ + public ColumnUpdateChange updateColumnComment(String name, String newDoc) { + checkColModifyIsLegal(name); + Types.Field field = internalSchema.findField(name); + if (field == null) { + throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name)); + } + // consider null + if (Objects.equals(field.doc(), newDoc)) { + // do nothings + return this; + } + // save update info + Types.Field update = updates.get(field.fieldId()); + if (update == null) { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), field.name(), field.type(), newDoc)); + } else { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), update.name(), update.type(), newDoc)); + } + return this; + } + + /** + * Rename a column in the schema. + * + * @param name name of the column to rename + * @param newName new name for the column + * @return this + * @throws IllegalArgumentException + */ + public ColumnUpdateChange renameColumn(String name, String newName) { + checkColModifyIsLegal(name); + Types.Field field = internalSchema.findField(name); + if (field == null) { + throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name)); + } + if (newName == null || newName.isEmpty()) { + throw new IllegalArgumentException(String.format("cannot rename column: %s to empty", name)); + } + // keep consisitent with hive. column names insensitive, so we check 'newName.toLowerCase(Locale.ROOT)' + if (internalSchema.findDuplicateCol(newName.toLowerCase(Locale.ROOT))) { + throw new IllegalArgumentException(String.format("cannot rename column: %s to a existing name", name)); + } + // save update info + Types.Field update = updates.get(field.fieldId()); + if (update == null) { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), newName, field.type(), field.doc())); + } else { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), newName, update.type(), update.doc())); + } + return this; + } + + /** + * update nullable for column. + * only support required type -> optional type + * + * @param name name of the column to update + * @param nullable nullable for updated name + * @return this + * @throws IllegalArgumentException + */ + public ColumnUpdateChange updateColumnNullability(String name, boolean nullable) { + return updateColumnNullability(name, nullable, false); + } + + public ColumnUpdateChange updateColumnNullability(String name, boolean nullable, boolean force) { + checkColModifyIsLegal(name); + Types.Field field = internalSchema.findField(name); + if (field == null) { + throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name)); + } + if (field.isOptional() == nullable) { + // do nothings + return this; + } + if (field.isOptional() && !nullable && !force) { + throw new IllegalArgumentException("cannot update column Nullability: optional to required"); + } + // save update info + Types.Field update = updates.get(field.fieldId()); + if (update == null) { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), nullable, field.name(), field.type(), field.doc())); + } else { + updates.put(field.fieldId(), Types.Field.get(field.fieldId(), nullable, update.name(), update.type(), update.doc())); + } + + return this; + } + + public Map> getPositionChangeMap() { + return positionChangeMap; + } + + @Override + public ColumnChangeID columnChangeId() { + return ColumnChangeID.UPDATE; + } + + @Override + protected Integer findIdByFullName(String fullName) { + Types.Field field = internalSchema.findField(fullName); + if (field != null) { + return field.fieldId(); + } else { + throw new IllegalArgumentException(String.format("cannot find col id for given column fullName: %s", fullName)); + } + } + } + + /** deal with delete columns changes for table. */ + public static class ColumnDeleteChange extends TableChange.BaseColumnChange { + private final Set deletes = new HashSet<>(); + + @Override + public ColumnChangeID columnChangeId() { + return ColumnChangeID.DELETE; + } + + public static ColumnDeleteChange get(InternalSchema schema) { + return new ColumnDeleteChange(schema); + } + + private ColumnDeleteChange(InternalSchema schema) { + super(schema); + } + + @Override + public boolean withPositionChange() { + return false; + } + + @Override + public BaseColumnChange addPositionChange(String srcId, String dsrId, String orderType) { + throw new UnsupportedOperationException("no support add position change for ColumnDeleteChange"); + } + + public ColumnDeleteChange deleteColumn(String name) { + checkColModifyIsLegal(name); + Types.Field field = internalSchema.findField(name); + if (field == null) { + throw new IllegalArgumentException(String.format("cannot delete missing columns: %s", name)); + } + deletes.add(field.fieldId()); + return this; + } + + public Type applyDelete(int id, Type type) { + if (deletes.contains(id)) { + return null; + } + return type; + } + + public Set getDeletes() { + return deletes; + } + + @Override + protected Integer findIdByFullName(String fullName) { + throw new UnsupportedOperationException("delete change cannot support this method"); + } + } + + /** deal with add columns changes for table. */ + public static class ColumnAddChange extends TableChange.BaseColumnChange { + private final Map fullColName2Id = new HashMap<>(); + private final Map> parentId2AddCols = new HashMap<>(); + private int nextId; + + public static ColumnAddChange get(InternalSchema internalSchema) { + return new ColumnAddChange(internalSchema); + } + + public Type applyAdd(Types.Field orignalField, Type type) { + int fieldId = orignalField.fieldId(); + ArrayList addFields = parentId2AddCols.getOrDefault(fieldId, new ArrayList<>()); + ArrayList pchanges = positionChangeMap.getOrDefault(fieldId, new ArrayList<>()); + + if (!addFields.isEmpty() || !pchanges.isEmpty()) { + // when we build ColumnAddChange,we have already done some check, so it's safe to convert newType to RecordType + List newFields = TableChangesHelper.applyAddChange2Fields(((Types.RecordType) type).fields(), addFields, pchanges); + return Types.RecordType.get(newFields); + } + return type; + } + + public ColumnAddChange addColumns(String name, Type type, String doc) { + checkColModifyIsLegal(name); + return addColumns("", name, type, doc); + } + + public ColumnAddChange addColumns(String parent, String name, Type type, String doc) { + checkColModifyIsLegal(name); + addColumnsInternal(parent, name, type, doc); + return this; + } + + private void addColumnsInternal(String parent, String name, Type type, String doc) { + // root record has no parent, so set parentId to -1 as default + int parentId = -1; + // do check + String fullName = name; + if (!parent.isEmpty()) { + Types.Field parentField = internalSchema.findField(parent); + if (parentField == null) { + throw new HoodieSchemaException(String.format("cannot add column: %s which parent: %s is not exist", name, parent)); + } + Type parentType = parentField.type(); + if (!(parentField.type() instanceof Types.RecordType)) { + throw new HoodieSchemaException("only support add nested columns to struct column"); + } + parentId = parentField.fieldId(); + Types.Field newParentField = internalSchema.findField(parent + "." + name); + if (newParentField != null) { + throw new HoodieSchemaException(String.format("cannot add column: %s which already exist", name)); + } + fullName = parent + "." + name; + } else { + // keep consistent with hive, column name case insensitive + if (internalSchema.findDuplicateCol(name.toLowerCase(Locale.ROOT))) { + throw new HoodieSchemaException(String.format("cannot add column: %s which already exist", name)); + } + } + if (fullColName2Id.containsKey(fullName)) { + throw new HoodieSchemaException(String.format("cannot repeat add column: %s", name)); + } + fullColName2Id.put(fullName, nextId); + if (parentId != -1) { + id2parent.put(nextId, parentId); + } + AtomicInteger assignNextId = new AtomicInteger(nextId + 1); + Type typeWithNewId = InternalSchemaUtils.refreshNewId(type, assignNextId); + // only allow add optional columns. + ArrayList adds = parentId2AddCols.getOrDefault(parentId, new ArrayList<>()); + adds.add(Types.Field.get(nextId, true, name, typeWithNewId, doc)); + parentId2AddCols.put(parentId, adds); + nextId = assignNextId.get(); + } + + private ColumnAddChange(InternalSchema internalSchema) { + super(internalSchema); + this.nextId = internalSchema.getMaxColumnId() + 1; + } + + public Map> getParentId2AddCols() { + return parentId2AddCols; + } + + public Map> getPositionChangeMap() { + return positionChangeMap; + } + + // expose to test + public Map getFullColName2Id() { + return fullColName2Id; + } + + protected Integer findIdByFullName(String fullName) { + Types.Field field = internalSchema.findField(fullName); + if (field != null) { + return field.fieldId(); + } + return fullColName2Id.getOrDefault(fullName, -1); + } + + @Override + public ColumnChangeID columnChangeId() { + return ColumnChangeID.ADD; + } + + @Override + public boolean withPositionChange() { + return true; + } + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChangesHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChangesHelper.java new file mode 100644 index 0000000000000..e8ef3cdcf96a2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChangesHelper.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.action; + +import org.apache.hudi.internal.schema.Types; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +/** + * helper class to support Table schema changes. + */ +public class TableChangesHelper { + /** + * apply add operation and column position change operation. + * + * @param fields origin column fields. + * @param adds column fields to be added. + * @param pchanges a wrapper class hold all the position change operations. + * @return column fields after adjusting the position. + */ + public static List applyAddChange2Fields(List fields, ArrayList adds, ArrayList pchanges) { + if (adds == null && pchanges == null) { + return fields; + } + LinkedList result = new LinkedList<>(fields); + // apply add columns + if (adds != null && !adds.isEmpty()) { + result.addAll(adds); + } + // apply position change + if (pchanges != null && !pchanges.isEmpty()) { + for (TableChange.ColumnPositionChange pchange : pchanges) { + Types.Field srcField = result.stream().filter(f -> f.fieldId() == pchange.getSrcId()).findFirst().get(); + Types.Field dsrField = result.stream().filter(f -> f.fieldId() == pchange.getDsrId()).findFirst().orElse(null); + // we remove srcField first + result.remove(srcField); + switch (pchange.type()) { + case AFTER: + // add srcField after dsrField + result.add(result.indexOf(dsrField) + 1, srcField); + break; + case BEFORE: + // add srcField before dsrField + result.add(result.indexOf(dsrField), srcField); + break; + case FIRST: + result.addFirst(srcField); + break; + default: + // should not reach here + } + } + } + return result; + } + + public static String getParentName(String fullColName) { + int offset = fullColName.lastIndexOf("."); + return offset > 0 ? fullColName.substring(0, offset) : ""; + } +} 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 new file mode 100644 index 0000000000000..e371e8ce7a8f1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.convert; + +import org.apache.avro.Schema; +import org.apache.hudi.internal.schema.HoodieSchemaException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; + +import java.util.List; + +import static org.apache.avro.Schema.Type.UNION; + +/** + * auxiliary class. + * Converts an avro schema into InternalSchema, or convert InternalSchema to an avro schema + */ +public class AvroInternalSchemaConverter { + + /** + * convert internalSchema to avro Schema. + * + * @param internalSchema internal schema. + * @param tableName the record name. + * @return an avro Schema. + */ + public static Schema convert(InternalSchema internalSchema, String tableName) { + return InternalSchemaUtils.buildAvroSchemaFromInternalSchema(internalSchema, tableName); + } + + /** + * convert RecordType to avro Schema. + * + * @param type internal schema. + * @param name the record name. + * @return an avro Schema. + */ + public static Schema convert(Types.RecordType type, String name) { + return InternalSchemaUtils.buildAvroSchemaFromType(type, name); + } + + /** + * convert internal type to avro Schema. + * + * @param type internal type. + * @param name the record name. + * @return an avro Schema. + */ + public static Schema convert(Type type, String name) { + return InternalSchemaUtils.buildAvroSchemaFromType(type, name); + } + + /** convert an avro schema into internal type. */ + public static Type convertToField(Schema schema) { + return InternalSchemaUtils.buildTypeFromAvroSchema(schema); + } + + /** convert an avro schema into internalSchema. */ + public static InternalSchema convert(Schema schema) { + List fields = ((Types.RecordType) convertToField(schema)).fields(); + return new InternalSchema(fields); + } + + /** check whether current avro schema is optional?. */ + public static boolean isOptional(Schema schema) { + if (schema.getType() == UNION && schema.getTypes().size() == 2) { + return schema.getTypes().get(0).getType() == Schema.Type.NULL || schema.getTypes().get(1).getType() == Schema.Type.NULL; + } + return false; + } + + /** Returns schema with nullable true. */ + public static Schema nullableSchema(Schema schema) { + if (schema.getType() == UNION) { + if (!isOptional(schema)) { + throw new HoodieSchemaException(String.format("Union schemas are not supported: %s", schema)); + } + return schema; + } else { + return Schema.createUnion(Schema.create(Schema.Type.NULL), schema); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/AbstractInternalSchemaStorageManager.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/AbstractInternalSchemaStorageManager.java new file mode 100644 index 0000000000000..86e1f4c29585c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/AbstractInternalSchemaStorageManager.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.internal.schema.io; + +import org.apache.hudi.common.util.Option; + +abstract class AbstractInternalSchemaStorageManager { + + /** + * persist history schema str. + */ + public abstract void persistHistorySchemaStr(String instantTime, String historySchemaStr); + + /** + * get latest history schema string. + */ + public abstract String getHistorySchemaStr(); + + /** + * Bulk Insert a batch of new records into Hoodie table at the supplied instantTime. + * + * @param versionId schema version_id need to search + * @return internalSchema + */ + public abstract Option getSchemaByKey(String versionId); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java new file mode 100644 index 0000000000000..ea521d361adfd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.io; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.utils.SerDeHelper; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVE_SCHEMA_ACTION; + +public class FileBasedInternalSchemaStorageManager extends AbstractInternalSchemaStorageManager { + private static final Logger LOG = LogManager.getLogger(FileBasedInternalSchemaStorageManager.class); + + public static final String SCHEMA_NAME = ".schema"; + private final Path baseSchemaPath; + private Configuration conf; + private HoodieTableMetaClient metaClient; + + public FileBasedInternalSchemaStorageManager(Configuration conf, Path baseTablePath) { + Path metaPath = new Path(baseTablePath, ".hoodie"); + this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME); + this.conf = conf; + this.metaClient = HoodieTableMetaClient.builder().setBasePath(metaPath.getParent().toString()).setConf(conf).build(); + } + + public FileBasedInternalSchemaStorageManager(HoodieTableMetaClient metaClient) { + Path metaPath = new Path(metaClient.getBasePath(), ".hoodie"); + this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME); + this.conf = metaClient.getHadoopConf(); + this.metaClient = metaClient; + } + + @Override + public void persistHistorySchemaStr(String instantTime, String historySchemaStr) { + cleanResidualFiles(); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + HoodieInstant hoodieInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, SAVE_SCHEMA_ACTION, instantTime); + timeline.createNewInstant(hoodieInstant); + byte[] writeContent = historySchemaStr.getBytes(StandardCharsets.UTF_8); + timeline.transitionRequestedToInflight(hoodieInstant, Option.empty()); + timeline.saveAsComplete(new HoodieInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.getTimestamp()), Option.of(writeContent)); + LOG.info(String.format("persist history schema success on commit time: %s", instantTime)); + } + + private void cleanResidualFiles() { + List validInstants = getValidInstants(); + try { + FileSystem fs = baseSchemaPath.getFileSystem(conf); + if (fs.exists(baseSchemaPath)) { + List candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile()) + .map(file -> file.getPath().getName()).collect(Collectors.toList()); + List residualSchemaFiles = candidateSchemaFiles.stream().filter(f -> !validInstants.contains(f.split("\\.")[0])).collect(Collectors.toList()); + // clean residual files + residualSchemaFiles.forEach(f -> { + try { + fs.delete(new Path(metaClient.getSchemaFolderName(), f)); + } catch (IOException o) { + throw new HoodieException(o); + } + }); + } + } catch (IOException e) { + throw new HoodieException(e); + } + } + + public void cleanOldFiles(List validateCommits) { + try { + FileSystem fs = baseSchemaPath.getFileSystem(conf); + if (fs.exists(baseSchemaPath)) { + List candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile()) + .map(file -> file.getPath().getName()).collect(Collectors.toList()); + List validSchemaFiles = candidateSchemaFiles.stream().filter(f -> validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList()); + for (int i = 0; i < validSchemaFiles.size(); i++) { + fs.delete(new Path(validSchemaFiles.get(i))); + } + } + } catch (IOException e) { + throw new HoodieException(e); + } + } + + private List getValidInstants() { + metaClient.reloadActiveTimeline(); + return metaClient.getCommitsTimeline() + .filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList()); + } + + @Override + public String getHistorySchemaStr() { + List validateCommits = getValidInstants(); + try { + if (metaClient.getFs().exists(baseSchemaPath)) { + List validateSchemaFiles = Arrays.stream(metaClient.getFs().listStatus(baseSchemaPath)) + .filter(f -> f.isFile() && f.getPath().getName().endsWith(SAVE_SCHEMA_ACTION)) + .map(file -> file.getPath().getName()).filter(f -> validateCommits.contains(f.split("\\.")[0])).sorted().collect(Collectors.toList()); + if (!validateSchemaFiles.isEmpty()) { + Path latestFilePath = new Path(baseSchemaPath, validateSchemaFiles.get(validateSchemaFiles.size() - 1)); + byte[] content; + try (FSDataInputStream is = metaClient.getFs().open(latestFilePath)) { + content = FileIOUtils.readAsByteArray(is); + LOG.info(String.format("read history schema success from file : %s", latestFilePath)); + return new String(content, StandardCharsets.UTF_8); + } catch (IOException e) { + throw new HoodieIOException("Could not read history schema from " + latestFilePath, e); + } + } + } + } catch (IOException io) { + throw new HoodieException(io); + } + LOG.info("failed to read history schema"); + return ""; + } + + @Override + public Option getSchemaByKey(String versionId) { + String historySchemaStr = getHistorySchemaStr(); + TreeMap treeMap; + if (historySchemaStr.isEmpty()) { + return Option.empty(); + } else { + treeMap = SerDeHelper.parseSchemas(historySchemaStr); + InternalSchema result = SerDeHelper.searchSchema(Long.valueOf(versionId), treeMap); + if (result == null) { + return Option.empty(); + } + return Option.of(result); + } + } +} + + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaUtil.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaUtil.java new file mode 100644 index 0000000000000..61402557801ca --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaUtil.java @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import static org.apache.avro.Schema.Type.UNION; + +import org.apache.avro.AvroRuntimeException; +import org.apache.avro.Conversions; +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaCompatibility; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.action.TableChanges; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; +import java.util.TreeMap; +import java.util.stream.Collectors; + +public class AvroSchemaUtil { + private AvroSchemaUtil() { + } + + private static final long MILLIS_PER_DAY = 86400000L; + + //Export for test + public static final Conversions.DecimalConversion DECIMAL_CONVERSION = new Conversions.DecimalConversion(); + + /** + * Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new schema. + * support deep rewrite for nested record. + * This particular method does the following things : + * a) Create a new empty GenericRecord with the new schema. + * b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this transformed schema + * + * @param oldRecord oldRecord to be rewrite + * @param newSchema newSchema used to rewrite oldRecord + * @return newRecord for new Schema + */ + public static GenericRecord rewriteRecord(IndexedRecord oldRecord, Schema newSchema) { + Object newRecord = rewriteRecord(oldRecord, oldRecord.getSchema(), newSchema); + return (GenericData.Record) newRecord; + } + + private static Object rewriteRecord(Object oldRecord, Schema oldSchema, Schema newSchema) { + if (oldRecord == null) { + return null; + } + switch (newSchema.getType()) { + case RECORD: + if (!(oldRecord instanceof IndexedRecord)) { + throw new IllegalArgumentException("cannot rewrite record with different type"); + } + IndexedRecord indexedRecord = (IndexedRecord) oldRecord; + List fields = newSchema.getFields(); + Map helper = new HashMap<>(); + + for (int i = 0; i < fields.size(); i++) { + Schema.Field field = fields.get(i); + if (oldSchema.getField(field.name()) != null) { + Schema.Field oldField = oldSchema.getField(field.name()); + helper.put(i, rewriteRecord(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema())); + } + } + GenericData.Record newRecord = new GenericData.Record(newSchema); + for (int i = 0; i < fields.size(); i++) { + if (helper.containsKey(i)) { + newRecord.put(i, helper.get(i)); + } else { + if (fields.get(i).defaultVal() instanceof JsonProperties.Null) { + newRecord.put(i, null); + } else { + newRecord.put(i, fields.get(i).defaultVal()); + } + } + } + return newRecord; + case ARRAY: + if (!(oldRecord instanceof Collection)) { + throw new IllegalArgumentException("cannot rewrite record with different type"); + } + Collection array = (Collection)oldRecord; + List newArray = new ArrayList(); + for (Object element : array) { + newArray.add(rewriteRecord(element, oldSchema.getElementType(), newSchema.getElementType())); + } + return newArray; + case MAP: + if (!(oldRecord instanceof Map)) { + throw new IllegalArgumentException("cannot rewrite record with different type"); + } + Map map = (Map) oldRecord; + Map newMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + newMap.put(entry.getKey(), rewriteRecord(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType())); + } + return newMap; + case UNION: + return rewriteRecord(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord)); + default: + return rewritePrimaryType(oldRecord, oldSchema, newSchema); + } + } + + private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schema newSchema) { + Schema realOldSchema = oldSchema; + if (realOldSchema.getType() == UNION) { + realOldSchema = getActualSchemaFromUnion(oldSchema, oldValue); + } + if (realOldSchema.getType() == newSchema.getType()) { + switch (realOldSchema.getType()) { + case NULL: + case BOOLEAN: + case INT: + case LONG: + case FLOAT: + case DOUBLE: + case BYTES: + case STRING: + return oldValue; + case FIXED: + // fixed size and name must match: + if (!SchemaCompatibility.schemaNameEquals(realOldSchema, newSchema) || realOldSchema.getFixedSize() != newSchema.getFixedSize()) { + // deal with the precision change for decimalType + if (realOldSchema.getLogicalType() instanceof LogicalTypes.Decimal) { + final byte[] bytes; + bytes = ((GenericFixed) oldValue).bytes(); + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) realOldSchema.getLogicalType(); + BigDecimal bd = new BigDecimal(new BigInteger(bytes), decimal.getScale()).setScale(((LogicalTypes.Decimal) newSchema.getLogicalType()).getScale()); + return DECIMAL_CONVERSION.toFixed(bd, newSchema, newSchema.getLogicalType()); + } + } else { + return oldValue; + } + return oldValue; + default: + throw new AvroRuntimeException("Unknown schema type: " + newSchema.getType()); + } + } else { + return rewritePrimaryTypeWithDiffSchemaType(oldValue, realOldSchema, newSchema); + } + } + + private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Schema oldSchema, Schema newSchema) { + switch (newSchema.getType()) { + case NULL: + case BOOLEAN: + break; + case INT: + if (newSchema.getLogicalType() == LogicalTypes.date() && oldSchema.getType() == Schema.Type.STRING) { + return fromJavaDate(Date.valueOf(oldValue.toString())); + } + break; + case LONG: + if (oldSchema.getType() == Schema.Type.INT) { + return ((Integer) oldValue).longValue(); + } + break; + case FLOAT: + if ((oldSchema.getType() == Schema.Type.INT) + || (oldSchema.getType() == Schema.Type.LONG)) { + return oldSchema.getType() == Schema.Type.INT ? ((Integer) oldValue).floatValue() : ((Long) oldValue).floatValue(); + } + break; + case DOUBLE: + if (oldSchema.getType() == Schema.Type.FLOAT) { + // java float cannot convert to double directly, deal with float precision change + return Double.valueOf(oldValue + ""); + } else if (oldSchema.getType() == Schema.Type.INT) { + return ((Integer) oldValue).doubleValue(); + } else if (oldSchema.getType() == Schema.Type.LONG) { + return ((Long) oldValue).doubleValue(); + } + break; + case BYTES: + if (oldSchema.getType() == Schema.Type.STRING) { + return ((String) oldValue).getBytes(StandardCharsets.UTF_8); + } + break; + case STRING: + if (oldSchema.getType() == Schema.Type.BYTES) { + return String.valueOf(((byte[]) oldValue)); + } + if (oldSchema.getLogicalType() == LogicalTypes.date()) { + return toJavaDate((Integer) oldValue).toString(); + } + if (oldSchema.getType() == Schema.Type.INT + || oldSchema.getType() == Schema.Type.LONG + || oldSchema.getType() == Schema.Type.FLOAT + || oldSchema.getType() == Schema.Type.DOUBLE) { + return oldValue.toString(); + } + if (oldSchema.getType() == Schema.Type.FIXED && oldSchema.getLogicalType() instanceof LogicalTypes.Decimal) { + final byte[] bytes; + bytes = ((GenericFixed) oldValue).bytes(); + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) oldSchema.getLogicalType(); + BigDecimal bd = new BigDecimal(new BigInteger(bytes), decimal.getScale()); + return bd.toString(); + } + break; + case FIXED: + // deal with decimal Type + if (newSchema.getLogicalType() instanceof LogicalTypes.Decimal) { + // TODO: support more types + if (oldSchema.getType() == Schema.Type.STRING + || oldSchema.getType() == Schema.Type.DOUBLE + || oldSchema.getType() == Schema.Type.INT + || oldSchema.getType() == Schema.Type.LONG + || oldSchema.getType() == Schema.Type.FLOAT) { + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) newSchema.getLogicalType(); + BigDecimal bigDecimal = null; + if (oldSchema.getType() == Schema.Type.STRING) { + bigDecimal = new java.math.BigDecimal((String) oldValue) + .setScale(decimal.getScale()); + } else { + // Due to Java, there will be precision problems in direct conversion, we should use string instead of use double + bigDecimal = new java.math.BigDecimal(oldValue.toString()) + .setScale(decimal.getScale()); + } + return DECIMAL_CONVERSION.toFixed(bigDecimal, newSchema, newSchema.getLogicalType()); + } + } + break; + default: + } + throw new AvroRuntimeException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema)); + } + + // convert days to Date + private static Date toJavaDate(int days) { + long localMillis = Math.multiplyExact(days, MILLIS_PER_DAY); + int timeZoneOffset; + TimeZone defaultTimeZone = TimeZone.getDefault(); + if (defaultTimeZone instanceof sun.util.calendar.ZoneInfo) { + timeZoneOffset = ((sun.util.calendar.ZoneInfo) defaultTimeZone).getOffsetsByWall(localMillis, null); + } else { + timeZoneOffset = defaultTimeZone.getOffset(localMillis - defaultTimeZone.getRawOffset()); + } + return new Date(localMillis - timeZoneOffset); + } + + // convert Date to days + private static int fromJavaDate(Date date) { + long millisUtc = date.getTime(); + long millisLocal = millisUtc + TimeZone.getDefault().getOffset(millisUtc); + int julianDays = Math.toIntExact(Math.floorDiv(millisLocal, MILLIS_PER_DAY)); + return julianDays; + } + + private static Schema getActualSchemaFromUnion(Schema schema, Object data) { + Schema actualSchema; + if (!schema.getType().equals(UNION)) { + return schema; + } + if (schema.getTypes().size() == 2 + && schema.getTypes().get(0).getType() == Schema.Type.NULL) { + actualSchema = schema.getTypes().get(1); + } else if (schema.getTypes().size() == 2 + && schema.getTypes().get(1).getType() == Schema.Type.NULL) { + actualSchema = schema.getTypes().get(0); + } else if (schema.getTypes().size() == 1) { + actualSchema = schema.getTypes().get(0); + } else { + // deal complex union. this should not happened in hoodie, + // since flink/spark do not write this type. + int i = GenericData.get().resolveUnion(schema, data); + actualSchema = schema.getTypes().get(i); + } + return actualSchema; + } + + /** + * Given avro records, rewrites them with new schema. + * + * @param oldRecords oldRecords to be rewrite + * @param newSchema newSchema used to rewrite oldRecord + * @return a iterator of rewrote GeneriRcords + */ + public static Iterator rewriteRecords(Iterator oldRecords, Schema newSchema) { + if (oldRecords == null || newSchema == null) { + return Collections.emptyIterator(); + } + return new Iterator() { + @Override + public boolean hasNext() { + return oldRecords.hasNext(); + } + + @Override + public GenericRecord next() { + return rewriteRecord(oldRecords.next(), newSchema); + } + }; + } + + /** + * support evolution from a new avroSchema. + * notice: this is not a universal method, + * now hoodie support implicitly add columns when hoodie write operation, + * This ability needs to be preserved, so implicitly evolution for internalSchema should supported. + * + * @param evolvedSchema implicitly evolution of avro when hoodie write operation + * @param oldSchema old internalSchema + * @param supportPositionReorder support position reorder + * @return evolution Schema + */ + public static InternalSchema evolutionSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema, Boolean supportPositionReorder) { + InternalSchema evolvedInternalSchema = AvroInternalSchemaConverter.convert(evolvedSchema); + // do check, only support add column evolution + List colNamesFromEvolved = evolvedInternalSchema.getAllColsFullName(); + List colNamesFromOldSchema = oldSchema.getAllColsFullName(); + List diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromEvolved.contains(f)).collect(Collectors.toList()); + List newFields = new ArrayList<>(); + if (colNamesFromEvolved.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) { + // no changes happen + if (supportPositionReorder) { + evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name()))); + return new InternalSchema(newFields); + } + return oldSchema; + } + // try to find all added columns + if (diffFromOldSchema.size() != 0) { + throw new UnsupportedOperationException("cannot evolution schema implicitly, find delete/rename operation"); + } + + List diffFromEvolutionSchema = colNamesFromEvolved.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList()); + // Remove redundancy from diffFromEvolutionSchema. + // for example, now we add a struct col in evolvedSchema, the struct col is " user struct " + // 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 < diffFromEvolutionSchema.size(); i++) { + String name = diffFromEvolutionSchema.get(i); + int splitPoint = name.lastIndexOf("."); + String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : ""; + if (!parentName.isEmpty() && diffFromEvolutionSchema.contains(parentName)) { + // find redundancy, skip it + continue; + } + finalAddAction.put(evolvedInternalSchema.findIdByName(name), name); + } + + TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema); + finalAddAction.entrySet().stream().forEach(f -> { + String name = f.getValue(); + int splitPoint = name.lastIndexOf("."); + String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : ""; + String rawName = splitPoint > 0 ? name.substring(splitPoint + 1) : name; + addChange.addColumns(parentName, rawName, evolvedInternalSchema.findType(name), null); + }); + + InternalSchema res = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange); + if (supportPositionReorder) { + evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name()))); + return new InternalSchema(newFields); + } else { + return res; + } + } + + public static InternalSchema evolutionSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema) { + return evolutionSchemaFromNewAvroSchema(evolvedSchema, oldSchema, false); + } + + /** + * canonical the nullability. + * do not allow change cols Nullability field from optional to required. + * if above problem occurs, try to correct it. + * + * @param writeSchema writeSchema hoodie used to write data. + * @param readSchema read schema + * @return canonical Schema + */ + public static Schema canonicalColumnNullability(Schema writeSchema, Schema readSchema) { + if (writeSchema.getFields().isEmpty() || readSchema.getFields().isEmpty()) { + return writeSchema; + } + InternalSchema writeInternalSchema = AvroInternalSchemaConverter.convert(writeSchema); + InternalSchema readInternalSchema = AvroInternalSchemaConverter.convert(readSchema); + List colNamesWriteSchema = writeInternalSchema.getAllColsFullName(); + List colNamesFromReadSchema = readInternalSchema.getAllColsFullName(); + // try to deal with optional change. now when we use sparksql to update hudi table, + // sparksql Will change the col type from optional to required, this is a bug. + List candidateUpdateCols = colNamesWriteSchema.stream().filter(f -> { + boolean exist = colNamesFromReadSchema.contains(f); + if (exist && (writeInternalSchema.findField(f).isOptional() != readInternalSchema.findField(f).isOptional())) { + return true; + } else { + return false; + } + }).collect(Collectors.toList()); + if (candidateUpdateCols.isEmpty()) { + return writeSchema; + } + // try to correct all changes + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(writeInternalSchema); + candidateUpdateCols.stream().forEach(f -> updateChange.updateColumnNullability(f, true)); + Schema result = AvroInternalSchemaConverter.convert(SchemaChangeUtils.applyTableChanges2Schema(writeInternalSchema, updateChange), writeSchema.getName()); + return result; + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java new file mode 100644 index 0000000000000..fc9656803e599 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java @@ -0,0 +1,856 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.internal.schema.HoodieSchemaException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.internal.schema.Types.RecordType; +import org.apache.hudi.internal.schema.action.InternalSchemaMerger; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; +import org.apache.hudi.internal.schema.visitor.InternalSchemaVisitor; +import org.apache.hudi.internal.schema.visitor.NameToIDVisitor; + +import java.util.ArrayList; +import java.util.Deque; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +public class InternalSchemaUtils { + + private InternalSchemaUtils() { + } + + /** + * build a mapping from id to full field name for a internal Type. + * if a field y belong to a struct filed x, then the full name of y is x.y + * + * @param type hoodie internal type + * @return a mapping from id to full field name + */ + public static Map buildIdToName(Type type) { + Map result = new HashMap<>(); + buildNameToId(type).forEach((k, v) -> result.put(v, k)); + return result; + } + + /** + * build a mapping from full field name to id for a internal Type. + * if a field y belong to a struct filed x, then the full name of y is x.y + * + * @param type hoodie internal type + * @return a mapping from full field name to id + */ + public static Map buildNameToId(Type type) { + return visit(type, new NameToIDVisitor()); + } + + /** + * use to traverse all types in internalSchema with visitor. + * + * @param schema hoodie internal schema + * @return vistor expected result. + */ + public static T visit(InternalSchema schema, InternalSchemaVisitor visitor) { + return visitor.schema(schema, visit(schema.getRecord(), visitor)); + } + + public static T visit(Type type, InternalSchemaVisitor visitor) { + switch (type.typeId()) { + case RECORD: + RecordType record = (RecordType) type; + List results = new ArrayList<>(); + for (Types.Field f : record.fields()) { + visitor.beforeField(f); + T result; + try { + result = visit(f.type(), visitor); + } finally { + visitor.afterField(f); + } + results.add(visitor.field(f, result)); + } + return visitor.record(record, results); + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + T elementResult; + Types.Field elementField = array.field(array.elementId()); + visitor.beforeArrayElement(elementField); + try { + elementResult = visit(elementField.type(), visitor); + } finally { + visitor.afterArrayElement(elementField); + } + return visitor.array(array, elementResult); + case MAP: + Types.MapType map = (Types.MapType) type; + T keyResult; + T valueResult; + Types.Field keyField = map.field(map.keyId()); + visitor.beforeMapKey(keyField); + try { + keyResult = visit(map.keyType(), visitor); + } finally { + visitor.afterMapKey(keyField); + } + Types.Field valueField = map.field(map.valueId()); + visitor.beforeMapValue(valueField); + try { + valueResult = visit(map.valueType(), visitor); + } finally { + visitor.afterMapValue(valueField); + } + return visitor.map(map, keyResult, valueResult); + default: + return visitor.primitive((Type.PrimitiveType)type); + } + } + + /** + * build a mapping from id to field for a internal Type. + * + * @param type hoodie internal type + * @return a mapping from id to field + */ + public static Map buildIdToField(Type type) { + Map idToField = new HashMap<>(); + visitIdToField(type, idToField); + return idToField; + } + + private static void visitIdToField(Type type, Map index) { + switch (type.typeId()) { + case RECORD: + RecordType record = (RecordType) type; + for (Field field : record.fields()) { + visitIdToField(field.type(), index); + index.put(field.fieldId(), field); + } + return; + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + visitIdToField(array.elementType(), index); + for (Field field : array.fields()) { + index.put(field.fieldId(), field); + } + return; + case MAP: + Types.MapType map = (Types.MapType) type; + visitIdToField(map.keyType(), index); + visitIdToField(map.valueType(), index); + for (Field field : map.fields()) { + index.put(field.fieldId(), field); + } + return; + default: + return; + } + } + + public static String createFullName(String name, Deque fieldNames) { + String result = name; + if (!fieldNames.isEmpty()) { + List parentNames = new ArrayList<>(); + fieldNames.descendingIterator().forEachRemaining(parentNames::add); + result = parentNames.stream().collect(Collectors.joining(".")) + "." + result; + } + return result; + } + + /** + * build a mapping which maintain the relation between child field id and it's parent field id. + * if a child field y(which id is 9) belong to a nest field x(which id is 6), then (9 -> 6) will be added to the result map. + * if a field has no parent field, nothings will be added. + * + * @param record hoodie record type. + * @return a mapping from id to parentId for a record Type + */ + public static Map index2Parents(Types.RecordType record) { + Map result = new HashMap<>(); + Deque parentIds = new LinkedList<>(); + index2Parents(record, parentIds, result); + return result; + } + + private static void index2Parents(Type type, Deque pids, Map id2p) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType)type; + for (Field f : record.fields()) { + pids.push(f.fieldId()); + index2Parents(f.type(), pids, id2p); + pids.pop(); + } + + for (Field f : record.fields()) { + // root record has no parent id. + if (!pids.isEmpty()) { + Integer pid = pids.peek(); + id2p.put(f.fieldId(), pid); + } + } + return; + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Types.Field elementField = array.field(array.elementId()); + pids.push(elementField.fieldId()); + index2Parents(elementField.type(), pids, id2p); + pids.pop(); + id2p.put(array.elementId(), pids.peek()); + return; + case MAP: + Types.MapType map = (Types.MapType) type; + Types.Field keyField = map.field(map.keyId()); + Types.Field valueField = map.field(map.valueId()); + // visit key + pids.push(map.keyId()); + index2Parents(keyField.type(), pids, id2p); + pids.pop(); + // visit value + pids.push(map.valueId()); + index2Parents(valueField.type(), pids, id2p); + pids.pop(); + id2p.put(map.keyId(), pids.peek()); + id2p.put(map.valueId(), pids.peek()); + return; + default: + } + } + + /** + * Assigns new ids for all fields in a Type, based on initial id. + * + * @param type a type. + * @param nextId initial id which used to fresh ids for all fields in a type + * @return a new type with new ids + */ + public static Type refreshNewId(Type type, AtomicInteger nextId) { + switch (type.typeId()) { + case RECORD: + RecordType record = (RecordType) type; + List oldFields = record.fields(); + int currentId = nextId.get(); + nextId.set(currentId + record.fields().size()); + List internalFields = new ArrayList<>(); + for (int i = 0; i < oldFields.size(); i++) { + Field oldField = oldFields.get(i); + Type fieldType = refreshNewId(oldField.type(), nextId); + internalFields.add(Types.Field.get(currentId++, oldField.isOptional(), oldField.name(), fieldType, oldField.doc())); + } + return Types.RecordType.get(internalFields); + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + int elementId = nextId.get(); + nextId.set(elementId + 1); + Type elementType = refreshNewId(array.elementType(), nextId); + return Types.ArrayType.get(elementId, array.isElementOptional(), elementType); + case MAP: + Types.MapType map = (Types.MapType) type; + int keyId = nextId.get(); + int valueId = keyId + 1; + nextId.set(keyId + 2); + Type keyType = refreshNewId(map.keyType(), nextId); + Type valueType = refreshNewId(map.valueType(), nextId); + return Types.MapType.get(keyId, valueId, keyType, valueType, map.isValueOptional()); + default: + return type; + } + } + + /** + * build hudi type from avro schema. + * + * @param schema a avro schema. + * @return a hudi type. + */ + public static Type buildTypeFromAvroSchema(Schema schema) { + // set flag to check this has not been visited. + Deque visited = new LinkedList(); + AtomicInteger nextId = new AtomicInteger(1); + return visitAvroSchemaToBuildType(schema, visited, true, nextId); + } + + /** + * converts an avro schema into hudi type. + * + * @param schema a avro schema. + * @param visited track the visit node when do traversal for avro schema; used to check if the name of avro record schema is correct. + * @param firstVisitRoot track whether the current visited schema node is a root node. + * @param nextId a initial id which used to create id for all fields. + * @return a hudi type match avro schema. + */ + private static Type visitAvroSchemaToBuildType(Schema schema, Deque visited, Boolean firstVisitRoot, AtomicInteger nextId) { + switch (schema.getType()) { + case RECORD: + String name = schema.getFullName(); + if (visited.contains(name)) { + throw new HoodieSchemaException(String.format("cannot convert recursive avro record %s", name)); + } + visited.push(name); + List fields = schema.getFields(); + List fieldTypes = new ArrayList<>(fields.size()); + int nextAssignId = nextId.get(); + // when first visit root record, set nextAssignId = 0; + if (firstVisitRoot) { + nextAssignId = 0; + } + nextId.set(nextAssignId + fields.size()); + fields.stream().forEach(field -> { + fieldTypes.add(visitAvroSchemaToBuildType(field.schema(), visited, false, nextId)); + }); + visited.pop(); + List internalFields = new ArrayList<>(fields.size()); + + for (int i = 0; i < fields.size(); i++) { + Schema.Field field = fields.get(i); + Type fieldType = fieldTypes.get(i); + internalFields.add(Types.Field.get(nextAssignId, AvroInternalSchemaConverter.isOptional(field.schema()), field.name(), fieldType, field.doc())); + nextAssignId += 1; + } + return Types.RecordType.get(internalFields); + case UNION: + List fTypes = new ArrayList<>(); + schema.getTypes().stream().forEach(t -> { + fTypes.add(visitAvroSchemaToBuildType(t, visited, false, nextId)); + }); + return fTypes.get(0) == null ? fTypes.get(1) : fTypes.get(0); + case ARRAY: + Schema elementSchema = schema.getElementType(); + int elementId = nextId.get(); + nextId.set(elementId + 1); + Type elementType = visitAvroSchemaToBuildType(elementSchema, visited, false, nextId); + return Types.ArrayType.get(elementId, AvroInternalSchemaConverter.isOptional(schema.getElementType()), elementType); + case MAP: + int keyId = nextId.get(); + int valueId = keyId + 1; + nextId.set(valueId + 1); + Type valueType = visitAvroSchemaToBuildType(schema.getValueType(), visited, false, nextId); + return Types.MapType.get(keyId, valueId, Types.StringType.get(), valueType, AvroInternalSchemaConverter.isOptional(schema.getValueType())); + default: + return visitAvroPrimitiveToBuildInternalType(schema); + } + } + + private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) { + LogicalType logical = primitive.getLogicalType(); + if (logical != null) { + String name = logical.getName(); + if (logical instanceof LogicalTypes.Decimal) { + return Types.DecimalType.get( + ((LogicalTypes.Decimal) logical).getPrecision(), + ((LogicalTypes.Decimal) logical).getScale()); + + } else if (logical instanceof LogicalTypes.Date) { + return Types.DateType.get(); + + } else if ( + logical instanceof LogicalTypes.TimeMillis + || logical instanceof LogicalTypes.TimeMicros) { + return Types.TimeType.get(); + + } else if ( + logical instanceof LogicalTypes.TimestampMillis + || logical instanceof LogicalTypes.TimestampMicros) { + return Types.TimestampType.get(); + } else if (LogicalTypes.uuid().getName().equals(name)) { + return Types.UUIDType.get(); + } + } + + switch (primitive.getType()) { + case BOOLEAN: + return Types.BooleanType.get(); + case INT: + return Types.IntType.get(); + case LONG: + return Types.LongType.get(); + case FLOAT: + return Types.FloatType.get(); + case DOUBLE: + return Types.DoubleType.get(); + case STRING: + case ENUM: + return Types.StringType.get(); + case FIXED: + return Types.FixedType.getFixed(primitive.getFixedSize()); + case BYTES: + return Types.BinaryType.get(); + case NULL: + return null; + default: + throw new UnsupportedOperationException("Unsupported primitive type: " + primitive); + } + } + + /** + * Converts hudi type into an Avro Schema. + * + * @param type a hudi type. + * @param recordName the record name + * @return a Avro schema match this type + */ + public static Schema buildAvroSchemaFromType(Type type, String recordName) { + Map cache = new HashMap<>(); + return visitInternalSchemaToBuildAvroSchema(type, cache, recordName); + } + + /** + * Converts hudi internal Schema into an Avro Schema. + * + * @param schema a hudi internal Schema. + * @param recordName the record name + * @return a Avro schema match hudi internal schema. + */ + public static Schema buildAvroSchemaFromInternalSchema(InternalSchema schema, String recordName) { + Map cache = new HashMap<>(); + return visitInternalSchemaToBuildAvroSchema(schema.getRecord(), cache, recordName); + } + + /** + * Converts hudi type into an Avro Schema. + * + * @param type a hudi type. + * @param cache use to cache intermediate convert result to save cost. + * @param recordName the record name + * @return a Avro schema match this type + */ + private static Schema visitInternalSchemaToBuildAvroSchema(Type type, Map cache, String recordName) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType) type; + List schemas = new ArrayList<>(); + record.fields().forEach(f -> { + Schema tempSchema = visitInternalSchemaToBuildAvroSchema(f.type(), cache, recordName + "_" + f.name()); + // convert tempSchema + Schema result = f.isOptional() ? AvroInternalSchemaConverter.nullableSchema(tempSchema) : tempSchema; + schemas.add(result); + }); + // check visited + Schema recordSchema; + recordSchema = cache.get(record); + if (recordSchema != null) { + return recordSchema; + } + recordSchema = visitInternalRecordToBuildAvroRecord(record, schemas, recordName); + cache.put(record, recordSchema); + return recordSchema; + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Schema elementSchema; + elementSchema = visitInternalSchemaToBuildAvroSchema(array.elementType(), cache, recordName); + Schema arraySchema; + arraySchema = cache.get(array); + if (arraySchema != null) { + return arraySchema; + } + arraySchema = visitInternalArrayToBuildAvroArray(array, elementSchema); + cache.put(array, arraySchema); + return arraySchema; + case MAP: + Types.MapType map = (Types.MapType) type; + Schema keySchema; + Schema valueSchema; + keySchema = visitInternalSchemaToBuildAvroSchema(map.keyType(), cache, recordName); + valueSchema = visitInternalSchemaToBuildAvroSchema(map.valueType(), cache, recordName); + Schema mapSchema; + mapSchema = cache.get(map); + if (mapSchema != null) { + return mapSchema; + } + mapSchema = visitInternalMapToBuildAvroMap(map, keySchema, valueSchema); + cache.put(map, mapSchema); + return mapSchema; + default: + Schema primitiveSchema = visitInternalPrimitiveToBuildAvroPrimitiveType((Type.PrimitiveType) type); + cache.put(type, primitiveSchema); + return primitiveSchema; + } + } + + /** + * Converts hudi RecordType to Avro RecordType. + * this is auxiliary function used by visitInternalSchemaToBuildAvroSchema + */ + private static Schema visitInternalRecordToBuildAvroRecord(Types.RecordType record, List fieldSchemas, String recordName) { + List fields = record.fields(); + List avroFields = new ArrayList<>(); + for (int i = 0; i < fields.size(); i++) { + Field f = fields.get(i); + Schema.Field field = new Schema.Field(f.name(), fieldSchemas.get(i), f.doc(), f.isOptional() ? JsonProperties.NULL_VALUE : null); + avroFields.add(field); + } + return Schema.createRecord(recordName, null, null, false, avroFields); + } + + /** + * Converts hudi ArrayType to Avro ArrayType. + * this is auxiliary function used by visitInternalSchemaToBuildAvroSchema + */ + private static Schema visitInternalArrayToBuildAvroArray(Types.ArrayType array, Schema elementSchema) { + Schema result; + if (array.isElementOptional()) { + result = Schema.createArray(AvroInternalSchemaConverter.nullableSchema(elementSchema)); + } else { + result = Schema.createArray(elementSchema); + } + return result; + } + + /** + * Converts hudi MapType to Avro MapType. + * this is auxiliary function used by visitInternalSchemaToBuildAvroSchema + */ + private static Schema visitInternalMapToBuildAvroMap(Types.MapType map, Schema keySchema, Schema valueSchema) { + Schema mapSchema; + if (keySchema.getType() == Schema.Type.STRING) { + mapSchema = Schema.createMap(map.isValueOptional() ? AvroInternalSchemaConverter.nullableSchema(valueSchema) : valueSchema); + } else { + throw new HoodieSchemaException("only support StringType key for avro MapType"); + } + return mapSchema; + } + + /** + * Converts hudi PrimitiveType to Avro PrimitiveType. + * this is auxiliary function used by visitInternalSchemaToBuildAvroSchema + */ + private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.PrimitiveType primitive) { + Schema primitiveSchema; + switch (primitive.typeId()) { + case BOOLEAN: + primitiveSchema = Schema.create(Schema.Type.BOOLEAN); + break; + case INT: + primitiveSchema = Schema.create(Schema.Type.INT); + break; + case LONG: + primitiveSchema = Schema.create(Schema.Type.LONG); + break; + case FLOAT: + primitiveSchema = Schema.create(Schema.Type.FLOAT); + break; + case DOUBLE: + primitiveSchema = Schema.create(Schema.Type.DOUBLE); + break; + case DATE: + primitiveSchema = LogicalTypes.date() + .addToSchema(Schema.create(Schema.Type.INT)); + break; + case TIME: + primitiveSchema = LogicalTypes.timeMicros() + .addToSchema(Schema.create(Schema.Type.LONG)); + break; + case TIMESTAMP: + primitiveSchema = LogicalTypes.timestampMicros() + .addToSchema(Schema.create(Schema.Type.LONG)); + break; + case STRING: + primitiveSchema = Schema.create(Schema.Type.STRING); + break; + case UUID: + primitiveSchema = LogicalTypes.uuid() + .addToSchema(Schema.createFixed("uuid_fixed", null, null, 16)); + break; + case FIXED: + Types.FixedType fixed = (Types.FixedType) primitive; + primitiveSchema = Schema.createFixed("fixed_" + fixed.getFixedSize(), null, null, fixed.getFixedSize()); + break; + case BINARY: + primitiveSchema = Schema.create(Schema.Type.BYTES); + break; + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + primitiveSchema = LogicalTypes.decimal(decimal.precision(), decimal.scale()) + .addToSchema(Schema.createFixed( + "decimal_" + decimal.precision() + "_" + decimal.scale(), + null, null, computeMinBytesForPrecision(decimal.precision()))); + break; + default: + throw new UnsupportedOperationException( + "Unsupported type ID: " + primitive.typeId()); + } + return primitiveSchema; + } + + /** + * return the minimum number of bytes needed to store a decimal with a give 'precision'. + * reference from Spark release 3.1 . + */ + private static int computeMinBytesForPrecision(int precision) { + int numBytes = 1; + while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { + numBytes += 1; + } + return numBytes; + } + + /** + * create final read schema to read avro/parquet file. + * + * @param fileSchema the real schema of avro/parquet file. + * @param querySchema the query schema which query engine produced. + * @return read schema to read avro/parquet file. + */ + public static InternalSchema mergeSchema(InternalSchema fileSchema, InternalSchema querySchema) { + return mergeSchema(fileSchema, querySchema, true, true); + } + + /** + * create final read schema to read avro/parquet file. + * + * @param fileSchema the real schema of avro/parquet file. + * @param querySchema the query schema which query engine produced. + * @param ignoreRequiredAttribute now sparksql will change col nullability attribute from optional to required which is a bug. + * if this situation occur and mergeRequiredFiledForce is set to be true, + * just ignore the nullability changes, and merge force. + * @param useColumnTypeFromFileSchema Whether to use column Type from file schema to read files when we find some column type changed. + * when read parquet/orc file this value should be set to true, + * when read log file this value should be set to false. + * @return read schema to read avro/parquet file. + */ + public static InternalSchema mergeSchema(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) { + InternalSchemaMerger schemaMerger = new InternalSchemaMerger(fileSchema, querySchema, ignoreRequiredAttribute, useColumnTypeFromFileSchema); + Types.RecordType record = (Types.RecordType) mergeType(schemaMerger, querySchema.getRecord(), 0); + return new InternalSchema(record.fields()); + } + + + /** + * create final read schema to read avro/parquet file. + * this is auxiliary function used by mergeSchema. + */ + private static Type mergeType(InternalSchemaMerger schemaMerger, Type type, int currentTypeId) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType) type; + List newTypes = new ArrayList<>(); + for (Types.Field f : record.fields()) { + Type newType = mergeType(schemaMerger, f.type(), f.fieldId()); + newTypes.add(newType); + } + return Types.RecordType.get(schemaMerger.buildRecordType(record.fields(), newTypes)); + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Type newElementType; + Types.Field elementField = array.fields().get(0); + newElementType = mergeType(schemaMerger, elementField.type(), elementField.fieldId()); + return schemaMerger.buildArrayType(array, newElementType); + case MAP: + Types.MapType map = (Types.MapType) type; + Type newValueType = mergeType(schemaMerger, map.valueType(), map.valueId()); + return schemaMerger.buildMapType(map, newValueType); + default: + return schemaMerger.buildPrimitiveType((Type.PrimitiveType) type, currentTypeId); + } + } + + /** + * create project internalSchema, based on the project names which produced by query engine. + * support nested project. + * + * @param schema a internal schema. + * @param names project names produced by query engine. + * @return a project internalSchema. + */ + public static InternalSchema pruneInternalSchema(InternalSchema schema, List names) { + // do check + List prunedIds = names.stream().map(name -> { + int id = schema.findIdByName(name); + if (id == -1) { + throw new IllegalArgumentException(String.format("cannot prune col: %s which not exisit in hudi table", name)); + } + return id; + }).collect(Collectors.toList()); + // find top parent field ID. eg: a.b.c, f.g.h, only collect id of a and f ignore all child field. + List topParentFieldIds = new ArrayList<>(); + names.stream().forEach(f -> { + int id = schema.findIdByName(f.split("\\.")[0]); + if (!topParentFieldIds.contains(id)) { + topParentFieldIds.add(id); + } + }); + return pruneInternalSchemaByID(schema, prunedIds, topParentFieldIds); + } + + /** + * create project internalSchema. + * support nested project. + * + * @param schema a internal schema. + * @param fieldIds project col field_ids. + * @return a project internalSchema. + */ + public static InternalSchema pruneInternalSchemaByID(InternalSchema schema, List fieldIds, List topParentFieldIds) { + Types.RecordType recordType = (Types.RecordType)pruneType(schema.getRecord(), fieldIds); + // reorder top parent fields, since the recordType.fields() produced by pruneType maybe out of order. + List newFields = new ArrayList<>(); + if (topParentFieldIds != null && !topParentFieldIds.isEmpty()) { + for (int id : topParentFieldIds) { + Types.Field f = recordType.field(id); + if (f != null) { + newFields.add(f); + } else { + throw new HoodieSchemaException(String.format("cannot find pruned id %s in currentSchema %s", id, schema.toString())); + } + } + } + return new InternalSchema(newFields.isEmpty() ? recordType.fields() : newFields); + } + + /** + * project hudi type by projected cols field_ids + * this is auxiliary function used by pruneInternalSchema. + */ + private static Type pruneType(Type type, List fieldIds) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType) type; + List fields = record.fields(); + List newTypes = new ArrayList<>(); + for (Types.Field f : fields) { + Type newType = pruneType(f.type(), fieldIds); + if (fieldIds.contains(f.fieldId())) { + newTypes.add(f.type()); + } else if (newType != null) { + newTypes.add(newType); + } else { + newTypes.add(null); + } + } + boolean changed = false; + List newFields = new ArrayList<>(); + for (int i = 0; i < fields.size(); i++) { + Types.Field oldField = fields.get(i); + Type newType = newTypes.get(i); + if (oldField.type() == newType) { + newFields.add(oldField); + } else if (newType != null) { + changed = true; + newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc())); + } + } + if (newFields.isEmpty()) { + return null; + } + if (newFields.size() == fields.size() && !changed) { + return record; + } else { + return Types.RecordType.get(newFields); + } + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Type newElementType = pruneType(array.elementType(), fieldIds); + if (fieldIds.contains(array.elementId())) { + return array; + } else if (newElementType != null) { + if (array.elementType() == newElementType) { + return array; + } + return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType); + } + return null; + case MAP: + Types.MapType map = (Types.MapType) type; + Type newValueType = pruneType(map.valueType(), fieldIds); + if (fieldIds.contains(map.valueId())) { + return map; + } else if (newValueType != null) { + if (map.valueType() == newValueType) { + return map; + } + return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional()); + } + return null; + default: + return null; + } + } + + /** + * a helper function to help correct the colName of pushed filters. + * + * @param name origin col name from pushed filters. + * @param fileSchema the real schema of avro/parquet file. + * @param querySchema the query schema which query engine produced. + * @return a corrected name. + */ + public static String reBuildFilterName(String name, InternalSchema fileSchema, InternalSchema querySchema) { + int nameId = querySchema.findIdByName(name); + if (nameId == -1) { + throw new IllegalArgumentException(String.format("cannot found filter col name:%s from querySchema: %s", name, querySchema)); + } + if (fileSchema.findField(nameId) == null) { + // added operation found + // the read file does not contain current col, so current colFilter is invalid + return ""; + } else { + if (name.equals(fileSchema.findfullName(nameId))) { + // no change happened on current col + return name; + } else { + // find rename operation on current col + // return the name from fileSchema + return fileSchema.findfullName(nameId); + } + } + } + + /** + * collect all type changed cols to build a colPosition -> (newColType, oldColType) map. + * only collect top level col changed. eg: a is a nest field(record(b int, d long), now a.b is changed from int to long, + * only a will be collected, a.b will excluded. + * + * @param schema a type changed internalSchema + * @param oldSchema an old internalSchema. + * @return a map. + */ + public static Map> collectTypeChangedCols(InternalSchema schema, InternalSchema oldSchema) { + Set ids = schema.getAllIds(); + Set otherIds = oldSchema.getAllIds(); + Map> result = new HashMap<>(); + ids.stream().filter(f -> otherIds.contains(f)).forEach(f -> { + if (!schema.findType(f).equals(oldSchema.findType(f))) { + String[] fieldNameParts = schema.findfullName(f).split("\\."); + String[] otherFieldNameParts = oldSchema.findfullName(f).split("\\."); + String parentName = fieldNameParts[0]; + String otherParentName = otherFieldNameParts[0]; + if (fieldNameParts.length == otherFieldNameParts.length && schema.findIdByName(parentName) == oldSchema.findIdByName(otherParentName)) { + int index = schema.findIdByName(parentName); + int position = schema.getRecord().fields().stream().map(s -> s.fieldId()).collect(Collectors.toList()).indexOf(index); + if (!result.containsKey(position)) { + result.put(position, Pair.of(schema.findType(parentName), oldSchema.findType(otherParentName))); + } + } + } + }); + return result; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangePersistHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangePersistHelper.java new file mode 100644 index 0000000000000..3c9013d04d7f2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangePersistHelper.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.action.TableChange; +import org.apache.hudi.internal.schema.action.TableChanges; +import org.apache.hudi.internal.schema.action.TableChangesHelper; + +import java.util.Arrays; + +public class SchemaChangePersistHelper { + private SchemaChangePersistHelper() {} + + /** + * add columns to table. + * + * @param latestSchema latest internal schema. + * @param colName col name to be added. if we want to add col to a nested filed, the fullName should be specify + * @param colType col type to be added. + * @param doc col doc to be added. + * @param position col position to be added + * @param positionType col position change type. now support three change types: first/after/before + */ + public static InternalSchema applyAddChange( + InternalSchema latestSchema, + String colName, + Type colType, + String doc, + String position, + TableChange.ColumnPositionChange.ColumnPositionType positionType) { + TableChanges.ColumnAddChange add = TableChanges.ColumnAddChange.get(latestSchema); + String parentName = TableChangesHelper.getParentName(colName); + add.addColumns(parentName, colName, colType, doc); + if (positionType != null) { + switch (positionType) { + case NO_OPERATION: + break; + case FIRST: + add.addPositionChange(colName, "", positionType); + break; + case AFTER: + case BEFORE: + if (position == null || position.isEmpty()) { + throw new IllegalArgumentException("position should not be null/empty_string when specify positionChangeType as after/before"); + } + String referParentName = TableChangesHelper.getParentName(position); + if (!parentName.equals(referParentName)) { + throw new IllegalArgumentException("cannot reorder two columns which has different parent"); + } + add.addPositionChange(colName, position, positionType); + break; + default: + throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", positionType)); + } + } else { + throw new IllegalArgumentException(String.format("positionType should be specified")); + } + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, add); + } + + /** + * delete columns to table. + * + * @param latestSchema latest internal schema. + * @param colNames col name to be deleted. if we want to delete col from a nested filed, the fullName should be specify + */ + public static InternalSchema applyDeleteChange(InternalSchema latestSchema, String... colNames) { + TableChanges.ColumnDeleteChange delete = TableChanges.ColumnDeleteChange.get(latestSchema); + Arrays.stream(colNames).forEach(colName -> delete.deleteColumn(colName)); + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, delete); + } + + /** + * rename col name for hudi table. + * + * @param latestSchema latest internal schema. + * @param colName col name to be renamed. if we want to rename col from a nested filed, the fullName should be specify + * @param newName new name for current col. no need to specify fullName. + */ + public static InternalSchema applyRenameChange(InternalSchema latestSchema, String colName, String newName) { + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema); + updateChange.renameColumn(colName, newName); + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange); + } + + /** + * update col nullability for hudi table. + * + * @param latestSchema latest internal schema. + * @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify + * @param nullable . + */ + public static InternalSchema applyColumnNullabilityChange(InternalSchema latestSchema, String colName, boolean nullable) { + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema); + updateChange.updateColumnNullability(colName, nullable); + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange); + } + + /** + * update col type for hudi table. + * + * @param latestSchema latest internal schema. + * @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify + * @param newType . + */ + public static InternalSchema applyColumnTypeChange(InternalSchema latestSchema, String colName, Type newType) { + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema); + updateChange.updateColumnType(colName, newType); + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange); + } + + /** + * update col comment for hudi table. + * + * @param latestSchema latest internal schema. + * @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify + * @param doc . + */ + public static InternalSchema applyColumnCommentChange(InternalSchema latestSchema, String colName, String doc) { + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema); + updateChange.updateColumnComment(colName, doc); + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange); + } + + /** + * reorder the position of col. + * + * @param latestSchema latest internal schema. + * @param colName column which need to be reordered. if we want to change col from a nested filed, the fullName should be specify. + * @param referColName reference position. + * @param positionType col position change type. now support three change types: first/after/before + */ + public static InternalSchema applyReOrderColPositionChange( + InternalSchema latestSchema, + String colName, + String referColName, + TableChange.ColumnPositionChange.ColumnPositionType positionType) { + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema); + String parentName = TableChangesHelper.getParentName(colName); + String referParentName = TableChangesHelper.getParentName(referColName); + if (positionType.equals(TableChange.ColumnPositionChange.ColumnPositionType.FIRST)) { + updateChange.addPositionChange(colName, "", positionType); + } else if (parentName.equals(referParentName)) { + updateChange.addPositionChange(colName, referColName, positionType); + } else { + throw new IllegalArgumentException("cannot reorder two columns which has different parent"); + } + return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange); + } +} 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 new file mode 100644 index 0000000000000..3e554566b5386 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.action.TableChanges; +import org.apache.hudi.internal.schema.action.TableChangesHelper; + +import java.util.ArrayList; +import java.util.List; + +/** + * Helper methods for schema Change. + */ +public class SchemaChangeUtils { + private SchemaChangeUtils() { + + } + + /** + * whether to allow the column type to be updated. + * now only support: + * int => long/float/double/string + * long => float/double/string + * float => double/String + * double => String/Decimal + * Decimal => Decimal/String + * String => date/decimal + * date => String + * TODO: support more type update. + * + * @param src origin column type. + * @param dsr new column type. + * @return whether to allow the column type to be updated. + */ + public static boolean isTypeUpdateAllow(Type src, Type dsr) { + if (src.isNestedType() || dsr.isNestedType()) { + throw new IllegalArgumentException("only support update primitive type"); + } + if (src.equals(dsr)) { + return true; + } + switch (src.typeId()) { + case INT: + return dsr == Types.LongType.get() || dsr == Types.FloatType.get() + || dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + case LONG: + return dsr == Types.FloatType.get() || dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + case FLOAT: + return dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + case DOUBLE: + return dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + case DATE: + return dsr == Types.StringType.get(); + case DECIMAL: + if (dsr.typeId() == Type.TypeID.DECIMAL) { + Types.DecimalType decimalSrc = (Types.DecimalType)src; + Types.DecimalType decimalDsr = (Types.DecimalType)dsr; + if (decimalDsr.isWiderThan(decimalSrc)) { + return true; + } + } else if (dsr.typeId() == Type.TypeID.STRING) { + return true; + } + break; + case STRING: + return dsr == Types.DateType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + default: + return false; + } + return false; + } + + /** + * Apply all the DDL add operations to internalSchema to produce a new internalSchema. + * + * @param internalSchema origin internalSchema. + * @param adds a wrapper class for all the DDL add operations. + * @return a new internalSchema. + */ + public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnAddChange adds) { + Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), adds); + // deal with root level changes + List newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(), + adds.getParentId2AddCols().get(-1), adds.getPositionChangeMap().get(-1)); + return new InternalSchema(newFields); + } + + /** + * Apply all the DDL add operations to Type to produce a new internalSchema. + * do not call this method directly. expose this method only for UT. + * + * @param type origin hudi Type. + * @param adds a wrapper class for all the DDL add operations. + * @return a new internalSchema. + */ + public static Type applyTableChange2Type(Type type, TableChanges.ColumnAddChange adds) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType) type; + List newTypes = new ArrayList<>(); + for (Types.Field f : record.fields()) { + Type newType = applyTableChange2Type(f.type(), adds); + // try to apply add + newTypes.add(newType.isNestedType() ? adds.applyAdd(f, newType) : newType); + } + List newFields = new ArrayList<>(); + boolean hasChanged = false; + for (int i = 0; i < newTypes.size(); i++) { + Type newType = newTypes.get(i); + Types.Field oldfield = record.fields().get(i); + if (oldfield.type() == newType) { + newFields.add(oldfield); + } else { + hasChanged = true; + newFields.add(Types.Field.get(oldfield.fieldId(), oldfield.isOptional(), oldfield.name(), newType, oldfield.doc())); + } + } + return hasChanged ? Types.RecordType.get(newFields) : record; + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Type newElementType; + Types.Field elementField = array.field(array.elementId()); + newElementType = applyTableChange2Type(array.elementType(), adds); + // try to apply add + newElementType = adds.applyAdd(elementField, newElementType); + if (newElementType == array.elementType()) { + return array; + } + return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType); + case MAP: + Types.MapType map = (Types.MapType) type; + Type newValueType; + Types.Field valueField = map.field(map.valueId()); + if (adds.getParentId2AddCols().containsKey(map.keyId())) { + throw new IllegalArgumentException("Cannot add fields to map keys: " + map); + } + newValueType = applyTableChange2Type(map.valueType(), adds); + // try to apply add + newValueType = adds.applyAdd(valueField, newValueType); + if (newValueType == map.valueType()) { + return map; + } + return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional()); + default: + return type; + } + } + + /** + * Apply all the DDL delete operations to internalSchema to produce a new internalSchema. + * + * @param internalSchema origin internalSchema. + * @param deletes a wrapper class for all the DDL delete operations. + * @return a new internalSchema. + */ + public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnDeleteChange deletes) { + Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), deletes); + return new InternalSchema(newType.fields()); + } + + /** + * Apply all the DDL delete operations to Type to produce a new internalSchema. + * do not call this method directly. expose this method only for UT. + * + * @param type origin type. + * @param deletes a wrapper class for all the DDL delete operations. + * @return a new internalSchema. + */ + public static Type applyTableChange2Type(Type type, TableChanges.ColumnDeleteChange deletes) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType) type; + List fields = new ArrayList<>(); + for (Types.Field f : record.fields()) { + Type newType = applyTableChange2Type(f.type(), deletes); + // apply delete + newType = deletes.applyDelete(f.fieldId(), newType); + if (newType != null) { + fields.add(Types.Field.get(f.fieldId(), f.isOptional(), f.name(), newType, f.doc())); + } + } + if (fields.isEmpty()) { + throw new UnsupportedOperationException("cannot support delete all columns from Struct"); + } + return Types.RecordType.get(fields); + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Type newElementType = applyTableChange2Type(array.elementType(), deletes); + newElementType = deletes.applyDelete(array.elementId(), newElementType); + if (newElementType == null) { + throw new IllegalArgumentException(String.format("cannot delete element from arrayType: %s", array)); + } + return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType); + case MAP: + Types.MapType map = (Types.MapType) type; + int keyId = map.fields().get(0).fieldId(); + if (deletes.getDeletes().contains(keyId)) { + throw new IllegalArgumentException(String.format("cannot delete key from mapType: %s", map)); + } + Type newValueType = applyTableChange2Type(map.valueType(), deletes); + newValueType = deletes.applyDelete(map.valueId(), newValueType); + if (newValueType == null) { + throw new IllegalArgumentException(String.format("cannot delete value from mapType: %s", map)); + } + return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional()); + default: + return type; + } + } + + /** + * Apply all the DDL update operations to internalSchema to produce a new internalSchema. + * + * @param internalSchema origin internalSchema. + * @param updates a wrapper class for all the DDL update operations. + * @return a new internalSchema. + */ + public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnUpdateChange updates) { + Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), updates); + // deal with root level changes + List newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(), + new ArrayList<>(), updates.getPositionChangeMap().get(-1)); + return new InternalSchema(newFields); + } + + /** + * Apply all the DDL update operations to type to produce a new internalSchema. + * do not call this method directly. expose this method only for UT. + * + * @param type origin internalSchema. + * @param updates a wrapper class for all the DDL update operations. + * @return a new internalSchema. + */ + public static Type applyTableChange2Type(Type type, TableChanges.ColumnUpdateChange updates) { + switch (type.typeId()) { + case RECORD: + Types.RecordType record = (Types.RecordType) type; + List newTypes = new ArrayList<>(); + for (Types.Field f : record.fields()) { + Type newType = applyTableChange2Type(f.type(), updates); + newTypes.add(updates.applyUpdates(f, newType)); + } + List newFields = new ArrayList<>(); + for (int i = 0; i < newTypes.size(); i++) { + Type newType = newTypes.get(i); + Types.Field oldField = record.fields().get(i); + Types.Field updateField = updates.getUpdates().get(oldField.fieldId()); + if (updateField != null) { + newFields.add(Types.Field.get(oldField.fieldId(), updateField.isOptional(), updateField.name(), newType, updateField.doc())); + } else if (!oldField.type().equals(newType)) { + newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc())); + } else { + newFields.add(oldField); + } + } + return Types.RecordType.get(newFields); + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + Type newElementType; + Types.Field elementField = array.fields().get(0); + newElementType = applyTableChange2Type(array.elementType(), updates); + newElementType = updates.applyUpdates(elementField, newElementType); + Types.Field elementUpdate = updates.getUpdates().get(elementField.fieldId()); + boolean optional = elementUpdate == null ? array.isElementOptional() : elementUpdate.isOptional(); + if (optional == elementField.isOptional() && array.elementType() == newElementType) { + return array; + } + return Types.ArrayType.get(array.elementId(), optional, newElementType); + case MAP: + Types.MapType map = (Types.MapType) type; + Types.Field valueFiled = map.fields().get(1); + Type newValueType; + newValueType = applyTableChange2Type(map.valueType(), updates); + newValueType = updates.applyUpdates(valueFiled, newValueType); + Types.Field valueUpdate = updates.getUpdates().get(valueFiled.fieldId()); + boolean valueOptional = valueUpdate == null ? map.isValueOptional() : valueUpdate.isOptional(); + if (valueOptional == map.isValueOptional() && map.valueType() == newValueType) { + return map; + } + return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, valueOptional); + default: + return type; + } + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java new file mode 100644 index 0000000000000..b14dc884279e5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java @@ -0,0 +1,384 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class SerDeHelper { + private SerDeHelper() { + + } + + public static final String LATESTSCHEMA = "latestSchema"; + public static final String SCHEMAS = "schemas"; + private static final String MAX_COLUMN_ID = "max_column_id"; + private static final String VERSION_ID = "version-id"; + private static final String TYPE = "type"; + private static final String RECORD = "record"; + private static final String ARRAY = "array"; + private static final String MAP = "map"; + private static final String FIELDS = "fields"; + private static final String ELEMENT = "element"; + private static final String KEY = "key"; + private static final String VALUE = "value"; + private static final String DOC = "doc"; + private static final String NAME = "name"; + private static final String ID = "id"; + private static final String ELEMENT_ID = "element-id"; + private static final String KEY_ID = "key-id"; + private static final String VALUE_ID = "value-id"; + private static final String OPTIONAL = "optional"; + private static final String ELEMENT_OPTIONAL = "element_optional"; + private static final String VALUE_OPTIONAL = "value_optional"; + + private static final Pattern FIXED = Pattern.compile("fixed\\[(\\d+)\\]"); + private static final Pattern DECIMAL = Pattern.compile("decimal\\((\\d+),\\s+(\\d+)\\)"); + + /** + * convert history internalSchemas to json. + * this is used when save history schemas into hudi. + * + * @param internalSchemas history internal schemas + * @return a string + */ + public static String toJson(List internalSchemas) { + try { + StringWriter writer = new StringWriter(); + JsonGenerator generator = (new JsonFactory()).createGenerator(writer); + generator.writeStartObject(); + generator.writeArrayFieldStart(SCHEMAS); + for (InternalSchema schema : internalSchemas) { + toJson(schema, generator); + } + generator.writeEndArray(); + generator.writeEndObject(); + generator.flush(); + return writer.toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * convert internalSchemas to json. + * + * @param internalSchema a internal schema + * @return a string + */ + public static String toJson(InternalSchema internalSchema) { + if (internalSchema == null) { + return ""; + } + try { + StringWriter writer = new StringWriter(); + JsonGenerator generator = (new JsonFactory()).createGenerator(writer); + toJson(internalSchema, generator); + generator.flush(); + return writer.toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static void toJson(InternalSchema internalSchema, JsonGenerator generator) throws IOException { + toJson(internalSchema.getRecord(), internalSchema.getMaxColumnId(), internalSchema.schemaId(), generator); + } + + private static void toJson(Types.RecordType record, Integer maxColumnId, Long versionId, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + if (maxColumnId != null) { + generator.writeNumberField(MAX_COLUMN_ID, maxColumnId); + } + if (versionId != null) { + generator.writeNumberField(VERSION_ID, versionId); + } + generator.writeStringField(TYPE, RECORD); + generator.writeArrayFieldStart(FIELDS); + for (Types.Field field : record.fields()) { + generator.writeStartObject(); + generator.writeNumberField(ID, field.fieldId()); + generator.writeStringField(NAME, field.name()); + generator.writeBooleanField(OPTIONAL, field.isOptional()); + generator.writeFieldName(TYPE); + toJson(field.type(), generator); + if (field.doc() != null) { + generator.writeStringField(DOC, field.doc()); + } + generator.writeEndObject(); + } + generator.writeEndArray(); + generator.writeEndObject(); + } + + private static void toJson(Type type, JsonGenerator generator) throws IOException { + switch (type.typeId()) { + case RECORD: + toJson((Types.RecordType) type, null, null, generator); + break; + case ARRAY: + Types.ArrayType array = (Types.ArrayType) type; + generator.writeStartObject(); + generator.writeStringField(TYPE, ARRAY); + generator.writeNumberField(ELEMENT_ID, array.elementId()); + generator.writeFieldName(ELEMENT); + toJson(array.elementType(), generator); + generator.writeBooleanField(ELEMENT_OPTIONAL, array.isElementOptional()); + generator.writeEndObject(); + break; + case MAP: + Types.MapType map = (Types.MapType) type; + generator.writeStartObject(); + generator.writeStringField(TYPE, MAP); + generator.writeNumberField(KEY_ID, map.keyId()); + generator.writeFieldName(KEY); + toJson(map.keyType(), generator); + generator.writeNumberField(VALUE_ID, map.valueId()); + generator.writeFieldName(VALUE); + toJson(map.valueType(), generator); + generator.writeBooleanField(VALUE_OPTIONAL, map.isValueOptional()); + generator.writeEndObject(); + break; + default: + if (!type.isNestedType()) { + generator.writeString(type.toString()); + } else { + throw new IllegalArgumentIOException(String.format("cannot write unknown types: %s", type)); + } + } + } + + private static Type parserTypeFromJson(JsonNode jsonNode) { + if (jsonNode.isTextual()) { + String type = jsonNode.asText().toLowerCase(Locale.ROOT); + // deal with fixed and decimal + Matcher fixed = FIXED.matcher(type); + if (fixed.matches()) { + return Types.FixedType.getFixed(Integer.parseInt(fixed.group(1))); + } + Matcher decimal = DECIMAL.matcher(type); + if (decimal.matches()) { + return Types.DecimalType.get( + Integer.parseInt(decimal.group(1)), + Integer.parseInt(decimal.group(2))); + } + // deal with other type + switch (Type.fromValue(type)) { + case BOOLEAN: + return Types.BooleanType.get(); + case INT: + return Types.IntType.get(); + case LONG: + return Types.LongType.get(); + case FLOAT: + return Types.FloatType.get(); + case DOUBLE: + return Types.DoubleType.get(); + case DATE: + return Types.DateType.get(); + case TIME: + return Types.TimeType.get(); + case TIMESTAMP: + return Types.TimestampType.get(); + case STRING: + return Types.StringType.get(); + case UUID: + return Types.UUIDType.get(); + case BINARY: + return Types.BinaryType.get(); + default: + throw new IllegalArgumentException("cannot parser types from jsonNode"); + } + } else if (jsonNode.isObject()) { + String typeStr = jsonNode.get(TYPE).asText(); + if (RECORD.equals(typeStr)) { + JsonNode fieldNodes = jsonNode.get(FIELDS); + Iterator iter = fieldNodes.elements(); + List fields = new ArrayList<>(); + while (iter.hasNext()) { + JsonNode field = iter.next(); + // extract + int id = field.get(ID).asInt(); + String name = field.get(NAME).asText(); + Type type = parserTypeFromJson(field.get(TYPE)); + String doc = field.has(DOC) ? field.get(DOC).asText() : null; + boolean optional = field.get(OPTIONAL).asBoolean(); + // build fields + fields.add(Types.Field.get(id, optional, name, type, doc)); + } + return Types.RecordType.get(fields); + } else if (ARRAY.equals(typeStr)) { + int elementId = jsonNode.get(ELEMENT_ID).asInt(); + Type elementType = parserTypeFromJson(jsonNode.get(ELEMENT)); + boolean optional = jsonNode.get(ELEMENT_OPTIONAL).asBoolean(); + return Types.ArrayType.get(elementId, optional, elementType); + } else if (MAP.equals(typeStr)) { + int keyId = jsonNode.get(KEY_ID).asInt(); + Type keyType = parserTypeFromJson(jsonNode.get(KEY)); + int valueId = jsonNode.get(VALUE_ID).asInt(); + Type valueType = parserTypeFromJson(jsonNode.get(VALUE)); + boolean optional = jsonNode.get(VALUE_OPTIONAL).asBoolean(); + return Types.MapType.get(keyId, valueId, keyType, valueType, optional); + } + } + throw new IllegalArgumentException(String.format("cannot parse type from jsonNode: %s", jsonNode)); + } + + /** + * convert jsonNode to internalSchema. + * + * @param jsonNode a jsonNode. + * @return a internalSchema. + */ + public static InternalSchema fromJson(JsonNode jsonNode) { + Integer maxColumnId = !jsonNode.has(MAX_COLUMN_ID) ? null : jsonNode.get(MAX_COLUMN_ID).asInt(); + Long versionId = !jsonNode.has(VERSION_ID) ? null : jsonNode.get(VERSION_ID).asLong(); + Types.RecordType type = (Types.RecordType)parserTypeFromJson(jsonNode); + if (versionId == null) { + return new InternalSchema(type.fields()); + } else { + if (maxColumnId != null) { + return new InternalSchema(versionId, maxColumnId, type.fields()); + } else { + return new InternalSchema(versionId, type.fields()); + } + } + } + + /** + * convert string to internalSchema. + * + * @param json a json string. + * @return a internalSchema. + */ + public static Option fromJson(String json) { + if (json == null || json.isEmpty()) { + return Option.empty(); + } + try { + return Option.of(fromJson((new ObjectMapper(new JsonFactory())).readValue(json, JsonNode.class))); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * convert json string to history internalSchemas. + * TreeMap is used to hold history internalSchemas. + * + * @param json a json string + * @return a TreeMap + */ + public static TreeMap parseSchemas(String json) { + TreeMap result = new TreeMap<>(); + try { + JsonNode jsonNode = (new ObjectMapper(new JsonFactory())).readValue(json, JsonNode.class); + if (!jsonNode.has(SCHEMAS)) { + throw new IllegalArgumentException(String.format("cannot parser schemas from current json string, missing key name: %s", SCHEMAS)); + } + JsonNode schemas = jsonNode.get(SCHEMAS); + Iterator iter = schemas.elements(); + while (iter.hasNext()) { + JsonNode schema = iter.next(); + InternalSchema current = fromJson(schema); + result.put(current.schemaId(), current); + } + } catch (IOException e) { + throw new HoodieException(e); + } + return result; + } + + /** + * search target internalSchema by version number. + * + * @param versionId the internalSchema version to be search. + * @param internalSchemas internalSchemas to be searched. + * @return a internalSchema. + */ + public static InternalSchema searchSchema(long versionId, List internalSchemas) { + TreeMap treeMap = new TreeMap<>(); + internalSchemas.forEach(s -> treeMap.put(s.schemaId(), s)); + return searchSchema(versionId, treeMap); + } + + /** + * search target internalSchema by version number. + * + * @param versionId the internalSchema version to be search. + * @param treeMap internalSchemas collections to be searched. + * @return a internalSchema. + */ + public static InternalSchema searchSchema(long versionId, TreeMap treeMap) { + if (treeMap.containsKey(versionId)) { + return treeMap.get(versionId); + } else { + SortedMap headMap = treeMap.headMap(versionId); + if (!headMap.isEmpty()) { + return headMap.get(headMap.lastKey()); + } + } + return null; + } + + /** + * add the new schema to the historical schemas. + * use string operations to reduce overhead. + * + * @param newSchema a new internalSchema + * @param oldSchemas historical schemas string. + * @return a string. + */ + public static String inheritSchemas(InternalSchema newSchema, String oldSchemas) { + if (newSchema == null) { + return ""; + } + if (oldSchemas == null || oldSchemas.isEmpty()) { + return toJson(Arrays.asList(newSchema)); + } + String checkedString = "{\"schemas\":["; + if (!oldSchemas.startsWith("{\"schemas\":")) { + return ""; + } + String oldSchemasSuffix = oldSchemas.substring(checkedString.length()); + return checkedString + toJson(newSchema) + "," + oldSchemasSuffix; + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/visitor/InternalSchemaVisitor.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/visitor/InternalSchemaVisitor.java new file mode 100644 index 0000000000000..99a086f60dcce --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/visitor/InternalSchemaVisitor.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.visitor; + +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; + +import java.util.List; + +/** + * base class of schema visitor. + */ +public abstract class InternalSchemaVisitor { + + public void beforeField(Types.Field field) { + } + + public void afterField(Types.Field field) { + } + + public void beforeArrayElement(Types.Field elementField) { + beforeField(elementField); + } + + public void afterArrayElement(Types.Field elementField) { + afterField(elementField); + } + + public void beforeMapKey(Types.Field keyField) { + beforeField(keyField); + } + + public void afterMapKey(Types.Field keyField) { + afterField(keyField); + } + + public void beforeMapValue(Types.Field valueField) { + beforeField(valueField); + } + + public void afterMapValue(Types.Field valueField) { + afterField(valueField); + } + + public T schema(InternalSchema schema, T recordResult) { + return null; + } + + public T record(Types.RecordType record, List fieldResults) { + return null; + } + + public T field(Types.Field field, T fieldResult) { + return null; + } + + public T array(Types.ArrayType array, T elementResult) { + return null; + } + + public T map(Types.MapType map, T keyResult, T valueResult) { + return null; + } + + public T primitive(Type.PrimitiveType primitive) { + return null; + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/visitor/NameToIDVisitor.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/visitor/NameToIDVisitor.java new file mode 100644 index 0000000000000..5598149b839c9 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/visitor/NameToIDVisitor.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.visitor; + +import static org.apache.hudi.internal.schema.utils.InternalSchemaUtils.createFullName; + +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; + +import java.util.Deque; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * schema visitor to produce name -> id map for internalSchema. + */ +public class NameToIDVisitor extends InternalSchemaVisitor> { + private final Deque fieldNames = new LinkedList<>(); + private final Map nameToId = new HashMap<>(); + + @Override + public void beforeField(Types.Field field) { + fieldNames.push(field.name()); + } + + @Override + public void afterField(Types.Field field) { + fieldNames.pop(); + } + + @Override + public void beforeArrayElement(Types.Field elementField) { + fieldNames.push(elementField.name()); + } + + @Override + public void afterArrayElement(Types.Field elementField) { + fieldNames.pop(); + } + + @Override + public void beforeMapKey(Types.Field keyField) { + fieldNames.push(keyField.name()); + } + + @Override + public void afterMapKey(Types.Field keyField) { + fieldNames.pop(); + } + + @Override + public void beforeMapValue(Types.Field valueField) { + fieldNames.push(valueField.name()); + } + + @Override + public void afterMapValue(Types.Field valueField) { + fieldNames.pop(); + } + + @Override + public Map schema(InternalSchema schema, Map recordResult) { + return nameToId; + } + + @Override + public Map record(Types.RecordType record, List> fieldResults) { + return nameToId; + } + + @Override + public Map field(Types.Field field, Map fieldResult) { + nameToId.put(createFullName(field.name(), fieldNames), field.fieldId()); + return nameToId; + } + + @Override + public Map array(Types.ArrayType array, Map elementResult) { + nameToId.put(createFullName("element", fieldNames), array.elementId()); + return nameToId; + } + + @Override + public Map map(Types.MapType map, Map keyResult, Map valueResult) { + nameToId.put(createFullName("key", fieldNames), map.keyId()); + nameToId.put(createFullName("value", fieldNames), map.valueId()); + return nameToId; + } + + @Override + public Map primitive(Type.PrimitiveType primitive) { + return nameToId; + } +} \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/TestSerDeHelper.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/TestSerDeHelper.java new file mode 100644 index 0000000000000..1212331da0469 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/TestSerDeHelper.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema; + +import org.apache.hudi.internal.schema.utils.SerDeHelper; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Assertions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.TreeMap; + +public class TestSerDeHelper { + + @Test + public void testComplexSchema2Json() { + InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(7, false, "feature1", + Types.BooleanType.get()), Types.Field.get(8, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(13, true, + Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))), + Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())), + Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get())) + ); + // test schema2json + String result = SerDeHelper.toJson(internalSchema); + InternalSchema convertedSchema = SerDeHelper.fromJson(result).get(); + Assertions.assertEquals(internalSchema, convertedSchema); + // test schemas2json + String results = SerDeHelper.toJson(Arrays.asList(internalSchema)); + TreeMap convertedSchemas = SerDeHelper.parseSchemas(results); + Assertions.assertEquals(1, convertedSchemas.size()); + } + + @Test + public void testPrimitive2Json() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "bool", Types.BooleanType.get()), + Types.Field.get(1, "int", Types.IntType.get()), + Types.Field.get(2, "long", Types.LongType.get()), + Types.Field.get(3, "float", Types.FloatType.get()), + Types.Field.get(4, "double", Types.DoubleType.get()), + Types.Field.get(5, "date", Types.DateType.get()), + Types.Field.get(6, "time", Types.TimeType.get()), + Types.Field.get(7, "timestamp", Types.TimestampType.get()), + Types.Field.get(8, "string", Types.StringType.get()), + Types.Field.get(9, "uuid", Types.UUIDType.get()), + Types.Field.get(10, "fixed", Types.FixedType.getFixed(10)), + Types.Field.get(11, "binary", Types.BinaryType.get()), + Types.Field.get(12, "decimal", Types.DecimalType.get(10, 2)) + })); + InternalSchema internalSchema = new InternalSchema(record.fields()); + String result = SerDeHelper.toJson(internalSchema); + InternalSchema convertedSchema = SerDeHelper.fromJson(result).get(); + Assertions.assertEquals(internalSchema, convertedSchema); + } + + @Test + public void testSearchSchema() { + List schemas = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + schemas.add(new InternalSchema(i * 10, + Arrays.asList(Types.Field.get(1, true, "schema" + i * 10, Types.LongType.get())))); + } + + Assertions.assertEquals(SerDeHelper.searchSchema(0, schemas).getRecord().fields().get(0), + Types.Field.get(1, true, "schema" + 0, Types.LongType.get())); + + Assertions.assertEquals(SerDeHelper.searchSchema(9, schemas).getRecord().fields().get(0), + Types.Field.get(1, true, "schema" + 0, Types.LongType.get())); + + Assertions.assertEquals(SerDeHelper.searchSchema(99, schemas).getRecord().fields().get(0), + Types.Field.get(1, true, "schema" + 90, Types.LongType.get())); + + Assertions.assertEquals(SerDeHelper.searchSchema(9999, schemas).getRecord().fields().get(0), + Types.Field.get(1, true, "schema" + 990, Types.LongType.get())); + } + + @Test + public void testInheritSchemas() { + List schemas = new ArrayList<>(); + for (int i = 0; i < 2; i++) { + schemas.add(new InternalSchema(i, + Arrays.asList(Types.Field.get(1, true, "schema" + i, Types.LongType.get())))); + } + String oldSchemas = SerDeHelper.toJson(schemas); + InternalSchema newSchema = new InternalSchema(3, + Arrays.asList(Types.Field.get(1, true, "schema" + 3, Types.LongType.get()))); + + String finalResult = SerDeHelper.inheritSchemas(newSchema, oldSchemas); + // convert back + Assertions.assertEquals(SerDeHelper.parseSchemas(finalResult).size(), 3); + } +} + diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java new file mode 100644 index 0000000000000..e4b6d057fa913 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.action; + +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; + +import org.apache.hudi.internal.schema.utils.SchemaChangeUtils; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +public class TestMergeSchema { + + @Test + public void testPrimitiveMerge() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "col1", Types.BooleanType.get()), + Types.Field.get(1, "col2", Types.IntType.get()), + Types.Field.get(2, "col3", Types.LongType.get()), + Types.Field.get(3, "col4", Types.FloatType.get())})); + + InternalSchema oldSchema = new InternalSchema(record.fields()); + // add c1 after 'col1', and c2 before 'col3' + TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema); + addChange.addColumns("c1", Types.BooleanType.get(), "add c1 after col1"); + addChange.addPositionChange("c1", "col1", "after"); + addChange.addColumns("c2", Types.IntType.get(), "add c2 before col3"); + addChange.addPositionChange("c2", "col3", "before"); + InternalSchema newAddSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange); + TableChanges.ColumnDeleteChange deleteChange = TableChanges.ColumnDeleteChange.get(newAddSchema); + deleteChange.deleteColumn("col1"); + deleteChange.deleteColumn("col3"); + InternalSchema newDeleteSchema = SchemaChangeUtils.applyTableChanges2Schema(newAddSchema, deleteChange); + + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(newDeleteSchema); + updateChange.updateColumnType("col2", Types.LongType.get()) + .updateColumnComment("col2", "alter col2 comments") + .renameColumn("col2", "colx").addPositionChange("col2", + "col4", "after"); + InternalSchema updateSchema = SchemaChangeUtils.applyTableChanges2Schema(newDeleteSchema, updateChange); + + // add col1 again + TableChanges.ColumnAddChange addChange1 = TableChanges.ColumnAddChange.get(updateSchema); + addChange1.addColumns("col1", Types.BooleanType.get(), "add new col1"); + InternalSchema finalSchema = SchemaChangeUtils.applyTableChanges2Schema(updateSchema, addChange1); + InternalSchema mergeSchema = InternalSchemaUtils.mergeSchema(oldSchema, finalSchema); + } +} + diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestTableChanges.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestTableChanges.java new file mode 100644 index 0000000000000..5bf817f4d8b28 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestTableChanges.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.action; + +import org.apache.hudi.internal.schema.HoodieSchemaException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; + +import org.apache.hudi.internal.schema.utils.SchemaChangeUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Assertions; + +import java.util.Arrays; + +public class TestTableChanges { + + @Test + public void testPrimitiveAdd() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "col1", Types.BooleanType.get()), + Types.Field.get(1, "col2", Types.IntType.get()), + Types.Field.get(2, "col3", Types.LongType.get()), + Types.Field.get(3, "col4", Types.FloatType.get())})); + + Types.RecordType checkRecord = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "col1", Types.BooleanType.get()), + Types.Field.get(4, true, "c1", Types.BooleanType.get(), "add c1 after col1"), + Types.Field.get(1, "col2", Types.IntType.get()), + Types.Field.get(5, true, "c2", Types.IntType.get(), "add c2 before col3"), + Types.Field.get(2, "col3", Types.LongType.get()), + Types.Field.get(3, "col4", Types.FloatType.get())})); + + InternalSchema oldSchema = new InternalSchema(record.fields()); + // add c1 after 'col1', and c2 before 'col3' + TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema); + addChange.addColumns("c1", Types.BooleanType.get(), "add c1 after col1"); + // check repeated add. + Assertions.assertThrows(HoodieSchemaException.class, () -> addChange.addColumns("c1", Types.BooleanType.get(), "add c1 after col1")); + addChange.addPositionChange("c1", "col1", "after"); + addChange.addColumns("c2", Types.IntType.get(), "add c2 before col3"); + addChange.addPositionChange("c2", "col3", "before"); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange); + Assertions.assertEquals(newSchema.getRecord(), checkRecord); + } + + @Test + public void testNestAdd() { + InternalSchema oldSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(7, false, "feature1", + Types.BooleanType.get()), Types.Field.get(8, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(13, true, + Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))), + Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())), + Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get())) + ); + + TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema); + // add c1 first + addChange.addColumns("c1", Types.StringType.get(), "add c1 first"); + addChange.addPositionChange("c1", "id", "before"); + //add preferences.cx before preferences.feature2 + addChange.addColumns("preferences", "cx", Types.BooleanType.get(), "add preferences.cx before preferences.feature2"); + // check repeated add. + Assertions.assertThrows(HoodieSchemaException.class, () -> addChange.addColumns("preferences", "cx", Types.BooleanType.get(), "add preferences.cx before preferences.feature2")); + addChange.addPositionChange("preferences.cx", "preferences.feature2", "before"); + // add locations.value.lax before locations.value.long + addChange.addColumns("locations.value", "lax", Types.BooleanType.get(), "add locations.value.lax before locations.value.long"); + addChange.addPositionChange("locations.value.lax", "locations.value.long", "before"); + // + // add points.element.z after points.element.y + addChange.addColumns("points.element", "z", Types.BooleanType.get(), "add points.element.z after points.element.y"); + addChange.addPositionChange("points.element.z", "points.element.y", "after"); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange); + InternalSchema checkedSchema = new InternalSchema( + Types.Field.get(19, true, "c1", Types.StringType.get(), "add c1 first"), + Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(7, false, "feature1", Types.BooleanType.get()), + Types.Field.get(20, true, "cx", Types.BooleanType.get(), "add preferences.cx before preferences.feature2"), + Types.Field.get(8, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), + Types.Field.get(21, true, "lax", Types.BooleanType.get(), "add locations.value.lax before locations.value.long"), + Types.Field.get(12, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(13, true, + Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), + Types.Field.get(15, false, "y", Types.LongType.get()), + Types.Field.get(22, true, "z", Types.BooleanType.get(), "add points.element.z after points.element.y")))), + Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())), + Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get())) + ); + Assertions.assertEquals(newSchema.getRecord(), checkedSchema.getRecord()); + } + + @Test + public void testPrimitiveDelete() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "col1", Types.BooleanType.get()), + Types.Field.get(1, "col2", Types.IntType.get()), + Types.Field.get(2, "col3", Types.LongType.get()), + Types.Field.get(3, "col4", Types.FloatType.get())})); + InternalSchema oldSchema = new InternalSchema(record.fields()); + TableChanges.ColumnDeleteChange deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema); + deleteChange.deleteColumn("col1"); + // check repeated delete. + // deletechange can handle deleting the same column multiple times, only keep one operation. + deleteChange.deleteColumn("col1"); + deleteChange.deleteColumn("col3"); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange); + Types.RecordType checkRecord = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(1, "col2", Types.IntType.get()), + Types.Field.get(3, "col4", Types.FloatType.get())})); + Assertions.assertEquals(newSchema.getRecord(), checkRecord); + } + + @Test + public void testNestDelete() { + InternalSchema oldSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "feature1", + Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(9, false, "lat", Types.FloatType.get()), Types.Field.get(10, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(11, true, + Types.RecordType.get(Types.Field.get(12, false, "x", Types.LongType.get()), Types.Field.get(13, false, "y", Types.LongType.get())))) + ); + TableChanges.ColumnDeleteChange deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema); + deleteChange.deleteColumn("data"); + deleteChange.deleteColumn("preferences.feature2"); + deleteChange.deleteColumn("preferences.feature2"); + deleteChange.deleteColumn("locations.value.lat"); + deleteChange.deleteColumn("points.element.y"); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange); + InternalSchema checkedSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "feature1", + Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(10, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(11, true, + Types.RecordType.get(Types.Field.get(12, false, "x", Types.LongType.get())))) + ); + Assertions.assertEquals(newSchema.getRecord(), checkedSchema.getRecord()); + } + + @Test + public void testPrimitiveUpdate() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "col1", Types.BooleanType.get()), + Types.Field.get(1, "col2", Types.IntType.get()), + Types.Field.get(2, "col3", Types.LongType.get()), + Types.Field.get(3, "col4", Types.FloatType.get())})); + InternalSchema oldSchema = new InternalSchema(record.fields()); + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(oldSchema); + updateChange.updateColumnType("col2", Types.LongType.get()) + .updateColumnComment("col2", "alter col2 comments") + .renameColumn("col2", "colx").addPositionChange("col2", "col4", "after"); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange); + Types.RecordType checkedRecord = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "col1", Types.BooleanType.get()), + Types.Field.get(2, "col3", Types.LongType.get()), + Types.Field.get(3, "col4", Types.FloatType.get()), + Types.Field.get(1, true, "colx", Types.LongType.get(), "alter col2 comments")})); + Assertions.assertEquals(newSchema.getRecord(), checkedRecord); + } + + @Test + public void testNestUpdate() { + InternalSchema oldSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "feature1", + Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(9, false, "lat", Types.FloatType.get()), Types.Field.get(10, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(11, true, + Types.RecordType.get(Types.Field.get(12, false, "x", Types.LongType.get()), Types.Field.get(13, false, "y", Types.LongType.get())))) + ); + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(oldSchema); + updateChange + .updateColumnNullability("id", true) + .renameColumn("id", "idx") + .addPositionChange("data", "points", "after"); + updateChange + .updateColumnComment("preferences.feature1", "add feature1 comment") + .renameColumn("preferences.feature1", "f1") + .addPositionChange("preferences.feature1", "preferences.feature1", "first"); + updateChange.updateColumnComment("locations.value.lat", "add lat comment") + .renameColumn("locations.value.lat", "lax") + .addPositionChange("locations.value.lat", "locations.value.lat", "first"); + updateChange.renameColumn("points.element.x", "z") + .addPositionChange("points.element.x", "points.element.y", "after"); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange); + InternalSchema checkSchema = new InternalSchema(Types.Field.get(0, true, "idx", Types.IntType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "f1", + Types.BooleanType.get(), "add feature1 comment"), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(9, false, "lax", Types.FloatType.get(), "add lat comment"), Types.Field.get(10, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(11, true, + Types.RecordType.get(Types.Field.get(13, false, "y", Types.LongType.get()), Types.Field.get(12, false, "z", Types.LongType.get())))), + Types.Field.get(1, true, "data", Types.StringType.get()) + ); + Assertions.assertEquals(newSchema.getRecord(), checkSchema.getRecord()); + } +} + diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java new file mode 100644 index 0000000000000..2276b93ee045c --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.io; + +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.utils.SerDeHelper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests {@link FileBasedInternalSchemaStorageManager}. + */ +public class TestFileBasedInternalSchemaStorageManager extends HoodieCommonTestHarness { + private HoodieActiveTimeline timeline; + + @BeforeEach + public void setUp() throws Exception { + initMetaClient(); + } + + @Test + public void testPersistAndReadHistorySchemaStr() throws IOException { + timeline = new HoodieActiveTimeline(metaClient); + FileBasedInternalSchemaStorageManager fm = new FileBasedInternalSchemaStorageManager(metaClient); + InternalSchema currentSchema = getSimpleSchema(); + currentSchema.setSchemaId(0L); + // save first schema. + fm.persistHistorySchemaStr("0000", SerDeHelper.inheritSchemas(currentSchema, "")); + // Simulate commit. + simulateCommit("0000"); + // try to read schema + InternalSchema readSchema = fm.getSchemaByKey("0").get(); + assertEquals(currentSchema, readSchema); + // save history schema again + InternalSchema secondSchema = getSimpleSchema(); + secondSchema.setSchemaId(1L); + fm.persistHistorySchemaStr("0001", SerDeHelper.inheritSchemas(secondSchema, fm.getHistorySchemaStr())); + // Simulate commit. + simulateCommit("0001"); + // try to read schema + assertEquals(secondSchema, fm.getSchemaByKey("1").get()); + + // test write failed and residual file clean. + InternalSchema thirdSchema = getSimpleSchema(); + thirdSchema.setSchemaId(2L); + fm.persistHistorySchemaStr("0002", SerDeHelper.inheritSchemas(thirdSchema, fm.getHistorySchemaStr())); + // do not simulate commit "0002", so current save file will be residual files. + // try 4st persist + InternalSchema lastSchema = getSimpleSchema(); + lastSchema.setSchemaId(3L); + fm.persistHistorySchemaStr("0004", SerDeHelper.inheritSchemas(lastSchema, fm.getHistorySchemaStr())); + simulateCommit("0004"); + // now the residual file created by 3st persist should be removed. + File f = new File(metaClient.getSchemaFolderName() + File.separator + "0002.schemacommit"); + assertTrue(!f.exists()); + assertEquals(lastSchema, fm.getSchemaByKey("3").get()); + } + + private void simulateCommit(String commitTime) { + if (timeline == null) { + timeline = new HoodieActiveTimeline(metaClient); + } + HoodieInstant instant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, commitTime); + timeline.createNewInstant(instant); + timeline.transitionRequestedToInflight(instant, Option.empty()); + timeline.saveAsComplete(new HoodieInstant(true, instant.getAction(), instant.getTimestamp()), + Option.empty()); + } + + private InternalSchema getSimpleSchema() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "bool", Types.BooleanType.get()), + Types.Field.get(1, "int", Types.IntType.get()), + })); + return new InternalSchema(record.fields()); + } +} + diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaUtil.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaUtil.java new file mode 100644 index 0000000000000..ae683817bca3f --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaUtil.java @@ -0,0 +1,428 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.action.TableChanges; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Assertions; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +public class TestAvroSchemaUtil { + + @Test + public void testPrimitiveTypes() { + Schema[] avroPrimitives = new Schema[] { + Schema.create(Schema.Type.BOOLEAN), + Schema.create(Schema.Type.INT), + Schema.create(Schema.Type.LONG), + Schema.create(Schema.Type.FLOAT), + Schema.create(Schema.Type.DOUBLE), + LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), + LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)), + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + Schema.create(Schema.Type.STRING), + LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16)), + Schema.createFixed("fixed_12", null, null, 12), + Schema.create(Schema.Type.BYTES), + LogicalTypes.decimal(9, 4).addToSchema(Schema.createFixed("decimal_9_4", null, null, 4))}; + + Type[] primitiveTypes = new Type[] { + Types.BooleanType.get(), + Types.IntType.get(), + Types.LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.DateType.get(), + Types.TimeType.get(), + Types.TimestampType.get(), + Types.StringType.get(), + Types.UUIDType.get(), + Types.FixedType.getFixed(12), + Types.BinaryType.get(), + Types.DecimalType.get(9, 4) + }; + + for (int i = 0; i < primitiveTypes.length; i++) { + Type convertPrimitiveResult = AvroInternalSchemaConverter.convertToField(avroPrimitives[i]); + Assertions.assertEquals(convertPrimitiveResult, primitiveTypes[i]); + Schema convertResult = AvroInternalSchemaConverter.convert(primitiveTypes[i], "t1"); + Assertions.assertEquals(convertResult, avroPrimitives[i]); + } + } + + @Test + public void testRecordAndPrimitiveTypes() { + Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "bool", Types.BooleanType.get()), + Types.Field.get(1, "int", Types.IntType.get()), + Types.Field.get(2, "long", Types.LongType.get()), + Types.Field.get(3, "float", Types.FloatType.get()), + Types.Field.get(4, "double", Types.DoubleType.get()), + Types.Field.get(5, "date", Types.DateType.get()), + Types.Field.get(6, "time", Types.TimeType.get()), + Types.Field.get(7, "timestamp", Types.TimestampType.get()), + Types.Field.get(8, "string", Types.StringType.get()), + Types.Field.get(9, "uuid", Types.UUIDType.get()), + Types.Field.get(10, "fixed", Types.FixedType.getFixed(10)), + Types.Field.get(11, "binary", Types.BinaryType.get()), + Types.Field.get(12, "decimal", Types.DecimalType.get(10, 2)) + })); + + Schema schema = create("t1", + new Schema.Field("bool", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BOOLEAN)), null, JsonProperties.NULL_VALUE), + new Schema.Field("int", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.INT)), null, JsonProperties.NULL_VALUE), + new Schema.Field("long", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("float", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.FLOAT)), null, JsonProperties.NULL_VALUE), + new Schema.Field("double", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.DOUBLE)), null, JsonProperties.NULL_VALUE), + new Schema.Field("date", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE), + new Schema.Field("time", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG))), null, JsonProperties.NULL_VALUE), + new Schema.Field("timestamp", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))), null, JsonProperties.NULL_VALUE), + new Schema.Field("string", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.STRING)), null, JsonProperties.NULL_VALUE), + new Schema.Field("uuid", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16))), null, JsonProperties.NULL_VALUE), + new Schema.Field("fixed", AvroInternalSchemaConverter.nullableSchema(Schema.createFixed("fixed_10", null, null, 10)), null, JsonProperties.NULL_VALUE), + new Schema.Field("binary", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BYTES)), null, JsonProperties.NULL_VALUE), + new Schema.Field("decimal", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.decimal(10, 2) + .addToSchema(Schema.createFixed("decimal_10_2", null, null, 5))), null, JsonProperties.NULL_VALUE)); + Schema convertedSchema = AvroInternalSchemaConverter.convert(record, "t1"); + Assertions.assertEquals(convertedSchema, schema); + Types.RecordType convertedRecord = AvroInternalSchemaConverter.convert(schema).getRecord(); + Assertions.assertEquals(convertedRecord, record); + } + + private Schema create(String name, Schema.Field... fields) { + return Schema.createRecord(name, null, null, false, Arrays.asList(fields)); + } + + @Test + public void testArrayType() { + Type arrayNestRecordType = Types.ArrayType.get(1, false, + Types.RecordType.get(Arrays.asList(Types.Field.get(2, false, "a", Types.FloatType.get()), + Types.Field.get(3, false, "b", Types.FloatType.get())))); + + Schema schema = SchemaBuilder.array().items(create("t1", + new Schema.Field("a", Schema.create(Schema.Type.FLOAT), null, null), + new Schema.Field("b", Schema.create(Schema.Type.FLOAT), null, null))); + Schema convertedSchema = AvroInternalSchemaConverter.convert(arrayNestRecordType, "t1"); + Assertions.assertEquals(convertedSchema, schema); + Types.ArrayType convertedRecord = (Types.ArrayType) AvroInternalSchemaConverter.convertToField(schema); + Assertions.assertEquals(convertedRecord, arrayNestRecordType); + } + + @Test + public void testComplexConvert() { + String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\"," + + "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\"," + + "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\"," + + "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}]," + + "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\"," + + "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," + + "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\"," + + "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," + + "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}}," + + "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}"; + Schema schema = new Schema.Parser().parse(schemaStr); + + InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(7, false, "feature1", + Types.BooleanType.get()), Types.Field.get(8, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)), + Types.Field.get(4, true, "points", Types.ArrayType.get(13, true, + Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))), + Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())), + Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get())) + ); + + Type convertRecord = AvroInternalSchemaConverter.convert(schema).getRecord(); + Assertions.assertEquals(convertRecord, internalSchema.getRecord()); + Assertions.assertEquals(schema, AvroInternalSchemaConverter.convert(internalSchema, "newTableName")); + } + + @Test + public void testRefreshNewId() { + Types.RecordType record = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(4, false, "feature1", + Types.BooleanType.get()), Types.Field.get(5, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "locations", Types.MapType.get(6, 7, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(8, false, "lat", Types.FloatType.get()), Types.Field.get(9, false, "long", Types.FloatType.get())), false)) + ); + AtomicInteger newId = new AtomicInteger(100); + Types.RecordType recordWithNewId = (Types.RecordType)InternalSchemaUtils.refreshNewId(record, newId); + + Types.RecordType newRecord = Types.RecordType.get(Types.Field.get(100, false, "id", Types.IntType.get()), + Types.Field.get(101, true, "data", Types.StringType.get()), + Types.Field.get(102, true, "preferences", + Types.RecordType.get(Types.Field.get(104, false, "feature1", + Types.BooleanType.get()), Types.Field.get(105, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(103, false, "locations", Types.MapType.get(106, 107, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(108, false, "lat", Types.FloatType.get()), Types.Field.get(109, false, "long", Types.FloatType.get())), false)) + ); + Assertions.assertEquals(newRecord, recordWithNewId); + } + + /** + * test record data type changes. + * int => long/float/double/string + * long => float/double/string + * float => double/String + * double => String/Decimal + * Decimal => Decimal/String + * String => date/decimal + * date => String + */ + @Test + public void testReWriteRecordWithTypeChanged() { + Schema avroSchema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"h0_record\",\"namespace\":\"hoodie.h0\",\"fields\"" + + ":[{\"name\":\"id\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"comb\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"com1\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"col0\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"col1\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"col11\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"col12\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"col2\",\"type\":[\"null\",\"float\"],\"default\":null}," + + "{\"name\":\"col21\",\"type\":[\"null\",\"float\"],\"default\":null}," + + "{\"name\":\"col3\",\"type\":[\"null\",\"double\"],\"default\":null}," + + "{\"name\":\"col31\",\"type\":[\"null\",\"double\"],\"default\":null}," + + "{\"name\":\"col4\",\"type\":[\"null\",{\"type\":\"fixed\",\"name\":\"fixed\",\"namespace\":\"hoodie.h0.h0_record.col4\"," + + "\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":4}],\"default\":null}," + + "{\"name\":\"col41\",\"type\":[\"null\",{\"type\":\"fixed\",\"name\":\"fixed\",\"namespace\":\"hoodie.h0.h0_record.col41\"," + + "\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":4}],\"default\":null}," + + "{\"name\":\"col5\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"col51\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"col6\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}," + + "{\"name\":\"col7\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}],\"default\":null}," + + "{\"name\":\"col8\",\"type\":[\"null\",\"boolean\"],\"default\":null}," + + "{\"name\":\"col9\",\"type\":[\"null\",\"bytes\"],\"default\":null},{\"name\":\"par\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}]}"); + // create a test record with avroSchema + GenericData.Record avroRecord = new GenericData.Record(avroSchema); + avroRecord.put("id", 1); + avroRecord.put("comb", 100); + avroRecord.put("com1", -100); + avroRecord.put("col0", 256); + avroRecord.put("col1", 1000L); + avroRecord.put("col11", -100L); + avroRecord.put("col12", 2000L); + avroRecord.put("col2", -5.001f); + avroRecord.put("col21", 5.001f); + avroRecord.put("col3", 12.999d); + avroRecord.put("col31", 9999.999d); + Schema currentDecimalType = avroSchema.getField("col4").schema().getTypes().get(1); + BigDecimal bd = new BigDecimal("123.456").setScale(((LogicalTypes.Decimal) currentDecimalType.getLogicalType()).getScale()); + avroRecord.put("col4", AvroSchemaUtil.DECIMAL_CONVERSION.toFixed(bd, currentDecimalType, currentDecimalType.getLogicalType())); + Schema currentDecimalType1 = avroSchema.getField("col41").schema().getTypes().get(1); + BigDecimal bd1 = new BigDecimal("7890.456").setScale(((LogicalTypes.Decimal) currentDecimalType1.getLogicalType()).getScale()); + avroRecord.put("col41", AvroSchemaUtil.DECIMAL_CONVERSION.toFixed(bd1, currentDecimalType1, currentDecimalType1.getLogicalType())); + + avroRecord.put("col5", "2011-01-01"); + avroRecord.put("col51", "199.342"); + avroRecord.put("col6", 18987); + avroRecord.put("col7", 1640491505000000L); + avroRecord.put("col8", false); + ByteBuffer bb = ByteBuffer.wrap(new byte[] {97, 48, 53}); + avroRecord.put("col9", bb); + Assertions.assertEquals(GenericData.get().validate(avroSchema, avroRecord), true); + InternalSchema internalSchema = AvroInternalSchemaConverter.convert(avroSchema); + // do change type operation + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(internalSchema); + updateChange + .updateColumnType("id", Types.LongType.get()) + .updateColumnType("comb", Types.FloatType.get()) + .updateColumnType("com1", Types.DoubleType.get()) + .updateColumnType("col0", Types.StringType.get()) + .updateColumnType("col1", Types.FloatType.get()) + .updateColumnType("col11", Types.DoubleType.get()) + .updateColumnType("col12", Types.StringType.get()) + .updateColumnType("col2", Types.DoubleType.get()) + .updateColumnType("col21", Types.StringType.get()) + .updateColumnType("col3", Types.StringType.get()) + .updateColumnType("col31", Types.DecimalType.get(18, 9)) + .updateColumnType("col4", Types.DecimalType.get(18, 9)) + .updateColumnType("col41", Types.StringType.get()) + .updateColumnType("col5", Types.DateType.get()) + .updateColumnType("col51", Types.DecimalType.get(18, 9)) + .updateColumnType("col6", Types.StringType.get()); + InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); + Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName()); + GenericRecord newRecord = AvroSchemaUtil.rewriteRecord(avroRecord, newAvroSchema); + + Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newRecord), true); + } + + @Test + public void testReWriteNestRecord() { + Types.RecordType record = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "feature1", + Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(10, false, "lat", Types.FloatType.get()), Types.Field.get(11, false, "long", Types.FloatType.get())), false)) + ); + Schema schema = AvroInternalSchemaConverter.convert(record, "test1"); + GenericData.Record avroRecord = new GenericData.Record(schema); + GenericData.get().validate(schema, avroRecord); + avroRecord.put("id", 2); + avroRecord.put("data", "xs"); + // fill record type + GenericData.Record preferencesRecord = new GenericData.Record(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences")); + preferencesRecord.put("feature1", false); + preferencesRecord.put("feature2", true); + Assertions.assertEquals(GenericData.get().validate(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences"), preferencesRecord), true); + avroRecord.put("preferences", preferencesRecord); + // fill mapType + Map locations = new HashMap<>(); + Schema mapSchema = AvroInternalSchemaConverter.convert(((Types.MapType)record.field("locations").type()).valueType(), "test1_locations"); + GenericData.Record locationsValue = new GenericData.Record(mapSchema); + locationsValue.put("lat", 1.2f); + locationsValue.put("long", 1.4f); + GenericData.Record locationsValue1 = new GenericData.Record(mapSchema); + locationsValue1.put("lat", 2.2f); + locationsValue1.put("long", 2.4f); + locations.put("key1", locationsValue); + locations.put("key2", locationsValue1); + avroRecord.put("locations", locations); + + List doubles = new ArrayList<>(); + doubles.add(2.0d); + doubles.add(3.0d); + avroRecord.put("doubles", doubles); + + // do check + Assertions.assertEquals(GenericData.get().validate(schema, avroRecord), true); + // create newSchema + Types.RecordType newRecord = Types.RecordType.get( + Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get( + Types.Field.get(5, false, "feature1", Types.BooleanType.get()), + Types.Field.get(5, true, "featurex", Types.BooleanType.get()), + Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(), + Types.RecordType.get( + Types.Field.get(10, true, "laty", Types.FloatType.get()), + Types.Field.get(11, false, "long", Types.FloatType.get())), false) + ) + ); + + Schema newAvroSchema = AvroInternalSchemaConverter.convert(newRecord, schema.getName()); + GenericRecord newAvroRecord = AvroSchemaUtil.rewriteRecord(avroRecord, newAvroSchema); + // test the correctly of rewrite + Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newAvroRecord), true); + + // test union type + Schema union = Schema.createUnion(Schema.create(Schema.Type.BOOLEAN), Schema.create(Schema.Type.STRING), Schema.create(Schema.Type.FLOAT)); + Schema schemaUnion = create("t1", new Schema.Field("ux", union, null, JsonProperties.NULL_VALUE)); + GenericData.Record unionRecord = new GenericData.Record(schemaUnion); + unionRecord.put(0, "ss"); + // test the correctly of rewrite + Assertions.assertEquals(GenericData.get().validate(schemaUnion, unionRecord), true); + } + + @Test + public void testEvolutionSchemaFromNewAvroSchema() { + Types.RecordType oldRecord = Types.RecordType.get( + Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get( + Types.Field.get(5, false, "feature1", Types.BooleanType.get()), + Types.Field.get(6, true, "featurex", Types.BooleanType.get()), + Types.Field.get(7, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false,"doubles", Types.ArrayType.get(8, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(), + Types.RecordType.get( + Types.Field.get(11, false, "laty", Types.FloatType.get()), + Types.Field.get(12, false, "long", Types.FloatType.get())), false) + ) + ); + InternalSchema oldSchema = new InternalSchema(oldRecord.fields()); + Types.RecordType evolvedRecord = Types.RecordType.get( + Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get( + Types.Field.get(5, false, "feature1", Types.BooleanType.get()), + Types.Field.get(5, true, "featurex", Types.BooleanType.get()), + Types.Field.get(6, true, "feature2", Types.BooleanType.get()), + Types.Field.get(5, true, "feature3", Types.BooleanType.get()))), + Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(), + Types.RecordType.get( + Types.Field.get(10, false, "laty", Types.FloatType.get()), + Types.Field.get(11, false, "long", Types.FloatType.get())), false) + ), + Types.Field.get(0, false, "add1", Types.IntType.get()), + Types.Field.get(2, true, "addStruct", + Types.RecordType.get( + Types.Field.get(5, false, "nest1", Types.BooleanType.get()), + Types.Field.get(5, true, "nest2", Types.BooleanType.get()))) + ); + evolvedRecord = (Types.RecordType)InternalSchemaUtils.refreshNewId(evolvedRecord, new AtomicInteger(0)); + Schema evolvedAvroSchema = AvroInternalSchemaConverter.convert(evolvedRecord, "test1"); + InternalSchema result = AvroSchemaUtil.evolutionSchemaFromNewAvroSchema(evolvedAvroSchema, oldSchema); + Types.RecordType checkedRecord = Types.RecordType.get( + Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get( + Types.Field.get(5, false, "feature1", Types.BooleanType.get()), + Types.Field.get(6, true, "featurex", Types.BooleanType.get()), + Types.Field.get(7, true, "feature2", Types.BooleanType.get()), + Types.Field.get(17, true, "feature3", Types.BooleanType.get()))), + Types.Field.get(3, false,"doubles", Types.ArrayType.get(8, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(), + Types.RecordType.get( + Types.Field.get(11, false, "laty", Types.FloatType.get()), + Types.Field.get(12, false, "long", Types.FloatType.get())), false) + ), + Types.Field.get(13, true, "add1", Types.IntType.get()), + Types.Field.get(14, true, "addStruct", + Types.RecordType.get( + Types.Field.get(15, false, "nest1", Types.BooleanType.get()), + Types.Field.get(16, true, "nest2", Types.BooleanType.get()))) + ); + Assertions.assertEquals(result.getRecord(), checkedRecord); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestInternalSchemaUtils.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestInternalSchemaUtils.java new file mode 100644 index 0000000000000..28da410546026 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestInternalSchemaUtils.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal.schema.utils; + +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Assertions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class TestInternalSchemaUtils { + @Test + public void testPruneSchema() { + Types.RecordType record = getSimpleRecordType(); + InternalSchema originSchema = new InternalSchema(record.fields()); + List prunedCols = new ArrayList<>(); + prunedCols.add(4); + prunedCols.add(3); + prunedCols.add(0); + prunedCols.add(2); + InternalSchema prunedSchema = InternalSchemaUtils.pruneInternalSchemaByID(originSchema, prunedCols, null); + InternalSchema checkedSchema = new InternalSchema(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "bool", Types.BooleanType.get()), + Types.Field.get(2, "long", Types.LongType.get()), + Types.Field.get(3, "float", Types.FloatType.get()), + Types.Field.get(4, "double", Types.DoubleType.get()) + })); + Assertions.assertEquals(prunedSchema, checkedSchema); + + // nest schema + Types.RecordType nestRecord = getNestRecordType(); + InternalSchema originNestSchema = new InternalSchema(nestRecord.fields()); + List prunedNestCols = new ArrayList<>(); + prunedNestCols.add(0); + prunedNestCols.add(1); + prunedNestCols.add(5); + prunedNestCols.add(11); + InternalSchema prunedNestSchema = InternalSchemaUtils.pruneInternalSchemaByID(originNestSchema, prunedNestCols, null); + } + + @Test + public void testInternalSchemaVisitor() { + Types.RecordType nestRecord = getNestRecordType(); + Map result = InternalSchemaUtils.buildNameToId(nestRecord); + Assertions.assertEquals(result.size(), 12); + Assertions.assertEquals(result.get("locations.value.long"), 11); + Assertions.assertEquals(result.get("locations.value.lat"), 10); + Assertions.assertEquals(result.get("locations.value"), 9); + Assertions.assertEquals(result.get("locations.key"), 8); + Assertions.assertEquals(result.get("doubles.element"), 7); + + Types.RecordType simpleRecord = getSimpleRecordType(); + Map result1 = InternalSchemaUtils.buildNameToId(simpleRecord); + Assertions.assertEquals(result1.size(), 5); + Assertions.assertEquals(result1.get("double"), 4); + } + + public Types.RecordType getNestRecordType() { + return Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "feature1", + Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(10, false, "lat", Types.FloatType.get()), Types.Field.get(11, false, "long", Types.FloatType.get())), false)) + ); + } + + public Types.RecordType getSimpleRecordType() { + return Types.RecordType.get(Arrays.asList(new Types.Field[] { + Types.Field.get(0, "bool", Types.BooleanType.get()), + Types.Field.get(1, "int", Types.IntType.get()), + Types.Field.get(2, "long", Types.LongType.get()), + Types.Field.get(3, "float", Types.FloatType.get()), + Types.Field.get(4, "double", Types.DoubleType.get()) + })); + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala index c4f5cd39f6073..1d65670f6d3f3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala @@ -110,7 +110,7 @@ object AlterHoodieTableAddColumnsCommand { HoodieWriterUtils.parametersWithWriteDefaults(hoodieCatalogTable.catalogProperties).asJava ) - val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, hoodieCatalogTable.tableType) + val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, hoodieCatalogTable.tableType) val instantTime = HoodieActiveTimeline.createNewInstantTime client.startCommitWithTime(instantTime, commitActionType) @@ -118,7 +118,7 @@ object AlterHoodieTableAddColumnsCommand { val timeLine = hoodieTable.getActiveTimeline val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime) val metadata = new HoodieCommitMetadata - metadata.setOperationType(WriteOperationType.INSERT) + metadata.setOperationType(WriteOperationType.ALTER_SCHEMA) timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8))) client.commit(instantTime, jsc.emptyRDD)