You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/09/23 19:28:22 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6745: Fix comment in RFC46

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


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/HoodieSparkRecordSerializer.scala:
##########
@@ -97,25 +90,26 @@ class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSeriali
    */
   def serializeDatum(datum: HoodieSparkRecord, output: Output): Unit = {
     val schema = datum.getStructType
-    val fingerprint = fingerprintCache.getOrElseUpdate(schema, {
-      SchemaNormalization.fingerprint64(schema.json.getBytes(StandardCharsets.UTF_8))
-    })
-    schemas.get(fingerprint) match {
-      case Some(_) =>
-        output.writeBoolean(true)
-        output.writeLong(fingerprint)
-      case None =>
-        output.writeBoolean(false)
-        val compressedSchema = compress(schema)
-        output.writeInt(compressedSchema.length)
-        output.writeBytes(compressedSchema)
+    if (HoodieInternalRowUtils.containsCompressedSchema(schema)) {
+      val fingerprint = HoodieInternalRowUtils.getCachedFingerPrintFromSchema(schema)
+      output.writeBoolean(true)
+      output.writeLong(fingerprint)
+    } else {
+      output.writeBoolean(false)
+      val compressedSchema = compress(schema)
+      output.writeInt(compressedSchema.length)
+      output.writeBytes(compressedSchema)
     }
 
     val record = datum.newInstance().asInstanceOf[HoodieSparkRecord]
     record.setStructType(null)
-    val stream = new ObjectOutputStream(output)
+    val byteStream = new ByteArrayOutputStream()

Review Comment:
   Sorry, i skimmed t/h this initially but overlooked how we're doing the serialization here:
   
    -   We should only serialize the `UnsafeRow` that HoodieSparkRecord holds (see my comment above); `UnsafeRow` implements KryoSerializable so we can use these methods to properly write it out into the stream; we should not be relying on Java's object serialization framework for that



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -47,42 +50,49 @@
 import java.util.Properties;
 
 import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.hudi.util.HoodieSparkRecordUtils.getNullableValAsString;
+import static org.apache.hudi.util.HoodieSparkRecordUtils.getValue;
 import static org.apache.spark.sql.types.DataTypes.BooleanType;
 import static org.apache.spark.sql.types.DataTypes.StringType;
 
 /**
  * Spark Engine-specific Implementations of `HoodieRecord`.
  */
+@DefaultSerializer(HoodieSparkRecordSerializer.class)
 public class HoodieSparkRecord extends HoodieRecord<InternalRow> {

Review Comment:
   We need to make sure that 
   
    - We only hold `UnsafeRow` w/in `HoodieSparkRecord` (for the same reasons we hold bytes in `HoodieLegacyAvroRecord`); whenever we instantiate HoodieSparkRecord we need to check whether passed in InternalRow is UnsafeRow and if not we should do UnsafeProjection (that will serialize it to bytes)
   



##########
hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatLists.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.common.util.collection;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoSerializable;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+/**
+ * Space-efficient, comparable, immutable lists, copied from calcite core.
+ */
+public class FlatLists {

Review Comment:
   Shall we call this `ComparableLists`? "Flat" reference is confusing



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -152,8 +156,7 @@ protected <T> void processNextRecord(HoodieRecord<T> hoodieRecord) throws IOExce
       T combinedValue = ((HoodieRecord<T>) recordMerger.merge(oldRecord, hoodieRecord, readerSchema, this.hoodieTableMetaClient.getTableConfig().getProps()).get()).getData();
       // If combinedValue is oldValue, no need rePut oldRecord
       if (combinedValue != oldValue) {
-        hoodieRecord.setData(combinedValue);
-        records.put(key, hoodieRecord);
+        records.put(key, hoodieRecord.newInstance(combinedValue));

Review Comment:
   @wzx140 why do we still need to do `newInstance` why can't we use `HoodieRecord` returned by the merger?
   



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/HoodieSparkRecordSerializer.scala:
##########
@@ -38,19 +38,12 @@ import scala.collection.mutable
  * schema, as to reduce network IO.
  * Actions like parsing or compressing schemas are computationally expensive so the serializer
  * caches all previously seen values as to reduce the amount of work needed to do.
- * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
- *                string representation of the Avro schema, used to decrease the amount of data
- *                that needs to be serialized.
  */
-class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {
+class HoodieSparkRecordSerializer() extends KSerializer[HoodieSparkRecord] {

Review Comment:
   You can drop ()



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -94,18 +62,11 @@ private static Option<String> getNullableValAsString(StructType structType, Inte
   public static Object getRecordColumnValues(InternalRow row,

Review Comment:
   Let's not forget to change the interface to return `Object[]`



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org