nsivabalan commented on a change in pull request #5031:
URL: https://github.com/apache/hudi/pull/5031#discussion_r829454938



##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/AddColumnSchemaPostProcessor.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.schema;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * A {@link SchemaPostProcessor} use to add column to given schema. Currently. 
only supports adding one column at a time.
+ * Users can specify the position of new column by config {@link 
Config#SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP},
+ * the new column will be added before this column.
+ * <p>
+ * Currently supported types : bytes, string, int, long, float, double, 
boolean, decimal
+ */

Review comment:
       I understand this is schema post processor. But can we add a line here 
in documentation that i hudi, adding a new column in the middle may not be 
advisable. users might run into issues. So, its always a good practice to 
append new columns to the end. 
   

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/AddColumnSchemaPostProcessor.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.schema;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * A {@link SchemaPostProcessor} use to add column to given schema. Currently. 
only supports adding one column at a time.
+ * Users can specify the position of new column by config {@link 
Config#SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP},
+ * the new column will be added before this column.
+ * <p>
+ * Currently supported types : bytes, string, int, long, float, double, 
boolean, decimal
+ */
+public class AddColumnSchemaPostProcessor extends SchemaPostProcessor {
+
+  private static final Logger LOG = 
LogManager.getLogger(AddColumnSchemaPostProcessor.class);
+
+  public AddColumnSchemaPostProcessor(TypedProperties props, JavaSparkContext 
jssc) {
+    super(props, jssc);
+  }
+
+  /**
+   * Configs supported.
+   */
+  public static class Config {
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_NAME_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.name")
+        .noDefaultValue()
+        .withDocumentation("New column's name");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_TYPE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.type")
+        .noDefaultValue()
+        .withDocumentation("New column's type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_DOC_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.doc")
+        .noDefaultValue()
+        .withDocumentation("New column's doc");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_DEFAULT_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.default")
+        .noDefaultValue()
+        .withDocumentation("New column's default value");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_SIZE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.size")
+        .noDefaultValue()
+        .withDocumentation("New column's size, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_PRECISION_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.precision")
+        .noDefaultValue()
+        .withDocumentation("New column's precision, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_SCALE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.scale")
+        .noDefaultValue()
+        .withDocumentation("New column's precision, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.next")
+        .defaultValue(HoodieRecord.HOODIE_IS_DELETED)
+        .withDocumentation("Column name which locate next to new column, 
`_hoodie_is_deleted` by default.");
+  }
+
+  public static final String BYTES = "BYTES";
+  public static final String STRING = "STRING";
+  public static final String INT = "INT";
+  public static final String LONG = "LONG";
+  public static final String FLOAT = "FLOAT";
+  public static final String DOUBLE = "DOUBLE";
+  public static final String BOOLEAN = "BOOLEAN";
+  public static final String DECIMAL = "DECIMAL";
+
+  @Override
+  public Schema processSchema(Schema schema) {
+    String newColumnName = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NAME_PROP.key());
+
+    if (schema.getField(newColumnName) != null) {
+      LOG.warn(String.format("Column %s already exist!", newColumnName));
+      return schema;
+    }
+
+    List<Schema.Field> sourceFields = schema.getFields();
+    List<Schema.Field> targetFields = new ArrayList<>(sourceFields.size() + 1);
+
+    String nextColumnName = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP.key(),
+        Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP.defaultValue());
+
+    // mark whether the new column is added
+    boolean isAdded = false;
+    for (Schema.Field sourceField : sourceFields) {
+      if (sourceField.name().equals(nextColumnName)) {
+        targetFields.add(buildNewColumn());
+        isAdded = true;
+      }
+      targetFields.add(new Schema.Field(sourceField.name(), 
sourceField.schema(), sourceField.doc(), sourceField.defaultVal()));

Review comment:
       from the documentation, I thought we will add the new column after the 
column specified in Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP. But 
guess, we are adding the new column prior to that. Can we please clarify this 
in documentation. Or may be give an example. 

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/AddColumnSchemaPostProcessor.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.schema;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * A {@link SchemaPostProcessor} use to add column to given schema. Currently. 
only supports adding one column at a time.
+ * Users can specify the position of new column by config {@link 
Config#SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP},
+ * the new column will be added before this column.
+ * <p>
+ * Currently supported types : bytes, string, int, long, float, double, 
boolean, decimal
+ */
+public class AddColumnSchemaPostProcessor extends SchemaPostProcessor {
+
+  private static final Logger LOG = 
LogManager.getLogger(AddColumnSchemaPostProcessor.class);
+
+  public AddColumnSchemaPostProcessor(TypedProperties props, JavaSparkContext 
jssc) {
+    super(props, jssc);
+  }
+
+  /**
+   * Configs supported.
+   */
+  public static class Config {
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_NAME_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.name")
+        .noDefaultValue()
+        .withDocumentation("New column's name");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_TYPE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.type")
+        .noDefaultValue()
+        .withDocumentation("New column's type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_DOC_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.doc")
+        .noDefaultValue()
+        .withDocumentation("New column's doc");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_DEFAULT_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.default")
+        .noDefaultValue()
+        .withDocumentation("New column's default value");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_SIZE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.size")
+        .noDefaultValue()
+        .withDocumentation("New column's size, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_PRECISION_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.precision")
+        .noDefaultValue()
+        .withDocumentation("New column's precision, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_SCALE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.scale")
+        .noDefaultValue()
+        .withDocumentation("New column's precision, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.next")
+        .defaultValue(HoodieRecord.HOODIE_IS_DELETED)
+        .withDocumentation("Column name which locate next to new column, 
`_hoodie_is_deleted` by default.");
+  }
+
+  public static final String BYTES = "BYTES";
+  public static final String STRING = "STRING";
+  public static final String INT = "INT";
+  public static final String LONG = "LONG";
+  public static final String FLOAT = "FLOAT";
+  public static final String DOUBLE = "DOUBLE";
+  public static final String BOOLEAN = "BOOLEAN";
+  public static final String DECIMAL = "DECIMAL";
+
+  @Override
+  public Schema processSchema(Schema schema) {
+    String newColumnName = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NAME_PROP.key());
+
+    if (schema.getField(newColumnName) != null) {
+      LOG.warn(String.format("Column %s already exist!", newColumnName));
+      return schema;
+    }
+
+    List<Schema.Field> sourceFields = schema.getFields();
+    List<Schema.Field> targetFields = new ArrayList<>(sourceFields.size() + 1);
+
+    String nextColumnName = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP.key(),
+        Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP.defaultValue());
+
+    // mark whether the new column is added
+    boolean isAdded = false;
+    for (Schema.Field sourceField : sourceFields) {
+      if (sourceField.name().equals(nextColumnName)) {
+        targetFields.add(buildNewColumn());
+        isAdded = true;
+      }
+      targetFields.add(new Schema.Field(sourceField.name(), 
sourceField.schema(), sourceField.doc(), sourceField.defaultVal()));
+    }
+
+    // this would happen when `nextColumn` does not exist. just append the new 
column to the end
+    if (!isAdded) {
+      targetFields.add(buildNewColumn());
+    }
+
+    return Schema.createRecord(schema.getName(), schema.getDoc(), 
schema.getNamespace(), false, targetFields);
+  }
+
+  private Schema.Field buildNewColumn() {
+    Schema.Field result;
+
+    String columnName = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_NAME_PROP.key());
+    String type = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_TYPE_PROP.key()).toUpperCase(Locale.ROOT);
+    String doc = 
this.config.getString(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_DOC_PROP.key(), 
null);
+    Object defaultValue = 
this.config.getOrDefault(Config.SCHEMA_POST_PROCESSOR_ADD_COLUMN_DEFAULT_PROP.key(),
+        null);
+
+    switch (type) {
+      case STRING:
+      case BYTES:
+      case INT:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case BOOLEAN:

Review comment:
       so, looks like we are supporting only primitives here. I don't see 
arrays or maps etc. Can we call it out in the class documentation. would like 
to understand why we have such a limitation. why not add all diff types? 
   

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/AddColumnSchemaPostProcessor.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.schema;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * A {@link SchemaPostProcessor} use to add column to given schema. Currently. 
only supports adding one column at a time.
+ * Users can specify the position of new column by config {@link 
Config#SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP},
+ * the new column will be added before this column.
+ * <p>
+ * Currently supported types : bytes, string, int, long, float, double, 
boolean, decimal
+ */
+public class AddColumnSchemaPostProcessor extends SchemaPostProcessor {
+
+  private static final Logger LOG = 
LogManager.getLogger(AddColumnSchemaPostProcessor.class);
+
+  public AddColumnSchemaPostProcessor(TypedProperties props, JavaSparkContext 
jssc) {
+    super(props, jssc);
+  }
+
+  /**
+   * Configs supported.
+   */
+  public static class Config {
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_NAME_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.name")
+        .noDefaultValue()
+        .withDocumentation("New column's name");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_TYPE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.type")
+        .noDefaultValue()
+        .withDocumentation("New column's type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_DOC_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.doc")
+        .noDefaultValue()
+        .withDocumentation("New column's doc");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_DEFAULT_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.default")
+        .noDefaultValue()
+        .withDocumentation("New column's default value");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_SIZE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.size")
+        .noDefaultValue()
+        .withDocumentation("New column's size, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_PRECISION_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.precision")
+        .noDefaultValue()
+        .withDocumentation("New column's precision, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_SCALE_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.scale")
+        .noDefaultValue()
+        .withDocumentation("New column's precision, used in decimal type");
+
+    public static final ConfigProperty<String> 
SCHEMA_POST_PROCESSOR_ADD_COLUMN_NEXT_PROP = ConfigProperty
+        
.key("hoodie.deltastreamer.schemaprovider.schema_post_processor.add.column.next")
+        .defaultValue(HoodieRecord.HOODIE_IS_DELETED)

Review comment:
       there should not be any default. We should just append to the end by 
defualt




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to