[HUDI-426] Bootstrap datasource integration (#1702)
This commit is contained in:
@@ -18,17 +18,25 @@
|
||||
|
||||
package org.apache.hudi.client.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.sql.avro.SchemaConverters;
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter;
|
||||
import org.apache.spark.sql.internal.SQLConf;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet
|
||||
@@ -50,7 +58,10 @@ public class BootstrapSchemaProvider {
|
||||
public final Schema getBootstrapSchema(JavaSparkContext jsc, List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
if (writeConfig.getSchema() != null) {
|
||||
// Use schema specified by user if set
|
||||
return Schema.parse(writeConfig.getSchema());
|
||||
Schema userSchema = Schema.parse(writeConfig.getSchema());
|
||||
if (!HoodieAvroUtils.getNullSchema().equals(userSchema)) {
|
||||
return userSchema;
|
||||
}
|
||||
}
|
||||
return getBootstrapSourceSchema(jsc, partitions);
|
||||
}
|
||||
@@ -64,14 +75,26 @@ public class BootstrapSchemaProvider {
|
||||
*/
|
||||
protected Schema getBootstrapSourceSchema(JavaSparkContext jsc,
|
||||
List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
return partitions.stream().flatMap(p -> p.getValue().stream())
|
||||
.map(fs -> {
|
||||
try {
|
||||
Path filePath = FileStatusUtils.toPath(fs.getPath());
|
||||
return ParquetUtils.readAvroSchema(jsc.hadoopConfiguration(), filePath);
|
||||
} catch (Exception ex) {
|
||||
return null;
|
||||
}
|
||||
}).filter(x -> x != null).findAny().get();
|
||||
MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> {
|
||||
try {
|
||||
Path filePath = FileStatusUtils.toPath(fs.getPath());
|
||||
return ParquetUtils.readSchema(jsc.hadoopConfiguration(), filePath);
|
||||
} catch (Exception ex) {
|
||||
return null;
|
||||
}
|
||||
}).filter(Objects::nonNull).findAny()
|
||||
.orElseThrow(() -> new HoodieException("Could not determine schema from the data files."));
|
||||
|
||||
|
||||
ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(
|
||||
Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()),
|
||||
Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()));
|
||||
StructType sparkSchema = converter.convert(parquetSchema);
|
||||
String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName());
|
||||
String structName = tableName + "_record";
|
||||
String recordNamespace = "hoodie." + tableName;
|
||||
|
||||
return SchemaConverters.toAvroType(sparkSchema, false, structName, recordNamespace);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.config;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.translator.IdentityBootstrapPartitionPathTranslator;
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import java.io.File;
|
||||
@@ -52,6 +53,9 @@ public class HoodieBootstrapConfig extends DefaultHoodieConfig {
|
||||
public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX = ".*";
|
||||
public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = BootstrapMode.METADATA_ONLY.name();
|
||||
|
||||
public static final String BOOTSTRAP_INDEX_CLASS_PROP = "hoodie.bootstrap.index.class";
|
||||
public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
|
||||
|
||||
public HoodieBootstrapConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
@@ -129,6 +133,8 @@ public class HoodieBootstrapConfig extends DefaultHoodieConfig {
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE),
|
||||
BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE);
|
||||
BootstrapMode.valueOf(props.getProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_INDEX_CLASS_PROP), BOOTSTRAP_INDEX_CLASS_PROP,
|
||||
DEFAULT_BOOTSTRAP_INDEX_CLASS);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -136,6 +136,7 @@ public class BootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getSchemaToStoreInCommit() {
|
||||
return bootstrapSchema;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user