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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -186,10 +186,18 @@ public HoodieRecord rewriteRecord(Schema recordSchema, 
Properties props, Schema
     boolean containMetaFields = hasMetaFields(structType);
     UTF8String[] metaFields = tryExtractMetaFields(data, structType);
 
-    // TODO add actual rewriting
-    InternalRow finalRow = new HoodieInternalRow(metaFields, data, 
containMetaFields);
+    // Rewrite if schema is not same
+    InternalRow finalRow = this.data;
+    StructType rawTargetSchema = 
HoodieInternalRowUtils.removeMetaField(targetStructType);
+    if (!HoodieInternalRowUtils.compareSchema(structType, rawTargetSchema)) {
+      InternalRow rewriteRecord = 
HoodieInternalRowUtils.rewriteRecord(this.data, structType, rawTargetSchema);
+      this.data = null;

Review Comment:
   We should not modify existing record



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -186,10 +186,18 @@ public HoodieRecord rewriteRecord(Schema recordSchema, 
Properties props, Schema
     boolean containMetaFields = hasMetaFields(structType);
     UTF8String[] metaFields = tryExtractMetaFields(data, structType);
 
-    // TODO add actual rewriting
-    InternalRow finalRow = new HoodieInternalRow(metaFields, data, 
containMetaFields);
+    // Rewrite if schema is not same
+    InternalRow finalRow = this.data;
+    StructType rawTargetSchema = 
HoodieInternalRowUtils.removeMetaField(targetStructType);

Review Comment:
   Why do we need to remove meta fields?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java:
##########
@@ -56,6 +63,8 @@ public class HoodieSparkParquetReader implements 
HoodieSparkFileReader {
   public HoodieSparkParquetReader(Configuration conf, Path path) {
     this.path = path;
     this.conf = conf;

Review Comment:
   We need to make sure we make the copy of the config first, before modifying 
it



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -862,22 +862,34 @@ object HoodieSparkSqlWriter {
       case HoodieRecord.HoodieRecordType.SPARK =>
         // ut will use AvroKeyGenerator, so we need to cast it in spark record
         val sparkKeyGenerator = 
keyGenerator.asInstanceOf[SparkKeyGeneratorInterface]
-        val structType = HoodieInternalRowUtils.getCachedSchema(schema)
+        val targetStructType = HoodieInternalRowUtils.getCachedSchema(schema)
+        val sourceStructType = df.schema
+        val (processedSourceStructType, processedStructType) = if 
(dropPartitionColumns) {
+          (generateSparkSchemaWithoutPartitionColumns(partitionCols, 
sourceStructType),
+            generateSparkSchemaWithoutPartitionColumns(partitionCols, 
targetStructType))
+        } else {
+          (sourceStructType, targetStructType)
+        }
         df.queryExecution.toRdd.mapPartitions { iter =>
-          val projection: Function[InternalRow, InternalRow] = if 
(dropPartitionColumns) {
-            val newSchema = 
generateSparkSchemaWithoutPartitionColumns(partitionCols, structType)
-            HoodieInternalRowUtils.getCachedUnsafeProjection(structType, 
newSchema)
+          val dropProjection: Function[InternalRow, InternalRow] = if 
(dropPartitionColumns) {
+            HoodieInternalRowUtils.getCachedUnsafeProjection(sourceStructType, 
processedSourceStructType)
           } else {
             identity
           }
 
           iter.map { internalRow =>
-            val processedRow = projection(internalRow)
-            val recordKey = sparkKeyGenerator.getRecordKey(internalRow, 
structType)
-            val partitionPath = 
sparkKeyGenerator.getPartitionPath(internalRow, structType)
+            val recordKey = sparkKeyGenerator.getRecordKey(internalRow, 
sourceStructType)
+            val partitionPath = 
sparkKeyGenerator.getPartitionPath(internalRow, sourceStructType)
             val key = new HoodieKey(recordKey.toString, partitionPath.toString)
 
-            new HoodieSparkRecord(key, processedRow, structType, false)
+            val processedRow = if (reconcileSchema) {
+              
HoodieInternalRowUtils.getCachedUnsafeProjection(processedStructType, 
processedStructType)

Review Comment:
   We'd need to reconcile this with https://github.com/apache/hudi/pull/6358



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -461,9 +460,12 @@ abstract class HoodieBaseRelation(val sqlContext: 
SQLContext,
   }
 
   protected def getTableState: HoodieTableState = {
-    val mergerImpls = 
ConfigUtils.getMergerImpls(optParams.asJava).asScala.toList
+    val mergerImpls = 
ConfigUtils.getMergerImpls(optParams.getOrElse(HoodieWriteConfig.MERGER_IMPLS.key(),

Review Comment:
   I meant to abstract utility that first looks up in optParams, then sqlContext



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql.hudi.command
+
+import org.apache.avro.Schema
+import org.apache.hudi.HoodieSparkRecordMerger
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.model.HoodieAvroRecordMerger.Config
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.util.{collection, Option => HOption}
+import org.apache.hudi.exception.HoodieDuplicateKeyException
+
+/**
+ * Validate the duplicate key for insert statement without enable the 
INSERT_DROP_DUPS_OPT
+ * config.
+ */
+class HoodieSparkValidateDuplicateKeyRecordMerger extends 
HoodieSparkRecordMerger {

Review Comment:
   I see now. Please update the java-doc to reflect that



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -200,11 +208,18 @@ public HoodieRecord rewriteRecordWithNewSchema(Schema 
recordSchema, Properties p
     boolean containMetaFields = hasMetaFields(structType);
     UTF8String[] metaFields = tryExtractMetaFields(data, structType);
 
-    InternalRow rewrittenRow =
-        HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, 
newStructType, renameCols);
-    HoodieInternalRow finalRow = new HoodieInternalRow(metaFields, 
rewrittenRow, containMetaFields);
+    // Rewrite if schema is not same
+    InternalRow finalRow = this.data;
+    StructType rawNewSchema = 
HoodieInternalRowUtils.removeMetaField(newStructType);

Review Comment:
   Same comments as above



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -420,6 +438,15 @@ public Pair<SchemaProvider, Pair<String, 
JavaRDD<HoodieRecord>>> readFromSource(
   }
 
   private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> 
fetchFromSource(Option<String> resumeCheckpointStr) {
+    HoodieRecordType recordType = HoodieRecordUtils.createRecordMerger(null, 
EngineType.SPARK,
+        
ConfigUtils.getMergerImpls(props.getProperty(HoodieWriteConfig.MERGER_IMPLS.key(),
 HoodieWriteConfig.MERGER_IMPLS.defaultValue())),
+        props.getProperty(HoodieWriteConfig.MERGER_STRATEGY.key(), 
HoodieWriteConfig.MERGER_STRATEGY.defaultValue())).getRecordType();
+    if (recordType == HoodieRecordType.SPARK && 
HoodieTableType.valueOf(cfg.tableType) == HoodieTableType.MERGE_ON_READ

Review Comment:
   We know at this stage that the `recordType` has to be Spark, right?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -492,15 +519,40 @@ private Pair<SchemaProvider, Pair<String, 
JavaRDD<HoodieRecord>>> fetchFromSourc
     boolean shouldCombine = cfg.filterDupes || 
cfg.operation.equals(WriteOperationType.UPSERT);
     Set<String> partitionColumns = getPartitionColumns(keyGenerator, props);
     JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
-    JavaRDD<HoodieRecord> records = avroRDD.map(record -> {
-      GenericRecord gr = isDropPartitionColumns() ? 
HoodieAvroUtils.removeFields(record, partitionColumns) : record;
-      HoodieRecordPayload payload = shouldCombine ? 
DataSourceUtils.createPayload(cfg.payloadClassName, gr,
-          (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, 
cfg.sourceOrderingField, false, props.getBoolean(
-              
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
-              
Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))))
-          : DataSourceUtils.createPayload(cfg.payloadClassName, gr);
-      return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload);
-    });
+
+    JavaRDD<HoodieRecord> records;
+    SerializableSchema avroSchema = new 
SerializableSchema(schemaProvider.getTargetSchema());
+    SerializableSchema processedAvroSchema = new 
SerializableSchema(isDropPartitionColumns() ? 
HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
+    if (recordType == HoodieRecordType.AVRO) {
+      records = avroRDD.map(record -> {
+        GenericRecord gr = isDropPartitionColumns() ? 
HoodieAvroUtils.removeFields(record, partitionColumns) : record;
+        HoodieRecordPayload payload = shouldCombine ? 
DataSourceUtils.createPayload(cfg.payloadClassName, gr,
+            (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, 
cfg.sourceOrderingField, false, props.getBoolean(
+                
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
+                
Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))))
+            : DataSourceUtils.createPayload(cfg.payloadClassName, gr);
+        return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload);
+      });
+    } else if (recordType == HoodieRecordType.SPARK) {
+      // TODO we should remove it if we can read InternalRow from source.

Review Comment:
   We actually already have Jira for it, let me find it



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java:
##########
@@ -105,8 +105,11 @@ public long getTotalRecords() {
   }
 
   private ClosableIterator<IndexedRecord> 
getIndexedRecordIteratorInternal(Schema schema, Option<Schema> requestedSchema) 
throws IOException {
-    AvroReadSupport.setAvroReadSchema(conf, schema);
-    if (requestedSchema.isPresent()) {
+    if (!requestedSchema.isPresent()) {
+      AvroReadSupport.setAvroReadSchema(conf, schema);
+    } else {
+      // Make record schema the same as requestedSchema(reader schema)
+      AvroReadSupport.setAvroReadSchema(conf, requestedSchema.get());

Review Comment:
   I don't think we can actually do that. Why do we need to change that?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -492,15 +519,40 @@ private Pair<SchemaProvider, Pair<String, 
JavaRDD<HoodieRecord>>> fetchFromSourc
     boolean shouldCombine = cfg.filterDupes || 
cfg.operation.equals(WriteOperationType.UPSERT);
     Set<String> partitionColumns = getPartitionColumns(keyGenerator, props);
     JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
-    JavaRDD<HoodieRecord> records = avroRDD.map(record -> {
-      GenericRecord gr = isDropPartitionColumns() ? 
HoodieAvroUtils.removeFields(record, partitionColumns) : record;
-      HoodieRecordPayload payload = shouldCombine ? 
DataSourceUtils.createPayload(cfg.payloadClassName, gr,
-          (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, 
cfg.sourceOrderingField, false, props.getBoolean(
-              
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
-              
Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))))
-          : DataSourceUtils.createPayload(cfg.payloadClassName, gr);
-      return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload);
-    });
+
+    JavaRDD<HoodieRecord> records;
+    SerializableSchema avroSchema = new 
SerializableSchema(schemaProvider.getTargetSchema());
+    SerializableSchema processedAvroSchema = new 
SerializableSchema(isDropPartitionColumns() ? 
HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
+    if (recordType == HoodieRecordType.AVRO) {
+      records = avroRDD.map(record -> {

Review Comment:
   Code seems unchanged, but want to double-check with you: this section didn't 
change, right?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java:
##########
@@ -108,7 +117,12 @@ private ClosableIterator<InternalRow> 
getInternalRowIterator(Schema readerSchema
 
   @Override
   public Schema getSchema() {
-    return parquetUtils.readAvroSchema(conf, path);
+    // Some types in avro are not compatible with parquet
+    // decimal as int32/int64
+    MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, 
path);
+    StructType structType = new 
ParquetToSparkSchemaConverter(conf).convert(messageType);

Review Comment:
   Oh, i see, let's capture that in the comment: that Avro only supports 
representing Decimals as fixed byte array and therefore if we convert to Avro 
directly we'll lose logical type-info



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