pratyakshsharma commented on code in PR #4910: URL: https://github.com/apache/hudi/pull/4910#discussion_r845211292
########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java: ########## @@ -1505,4 +1549,138 @@ private void tryUpgrade(HoodieTableMetaClient metaClient, Option<String> 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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()) + .applyAddChange(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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyDeleteChange(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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyRenameChange(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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyColumnNullabilityChange(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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyColumnTypeChange(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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyColumnCommentChange(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<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient(); + InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()) + .applyReOrderColPositionChange(colName, referColName, orderType); + commitTableChange(newSchema, pair.getRight()); + } + + private Pair<InternalSchema, HoodieTableMetaClient> getInternalSchemaAndMetaClient() { + HoodieTableMetaClient metaClient = createMetaClient(true); + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + Option<InternalSchema> 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<String, String> extraMeta = new HashMap<>(); + extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(newSchema.setSchemaId(Long.getLong(instantTime)))); + // try to save history schemas + FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient); + schemasManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(newSchema, historySchemaStr)); Review Comment: what is the purpose of storing history schema here, I guess this is redundant since we are anyways storing the evolved schema as history schema in saveInternalSchema() method which gets called from commitStats(). WDYT @xiarixiaoyao ? Also can you share your slack id with me, it will be easier to coordinate with you. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org