1
0

[HUDI-3514] Rebase Data Skipping flow to rely on MT Column Stats index (#4948)

This commit is contained in:
Alexey Kudinkin
2022-03-15 10:38:36 -07:00
committed by GitHub
parent 9bdda2a312
commit 5e8ff8d793
19 changed files with 359 additions and 224 deletions

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.hadoop;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -47,7 +48,6 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import org.apache.parquet.schema.MessageType;
import javax.annotation.Nonnull;
import java.io.IOException;
@@ -278,10 +278,13 @@ public class HoodieCopyOnWriteTableInputFormat extends HoodieTableInputFormat {
TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient);
try {
MessageType parquetSchema = tableSchemaResolver.getTableParquetSchema();
return Option.of(new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(),
tableConfig.getPartitionFieldProp(), parquetSchema.getFieldIndex(tableConfig.getRecordKeyFieldProp()),
parquetSchema.getFieldIndex(tableConfig.getPartitionFieldProp())));
Schema schema = tableSchemaResolver.getTableAvroSchema();
return Option.of(
new HoodieVirtualKeyInfo(
tableConfig.getRecordKeyFieldProp(),
tableConfig.getPartitionFieldProp(),
schema.getField(tableConfig.getRecordKeyFieldProp()).pos(),
schema.getField(tableConfig.getPartitionFieldProp()).pos()));
} catch (Exception exception) {
throw new HoodieException("Fetching table schema failed with exception ", exception);
}