From a5b48e21ab1ac232d11efc477ecbe222267c912e Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 11 Nov 2024 14:08:20 -0800 Subject: [PATCH] [GH-3035][ParquetRewriter] removed ColumnChunkMetaData.get(path, type) --- .../hadoop/metadata/ColumnChunkMetaData.java | 23 ------------------- .../hadoop/rewrite/ParquetRewriter.java | 17 +++++++++++--- .../hadoop/rewrite/RewriteOptions.java | 2 +- 3 files changed, 15 insertions(+), 27 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java index 6c0967ad3e..14a949b0e0 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java @@ -504,29 +504,6 @@ public boolean hasDictionaryPage() { public boolean isEncrypted() { return false; } - - /** - * Copies this ColumnChunkMetaData with path and type changed to provided ones. - * - * @param path a new ColumnPath of a chunk - * @param type a new PrimitiveType of a chunk - * @return resulting chunk - */ - public ColumnChunkMetaData copy(ColumnPath path, PrimitiveType type) { - return ColumnChunkMetaData.get( - path, - type, - this.getCodec(), - this.getEncodingStats(), - this.getEncodings(), - this.getStatistics(), - this.getFirstDataPageOffset(), - this.getDictionaryPageOffset(), - this.getValueCount(), - this.getTotalSize(), - this.getTotalUncompressedSize(), - this.getSizeStatistics()); - } } class IntColumnChunkMetaData extends ColumnChunkMetaData { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index e6c9d8e70a..865ec0fce6 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -155,7 +155,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { this.newCodecName = options.getNewCodecName(); this.indexCacheStrategy = options.getIndexCacheStrategy(); this.overwriteInputWithJoinColumns = options.getOverwriteInputWithJoinColumns(); - this.renamedColumns = options.gerRenameColumns(); + this.renamedColumns = options.getRenameColumns(); ParquetConfiguration conf = options.getParquetConfiguration(); this.inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); this.inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoin(), conf)); @@ -495,8 +495,19 @@ private void processBlock( ColumnChunkMetaData chunkNormalized = chunk; if (!renamedColumns.isEmpty()) { - chunkNormalized = - chunk.copy(normalizeFieldsInPath(chunk.getPath()), normalizeNameInType(chunk.getPrimitiveType())); + chunkNormalized = ColumnChunkMetaData.get( + normalizeFieldsInPath(chunk.getPath()), + normalizeNameInType(chunk.getPrimitiveType()), + chunk.getCodec(), + chunk.getEncodingStats(), + chunk.getEncodings(), + chunk.getStatistics(), + chunk.getFirstDataPageOffset(), + chunk.getDictionaryPageOffset(), + chunk.getValueCount(), + chunk.getTotalSize(), + chunk.getTotalUncompressedSize(), + chunk.getSizeStatistics()); } ColumnDescriptor descriptorOriginal = outSchema.getColumns().get(outColumnIdx); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index a2a1c18e32..f85b65ea3d 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -198,7 +198,7 @@ public Map getMaskColumns() { return maskColumns; } - public Map gerRenameColumns() { + public Map getRenameColumns() { return renameColumns; }