[RFC-33] [HUDI-2429][Stacked on HUDI-2560] Support full Schema evolution for Spark (#4910)
* [HUDI-2560] introduce id_based schema to support full schema evolution. * add test for FileBasedInternalSchemaStorageManger and rebase code * add support for change column type and fix some test case * fix some bugs encountered in the production env and delete useless code * fix test error * rebase code * fixed some nested schema change bugs * [HUDI-2429][Stacked On HUDI-2560]Support full schema evolution for spark * [use dummyInternalSchema instead of null] * add support for spark3.1.x * remove support for spark3.1.x , sicne some compile fail * support spark3.1.x * rebase and prepare solve all comments * address all comments * rebase code * fixed the count(*) bug * try to get internalSchema by parser commit file/history file directly, not use metaclient which is time cost address some comments * fixed all comments * fix new comments * rebase code,fix UT failed * fixed mistake * rebase code ,fixed new comments * rebase code , and prepare for address new comments * address commits * address new comments * fix new issues * control fallback original write logical
This commit is contained in:
@@ -21,6 +21,7 @@ package org.apache.hudi.client;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.async.AsyncArchiveService;
|
||||
import org.apache.hudi.async.AsyncCleanerService;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
@@ -49,6 +50,7 @@ import org.apache.hudi.common.model.TableServiceType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTableVersion;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
@@ -71,6 +73,15 @@ import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.exception.HoodieSavepointException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaChangeApplier;
|
||||
import org.apache.hudi.internal.schema.action.TableChange;
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
|
||||
import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils;
|
||||
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.MetadataPartitionType;
|
||||
import org.apache.hudi.metrics.HoodieMetrics;
|
||||
@@ -85,6 +96,7 @@ import org.apache.hudi.table.upgrade.UpgradeDowngrade;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -101,6 +113,8 @@ import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY;
|
||||
|
||||
/**
|
||||
* Abstract Write Client providing functionality for performing commit, index updates and rollback
|
||||
* Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap
|
||||
@@ -246,12 +260,42 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
// Finalize write
|
||||
finalizeWrite(table, instantTime, stats);
|
||||
// do save internal schema to support Implicitly add columns in write process
|
||||
if (!metadata.getExtraMetadata().containsKey(SerDeHelper.LATEST_SCHEMA)
|
||||
&& metadata.getExtraMetadata().containsKey(SCHEMA_KEY) && table.getConfig().getSchemaEvolutionEnable()) {
|
||||
saveInternalSchema(table, instantTime, metadata);
|
||||
}
|
||||
// update Metadata table
|
||||
writeTableMetadata(table, instantTime, commitActionType, metadata);
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
}
|
||||
|
||||
// Save internal schema
|
||||
private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCommitMetadata metadata) {
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(table.getMetaClient());
|
||||
String historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata().orElse("");
|
||||
FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(table.getMetaClient());
|
||||
if (!historySchemaStr.isEmpty()) {
|
||||
InternalSchema internalSchema = InternalSchemaUtils.searchSchema(Long.parseLong(instantTime),
|
||||
SerDeHelper.parseSchemas(historySchemaStr));
|
||||
Schema avroSchema = HoodieAvroUtils.createHoodieWriteSchema(new Schema.Parser().parse(config.getSchema()));
|
||||
InternalSchema evolvedSchema = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(avroSchema, internalSchema);
|
||||
if (evolvedSchema.equals(internalSchema)) {
|
||||
metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(evolvedSchema));
|
||||
//TODO save history schema by metaTable
|
||||
schemasManager.persistHistorySchemaStr(instantTime, historySchemaStr);
|
||||
} else {
|
||||
evolvedSchema.setSchemaId(Long.parseLong(instantTime));
|
||||
String newSchemaStr = SerDeHelper.toJson(evolvedSchema);
|
||||
metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, newSchemaStr);
|
||||
schemasManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(evolvedSchema, historySchemaStr));
|
||||
}
|
||||
// update SCHEMA_KEY
|
||||
metadata.addMetadata(SCHEMA_KEY, AvroInternalSchemaConverter.convert(evolvedSchema, avroSchema.getName()).toString());
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) {
|
||||
return createTable(config, hadoopConf, false);
|
||||
}
|
||||
@@ -1442,8 +1486,8 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
|
||||
if (commitMetadata.getExtraMetadata().containsKey(HoodieCommitMetadata.SCHEMA_KEY)) {
|
||||
config.setSchema(commitMetadata.getExtraMetadata().get(HoodieCommitMetadata.SCHEMA_KEY));
|
||||
if (commitMetadata.getExtraMetadata().containsKey(SCHEMA_KEY)) {
|
||||
config.setSchema(commitMetadata.getExtraMetadata().get(SCHEMA_KEY));
|
||||
} else {
|
||||
throw new HoodieIOException("Latest commit does not have any schema in commit metadata");
|
||||
}
|
||||
@@ -1505,4 +1549,138 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
metaClient.reloadActiveTimeline();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* add columns to table.
|
||||
*
|
||||
* @param colName col name to be added. if we want to add col to a nested filed, the fullName should be specify
|
||||
* @param schema col type to be added.
|
||||
* @param doc col doc to be added.
|
||||
* @param position col position to be added
|
||||
* @param positionType col position change type. now support three change types: first/after/before
|
||||
*/
|
||||
public void addColumn(String colName, Schema schema, String doc, String position, TableChange.ColumnPositionChange.ColumnPositionType positionType) {
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft())
|
||||
.applyAddChange(colName, AvroInternalSchemaConverter.convertToField(schema), doc, position, positionType);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
public void addColumn(String colName, Schema schema) {
|
||||
addColumn(colName, schema, null, "", TableChange.ColumnPositionChange.ColumnPositionType.NO_OPERATION);
|
||||
}
|
||||
|
||||
/**
|
||||
* delete columns to table.
|
||||
*
|
||||
* @param colNames col name to be deleted. if we want to delete col from a nested filed, the fullName should be specify
|
||||
*/
|
||||
public void deleteColumns(String... colNames) {
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyDeleteChange(colNames);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
/**
|
||||
* rename col name for hudi table.
|
||||
*
|
||||
* @param colName col name to be renamed. if we want to rename col from a nested filed, the fullName should be specify
|
||||
* @param newName new name for current col. no need to specify fullName.
|
||||
*/
|
||||
public void renameColumn(String colName, String newName) {
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyRenameChange(colName, newName);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
/**
|
||||
* update col nullable attribute for hudi table.
|
||||
*
|
||||
* @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify
|
||||
* @param nullable .
|
||||
*/
|
||||
public void updateColumnNullability(String colName, boolean nullable) {
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyColumnNullabilityChange(colName, nullable);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
/**
|
||||
* update col Type for hudi table.
|
||||
* only support update primitive type to primitive type.
|
||||
* cannot update nest type to nest type or primitive type eg: RecordType -> MapType, MapType -> LongType.
|
||||
*
|
||||
* @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify
|
||||
* @param newType .
|
||||
*/
|
||||
public void updateColumnType(String colName, Type newType) {
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyColumnTypeChange(colName, newType);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
/**
|
||||
* update col comment for hudi table.
|
||||
*
|
||||
* @param colName col name to be changed. if we want to change col from a nested filed, the fullName should be specify
|
||||
* @param doc .
|
||||
*/
|
||||
public void updateColumnComment(String colName, String doc) {
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft()).applyColumnCommentChange(colName, doc);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
/**
|
||||
* reorder the position of col.
|
||||
*
|
||||
* @param colName column which need to be reordered. if we want to change col from a nested filed, the fullName should be specify.
|
||||
* @param referColName reference position.
|
||||
* @param orderType col position change type. now support three change types: first/after/before
|
||||
*/
|
||||
public void reOrderColPosition(String colName, String referColName, TableChange.ColumnPositionChange.ColumnPositionType orderType) {
|
||||
if (colName == null || orderType == null || referColName == null) {
|
||||
return;
|
||||
}
|
||||
//get internalSchema
|
||||
Pair<InternalSchema, HoodieTableMetaClient> pair = getInternalSchemaAndMetaClient();
|
||||
InternalSchema newSchema = new InternalSchemaChangeApplier(pair.getLeft())
|
||||
.applyReOrderColPositionChange(colName, referColName, orderType);
|
||||
commitTableChange(newSchema, pair.getRight());
|
||||
}
|
||||
|
||||
private Pair<InternalSchema, HoodieTableMetaClient> getInternalSchemaAndMetaClient() {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
|
||||
Option<InternalSchema> internalSchemaOption = schemaUtil.getTableInternalSchemaFromCommitMetadata();
|
||||
if (!internalSchemaOption.isPresent()) {
|
||||
throw new HoodieException(String.format("cannot find schema for current table: %s", config.getBasePath()));
|
||||
}
|
||||
return Pair.of(internalSchemaOption.get(), metaClient);
|
||||
}
|
||||
|
||||
private void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient metaClient) {
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
|
||||
String historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata().orElse("");
|
||||
Schema schema = AvroInternalSchemaConverter.convert(newSchema, config.getTableName());
|
||||
String commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType());
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
startCommitWithTime(instantTime, commitActionType, metaClient);
|
||||
config.setSchema(schema.toString());
|
||||
HoodieActiveTimeline timeLine = metaClient.getActiveTimeline();
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime);
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA);
|
||||
try {
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
Map<String, String> extraMeta = new HashMap<>();
|
||||
extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(newSchema.setSchemaId(Long.getLong(instantTime))));
|
||||
// try to save history schemas
|
||||
FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient);
|
||||
schemasManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(newSchema, historySchemaStr));
|
||||
commitStats(instantTime, Collections.EMPTY_LIST, Option.of(extraMeta), commitActionType);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -167,6 +167,22 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
+ "implementations of HoodieRecordPayload to convert incoming records to avro. This is also used as the write schema "
|
||||
+ "evolving records during an update.");
|
||||
|
||||
public static final ConfigProperty<String> INTERNAL_SCHEMA_STRING = ConfigProperty
|
||||
.key("hoodie.internal.schema")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Schema string representing the latest schema of the table. Hudi passes this to "
|
||||
+ "implementations of evolution of schema");
|
||||
|
||||
public static final ConfigProperty<Boolean> SCHEMA_EVOLUTION_ENABLE = ConfigProperty
|
||||
.key("hoodie.schema.on.read.enable")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("enable full schema evolution for hoodie");
|
||||
|
||||
public static final ConfigProperty<Boolean> ENABLE_INTERNAL_SCHEMA_CACHE = ConfigProperty
|
||||
.key("hoodie.schema.cache.enable")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("cache query internalSchemas in driver/executor side");
|
||||
|
||||
public static final ConfigProperty<String> AVRO_SCHEMA_VALIDATE_ENABLE = ConfigProperty
|
||||
.key("hoodie.avro.schema.validate")
|
||||
.defaultValue("false")
|
||||
@@ -886,6 +902,30 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
setValue(AVRO_SCHEMA_STRING, schemaStr);
|
||||
}
|
||||
|
||||
public String getInternalSchema() {
|
||||
return getString(INTERNAL_SCHEMA_STRING);
|
||||
}
|
||||
|
||||
public boolean getInternalSchemaCacheEnable() {
|
||||
return getBoolean(ENABLE_INTERNAL_SCHEMA_CACHE);
|
||||
}
|
||||
|
||||
public void setInternalSchemaString(String internalSchemaString) {
|
||||
setValue(INTERNAL_SCHEMA_STRING, internalSchemaString);
|
||||
}
|
||||
|
||||
public void setInternalSchemaCacheEnable(boolean enable) {
|
||||
setValue(ENABLE_INTERNAL_SCHEMA_CACHE, String.valueOf(enable));
|
||||
}
|
||||
|
||||
public boolean getSchemaEvolutionEnable() {
|
||||
return getBoolean(SCHEMA_EVOLUTION_ENABLE);
|
||||
}
|
||||
|
||||
public void setSchemaEvolutionEnable(boolean enable) {
|
||||
setValue(SCHEMA_EVOLUTION_ENABLE, String.valueOf(enable));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the write schema for written records.
|
||||
*
|
||||
@@ -2075,6 +2115,16 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSchemaEvolutionEnable(boolean enable) {
|
||||
writeConfig.setValue(SCHEMA_EVOLUTION_ENABLE, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInternalSchemaCacheEnable(boolean enable) {
|
||||
writeConfig.setValue(ENABLE_INTERNAL_SCHEMA_CACHE, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAvroSchemaValidate(boolean enable) {
|
||||
writeConfig.setValue(AVRO_SCHEMA_VALIDATE_ENABLE, String.valueOf(enable));
|
||||
return this;
|
||||
|
||||
@@ -39,6 +39,7 @@ import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||
|
||||
@@ -240,6 +241,14 @@ public class CleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends
|
||||
List<HoodieInstant> pendingCleanInstants = table.getCleanTimeline()
|
||||
.filterInflightsAndRequested().getInstants().collect(Collectors.toList());
|
||||
if (pendingCleanInstants.size() > 0) {
|
||||
// try to clean old history schema.
|
||||
try {
|
||||
FileBasedInternalSchemaStorageManager fss = new FileBasedInternalSchemaStorageManager(table.getMetaClient());
|
||||
fss.cleanOldFiles(pendingCleanInstants.stream().map(is -> is.getTimestamp()).collect(Collectors.toList()));
|
||||
} catch (Exception e) {
|
||||
// we should not affect original clean logic. Swallow exception and log warn.
|
||||
LOG.warn("failed to clean old history schema");
|
||||
}
|
||||
pendingCleanInstants.forEach(hoodieInstant -> {
|
||||
if (table.getCleanTimeline().isEmpty(hoodieInstant)) {
|
||||
table.getActiveTimeline().deleteEmptyInstantIfExists(hoodieInstant);
|
||||
|
||||
@@ -18,14 +18,24 @@
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.avro.SchemaCompatibility;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.InternalSchemaCache;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaMerger;
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
|
||||
import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
@@ -41,6 +51,8 @@ import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class HoodieMergeHelper<T extends HoodieRecordPayload> extends
|
||||
BaseMergeHelper<T, HoodieData<HoodieRecord<T>>, HoodieData<HoodieKey>, HoodieData<WriteStatus>> {
|
||||
@@ -78,12 +90,41 @@ public class HoodieMergeHelper<T extends HoodieRecordPayload> extends
|
||||
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
|
||||
|
||||
Option<InternalSchema> querySchemaOpt = SerDeHelper.fromJson(table.getConfig().getInternalSchema());
|
||||
boolean needToReWriteRecord = false;
|
||||
// TODO support bootstrap
|
||||
if (querySchemaOpt.isPresent() && !baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
// check implicitly add columns, and position reorder(spark sql may change cols order)
|
||||
InternalSchema querySchema = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(readSchema, querySchemaOpt.get(), true);
|
||||
long commitInstantTime = Long.valueOf(FSUtils.getCommitTime(mergeHandle.getOldFilePath().getName()));
|
||||
InternalSchema writeInternalSchema = InternalSchemaCache.searchSchemaAndCache(commitInstantTime, table.getMetaClient(), table.getConfig().getInternalSchemaCacheEnable());
|
||||
if (writeInternalSchema.isEmptySchema()) {
|
||||
throw new HoodieException(String.format("cannot find file schema for current commit %s", commitInstantTime));
|
||||
}
|
||||
List<String> colNamesFromQuerySchema = querySchema.getAllColsFullName();
|
||||
List<String> colNamesFromWriteSchema = writeInternalSchema.getAllColsFullName();
|
||||
List<String> sameCols = colNamesFromWriteSchema.stream()
|
||||
.filter(f -> colNamesFromQuerySchema.contains(f)
|
||||
&& writeInternalSchema.findIdByName(f) == querySchema.findIdByName(f)
|
||||
&& writeInternalSchema.findIdByName(f) != -1
|
||||
&& writeInternalSchema.findType(writeInternalSchema.findIdByName(f)).equals(querySchema.findType(writeInternalSchema.findIdByName(f)))).collect(Collectors.toList());
|
||||
readSchema = AvroInternalSchemaConverter.convert(new InternalSchemaMerger(writeInternalSchema, querySchema, true, false).mergeSchema(), readSchema.getName());
|
||||
Schema writeSchemaFromFile = AvroInternalSchemaConverter.convert(writeInternalSchema, readSchema.getName());
|
||||
needToReWriteRecord = sameCols.size() != colNamesFromWriteSchema.size()
|
||||
|| SchemaCompatibility.checkReaderWriterCompatibility(writeSchemaFromFile, readSchema).getType() == org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE;
|
||||
}
|
||||
|
||||
try {
|
||||
final Iterator<GenericRecord> readerIterator;
|
||||
if (baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
|
||||
} else {
|
||||
readerIterator = reader.getRecordIterator(readSchema);
|
||||
if (needToReWriteRecord) {
|
||||
readerIterator = HoodieAvroUtils.rewriteRecordWithNewSchema(reader.getRecordIterator(), readSchema);
|
||||
} else {
|
||||
readerIterator = reader.getRecordIterator(readSchema);
|
||||
}
|
||||
}
|
||||
|
||||
ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
|
||||
|
||||
@@ -44,9 +44,12 @@ import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
import org.apache.hudi.io.IOUtils;
|
||||
import org.apache.hudi.table.HoodieCompactionHandler;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -117,8 +120,10 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|
||||
// log file.That is because in the case of MergeInto, the config.getSchema may not
|
||||
// the same with the table schema.
|
||||
try {
|
||||
Schema readerSchema = schemaResolver.getTableAvroSchema(false);
|
||||
config.setSchema(readerSchema.toString());
|
||||
if (StringUtils.isNullOrEmpty(config.getInternalSchema())) {
|
||||
Schema readerSchema = schemaResolver.getTableAvroSchema(false);
|
||||
config.setSchema(readerSchema.toString());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// If there is no commit in the table, just ignore the exception.
|
||||
}
|
||||
@@ -145,9 +150,17 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|
||||
String instantTime,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(
|
||||
new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
|
||||
Schema readerSchema;
|
||||
Option<InternalSchema> internalSchemaOption = Option.empty();
|
||||
if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
|
||||
readerSchema = new Schema.Parser().parse(config.getSchema());
|
||||
internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
|
||||
// its safe to modify config here, since we running in task side.
|
||||
((HoodieTable) compactionHandler).getConfig().setDefault(config);
|
||||
} else {
|
||||
readerSchema = HoodieAvroUtils.addMetadataFields(
|
||||
new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
|
||||
}
|
||||
LOG.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation.getDeltaFileNames()
|
||||
+ " for commit " + instantTime);
|
||||
// TODO - FIX THIS
|
||||
@@ -172,6 +185,7 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|
||||
.withLogFilePaths(logFiles)
|
||||
.withReaderSchema(readerSchema)
|
||||
.withLatestInstantTime(maxInstantTime)
|
||||
.withInternalSchema(internalSchemaOption.orElse(InternalSchema.getEmptyInternalSchema()))
|
||||
.withMaxMemorySizeInBytes(maxMemoryPerCompaction)
|
||||
.withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled())
|
||||
.withReverseReader(config.getCompactionReverseLogReadEnabled())
|
||||
|
||||
@@ -30,8 +30,11 @@ import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.InternalSchemaCache;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
import org.apache.hudi.table.HoodieCompactionHandler;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||
@@ -70,8 +73,19 @@ public class RunCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.getCompactionPlan(table.getMetaClient(), instantTime);
|
||||
|
||||
// try to load internalSchema to support schema Evolution
|
||||
HoodieWriteConfig configCopy = config;
|
||||
Pair<Option<String>, Option<String>> schemaPair = InternalSchemaCache
|
||||
.getInternalSchemaAndAvroSchemaForClusteringAndCompaction(table.getMetaClient(), instantTime);
|
||||
if (schemaPair.getLeft().isPresent() && schemaPair.getRight().isPresent()) {
|
||||
// should not influence the original config, just copy it
|
||||
configCopy = HoodieWriteConfig.newBuilder().withProperties(config.getProps()).build();
|
||||
configCopy.setInternalSchemaString(schemaPair.getLeft().get());
|
||||
configCopy.setSchema(schemaPair.getRight().get());
|
||||
}
|
||||
|
||||
HoodieData<WriteStatus> statuses = compactor.compact(
|
||||
context, compactionPlan, table, config, instantTime, compactionHandler);
|
||||
context, compactionPlan, table, configCopy, instantTime, compactionHandler);
|
||||
|
||||
compactor.maybePersist(statuses, config);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata");
|
||||
@@ -81,7 +95,10 @@ public class RunCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
metadata.addWriteStat(stat.getPartitionPath(), stat);
|
||||
}
|
||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
|
||||
|
||||
if (schemaPair.getLeft().isPresent()) {
|
||||
metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, schemaPair.getLeft().get());
|
||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaPair.getRight().get());
|
||||
}
|
||||
compactionMetadata.setWriteStatuses(statuses);
|
||||
compactionMetadata.setCommitted(false);
|
||||
compactionMetadata.setCommitMetadata(Option.of(metadata));
|
||||
|
||||
@@ -0,0 +1,502 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaMerger;
|
||||
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
|
||||
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
|
||||
import org.apache.spark.sql.types.ArrayType;
|
||||
import org.apache.spark.sql.types.ArrayType$;
|
||||
import org.apache.spark.sql.types.BinaryType;
|
||||
import org.apache.spark.sql.types.BinaryType$;
|
||||
import org.apache.spark.sql.types.BooleanType;
|
||||
import org.apache.spark.sql.types.BooleanType$;
|
||||
import org.apache.spark.sql.types.ByteType;
|
||||
import org.apache.spark.sql.types.CharType;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.DateType;
|
||||
import org.apache.spark.sql.types.DateType$;
|
||||
import org.apache.spark.sql.types.Decimal;
|
||||
import org.apache.spark.sql.types.DecimalType;
|
||||
import org.apache.spark.sql.types.DecimalType$;
|
||||
import org.apache.spark.sql.types.DoubleType;
|
||||
import org.apache.spark.sql.types.DoubleType$;
|
||||
import org.apache.spark.sql.types.FloatType;
|
||||
import org.apache.spark.sql.types.FloatType$;
|
||||
import org.apache.spark.sql.types.IntegerType;
|
||||
import org.apache.spark.sql.types.IntegerType$;
|
||||
import org.apache.spark.sql.types.LongType;
|
||||
import org.apache.spark.sql.types.LongType$;
|
||||
import org.apache.spark.sql.types.MapType;
|
||||
import org.apache.spark.sql.types.MapType$;
|
||||
import org.apache.spark.sql.types.Metadata;
|
||||
import org.apache.spark.sql.types.ShortType;
|
||||
import org.apache.spark.sql.types.StringType;
|
||||
import org.apache.spark.sql.types.StringType$;
|
||||
import org.apache.spark.sql.types.StructField;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.types.StructType$;
|
||||
import org.apache.spark.sql.types.TimestampType;
|
||||
import org.apache.spark.sql.types.TimestampType$;
|
||||
import org.apache.spark.sql.types.UserDefinedType;
|
||||
import org.apache.spark.sql.types.VarcharType;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Deque;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class SparkInternalSchemaConverter {
|
||||
private SparkInternalSchemaConverter() {
|
||||
|
||||
}
|
||||
|
||||
public static final String HOODIE_QUERY_SCHEMA = "hoodie.schema.internal.querySchema";
|
||||
public static final String HOODIE_TABLE_PATH = "hoodie.tablePath";
|
||||
public static final String HOODIE_VALID_COMMITS_LIST = "hoodie.valid.commits.list";
|
||||
|
||||
/**
|
||||
* Converts a spark schema to an hudi internal schema. Fields without IDs are kept and assigned fallback IDs.
|
||||
*
|
||||
* @param sparkSchema a spark schema
|
||||
* @return a matching internal schema for the provided spark schema
|
||||
*/
|
||||
public static InternalSchema convertStructTypeToInternalSchema(StructType sparkSchema) {
|
||||
Type newType = buildTypeFromStructType(sparkSchema, true, new AtomicInteger(0));
|
||||
return new InternalSchema(((Types.RecordType)newType).fields());
|
||||
}
|
||||
|
||||
public static Type buildTypeFromStructType(DataType sparkType, Boolean firstVisitRoot, AtomicInteger nextId) {
|
||||
if (sparkType instanceof StructType) {
|
||||
StructField[] fields = ((StructType) sparkType).fields();
|
||||
int nextAssignId = firstVisitRoot ? 0 : nextId.get();
|
||||
nextId.set(nextAssignId + fields.length);
|
||||
List<Type> newTypes = new ArrayList<>();
|
||||
for (StructField f : fields) {
|
||||
newTypes.add(buildTypeFromStructType(f.dataType(), false, nextId));
|
||||
}
|
||||
List<Types.Field> newFields = new ArrayList<>();
|
||||
for (int i = 0; i < newTypes.size(); i++) {
|
||||
StructField f = fields[i];
|
||||
newFields.add(Types.Field.get(nextAssignId + i, f.nullable(), f.name(), newTypes.get(i),
|
||||
f.getComment().isDefined() ? f.getComment().get() : null));
|
||||
}
|
||||
return Types.RecordType.get(newFields);
|
||||
} else if (sparkType instanceof MapType) {
|
||||
MapType map = (MapType) sparkType;
|
||||
DataType keyType = map.keyType();
|
||||
DataType valueType = map.valueType();
|
||||
int keyId = nextId.get();
|
||||
int valueId = keyId + 1;
|
||||
nextId.set(valueId + 1);
|
||||
return Types.MapType.get(keyId, valueId, buildTypeFromStructType(keyType, false, nextId),
|
||||
buildTypeFromStructType(valueType, false, nextId), map.valueContainsNull());
|
||||
} else if (sparkType instanceof ArrayType) {
|
||||
ArrayType array = (ArrayType) sparkType;
|
||||
DataType et = array.elementType();
|
||||
int elementId = nextId.get();
|
||||
nextId.set(elementId + 1);
|
||||
return Types.ArrayType.get(elementId, array.containsNull(), buildTypeFromStructType(et, false, nextId));
|
||||
} else if (sparkType instanceof UserDefinedType) {
|
||||
throw new UnsupportedOperationException("User-defined types are not supported");
|
||||
} else if (sparkType instanceof BooleanType) {
|
||||
return Types.BooleanType.get();
|
||||
} else if (sparkType instanceof IntegerType
|
||||
|| sparkType instanceof ShortType
|
||||
|| sparkType instanceof ByteType) {
|
||||
return Types.IntType.get();
|
||||
} else if (sparkType instanceof LongType) {
|
||||
return Types.LongType.get();
|
||||
} else if (sparkType instanceof FloatType) {
|
||||
return Types.FloatType.get();
|
||||
} else if (sparkType instanceof DoubleType) {
|
||||
return Types.DoubleType.get();
|
||||
} else if (sparkType instanceof StringType
|
||||
|| sparkType instanceof CharType
|
||||
|| sparkType instanceof VarcharType) {
|
||||
return Types.StringType.get();
|
||||
} else if (sparkType instanceof DateType) {
|
||||
return Types.DateType.get();
|
||||
// spark 3.3.0 support TimeStampNTZ, to do support spark3.3.0
|
||||
} else if (sparkType instanceof TimestampType) {
|
||||
return Types.TimestampType.get();
|
||||
} else if (sparkType instanceof DecimalType) {
|
||||
return Types.DecimalType.get(
|
||||
((DecimalType) sparkType).precision(),
|
||||
((DecimalType) sparkType).scale());
|
||||
} else if (sparkType instanceof BinaryType) {
|
||||
return Types.BinaryType.get();
|
||||
} else {
|
||||
throw new UnsupportedOperationException(String.format("Not a supported type: %s", sparkType.catalogString()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts Spark schema to Hudi internal schema, and prune fields.
|
||||
* Fields without IDs are kept and assigned fallback IDs.
|
||||
*
|
||||
* @param sparkSchema a pruned spark schema
|
||||
* @param originSchema a internal schema for hoodie table
|
||||
* @return a pruned internal schema for the provided spark schema
|
||||
*/
|
||||
public static InternalSchema convertAndPruneStructTypeToInternalSchema(StructType sparkSchema, InternalSchema originSchema) {
|
||||
List<String> pruneNames = collectColNamesFromSparkStruct(sparkSchema);
|
||||
return InternalSchemaUtils.pruneInternalSchema(originSchema, pruneNames);
|
||||
}
|
||||
|
||||
/**
|
||||
* Collect all the leaf nodes names.
|
||||
*
|
||||
* @param sparkSchema a spark schema
|
||||
* @return leaf nodes full names.
|
||||
*/
|
||||
public static List<String> collectColNamesFromSparkStruct(StructType sparkSchema) {
|
||||
List<String> result = new ArrayList<>();
|
||||
collectColNamesFromStructType(sparkSchema, new LinkedList<>(), result);
|
||||
return result;
|
||||
}
|
||||
|
||||
private static void collectColNamesFromStructType(DataType sparkType, Deque<String> fieldNames, List<String> resultSet) {
|
||||
if (sparkType instanceof StructType) {
|
||||
StructField[] fields = ((StructType) sparkType).fields();
|
||||
for (StructField f : fields) {
|
||||
fieldNames.push(f.name());
|
||||
collectColNamesFromStructType(f.dataType(), fieldNames, resultSet);
|
||||
fieldNames.pop();
|
||||
addFullName(f.dataType(), f.name(), fieldNames, resultSet);
|
||||
}
|
||||
} else if (sparkType instanceof MapType) {
|
||||
MapType map = (MapType) sparkType;
|
||||
DataType keyType = map.keyType();
|
||||
DataType valueType = map.valueType();
|
||||
// key
|
||||
fieldNames.push("key");
|
||||
collectColNamesFromStructType(keyType, fieldNames, resultSet);
|
||||
fieldNames.pop();
|
||||
addFullName(keyType,"key", fieldNames, resultSet);
|
||||
// value
|
||||
fieldNames.push("value");
|
||||
collectColNamesFromStructType(valueType, fieldNames, resultSet);
|
||||
fieldNames.poll();
|
||||
addFullName(valueType,"value", fieldNames, resultSet);
|
||||
} else if (sparkType instanceof ArrayType) {
|
||||
ArrayType array = (ArrayType) sparkType;
|
||||
DataType et = array.elementType();
|
||||
fieldNames.push("element");
|
||||
collectColNamesFromStructType(et, fieldNames, resultSet);
|
||||
fieldNames.pop();
|
||||
addFullName(et, "element", fieldNames, resultSet);
|
||||
} else if (sparkType instanceof UserDefinedType) {
|
||||
throw new UnsupportedOperationException("User-defined types are not supported");
|
||||
} else {
|
||||
// do nothings
|
||||
}
|
||||
}
|
||||
|
||||
private static void addFullName(DataType sparkType, String name, Deque<String> fieldNames, List<String> resultSet) {
|
||||
if (!(sparkType instanceof StructType) && !(sparkType instanceof ArrayType) && !(sparkType instanceof MapType)) {
|
||||
resultSet.add(InternalSchemaUtils.createFullName(name, fieldNames));
|
||||
}
|
||||
}
|
||||
|
||||
public static StructType mergeSchema(InternalSchema fileSchema, InternalSchema querySchema) {
|
||||
InternalSchema schema = new InternalSchemaMerger(fileSchema, querySchema, true, true).mergeSchema();
|
||||
return constructSparkSchemaFromInternalSchema(schema);
|
||||
}
|
||||
|
||||
public static Map<Integer, Pair<DataType, DataType>> collectTypeChangedCols(InternalSchema schema, InternalSchema other) {
|
||||
return InternalSchemaUtils
|
||||
.collectTypeChangedCols(schema, other)
|
||||
.entrySet()
|
||||
.stream()
|
||||
.collect(Collectors.toMap(e -> e.getKey(), e -> Pair.of(constructSparkSchemaFromType(e.getValue().getLeft()), constructSparkSchemaFromType(e.getValue().getRight()))));
|
||||
}
|
||||
|
||||
public static StructType constructSparkSchemaFromInternalSchema(InternalSchema schema) {
|
||||
return (StructType) constructSparkSchemaFromType(schema.getRecord());
|
||||
}
|
||||
|
||||
private static DataType constructSparkSchemaFromType(Type type) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Types.Field> fields = record.fields();
|
||||
List<StructField> structFields = new ArrayList<>();
|
||||
for (Types.Field f : fields) {
|
||||
DataType dataType = constructSparkSchemaFromType(f.type());
|
||||
StructField structField = StructField.apply(f.name(), dataType, f.isOptional(), Metadata.empty());
|
||||
structField = f.doc() == null ? structField : structField.withComment(f.doc());
|
||||
structFields.add(structField);
|
||||
}
|
||||
return StructType$.MODULE$.apply(structFields);
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
DataType elementType = constructSparkSchemaFromType(array.elementType());
|
||||
return ArrayType$.MODULE$.apply(elementType, array.isElementOptional());
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
DataType keyDataType = constructSparkSchemaFromType(map.keyType());
|
||||
DataType valueDataType = constructSparkSchemaFromType(map.valueType());
|
||||
return MapType$.MODULE$.apply(keyDataType, valueDataType, map.isValueOptional());
|
||||
case BOOLEAN:
|
||||
return BooleanType$.MODULE$;
|
||||
case INT:
|
||||
return IntegerType$.MODULE$;
|
||||
case LONG:
|
||||
return LongType$.MODULE$;
|
||||
case FLOAT:
|
||||
return FloatType$.MODULE$;
|
||||
case DOUBLE:
|
||||
return DoubleType$.MODULE$;
|
||||
case DATE:
|
||||
return DateType$.MODULE$;
|
||||
case TIME:
|
||||
throw new UnsupportedOperationException(String.format("cannot convert %s type to Spark", type));
|
||||
case TIMESTAMP:
|
||||
// todo support TimeStampNTZ
|
||||
return TimestampType$.MODULE$;
|
||||
case STRING:
|
||||
return StringType$.MODULE$;
|
||||
case UUID:
|
||||
return StringType$.MODULE$;
|
||||
case FIXED:
|
||||
return BinaryType$.MODULE$;
|
||||
case BINARY:
|
||||
return BinaryType$.MODULE$;
|
||||
case DECIMAL:
|
||||
Types.DecimalType decimal = (Types.DecimalType) type;
|
||||
return DecimalType$.MODULE$.apply(decimal.precision(), decimal.scale());
|
||||
default:
|
||||
throw new UnsupportedOperationException(String.format("cannot convert unknown type: %s to Spark", type));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert Int/long type to other Type.
|
||||
* Now only support int/long -> long/float/double/string
|
||||
* TODO: support more types
|
||||
*/
|
||||
private static boolean convertIntLongType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) {
|
||||
boolean isInt = oldV.dataType() instanceof IntegerType;
|
||||
if (newType instanceof LongType || newType instanceof FloatType
|
||||
|| newType instanceof DoubleType || newType instanceof StringType || newType instanceof DecimalType) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (oldV.isNullAt(i)) {
|
||||
newV.putNull(i);
|
||||
continue;
|
||||
}
|
||||
// int/long -> long/float/double/string/decimal
|
||||
if (newType instanceof LongType) {
|
||||
newV.putLong(i, isInt ? oldV.getInt(i) : oldV.getLong(i));
|
||||
} else if (newType instanceof FloatType) {
|
||||
newV.putFloat(i, isInt ? oldV.getInt(i) : oldV.getLong(i));
|
||||
} else if (newType instanceof DoubleType) {
|
||||
newV.putDouble(i, isInt ? oldV.getInt(i) : oldV.getLong(i));
|
||||
} else if (newType instanceof StringType) {
|
||||
newV.putByteArray(i, ((isInt ? oldV.getInt(i) : oldV.getLong(i)) + "").getBytes(StandardCharsets.UTF_8));
|
||||
} else if (newType instanceof DecimalType) {
|
||||
Decimal oldDecimal = Decimal.apply(isInt ? oldV.getInt(i) : oldV.getLong(i));
|
||||
oldDecimal.changePrecision(((DecimalType) newType).precision(), ((DecimalType) newType).scale());
|
||||
newV.putDecimal(i, oldDecimal, ((DecimalType) newType).precision());
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert float type to other Type.
|
||||
* Now only support float -> double/String
|
||||
* TODO: support more types
|
||||
*/
|
||||
private static boolean convertFloatType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) {
|
||||
if (newType instanceof DoubleType || newType instanceof StringType || newType instanceof DecimalType) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (oldV.isNullAt(i)) {
|
||||
newV.putNull(i);
|
||||
continue;
|
||||
}
|
||||
// float -> double/string/decimal
|
||||
if (newType instanceof DoubleType) {
|
||||
newV.putDouble(i, Double.valueOf(oldV.getFloat(i) + ""));
|
||||
} else if (newType instanceof StringType) {
|
||||
newV.putByteArray(i, (oldV.getFloat(i) + "").getBytes(StandardCharsets.UTF_8));
|
||||
} else if (newType instanceof DecimalType) {
|
||||
Decimal oldDecimal = Decimal.apply(oldV.getFloat(i));
|
||||
oldDecimal.changePrecision(((DecimalType) newType).precision(), ((DecimalType) newType).scale());
|
||||
newV.putDecimal(i, oldDecimal, ((DecimalType) newType).precision());
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert double type to other Type.
|
||||
* Now only support Double -> Decimal/String
|
||||
* TODO: support more types
|
||||
*/
|
||||
private static boolean convertDoubleType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) {
|
||||
if (newType instanceof DecimalType || newType instanceof StringType) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (oldV.isNullAt(i)) {
|
||||
newV.putNull(i);
|
||||
continue;
|
||||
}
|
||||
// double -> decimal/string
|
||||
if (newType instanceof DecimalType) {
|
||||
Decimal oldDecimal = Decimal.apply(oldV.getDouble(i));
|
||||
oldDecimal.changePrecision(((DecimalType) newType).precision(), ((DecimalType) newType).scale());
|
||||
newV.putDecimal(i, oldDecimal, ((DecimalType) newType).precision());
|
||||
} else if (newType instanceof StringType) {
|
||||
newV.putByteArray(i, (oldV.getDouble(i) + "").getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert decimal type to other Type.
|
||||
* Now only support Decimal -> Decimal/String
|
||||
* TODO: support more types
|
||||
*/
|
||||
private static boolean convertDecimalType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) {
|
||||
DataType oldType = oldV.dataType();
|
||||
if (newType instanceof DecimalType || newType instanceof StringType) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (oldV.isNullAt(i)) {
|
||||
newV.putNull(i);
|
||||
continue;
|
||||
}
|
||||
Decimal oldDecimal = oldV.getDecimal(i, ((DecimalType) oldType).precision(), ((DecimalType) oldType).scale());
|
||||
if (newType instanceof DecimalType) {
|
||||
oldDecimal.changePrecision(((DecimalType) newType).precision(), ((DecimalType) newType).scale());
|
||||
newV.putDecimal(i, oldDecimal, ((DecimalType) newType).precision());
|
||||
} else if (newType instanceof StringType) {
|
||||
newV.putByteArray(i, oldDecimal.toString().getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert date type to other Type.
|
||||
* Now only support Date -> String
|
||||
* TODO: support more types
|
||||
*/
|
||||
private static boolean convertDateType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) {
|
||||
if (newType instanceof StringType) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (oldV.isNullAt(i)) {
|
||||
newV.putNull(i);
|
||||
continue;
|
||||
}
|
||||
// to do support rebaseDate
|
||||
String res = org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(oldV.getInt(i)).toString();
|
||||
newV.putByteArray(i, res.getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert String type to other Type.
|
||||
* Now only support String -> Decimal/Date.
|
||||
* Notice: This convert maybe failed!!!
|
||||
* TODO: support more types
|
||||
*/
|
||||
private static boolean convertStringType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) {
|
||||
if (newType instanceof DateType || newType instanceof DecimalType) {
|
||||
for (int i = 0; i < len; i++) {
|
||||
if (oldV.isNullAt(i)) {
|
||||
newV.putNull(i);
|
||||
continue;
|
||||
}
|
||||
// to do support rebaseDate
|
||||
if (newType instanceof DateType) {
|
||||
int days = org.apache.spark.sql.catalyst.util.DateTimeUtils.fromJavaDate(Date.valueOf(oldV.getUTF8String(i).toString()));
|
||||
newV.putInt(i, days);
|
||||
} else if (newType instanceof DecimalType) {
|
||||
DecimalType decimalType = (DecimalType) newType;
|
||||
java.math.BigDecimal bigDecimal = new java.math.BigDecimal(oldV.getUTF8String(i).toString().trim());
|
||||
Decimal sparkDecimal = Decimal.apply(bigDecimal);
|
||||
sparkDecimal.changePrecision(decimalType.precision(), decimalType.scale());
|
||||
newV.putDecimal(i, sparkDecimal, decimalType.precision());
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static boolean convertColumnVectorType(WritableColumnVector oldV, WritableColumnVector newV, int len) {
|
||||
if (len == 0 || oldV == null || newV == null) {
|
||||
return false;
|
||||
}
|
||||
DataType oldType = oldV.dataType(); // old colType eg: floatType
|
||||
DataType newType = newV.dataType(); // new colType eg: doubleType
|
||||
if (oldV != null && newType != null) {
|
||||
if (oldType instanceof BooleanType) {
|
||||
return false;
|
||||
} else if (oldType instanceof ByteType) {
|
||||
return false;
|
||||
} else if (oldType instanceof ShortType) {
|
||||
return false;
|
||||
} else if (oldType instanceof IntegerType) {
|
||||
return convertIntLongType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof LongType) {
|
||||
return convertIntLongType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof FloatType) {
|
||||
return convertFloatType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof DoubleType) {
|
||||
return convertDoubleType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof StringType) {
|
||||
return convertStringType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof BinaryType) {
|
||||
return false;
|
||||
} else if (oldType instanceof DecimalType) {
|
||||
return convertDecimalType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof DateType) {
|
||||
return convertDateType(oldV, newV, newType, len);
|
||||
} else if (oldType instanceof TimestampType) {
|
||||
return false;
|
||||
} else {
|
||||
throw new UnsupportedOperationException("Datatype not supported " + oldV);
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -37,8 +37,12 @@ import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFile
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
@@ -55,6 +59,10 @@ object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
|
||||
def gteqSpark3_2: Boolean = SPARK_VERSION > "3.2"
|
||||
|
||||
def gteqSpark3_1: Boolean = SPARK_VERSION > "3.1"
|
||||
|
||||
def gteqSpark3_1_3: Boolean = SPARK_VERSION >= "3.1.3"
|
||||
|
||||
def getMetaSchema: StructType = {
|
||||
StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => {
|
||||
StructField(col, StringType, nullable = true)
|
||||
@@ -302,17 +310,25 @@ object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
AttributeReference(columnName, field.get.dataType, field.get.nullable)()
|
||||
}
|
||||
|
||||
def getRequiredSchema(tableAvroSchema: Schema, requiredColumns: Array[String]): (Schema, StructType) = {
|
||||
// First get the required avro-schema, then convert the avro-schema to spark schema.
|
||||
val name2Fields = tableAvroSchema.getFields.asScala.map(f => f.name() -> f).toMap
|
||||
// Here have to create a new Schema.Field object
|
||||
// to prevent throwing exceptions like "org.apache.avro.AvroRuntimeException: Field already used".
|
||||
val requiredFields = requiredColumns.map(c => name2Fields(c))
|
||||
.map(f => new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order())).toList
|
||||
val requiredAvroSchema = Schema.createRecord(tableAvroSchema.getName, tableAvroSchema.getDoc,
|
||||
tableAvroSchema.getNamespace, tableAvroSchema.isError, requiredFields.asJava)
|
||||
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
|
||||
(requiredAvroSchema, requiredStructSchema)
|
||||
def getRequiredSchema(tableAvroSchema: Schema, requiredColumns: Array[String], internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema): (Schema, StructType, InternalSchema) = {
|
||||
if (internalSchema.isEmptySchema || requiredColumns.isEmpty) {
|
||||
// First get the required avro-schema, then convert the avro-schema to spark schema.
|
||||
val name2Fields = tableAvroSchema.getFields.asScala.map(f => f.name() -> f).toMap
|
||||
// Here have to create a new Schema.Field object
|
||||
// to prevent throwing exceptions like "org.apache.avro.AvroRuntimeException: Field already used".
|
||||
val requiredFields = requiredColumns.map(c => name2Fields(c))
|
||||
.map(f => new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order())).toList
|
||||
val requiredAvroSchema = Schema.createRecord(tableAvroSchema.getName, tableAvroSchema.getDoc,
|
||||
tableAvroSchema.getNamespace, tableAvroSchema.isError, requiredFields.asJava)
|
||||
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
|
||||
(requiredAvroSchema, requiredStructSchema, internalSchema)
|
||||
} else {
|
||||
// now we support nested project
|
||||
val prunedInternalSchema = InternalSchemaUtils.pruneInternalSchema(internalSchema, requiredColumns.toList.asJava)
|
||||
val requiredAvroSchema = AvroInternalSchemaConverter.convert(prunedInternalSchema, tableAvroSchema.getName)
|
||||
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
|
||||
(requiredAvroSchema, requiredStructSchema, prunedInternalSchema)
|
||||
}
|
||||
}
|
||||
|
||||
def toAttribute(tableSchema: StructType): Seq[AttributeReference] = {
|
||||
|
||||
@@ -33,6 +33,9 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelatio
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row, SparkSession}
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
|
||||
import java.util.Locale
|
||||
|
||||
@@ -167,4 +170,14 @@ trait SparkAdapter extends Serializable {
|
||||
other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create customresolutionRule to deal with alter command for hudi.
|
||||
*/
|
||||
def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan]
|
||||
|
||||
/**
|
||||
* Create hoodie parquet file format.
|
||||
*/
|
||||
def createHoodieParquetFileFormat(): Option[ParquetFileFormat]
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user