Skip to content

Commit 686f071

Browse files
authored
GH-3035: ParquetRewriter: Add a column renaming feature (#3036)
1 parent 34359c9 commit 686f071

File tree

3 files changed

+363
-106
lines changed

3 files changed

+363
-106
lines changed

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java

Lines changed: 114 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -109,7 +109,7 @@
109109
* Please note the schema of all <code>inputFiles</code> must be the same, otherwise the rewrite will fail.
110110
* <p>
111111
* <h2>Applying column transformations</h2>
112-
* Some supported column transformations: pruning, masking, encrypting, changing a codec.
112+
* Some supported column transformations: pruning, masking, renaming, encrypting, changing a codec.
113113
* See {@link RewriteOptions} and {@link RewriteOptions.Builder} for the full list with description.
114114
* <p>
115115
* <h2><i>Joining</i> with extra files with a different schema</h2>
@@ -149,28 +149,30 @@ public class ParquetRewriter implements Closeable {
149149
private final IndexCache.CacheStrategy indexCacheStrategy;
150150
private final boolean overwriteInputWithJoinColumns;
151151
private final InternalFileEncryptor nullColumnEncryptor;
152+
private final Map<String, String> renamedColumns;
152153

153154
public ParquetRewriter(RewriteOptions options) throws IOException {
154155
this.newCodecName = options.getNewCodecName();
155156
this.indexCacheStrategy = options.getIndexCacheStrategy();
156157
this.overwriteInputWithJoinColumns = options.getOverwriteInputWithJoinColumns();
158+
this.renamedColumns = options.getRenameColumns();
157159
ParquetConfiguration conf = options.getParquetConfiguration();
158-
OutputFile out = options.getParquetOutputFile();
159-
inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf));
160-
inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoin(), conf));
160+
this.inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf));
161+
this.inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoin(), conf));
162+
this.outSchema = pruneColumnsInSchema(getSchema(), options.getPruneColumns());
163+
this.extraMetaData = getExtraMetadata(options);
161164
ensureSameSchema(inputFiles);
162165
ensureSameSchema(inputFilesToJoin);
163166
ensureRowCount();
167+
ensureRenamingCorrectness(outSchema, renamedColumns);
168+
OutputFile out = options.getParquetOutputFile();
164169
LOG.info(
165170
"Start rewriting {} input file(s) {} to {}",
166171
inputFiles.size() + inputFilesToJoin.size(),
167172
Stream.concat(options.getParquetInputFiles().stream(), options.getParquetInputFilesToJoin().stream())
168173
.collect(Collectors.toList()),
169174
out);
170175

171-
this.outSchema = pruneColumnsInSchema(getSchema(), options.getPruneColumns());
172-
this.extraMetaData = getExtraMetadata(options);
173-
174176
if (options.getMaskColumns() != null) {
175177
this.maskColumns = new HashMap<>();
176178
for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
@@ -184,9 +186,9 @@ public ParquetRewriter(RewriteOptions options) throws IOException {
184186
}
185187

186188
ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
187-
writer = new ParquetFileWriter(
189+
this.writer = new ParquetFileWriter(
188190
out,
189-
outSchema,
191+
renamedColumns.isEmpty() ? outSchema : getSchemaWithRenamedColumns(this.outSchema),
190192
writerMode,
191193
DEFAULT_BLOCK_SIZE,
192194
MAX_PADDING_SIZE_DEFAULT,
@@ -200,7 +202,8 @@ public ParquetRewriter(RewriteOptions options) throws IOException {
200202
this.nullColumnEncryptor = null;
201203
} else {
202204
this.nullColumnEncryptor = new InternalFileEncryptor(options.getFileEncryptionProperties());
203-
List<ColumnDescriptor> columns = outSchema.getColumns();
205+
List<ColumnDescriptor> columns =
206+
getSchemaWithRenamedColumns(this.outSchema).getColumns();
204207
for (int i = 0; i < columns.size(); i++) {
205208
writer.getEncryptor()
206209
.getColumnSetup(ColumnPath.get(columns.get(i).getPath()), true, i);
@@ -223,8 +226,8 @@ public ParquetRewriter(
223226
this.writer = writer;
224227
this.outSchema = outSchema;
225228
this.newCodecName = codecName;
226-
extraMetaData = new HashMap<>(meta.getFileMetaData().getKeyValueMetaData());
227-
extraMetaData.put(
229+
this.extraMetaData = new HashMap<>(meta.getFileMetaData().getKeyValueMetaData());
230+
this.extraMetaData.put(
228231
ORIGINAL_CREATED_BY_KEY,
229232
originalCreatedBy != null
230233
? originalCreatedBy
@@ -239,6 +242,7 @@ public ParquetRewriter(
239242
this.indexCacheStrategy = IndexCache.CacheStrategy.NONE;
240243
this.overwriteInputWithJoinColumns = false;
241244
this.nullColumnEncryptor = null;
245+
this.renamedColumns = new HashMap<>();
242246
}
243247

244248
private MessageType getSchema() {
@@ -266,6 +270,27 @@ private MessageType getSchema() {
266270
}
267271
}
268272

273+
private MessageType getSchemaWithRenamedColumns(MessageType schema) {
274+
List<Type> fields = schema.getFields().stream()
275+
.map(type -> {
276+
if (!renamedColumns.containsKey(type.getName())) {
277+
return type;
278+
} else if (type.isPrimitive()) {
279+
return new PrimitiveType(
280+
type.getRepetition(),
281+
type.asPrimitiveType().getPrimitiveTypeName(),
282+
renamedColumns.get(type.getName()));
283+
} else {
284+
return new GroupType(
285+
type.getRepetition(),
286+
renamedColumns.get(type.getName()),
287+
type.asGroupType().getFields());
288+
}
289+
})
290+
.collect(Collectors.toList());
291+
return new MessageType(schema.getName(), fields);
292+
}
293+
269294
private Map<String, String> getExtraMetadata(RewriteOptions options) {
270295
List<TransParquetFileReader> allFiles;
271296
if (options.getIgnoreJoinFilesMetadata()) {
@@ -338,6 +363,21 @@ private void ensureSameSchema(Queue<TransParquetFileReader> inputFileReaders) {
338363
}
339364
}
340365

366+
private void ensureRenamingCorrectness(MessageType schema, Map<String, String> renameMap) {
367+
Set<String> columns = schema.getFields().stream().map(Type::getName).collect(Collectors.toSet());
368+
renameMap.forEach((src, dst) -> {
369+
if (!columns.contains(src)) {
370+
String msg = String.format("Column to rename '%s' is not found in input files schema", src);
371+
LOG.error(msg);
372+
throw new IllegalArgumentException(msg);
373+
} else if (columns.contains(dst)) {
374+
String msg = String.format("Renamed column target name '%s' is already present in a schema", dst);
375+
LOG.error(msg);
376+
throw new IllegalArgumentException(msg);
377+
}
378+
});
379+
}
380+
341381
@Override
342382
public void close() throws IOException {
343383
writer.end(extraMetaData);
@@ -421,6 +461,27 @@ public void processBlocks() throws IOException {
421461
if (readerToJoin != null) readerToJoin.close();
422462
}
423463

464+
private ColumnPath normalizeFieldsInPath(ColumnPath path) {
465+
if (renamedColumns.isEmpty()) {
466+
return path;
467+
} else {
468+
String[] pathArray = path.toArray();
469+
pathArray[0] = renamedColumns.getOrDefault(pathArray[0], pathArray[0]);
470+
return ColumnPath.get(pathArray);
471+
}
472+
}
473+
474+
private PrimitiveType normalizeNameInType(PrimitiveType type) {
475+
if (renamedColumns.isEmpty()) {
476+
return type;
477+
} else {
478+
return new PrimitiveType(
479+
type.getRepetition(),
480+
type.asPrimitiveType().getPrimitiveTypeName(),
481+
renamedColumns.getOrDefault(type.getName(), type.getName()));
482+
}
483+
}
484+
424485
private void processBlock(
425486
TransParquetFileReader reader,
426487
int blockIdx,
@@ -431,7 +492,28 @@ private void processBlock(
431492
if (chunk.isEncrypted()) {
432493
throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
433494
}
434-
ColumnDescriptor descriptor = outSchema.getColumns().get(outColumnIdx);
495+
496+
ColumnChunkMetaData chunkNormalized = chunk;
497+
if (!renamedColumns.isEmpty()) {
498+
// Keep an eye if this get stale because of ColumnChunkMetaData change
499+
chunkNormalized = ColumnChunkMetaData.get(
500+
normalizeFieldsInPath(chunk.getPath()),
501+
normalizeNameInType(chunk.getPrimitiveType()),
502+
chunk.getCodec(),
503+
chunk.getEncodingStats(),
504+
chunk.getEncodings(),
505+
chunk.getStatistics(),
506+
chunk.getFirstDataPageOffset(),
507+
chunk.getDictionaryPageOffset(),
508+
chunk.getValueCount(),
509+
chunk.getTotalSize(),
510+
chunk.getTotalUncompressedSize(),
511+
chunk.getSizeStatistics());
512+
}
513+
514+
ColumnDescriptor descriptorOriginal = outSchema.getColumns().get(outColumnIdx);
515+
ColumnDescriptor descriptorRenamed =
516+
getSchemaWithRenamedColumns(outSchema).getColumns().get(outColumnIdx);
435517
BlockMetaData blockMetaData = reader.getFooter().getBlocks().get(blockIdx);
436518
String originalCreatedBy = reader.getFileMetaData().getCreatedBy();
437519

@@ -443,13 +525,21 @@ private void processBlock(
443525
// Mask column and compress it again.
444526
MaskMode maskMode = maskColumns.get(chunk.getPath());
445527
if (maskMode.equals(MaskMode.NULLIFY)) {
446-
Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
528+
Type.Repetition repetition =
529+
descriptorOriginal.getPrimitiveType().getRepetition();
447530
if (repetition.equals(Type.Repetition.REQUIRED)) {
448-
throw new IOException(
449-
"Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
531+
throw new IOException("Required column ["
532+
+ descriptorOriginal.getPrimitiveType().getName() + "] cannot be nullified");
450533
}
451534
nullifyColumn(
452-
reader, blockIdx, descriptor, chunk, writer, newCodecName, encryptColumn, originalCreatedBy);
535+
reader,
536+
blockIdx,
537+
descriptorOriginal,
538+
chunk,
539+
writer,
540+
newCodecName,
541+
encryptColumn,
542+
originalCreatedBy);
453543
} else {
454544
throw new UnsupportedOperationException("Only nullify is supported for now");
455545
}
@@ -462,7 +552,7 @@ private void processBlock(
462552
}
463553

464554
// Translate compression and/or encryption
465-
writer.startColumn(descriptor, chunk.getValueCount(), newCodecName);
555+
writer.startColumn(descriptorRenamed, chunk.getValueCount(), newCodecName);
466556
processChunk(
467557
reader,
468558
blockMetaData.getRowCount(),
@@ -480,7 +570,8 @@ private void processBlock(
480570
BloomFilter bloomFilter = indexCache.getBloomFilter(chunk);
481571
ColumnIndex columnIndex = indexCache.getColumnIndex(chunk);
482572
OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk);
483-
writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
573+
writer.appendColumnChunk(
574+
descriptorRenamed, reader.getStream(), chunkNormalized, bloomFilter, columnIndex, offsetIndex);
484575
}
485576
}
486577

@@ -522,7 +613,7 @@ private void processChunk(
522613
}
523614

524615
if (bloomFilter != null) {
525-
writer.addBloomFilter(chunk.getPath().toDotString(), bloomFilter);
616+
writer.addBloomFilter(normalizeFieldsInPath(chunk.getPath()).toDotString(), bloomFilter);
526617
}
527618

528619
reader.setStreamPosition(chunk.getStartingPos());
@@ -580,7 +671,7 @@ private void processChunk(
580671
dataPageAAD);
581672
statistics = convertStatistics(
582673
originalCreatedBy,
583-
chunk.getPrimitiveType(),
674+
normalizeNameInType(chunk.getPrimitiveType()),
584675
headerV1.getStatistics(),
585676
columnIndex,
586677
pageOrdinal,
@@ -648,7 +739,7 @@ private void processChunk(
648739
dataPageAAD);
649740
statistics = convertStatistics(
650741
originalCreatedBy,
651-
chunk.getPrimitiveType(),
742+
normalizeNameInType(chunk.getPrimitiveType()),
652743
headerV2.getStatistics(),
653744
columnIndex,
654745
pageOrdinal,
@@ -887,7 +978,7 @@ private void nullifyColumn(
887978
CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(newCodecName);
888979

889980
// Create new schema that only has the current column
890-
MessageType newSchema = newSchema(outSchema, descriptor);
981+
MessageType newSchema = getSchemaWithRenamedColumns(newSchema(outSchema, descriptor));
891982
ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
892983
compressor,
893984
newSchema,

0 commit comments

Comments
 (0)