alexeykudinkin commented on code in PR #5523:
URL: https://github.com/apache/hudi/pull/5523#discussion_r927206578


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java:
##########
@@ -19,132 +19,560 @@
 package org.apache.hudi.keygen;
 
 import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.ApiMaturityLevel;
 import org.apache.hudi.AvroConversionUtils;
-import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.client.utils.SparkRowSerDe;
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
 import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieKeyException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils$;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DateType;
 import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.types.TimestampType;
+import org.apache.spark.unsafe.types.UTF8String;
 import scala.Function1;
 
-import java.util.HashMap;
+import javax.annotation.concurrent.ThreadSafe;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDate;
 import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import static org.apache.hudi.common.util.CollectionUtils.tail;
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+import static 
org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR;
+import static 
org.apache.hudi.keygen.KeyGenUtils.DEFAULT_RECORD_KEY_PARTS_SEPARATOR;
+import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
+import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
+import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
 
 /**
- * Base class for the built-in key generators. Contains methods structured for
- * code reuse amongst them.
+ * Base class for all built-in key generators.
+ *
+ * NOTE: By default it implements all the methods of {@link 
SparkKeyGeneratorInterface}, which
+ *       by default however fallback to Avro implementation. For maximum 
performance (to avoid
+ *       conversion from Spark's internal data-types to Avro) you should 
override these methods
+ *       in your implementation.
+ *
+ * TODO rename to AvroFallbackBaseKeyGenerator
  */
+@ThreadSafe
 public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements 
SparkKeyGeneratorInterface {
 
-  private static final String STRUCT_NAME = "hoodieRowTopLevelField";
-  private static final String NAMESPACE = "hoodieRow";
-  private Function1<Row, GenericRecord> converterFn = null;
-  private final AtomicBoolean validatePartitionFields = new 
AtomicBoolean(false);
-  protected StructType structType;
+  private static final Logger LOG = 
LogManager.getLogger(BuiltinKeyGenerator.class);
+
+  private static final String COMPOSITE_KEY_FIELD_VALUE_INFIX = ":";
+
+  protected static final UTF8String HUDI_DEFAULT_PARTITION_PATH_UTF8 = 
UTF8String.fromString(HUDI_DEFAULT_PARTITION_PATH);
+  protected static final UTF8String NULL_RECORD_KEY_PLACEHOLDER_UTF8 = 
UTF8String.fromString(NULL_RECORDKEY_PLACEHOLDER);
+  protected static final UTF8String EMPTY_RECORD_KEY_PLACEHOLDER_UTF8 = 
UTF8String.fromString(EMPTY_RECORDKEY_PLACEHOLDER);
+
 
-  protected Map<String, Pair<List<Integer>, DataType>> recordKeySchemaInfo = 
new HashMap<>();
-  protected Map<String, Pair<List<Integer>, DataType>> partitionPathSchemaInfo 
= new HashMap<>();
+  protected transient volatile SparkRowConverter rowConverter;
+  protected transient volatile SparkRowAccessor rowAccessor;
 
   protected BuiltinKeyGenerator(TypedProperties config) {
     super(config);
   }
 
-  /**
-   * Fetch record key from {@link Row}.
-   *
-   * @param row instance of {@link Row} from which record key is requested.
-   * @return the record key of interest from {@link Row}.
-   */
   @Override
-  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)

Review Comment:
   Correct. Should just be part of the interface not impl



-- 
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