amrishlal commented on code in PR #8956: URL: https://github.com/apache/hudi/pull/8956#discussion_r1234440258
########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCreateRecordUtils.scala: ########## @@ -0,0 +1,285 @@ +/* + * 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 + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericRecord +import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, PAYLOAD_CLASS_NAME, PRECOMBINE_FIELD} +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.{HoodieKey, HoodieRecord, HoodieRecordLocation, HoodieSparkRecord, WriteOperationType} +import org.apache.hudi.common.model.HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.exception.HoodieException +import org.apache.hudi.keygen.constant.KeyGeneratorOptions +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory +import org.apache.hudi.keygen.{BaseKeyGenerator, KeyGenUtils, SparkKeyGeneratorInterface} +import org.apache.spark.TaskContext +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.HoodieInternalRowUtils.getCachedUnsafeRowWriter +import org.apache.spark.sql.{DataFrame, HoodieInternalRowUtils} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.StructType +import org.slf4j.LoggerFactory + +import scala.collection.JavaConversions.mapAsJavaMap + +/** Utility class for converting dataframe into RDD[HoodieRecord]. */ +object HoodieCreateRecordUtils { + private val log = LoggerFactory.getLogger(getClass) + + def createHoodieRecordRdd(df: DataFrame, + config: HoodieWriteConfig, + parameters: Map[String, String], + recordName: String, + recordNameSpace: String, + writerSchema: Schema, + dataFileSchema: Schema, + operation: WriteOperationType, + instantTime: String, + isPrepped: Boolean) = { + val shouldDropPartitionColumns = config.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS) + val recordType = config.getRecordMerger.getRecordType + val autoGenerateRecordKeys: Boolean = !parameters.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()); + + val shouldCombine = if (!isPrepped && WriteOperationType.isInsert(operation)) { + parameters(INSERT_DROP_DUPS.key()).toBoolean || + parameters.getOrElse( + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(), + HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue() + ).toBoolean + } else if (!isPrepped && WriteOperationType.isUpsert(operation)) { + parameters.getOrElse( + HoodieWriteConfig.COMBINE_BEFORE_UPSERT.key(), + HoodieWriteConfig.COMBINE_BEFORE_UPSERT.defaultValue() + ).toBoolean + } else { + !isPrepped + } + + // NOTE: Avro's [[Schema]] can't be effectively serialized by JVM native serialization framework + // (due to containing cyclic refs), therefore we have to convert it to string before + // passing onto the Executor + val dataFileSchemaStr = dataFileSchema.toString + + log.debug(s"Creating HoodieRecords (as $recordType)") + + recordType match { + case HoodieRecord.HoodieRecordType.AVRO => + // avroRecords will contain meta fields when isPrepped is true. + val avroRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, recordName, recordNameSpace, + Some(writerSchema)) + + if (isPrepped) { + // For prepped records, check the first record to make sure it has meta fields set. + validateMetaFieldsInAvroRecords(avroRecords.first()) Review Comment: Fixed. ########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCreateRecordUtils.scala: ########## @@ -0,0 +1,285 @@ +/* + * 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 + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericRecord +import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, PAYLOAD_CLASS_NAME, PRECOMBINE_FIELD} +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.{HoodieKey, HoodieRecord, HoodieRecordLocation, HoodieSparkRecord, WriteOperationType} +import org.apache.hudi.common.model.HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.exception.HoodieException +import org.apache.hudi.keygen.constant.KeyGeneratorOptions +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory +import org.apache.hudi.keygen.{BaseKeyGenerator, KeyGenUtils, SparkKeyGeneratorInterface} +import org.apache.spark.TaskContext +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.HoodieInternalRowUtils.getCachedUnsafeRowWriter +import org.apache.spark.sql.{DataFrame, HoodieInternalRowUtils} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.StructType +import org.slf4j.LoggerFactory + +import scala.collection.JavaConversions.mapAsJavaMap + +/** Utility class for converting dataframe into RDD[HoodieRecord]. */ +object HoodieCreateRecordUtils { + private val log = LoggerFactory.getLogger(getClass) + + def createHoodieRecordRdd(df: DataFrame, + config: HoodieWriteConfig, + parameters: Map[String, String], + recordName: String, + recordNameSpace: String, + writerSchema: Schema, + dataFileSchema: Schema, + operation: WriteOperationType, + instantTime: String, + isPrepped: Boolean) = { + val shouldDropPartitionColumns = config.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS) + val recordType = config.getRecordMerger.getRecordType + val autoGenerateRecordKeys: Boolean = !parameters.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()); + + val shouldCombine = if (!isPrepped && WriteOperationType.isInsert(operation)) { + parameters(INSERT_DROP_DUPS.key()).toBoolean || + parameters.getOrElse( + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(), + HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue() + ).toBoolean + } else if (!isPrepped && WriteOperationType.isUpsert(operation)) { + parameters.getOrElse( + HoodieWriteConfig.COMBINE_BEFORE_UPSERT.key(), + HoodieWriteConfig.COMBINE_BEFORE_UPSERT.defaultValue() + ).toBoolean + } else { + !isPrepped + } + + // NOTE: Avro's [[Schema]] can't be effectively serialized by JVM native serialization framework + // (due to containing cyclic refs), therefore we have to convert it to string before + // passing onto the Executor + val dataFileSchemaStr = dataFileSchema.toString + + log.debug(s"Creating HoodieRecords (as $recordType)") + + recordType match { + case HoodieRecord.HoodieRecordType.AVRO => + // avroRecords will contain meta fields when isPrepped is true. + val avroRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, recordName, recordNameSpace, + Some(writerSchema)) + + if (isPrepped) { + // For prepped records, check the first record to make sure it has meta fields set. + validateMetaFieldsInAvroRecords(avroRecords.first()) + } + + avroRecords.mapPartitions(it => { + val sparkPartitionId = TaskContext.getPartitionId() + val keyGenProps = new TypedProperties(config.getProps) + if (autoGenerateRecordKeys) { + keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(sparkPartitionId)) + keyGenProps.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime) + } + val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(keyGenProps) + .asInstanceOf[BaseKeyGenerator] + + val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr) + val consistentLogicalTimestampEnabled = parameters.getOrElse( + DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean + + // handle dropping partition columns + it.map { avroRec => + val (hoodieKey: HoodieKey, recordLocation: Option[HoodieRecordLocation]) = HoodieCreateRecordUtils.getHoodieKeyAndMaybeLocationFromAvroRecord(keyGenerator, avroRec, + isPrepped) + val avroRecWithoutMeta: GenericRecord = if (isPrepped) { + HoodieAvroUtils.rewriteRecord(avroRec, HoodieAvroUtils.removeMetadataFields(dataFileSchema)) + } else { + avroRec + } + + val processedRecord = if (shouldDropPartitionColumns) { + HoodieAvroUtils.rewriteRecord(avroRecWithoutMeta, dataFileSchema) + } else { + avroRecWithoutMeta + } + + val hoodieRecord = if (shouldCombine) { + val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRec, config.getString(PRECOMBINE_FIELD), + false, consistentLogicalTimestampEnabled).asInstanceOf[Comparable[_]] + DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, hoodieKey, + config.getString(PAYLOAD_CLASS_NAME), recordLocation) + } else { + DataSourceUtils.createHoodieRecord(processedRecord, hoodieKey, + config.getString(PAYLOAD_CLASS_NAME), recordLocation) + } + hoodieRecord + } + }).toJavaRDD() + + case HoodieRecord.HoodieRecordType.SPARK => + val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr) + val dataFileStructType = HoodieInternalRowUtils.getCachedSchema(dataFileSchema) + val writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema) + val sourceStructType = df.schema + + if (isPrepped) { + // For prepped records, check the record schema to make sure it has meta fields set. + validateMetaFieldsInSparkRecords(sourceStructType) Review Comment: Fixed. -- 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]
