alexeykudinkin commented on code in PR #5629: URL: https://github.com/apache/hudi/pull/5629#discussion_r938247328
########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java: ########## @@ -0,0 +1,140 @@ +/* + * 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.util; + +import org.apache.hudi.HoodieInternalRowUtils; +import org.apache.hudi.commmon.model.HoodieSparkRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.RowKeyGeneratorHelper; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import java.util.List; + +import scala.Tuple2; + +public class HoodieSparkRecordUtils { + + /** + * Utility method to convert bytes to HoodieRecord using schema and payload class. + */ + public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) { + return new HoodieSparkRecord(data, structType); + } + + /** + * Utility method to convert InternalRow to HoodieRecord using schema and payload class. + */ + public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) { + return convertToHoodieSparkRecord(structType, data, preCombineField, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, Option.empty()); + } + + public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField, + Option<String> partitionName) { + return convertToHoodieSparkRecord(structType, data, preCombineField, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, partitionName); + } + + /** + * Utility method to convert bytes to HoodieRecord using schema and payload class. + */ + public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair, + boolean withOperationField, Option<String> partitionName) { + final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString(); + final String partitionPath = (partitionName.isPresent() ? partitionName.get() : + getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString()); + + Object preCombineVal = getPreCombineVal(structType, data, preCombineField); Review Comment: @wzx140 let's pull back here a little bit: > If every SparkRecord has a schema, it will bring huge performance loss (shuffle schema). The current solution is that those spark records that need to be shuffled do not save the schema like record payload. Yeah, we already had that discussion on Slack and it was a fair call-out that i previously had as a hunch and didn't get a chance to explore more thoroughly, which i'll try to rectify now: See, i was coming from an angle that Hudi currently passes fully-deserialized payload (Avro) bearing native Java types across the board. Therefore transitioning to a state where we'd be able to instead pass a *non-serialized* payload will require non-trivial amount of effort, since assumptions that you can get any value from the record *w/o providing any schema* have been baked in pretty firmly in some areas (like the one you're referring to). In that sense, i was thinking that our v1 `HoodieRecord` impl should carry schema as well so that transition from "fully-deserialized" to "non-deserialized" records being passed around would be a smoother sail. We can approach it actually in a way that is similar to how Spark is currently handling it for... Avro actually: every Avro record bears the schema (exposed by `getSchema`) and Spark have solved the problem of needing to pump this superfluous I/O by implementing custom Kryo serializer (`GenericAvroSerializer`) that does NOT serialize the schema, but instead serializes its fingerprint w/ every record and when deserializing puts it back in. This would avoid this "schema-penalty" altogether. cc @vinothchandar > Orderingval needs to be passed in SparkWriteHelper#deduicateRecords for deduplication. Passing schema from HoodieSparkSqlWriter to SparkWriteHelper will need to change many basic APIs We actually don't need to do that: Avro schema is passed w/in `HoodieWriteConfig` which is available in the caller of this method so you can easily extract it and parse it there and pass to `getOrderingVal` method -- 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]
