1
0

[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:
Alexey Kudinkin
2022-06-24 20:52:28 -07:00
committed by GitHub
parent 360df576a9
commit c86edfc28e
16 changed files with 50 additions and 158 deletions

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.HoodieParquetConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.Path;
@@ -68,14 +69,17 @@ public class HoodieInternalRowFileWriterFactory {
HoodieRowParquetWriteSupport writeSupport =
new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, filter, writeConfig);
return new HoodieInternalRowParquetWriter(
path, new HoodieRowParquetConfig(
path,
new HoodieParquetConfig<>(
writeSupport,
writeConfig.getParquetCompressionCodec(),
writeConfig.getParquetBlockSize(),
writeConfig.getParquetPageSize(),
writeConfig.getParquetMaxFileSize(),
writeSupport.getHadoopConf(),
writeConfig.getParquetCompressionRatio()));
writeConfig.getParquetCompressionRatio(),
writeConfig.parquetDictionaryEnabled()
));
}
public static HoodieInternalRowFileWriter getInternalRowFileWriterWithoutMetaFields(
@@ -93,13 +97,15 @@ public class HoodieInternalRowFileWriterFactory {
HoodieRowParquetWriteSupport writeSupport =
new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, null, writeConfig);
return new HoodieInternalRowParquetWriter(
path, new HoodieRowParquetConfig(
path, new HoodieParquetConfig<>(
writeSupport,
writeConfig.getParquetCompressionCodec(),
writeConfig.getParquetBlockSize(),
writeConfig.getParquetPageSize(),
writeConfig.getParquetMaxFileSize(),
writeSupport.getHadoopConf(),
writeConfig.getParquetCompressionRatio()));
writeConfig.getParquetCompressionRatio(),
writeConfig.parquetDictionaryEnabled())
);
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.io.storage.row;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.io.storage.HoodieParquetConfig;
import org.apache.hudi.io.storage.HoodieBaseParquetWriter;
import org.apache.spark.sql.catalyst.InternalRow;
@@ -32,7 +33,7 @@ public class HoodieInternalRowParquetWriter extends HoodieBaseParquetWriter<Inte
private final HoodieRowParquetWriteSupport writeSupport;
public HoodieInternalRowParquetWriter(Path file, HoodieRowParquetConfig parquetConfig)
public HoodieInternalRowParquetWriter(Path file, HoodieParquetConfig<HoodieRowParquetWriteSupport> parquetConfig)
throws IOException {
super(file, parquetConfig);

View File

@@ -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.hudi.client.model.HoodieInternalRow}.
*/
public class HoodieRowParquetConfig extends HoodieBaseParquetConfig<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);
}
}