xushiyan commented on code in PR #5629: URL: https://github.com/apache/hudi/pull/5629#discussion_r919594262
########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java: ########## @@ -0,0 +1,114 @@ +/* + * 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.hudi.HoodieInternalRowUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; +import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; + +public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory { + + private static class SingletonHolder { + + private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory(); + } + + public static HoodieFileWriterFactory getFileWriterFactory() { + return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE; + } + + @Override + protected HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); + boolean enableBloomFilter = populateMetaFields; + Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf, + HoodieInternalRowUtils.getCachedSchema(schema), filter.get(), + HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build()); + HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport, + CompressionCodecName.fromConf(compressionCodecName), + config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), + conf, + config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf()); + + return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)); + } + + protected HoodieFileWriter newParquetFileWriter( + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException { + boolean populateMetaFields = true; + boolean enableBloomFilter = false; + BloomFilter filter = enableBloomFilter ? createBloomFilter(config) : null; + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf, + HoodieInternalRowUtils.getCachedSchema(schema), filter, + HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build()); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } + HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport, + CompressionCodecName.fromConf(compressionCodecName), + config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), + writeSupport.getHadoopConf(), config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf()); Review Comment: looks duplicate from the other newParquetFileWriter() ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java: ########## @@ -0,0 +1,42 @@ +/* + * 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.row; + +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +/** + * ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}. + */ +public class HoodieRowParquetConfig extends HoodieParquetConfig<HoodieRowParquetWriteSupport> { + + public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName, + int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, + double compressionRatio) { + super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio); + } + + public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName, + int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, + double compressionRatio, boolean dictionEnable) { Review Comment: enableDictionary ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java: ########## @@ -0,0 +1,105 @@ +/* + * 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.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HoodieInternalRowUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.api.ReadSupport; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.InputFile; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +public class HoodieSparkParquetReader implements HoodieSparkFileReader { + + private final Path path; + private final Configuration conf; + private final BaseFileUtils parquetUtils; + private List<ParquetReaderIterator> readerIterators = new ArrayList<>(); + + public HoodieSparkParquetReader(Configuration conf, Path path) { + this.path = path; + this.conf = conf; + this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); + } + + @Override + public String[] readMinMaxRecordKeys() { + return parquetUtils.readMinMaxRecordKeys(conf, path); + } + + @Override + public BloomFilter readBloomFilter() { + return parquetUtils.readBloomFilterFromMetadata(conf, path); + } + + @Override + public Set<String> filterRowKeys(Set<String> candidateRowKeys) { + return parquetUtils.filterRowKeys(conf, path, candidateRowKeys); + } + + @Override + public ClosableIterator<InternalRow> getInternalRowIterator(Schema schema) throws IOException { + StructType structType = HoodieInternalRowUtils.getCachedSchema(schema); + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), structType.json()); + // todo: get it from spark context + conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(),false); + conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), true); Review Comment: pls fix this by setting it from environment ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala: ########## @@ -188,29 +211,82 @@ object HoodieInternalRowUtils { schemaMap.get(schema) } + def getProjection(from: Schema, to: Schema): Projection = { + getCachedUnsafeProjection(getCachedSchema(from), getCachedSchema(to)) + } + private def getCachedProjection(from: StructType, to: StructType): Projection = { val schemaPair = (from, to) if (!projectionMap.contains(schemaPair)) { projectionMap.synchronized { if (!projectionMap.contains(schemaPair)) { - val projection = generateMutableProjection(from, to) + val utilsClazz = ReflectionUtils.getClass("org.apache.hudi.HoodieSparkProjectionUtils") Review Comment: @wzx140 i thought we made changes to avoid per-record reflection from previous step. can we stick to the no-reflection principle here? ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java: ########## @@ -0,0 +1,114 @@ +/* + * 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.hudi.HoodieInternalRowUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; +import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; + +public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory { + + private static class SingletonHolder { + + private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory(); + } + + public static HoodieFileWriterFactory getFileWriterFactory() { + return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE; + } + + @Override + protected HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); + boolean enableBloomFilter = populateMetaFields; + Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf, + HoodieInternalRowUtils.getCachedSchema(schema), filter.get(), + HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build()); + HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport, + CompressionCodecName.fromConf(compressionCodecName), + config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), + conf, + config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf()); + + return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)); + } + + protected HoodieFileWriter newParquetFileWriter( + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException { + boolean populateMetaFields = true; + boolean enableBloomFilter = false; + BloomFilter filter = enableBloomFilter ? createBloomFilter(config) : null; + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf, + HoodieInternalRowUtils.getCachedSchema(schema), filter, + HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build()); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } Review Comment: ditto ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java: ########## @@ -0,0 +1,114 @@ +/* + * 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.hudi.HoodieInternalRowUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; +import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; + +public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory { + + private static class SingletonHolder { + + private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory(); + } + + public static HoodieFileWriterFactory getFileWriterFactory() { + return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE; + } + + @Override + protected HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); + boolean enableBloomFilter = populateMetaFields; + Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } Review Comment: this is weird.. can we get rid of this check and validate the values set to PARQUET_COMPRESSION_CODEC_NAME at config level? -- 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]
