vinothchandar commented on a change in pull request #1834: URL: https://github.com/apache/hudi/pull/1834#discussion_r469399755
########## File path: hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala ########## @@ -0,0 +1,76 @@ +/* + * 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.hudi.DataSourceWriteOptions._ +import org.apache.hudi.common.config.TypedProperties + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + +/** + * WriterUtils to assist in write path in Datasource and tests. + */ +object HoodieWriterUtils { + + def javaParametersWithWriteDefaults(parameters: java.util.Map[String, String]): java.util.Map[String, String] = { + mapAsJavaMap(parametersWithWriteDefaults(parameters.asScala.toMap)) + } + + /** + * Add default options for unspecified write options keys. + * + * @param parameters + * @return + */ + def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = { + Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL, + TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL, + PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL, + PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL, + RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL, + PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL, + KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, + COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL, + INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL, + STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL, + STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL, + STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL, + META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS, + HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL, + META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL, + HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL, + HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL, + HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL, + HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL, + HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL, + HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL, + HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL, + HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL, + HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL, + HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL, + ASYNC_COMPACT_ENABLE_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_VAL Review comment: I added this from `HoodieSparkSQLWriter`so we can have.just one method ########## File path: hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java ########## @@ -0,0 +1,89 @@ +/* + * 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.io.storage; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; +import org.apache.spark.sql.types.StructType; + +import java.util.HashMap; + +import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY; +import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE; +import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER; +import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER; + +/** + * Hoodie Write Support for directly writing Row to Parquet. + */ +public class HoodieRowParquetWriteSupport extends ParquetWriteSupport { + + private Configuration hadoopConf; + private BloomFilter bloomFilter; + private String minRecordKey; + private String maxRecordKey; + + public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, BloomFilter bloomFilter) { + super(); + Configuration hadoopConf = new Configuration(conf); + hadoopConf.set("spark.sql.parquet.writeLegacyFormat", "false"); Review comment: yes. why we are hardcoding this. any ideas @bvaradar ? ########## File path: hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java ########## @@ -35,6 +35,7 @@ // bulk insert BULK_INSERT("bulk_insert"), BULK_INSERT_PREPPED("bulk_insert_prepped"), + BULK_INSERT_DATASET("bulk_insert_dataset"), Review comment: removing this. it was easy enough. ########## File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala ########## @@ -297,8 +298,9 @@ object DataSourceWriteOptions { val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false" val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" val DEFAULT_HIVE_USE_JDBC_OPT_VAL = "true" - // Async Compaction - Enabled by default for MOR val ASYNC_COMPACT_ENABLE_KEY = "hoodie.datasource.compaction.async.enable" val DEFAULT_ASYNC_COMPACT_ENABLE_VAL = "true" + // Internal configs Review comment: this is publicly visible. cannot add this here. ########## File path: hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java ########## @@ -0,0 +1,243 @@ +/* + * 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.client.model; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * Internal Row implementation for Hoodie Row. It wraps an {@link InternalRow} and keeps meta columns locally. But the {@link InternalRow} + * does include the meta columns as well just that {@link HoodieInternalRow} will intercept queries for meta columns and serve from its + * copy rather than fetching from {@link InternalRow}. + */ +public class HoodieInternalRow extends InternalRow { + + private String commitTime; + private String commitSeqNumber; + private String recordKey; + private String partitionPath; + private String fileName; + private InternalRow row; + + public HoodieInternalRow(String commitTime, String commitSeqNumber, String recordKey, String partitionPath, + String fileName, InternalRow row) { + this.commitTime = commitTime; + this.commitSeqNumber = commitSeqNumber; + this.recordKey = recordKey; + this.partitionPath = partitionPath; + this.fileName = fileName; + this.row = row; + } + + @Override + public int numFields() { + return row.numFields(); + } + + @Override + public void setNullAt(int i) { + if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) { + switch (i) { + case 0: { + this.commitTime = null; + break; + } + case 1: { + this.commitSeqNumber = null; + break; + } + case 2: { + this.recordKey = null; + break; + } + case 3: { + this.partitionPath = null; + break; + } + case 4: { + this.fileName = null; + break; + } + default: throw new IllegalArgumentException("Not expected"); + } + } else { + row.setNullAt(i); Review comment: I think this is because row already has these metafields per se in the schema ########## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java ########## @@ -670,7 +670,9 @@ public Builder withPath(String basePath) { } public Builder withSchema(String schemaStr) { - props.setProperty(AVRO_SCHEMA, schemaStr); + if (null != schemaStr) { Review comment: I did that. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
