[HUDI-4319] Fixed Parquet's PLAIN_DICTIONARY encoding not being applied when bulk-inserting (#5966)
* Fixed Dictionary encoding config not being properly propagated to Parquet writer (making it unable to apply it, substantially bloating the storage footprint)
This commit is contained in:
@@ -22,6 +22,7 @@ import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.storage.HoodieParquetConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
@@ -67,7 +68,7 @@ public class HoodieRowDataFileWriterFactory {
|
||||
HoodieRowDataParquetWriteSupport writeSupport =
|
||||
new HoodieRowDataParquetWriteSupport(table.getHadoopConf(), rowType, filter);
|
||||
return new HoodieRowDataParquetWriter(
|
||||
path, new HoodieRowDataParquetConfig(
|
||||
path, new HoodieParquetConfig<>(
|
||||
writeSupport,
|
||||
writeConfig.getParquetCompressionCodec(),
|
||||
writeConfig.getParquetBlockSize(),
|
||||
|
||||
@@ -1,36 +0,0 @@
|
||||
/*
|
||||
* 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.HoodieBaseParquetConfig;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
/**
|
||||
* ParquetConfig for datasource implementation with {@link org.apache.flink.table.data.RowData}.
|
||||
*/
|
||||
public class HoodieRowDataParquetConfig extends HoodieBaseParquetConfig<HoodieRowDataParquetWriteSupport> {
|
||||
|
||||
public HoodieRowDataParquetConfig(HoodieRowDataParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
|
||||
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
|
||||
double compressionRatio) {
|
||||
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
|
||||
}
|
||||
}
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.io.storage.HoodieParquetConfig;
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
|
||||
@@ -39,7 +40,7 @@ public class HoodieRowDataParquetWriter extends ParquetWriter<RowData>
|
||||
private final long maxFileSize;
|
||||
private final HoodieRowDataParquetWriteSupport writeSupport;
|
||||
|
||||
public HoodieRowDataParquetWriter(Path file, HoodieRowDataParquetConfig parquetConfig)
|
||||
public HoodieRowDataParquetWriter(Path file, HoodieParquetConfig<HoodieRowDataParquetWriteSupport> parquetConfig)
|
||||
throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
|
||||
|
||||
Reference in New Issue
Block a user