1
0

[HUDI-2632] Schema evolution for flink parquet reader (#3872)

This commit is contained in:
Danny Chan
2021-10-27 20:00:24 +08:00
committed by GitHub
parent ae000795d7
commit 909c3ba45e
6 changed files with 131 additions and 28 deletions

View File

@@ -110,10 +110,14 @@ public class FormatUtils {
List<Schema.Field> requiredFields = requiredSchema.getFields();
assert (requiredFields.size() == requiredPos.length);
Iterator<Integer> positionIterator = Arrays.stream(requiredPos).iterator();
requiredFields.forEach(f -> recordBuilder.set(f, record.get(positionIterator.next())));
requiredFields.forEach(f -> recordBuilder.set(f, getVal(record, positionIterator.next())));
return recordBuilder.build();
}
private static Object getVal(IndexedRecord record, int pos) {
return pos == -1 ? null : record.get(pos);
}
public static HoodieMergedLogRecordScanner logScanner(
MergeOnReadInputSplit split,
Schema logSchema,

View File

@@ -46,6 +46,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.stream.IntStream;
import static org.apache.hudi.table.format.cow.ParquetSplitReaderUtil.createColumnReader;
import static org.apache.hudi.table.format.cow.ParquetSplitReaderUtil.createWritableColumnVector;
@@ -67,6 +68,8 @@ public class ParquetColumnarRowSplitReader implements Closeable {
private final MessageType fileSchema;
private final LogicalType[] requestedTypes;
private final MessageType requestedSchema;
/**
@@ -81,8 +84,6 @@ public class ParquetColumnarRowSplitReader implements Closeable {
private final ColumnarRowData row;
private final LogicalType[] selectedTypes;
private final int batchSize;
private ParquetFileReader reader;
@@ -121,7 +122,6 @@ public class ParquetColumnarRowSplitReader implements Closeable {
long splitStart,
long splitLength) throws IOException {
this.utcTimestamp = utcTimestamp;
this.selectedTypes = selectedTypes;
this.batchSize = batchSize;
// then we need to apply the predicate push down filter
ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitStart + splitLength));
@@ -130,7 +130,13 @@ public class ParquetColumnarRowSplitReader implements Closeable {
List<BlockMetaData> blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
this.fileSchema = footer.getFileMetaData().getSchema();
this.requestedSchema = clipParquetSchema(fileSchema, selectedFieldNames, caseSensitive);
Type[] types = clipParquetSchema(fileSchema, selectedFieldNames, caseSensitive);
int[] requestedIndices = IntStream.range(0, types.length).filter(i -> types[i] != null).toArray();
Type[] readTypes = Arrays.stream(requestedIndices).mapToObj(i -> types[i]).toArray(Type[]::new);
this.requestedTypes = Arrays.stream(requestedIndices).mapToObj(i -> selectedTypes[i]).toArray(LogicalType[]::new);
this.requestedSchema = Types.buildMessage().addFields(readTypes).named("flink-parquet");
this.reader = new ParquetFileReader(
conf, footer.getFileMetaData(), path, blocks, requestedSchema.getColumns());
@@ -146,23 +152,37 @@ public class ParquetColumnarRowSplitReader implements Closeable {
checkSchema();
this.writableVectors = createWritableVectors();
this.columnarBatch = generator.generate(createReadableVectors());
ColumnVector[] columnVectors = patchedVector(selectedFieldNames.length, createReadableVectors(), requestedIndices);
this.columnarBatch = generator.generate(columnVectors);
this.row = new ColumnarRowData(columnarBatch);
}
/**
* Patches the given vectors with nulls.
* The vector position that is not requested (or read from file) is patched as null.
*
* @param fields The total selected fields number
* @param vectors The readable vectors
* @param indices The requested indices from the selected fields
*/
private static ColumnVector[] patchedVector(int fields, ColumnVector[] vectors, int[] indices) {
ColumnVector[] patched = new ColumnVector[fields];
for (int i = 0; i < indices.length; i++) {
patched[indices[i]] = vectors[i];
}
return patched;
}
/**
* Clips `parquetSchema` according to `fieldNames`.
*/
private static MessageType clipParquetSchema(
private static Type[] clipParquetSchema(
GroupType parquetSchema, String[] fieldNames, boolean caseSensitive) {
Type[] types = new Type[fieldNames.length];
if (caseSensitive) {
for (int i = 0; i < fieldNames.length; ++i) {
String fieldName = fieldNames[i];
if (parquetSchema.getFieldIndex(fieldName) < 0) {
throw new IllegalArgumentException(fieldName + " does not exist");
}
types[i] = parquetSchema.getType(fieldName);
types[i] = parquetSchema.containsField(fieldName) ? parquetSchema.getType(fieldName) : null;
}
} else {
Map<String, Type> caseInsensitiveFieldMap = new HashMap<>();
@@ -178,23 +198,20 @@ public class ParquetColumnarRowSplitReader implements Closeable {
}
for (int i = 0; i < fieldNames.length; ++i) {
Type type = caseInsensitiveFieldMap.get(fieldNames[i].toLowerCase(Locale.ROOT));
if (type == null) {
throw new IllegalArgumentException(fieldNames[i] + " does not exist");
}
// TODO clip for array,map,row types.
types[i] = type;
}
}
return Types.buildMessage().addFields(types).named("flink-parquet");
return types;
}
private WritableColumnVector[] createWritableVectors() {
WritableColumnVector[] columns = new WritableColumnVector[selectedTypes.length];
for (int i = 0; i < selectedTypes.length; i++) {
WritableColumnVector[] columns = new WritableColumnVector[requestedTypes.length];
for (int i = 0; i < requestedTypes.length; i++) {
columns[i] = createWritableColumnVector(
batchSize,
selectedTypes[i],
requestedTypes[i],
requestedSchema.getColumns().get(i).getPrimitiveType());
}
return columns;
@@ -207,7 +224,7 @@ public class ParquetColumnarRowSplitReader implements Closeable {
private ColumnVector[] createReadableVectors() {
ColumnVector[] vectors = new ColumnVector[writableVectors.length];
for (int i = 0; i < writableVectors.length; i++) {
vectors[i] = selectedTypes[i].getTypeRoot() == LogicalTypeRoot.DECIMAL
vectors[i] = requestedTypes[i].getTypeRoot() == LogicalTypeRoot.DECIMAL
? new ParquetDecimalVector(writableVectors[i])
: writableVectors[i];
}
@@ -215,10 +232,6 @@ public class ParquetColumnarRowSplitReader implements Closeable {
}
private void checkSchema() throws IOException, UnsupportedOperationException {
if (selectedTypes.length != requestedSchema.getFieldCount()) {
throw new RuntimeException("The quality of field type is incompatible with the request schema!");
}
/*
* Check that the requested schema is supported.
*/
@@ -314,7 +327,7 @@ public class ParquetColumnarRowSplitReader implements Closeable {
for (int i = 0; i < columns.size(); ++i) {
columnReaders[i] = createColumnReader(
utcTimestamp,
selectedTypes[i],
requestedTypes[i],
columns.get(i),
pages.getPageReader(columns.get(i)));
}

View File

@@ -108,9 +108,7 @@ public class ParquetSplitReaderUtil {
for (int i = 0; i < vectors.length; i++) {
String name = fullFieldNames[selectedFields[i]];
LogicalType type = fullFieldTypes[selectedFields[i]].getLogicalType();
vectors[i] = partitionSpec.containsKey(name)
? createVectorFromConstant(type, partitionSpec.get(name), batchSize)
: readVectors[selNonPartNames.indexOf(name)];
vectors[i] = createVector(readVectors, selNonPartNames, name, type, partitionSpec, batchSize);
}
return new VectorizedColumnBatch(vectors);
};
@@ -130,6 +128,24 @@ public class ParquetSplitReaderUtil {
splitLength);
}
private static ColumnVector createVector(
ColumnVector[] readVectors,
List<String> selNonPartNames,
String name,
LogicalType type,
Map<String, Object> partitionSpec,
int batchSize) {
if (partitionSpec.containsKey(name)) {
return createVectorFromConstant(type, partitionSpec.get(name), batchSize);
}
ColumnVector readVector = readVectors[selNonPartNames.indexOf(name)];
if (readVector == null) {
// when the read vector is null, use a constant null vector instead
readVector = createVectorFromConstant(type, null, batchSize);
}
return readVector;
}
private static ColumnVector createVectorFromConstant(
LogicalType type,
Object value,