[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]
|
||||
}
|
||||
|
||||
@@ -117,6 +117,12 @@
|
||||
<artifactId>avro</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- caffeine -->
|
||||
<dependency>
|
||||
<groupId>com.github.ben-manes.caffeine</groupId>
|
||||
<artifactId>caffeine</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- Parquet -->
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
|
||||
@@ -19,6 +19,8 @@
|
||||
package org.apache.hudi.avro;
|
||||
|
||||
import org.apache.avro.AvroRuntimeException;
|
||||
import org.apache.avro.SchemaCompatibility;
|
||||
import org.apache.avro.Conversions;
|
||||
import org.apache.avro.Conversions.DecimalConversion;
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
@@ -55,16 +57,26 @@ import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.BigInteger;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
import java.util.TimeZone;
|
||||
import java.util.Iterator;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.avro.Schema.Type.UNION;
|
||||
|
||||
/**
|
||||
* Helper class to do common stuff across Avro.
|
||||
*/
|
||||
@@ -74,6 +86,11 @@ public class HoodieAvroUtils {
|
||||
|
||||
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
|
||||
|
||||
private static final long MILLIS_PER_DAY = 86400000L;
|
||||
|
||||
//Export for test
|
||||
public static final Conversions.DecimalConversion DECIMAL_CONVERSION = new Conversions.DecimalConversion();
|
||||
|
||||
// As per https://avro.apache.org/docs/current/spec.html#names
|
||||
private static String INVALID_AVRO_CHARS_IN_NAMES = "[^A-Za-z0-9_]";
|
||||
private static String INVALID_AVRO_FIRST_CHAR_IN_NAMES = "[^A-Za-z_]";
|
||||
@@ -655,4 +672,271 @@ public class HoodieAvroUtils {
|
||||
|
||||
return nonNullType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new schema.
|
||||
* support deep rewrite for nested record.
|
||||
* This particular method does the following things :
|
||||
* a) Create a new empty GenericRecord with the new schema.
|
||||
* b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this transformed schema
|
||||
*
|
||||
* @param oldRecord oldRecord to be rewritten
|
||||
* @param newSchema newSchema used to rewrite oldRecord
|
||||
* @return newRecord for new Schema
|
||||
*/
|
||||
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema) {
|
||||
Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema);
|
||||
return (GenericData.Record) newRecord;
|
||||
}
|
||||
|
||||
private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema) {
|
||||
if (oldRecord == null) {
|
||||
return null;
|
||||
}
|
||||
switch (newSchema.getType()) {
|
||||
case RECORD:
|
||||
if (!(oldRecord instanceof IndexedRecord)) {
|
||||
throw new IllegalArgumentException("cannot rewrite record with different type");
|
||||
}
|
||||
IndexedRecord indexedRecord = (IndexedRecord) oldRecord;
|
||||
List<Schema.Field> fields = newSchema.getFields();
|
||||
Map<Integer, Object> helper = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
Schema.Field field = fields.get(i);
|
||||
if (oldSchema.getField(field.name()) != null) {
|
||||
Schema.Field oldField = oldSchema.getField(field.name());
|
||||
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema()));
|
||||
}
|
||||
}
|
||||
GenericData.Record newRecord = new GenericData.Record(newSchema);
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
if (helper.containsKey(i)) {
|
||||
newRecord.put(i, helper.get(i));
|
||||
} else {
|
||||
if (fields.get(i).defaultVal() instanceof JsonProperties.Null) {
|
||||
newRecord.put(i, null);
|
||||
} else {
|
||||
newRecord.put(i, fields.get(i).defaultVal());
|
||||
}
|
||||
}
|
||||
}
|
||||
return newRecord;
|
||||
case ARRAY:
|
||||
if (!(oldRecord instanceof Collection)) {
|
||||
throw new IllegalArgumentException("cannot rewrite record with different type");
|
||||
}
|
||||
Collection array = (Collection)oldRecord;
|
||||
List<Object> newArray = new ArrayList();
|
||||
for (Object element : array) {
|
||||
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType()));
|
||||
}
|
||||
return newArray;
|
||||
case MAP:
|
||||
if (!(oldRecord instanceof Map)) {
|
||||
throw new IllegalArgumentException("cannot rewrite record with different type");
|
||||
}
|
||||
Map<Object, Object> map = (Map<Object, Object>) oldRecord;
|
||||
Map<Object, Object> newMap = new HashMap<>();
|
||||
for (Map.Entry<Object, Object> entry : map.entrySet()) {
|
||||
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType()));
|
||||
}
|
||||
return newMap;
|
||||
case UNION:
|
||||
return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord));
|
||||
default:
|
||||
return rewritePrimaryType(oldRecord, oldSchema, newSchema);
|
||||
}
|
||||
}
|
||||
|
||||
private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schema newSchema) {
|
||||
Schema realOldSchema = oldSchema;
|
||||
if (realOldSchema.getType() == UNION) {
|
||||
realOldSchema = getActualSchemaFromUnion(oldSchema, oldValue);
|
||||
}
|
||||
if (realOldSchema.getType() == newSchema.getType()) {
|
||||
switch (realOldSchema.getType()) {
|
||||
case NULL:
|
||||
case BOOLEAN:
|
||||
case INT:
|
||||
case LONG:
|
||||
case FLOAT:
|
||||
case DOUBLE:
|
||||
case BYTES:
|
||||
case STRING:
|
||||
return oldValue;
|
||||
case FIXED:
|
||||
// fixed size and name must match:
|
||||
if (!SchemaCompatibility.schemaNameEquals(realOldSchema, newSchema) || realOldSchema.getFixedSize() != newSchema.getFixedSize()) {
|
||||
// deal with the precision change for decimalType
|
||||
if (realOldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
final byte[] bytes;
|
||||
bytes = ((GenericFixed) oldValue).bytes();
|
||||
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) realOldSchema.getLogicalType();
|
||||
BigDecimal bd = new BigDecimal(new BigInteger(bytes), decimal.getScale()).setScale(((LogicalTypes.Decimal) newSchema.getLogicalType()).getScale());
|
||||
return DECIMAL_CONVERSION.toFixed(bd, newSchema, newSchema.getLogicalType());
|
||||
}
|
||||
} else {
|
||||
return oldValue;
|
||||
}
|
||||
return oldValue;
|
||||
default:
|
||||
throw new AvroRuntimeException("Unknown schema type: " + newSchema.getType());
|
||||
}
|
||||
} else {
|
||||
return rewritePrimaryTypeWithDiffSchemaType(oldValue, realOldSchema, newSchema);
|
||||
}
|
||||
}
|
||||
|
||||
private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Schema oldSchema, Schema newSchema) {
|
||||
switch (newSchema.getType()) {
|
||||
case NULL:
|
||||
case BOOLEAN:
|
||||
break;
|
||||
case INT:
|
||||
if (newSchema.getLogicalType() == LogicalTypes.date() && oldSchema.getType() == Schema.Type.STRING) {
|
||||
return fromJavaDate(java.sql.Date.valueOf(oldValue.toString()));
|
||||
}
|
||||
break;
|
||||
case LONG:
|
||||
if (oldSchema.getType() == Schema.Type.INT) {
|
||||
return ((Integer) oldValue).longValue();
|
||||
}
|
||||
break;
|
||||
case FLOAT:
|
||||
if ((oldSchema.getType() == Schema.Type.INT)
|
||||
|| (oldSchema.getType() == Schema.Type.LONG)) {
|
||||
return oldSchema.getType() == Schema.Type.INT ? ((Integer) oldValue).floatValue() : ((Long) oldValue).floatValue();
|
||||
}
|
||||
break;
|
||||
case DOUBLE:
|
||||
if (oldSchema.getType() == Schema.Type.FLOAT) {
|
||||
// java float cannot convert to double directly, deal with float precision change
|
||||
return Double.valueOf(oldValue + "");
|
||||
} else if (oldSchema.getType() == Schema.Type.INT) {
|
||||
return ((Integer) oldValue).doubleValue();
|
||||
} else if (oldSchema.getType() == Schema.Type.LONG) {
|
||||
return ((Long) oldValue).doubleValue();
|
||||
}
|
||||
break;
|
||||
case BYTES:
|
||||
if (oldSchema.getType() == Schema.Type.STRING) {
|
||||
return (oldValue.toString()).getBytes(StandardCharsets.UTF_8);
|
||||
}
|
||||
break;
|
||||
case STRING:
|
||||
if (oldSchema.getType() == Schema.Type.BYTES) {
|
||||
return String.valueOf(((byte[]) oldValue));
|
||||
}
|
||||
if (oldSchema.getLogicalType() == LogicalTypes.date()) {
|
||||
return toJavaDate((Integer) oldValue).toString();
|
||||
}
|
||||
if (oldSchema.getType() == Schema.Type.INT
|
||||
|| oldSchema.getType() == Schema.Type.LONG
|
||||
|| oldSchema.getType() == Schema.Type.FLOAT
|
||||
|| oldSchema.getType() == Schema.Type.DOUBLE) {
|
||||
return oldValue.toString();
|
||||
}
|
||||
if (oldSchema.getType() == Schema.Type.FIXED && oldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
final byte[] bytes;
|
||||
bytes = ((GenericFixed) oldValue).bytes();
|
||||
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) oldSchema.getLogicalType();
|
||||
BigDecimal bd = new BigDecimal(new BigInteger(bytes), decimal.getScale());
|
||||
return bd.toString();
|
||||
}
|
||||
break;
|
||||
case FIXED:
|
||||
// deal with decimal Type
|
||||
if (newSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
// TODO: support more types
|
||||
if (oldSchema.getType() == Schema.Type.STRING
|
||||
|| oldSchema.getType() == Schema.Type.DOUBLE
|
||||
|| oldSchema.getType() == Schema.Type.INT
|
||||
|| oldSchema.getType() == Schema.Type.LONG
|
||||
|| oldSchema.getType() == Schema.Type.FLOAT) {
|
||||
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) newSchema.getLogicalType();
|
||||
BigDecimal bigDecimal = null;
|
||||
if (oldSchema.getType() == Schema.Type.STRING) {
|
||||
bigDecimal = new java.math.BigDecimal(oldValue.toString())
|
||||
.setScale(decimal.getScale());
|
||||
} else {
|
||||
// Due to Java, there will be precision problems in direct conversion, we should use string instead of use double
|
||||
bigDecimal = new java.math.BigDecimal(oldValue.toString())
|
||||
.setScale(decimal.getScale());
|
||||
}
|
||||
return DECIMAL_CONVERSION.toFixed(bigDecimal, newSchema, newSchema.getLogicalType());
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
}
|
||||
throw new AvroRuntimeException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema));
|
||||
}
|
||||
|
||||
// convert days to Date
|
||||
private static java.sql.Date toJavaDate(int days) {
|
||||
long localMillis = Math.multiplyExact(days, MILLIS_PER_DAY);
|
||||
int timeZoneOffset;
|
||||
TimeZone defaultTimeZone = TimeZone.getDefault();
|
||||
if (defaultTimeZone instanceof sun.util.calendar.ZoneInfo) {
|
||||
timeZoneOffset = ((sun.util.calendar.ZoneInfo) defaultTimeZone).getOffsetsByWall(localMillis, null);
|
||||
} else {
|
||||
timeZoneOffset = defaultTimeZone.getOffset(localMillis - defaultTimeZone.getRawOffset());
|
||||
}
|
||||
return new java.sql.Date(localMillis - timeZoneOffset);
|
||||
}
|
||||
|
||||
// convert Date to days
|
||||
private static int fromJavaDate(Date date) {
|
||||
long millisUtc = date.getTime();
|
||||
long millisLocal = millisUtc + TimeZone.getDefault().getOffset(millisUtc);
|
||||
int julianDays = Math.toIntExact(Math.floorDiv(millisLocal, MILLIS_PER_DAY));
|
||||
return julianDays;
|
||||
}
|
||||
|
||||
private static Schema getActualSchemaFromUnion(Schema schema, Object data) {
|
||||
Schema actualSchema;
|
||||
if (!schema.getType().equals(UNION)) {
|
||||
return schema;
|
||||
}
|
||||
if (schema.getTypes().size() == 2
|
||||
&& schema.getTypes().get(0).getType() == Schema.Type.NULL) {
|
||||
actualSchema = schema.getTypes().get(1);
|
||||
} else if (schema.getTypes().size() == 2
|
||||
&& schema.getTypes().get(1).getType() == Schema.Type.NULL) {
|
||||
actualSchema = schema.getTypes().get(0);
|
||||
} else if (schema.getTypes().size() == 1) {
|
||||
actualSchema = schema.getTypes().get(0);
|
||||
} else {
|
||||
// deal complex union. this should not happened in hoodie,
|
||||
// since flink/spark do not write this type.
|
||||
int i = GenericData.get().resolveUnion(schema, data);
|
||||
actualSchema = schema.getTypes().get(i);
|
||||
}
|
||||
return actualSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given avro records, rewrites them with new schema.
|
||||
*
|
||||
* @param oldRecords oldRecords to be rewrite
|
||||
* @param newSchema newSchema used to rewrite oldRecord
|
||||
* @return a iterator of rewrote GeneriRcords
|
||||
*/
|
||||
public static Iterator<GenericRecord> rewriteRecordWithNewSchema(Iterator<GenericRecord> oldRecords, Schema newSchema) {
|
||||
if (oldRecords == null || newSchema == null) {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
return new Iterator<GenericRecord>() {
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return oldRecords.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericRecord next() {
|
||||
return rewriteRecordWithNewSchema(oldRecords.next(), newSchema);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@@ -50,6 +50,10 @@ public enum WriteOperationType {
|
||||
COMPACT("compact"),
|
||||
|
||||
INDEX("index"),
|
||||
|
||||
// alter schema
|
||||
ALTER_SCHEMA("alter_schema"),
|
||||
|
||||
// used for old version
|
||||
UNKNOWN("unknown");
|
||||
|
||||
@@ -90,6 +94,8 @@ public enum WriteOperationType {
|
||||
return COMPACT;
|
||||
case "index":
|
||||
return INDEX;
|
||||
case "alter_schema":
|
||||
return ALTER_SCHEMA;
|
||||
case "unknown":
|
||||
return UNKNOWN;
|
||||
default:
|
||||
|
||||
@@ -90,6 +90,8 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR
|
||||
+ ".fileids";
|
||||
|
||||
public static final String SCHEMA_FOLDER_NAME = ".schema";
|
||||
|
||||
public static final String MARKER_EXTN = ".marker";
|
||||
|
||||
private String basePath;
|
||||
@@ -192,6 +194,13 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return new Path(metaPath, COLUMN_STATISTICS_INDEX_NAME).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return schema folder path
|
||||
*/
|
||||
public String getSchemaFolderName() {
|
||||
return new Path(metaPath, SCHEMA_FOLDER_NAME).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Temp Folder path
|
||||
*/
|
||||
@@ -392,6 +401,11 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
if (!fs.exists(metaPathDir)) {
|
||||
fs.mkdirs(metaPathDir);
|
||||
}
|
||||
// create schema folder
|
||||
Path schemaPathDir = new Path(metaPathDir, SCHEMA_FOLDER_NAME);
|
||||
if (!fs.exists(schemaPathDir)) {
|
||||
fs.mkdirs(schemaPathDir);
|
||||
}
|
||||
|
||||
// if anything other than default archive log folder is specified, create that too
|
||||
String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER);
|
||||
|
||||
@@ -18,13 +18,6 @@
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.SchemaCompatibility;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
@@ -44,8 +37,18 @@ import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.InvalidTableException;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import org.apache.hudi.io.storage.HoodieOrcReader;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.SchemaCompatibility;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
@@ -534,4 +537,51 @@ public class TableSchemaResolver {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the InternalSchema for a hoodie table from the HoodieCommitMetadata of the instant.
|
||||
*
|
||||
* @return InternalSchema for this table
|
||||
*/
|
||||
public Option<InternalSchema> getTableInternalSchemaFromCommitMetadata() {
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
if (timeline.lastInstant().isPresent()) {
|
||||
return getTableInternalSchemaFromCommitMetadata(timeline.lastInstant().get());
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the InternalSchema for a hoodie table from the HoodieCommitMetadata of the instant.
|
||||
*
|
||||
* @return InternalSchema for this table
|
||||
*/
|
||||
private Option<InternalSchema> getTableInternalSchemaFromCommitMetadata(HoodieInstant instant) {
|
||||
try {
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedInstants();
|
||||
byte[] data = timeline.getInstantDetails(instant).get();
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
|
||||
String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
|
||||
if (latestInternalSchemaStr != null) {
|
||||
return SerDeHelper.fromJson(latestInternalSchemaStr);
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Failed to read schema from commit metadata", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the history schemas as String for a hoodie table from the HoodieCommitMetadata of the instant.
|
||||
*
|
||||
* @return history schemas string for this table
|
||||
*/
|
||||
public Option<String> getTableHistorySchemaStrFromCommitMetadata() {
|
||||
// now we only support FileBaseInternalSchemaManager
|
||||
FileBasedInternalSchemaStorageManager manager = new FileBasedInternalSchemaStorageManager(metaClient);
|
||||
String result = manager.getHistorySchemaStr();
|
||||
return result.isEmpty() ? Option.empty() : Option.of(result);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.table.log;
|
||||
|
||||
import org.apache.hudi.common.model.DeleteRecord;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -36,6 +37,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SpillableMapUtils;
|
||||
import org.apache.hudi.common.util.InternalSchemaCache;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -46,6 +48,9 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -109,6 +114,10 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
private final FileSystem fs;
|
||||
// Total log files read - for metrics
|
||||
private AtomicLong totalLogFiles = new AtomicLong(0);
|
||||
// Internal schema, used to support full schema evolution.
|
||||
private InternalSchema internalSchema;
|
||||
// Hoodie table path.
|
||||
private final String path;
|
||||
// Total log blocks read - for metrics
|
||||
private AtomicLong totalLogBlocks = new AtomicLong(0);
|
||||
// Total log records read - for metrics
|
||||
@@ -135,14 +144,14 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
int bufferSize, Option<InstantRange> instantRange,
|
||||
boolean withOperationField) {
|
||||
this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
|
||||
instantRange, withOperationField, true, Option.empty());
|
||||
instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema());
|
||||
}
|
||||
|
||||
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
|
||||
boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
|
||||
boolean withOperationField, boolean enableFullScan,
|
||||
Option<String> partitionName) {
|
||||
Option<String> partitionName, InternalSchema internalSchema) {
|
||||
this.readerSchema = readerSchema;
|
||||
this.latestInstantTime = latestInstantTime;
|
||||
this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
|
||||
@@ -159,6 +168,8 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
this.instantRange = instantRange;
|
||||
this.withOperationField = withOperationField;
|
||||
this.enableFullScan = enableFullScan;
|
||||
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
|
||||
this.path = basePath;
|
||||
|
||||
// Key fields when populate meta fields is disabled (that is, virtual keys enabled)
|
||||
if (!tableConfig.populateMetaFields()) {
|
||||
@@ -202,7 +213,7 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
// Iterate over the paths
|
||||
logFormatReaderWrapper = new HoodieLogFormatReader(fs,
|
||||
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()),
|
||||
readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField);
|
||||
readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField, internalSchema);
|
||||
Set<HoodieLogFile> scannedLogFiles = new HashSet<>();
|
||||
while (logFormatReaderWrapper.hasNext()) {
|
||||
HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
|
||||
@@ -361,8 +372,10 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
*/
|
||||
private void processDataBlock(HoodieDataBlock dataBlock, Option<List<String>> keys) throws Exception {
|
||||
try (ClosableIterator<IndexedRecord> recordItr = dataBlock.getRecordItr(keys.orElse(Collections.emptyList()))) {
|
||||
Option<Schema> schemaOption = getMergedSchema(dataBlock);
|
||||
while (recordItr.hasNext()) {
|
||||
IndexedRecord record = recordItr.next();
|
||||
IndexedRecord currentRecord = recordItr.next();
|
||||
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get()) : currentRecord;
|
||||
processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
|
||||
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
|
||||
totalLogRecords.incrementAndGet();
|
||||
@@ -370,6 +383,28 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get final Read Schema for support evolution.
|
||||
* step1: find the fileSchema for current dataBlock.
|
||||
* step2: determine whether fileSchema is compatible with the final read internalSchema.
|
||||
* step3: merge fileSchema and read internalSchema to produce final read schema.
|
||||
*
|
||||
* @param dataBlock current processed block
|
||||
* @return final read schema.
|
||||
*/
|
||||
private Option<Schema> getMergedSchema(HoodieDataBlock dataBlock) {
|
||||
Option<Schema> result = Option.empty();
|
||||
if (!internalSchema.isEmptySchema()) {
|
||||
Long currentInstantTime = Long.parseLong(dataBlock.getLogBlockHeader().get(INSTANT_TIME));
|
||||
InternalSchema fileSchema = InternalSchemaCache
|
||||
.searchSchemaAndCache(currentInstantTime, hoodieTableMetaClient, false);
|
||||
Schema mergeSchema = AvroInternalSchemaConverter
|
||||
.convert(new InternalSchemaMerger(fileSchema, internalSchema, true, false).mergeSchema(), readerSchema.getName());
|
||||
result = Option.of(mergeSchema);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create @{@link HoodieRecord} from the @{@link IndexedRecord}.
|
||||
*
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.CorruptedLogFileException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -44,6 +45,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -74,6 +76,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
private final HoodieLogFile logFile;
|
||||
private final byte[] magicBuffer = new byte[6];
|
||||
private final Schema readerSchema;
|
||||
private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
|
||||
private final String keyField;
|
||||
private boolean readBlockLazily;
|
||||
private long reverseLogFilePosition;
|
||||
@@ -97,6 +100,12 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
|
||||
boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
|
||||
String keyField) throws IOException {
|
||||
this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, enableRecordLookups, keyField, InternalSchema.getEmptyInternalSchema());
|
||||
}
|
||||
|
||||
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
|
||||
boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
|
||||
String keyField, InternalSchema internalSchema) throws IOException {
|
||||
this.hadoopConf = fs.getConf();
|
||||
// NOTE: We repackage {@code HoodieLogFile} here to make sure that the provided path
|
||||
// is prefixed with an appropriate scheme given that we're not propagating the FS
|
||||
@@ -108,6 +117,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
this.reverseReader = reverseReader;
|
||||
this.enableRecordLookups = enableRecordLookups;
|
||||
this.keyField = keyField;
|
||||
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
|
||||
if (this.reverseReader) {
|
||||
this.reverseLogFilePosition = this.lastReverseLogFilePosition = this.logFile.getFileSize();
|
||||
}
|
||||
@@ -197,10 +207,10 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
switch (Objects.requireNonNull(blockType)) {
|
||||
case AVRO_DATA_BLOCK:
|
||||
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
|
||||
return HoodieAvroDataBlock.getBlock(content.get(), readerSchema);
|
||||
return HoodieAvroDataBlock.getBlock(content.get(), readerSchema, internalSchema);
|
||||
} else {
|
||||
return new HoodieAvroDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
|
||||
Option.ofNullable(readerSchema), header, footer, keyField);
|
||||
Option.ofNullable(readerSchema), header, footer, keyField, internalSchema);
|
||||
}
|
||||
|
||||
case HFILE_DATA_BLOCK:
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -42,6 +43,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
private HoodieLogFileReader currentReader;
|
||||
private final FileSystem fs;
|
||||
private final Schema readerSchema;
|
||||
private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
|
||||
private final boolean readBlocksLazily;
|
||||
private final boolean reverseLogReader;
|
||||
private final String recordKeyField;
|
||||
@@ -53,6 +55,12 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles, Schema readerSchema, boolean readBlocksLazily,
|
||||
boolean reverseLogReader, int bufferSize, boolean enableInlineReading,
|
||||
String recordKeyField) throws IOException {
|
||||
this(fs, logFiles, readerSchema, readBlocksLazily, reverseLogReader, bufferSize, enableInlineReading, recordKeyField, InternalSchema.getEmptyInternalSchema());
|
||||
}
|
||||
|
||||
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles, Schema readerSchema, boolean readBlocksLazily,
|
||||
boolean reverseLogReader, int bufferSize, boolean enableInlineReading,
|
||||
String recordKeyField, InternalSchema internalSchema) throws IOException {
|
||||
this.logFiles = logFiles;
|
||||
this.fs = fs;
|
||||
this.readerSchema = readerSchema;
|
||||
@@ -62,10 +70,11 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
this.prevReadersInOpenState = new ArrayList<>();
|
||||
this.recordKeyField = recordKeyField;
|
||||
this.enableInlineReading = enableInlineReading;
|
||||
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
|
||||
if (logFiles.size() > 0) {
|
||||
HoodieLogFile nextLogFile = logFiles.remove(0);
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
|
||||
enableInlineReading, recordKeyField);
|
||||
enableInlineReading, recordKeyField, internalSchema);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -105,7 +114,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
this.prevReadersInOpenState.add(currentReader);
|
||||
}
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
|
||||
enableInlineReading, recordKeyField);
|
||||
enableInlineReading, recordKeyField, internalSchema);
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to initialize read with log file ", io);
|
||||
}
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -79,10 +80,10 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
|
||||
ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isBitCaskDiskMapCompressionEnabled,
|
||||
boolean withOperationField, boolean enableFullScan,
|
||||
Option<String> partitionName) {
|
||||
Option<String> partitionName, InternalSchema internalSchema) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
|
||||
instantRange, withOperationField,
|
||||
enableFullScan, partitionName);
|
||||
enableFullScan, partitionName, internalSchema);
|
||||
try {
|
||||
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
|
||||
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
|
||||
@@ -197,6 +198,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
|
||||
protected String basePath;
|
||||
protected List<String> logFilePaths;
|
||||
protected Schema readerSchema;
|
||||
private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
|
||||
protected String latestInstantTime;
|
||||
protected boolean readBlocksLazily;
|
||||
protected boolean reverseReader;
|
||||
@@ -293,6 +295,11 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInternalSchema(InternalSchema internalSchema) {
|
||||
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withOperationField(boolean withOperationField) {
|
||||
this.withOperationField = withOperationField;
|
||||
return this;
|
||||
@@ -310,7 +317,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
|
||||
latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
|
||||
bufferSize, spillableMapBasePath, instantRange, autoScan,
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true,
|
||||
Option.ofNullable(partitionName));
|
||||
Option.ofNullable(partitionName), internalSchema);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.io.ByteArrayInputStream;
|
||||
@@ -62,6 +63,17 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
|
||||
|
||||
private final ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
|
||||
|
||||
public HoodieAvroDataBlock(FSDataInputStream inputStream,
|
||||
Option<byte[]> content,
|
||||
boolean readBlockLazily,
|
||||
HoodieLogBlockContentLocation logBlockContentLocation,
|
||||
Option<Schema> readerSchema,
|
||||
Map<HeaderMetadataType, String> header,
|
||||
Map<HeaderMetadataType, String> footer,
|
||||
String keyField, InternalSchema internalSchema) {
|
||||
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false, internalSchema);
|
||||
}
|
||||
|
||||
public HoodieAvroDataBlock(FSDataInputStream inputStream,
|
||||
Option<byte[]> content,
|
||||
boolean readBlockLazily,
|
||||
@@ -126,7 +138,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
|
||||
@Override
|
||||
protected ClosableIterator<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
|
||||
checkState(this.readerSchema != null, "Reader's schema has to be non-null");
|
||||
return RecordIterator.getInstance(this, content);
|
||||
return RecordIterator.getInstance(this, content, internalSchema);
|
||||
}
|
||||
|
||||
private static class RecordIterator implements ClosableIterator<IndexedRecord> {
|
||||
@@ -138,7 +150,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
|
||||
private int totalRecords = 0;
|
||||
private int readRecords = 0;
|
||||
|
||||
private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) throws IOException {
|
||||
private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, InternalSchema internalSchema) throws IOException {
|
||||
this.content = content;
|
||||
|
||||
this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content)));
|
||||
@@ -147,17 +159,26 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
|
||||
int version = this.dis.readInt();
|
||||
HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version);
|
||||
|
||||
this.reader = new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
Schema finalReadSchema = readerSchema;
|
||||
if (!internalSchema.isEmptySchema()) {
|
||||
// we should use write schema to read log file,
|
||||
// since when we have done some DDL operation, the readerSchema maybe different from writeSchema, avro reader will throw exception.
|
||||
// eg: origin writeSchema is: "a String, b double" then we add a new column now the readerSchema will be: "a string, c int, b double". it's wrong to use readerSchema to read old log file.
|
||||
// after we read those record by writeSchema, we rewrite those record with readerSchema in AbstractHoodieLogRecordReader
|
||||
finalReadSchema = writerSchema;
|
||||
}
|
||||
|
||||
this.reader = new GenericDatumReader<>(writerSchema, finalReadSchema);
|
||||
|
||||
if (logBlockVersion.hasRecordCount()) {
|
||||
this.totalRecords = this.dis.readInt();
|
||||
}
|
||||
}
|
||||
|
||||
public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException {
|
||||
public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, InternalSchema internalSchema) throws IOException {
|
||||
// Get schema from the header
|
||||
Schema writerSchema = new Schema.Parser().parse(dataBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
|
||||
return new RecordIterator(dataBlock.readerSchema, writerSchema, content);
|
||||
return new RecordIterator(dataBlock.readerSchema, writerSchema, content, internalSchema);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -209,12 +230,16 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
|
||||
super(records, Collections.singletonMap(HeaderMetadataType.SCHEMA, schema.toString()), new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
|
||||
}
|
||||
|
||||
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
|
||||
return getBlock(content, readerSchema, InternalSchema.getEmptyInternalSchema());
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using
|
||||
* HoodieLogFormat V1.
|
||||
*/
|
||||
@Deprecated
|
||||
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
|
||||
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema, InternalSchema internalSchema) throws IOException {
|
||||
|
||||
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
|
||||
|
||||
@@ -228,6 +253,10 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
|
||||
readerSchema = writerSchema;
|
||||
}
|
||||
|
||||
if (!internalSchema.isEmptySchema()) {
|
||||
readerSchema = writerSchema;
|
||||
}
|
||||
|
||||
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
// 2. Get the total records
|
||||
int totalRecords = dis.readInt();
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
@@ -60,6 +61,8 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
|
||||
|
||||
protected final Schema readerSchema;
|
||||
|
||||
protected InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema();
|
||||
|
||||
/**
|
||||
* NOTE: This ctor is used on the write-path (ie when records ought to be written into the log)
|
||||
*/
|
||||
@@ -95,6 +98,25 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
|
||||
this.enablePointLookups = enablePointLookups;
|
||||
}
|
||||
|
||||
protected HoodieDataBlock(Option<byte[]> content,
|
||||
FSDataInputStream inputStream,
|
||||
boolean readBlockLazily,
|
||||
Option<HoodieLogBlockContentLocation> blockContentLocation,
|
||||
Option<Schema> readerSchema,
|
||||
Map<HeaderMetadataType, String> headers,
|
||||
Map<HeaderMetadataType, String> footer,
|
||||
String keyFieldName,
|
||||
boolean enablePointLookups,
|
||||
InternalSchema internalSchema) {
|
||||
super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
|
||||
this.records = Option.empty();
|
||||
this.keyFieldName = keyFieldName;
|
||||
// If no reader-schema has been provided assume writer-schema as one
|
||||
this.readerSchema = readerSchema.orElseGet(() -> getWriterSchema(super.getLogBlockHeader()));
|
||||
this.enablePointLookups = enablePointLookups;
|
||||
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getContentBytes() throws IOException {
|
||||
// In case this method is called before realizing records from content
|
||||
|
||||
@@ -74,7 +74,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
|
||||
ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION,
|
||||
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION,
|
||||
REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION));
|
||||
REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION,
|
||||
REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION));
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
|
||||
@@ -227,7 +228,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public void deleteInstantFileIfExists(HoodieInstant instant) {
|
||||
LOG.info("Deleting instant " + instant);
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName());
|
||||
try {
|
||||
if (metaClient.getFs().exists(inFlightCommitFilePath)) {
|
||||
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
|
||||
@@ -246,7 +247,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
private void deleteInstantFile(HoodieInstant instant) {
|
||||
LOG.info("Deleting instant " + instant);
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName());
|
||||
try {
|
||||
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
|
||||
if (result) {
|
||||
@@ -261,7 +262,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
@Override
|
||||
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
|
||||
Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
Path detailPath = getInstantFileNamePath(instant.getFileName());
|
||||
return readDataFromPath(detailPath);
|
||||
}
|
||||
|
||||
@@ -307,12 +308,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public Option<byte[]> readCleanerInfoAsBytes(HoodieInstant instant) {
|
||||
// Cleaner metadata are always stored only in timeline .hoodie
|
||||
return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName()));
|
||||
return readDataFromPath(getInstantFileNamePath(instant.getFileName()));
|
||||
}
|
||||
|
||||
public Option<byte[]> readRollbackInfoAsBytes(HoodieInstant instant) {
|
||||
// Rollback metadata are always stored only in timeline .hoodie
|
||||
return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName()));
|
||||
return readDataFromPath(getInstantFileNamePath(instant.getFileName()));
|
||||
}
|
||||
|
||||
public Option<byte[]> readRestoreInfoAsBytes(HoodieInstant instant) {
|
||||
@@ -542,24 +543,23 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
// Re-create the .inflight file by opening a new file and write the commit metadata in
|
||||
createFileInMetaPath(fromInstant.getFileName(), data, allowRedundantTransitions);
|
||||
Path fromInstantPath = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
|
||||
Path toInstantPath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
|
||||
Path fromInstantPath = getInstantFileNamePath(fromInstant.getFileName());
|
||||
Path toInstantPath = getInstantFileNamePath(toInstant.getFileName());
|
||||
boolean success = metaClient.getFs().rename(fromInstantPath, toInstantPath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath);
|
||||
}
|
||||
} else {
|
||||
// Ensures old state exists in timeline
|
||||
LOG.info("Checking for file exists ?" + new Path(metaClient.getMetaPath(), fromInstant.getFileName()));
|
||||
ValidationUtils.checkArgument(metaClient.getFs().exists(new Path(metaClient.getMetaPath(),
|
||||
fromInstant.getFileName())));
|
||||
LOG.info("Checking for file exists ?" + getInstantFileNamePath(fromInstant.getFileName()));
|
||||
ValidationUtils.checkArgument(metaClient.getFs().exists(getInstantFileNamePath(fromInstant.getFileName())));
|
||||
// Use Write Once to create Target File
|
||||
if (allowRedundantTransitions) {
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), getInstantFileNamePath(toInstant.getFileName()), data);
|
||||
} else {
|
||||
createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
|
||||
createImmutableFileInPath(getInstantFileNamePath(toInstant.getFileName()), data);
|
||||
}
|
||||
LOG.info("Create new file for toInstant ?" + new Path(metaClient.getMetaPath(), toInstant.getFileName()));
|
||||
LOG.info("Create new file for toInstant ?" + getInstantFileNamePath(toInstant.getFileName()));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not complete " + fromInstant, e);
|
||||
@@ -568,8 +568,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
|
||||
ValidationUtils.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
|
||||
Path inFlightCommitFilePath = getInstantFileNamePath(inflight.getFileName());
|
||||
Path commitFilePath = getInstantFileNamePath(completed.getFileName());
|
||||
try {
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
|
||||
@@ -580,8 +580,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Path requestedInstantFilePath = new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()).getFileName());
|
||||
Path requestedInstantFilePath = getInstantFileNamePath(new HoodieInstant(State.REQUESTED,
|
||||
inflight.getAction(), inflight.getTimestamp()).getFileName());
|
||||
|
||||
// If inflight and requested files do not exist, create one
|
||||
if (!metaClient.getFs().exists(requestedInstantFilePath)) {
|
||||
@@ -600,6 +600,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
}
|
||||
|
||||
private Path getInstantFileNamePath(String fileName) {
|
||||
return new Path(fileName.contains(SCHEMA_COMMIT_ACTION) ? metaClient.getSchemaFolderName() : metaClient.getMetaPath(), fileName);
|
||||
}
|
||||
|
||||
public void transitionRequestedToInflight(String commitType, String inFlightInstant) {
|
||||
HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant);
|
||||
transitionRequestedToInflight(requested, Option.empty(), false);
|
||||
@@ -716,7 +720,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
private void createFileInMetaPath(String filename, Option<byte[]> content, boolean allowOverwrite) {
|
||||
Path fullPath = new Path(metaClient.getMetaPath(), filename);
|
||||
Path fullPath = getInstantFileNamePath(filename);
|
||||
if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), fullPath, content);
|
||||
} else {
|
||||
|
||||
@@ -176,6 +176,10 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
|
||||
return isInflight() ? HoodieTimeline.makeInflightIndexFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestedIndexFileName(timestamp)
|
||||
: HoodieTimeline.makeIndexCommitFileName(timestamp);
|
||||
} else if (HoodieTimeline.SCHEMA_COMMIT_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightSchemaFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestSchemaFileName(timestamp)
|
||||
: HoodieTimeline.makeSchemaFileName(timestamp);
|
||||
}
|
||||
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
|
||||
}
|
||||
|
||||
@@ -56,6 +56,8 @@ public interface HoodieTimeline extends Serializable {
|
||||
String REQUESTED_EXTENSION = ".requested";
|
||||
String RESTORE_ACTION = "restore";
|
||||
String INDEXING_ACTION = "indexing";
|
||||
// only for schema save
|
||||
String SCHEMA_COMMIT_ACTION = "schemacommit";
|
||||
|
||||
String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION,
|
||||
CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION,
|
||||
@@ -88,6 +90,9 @@ public interface HoodieTimeline extends Serializable {
|
||||
String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + REQUESTED_EXTENSION;
|
||||
String INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION;
|
||||
String SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION;
|
||||
String INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + REQUESTED_EXTENSION;
|
||||
|
||||
String INVALID_INSTANT_TS = "0";
|
||||
|
||||
@@ -497,4 +502,16 @@ public interface HoodieTimeline extends Serializable {
|
||||
static String makeRequestedIndexFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeSchemaFileName(String instantTime) {
|
||||
return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeInflightSchemaFileName(String instantTime) {
|
||||
return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeRequestSchemaFileName(String instantTime) {
|
||||
return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,212 @@
|
||||
/*
|
||||
* 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.common.util;
|
||||
|
||||
import com.github.benmanes.caffeine.cache.Cache;
|
||||
import com.github.benmanes.caffeine.cache.Caffeine;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
|
||||
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class InternalSchemaCache {
|
||||
private static final Logger LOG = LogManager.getLogger(InternalSchemaCache.class);
|
||||
// Use segment lock to reduce competition.
|
||||
// the lock size should be powers of 2 for better hash.
|
||||
private static Object[] lockList = new Object[16];
|
||||
|
||||
static {
|
||||
for (int i = 0; i < lockList.length; i++) {
|
||||
lockList[i] = new Object();
|
||||
}
|
||||
}
|
||||
|
||||
// historySchemas cache maintain a map about (tablePath, HistorySchemas).
|
||||
// this is a Global cache, all threads in one container/executor share the same cache.
|
||||
private static final Cache<String, TreeMap<Long, InternalSchema>>
|
||||
HISTORICAL_SCHEMA_CACHE = Caffeine.newBuilder().maximumSize(1000).weakValues().build();
|
||||
|
||||
/**
|
||||
* Search internalSchema based on versionID.
|
||||
* first step: try to get internalSchema from hoodie commit files, we no need to add lock.
|
||||
* if we cannot get internalSchema by first step, then we try to get internalSchema from cache.
|
||||
*
|
||||
* @param versionID schema version_id need to search
|
||||
* @param metaClient current hoodie metaClient
|
||||
* @return internalSchema
|
||||
*/
|
||||
public static InternalSchema searchSchemaAndCache(long versionID, HoodieTableMetaClient metaClient, boolean cacheEnable) {
|
||||
Option<InternalSchema> candidateSchema = getSchemaByReadingCommitFile(versionID, metaClient);
|
||||
if (candidateSchema.isPresent()) {
|
||||
return candidateSchema.get();
|
||||
}
|
||||
if (!cacheEnable) {
|
||||
// parse history schema and return directly
|
||||
return InternalSchemaUtils.searchSchema(versionID, getHistoricalSchemas(metaClient));
|
||||
}
|
||||
String tablePath = metaClient.getBasePath();
|
||||
// use segment lock to reduce competition.
|
||||
synchronized (lockList[tablePath.hashCode() & (lockList.length - 1)]) {
|
||||
TreeMap<Long, InternalSchema> historicalSchemas = HISTORICAL_SCHEMA_CACHE.getIfPresent(tablePath);
|
||||
if (historicalSchemas == null || InternalSchemaUtils.searchSchema(versionID, historicalSchemas) == null) {
|
||||
historicalSchemas = getHistoricalSchemas(metaClient);
|
||||
HISTORICAL_SCHEMA_CACHE.put(tablePath, historicalSchemas);
|
||||
} else {
|
||||
long maxVersionId = historicalSchemas.keySet().stream().max(Long::compareTo).get();
|
||||
if (versionID > maxVersionId) {
|
||||
historicalSchemas = getHistoricalSchemas(metaClient);
|
||||
HISTORICAL_SCHEMA_CACHE.put(tablePath, historicalSchemas);
|
||||
}
|
||||
}
|
||||
return InternalSchemaUtils.searchSchema(versionID, historicalSchemas);
|
||||
}
|
||||
}
|
||||
|
||||
private static TreeMap<Long, InternalSchema> getHistoricalSchemas(HoodieTableMetaClient metaClient) {
|
||||
TreeMap<Long, InternalSchema> result = new TreeMap<>();
|
||||
FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient);
|
||||
String historySchemaStr = schemasManager.getHistorySchemaStr();
|
||||
if (!StringUtils.isNullOrEmpty(historySchemaStr)) {
|
||||
result = SerDeHelper.parseSchemas(historySchemaStr);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private static Option<InternalSchema> getSchemaByReadingCommitFile(long versionID, HoodieTableMetaClient metaClient) {
|
||||
try {
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
List<HoodieInstant> instants = timeline.getInstants().filter(f -> f.getTimestamp().equals(String.valueOf(versionID))).collect(Collectors.toList());
|
||||
if (instants.isEmpty()) {
|
||||
return Option.empty();
|
||||
}
|
||||
byte[] data = timeline.getInstantDetails(instants.get(0)).get();
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
|
||||
String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
|
||||
return SerDeHelper.fromJson(latestInternalSchemaStr);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Failed to read schema from commit metadata", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get internalSchema and avroSchema for compaction/cluster operation.
|
||||
*
|
||||
* @param metaClient current hoodie metaClient
|
||||
* @param compactionAndClusteringInstant first instant before current compaction/cluster instant
|
||||
* @return (internalSchemaStrOpt, avroSchemaStrOpt) a pair of InternalSchema/avroSchema
|
||||
*/
|
||||
public static Pair<Option<String>, Option<String>> getInternalSchemaAndAvroSchemaForClusteringAndCompaction(HoodieTableMetaClient metaClient, String compactionAndClusteringInstant) {
|
||||
// try to load internalSchema to support Schema Evolution
|
||||
HoodieTimeline timelineBeforeCurrentCompaction = metaClient.getCommitsAndCompactionTimeline().findInstantsBefore(compactionAndClusteringInstant).filterCompletedInstants();
|
||||
Option<HoodieInstant> lastInstantBeforeCurrentCompaction = timelineBeforeCurrentCompaction.lastInstant();
|
||||
if (lastInstantBeforeCurrentCompaction.isPresent()) {
|
||||
// try to find internalSchema
|
||||
byte[] data = timelineBeforeCurrentCompaction.getInstantDetails(lastInstantBeforeCurrentCompaction.get()).get();
|
||||
HoodieCommitMetadata metadata;
|
||||
try {
|
||||
metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(String.format("cannot read metadata from commit: %s", lastInstantBeforeCurrentCompaction.get()), e);
|
||||
}
|
||||
String internalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
|
||||
if (internalSchemaStr != null) {
|
||||
String existingSchemaStr = metadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY);
|
||||
return Pair.of(Option.of(internalSchemaStr), Option.of(existingSchemaStr));
|
||||
}
|
||||
}
|
||||
return Pair.of(Option.empty(), Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Give a schema versionId return its internalSchema.
|
||||
* This method will be called by spark tasks, we should minimize time cost.
|
||||
* We try our best to not use metaClient, since the initialization of metaClient is time cost
|
||||
* step1:
|
||||
* try to parser internalSchema from HoodieInstant directly
|
||||
* step2:
|
||||
* if we cannot parser internalSchema in step1,
|
||||
* try to find internalSchema in historySchema.
|
||||
*
|
||||
* @param versionId the internalSchema version to be search.
|
||||
* @param tablePath table path
|
||||
* @param hadoopConf conf
|
||||
* @param validCommits current validate commits, use to make up the commit file path/verify the validity of the history schema files
|
||||
* @return a internalSchema.
|
||||
*/
|
||||
public static InternalSchema getInternalSchemaByVersionId(long versionId, String tablePath, Configuration hadoopConf, String validCommits) {
|
||||
Set<String> commitSet = Arrays.stream(validCommits.split(",")).collect(Collectors.toSet());
|
||||
List<String> validateCommitList = commitSet.stream().map(fileName -> {
|
||||
String fileExtension = HoodieInstant.getTimelineFileExtension(fileName);
|
||||
return fileName.replace(fileExtension, "");
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
FileSystem fs = FSUtils.getFs(tablePath, hadoopConf);
|
||||
Path hoodieMetaPath = new Path(tablePath, HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
//step1:
|
||||
Path candidateCommitFile = commitSet.stream().filter(fileName -> {
|
||||
String fileExtension = HoodieInstant.getTimelineFileExtension(fileName);
|
||||
return fileName.replace(fileExtension, "").equals(versionId + "");
|
||||
}).findFirst().map(f -> new Path(hoodieMetaPath, f)).orElse(null);
|
||||
if (candidateCommitFile != null) {
|
||||
try {
|
||||
byte[] data;
|
||||
try (FSDataInputStream is = fs.open(candidateCommitFile)) {
|
||||
data = FileIOUtils.readAsByteArray(is);
|
||||
} catch (IOException e) {
|
||||
throw e;
|
||||
}
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
|
||||
String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA);
|
||||
if (latestInternalSchemaStr != null) {
|
||||
return SerDeHelper.fromJson(latestInternalSchemaStr).orElse(null);
|
||||
}
|
||||
} catch (Exception e1) {
|
||||
// swallow this exception.
|
||||
LOG.warn(String.format("Cannot find internal schema from commit file %s. Falling back to parsing historical internal schema", candidateCommitFile.toString()));
|
||||
}
|
||||
}
|
||||
// step2:
|
||||
FileBasedInternalSchemaStorageManager fileBasedInternalSchemaStorageManager = new FileBasedInternalSchemaStorageManager(hadoopConf, new Path(tablePath));
|
||||
String lastestHistorySchema = fileBasedInternalSchemaStorageManager.getHistorySchemaStrByGivenValidCommits(validateCommitList);
|
||||
return InternalSchemaUtils.searchSchema(versionId, SerDeHelper.parseSchemas(lastestHistorySchema));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,44 @@
|
||||
/*
|
||||
* 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.internal.schema;
|
||||
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Exception thrown for Hoodie schema convert failures. The root of the exception hierarchy.
|
||||
* Hoodie Write/Read clients will throw this exception if any of its operations fail. This is a runtime (unchecked)
|
||||
* exception.
|
||||
*/
|
||||
public class HoodieSchemaException extends HoodieException {
|
||||
public HoodieSchemaException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HoodieSchemaException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public HoodieSchemaException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public HoodieSchemaException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,291 @@
|
||||
/*
|
||||
* 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.internal.schema;
|
||||
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.internal.schema.Types.Field;
|
||||
import org.apache.hudi.internal.schema.Types.RecordType;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Internal schema for hudi table.
|
||||
* used to support schema evolution.
|
||||
*/
|
||||
public class InternalSchema implements Serializable {
|
||||
|
||||
private static final long DEFAULT_VERSION_ID = 0;
|
||||
|
||||
private final RecordType record;
|
||||
|
||||
private int maxColumnId;
|
||||
private long versionId;
|
||||
|
||||
private transient Map<Integer, Field> idToField = null;
|
||||
private transient Map<String, Integer> nameToId = null;
|
||||
private transient Map<Integer, String> idToName = null;
|
||||
|
||||
public static InternalSchema getEmptyInternalSchema() {
|
||||
return new InternalSchema(-1L, new ArrayList<>());
|
||||
}
|
||||
|
||||
public boolean isEmptySchema() {
|
||||
return versionId < 0;
|
||||
}
|
||||
|
||||
public InternalSchema(List<Field> columns) {
|
||||
this(DEFAULT_VERSION_ID, columns);
|
||||
}
|
||||
|
||||
public InternalSchema(Field... columns) {
|
||||
this(DEFAULT_VERSION_ID, Arrays.asList(columns));
|
||||
}
|
||||
|
||||
public InternalSchema(long versionId, List<Field> cols) {
|
||||
this.versionId = versionId;
|
||||
this.record = RecordType.get(cols);
|
||||
idToName = cols.isEmpty() ? new HashMap<>() : InternalSchemaBuilder.getBuilder().buildIdToName(record);
|
||||
nameToId = cols.isEmpty() ? new HashMap<>() : idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
|
||||
maxColumnId = idToName.isEmpty() ? -1 : idToName.keySet().stream().max(Comparator.comparing(Integer::valueOf)).get();
|
||||
}
|
||||
|
||||
public InternalSchema(long versionId, int maxColumnId, List<Field> cols) {
|
||||
this.maxColumnId = maxColumnId;
|
||||
this.versionId = versionId;
|
||||
this.record = RecordType.get(cols);
|
||||
buildIdToName();
|
||||
}
|
||||
|
||||
public InternalSchema(long versionId, int maxColumnId, Field... cols) {
|
||||
this(versionId, maxColumnId, Arrays.asList(cols));
|
||||
}
|
||||
|
||||
public RecordType getRecord() {
|
||||
return record;
|
||||
}
|
||||
|
||||
private Map<Integer, String> buildIdToName() {
|
||||
if (idToName == null) {
|
||||
idToName = InternalSchemaBuilder.getBuilder().buildIdToName(record);
|
||||
}
|
||||
return idToName;
|
||||
}
|
||||
|
||||
private Map<String, Integer> buildNameToId() {
|
||||
if (nameToId == null) {
|
||||
if (idToName != null && !idToName.isEmpty()) {
|
||||
nameToId = idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
|
||||
return nameToId;
|
||||
}
|
||||
nameToId = InternalSchemaBuilder.getBuilder().buildNameToId(record);
|
||||
}
|
||||
return nameToId;
|
||||
}
|
||||
|
||||
private Map<Integer, Field> buildIdToField() {
|
||||
if (idToField == null) {
|
||||
idToField = InternalSchemaBuilder.getBuilder().buildIdToField(record);
|
||||
}
|
||||
return idToField;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all columns full name.
|
||||
*/
|
||||
public List<String> getAllColsFullName() {
|
||||
if (nameToId == null) {
|
||||
nameToId = InternalSchemaBuilder.getBuilder().buildNameToId(record);
|
||||
}
|
||||
return Arrays.asList(nameToId.keySet().toArray(new String[0]));
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the version ID for this schema.
|
||||
*/
|
||||
public InternalSchema setSchemaId(long versionId) {
|
||||
this.versionId = versionId;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the version ID for this schema.
|
||||
*/
|
||||
public long schemaId() {
|
||||
return this.versionId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the version ID for this schema.
|
||||
*/
|
||||
public void setMaxColumnId(int maxColumnId) {
|
||||
this.maxColumnId = maxColumnId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the max column id for this schema.
|
||||
*/
|
||||
public int getMaxColumnId() {
|
||||
return this.maxColumnId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a List of the {@link Field columns} in this Schema.
|
||||
*/
|
||||
public List<Field> columns() {
|
||||
return record.fields();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link Type} of a sub-field identified by the field name.
|
||||
*
|
||||
* @param id a field id
|
||||
* @return fullName of field of
|
||||
*/
|
||||
public String findfullName(int id) {
|
||||
if (idToName == null) {
|
||||
buildIdToName();
|
||||
}
|
||||
String result = idToName.get(id);
|
||||
return result == null ? "" : result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link Type} of a sub-field identified by the field name.
|
||||
*
|
||||
* @param name a field name
|
||||
* @return a Type for the sub-field or null if it is not found
|
||||
*/
|
||||
public Type findType(String name) {
|
||||
if (name == null || name.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
Integer id = buildNameToId().get(name);
|
||||
if (id != null) { // name is found
|
||||
return findType(id);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link Type} of a sub-field identified by the field id.
|
||||
*
|
||||
* @param id a field id
|
||||
* @return a Type for the sub-field or null if it is not found
|
||||
*/
|
||||
public Type findType(int id) {
|
||||
Field field = buildIdToField().get(id);
|
||||
if (field != null) {
|
||||
return field.type();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all field ids
|
||||
*/
|
||||
public Set<Integer> getAllIds() {
|
||||
if (idToName == null) {
|
||||
buildIdToName();
|
||||
}
|
||||
return idToName.keySet();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the sub-field identified by the field id.
|
||||
*
|
||||
* @param id a field id
|
||||
* @return the sub-field or null if it is not found
|
||||
*/
|
||||
public Field findField(int id) {
|
||||
return buildIdToField().get(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a sub-field by name as a {@link Field}.
|
||||
* The result may be a top-level or a nested field.
|
||||
*
|
||||
* @param name a String name
|
||||
* @return a Type for the sub-field or null if it is not found
|
||||
*/
|
||||
public Field findField(String name) {
|
||||
if (name == null || name.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
Integer id = buildNameToId().get(name);
|
||||
if (id != null) {
|
||||
return buildIdToField().get(id);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether colName exists in current Schema.
|
||||
* Case insensitive.
|
||||
*
|
||||
* @param colName a colName
|
||||
* @return Whether colName exists in current Schema
|
||||
*/
|
||||
public boolean findDuplicateCol(String colName) {
|
||||
return idToName.entrySet().stream().map(e -> e.getValue().toLowerCase(Locale.ROOT))
|
||||
.collect(Collectors.toSet()).contains(colName);
|
||||
}
|
||||
|
||||
public int findIdByName(String name) {
|
||||
if (name == null || name.isEmpty()) {
|
||||
return -1;
|
||||
}
|
||||
return buildNameToId().getOrDefault(name, -1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("table {\n%s\n}",
|
||||
StringUtils.join(record.fields().stream()
|
||||
.map(f -> " " + f)
|
||||
.collect(Collectors.toList()).toArray(new String[0]), "\n"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof InternalSchema)) {
|
||||
return false;
|
||||
}
|
||||
InternalSchema that = (InternalSchema) o;
|
||||
if (versionId != that.schemaId()) {
|
||||
return false;
|
||||
}
|
||||
return record.equals(that.record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return record.hashCode();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,272 @@
|
||||
/*
|
||||
* 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.internal.schema;
|
||||
|
||||
import org.apache.hudi.internal.schema.visitor.InternalSchemaVisitor;
|
||||
import org.apache.hudi.internal.schema.visitor.NameToIDVisitor;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* A build class to help build fields for InternalSchema
|
||||
*/
|
||||
public class InternalSchemaBuilder implements Serializable {
|
||||
private static final InternalSchemaBuilder INSTANCE = new InternalSchemaBuilder();
|
||||
|
||||
public static InternalSchemaBuilder getBuilder() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
private InternalSchemaBuilder() {
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Build a mapping from id to full field name for a internal Type.
|
||||
* if a field y belong to a struct filed x, then the full name of y is x.y
|
||||
*
|
||||
* @param type hoodie internal type
|
||||
* @return a mapping from id to full field name
|
||||
*/
|
||||
public Map<Integer, String> buildIdToName(Type type) {
|
||||
Map<Integer, String> result = new HashMap<>();
|
||||
buildNameToId(type).forEach((k, v) -> result.put(v, k));
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a mapping from full field name to id for a internal Type.
|
||||
* if a field y belong to a struct filed x, then the full name of y is x.y
|
||||
*
|
||||
* @param type hoodie internal type
|
||||
* @return a mapping from full field name to id
|
||||
*/
|
||||
public Map<String, Integer> buildNameToId(Type type) {
|
||||
return visit(type, new NameToIDVisitor());
|
||||
}
|
||||
|
||||
/**
|
||||
* Use to traverse all types in internalSchema with visitor.
|
||||
*
|
||||
* @param schema hoodie internal schema
|
||||
* @return vistor expected result.
|
||||
*/
|
||||
public <T> T visit(InternalSchema schema, InternalSchemaVisitor<T> visitor) {
|
||||
return visitor.schema(schema, visit(schema.getRecord(), visitor));
|
||||
}
|
||||
|
||||
public <T> T visit(Type type, InternalSchemaVisitor<T> visitor) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<T> results = new ArrayList<>();
|
||||
for (Types.Field f : record.fields()) {
|
||||
visitor.beforeField(f);
|
||||
T result;
|
||||
try {
|
||||
result = visit(f.type(), visitor);
|
||||
} finally {
|
||||
visitor.afterField(f);
|
||||
}
|
||||
results.add(visitor.field(f, result));
|
||||
}
|
||||
return visitor.record(record, results);
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
T elementResult;
|
||||
Types.Field elementField = array.field(array.elementId());
|
||||
visitor.beforeArrayElement(elementField);
|
||||
try {
|
||||
elementResult = visit(elementField.type(), visitor);
|
||||
} finally {
|
||||
visitor.afterArrayElement(elementField);
|
||||
}
|
||||
return visitor.array(array, elementResult);
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
T keyResult;
|
||||
T valueResult;
|
||||
Types.Field keyField = map.field(map.keyId());
|
||||
visitor.beforeMapKey(keyField);
|
||||
try {
|
||||
keyResult = visit(map.keyType(), visitor);
|
||||
} finally {
|
||||
visitor.afterMapKey(keyField);
|
||||
}
|
||||
Types.Field valueField = map.field(map.valueId());
|
||||
visitor.beforeMapValue(valueField);
|
||||
try {
|
||||
valueResult = visit(map.valueType(), visitor);
|
||||
} finally {
|
||||
visitor.afterMapValue(valueField);
|
||||
}
|
||||
return visitor.map(map, keyResult, valueResult);
|
||||
default:
|
||||
return visitor.primitive((Type.PrimitiveType)type);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a mapping from id to field for a internal Type.
|
||||
*
|
||||
* @param type hoodie internal type
|
||||
* @return a mapping from id to field
|
||||
*/
|
||||
public Map<Integer, Types.Field> buildIdToField(Type type) {
|
||||
Map<Integer, Types.Field> idToField = new HashMap<>();
|
||||
visitIdToField(type, idToField);
|
||||
return idToField;
|
||||
}
|
||||
|
||||
private void visitIdToField(Type type, Map<Integer, Types.Field> index) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
for (Types.Field field : record.fields()) {
|
||||
visitIdToField(field.type(), index);
|
||||
index.put(field.fieldId(), field);
|
||||
}
|
||||
return;
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
visitIdToField(array.elementType(), index);
|
||||
for (Types.Field field : array.fields()) {
|
||||
index.put(field.fieldId(), field);
|
||||
}
|
||||
return;
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
visitIdToField(map.keyType(), index);
|
||||
visitIdToField(map.valueType(), index);
|
||||
for (Types.Field field : map.fields()) {
|
||||
index.put(field.fieldId(), field);
|
||||
}
|
||||
return;
|
||||
default:
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a mapping which maintain the relation between child field id and it's parent field id.
|
||||
* if a child field y(which id is 9) belong to a nest field x(which id is 6), then (9 -> 6) will be added to the result map.
|
||||
* if a field has no parent field, nothings will be added.
|
||||
*
|
||||
* @param record hoodie record type.
|
||||
* @return a mapping from id to parentId for a record Type
|
||||
*/
|
||||
public Map<Integer, Integer> index2Parents(Types.RecordType record) {
|
||||
Map<Integer, Integer> result = new HashMap<>();
|
||||
Deque<Integer> parentIds = new LinkedList<>();
|
||||
index2Parents(record, parentIds, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
private void index2Parents(Type type, Deque<Integer> pids, Map<Integer, Integer> id2p) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType)type;
|
||||
for (Types.Field f : record.fields()) {
|
||||
pids.push(f.fieldId());
|
||||
index2Parents(f.type(), pids, id2p);
|
||||
pids.pop();
|
||||
}
|
||||
|
||||
for (Types.Field f : record.fields()) {
|
||||
// root record has no parent id.
|
||||
if (!pids.isEmpty()) {
|
||||
Integer pid = pids.peek();
|
||||
id2p.put(f.fieldId(), pid);
|
||||
}
|
||||
}
|
||||
return;
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Types.Field elementField = array.field(array.elementId());
|
||||
pids.push(elementField.fieldId());
|
||||
index2Parents(elementField.type(), pids, id2p);
|
||||
pids.pop();
|
||||
id2p.put(array.elementId(), pids.peek());
|
||||
return;
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
Types.Field keyField = map.field(map.keyId());
|
||||
Types.Field valueField = map.field(map.valueId());
|
||||
// visit key
|
||||
pids.push(map.keyId());
|
||||
index2Parents(keyField.type(), pids, id2p);
|
||||
pids.pop();
|
||||
// visit value
|
||||
pids.push(map.valueId());
|
||||
index2Parents(valueField.type(), pids, id2p);
|
||||
pids.pop();
|
||||
id2p.put(map.keyId(), pids.peek());
|
||||
id2p.put(map.valueId(), pids.peek());
|
||||
return;
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Assigns new ids for all fields in a Type, based on initial id.
|
||||
*
|
||||
* @param type a type.
|
||||
* @param nextId initial id which used to fresh ids for all fields in a type
|
||||
* @return a new type with new ids
|
||||
*/
|
||||
public Type refreshNewId(Type type, AtomicInteger nextId) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Types.Field> oldFields = record.fields();
|
||||
int currentId = nextId.get();
|
||||
nextId.set(currentId + record.fields().size());
|
||||
List<Types.Field> internalFields = new ArrayList<>();
|
||||
for (int i = 0; i < oldFields.size(); i++) {
|
||||
Types.Field oldField = oldFields.get(i);
|
||||
Type fieldType = refreshNewId(oldField.type(), nextId);
|
||||
internalFields.add(Types.Field.get(currentId++, oldField.isOptional(), oldField.name(), fieldType, oldField.doc()));
|
||||
}
|
||||
return Types.RecordType.get(internalFields);
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
int elementId = nextId.get();
|
||||
nextId.set(elementId + 1);
|
||||
Type elementType = refreshNewId(array.elementType(), nextId);
|
||||
return Types.ArrayType.get(elementId, array.isElementOptional(), elementType);
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
int keyId = nextId.get();
|
||||
int valueId = keyId + 1;
|
||||
nextId.set(keyId + 2);
|
||||
Type keyType = refreshNewId(map.keyType(), nextId);
|
||||
Type valueType = refreshNewId(map.valueType(), nextId);
|
||||
return Types.MapType.get(keyId, valueId, keyType, valueType, map.isValueOptional());
|
||||
default:
|
||||
return type;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,78 @@
|
||||
/*
|
||||
* 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.internal.schema;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* The type of a schema, reference avro schema.
|
||||
* now avro version used by hoodie, not support localTime.
|
||||
* to do add support for localTime if avro version is updated
|
||||
*/
|
||||
public interface Type extends Serializable {
|
||||
enum TypeID {
|
||||
RECORD, ARRAY, MAP, FIXED, STRING, BINARY,
|
||||
INT, LONG, FLOAT, DOUBLE, DATE, BOOLEAN, TIME, TIMESTAMP, DECIMAL, UUID;
|
||||
private String name;
|
||||
TypeID() {
|
||||
this.name = this.name().toLowerCase(Locale.ROOT);
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
}
|
||||
|
||||
static TypeID fromValue(String value) {
|
||||
try {
|
||||
return TypeID.valueOf(value.toUpperCase(Locale.ROOT));
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new IllegalArgumentException(String.format("Invalid value of Type: %s", value));
|
||||
}
|
||||
}
|
||||
|
||||
TypeID typeId();
|
||||
|
||||
default boolean isNestedType() {
|
||||
return false;
|
||||
}
|
||||
|
||||
abstract class PrimitiveType implements Type {
|
||||
@Override
|
||||
public boolean isNestedType() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
abstract class NestedType implements Type {
|
||||
|
||||
@Override
|
||||
public boolean isNestedType() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public abstract List<Types.Field> fields();
|
||||
|
||||
public abstract Type fieldType(String name);
|
||||
|
||||
public abstract Types.Field field(int id);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,716 @@
|
||||
/*
|
||||
* 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.internal.schema;
|
||||
|
||||
import org.apache.hudi.internal.schema.Type.PrimitiveType;
|
||||
import org.apache.hudi.internal.schema.Type.NestedType;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class Types {
|
||||
private Types() {
|
||||
}
|
||||
|
||||
public static class BooleanType extends PrimitiveType {
|
||||
private static final BooleanType INSTANCE = new BooleanType();
|
||||
|
||||
public static BooleanType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return Type.TypeID.BOOLEAN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "boolean";
|
||||
}
|
||||
}
|
||||
|
||||
public static class IntType extends PrimitiveType {
|
||||
private static final IntType INSTANCE = new IntType();
|
||||
|
||||
public static IntType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.INT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "int";
|
||||
}
|
||||
}
|
||||
|
||||
public static class LongType extends PrimitiveType {
|
||||
private static final LongType INSTANCE = new LongType();
|
||||
|
||||
public static LongType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.LONG;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "long";
|
||||
}
|
||||
}
|
||||
|
||||
public static class FloatType extends PrimitiveType {
|
||||
private static final FloatType INSTANCE = new FloatType();
|
||||
|
||||
public static FloatType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.FLOAT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "float";
|
||||
}
|
||||
}
|
||||
|
||||
public static class DoubleType extends PrimitiveType {
|
||||
private static final DoubleType INSTANCE = new DoubleType();
|
||||
|
||||
public static DoubleType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.DOUBLE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "double";
|
||||
}
|
||||
}
|
||||
|
||||
public static class DateType extends PrimitiveType {
|
||||
private static final DateType INSTANCE = new DateType();
|
||||
|
||||
public static DateType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.DATE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "date";
|
||||
}
|
||||
}
|
||||
|
||||
public static class TimeType extends PrimitiveType {
|
||||
private static final TimeType INSTANCE = new TimeType();
|
||||
|
||||
public static TimeType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
private TimeType() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.TIME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "time";
|
||||
}
|
||||
}
|
||||
|
||||
public static class TimestampType extends PrimitiveType {
|
||||
private static final TimestampType INSTANCE = new TimestampType();
|
||||
|
||||
public static TimestampType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
private TimestampType() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.TIMESTAMP;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "timestamp";
|
||||
}
|
||||
}
|
||||
|
||||
public static class StringType extends PrimitiveType {
|
||||
private static final StringType INSTANCE = new StringType();
|
||||
|
||||
public static StringType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.STRING;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "string";
|
||||
}
|
||||
}
|
||||
|
||||
public static class BinaryType extends PrimitiveType {
|
||||
private static final BinaryType INSTANCE = new BinaryType();
|
||||
|
||||
public static BinaryType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.BINARY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "binary";
|
||||
}
|
||||
}
|
||||
|
||||
public static class FixedType extends PrimitiveType {
|
||||
public static FixedType getFixed(int size) {
|
||||
return new FixedType(size);
|
||||
}
|
||||
|
||||
private final int size;
|
||||
|
||||
private FixedType(int length) {
|
||||
this.size = length;
|
||||
}
|
||||
|
||||
public int getFixedSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.FIXED;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("fixed[%d]", size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof FixedType)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
FixedType fixedType = (FixedType) o;
|
||||
return size == fixedType.size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(FixedType.class, size);
|
||||
}
|
||||
}
|
||||
|
||||
public static class DecimalType extends PrimitiveType {
|
||||
public static DecimalType get(int precision, int scale) {
|
||||
return new DecimalType(precision, scale);
|
||||
}
|
||||
|
||||
private final int scale;
|
||||
private final int precision;
|
||||
|
||||
private DecimalType(int precision, int scale) {
|
||||
this.scale = scale;
|
||||
this.precision = precision;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether this DecimalType is wider than `other`. If yes, it means `other`
|
||||
* can be casted into `this` safely without losing any precision or range.
|
||||
*/
|
||||
public boolean isWiderThan(PrimitiveType other) {
|
||||
if (other instanceof DecimalType) {
|
||||
DecimalType dt = (DecimalType) other;
|
||||
return (precision - scale) >= (dt.precision - dt.scale) && scale > dt.scale;
|
||||
}
|
||||
if (other instanceof IntType) {
|
||||
return isWiderThan(get(10, 0));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether this DecimalType is tighter than `other`. If yes, it means `this`
|
||||
* can be casted into `other` safely without losing any precision or range.
|
||||
*/
|
||||
public boolean isTighterThan(PrimitiveType other) {
|
||||
if (other instanceof DecimalType) {
|
||||
DecimalType dt = (DecimalType) other;
|
||||
return (precision - scale) <= (dt.precision - dt.scale) && scale <= dt.scale;
|
||||
}
|
||||
if (other instanceof IntType) {
|
||||
return isTighterThan(get(10, 0));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public int scale() {
|
||||
return scale;
|
||||
}
|
||||
|
||||
public int precision() {
|
||||
return precision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.DECIMAL;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("decimal(%d, %d)", precision, scale);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof DecimalType)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DecimalType that = (DecimalType) o;
|
||||
if (scale != that.scale) {
|
||||
return false;
|
||||
}
|
||||
return precision == that.precision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(DecimalType.class, scale, precision);
|
||||
}
|
||||
}
|
||||
|
||||
public static class UUIDType extends PrimitiveType {
|
||||
private static final UUIDType INSTANCE = new UUIDType();
|
||||
|
||||
public static UUIDType get() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.UUID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "uuid";
|
||||
}
|
||||
}
|
||||
|
||||
/** A field within a record. */
|
||||
public static class Field implements Serializable {
|
||||
// Experimental method to support defaultValue
|
||||
public static Field get(int id, boolean isOptional, String name, Type type, String doc, Object defaultValue) {
|
||||
return new Field(isOptional, id, name, type, doc, defaultValue);
|
||||
}
|
||||
|
||||
public static Field get(int id, boolean isOptional, String name, Type type, String doc) {
|
||||
return new Field(isOptional, id, name, type, doc, null);
|
||||
}
|
||||
|
||||
public static Field get(int id, boolean isOptional, String name, Type type) {
|
||||
return new Field(isOptional, id, name, type, null, null);
|
||||
}
|
||||
|
||||
public static Field get(int id, String name, Type type) {
|
||||
return new Field(true, id, name, type, null, null);
|
||||
}
|
||||
|
||||
private final boolean isOptional;
|
||||
private final int id;
|
||||
private final String name;
|
||||
private final Type type;
|
||||
private final String doc;
|
||||
// Experimental properties
|
||||
private final Object defaultValue;
|
||||
|
||||
private Field(boolean isOptional, int id, String name, Type type, String doc, Object defaultValue) {
|
||||
this.isOptional = isOptional;
|
||||
this.id = id;
|
||||
this.name = name;
|
||||
this.type = type;
|
||||
this.doc = doc;
|
||||
this.defaultValue = defaultValue;
|
||||
}
|
||||
|
||||
public Object getDefaultValue() {
|
||||
return defaultValue;
|
||||
}
|
||||
|
||||
public boolean isOptional() {
|
||||
return isOptional;
|
||||
}
|
||||
|
||||
public int fieldId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public String name() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public String doc() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("%d: %s: %s %s",
|
||||
id, name, isOptional ? "optional" : "required", type) + (doc != null ? " (" + doc + ")" : "");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof Field)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
Field that = (Field) o;
|
||||
if (isOptional != that.isOptional) {
|
||||
return false;
|
||||
} else if (id != that.id) {
|
||||
return false;
|
||||
} else if (!name.equals(that.name)) {
|
||||
return false;
|
||||
} else if (!Objects.equals(doc, that.doc)) {
|
||||
return false;
|
||||
}
|
||||
return type.equals(that.type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(Field.class, id, isOptional, name, type);
|
||||
}
|
||||
}
|
||||
|
||||
public static class RecordType extends NestedType {
|
||||
|
||||
public static RecordType get(List<Field> fields) {
|
||||
return new RecordType(fields);
|
||||
}
|
||||
|
||||
public static RecordType get(Field... fields) {
|
||||
return new RecordType(Arrays.asList(fields));
|
||||
}
|
||||
|
||||
private final Field[] fields;
|
||||
|
||||
private transient Map<String, Field> nameToFields = null;
|
||||
private transient Map<Integer, Field> idToFields = null;
|
||||
|
||||
private RecordType(List<Field> fields) {
|
||||
this.fields = new Field[fields.size()];
|
||||
for (int i = 0; i < this.fields.length; i += 1) {
|
||||
this.fields[i] = fields.get(i);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Field> fields() {
|
||||
return Arrays.asList(fields);
|
||||
}
|
||||
|
||||
public Field field(String name) {
|
||||
if (nameToFields == null) {
|
||||
nameToFields = new HashMap<>();
|
||||
for (Field field : fields) {
|
||||
nameToFields.put(field.name().toLowerCase(Locale.ROOT), field);
|
||||
}
|
||||
}
|
||||
return nameToFields.get(name.toLowerCase(Locale.ROOT));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Field field(int id) {
|
||||
if (idToFields == null) {
|
||||
idToFields = new HashMap<>();
|
||||
for (Field field : fields) {
|
||||
idToFields.put(field.fieldId(), field);
|
||||
}
|
||||
}
|
||||
return idToFields.get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type fieldType(String name) {
|
||||
Field field = field(name);
|
||||
if (field != null) {
|
||||
return field.type();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.RECORD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Record<%s>", Arrays.stream(fields).map(f -> f.toString()).collect(Collectors.joining("-")));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof RecordType)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
RecordType that = (RecordType) o;
|
||||
return Arrays.equals(fields, that.fields);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(Field.class, Arrays.hashCode(fields));
|
||||
}
|
||||
}
|
||||
|
||||
public static class ArrayType extends NestedType {
|
||||
public static ArrayType get(int elementId, boolean isOptional, Type elementType) {
|
||||
return new ArrayType(Field.get(elementId, isOptional,"element", elementType));
|
||||
}
|
||||
|
||||
private final Field elementField;
|
||||
|
||||
private ArrayType(Field elementField) {
|
||||
this.elementField = elementField;
|
||||
}
|
||||
|
||||
public Type elementType() {
|
||||
return elementField.type();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type fieldType(String name) {
|
||||
if ("element".equals(name)) {
|
||||
return elementType();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Field field(int id) {
|
||||
if (elementField.fieldId() == id) {
|
||||
return elementField;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Field> fields() {
|
||||
return Arrays.asList(elementField);
|
||||
}
|
||||
|
||||
public int elementId() {
|
||||
return elementField.fieldId();
|
||||
}
|
||||
|
||||
public boolean isElementOptional() {
|
||||
return elementField.isOptional;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.ARRAY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("list<%s>", elementField.type());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof ArrayType)) {
|
||||
return false;
|
||||
}
|
||||
ArrayType listType = (ArrayType) o;
|
||||
return elementField.equals(listType.elementField);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(ArrayType.class, elementField);
|
||||
}
|
||||
}
|
||||
|
||||
public static class MapType extends NestedType {
|
||||
|
||||
public static MapType get(int keyId, int valueId, Type keyType, Type valueType) {
|
||||
return new MapType(
|
||||
Field.get(keyId, "key", keyType),
|
||||
Field.get(valueId, "value", valueType));
|
||||
}
|
||||
|
||||
public static MapType get(int keyId, int valueId, Type keyType, Type valueType, boolean isOptional) {
|
||||
return new MapType(
|
||||
Field.get(keyId, isOptional, "key", keyType),
|
||||
Field.get(valueId, isOptional, "value", valueType));
|
||||
}
|
||||
|
||||
private final Field keyField;
|
||||
private final Field valueField;
|
||||
private transient List<Field> fields = null;
|
||||
|
||||
private MapType(Field keyField, Field valueField) {
|
||||
this.keyField = keyField;
|
||||
this.valueField = valueField;
|
||||
}
|
||||
|
||||
public Type keyType() {
|
||||
return keyField.type();
|
||||
}
|
||||
|
||||
public Type valueType() {
|
||||
return valueField.type();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type fieldType(String name) {
|
||||
if ("key".equals(name)) {
|
||||
return keyField.type();
|
||||
} else if ("value".equals(name)) {
|
||||
return valueField.type();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Field field(int id) {
|
||||
if (keyField.fieldId() == id) {
|
||||
return keyField;
|
||||
} else if (valueField.fieldId() == id) {
|
||||
return valueField;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Field> fields() {
|
||||
if (fields == null) {
|
||||
fields = Arrays.asList(keyField, valueField);
|
||||
}
|
||||
return fields;
|
||||
}
|
||||
|
||||
public int keyId() {
|
||||
return keyField.fieldId();
|
||||
}
|
||||
|
||||
public int valueId() {
|
||||
return valueField.fieldId();
|
||||
}
|
||||
|
||||
public boolean isValueOptional() {
|
||||
return valueField.isOptional;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeID typeId() {
|
||||
return TypeID.MAP;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("map<%s, %s>", keyField.type(), valueField.type());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
} else if (!(o instanceof MapType)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
MapType mapType = (MapType) o;
|
||||
if (!keyField.equals(mapType.keyField)) {
|
||||
return false;
|
||||
}
|
||||
return valueField.equals(mapType.valueField);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(MapType.class, keyField, valueField);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,164 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.utils.SchemaChangeUtils;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
/**
|
||||
* Manage schema change for HoodieWriteClient.
|
||||
*/
|
||||
public class InternalSchemaChangeApplier {
|
||||
private InternalSchema latestSchema;
|
||||
|
||||
public InternalSchemaChangeApplier(InternalSchema latestSchema) {
|
||||
this.latestSchema = latestSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 colType 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 InternalSchema applyAddChange(
|
||||
String colName,
|
||||
Type colType,
|
||||
String doc,
|
||||
String position,
|
||||
TableChange.ColumnPositionChange.ColumnPositionType positionType) {
|
||||
TableChanges.ColumnAddChange add = TableChanges.ColumnAddChange.get(latestSchema);
|
||||
String parentName = TableChangesHelper.getParentName(colName);
|
||||
add.addColumns(parentName, colName, colType, doc);
|
||||
if (positionType != null) {
|
||||
switch (positionType) {
|
||||
case NO_OPERATION:
|
||||
break;
|
||||
case FIRST:
|
||||
add.addPositionChange(colName, "", positionType);
|
||||
break;
|
||||
case AFTER:
|
||||
case BEFORE:
|
||||
if (position == null || position.isEmpty()) {
|
||||
throw new IllegalArgumentException("position should not be null/empty_string when specify positionChangeType as after/before");
|
||||
}
|
||||
String referParentName = TableChangesHelper.getParentName(position);
|
||||
if (!parentName.equals(referParentName)) {
|
||||
throw new IllegalArgumentException("cannot reorder two columns which has different parent");
|
||||
}
|
||||
add.addPositionChange(colName, position, positionType);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", positionType));
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException(String.format("positionType should be specified"));
|
||||
}
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, add);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 InternalSchema applyDeleteChange(String... colNames) {
|
||||
TableChanges.ColumnDeleteChange delete = TableChanges.ColumnDeleteChange.get(latestSchema);
|
||||
Arrays.stream(colNames).forEach(colName -> delete.deleteColumn(colName));
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, delete);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 InternalSchema applyRenameChange(String colName, String newName) {
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
|
||||
updateChange.renameColumn(colName, newName);
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
|
||||
}
|
||||
|
||||
/**
|
||||
* Update col nullability 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 InternalSchema applyColumnNullabilityChange(String colName, boolean nullable) {
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
|
||||
updateChange.updateColumnNullability(colName, nullable);
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
|
||||
}
|
||||
|
||||
/**
|
||||
* Update col type 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 newType .
|
||||
*/
|
||||
public InternalSchema applyColumnTypeChange(String colName, Type newType) {
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
|
||||
updateChange.updateColumnType(colName, newType);
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 InternalSchema applyColumnCommentChange(String colName, String doc) {
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
|
||||
updateChange.updateColumnComment(colName, doc);
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 positionType col position change type. now support three change types: first/after/before
|
||||
*/
|
||||
public InternalSchema applyReOrderColPositionChange(
|
||||
String colName,
|
||||
String referColName,
|
||||
TableChange.ColumnPositionChange.ColumnPositionType positionType) {
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(latestSchema);
|
||||
String parentName = TableChangesHelper.getParentName(colName);
|
||||
String referParentName = TableChangesHelper.getParentName(referColName);
|
||||
if (positionType.equals(TableChange.ColumnPositionChange.ColumnPositionType.FIRST)) {
|
||||
updateChange.addPositionChange(colName, "", positionType);
|
||||
} else if (parentName.equals(referParentName)) {
|
||||
updateChange.addPositionChange(colName, referColName, positionType);
|
||||
} else {
|
||||
throw new IllegalArgumentException("cannot reorder two columns which has different parent");
|
||||
}
|
||||
return SchemaChangeUtils.applyTableChanges2Schema(latestSchema, updateChange);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,197 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Auxiliary class.
|
||||
* help to merge file schema and query schema to produce final read schema for avro/parquet file
|
||||
*/
|
||||
public class InternalSchemaMerger {
|
||||
private final InternalSchema fileSchema;
|
||||
private final InternalSchema querySchema;
|
||||
// now there exist some bugs when we use spark update/merge api,
|
||||
// those operation will change col nullability from optional to required which is wrong.
|
||||
// Before that bug is fixed, we need to do adapt.
|
||||
// if mergeRequiredFiledForce is true, we will ignore the col's required attribute.
|
||||
private final boolean ignoreRequiredAttribute;
|
||||
// Whether to use column Type from file schema to read files when we find some column type has changed.
|
||||
// spark parquetReader need the original column type to read data, otherwise the parquetReader will failed.
|
||||
// eg: current column type is StringType, now we changed it to decimalType,
|
||||
// we should not pass decimalType to parquetReader, we must pass StringType to it; when we read out the data, we convert data from String to Decimal, everything is ok.
|
||||
// for log reader
|
||||
// since our reWriteRecordWithNewSchema function support rewrite directly, so we no need this parameter
|
||||
// eg: current column type is StringType, now we changed it to decimalType,
|
||||
// we can pass decimalType to reWriteRecordWithNewSchema directly, everything is ok.
|
||||
private boolean useColumnTypeFromFileSchema = true;
|
||||
|
||||
public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) {
|
||||
this.fileSchema = fileSchema;
|
||||
this.querySchema = querySchema;
|
||||
this.ignoreRequiredAttribute = ignoreRequiredAttribute;
|
||||
this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create final read schema to read avro/parquet file.
|
||||
*
|
||||
* @return read schema to read avro/parquet file.
|
||||
*/
|
||||
public InternalSchema mergeSchema() {
|
||||
Types.RecordType record = (Types.RecordType) mergeType(querySchema.getRecord(), 0);
|
||||
return new InternalSchema(record.fields());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create final read schema to read avro/parquet file.
|
||||
* this is auxiliary function used by mergeSchema.
|
||||
*/
|
||||
private Type mergeType(Type type, int currentTypeId) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Type> newTypes = new ArrayList<>();
|
||||
for (Types.Field f : record.fields()) {
|
||||
Type newType = mergeType(f.type(), f.fieldId());
|
||||
newTypes.add(newType);
|
||||
}
|
||||
return Types.RecordType.get(buildRecordType(record.fields(), newTypes));
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Type newElementType;
|
||||
Types.Field elementField = array.fields().get(0);
|
||||
newElementType = mergeType(elementField.type(), elementField.fieldId());
|
||||
return buildArrayType(array, newElementType);
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
Type newValueType = mergeType(map.valueType(), map.valueId());
|
||||
return buildMapType(map, newValueType);
|
||||
default:
|
||||
return buildPrimitiveType((Type.PrimitiveType) type, currentTypeId);
|
||||
}
|
||||
}
|
||||
|
||||
private List<Types.Field> buildRecordType(List<Types.Field> oldFields, List<Type> newTypes) {
|
||||
List<Types.Field> newFields = new ArrayList<>();
|
||||
for (int i = 0; i < newTypes.size(); i++) {
|
||||
Type newType = newTypes.get(i);
|
||||
Types.Field oldField = oldFields.get(i);
|
||||
int fieldId = oldField.fieldId();
|
||||
String fullName = querySchema.findfullName(fieldId);
|
||||
if (fileSchema.findField(fieldId) != null) {
|
||||
if (fileSchema.findfullName(fieldId).equals(fullName)) {
|
||||
// maybe col type changed, deal with it.
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
|
||||
} else {
|
||||
// find rename, deal with it.
|
||||
newFields.add(dealWithRename(fieldId, newType, oldField));
|
||||
}
|
||||
} else {
|
||||
// buildFullName
|
||||
fullName = normalizeFullName(fullName);
|
||||
if (fileSchema.findField(fullName) != null) {
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name() + "suffix", oldField.type(), oldField.doc()));
|
||||
} else {
|
||||
// find add column
|
||||
// now there exist some bugs when we use spark update/merge api, those operation will change col optional to required.
|
||||
if (ignoreRequiredAttribute) {
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), true, oldField.name(), newType, oldField.doc()));
|
||||
} else {
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return newFields;
|
||||
}
|
||||
|
||||
private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldField) {
|
||||
Types.Field fieldFromFileSchema = fileSchema.findField(fieldId);
|
||||
String nameFromFileSchema = fieldFromFileSchema.name();
|
||||
Type typeFromFileSchema = fieldFromFileSchema.type();
|
||||
// Current design mechanism guarantees nestedType change is not allowed, so no need to consider.
|
||||
if (newType.isNestedType()) {
|
||||
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, newType, oldField.doc());
|
||||
} else {
|
||||
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc());
|
||||
}
|
||||
}
|
||||
|
||||
private String normalizeFullName(String fullName) {
|
||||
// find parent rename, and normalize fullName
|
||||
// eg: we renamed a nest field struct(c, d) to aa, the we delete a.d and add it back later.
|
||||
String[] nameParts = fullName.split("\\.");
|
||||
String[] normalizedNameParts = new String[nameParts.length];
|
||||
System.arraycopy(nameParts, 0, normalizedNameParts, 0, nameParts.length);
|
||||
for (int j = 0; j < nameParts.length - 1; j++) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (int k = 0; k <= j; k++) {
|
||||
sb.append(nameParts[k]);
|
||||
}
|
||||
String parentName = sb.toString();
|
||||
int parentFieldIdFromQuerySchema = querySchema.findIdByName(parentName);
|
||||
String parentNameFromFileSchema = fileSchema.findfullName(parentFieldIdFromQuerySchema);
|
||||
if (parentNameFromFileSchema.isEmpty()) {
|
||||
break;
|
||||
}
|
||||
if (!parentNameFromFileSchema.equalsIgnoreCase(parentName)) {
|
||||
// find parent rename, update nameParts
|
||||
String[] parentNameParts = parentNameFromFileSchema.split("\\.");
|
||||
System.arraycopy(parentNameParts, 0, normalizedNameParts, 0, parentNameParts.length);
|
||||
}
|
||||
}
|
||||
return StringUtils.join(normalizedNameParts, ".");
|
||||
}
|
||||
|
||||
private Type buildArrayType(Types.ArrayType array, Type newType) {
|
||||
Types.Field elementField = array.fields().get(0);
|
||||
int elementId = elementField.fieldId();
|
||||
if (elementField.type() == newType) {
|
||||
return array;
|
||||
} else {
|
||||
return Types.ArrayType.get(elementId, elementField.isOptional(), newType);
|
||||
}
|
||||
}
|
||||
|
||||
private Type buildMapType(Types.MapType map, Type newValue) {
|
||||
Types.Field valueFiled = map.fields().get(1);
|
||||
if (valueFiled.type() == newValue) {
|
||||
return map;
|
||||
} else {
|
||||
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValue, map.isValueOptional());
|
||||
}
|
||||
}
|
||||
|
||||
private Type buildPrimitiveType(Type.PrimitiveType typeFromQuerySchema, int currentPrimitiveTypeId) {
|
||||
Type typeFromFileSchema = fileSchema.findType(currentPrimitiveTypeId);
|
||||
if (typeFromFileSchema == null) {
|
||||
return typeFromQuerySchema;
|
||||
} else {
|
||||
return useColumnTypeFromFileSchema ? typeFromFileSchema : typeFromQuerySchema;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,252 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.internal.schema.HoodieSchemaException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* TableChange subclasses represent requested changes to a table.
|
||||
* now only column changes support.
|
||||
* to do support partition changes
|
||||
*/
|
||||
public interface TableChange {
|
||||
/* The action Type of schema change. */
|
||||
enum ColumnChangeID {
|
||||
ADD, UPDATE, DELETE, PROPERTY_CHANGE, REPLACE;
|
||||
private String name;
|
||||
|
||||
private ColumnChangeID() {
|
||||
this.name = this.name().toLowerCase(Locale.ROOT);
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
}
|
||||
|
||||
static ColumnChangeID fromValue(String value) {
|
||||
switch (value.toLowerCase(Locale.ROOT)) {
|
||||
case "add":
|
||||
return ColumnChangeID.ADD;
|
||||
case "change":
|
||||
return ColumnChangeID.UPDATE;
|
||||
case "delete":
|
||||
return ColumnChangeID.DELETE;
|
||||
case "property":
|
||||
return ColumnChangeID.PROPERTY_CHANGE;
|
||||
case "replace":
|
||||
return ColumnChangeID.REPLACE;
|
||||
default:
|
||||
throw new IllegalArgumentException("Invalid value of Type.");
|
||||
}
|
||||
}
|
||||
|
||||
ColumnChangeID columnChangeId();
|
||||
|
||||
default boolean withPositionChange() {
|
||||
return false;
|
||||
}
|
||||
|
||||
abstract class BaseColumnChange implements TableChange {
|
||||
protected final InternalSchema internalSchema;
|
||||
protected final Map<Integer, Integer> id2parent;
|
||||
protected final Map<Integer, ArrayList<ColumnPositionChange>> positionChangeMap = new HashMap<>();
|
||||
|
||||
BaseColumnChange(InternalSchema schema) {
|
||||
this.internalSchema = schema;
|
||||
this.id2parent = InternalSchemaBuilder.getBuilder().index2Parents(schema.getRecord());
|
||||
}
|
||||
|
||||
/**
|
||||
* Add position change.
|
||||
*
|
||||
* @param srcName column which need to be reordered
|
||||
* @param dsrName reference position
|
||||
* @param orderType change types
|
||||
* @return this
|
||||
*/
|
||||
public BaseColumnChange addPositionChange(String srcName, String dsrName, ColumnPositionChange.ColumnPositionType orderType) {
|
||||
Integer srcId = findIdByFullName(srcName);
|
||||
Option<Integer> dsrIdOpt = dsrName.isEmpty() ? Option.empty() : Option.of(findIdByFullName(dsrName));
|
||||
Integer srcParentId = id2parent.get(srcId);
|
||||
Option<Integer> dsrParentIdOpt = dsrIdOpt.map(id2parent::get);
|
||||
// forbid adjust hoodie metadata columns.
|
||||
switch (orderType) {
|
||||
case BEFORE:
|
||||
checkColModifyIsLegal(dsrName);
|
||||
break;
|
||||
case FIRST:
|
||||
if (srcId == null || srcId == -1 || srcParentId == null || srcParentId == -1) {
|
||||
throw new HoodieSchemaException("forbid adjust top-level columns position by using through first syntax");
|
||||
}
|
||||
break;
|
||||
case AFTER:
|
||||
List<String> checkColumns = HoodieRecord.HOODIE_META_COLUMNS.subList(0, HoodieRecord.HOODIE_META_COLUMNS.size() - 2);
|
||||
if (checkColumns.stream().anyMatch(f -> f.equalsIgnoreCase(dsrName))) {
|
||||
throw new HoodieSchemaException("forbid adjust the position of ordinary columns between meta columns");
|
||||
}
|
||||
break;
|
||||
case NO_OPERATION:
|
||||
default:
|
||||
break;
|
||||
}
|
||||
int parentId;
|
||||
if (srcParentId != null && dsrParentIdOpt.isPresent() && srcParentId.equals(dsrParentIdOpt.get())) {
|
||||
Types.Field parentField = internalSchema.findField(srcParentId);
|
||||
if (!(parentField.type() instanceof Types.RecordType)) {
|
||||
throw new HoodieSchemaException(String.format("only support reorder fields in struct type, but find: %s", parentField.type()));
|
||||
}
|
||||
parentId = parentField.fieldId();
|
||||
} else if (srcParentId == null && !dsrParentIdOpt.isPresent()) {
|
||||
parentId = -1;
|
||||
} else if (srcParentId != null && !dsrParentIdOpt.isPresent() && orderType.equals(ColumnPositionChange.ColumnPositionType.FIRST)) {
|
||||
parentId = srcParentId;
|
||||
} else {
|
||||
throw new HoodieSchemaException("cannot order position from different parent");
|
||||
}
|
||||
|
||||
ArrayList<ColumnPositionChange> changes = positionChangeMap.getOrDefault(parentId, new ArrayList<>());
|
||||
changes.add(ColumnPositionChange.get(srcId, dsrIdOpt.orElse(-1), orderType));
|
||||
positionChangeMap.put(parentId, changes);
|
||||
return this;
|
||||
}
|
||||
|
||||
public BaseColumnChange addPositionChange(String srcName, String dsrName, String orderType) {
|
||||
return addPositionChange(srcName, dsrName, ColumnPositionChange.fromTypeValue(orderType));
|
||||
}
|
||||
|
||||
/**
|
||||
* Abstract method.
|
||||
* give a column fullName and return the field id
|
||||
*
|
||||
* @param fullName column fullName
|
||||
* @return field id of current column
|
||||
*/
|
||||
protected abstract Integer findIdByFullName(String fullName);
|
||||
|
||||
// Modify hudi meta columns is prohibited
|
||||
protected void checkColModifyIsLegal(String colNeedToModfiy) {
|
||||
if (HoodieRecord.HOODIE_META_COLUMNS.stream().anyMatch(f -> f.equalsIgnoreCase(colNeedToModfiy))) {
|
||||
throw new IllegalArgumentException(String.format("cannot modify hudi meta col: %s", colNeedToModfiy));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean withPositionChange() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Column position change.
|
||||
* now support three change types: FIRST/AFTER/BEFORE
|
||||
* FIRST means the specified column should be the first column.
|
||||
* AFTER means the specified column should be put after the given column.
|
||||
* BEFORE means the specified column should be put before the given column.
|
||||
* Note that, the specified column may be a nested field:
|
||||
* AFTER/BEFORE means the given columns should in the same struct;
|
||||
* FIRST means this field should be the first one within the struct.
|
||||
*/
|
||||
class ColumnPositionChange {
|
||||
public enum ColumnPositionType {
|
||||
FIRST,
|
||||
BEFORE,
|
||||
AFTER,
|
||||
// only expose to internal use.
|
||||
NO_OPERATION
|
||||
}
|
||||
|
||||
static ColumnPositionType fromTypeValue(String value) {
|
||||
switch (value.toLowerCase(Locale.ROOT)) {
|
||||
case "first":
|
||||
return ColumnPositionType.FIRST;
|
||||
case "before":
|
||||
return ColumnPositionType.BEFORE;
|
||||
case "after":
|
||||
return ColumnPositionType.AFTER;
|
||||
case "no_operation":
|
||||
return ColumnPositionType.NO_OPERATION;
|
||||
default:
|
||||
throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", value));
|
||||
}
|
||||
}
|
||||
|
||||
private final int srcId;
|
||||
private final int dsrId;
|
||||
private final ColumnPositionType type;
|
||||
|
||||
static ColumnPositionChange first(int srcId) {
|
||||
return new ColumnPositionChange(srcId, -1, ColumnPositionType.FIRST);
|
||||
}
|
||||
|
||||
static ColumnPositionChange before(int srcId, int dsrId) {
|
||||
return new ColumnPositionChange(srcId, dsrId, ColumnPositionType.BEFORE);
|
||||
}
|
||||
|
||||
static ColumnPositionChange after(int srcId, int dsrId) {
|
||||
return new ColumnPositionChange(srcId, dsrId, ColumnPositionType.AFTER);
|
||||
}
|
||||
|
||||
static ColumnPositionChange get(int srcId, int dsrId, String type) {
|
||||
return get(srcId, dsrId, fromTypeValue(type));
|
||||
}
|
||||
|
||||
static ColumnPositionChange get(int srcId, int dsrId, ColumnPositionType type) {
|
||||
switch (type) {
|
||||
case FIRST:
|
||||
return ColumnPositionChange.first(srcId);
|
||||
case BEFORE:
|
||||
return ColumnPositionChange.before(srcId, dsrId);
|
||||
case AFTER:
|
||||
return ColumnPositionChange.after(srcId, dsrId);
|
||||
default:
|
||||
throw new IllegalArgumentException(String.format("only support first/before/after but found: %s", type));
|
||||
}
|
||||
}
|
||||
|
||||
private ColumnPositionChange(int srcId, int dsrId, ColumnPositionType type) {
|
||||
this.srcId = srcId;
|
||||
this.dsrId = dsrId;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public int getSrcId() {
|
||||
return srcId;
|
||||
}
|
||||
|
||||
public int getDsrId() {
|
||||
return dsrId;
|
||||
}
|
||||
|
||||
public ColumnPositionType type() {
|
||||
return type;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,398 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.internal.schema.HoodieSchemaException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
import org.apache.hudi.internal.schema.utils.SchemaChangeUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class TableChanges {
|
||||
|
||||
/** Deal with update columns changes for table. */
|
||||
public static class ColumnUpdateChange extends TableChange.BaseColumnChange {
|
||||
private final Map<Integer, Types.Field> updates = new HashMap<>();
|
||||
|
||||
public static ColumnUpdateChange get(InternalSchema schema) {
|
||||
return new ColumnUpdateChange(schema);
|
||||
}
|
||||
|
||||
private ColumnUpdateChange(InternalSchema schema) {
|
||||
super(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean withPositionChange() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public Type applyUpdates(Types.Field oldField, Type type) {
|
||||
Types.Field update = updates.get(oldField.fieldId());
|
||||
if (update != null && update.type() != oldField.type()) {
|
||||
return update.type();
|
||||
}
|
||||
//
|
||||
ArrayList<ColumnPositionChange> pchanges = positionChangeMap.getOrDefault(oldField.fieldId(), new ArrayList<>());
|
||||
if (!pchanges.isEmpty()) {
|
||||
// when we build ColumnAddChange,we have already done some check, so it's safe to convert newType to RecordType
|
||||
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(((Types.RecordType) type).fields(), new ArrayList<>(), pchanges);
|
||||
return Types.RecordType.get(newFields);
|
||||
}
|
||||
return type;
|
||||
}
|
||||
|
||||
public Map<Integer, Types.Field> getUpdates() {
|
||||
return updates;
|
||||
}
|
||||
|
||||
/**
|
||||
* Update a column in the schema to a new type.
|
||||
* only support update primitive type.
|
||||
* Only updates that widen types are allowed.
|
||||
*
|
||||
* @param name name of the column to update
|
||||
* @param newType new type for the column
|
||||
* @return this
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public ColumnUpdateChange updateColumnType(String name, Type newType) {
|
||||
checkColModifyIsLegal(name);
|
||||
if (newType.isNestedType()) {
|
||||
throw new IllegalArgumentException(String.format("only support update primitive type but find nest column: %s", name));
|
||||
}
|
||||
Types.Field field = internalSchema.findField(name);
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
|
||||
}
|
||||
|
||||
if (!SchemaChangeUtils.isTypeUpdateAllow(field.type(), newType)) {
|
||||
throw new IllegalArgumentException(String.format("cannot update origin type: %s to a incompatibility type: %s", field.type(), newType));
|
||||
}
|
||||
|
||||
if (field.type().equals(newType)) {
|
||||
// do nothings
|
||||
return this;
|
||||
}
|
||||
// save update info
|
||||
Types.Field update = updates.get(field.fieldId());
|
||||
if (update == null) {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), field.name(), newType, field.doc()));
|
||||
} else {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), update.name(), newType, update.doc()));
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Update a column doc in the schema to a new primitive type.
|
||||
*
|
||||
* @param name name of the column to update
|
||||
* @param newDoc new documentation for the column
|
||||
* @return this
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public ColumnUpdateChange updateColumnComment(String name, String newDoc) {
|
||||
checkColModifyIsLegal(name);
|
||||
Types.Field field = internalSchema.findField(name);
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
|
||||
}
|
||||
// consider null
|
||||
if (Objects.equals(field.doc(), newDoc)) {
|
||||
// do nothings
|
||||
return this;
|
||||
}
|
||||
// save update info
|
||||
Types.Field update = updates.get(field.fieldId());
|
||||
if (update == null) {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), field.name(), field.type(), newDoc));
|
||||
} else {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), update.name(), update.type(), newDoc));
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Rename a column in the schema.
|
||||
*
|
||||
* @param name name of the column to rename
|
||||
* @param newName new name for the column
|
||||
* @return this
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public ColumnUpdateChange renameColumn(String name, String newName) {
|
||||
checkColModifyIsLegal(name);
|
||||
Types.Field field = internalSchema.findField(name);
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
|
||||
}
|
||||
if (newName == null || newName.isEmpty()) {
|
||||
throw new IllegalArgumentException(String.format("cannot rename column: %s to empty", name));
|
||||
}
|
||||
// keep consisitent with hive. column names insensitive, so we check 'newName.toLowerCase(Locale.ROOT)'
|
||||
if (internalSchema.findDuplicateCol(newName.toLowerCase(Locale.ROOT))) {
|
||||
throw new IllegalArgumentException(String.format("cannot rename column: %s to a existing name", name));
|
||||
}
|
||||
// save update info
|
||||
Types.Field update = updates.get(field.fieldId());
|
||||
if (update == null) {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), field.isOptional(), newName, field.type(), field.doc()));
|
||||
} else {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), update.isOptional(), newName, update.type(), update.doc()));
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Update nullable for column.
|
||||
* only support required type -> optional type
|
||||
*
|
||||
* @param name name of the column to update
|
||||
* @param nullable nullable for updated name
|
||||
* @return this
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public ColumnUpdateChange updateColumnNullability(String name, boolean nullable) {
|
||||
return updateColumnNullability(name, nullable, false);
|
||||
}
|
||||
|
||||
public ColumnUpdateChange updateColumnNullability(String name, boolean nullable, boolean force) {
|
||||
checkColModifyIsLegal(name);
|
||||
Types.Field field = internalSchema.findField(name);
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot update a missing column: %s", name));
|
||||
}
|
||||
if (field.isOptional() == nullable) {
|
||||
// do nothings
|
||||
return this;
|
||||
}
|
||||
if (field.isOptional() && !nullable && !force) {
|
||||
throw new IllegalArgumentException("cannot update column Nullability: optional to required");
|
||||
}
|
||||
// save update info
|
||||
Types.Field update = updates.get(field.fieldId());
|
||||
if (update == null) {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), nullable, field.name(), field.type(), field.doc()));
|
||||
} else {
|
||||
updates.put(field.fieldId(), Types.Field.get(field.fieldId(), nullable, update.name(), update.type(), update.doc()));
|
||||
}
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
public Map<Integer, ArrayList<ColumnPositionChange>> getPositionChangeMap() {
|
||||
return positionChangeMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnChangeID columnChangeId() {
|
||||
return ColumnChangeID.UPDATE;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Integer findIdByFullName(String fullName) {
|
||||
Types.Field field = internalSchema.findField(fullName);
|
||||
if (field != null) {
|
||||
return field.fieldId();
|
||||
} else {
|
||||
throw new IllegalArgumentException(String.format("cannot find col id for given column fullName: %s", fullName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Deal with delete columns changes for table. */
|
||||
public static class ColumnDeleteChange extends TableChange.BaseColumnChange {
|
||||
private final Set deletes = new HashSet<>();
|
||||
|
||||
@Override
|
||||
public ColumnChangeID columnChangeId() {
|
||||
return ColumnChangeID.DELETE;
|
||||
}
|
||||
|
||||
public static ColumnDeleteChange get(InternalSchema schema) {
|
||||
return new ColumnDeleteChange(schema);
|
||||
}
|
||||
|
||||
private ColumnDeleteChange(InternalSchema schema) {
|
||||
super(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean withPositionChange() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BaseColumnChange addPositionChange(String srcId, String dsrId, String orderType) {
|
||||
throw new UnsupportedOperationException("no support add position change for ColumnDeleteChange");
|
||||
}
|
||||
|
||||
public ColumnDeleteChange deleteColumn(String name) {
|
||||
checkColModifyIsLegal(name);
|
||||
Types.Field field = internalSchema.findField(name);
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot delete missing columns: %s", name));
|
||||
}
|
||||
deletes.add(field.fieldId());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Type applyDelete(int id, Type type) {
|
||||
if (deletes.contains(id)) {
|
||||
return null;
|
||||
}
|
||||
return type;
|
||||
}
|
||||
|
||||
public Set<Integer> getDeletes() {
|
||||
return deletes;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Integer findIdByFullName(String fullName) {
|
||||
throw new UnsupportedOperationException("delete change cannot support this method");
|
||||
}
|
||||
}
|
||||
|
||||
/** Deal with add columns changes for table. */
|
||||
public static class ColumnAddChange extends TableChange.BaseColumnChange {
|
||||
private final Map<String, Integer> fullColName2Id = new HashMap<>();
|
||||
private final Map<Integer, ArrayList<Types.Field>> parentId2AddCols = new HashMap<>();
|
||||
private int nextId;
|
||||
|
||||
public static ColumnAddChange get(InternalSchema internalSchema) {
|
||||
return new ColumnAddChange(internalSchema);
|
||||
}
|
||||
|
||||
public Type applyAdd(Types.Field orignalField, Type type) {
|
||||
int fieldId = orignalField.fieldId();
|
||||
ArrayList<Types.Field> addFields = parentId2AddCols.getOrDefault(fieldId, new ArrayList<>());
|
||||
ArrayList<ColumnPositionChange> pchanges = positionChangeMap.getOrDefault(fieldId, new ArrayList<>());
|
||||
|
||||
if (!addFields.isEmpty() || !pchanges.isEmpty()) {
|
||||
// when we build ColumnAddChange,we have already done some check, so it's safe to convert newType to RecordType
|
||||
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(((Types.RecordType) type).fields(), addFields, pchanges);
|
||||
return Types.RecordType.get(newFields);
|
||||
}
|
||||
return type;
|
||||
}
|
||||
|
||||
public ColumnAddChange addColumns(String name, Type type, String doc) {
|
||||
checkColModifyIsLegal(name);
|
||||
return addColumns("", name, type, doc);
|
||||
}
|
||||
|
||||
public ColumnAddChange addColumns(String parent, String name, Type type, String doc) {
|
||||
checkColModifyIsLegal(name);
|
||||
addColumnsInternal(parent, name, type, doc);
|
||||
return this;
|
||||
}
|
||||
|
||||
private void addColumnsInternal(String parent, String name, Type type, String doc) {
|
||||
// root record has no parent, so set parentId to -1 as default
|
||||
int parentId = -1;
|
||||
// do check
|
||||
String fullName = name;
|
||||
if (!parent.isEmpty()) {
|
||||
Types.Field parentField = internalSchema.findField(parent);
|
||||
if (parentField == null) {
|
||||
throw new HoodieSchemaException(String.format("cannot add column: %s which parent: %s is not exist", name, parent));
|
||||
}
|
||||
Type parentType = parentField.type();
|
||||
if (!(parentField.type() instanceof Types.RecordType)) {
|
||||
throw new HoodieSchemaException("only support add nested columns to struct column");
|
||||
}
|
||||
parentId = parentField.fieldId();
|
||||
Types.Field newParentField = internalSchema.findField(parent + "." + name);
|
||||
if (newParentField != null) {
|
||||
throw new HoodieSchemaException(String.format("cannot add column: %s which already exist", name));
|
||||
}
|
||||
fullName = parent + "." + name;
|
||||
} else {
|
||||
// keep consistent with hive, column name case insensitive
|
||||
if (internalSchema.findDuplicateCol(name.toLowerCase(Locale.ROOT))) {
|
||||
throw new HoodieSchemaException(String.format("cannot add column: %s which already exist", name));
|
||||
}
|
||||
}
|
||||
if (fullColName2Id.containsKey(fullName)) {
|
||||
throw new HoodieSchemaException(String.format("cannot repeat add column: %s", name));
|
||||
}
|
||||
fullColName2Id.put(fullName, nextId);
|
||||
if (parentId != -1) {
|
||||
id2parent.put(nextId, parentId);
|
||||
}
|
||||
AtomicInteger assignNextId = new AtomicInteger(nextId + 1);
|
||||
Type typeWithNewId = InternalSchemaBuilder.getBuilder().refreshNewId(type, assignNextId);
|
||||
// only allow add optional columns.
|
||||
ArrayList<Types.Field> adds = parentId2AddCols.getOrDefault(parentId, new ArrayList<>());
|
||||
adds.add(Types.Field.get(nextId, true, name, typeWithNewId, doc));
|
||||
parentId2AddCols.put(parentId, adds);
|
||||
nextId = assignNextId.get();
|
||||
}
|
||||
|
||||
private ColumnAddChange(InternalSchema internalSchema) {
|
||||
super(internalSchema);
|
||||
this.nextId = internalSchema.getMaxColumnId() + 1;
|
||||
}
|
||||
|
||||
public Map<Integer, ArrayList<Types.Field>> getParentId2AddCols() {
|
||||
return parentId2AddCols;
|
||||
}
|
||||
|
||||
public Map<Integer, ArrayList<ColumnPositionChange>> getPositionChangeMap() {
|
||||
return positionChangeMap;
|
||||
}
|
||||
|
||||
// expose to test
|
||||
public Map<String, Integer> getFullColName2Id() {
|
||||
return fullColName2Id;
|
||||
}
|
||||
|
||||
protected Integer findIdByFullName(String fullName) {
|
||||
Types.Field field = internalSchema.findField(fullName);
|
||||
if (field != null) {
|
||||
return field.fieldId();
|
||||
}
|
||||
return fullColName2Id.getOrDefault(fullName, -1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnChangeID columnChangeId() {
|
||||
return ColumnChangeID.ADD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean withPositionChange() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Helper class to support Table schema changes.
|
||||
*/
|
||||
public class TableChangesHelper {
|
||||
/**
|
||||
* Apply add operation and column position change operation.
|
||||
*
|
||||
* @param fields origin column fields.
|
||||
* @param adds column fields to be added.
|
||||
* @param pchanges a wrapper class hold all the position change operations.
|
||||
* @return column fields after adjusting the position.
|
||||
*/
|
||||
public static List<Types.Field> applyAddChange2Fields(List<Types.Field> fields, ArrayList<Types.Field> adds, ArrayList<TableChange.ColumnPositionChange> pchanges) {
|
||||
if (adds == null && pchanges == null) {
|
||||
return fields;
|
||||
}
|
||||
LinkedList<Types.Field> result = new LinkedList<>(fields);
|
||||
// apply add columns
|
||||
if (adds != null && !adds.isEmpty()) {
|
||||
result.addAll(adds);
|
||||
}
|
||||
// apply position change
|
||||
if (pchanges != null && !pchanges.isEmpty()) {
|
||||
for (TableChange.ColumnPositionChange pchange : pchanges) {
|
||||
Types.Field srcField = result.stream().filter(f -> f.fieldId() == pchange.getSrcId()).findFirst().get();
|
||||
Types.Field dsrField = result.stream().filter(f -> f.fieldId() == pchange.getDsrId()).findFirst().orElse(null);
|
||||
// we remove srcField first
|
||||
result.remove(srcField);
|
||||
switch (pchange.type()) {
|
||||
case AFTER:
|
||||
// add srcField after dsrField
|
||||
result.add(result.indexOf(dsrField) + 1, srcField);
|
||||
break;
|
||||
case BEFORE:
|
||||
// add srcField before dsrField
|
||||
result.add(result.indexOf(dsrField), srcField);
|
||||
break;
|
||||
case FIRST:
|
||||
result.addFirst(srcField);
|
||||
break;
|
||||
default:
|
||||
// should not reach here
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public static String getParentName(String fullColName) {
|
||||
int offset = fullColName.lastIndexOf(".");
|
||||
return offset > 0 ? fullColName.substring(0, offset) : "";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,436 @@
|
||||
/*
|
||||
* 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.internal.schema.convert;
|
||||
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.LogicalType;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.internal.schema.HoodieSchemaException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.apache.avro.Schema.Type.UNION;
|
||||
|
||||
/**
|
||||
* Auxiliary class.
|
||||
* Converts an avro schema into InternalSchema, or convert InternalSchema to an avro schema
|
||||
*/
|
||||
public class AvroInternalSchemaConverter {
|
||||
|
||||
/**
|
||||
* Convert internalSchema to avro Schema.
|
||||
*
|
||||
* @param internalSchema internal schema.
|
||||
* @param tableName the record name.
|
||||
* @return an avro Schema.
|
||||
*/
|
||||
public static Schema convert(InternalSchema internalSchema, String tableName) {
|
||||
return buildAvroSchemaFromInternalSchema(internalSchema, tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert RecordType to avro Schema.
|
||||
*
|
||||
* @param type internal schema.
|
||||
* @param name the record name.
|
||||
* @return an avro Schema.
|
||||
*/
|
||||
public static Schema convert(Types.RecordType type, String name) {
|
||||
return buildAvroSchemaFromType(type, name);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert internal type to avro Schema.
|
||||
*
|
||||
* @param type internal type.
|
||||
* @param name the record name.
|
||||
* @return an avro Schema.
|
||||
*/
|
||||
public static Schema convert(Type type, String name) {
|
||||
return buildAvroSchemaFromType(type, name);
|
||||
}
|
||||
|
||||
/** Convert an avro schema into internal type. */
|
||||
public static Type convertToField(Schema schema) {
|
||||
return buildTypeFromAvroSchema(schema);
|
||||
}
|
||||
|
||||
/** Convert an avro schema into internalSchema. */
|
||||
public static InternalSchema convert(Schema schema) {
|
||||
List<Types.Field> fields = ((Types.RecordType) convertToField(schema)).fields();
|
||||
return new InternalSchema(fields);
|
||||
}
|
||||
|
||||
/** Check whether current avro schema is optional?. */
|
||||
public static boolean isOptional(Schema schema) {
|
||||
if (schema.getType() == UNION && schema.getTypes().size() == 2) {
|
||||
return schema.getTypes().get(0).getType() == Schema.Type.NULL || schema.getTypes().get(1).getType() == Schema.Type.NULL;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/** Returns schema with nullable true. */
|
||||
public static Schema nullableSchema(Schema schema) {
|
||||
if (schema.getType() == UNION) {
|
||||
if (!isOptional(schema)) {
|
||||
throw new HoodieSchemaException(String.format("Union schemas are not supported: %s", schema));
|
||||
}
|
||||
return schema;
|
||||
} else {
|
||||
return Schema.createUnion(Schema.create(Schema.Type.NULL), schema);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Build hudi type from avro schema.
|
||||
*
|
||||
* @param schema a avro schema.
|
||||
* @return a hudi type.
|
||||
*/
|
||||
public static Type buildTypeFromAvroSchema(Schema schema) {
|
||||
// set flag to check this has not been visited.
|
||||
Deque<String> visited = new LinkedList();
|
||||
AtomicInteger nextId = new AtomicInteger(1);
|
||||
return visitAvroSchemaToBuildType(schema, visited, true, nextId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts an avro schema into hudi type.
|
||||
*
|
||||
* @param schema a avro schema.
|
||||
* @param visited track the visit node when do traversal for avro schema; used to check if the name of avro record schema is correct.
|
||||
* @param firstVisitRoot track whether the current visited schema node is a root node.
|
||||
* @param nextId a initial id which used to create id for all fields.
|
||||
* @return a hudi type match avro schema.
|
||||
*/
|
||||
private static Type visitAvroSchemaToBuildType(Schema schema, Deque<String> visited, Boolean firstVisitRoot, AtomicInteger nextId) {
|
||||
switch (schema.getType()) {
|
||||
case RECORD:
|
||||
String name = schema.getFullName();
|
||||
if (visited.contains(name)) {
|
||||
throw new HoodieSchemaException(String.format("cannot convert recursive avro record %s", name));
|
||||
}
|
||||
visited.push(name);
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
List<Type> fieldTypes = new ArrayList<>(fields.size());
|
||||
int nextAssignId = nextId.get();
|
||||
// when first visit root record, set nextAssignId = 0;
|
||||
if (firstVisitRoot) {
|
||||
nextAssignId = 0;
|
||||
}
|
||||
nextId.set(nextAssignId + fields.size());
|
||||
fields.stream().forEach(field -> {
|
||||
fieldTypes.add(visitAvroSchemaToBuildType(field.schema(), visited, false, nextId));
|
||||
});
|
||||
visited.pop();
|
||||
List<Types.Field> internalFields = new ArrayList<>(fields.size());
|
||||
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
Schema.Field field = fields.get(i);
|
||||
Type fieldType = fieldTypes.get(i);
|
||||
internalFields.add(Types.Field.get(nextAssignId, AvroInternalSchemaConverter.isOptional(field.schema()), field.name(), fieldType, field.doc()));
|
||||
nextAssignId += 1;
|
||||
}
|
||||
return Types.RecordType.get(internalFields);
|
||||
case UNION:
|
||||
List<Type> fTypes = new ArrayList<>();
|
||||
schema.getTypes().stream().forEach(t -> {
|
||||
fTypes.add(visitAvroSchemaToBuildType(t, visited, false, nextId));
|
||||
});
|
||||
return fTypes.get(0) == null ? fTypes.get(1) : fTypes.get(0);
|
||||
case ARRAY:
|
||||
Schema elementSchema = schema.getElementType();
|
||||
int elementId = nextId.get();
|
||||
nextId.set(elementId + 1);
|
||||
Type elementType = visitAvroSchemaToBuildType(elementSchema, visited, false, nextId);
|
||||
return Types.ArrayType.get(elementId, AvroInternalSchemaConverter.isOptional(schema.getElementType()), elementType);
|
||||
case MAP:
|
||||
int keyId = nextId.get();
|
||||
int valueId = keyId + 1;
|
||||
nextId.set(valueId + 1);
|
||||
Type valueType = visitAvroSchemaToBuildType(schema.getValueType(), visited, false, nextId);
|
||||
return Types.MapType.get(keyId, valueId, Types.StringType.get(), valueType, AvroInternalSchemaConverter.isOptional(schema.getValueType()));
|
||||
default:
|
||||
return visitAvroPrimitiveToBuildInternalType(schema);
|
||||
}
|
||||
}
|
||||
|
||||
private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) {
|
||||
LogicalType logical = primitive.getLogicalType();
|
||||
if (logical != null) {
|
||||
String name = logical.getName();
|
||||
if (logical instanceof LogicalTypes.Decimal) {
|
||||
return Types.DecimalType.get(
|
||||
((LogicalTypes.Decimal) logical).getPrecision(),
|
||||
((LogicalTypes.Decimal) logical).getScale());
|
||||
|
||||
} else if (logical instanceof LogicalTypes.Date) {
|
||||
return Types.DateType.get();
|
||||
|
||||
} else if (
|
||||
logical instanceof LogicalTypes.TimeMillis
|
||||
|| logical instanceof LogicalTypes.TimeMicros) {
|
||||
return Types.TimeType.get();
|
||||
|
||||
} else if (
|
||||
logical instanceof LogicalTypes.TimestampMillis
|
||||
|| logical instanceof LogicalTypes.TimestampMicros) {
|
||||
return Types.TimestampType.get();
|
||||
} else if (LogicalTypes.uuid().getName().equals(name)) {
|
||||
return Types.UUIDType.get();
|
||||
}
|
||||
}
|
||||
|
||||
switch (primitive.getType()) {
|
||||
case BOOLEAN:
|
||||
return Types.BooleanType.get();
|
||||
case INT:
|
||||
return Types.IntType.get();
|
||||
case LONG:
|
||||
return Types.LongType.get();
|
||||
case FLOAT:
|
||||
return Types.FloatType.get();
|
||||
case DOUBLE:
|
||||
return Types.DoubleType.get();
|
||||
case STRING:
|
||||
case ENUM:
|
||||
return Types.StringType.get();
|
||||
case FIXED:
|
||||
return Types.FixedType.getFixed(primitive.getFixedSize());
|
||||
case BYTES:
|
||||
return Types.BinaryType.get();
|
||||
case NULL:
|
||||
return null;
|
||||
default:
|
||||
throw new UnsupportedOperationException("Unsupported primitive type: " + primitive);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi type into an Avro Schema.
|
||||
*
|
||||
* @param type a hudi type.
|
||||
* @param recordName the record name
|
||||
* @return a Avro schema match this type
|
||||
*/
|
||||
public static Schema buildAvroSchemaFromType(Type type, String recordName) {
|
||||
Map<Type, Schema> cache = new HashMap<>();
|
||||
return visitInternalSchemaToBuildAvroSchema(type, cache, recordName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi internal Schema into an Avro Schema.
|
||||
*
|
||||
* @param schema a hudi internal Schema.
|
||||
* @param recordName the record name
|
||||
* @return a Avro schema match hudi internal schema.
|
||||
*/
|
||||
public static Schema buildAvroSchemaFromInternalSchema(InternalSchema schema, String recordName) {
|
||||
Map<Type, Schema> cache = new HashMap<>();
|
||||
return visitInternalSchemaToBuildAvroSchema(schema.getRecord(), cache, recordName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi type into an Avro Schema.
|
||||
*
|
||||
* @param type a hudi type.
|
||||
* @param cache use to cache intermediate convert result to save cost.
|
||||
* @param recordName the record name
|
||||
* @return a Avro schema match this type
|
||||
*/
|
||||
private static Schema visitInternalSchemaToBuildAvroSchema(Type type, Map<Type, Schema> cache, String recordName) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Schema> schemas = new ArrayList<>();
|
||||
record.fields().forEach(f -> {
|
||||
Schema tempSchema = visitInternalSchemaToBuildAvroSchema(f.type(), cache, recordName + "_" + f.name());
|
||||
// convert tempSchema
|
||||
Schema result = f.isOptional() ? AvroInternalSchemaConverter.nullableSchema(tempSchema) : tempSchema;
|
||||
schemas.add(result);
|
||||
});
|
||||
// check visited
|
||||
Schema recordSchema;
|
||||
recordSchema = cache.get(record);
|
||||
if (recordSchema != null) {
|
||||
return recordSchema;
|
||||
}
|
||||
recordSchema = visitInternalRecordToBuildAvroRecord(record, schemas, recordName);
|
||||
cache.put(record, recordSchema);
|
||||
return recordSchema;
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Schema elementSchema;
|
||||
elementSchema = visitInternalSchemaToBuildAvroSchema(array.elementType(), cache, recordName);
|
||||
Schema arraySchema;
|
||||
arraySchema = cache.get(array);
|
||||
if (arraySchema != null) {
|
||||
return arraySchema;
|
||||
}
|
||||
arraySchema = visitInternalArrayToBuildAvroArray(array, elementSchema);
|
||||
cache.put(array, arraySchema);
|
||||
return arraySchema;
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
Schema keySchema;
|
||||
Schema valueSchema;
|
||||
keySchema = visitInternalSchemaToBuildAvroSchema(map.keyType(), cache, recordName);
|
||||
valueSchema = visitInternalSchemaToBuildAvroSchema(map.valueType(), cache, recordName);
|
||||
Schema mapSchema;
|
||||
mapSchema = cache.get(map);
|
||||
if (mapSchema != null) {
|
||||
return mapSchema;
|
||||
}
|
||||
mapSchema = visitInternalMapToBuildAvroMap(map, keySchema, valueSchema);
|
||||
cache.put(map, mapSchema);
|
||||
return mapSchema;
|
||||
default:
|
||||
Schema primitiveSchema = visitInternalPrimitiveToBuildAvroPrimitiveType((Type.PrimitiveType) type);
|
||||
cache.put(type, primitiveSchema);
|
||||
return primitiveSchema;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi RecordType to Avro RecordType.
|
||||
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
|
||||
*/
|
||||
private static Schema visitInternalRecordToBuildAvroRecord(Types.RecordType record, List<Schema> fieldSchemas, String recordName) {
|
||||
List<Types.Field> fields = record.fields();
|
||||
List<Schema.Field> avroFields = new ArrayList<>();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
Types.Field f = fields.get(i);
|
||||
Schema.Field field = new Schema.Field(f.name(), fieldSchemas.get(i), f.doc(), f.isOptional() ? JsonProperties.NULL_VALUE : null);
|
||||
avroFields.add(field);
|
||||
}
|
||||
return Schema.createRecord(recordName, null, null, false, avroFields);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi ArrayType to Avro ArrayType.
|
||||
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
|
||||
*/
|
||||
private static Schema visitInternalArrayToBuildAvroArray(Types.ArrayType array, Schema elementSchema) {
|
||||
Schema result;
|
||||
if (array.isElementOptional()) {
|
||||
result = Schema.createArray(AvroInternalSchemaConverter.nullableSchema(elementSchema));
|
||||
} else {
|
||||
result = Schema.createArray(elementSchema);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi MapType to Avro MapType.
|
||||
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
|
||||
*/
|
||||
private static Schema visitInternalMapToBuildAvroMap(Types.MapType map, Schema keySchema, Schema valueSchema) {
|
||||
Schema mapSchema;
|
||||
if (keySchema.getType() == Schema.Type.STRING) {
|
||||
mapSchema = Schema.createMap(map.isValueOptional() ? AvroInternalSchemaConverter.nullableSchema(valueSchema) : valueSchema);
|
||||
} else {
|
||||
throw new HoodieSchemaException("only support StringType key for avro MapType");
|
||||
}
|
||||
return mapSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts hudi PrimitiveType to Avro PrimitiveType.
|
||||
* this is auxiliary function used by visitInternalSchemaToBuildAvroSchema
|
||||
*/
|
||||
private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.PrimitiveType primitive) {
|
||||
Schema primitiveSchema;
|
||||
switch (primitive.typeId()) {
|
||||
case BOOLEAN:
|
||||
primitiveSchema = Schema.create(Schema.Type.BOOLEAN);
|
||||
break;
|
||||
case INT:
|
||||
primitiveSchema = Schema.create(Schema.Type.INT);
|
||||
break;
|
||||
case LONG:
|
||||
primitiveSchema = Schema.create(Schema.Type.LONG);
|
||||
break;
|
||||
case FLOAT:
|
||||
primitiveSchema = Schema.create(Schema.Type.FLOAT);
|
||||
break;
|
||||
case DOUBLE:
|
||||
primitiveSchema = Schema.create(Schema.Type.DOUBLE);
|
||||
break;
|
||||
case DATE:
|
||||
primitiveSchema = LogicalTypes.date()
|
||||
.addToSchema(Schema.create(Schema.Type.INT));
|
||||
break;
|
||||
case TIME:
|
||||
primitiveSchema = LogicalTypes.timeMicros()
|
||||
.addToSchema(Schema.create(Schema.Type.LONG));
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
primitiveSchema = LogicalTypes.timestampMicros()
|
||||
.addToSchema(Schema.create(Schema.Type.LONG));
|
||||
break;
|
||||
case STRING:
|
||||
primitiveSchema = Schema.create(Schema.Type.STRING);
|
||||
break;
|
||||
case UUID:
|
||||
primitiveSchema = LogicalTypes.uuid()
|
||||
.addToSchema(Schema.createFixed("uuid_fixed", null, null, 16));
|
||||
break;
|
||||
case FIXED:
|
||||
Types.FixedType fixed = (Types.FixedType) primitive;
|
||||
primitiveSchema = Schema.createFixed("fixed_" + fixed.getFixedSize(), null, null, fixed.getFixedSize());
|
||||
break;
|
||||
case BINARY:
|
||||
primitiveSchema = Schema.create(Schema.Type.BYTES);
|
||||
break;
|
||||
case DECIMAL:
|
||||
Types.DecimalType decimal = (Types.DecimalType) primitive;
|
||||
primitiveSchema = LogicalTypes.decimal(decimal.precision(), decimal.scale())
|
||||
.addToSchema(Schema.createFixed(
|
||||
"decimal_" + decimal.precision() + "_" + decimal.scale(),
|
||||
null, null, computeMinBytesForPrecision(decimal.precision())));
|
||||
break;
|
||||
default:
|
||||
throw new UnsupportedOperationException(
|
||||
"Unsupported type ID: " + primitive.typeId());
|
||||
}
|
||||
return primitiveSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the minimum number of bytes needed to store a decimal with a give 'precision'.
|
||||
* reference from Spark release 3.1 .
|
||||
*/
|
||||
private static int computeMinBytesForPrecision(int precision) {
|
||||
int numBytes = 1;
|
||||
while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) {
|
||||
numBytes += 1;
|
||||
}
|
||||
return numBytes;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.internal.schema.io;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
abstract class AbstractInternalSchemaStorageManager {
|
||||
|
||||
/**
|
||||
* Persist history schema str.
|
||||
*/
|
||||
public abstract void persistHistorySchemaStr(String instantTime, String historySchemaStr);
|
||||
|
||||
/**
|
||||
* Get latest history schema string.
|
||||
*/
|
||||
public abstract String getHistorySchemaStr();
|
||||
|
||||
/**
|
||||
* Get latest history schema string.
|
||||
* Using give validCommits to validate all legal histroy Schema files, and return the latest one.
|
||||
* If the passed valid commits is null or empty, valid instants will be fetched from the file-system and used.
|
||||
*/
|
||||
public abstract String getHistorySchemaStrByGivenValidCommits(List<String> validCommits);
|
||||
|
||||
/**
|
||||
* Get internalSchema by using given versionId
|
||||
*
|
||||
* @param versionId schema version_id need to search
|
||||
* @return internalSchema
|
||||
*/
|
||||
public abstract Option getSchemaByKey(String versionId);
|
||||
}
|
||||
@@ -0,0 +1,184 @@
|
||||
/*
|
||||
* 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.internal.schema.io;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.SCHEMA_COMMIT_ACTION;
|
||||
|
||||
public class FileBasedInternalSchemaStorageManager extends AbstractInternalSchemaStorageManager {
|
||||
private static final Logger LOG = LogManager.getLogger(FileBasedInternalSchemaStorageManager.class);
|
||||
|
||||
public static final String SCHEMA_NAME = ".schema";
|
||||
private final Path baseSchemaPath;
|
||||
private final Configuration conf;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
public FileBasedInternalSchemaStorageManager(Configuration conf, Path baseTablePath) {
|
||||
Path metaPath = new Path(baseTablePath, ".hoodie");
|
||||
this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
public FileBasedInternalSchemaStorageManager(HoodieTableMetaClient metaClient) {
|
||||
Path metaPath = new Path(metaClient.getBasePath(), ".hoodie");
|
||||
this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
|
||||
this.conf = metaClient.getHadoopConf();
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
// make metaClient build lazy
|
||||
private HoodieTableMetaClient getMetaClient() {
|
||||
if (metaClient == null) {
|
||||
metaClient = HoodieTableMetaClient.builder().setBasePath(baseSchemaPath.getParent().getParent().toString()).setConf(conf).build();
|
||||
}
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persistHistorySchemaStr(String instantTime, String historySchemaStr) {
|
||||
cleanResidualFiles();
|
||||
HoodieActiveTimeline timeline = getMetaClient().getActiveTimeline();
|
||||
HoodieInstant hoodieInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, SCHEMA_COMMIT_ACTION, instantTime);
|
||||
timeline.createNewInstant(hoodieInstant);
|
||||
byte[] writeContent = historySchemaStr.getBytes(StandardCharsets.UTF_8);
|
||||
timeline.transitionRequestedToInflight(hoodieInstant, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.getTimestamp()), Option.of(writeContent));
|
||||
LOG.info(String.format("persist history schema success on commit time: %s", instantTime));
|
||||
}
|
||||
|
||||
private void cleanResidualFiles() {
|
||||
List<String> validateCommits = getValidInstants();
|
||||
try {
|
||||
FileSystem fs = baseSchemaPath.getFileSystem(conf);
|
||||
if (fs.exists(baseSchemaPath)) {
|
||||
List<String> candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile())
|
||||
.map(file -> file.getPath().getName()).collect(Collectors.toList());
|
||||
List<String> residualSchemaFiles = candidateSchemaFiles.stream().filter(f -> !validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList());
|
||||
// clean residual files
|
||||
residualSchemaFiles.forEach(f -> {
|
||||
try {
|
||||
fs.delete(new Path(getMetaClient().getSchemaFolderName(), f));
|
||||
} catch (IOException o) {
|
||||
throw new HoodieException(o);
|
||||
}
|
||||
});
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public void cleanOldFiles(List<String> validateCommits) {
|
||||
try {
|
||||
FileSystem fs = baseSchemaPath.getFileSystem(conf);
|
||||
if (fs.exists(baseSchemaPath)) {
|
||||
List<String> candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile())
|
||||
.map(file -> file.getPath().getName()).collect(Collectors.toList());
|
||||
List<String> validateSchemaFiles = candidateSchemaFiles.stream().filter(f -> validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList());
|
||||
for (int i = 0; i < validateSchemaFiles.size(); i++) {
|
||||
fs.delete(new Path(validateSchemaFiles.get(i)));
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private List<String> getValidInstants() {
|
||||
return getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getHistorySchemaStr() {
|
||||
return getHistorySchemaStrByGivenValidCommits(Collections.EMPTY_LIST);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getHistorySchemaStrByGivenValidCommits(List<String> validCommits) {
|
||||
List<String> commitList = validCommits == null || validCommits.isEmpty() ? getValidInstants() : validCommits;
|
||||
try {
|
||||
FileSystem fs = FSUtils.getFs(baseSchemaPath.toString(), conf);
|
||||
if (fs.exists(baseSchemaPath)) {
|
||||
List<String> validaSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath))
|
||||
.filter(f -> f.isFile() && f.getPath().getName().endsWith(SCHEMA_COMMIT_ACTION))
|
||||
.map(file -> file.getPath().getName()).filter(f -> commitList.contains(f.split("\\.")[0])).sorted().collect(Collectors.toList());
|
||||
if (!validaSchemaFiles.isEmpty()) {
|
||||
Path latestFilePath = new Path(baseSchemaPath, validaSchemaFiles.get(validaSchemaFiles.size() - 1));
|
||||
byte[] content;
|
||||
try (FSDataInputStream is = fs.open(latestFilePath)) {
|
||||
content = FileIOUtils.readAsByteArray(is);
|
||||
LOG.info(String.format("read history schema success from file : %s", latestFilePath));
|
||||
return new String(content, StandardCharsets.UTF_8);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read history schema from " + latestFilePath, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieException(io);
|
||||
}
|
||||
LOG.info("failed to read history schema");
|
||||
return "";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<InternalSchema> getSchemaByKey(String versionId) {
|
||||
String historySchemaStr = getHistorySchemaStr();
|
||||
TreeMap<Long, InternalSchema> treeMap;
|
||||
if (historySchemaStr.isEmpty()) {
|
||||
return Option.empty();
|
||||
} else {
|
||||
treeMap = SerDeHelper.parseSchemas(historySchemaStr);
|
||||
InternalSchema result = InternalSchemaUtils.searchSchema(Long.valueOf(versionId), treeMap);
|
||||
if (result == null) {
|
||||
return Option.empty();
|
||||
}
|
||||
return Option.of(result);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* 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.internal.schema.utils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
import org.apache.hudi.internal.schema.action.TableChanges;
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility methods to support evolve old avro schema based on a given schema.
|
||||
*/
|
||||
public class AvroSchemaEvolutionUtils {
|
||||
/**
|
||||
* Support evolution from a new avroSchema.
|
||||
* Now hoodie support implicitly add columns when hoodie write operation,
|
||||
* This ability needs to be preserved, so implicitly evolution for internalSchema should supported.
|
||||
*
|
||||
* @param evolvedSchema implicitly evolution of avro when hoodie write operation
|
||||
* @param oldSchema old internalSchema
|
||||
* @param supportPositionReorder support position reorder
|
||||
* @return evolution Schema
|
||||
*/
|
||||
public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema, Boolean supportPositionReorder) {
|
||||
InternalSchema evolvedInternalSchema = AvroInternalSchemaConverter.convert(evolvedSchema);
|
||||
// do check, only support add column evolution
|
||||
List<String> colNamesFromEvolved = evolvedInternalSchema.getAllColsFullName();
|
||||
List<String> colNamesFromOldSchema = oldSchema.getAllColsFullName();
|
||||
List<String> diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromEvolved.contains(f)).collect(Collectors.toList());
|
||||
List<Types.Field> newFields = new ArrayList<>();
|
||||
if (colNamesFromEvolved.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) {
|
||||
// no changes happen
|
||||
if (supportPositionReorder) {
|
||||
evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name())));
|
||||
return new InternalSchema(newFields);
|
||||
}
|
||||
return oldSchema;
|
||||
}
|
||||
// try to find all added columns
|
||||
if (diffFromOldSchema.size() != 0) {
|
||||
throw new UnsupportedOperationException("Cannot evolve schema implicitly, find delete/rename operation");
|
||||
}
|
||||
|
||||
List<String> diffFromEvolutionSchema = colNamesFromEvolved.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList());
|
||||
// Remove redundancy from diffFromEvolutionSchema.
|
||||
// for example, now we add a struct col in evolvedSchema, the struct col is " user struct<name:string, age:int> "
|
||||
// when we do diff operation: user, user.name, user.age will appeared in the resultSet which is redundancy, user.name and user.age should be excluded.
|
||||
// deal with add operation
|
||||
TreeMap<Integer, String> finalAddAction = new TreeMap<>();
|
||||
for (int i = 0; i < diffFromEvolutionSchema.size(); i++) {
|
||||
String name = diffFromEvolutionSchema.get(i);
|
||||
int splitPoint = name.lastIndexOf(".");
|
||||
String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : "";
|
||||
if (!parentName.isEmpty() && diffFromEvolutionSchema.contains(parentName)) {
|
||||
// find redundancy, skip it
|
||||
continue;
|
||||
}
|
||||
finalAddAction.put(evolvedInternalSchema.findIdByName(name), name);
|
||||
}
|
||||
|
||||
TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema);
|
||||
finalAddAction.entrySet().stream().forEach(f -> {
|
||||
String name = f.getValue();
|
||||
int splitPoint = name.lastIndexOf(".");
|
||||
String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : "";
|
||||
String rawName = splitPoint > 0 ? name.substring(splitPoint + 1) : name;
|
||||
addChange.addColumns(parentName, rawName, evolvedInternalSchema.findType(name), null);
|
||||
});
|
||||
|
||||
InternalSchema res = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange);
|
||||
if (supportPositionReorder) {
|
||||
evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name())));
|
||||
return new InternalSchema(newFields);
|
||||
} else {
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema) {
|
||||
return evolveSchemaFromNewAvroSchema(evolvedSchema, oldSchema, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Canonical the nullability.
|
||||
* Do not allow change cols Nullability field from optional to required.
|
||||
* If above problem occurs, try to correct it.
|
||||
*
|
||||
* @param writeSchema writeSchema hoodie used to write data.
|
||||
* @param readSchema read schema
|
||||
* @return canonical Schema
|
||||
*/
|
||||
public static Schema canonicalizeColumnNullability(Schema writeSchema, Schema readSchema) {
|
||||
if (writeSchema.getFields().isEmpty() || readSchema.getFields().isEmpty()) {
|
||||
return writeSchema;
|
||||
}
|
||||
InternalSchema writeInternalSchema = AvroInternalSchemaConverter.convert(writeSchema);
|
||||
InternalSchema readInternalSchema = AvroInternalSchemaConverter.convert(readSchema);
|
||||
List<String> colNamesWriteSchema = writeInternalSchema.getAllColsFullName();
|
||||
List<String> colNamesFromReadSchema = readInternalSchema.getAllColsFullName();
|
||||
// try to deal with optional change. now when we use sparksql to update hudi table,
|
||||
// sparksql Will change the col type from optional to required, this is a bug.
|
||||
List<String> candidateUpdateCols = colNamesWriteSchema.stream().filter(f -> {
|
||||
boolean exist = colNamesFromReadSchema.contains(f);
|
||||
if (exist && (writeInternalSchema.findField(f).isOptional() != readInternalSchema.findField(f).isOptional())) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
if (candidateUpdateCols.isEmpty()) {
|
||||
return writeSchema;
|
||||
}
|
||||
// try to correct all changes
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(writeInternalSchema);
|
||||
candidateUpdateCols.stream().forEach(f -> updateChange.updateColumnNullability(f, true));
|
||||
Schema result = AvroInternalSchemaConverter.convert(SchemaChangeUtils.applyTableChanges2Schema(writeInternalSchema, updateChange), writeSchema.getName());
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,270 @@
|
||||
/*
|
||||
* 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.internal.schema.utils;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.internal.schema.HoodieSchemaException;
|
||||
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.Types.Field;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.SortedMap;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Util methods to help us do some operations on InternalSchema.
|
||||
* eg: column prune, filter rebuild for query engine...
|
||||
*/
|
||||
public class InternalSchemaUtils {
|
||||
|
||||
private InternalSchemaUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create project internalSchema, based on the project names which produced by query engine.
|
||||
* support nested project.
|
||||
*
|
||||
* @param schema a internal schema.
|
||||
* @param names project names produced by query engine.
|
||||
* @return a project internalSchema.
|
||||
*/
|
||||
public static InternalSchema pruneInternalSchema(InternalSchema schema, List<String> names) {
|
||||
// do check
|
||||
List<Integer> prunedIds = names.stream().map(name -> {
|
||||
int id = schema.findIdByName(name);
|
||||
if (id == -1) {
|
||||
throw new IllegalArgumentException(String.format("cannot prune col: %s which not exisit in hudi table", name));
|
||||
}
|
||||
return id;
|
||||
}).collect(Collectors.toList());
|
||||
// find top parent field ID. eg: a.b.c, f.g.h, only collect id of a and f ignore all child field.
|
||||
List<Integer> topParentFieldIds = new ArrayList<>();
|
||||
names.stream().forEach(f -> {
|
||||
int id = schema.findIdByName(f.split("\\.")[0]);
|
||||
if (!topParentFieldIds.contains(id)) {
|
||||
topParentFieldIds.add(id);
|
||||
}
|
||||
});
|
||||
return pruneInternalSchemaByID(schema, prunedIds, topParentFieldIds);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create project internalSchema.
|
||||
* support nested project.
|
||||
*
|
||||
* @param schema a internal schema.
|
||||
* @param fieldIds project col field_ids.
|
||||
* @return a project internalSchema.
|
||||
*/
|
||||
public static InternalSchema pruneInternalSchemaByID(InternalSchema schema, List<Integer> fieldIds, List<Integer> topParentFieldIds) {
|
||||
Types.RecordType recordType = (Types.RecordType)pruneType(schema.getRecord(), fieldIds);
|
||||
// reorder top parent fields, since the recordType.fields() produced by pruneType maybe out of order.
|
||||
List<Types.Field> newFields = new ArrayList<>();
|
||||
if (topParentFieldIds != null && !topParentFieldIds.isEmpty()) {
|
||||
for (int id : topParentFieldIds) {
|
||||
Types.Field f = recordType.field(id);
|
||||
if (f != null) {
|
||||
newFields.add(f);
|
||||
} else {
|
||||
throw new HoodieSchemaException(String.format("cannot find pruned id %s in currentSchema %s", id, schema.toString()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return new InternalSchema(newFields.isEmpty() ? recordType.fields() : newFields);
|
||||
}
|
||||
|
||||
/**
|
||||
* Project hudi type by projected cols field_ids
|
||||
* this is auxiliary function used by pruneInternalSchema.
|
||||
*/
|
||||
private static Type pruneType(Type type, List<Integer> fieldIds) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Types.Field> fields = record.fields();
|
||||
List<Type> newTypes = new ArrayList<>();
|
||||
for (Types.Field f : fields) {
|
||||
Type newType = pruneType(f.type(), fieldIds);
|
||||
if (fieldIds.contains(f.fieldId())) {
|
||||
newTypes.add(f.type());
|
||||
} else if (newType != null) {
|
||||
newTypes.add(newType);
|
||||
} else {
|
||||
newTypes.add(null);
|
||||
}
|
||||
}
|
||||
boolean changed = false;
|
||||
List<Field> newFields = new ArrayList<>();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
Types.Field oldField = fields.get(i);
|
||||
Type newType = newTypes.get(i);
|
||||
if (oldField.type() == newType) {
|
||||
newFields.add(oldField);
|
||||
} else if (newType != null) {
|
||||
changed = true;
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
|
||||
}
|
||||
}
|
||||
if (newFields.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
if (newFields.size() == fields.size() && !changed) {
|
||||
return record;
|
||||
} else {
|
||||
return Types.RecordType.get(newFields);
|
||||
}
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Type newElementType = pruneType(array.elementType(), fieldIds);
|
||||
if (fieldIds.contains(array.elementId())) {
|
||||
return array;
|
||||
} else if (newElementType != null) {
|
||||
if (array.elementType() == newElementType) {
|
||||
return array;
|
||||
}
|
||||
return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType);
|
||||
}
|
||||
return null;
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
Type newValueType = pruneType(map.valueType(), fieldIds);
|
||||
if (fieldIds.contains(map.valueId())) {
|
||||
return map;
|
||||
} else if (newValueType != null) {
|
||||
if (map.valueType() == newValueType) {
|
||||
return map;
|
||||
}
|
||||
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional());
|
||||
}
|
||||
return null;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper function to help correct the colName of pushed filters.
|
||||
*
|
||||
* @param name origin col name from pushed filters.
|
||||
* @param fileSchema the real schema of avro/parquet file.
|
||||
* @param querySchema the query schema which query engine produced.
|
||||
* @return a corrected name.
|
||||
*/
|
||||
public static String reBuildFilterName(String name, InternalSchema fileSchema, InternalSchema querySchema) {
|
||||
int nameId = querySchema.findIdByName(name);
|
||||
if (nameId == -1) {
|
||||
throw new IllegalArgumentException(String.format("cannot found filter col name:%s from querySchema: %s", name, querySchema));
|
||||
}
|
||||
if (fileSchema.findField(nameId) == null) {
|
||||
// added operation found
|
||||
// the read file does not contain current col, so current colFilter is invalid
|
||||
return "";
|
||||
} else {
|
||||
if (name.equals(fileSchema.findfullName(nameId))) {
|
||||
// no change happened on current col
|
||||
return name;
|
||||
} else {
|
||||
// find rename operation on current col
|
||||
// return the name from fileSchema
|
||||
return fileSchema.findfullName(nameId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Collect all type changed cols to build a colPosition -> (newColType, oldColType) map.
|
||||
* only collect top level col changed. eg: a is a nest field(record(b int, d long), now a.b is changed from int to long,
|
||||
* only a will be collected, a.b will excluded.
|
||||
*
|
||||
* @param schema a type changed internalSchema
|
||||
* @param oldSchema an old internalSchema.
|
||||
* @return a map.
|
||||
*/
|
||||
public static Map<Integer, Pair<Type, Type>> collectTypeChangedCols(InternalSchema schema, InternalSchema oldSchema) {
|
||||
Set<Integer> ids = schema.getAllIds();
|
||||
Set<Integer> otherIds = oldSchema.getAllIds();
|
||||
Map<Integer, Pair<Type, Type>> result = new HashMap<>();
|
||||
ids.stream().filter(f -> otherIds.contains(f)).forEach(f -> {
|
||||
if (!schema.findType(f).equals(oldSchema.findType(f))) {
|
||||
String[] fieldNameParts = schema.findfullName(f).split("\\.");
|
||||
String[] otherFieldNameParts = oldSchema.findfullName(f).split("\\.");
|
||||
String parentName = fieldNameParts[0];
|
||||
String otherParentName = otherFieldNameParts[0];
|
||||
if (fieldNameParts.length == otherFieldNameParts.length && schema.findIdByName(parentName) == oldSchema.findIdByName(otherParentName)) {
|
||||
int index = schema.findIdByName(parentName);
|
||||
int position = schema.getRecord().fields().stream().map(s -> s.fieldId()).collect(Collectors.toList()).indexOf(index);
|
||||
if (!result.containsKey(position)) {
|
||||
result.put(position, Pair.of(schema.findType(parentName), oldSchema.findType(otherParentName)));
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Search target internalSchema by version number.
|
||||
*
|
||||
* @param versionId the internalSchema version to be search.
|
||||
* @param internalSchemas internalSchemas to be searched.
|
||||
* @return a internalSchema.
|
||||
*/
|
||||
public static InternalSchema searchSchema(long versionId, List<InternalSchema> internalSchemas) {
|
||||
TreeMap<Long, InternalSchema> treeMap = new TreeMap<>();
|
||||
internalSchemas.forEach(s -> treeMap.put(s.schemaId(), s));
|
||||
return searchSchema(versionId, treeMap);
|
||||
}
|
||||
|
||||
/**
|
||||
* Search target internalSchema by version number.
|
||||
*
|
||||
* @param versionId the internalSchema version to be search.
|
||||
* @param treeMap internalSchemas collections to be searched.
|
||||
* @return a internalSchema.
|
||||
*/
|
||||
public static InternalSchema searchSchema(long versionId, TreeMap<Long, InternalSchema> treeMap) {
|
||||
if (treeMap.containsKey(versionId)) {
|
||||
return treeMap.get(versionId);
|
||||
} else {
|
||||
SortedMap<Long, InternalSchema> headMap = treeMap.headMap(versionId);
|
||||
if (!headMap.isEmpty()) {
|
||||
return headMap.get(headMap.lastKey());
|
||||
}
|
||||
}
|
||||
return InternalSchema.getEmptyInternalSchema();
|
||||
}
|
||||
|
||||
public static String createFullName(String name, Deque<String> fieldNames) {
|
||||
String result = name;
|
||||
if (!fieldNames.isEmpty()) {
|
||||
List<String> parentNames = new ArrayList<>();
|
||||
fieldNames.descendingIterator().forEachRemaining(parentNames::add);
|
||||
result = parentNames.stream().collect(Collectors.joining(".")) + "." + result;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,305 @@
|
||||
/*
|
||||
* 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.internal.schema.utils;
|
||||
|
||||
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.TableChanges;
|
||||
import org.apache.hudi.internal.schema.action.TableChangesHelper;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Helper methods for schema Change.
|
||||
*/
|
||||
public class SchemaChangeUtils {
|
||||
private SchemaChangeUtils() {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether to allow the column type to be updated.
|
||||
* now only support:
|
||||
* int => long/float/double/string
|
||||
* long => float/double/string
|
||||
* float => double/String
|
||||
* double => String/Decimal
|
||||
* Decimal => Decimal/String
|
||||
* String => date/decimal
|
||||
* date => String
|
||||
* TODO: support more type update.
|
||||
*
|
||||
* @param src origin column type.
|
||||
* @param dsr new column type.
|
||||
* @return whether to allow the column type to be updated.
|
||||
*/
|
||||
public static boolean isTypeUpdateAllow(Type src, Type dsr) {
|
||||
if (src.isNestedType() || dsr.isNestedType()) {
|
||||
throw new IllegalArgumentException("only support update primitive type");
|
||||
}
|
||||
if (src.equals(dsr)) {
|
||||
return true;
|
||||
}
|
||||
switch (src.typeId()) {
|
||||
case INT:
|
||||
return dsr == Types.LongType.get() || dsr == Types.FloatType.get()
|
||||
|| dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
|
||||
case LONG:
|
||||
return dsr == Types.FloatType.get() || dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
|
||||
case FLOAT:
|
||||
return dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
|
||||
case DOUBLE:
|
||||
return dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
|
||||
case DATE:
|
||||
return dsr == Types.StringType.get();
|
||||
case DECIMAL:
|
||||
if (dsr.typeId() == Type.TypeID.DECIMAL) {
|
||||
Types.DecimalType decimalSrc = (Types.DecimalType)src;
|
||||
Types.DecimalType decimalDsr = (Types.DecimalType)dsr;
|
||||
if (decimalDsr.isWiderThan(decimalSrc)) {
|
||||
return true;
|
||||
}
|
||||
} else if (dsr.typeId() == Type.TypeID.STRING) {
|
||||
return true;
|
||||
}
|
||||
break;
|
||||
case STRING:
|
||||
return dsr == Types.DateType.get() || dsr.typeId() == Type.TypeID.DECIMAL;
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply all the DDL add operations to internalSchema to produce a new internalSchema.
|
||||
*
|
||||
* @param internalSchema origin internalSchema.
|
||||
* @param adds a wrapper class for all the DDL add operations.
|
||||
* @return a new internalSchema.
|
||||
*/
|
||||
public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnAddChange adds) {
|
||||
Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), adds);
|
||||
// deal with root level changes
|
||||
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(),
|
||||
adds.getParentId2AddCols().get(-1), adds.getPositionChangeMap().get(-1));
|
||||
return new InternalSchema(newFields);
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply all the DDL add operations to Type to produce a new internalSchema.
|
||||
* do not call this method directly. expose this method only for UT.
|
||||
*
|
||||
* @param type origin hudi Type.
|
||||
* @param adds a wrapper class for all the DDL add operations.
|
||||
* @return a new internalSchema.
|
||||
*/
|
||||
public static Type applyTableChange2Type(Type type, TableChanges.ColumnAddChange adds) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Type> newTypes = new ArrayList<>();
|
||||
for (Types.Field f : record.fields()) {
|
||||
Type newType = applyTableChange2Type(f.type(), adds);
|
||||
// try to apply add
|
||||
newTypes.add(newType.isNestedType() ? adds.applyAdd(f, newType) : newType);
|
||||
}
|
||||
List<Types.Field> newFields = new ArrayList<>();
|
||||
boolean hasChanged = false;
|
||||
for (int i = 0; i < newTypes.size(); i++) {
|
||||
Type newType = newTypes.get(i);
|
||||
Types.Field oldfield = record.fields().get(i);
|
||||
if (oldfield.type() == newType) {
|
||||
newFields.add(oldfield);
|
||||
} else {
|
||||
hasChanged = true;
|
||||
newFields.add(Types.Field.get(oldfield.fieldId(), oldfield.isOptional(), oldfield.name(), newType, oldfield.doc()));
|
||||
}
|
||||
}
|
||||
return hasChanged ? Types.RecordType.get(newFields) : record;
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Type newElementType;
|
||||
Types.Field elementField = array.field(array.elementId());
|
||||
newElementType = applyTableChange2Type(array.elementType(), adds);
|
||||
// try to apply add
|
||||
newElementType = adds.applyAdd(elementField, newElementType);
|
||||
if (newElementType == array.elementType()) {
|
||||
return array;
|
||||
}
|
||||
return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType);
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
Type newValueType;
|
||||
Types.Field valueField = map.field(map.valueId());
|
||||
if (adds.getParentId2AddCols().containsKey(map.keyId())) {
|
||||
throw new IllegalArgumentException("Cannot add fields to map keys: " + map);
|
||||
}
|
||||
newValueType = applyTableChange2Type(map.valueType(), adds);
|
||||
// try to apply add
|
||||
newValueType = adds.applyAdd(valueField, newValueType);
|
||||
if (newValueType == map.valueType()) {
|
||||
return map;
|
||||
}
|
||||
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional());
|
||||
default:
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply all the DDL delete operations to internalSchema to produce a new internalSchema.
|
||||
*
|
||||
* @param internalSchema origin internalSchema.
|
||||
* @param deletes a wrapper class for all the DDL delete operations.
|
||||
* @return a new internalSchema.
|
||||
*/
|
||||
public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnDeleteChange deletes) {
|
||||
Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), deletes);
|
||||
return new InternalSchema(newType.fields());
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply all the DDL delete operations to Type to produce a new internalSchema.
|
||||
* do not call this method directly. expose this method only for UT.
|
||||
*
|
||||
* @param type origin type.
|
||||
* @param deletes a wrapper class for all the DDL delete operations.
|
||||
* @return a new internalSchema.
|
||||
*/
|
||||
private static Type applyTableChange2Type(Type type, TableChanges.ColumnDeleteChange deletes) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Types.Field> fields = new ArrayList<>();
|
||||
for (Types.Field f : record.fields()) {
|
||||
Type newType = applyTableChange2Type(f.type(), deletes);
|
||||
// apply delete
|
||||
newType = deletes.applyDelete(f.fieldId(), newType);
|
||||
if (newType != null) {
|
||||
fields.add(Types.Field.get(f.fieldId(), f.isOptional(), f.name(), newType, f.doc()));
|
||||
}
|
||||
}
|
||||
if (fields.isEmpty()) {
|
||||
throw new UnsupportedOperationException("cannot support delete all columns from Struct");
|
||||
}
|
||||
return Types.RecordType.get(fields);
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Type newElementType = applyTableChange2Type(array.elementType(), deletes);
|
||||
newElementType = deletes.applyDelete(array.elementId(), newElementType);
|
||||
if (newElementType == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot delete element from arrayType: %s", array));
|
||||
}
|
||||
return Types.ArrayType.get(array.elementId(), array.isElementOptional(), newElementType);
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
int keyId = map.fields().get(0).fieldId();
|
||||
if (deletes.getDeletes().contains(keyId)) {
|
||||
throw new IllegalArgumentException(String.format("cannot delete key from mapType: %s", map));
|
||||
}
|
||||
Type newValueType = applyTableChange2Type(map.valueType(), deletes);
|
||||
newValueType = deletes.applyDelete(map.valueId(), newValueType);
|
||||
if (newValueType == null) {
|
||||
throw new IllegalArgumentException(String.format("cannot delete value from mapType: %s", map));
|
||||
}
|
||||
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, map.isValueOptional());
|
||||
default:
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply all the DDL update operations to internalSchema to produce a new internalSchema.
|
||||
*
|
||||
* @param internalSchema origin internalSchema.
|
||||
* @param updates a wrapper class for all the DDL update operations.
|
||||
* @return a new internalSchema.
|
||||
*/
|
||||
public static InternalSchema applyTableChanges2Schema(InternalSchema internalSchema, TableChanges.ColumnUpdateChange updates) {
|
||||
Types.RecordType newType = (Types.RecordType)applyTableChange2Type(internalSchema.getRecord(), updates);
|
||||
// deal with root level changes
|
||||
List<Types.Field> newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(),
|
||||
new ArrayList<>(), updates.getPositionChangeMap().get(-1));
|
||||
return new InternalSchema(newFields);
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply all the DDL update operations to type to produce a new internalSchema.
|
||||
* do not call this method directly. expose this method only for UT.
|
||||
*
|
||||
* @param type origin internalSchema.
|
||||
* @param updates a wrapper class for all the DDL update operations.
|
||||
* @return a new internalSchema.
|
||||
*/
|
||||
private static Type applyTableChange2Type(Type type, TableChanges.ColumnUpdateChange updates) {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
Types.RecordType record = (Types.RecordType) type;
|
||||
List<Type> newTypes = new ArrayList<>();
|
||||
for (Types.Field f : record.fields()) {
|
||||
Type newType = applyTableChange2Type(f.type(), updates);
|
||||
newTypes.add(updates.applyUpdates(f, newType));
|
||||
}
|
||||
List<Types.Field> newFields = new ArrayList<>();
|
||||
for (int i = 0; i < newTypes.size(); i++) {
|
||||
Type newType = newTypes.get(i);
|
||||
Types.Field oldField = record.fields().get(i);
|
||||
Types.Field updateField = updates.getUpdates().get(oldField.fieldId());
|
||||
if (updateField != null) {
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), updateField.isOptional(), updateField.name(), newType, updateField.doc()));
|
||||
} else if (!oldField.type().equals(newType)) {
|
||||
newFields.add(Types.Field.get(oldField.fieldId(), oldField.isOptional(), oldField.name(), newType, oldField.doc()));
|
||||
} else {
|
||||
newFields.add(oldField);
|
||||
}
|
||||
}
|
||||
return Types.RecordType.get(newFields);
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
Type newElementType;
|
||||
Types.Field elementField = array.fields().get(0);
|
||||
newElementType = applyTableChange2Type(array.elementType(), updates);
|
||||
newElementType = updates.applyUpdates(elementField, newElementType);
|
||||
Types.Field elementUpdate = updates.getUpdates().get(elementField.fieldId());
|
||||
boolean optional = elementUpdate == null ? array.isElementOptional() : elementUpdate.isOptional();
|
||||
if (optional == elementField.isOptional() && array.elementType() == newElementType) {
|
||||
return array;
|
||||
}
|
||||
return Types.ArrayType.get(array.elementId(), optional, newElementType);
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
Types.Field valueFiled = map.fields().get(1);
|
||||
Type newValueType;
|
||||
newValueType = applyTableChange2Type(map.valueType(), updates);
|
||||
newValueType = updates.applyUpdates(valueFiled, newValueType);
|
||||
Types.Field valueUpdate = updates.getUpdates().get(valueFiled.fieldId());
|
||||
boolean valueOptional = valueUpdate == null ? map.isValueOptional() : valueUpdate.isOptional();
|
||||
if (valueOptional == map.isValueOptional() && map.valueType() == newValueType) {
|
||||
return map;
|
||||
}
|
||||
return Types.MapType.get(map.keyId(), map.valueId(), map.keyType(), newValueType, valueOptional);
|
||||
default:
|
||||
return type;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,351 @@
|
||||
/*
|
||||
* 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.internal.schema.utils;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.TreeMap;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class SerDeHelper {
|
||||
private SerDeHelper() {
|
||||
|
||||
}
|
||||
|
||||
public static final String LATEST_SCHEMA = "latest_schema";
|
||||
public static final String SCHEMAS = "schemas";
|
||||
private static final String MAX_COLUMN_ID = "max_column_id";
|
||||
private static final String VERSION_ID = "version_id";
|
||||
private static final String TYPE = "type";
|
||||
private static final String RECORD = "record";
|
||||
private static final String ARRAY = "array";
|
||||
private static final String MAP = "map";
|
||||
private static final String FIELDS = "fields";
|
||||
private static final String ELEMENT = "element";
|
||||
private static final String KEY = "key";
|
||||
private static final String VALUE = "value";
|
||||
private static final String DOC = "doc";
|
||||
private static final String NAME = "name";
|
||||
private static final String ID = "id";
|
||||
private static final String ELEMENT_ID = "element_id";
|
||||
private static final String KEY_ID = "key_id";
|
||||
private static final String VALUE_ID = "value_id";
|
||||
private static final String OPTIONAL = "optional";
|
||||
private static final String ELEMENT_OPTIONAL = "element_optional";
|
||||
private static final String VALUE_OPTIONAL = "value_optional";
|
||||
|
||||
private static final Pattern FIXED = Pattern.compile("fixed\\[(\\d+)\\]");
|
||||
private static final Pattern DECIMAL = Pattern.compile("decimal\\((\\d+),\\s+(\\d+)\\)");
|
||||
|
||||
/**
|
||||
* Convert history internalSchemas to json.
|
||||
* this is used when save history schemas into hudi.
|
||||
*
|
||||
* @param internalSchemas history internal schemas
|
||||
* @return a string
|
||||
*/
|
||||
public static String toJson(List<InternalSchema> internalSchemas) {
|
||||
try {
|
||||
StringWriter writer = new StringWriter();
|
||||
JsonGenerator generator = (new JsonFactory()).createGenerator(writer);
|
||||
generator.writeStartObject();
|
||||
generator.writeArrayFieldStart(SCHEMAS);
|
||||
for (InternalSchema schema : internalSchemas) {
|
||||
toJson(schema, generator);
|
||||
}
|
||||
generator.writeEndArray();
|
||||
generator.writeEndObject();
|
||||
generator.flush();
|
||||
return writer.toString();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert internalSchemas to json.
|
||||
*
|
||||
* @param internalSchema a internal schema
|
||||
* @return a string
|
||||
*/
|
||||
public static String toJson(InternalSchema internalSchema) {
|
||||
if (internalSchema == null || internalSchema.isEmptySchema()) {
|
||||
return "";
|
||||
}
|
||||
try {
|
||||
StringWriter writer = new StringWriter();
|
||||
JsonGenerator generator = (new JsonFactory()).createGenerator(writer);
|
||||
toJson(internalSchema, generator);
|
||||
generator.flush();
|
||||
return writer.toString();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static void toJson(InternalSchema internalSchema, JsonGenerator generator) throws IOException {
|
||||
toJson(internalSchema.getRecord(), internalSchema.getMaxColumnId(), internalSchema.schemaId(), generator);
|
||||
}
|
||||
|
||||
private static void toJson(Types.RecordType record, Integer maxColumnId, Long versionId, JsonGenerator generator) throws IOException {
|
||||
generator.writeStartObject();
|
||||
if (maxColumnId != null) {
|
||||
generator.writeNumberField(MAX_COLUMN_ID, maxColumnId);
|
||||
}
|
||||
if (versionId != null) {
|
||||
generator.writeNumberField(VERSION_ID, versionId);
|
||||
}
|
||||
generator.writeStringField(TYPE, RECORD);
|
||||
generator.writeArrayFieldStart(FIELDS);
|
||||
for (Types.Field field : record.fields()) {
|
||||
generator.writeStartObject();
|
||||
generator.writeNumberField(ID, field.fieldId());
|
||||
generator.writeStringField(NAME, field.name());
|
||||
generator.writeBooleanField(OPTIONAL, field.isOptional());
|
||||
generator.writeFieldName(TYPE);
|
||||
toJson(field.type(), generator);
|
||||
if (field.doc() != null) {
|
||||
generator.writeStringField(DOC, field.doc());
|
||||
}
|
||||
generator.writeEndObject();
|
||||
}
|
||||
generator.writeEndArray();
|
||||
generator.writeEndObject();
|
||||
}
|
||||
|
||||
private static void toJson(Type type, JsonGenerator generator) throws IOException {
|
||||
switch (type.typeId()) {
|
||||
case RECORD:
|
||||
toJson((Types.RecordType) type, null, null, generator);
|
||||
break;
|
||||
case ARRAY:
|
||||
Types.ArrayType array = (Types.ArrayType) type;
|
||||
generator.writeStartObject();
|
||||
generator.writeStringField(TYPE, ARRAY);
|
||||
generator.writeNumberField(ELEMENT_ID, array.elementId());
|
||||
generator.writeFieldName(ELEMENT);
|
||||
toJson(array.elementType(), generator);
|
||||
generator.writeBooleanField(ELEMENT_OPTIONAL, array.isElementOptional());
|
||||
generator.writeEndObject();
|
||||
break;
|
||||
case MAP:
|
||||
Types.MapType map = (Types.MapType) type;
|
||||
generator.writeStartObject();
|
||||
generator.writeStringField(TYPE, MAP);
|
||||
generator.writeNumberField(KEY_ID, map.keyId());
|
||||
generator.writeFieldName(KEY);
|
||||
toJson(map.keyType(), generator);
|
||||
generator.writeNumberField(VALUE_ID, map.valueId());
|
||||
generator.writeFieldName(VALUE);
|
||||
toJson(map.valueType(), generator);
|
||||
generator.writeBooleanField(VALUE_OPTIONAL, map.isValueOptional());
|
||||
generator.writeEndObject();
|
||||
break;
|
||||
default:
|
||||
if (!type.isNestedType()) {
|
||||
generator.writeString(type.toString());
|
||||
} else {
|
||||
throw new IllegalArgumentIOException(String.format("cannot write unknown types: %s", type));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static Type parserTypeFromJson(JsonNode jsonNode) {
|
||||
if (jsonNode.isTextual()) {
|
||||
String type = jsonNode.asText().toLowerCase(Locale.ROOT);
|
||||
// deal with fixed and decimal
|
||||
Matcher fixed = FIXED.matcher(type);
|
||||
if (fixed.matches()) {
|
||||
return Types.FixedType.getFixed(Integer.parseInt(fixed.group(1)));
|
||||
}
|
||||
Matcher decimal = DECIMAL.matcher(type);
|
||||
if (decimal.matches()) {
|
||||
return Types.DecimalType.get(
|
||||
Integer.parseInt(decimal.group(1)),
|
||||
Integer.parseInt(decimal.group(2)));
|
||||
}
|
||||
// deal with other type
|
||||
switch (Type.fromValue(type)) {
|
||||
case BOOLEAN:
|
||||
return Types.BooleanType.get();
|
||||
case INT:
|
||||
return Types.IntType.get();
|
||||
case LONG:
|
||||
return Types.LongType.get();
|
||||
case FLOAT:
|
||||
return Types.FloatType.get();
|
||||
case DOUBLE:
|
||||
return Types.DoubleType.get();
|
||||
case DATE:
|
||||
return Types.DateType.get();
|
||||
case TIME:
|
||||
return Types.TimeType.get();
|
||||
case TIMESTAMP:
|
||||
return Types.TimestampType.get();
|
||||
case STRING:
|
||||
return Types.StringType.get();
|
||||
case UUID:
|
||||
return Types.UUIDType.get();
|
||||
case BINARY:
|
||||
return Types.BinaryType.get();
|
||||
default:
|
||||
throw new IllegalArgumentException("cannot parser types from jsonNode");
|
||||
}
|
||||
} else if (jsonNode.isObject()) {
|
||||
String typeStr = jsonNode.get(TYPE).asText();
|
||||
if (RECORD.equals(typeStr)) {
|
||||
JsonNode fieldNodes = jsonNode.get(FIELDS);
|
||||
Iterator<JsonNode> iter = fieldNodes.elements();
|
||||
List<Types.Field> fields = new ArrayList<>();
|
||||
while (iter.hasNext()) {
|
||||
JsonNode field = iter.next();
|
||||
// extract
|
||||
int id = field.get(ID).asInt();
|
||||
String name = field.get(NAME).asText();
|
||||
Type type = parserTypeFromJson(field.get(TYPE));
|
||||
String doc = field.has(DOC) ? field.get(DOC).asText() : null;
|
||||
boolean optional = field.get(OPTIONAL).asBoolean();
|
||||
// build fields
|
||||
fields.add(Types.Field.get(id, optional, name, type, doc));
|
||||
}
|
||||
return Types.RecordType.get(fields);
|
||||
} else if (ARRAY.equals(typeStr)) {
|
||||
int elementId = jsonNode.get(ELEMENT_ID).asInt();
|
||||
Type elementType = parserTypeFromJson(jsonNode.get(ELEMENT));
|
||||
boolean optional = jsonNode.get(ELEMENT_OPTIONAL).asBoolean();
|
||||
return Types.ArrayType.get(elementId, optional, elementType);
|
||||
} else if (MAP.equals(typeStr)) {
|
||||
int keyId = jsonNode.get(KEY_ID).asInt();
|
||||
Type keyType = parserTypeFromJson(jsonNode.get(KEY));
|
||||
int valueId = jsonNode.get(VALUE_ID).asInt();
|
||||
Type valueType = parserTypeFromJson(jsonNode.get(VALUE));
|
||||
boolean optional = jsonNode.get(VALUE_OPTIONAL).asBoolean();
|
||||
return Types.MapType.get(keyId, valueId, keyType, valueType, optional);
|
||||
}
|
||||
}
|
||||
throw new IllegalArgumentException(String.format("cannot parse type from jsonNode: %s", jsonNode));
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert jsonNode to internalSchema.
|
||||
*
|
||||
* @param jsonNode a jsonNode.
|
||||
* @return a internalSchema.
|
||||
*/
|
||||
public static InternalSchema fromJson(JsonNode jsonNode) {
|
||||
Integer maxColumnId = !jsonNode.has(MAX_COLUMN_ID) ? null : jsonNode.get(MAX_COLUMN_ID).asInt();
|
||||
Long versionId = !jsonNode.has(VERSION_ID) ? null : jsonNode.get(VERSION_ID).asLong();
|
||||
Types.RecordType type = (Types.RecordType)parserTypeFromJson(jsonNode);
|
||||
if (versionId == null) {
|
||||
return new InternalSchema(type.fields());
|
||||
} else {
|
||||
if (maxColumnId != null) {
|
||||
return new InternalSchema(versionId, maxColumnId, type.fields());
|
||||
} else {
|
||||
return new InternalSchema(versionId, type.fields());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert string to internalSchema.
|
||||
*
|
||||
* @param json a json string.
|
||||
* @return a internalSchema.
|
||||
*/
|
||||
public static Option<InternalSchema> fromJson(String json) {
|
||||
if (json == null || json.isEmpty()) {
|
||||
return Option.empty();
|
||||
}
|
||||
try {
|
||||
return Option.of(fromJson((new ObjectMapper(new JsonFactory())).readValue(json, JsonNode.class)));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert json string to history internalSchemas.
|
||||
* TreeMap is used to hold history internalSchemas.
|
||||
*
|
||||
* @param json a json string
|
||||
* @return a TreeMap
|
||||
*/
|
||||
public static TreeMap<Long, InternalSchema> parseSchemas(String json) {
|
||||
TreeMap<Long, InternalSchema> result = new TreeMap<>();
|
||||
try {
|
||||
JsonNode jsonNode = (new ObjectMapper(new JsonFactory())).readValue(json, JsonNode.class);
|
||||
if (!jsonNode.has(SCHEMAS)) {
|
||||
throw new IllegalArgumentException(String.format("cannot parser schemas from current json string, missing key name: %s", SCHEMAS));
|
||||
}
|
||||
JsonNode schemas = jsonNode.get(SCHEMAS);
|
||||
Iterator<JsonNode> iter = schemas.elements();
|
||||
while (iter.hasNext()) {
|
||||
JsonNode schema = iter.next();
|
||||
InternalSchema current = fromJson(schema);
|
||||
result.put(current.schemaId(), current);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the new schema to the historical schemas.
|
||||
* use string operations to reduce overhead.
|
||||
*
|
||||
* @param newSchema a new internalSchema
|
||||
* @param oldSchemas historical schemas string.
|
||||
* @return a string.
|
||||
*/
|
||||
public static String inheritSchemas(InternalSchema newSchema, String oldSchemas) {
|
||||
if (newSchema == null) {
|
||||
return "";
|
||||
}
|
||||
if (oldSchemas == null || oldSchemas.isEmpty()) {
|
||||
return toJson(Arrays.asList(newSchema));
|
||||
}
|
||||
String checkedString = "{\"schemas\":[";
|
||||
if (!oldSchemas.startsWith("{\"schemas\":")) {
|
||||
return "";
|
||||
}
|
||||
String oldSchemasSuffix = oldSchemas.substring(checkedString.length());
|
||||
return checkedString + toJson(newSchema) + "," + oldSchemasSuffix;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,86 @@
|
||||
/*
|
||||
* 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.internal.schema.visitor;
|
||||
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Base class of schema visitor.
|
||||
*/
|
||||
public abstract class InternalSchemaVisitor<T> {
|
||||
|
||||
public void beforeField(Types.Field field) {
|
||||
}
|
||||
|
||||
public void afterField(Types.Field field) {
|
||||
}
|
||||
|
||||
public void beforeArrayElement(Types.Field elementField) {
|
||||
beforeField(elementField);
|
||||
}
|
||||
|
||||
public void afterArrayElement(Types.Field elementField) {
|
||||
afterField(elementField);
|
||||
}
|
||||
|
||||
public void beforeMapKey(Types.Field keyField) {
|
||||
beforeField(keyField);
|
||||
}
|
||||
|
||||
public void afterMapKey(Types.Field keyField) {
|
||||
afterField(keyField);
|
||||
}
|
||||
|
||||
public void beforeMapValue(Types.Field valueField) {
|
||||
beforeField(valueField);
|
||||
}
|
||||
|
||||
public void afterMapValue(Types.Field valueField) {
|
||||
afterField(valueField);
|
||||
}
|
||||
|
||||
public T schema(InternalSchema schema, T recordResult) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public T record(Types.RecordType record, List<T> fieldResults) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public T field(Types.Field field, T fieldResult) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public T array(Types.ArrayType array, T elementResult) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public T map(Types.MapType map, T keyResult, T valueResult) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public T primitive(Type.PrimitiveType primitive) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,113 @@
|
||||
/*
|
||||
* 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.internal.schema.visitor;
|
||||
|
||||
import static org.apache.hudi.internal.schema.utils.InternalSchemaUtils.createFullName;
|
||||
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Schema visitor to produce name -> id map for internalSchema.
|
||||
*/
|
||||
public class NameToIDVisitor extends InternalSchemaVisitor<Map<String, Integer>> {
|
||||
private final Deque fieldNames = new LinkedList<>();
|
||||
private final Map<String, Integer> nameToId = new HashMap<>();
|
||||
|
||||
@Override
|
||||
public void beforeField(Types.Field field) {
|
||||
fieldNames.push(field.name());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterField(Types.Field field) {
|
||||
fieldNames.pop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeArrayElement(Types.Field elementField) {
|
||||
fieldNames.push(elementField.name());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterArrayElement(Types.Field elementField) {
|
||||
fieldNames.pop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeMapKey(Types.Field keyField) {
|
||||
fieldNames.push(keyField.name());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterMapKey(Types.Field keyField) {
|
||||
fieldNames.pop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeMapValue(Types.Field valueField) {
|
||||
fieldNames.push(valueField.name());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterMapValue(Types.Field valueField) {
|
||||
fieldNames.pop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> schema(InternalSchema schema, Map<String, Integer> recordResult) {
|
||||
return nameToId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> record(Types.RecordType record, List<Map<String, Integer>> fieldResults) {
|
||||
return nameToId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> field(Types.Field field, Map<String, Integer> fieldResult) {
|
||||
nameToId.put(createFullName(field.name(), fieldNames), field.fieldId());
|
||||
return nameToId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> array(Types.ArrayType array, Map<String, Integer> elementResult) {
|
||||
nameToId.put(createFullName("element", fieldNames), array.elementId());
|
||||
return nameToId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> map(Types.MapType map, Map<String, Integer> keyResult, Map<String, Integer> valueResult) {
|
||||
nameToId.put(createFullName("key", fieldNames), map.keyId());
|
||||
nameToId.put(createFullName("value", fieldNames), map.valueId());
|
||||
return nameToId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> primitive(Type.PrimitiveType primitive) {
|
||||
return nameToId;
|
||||
}
|
||||
}
|
||||
@@ -30,6 +30,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SpillableMapUtils;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -65,7 +66,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
|
||||
Option<InstantRange> instantRange, boolean enableFullScan) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize,
|
||||
spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false,
|
||||
enableFullScan, Option.of(partitionName));
|
||||
enableFullScan, Option.of(partitionName), InternalSchema.getEmptyInternalSchema());
|
||||
this.mergeKeyFilter = mergeKeyFilter;
|
||||
if (enableFullScan) {
|
||||
performScan();
|
||||
|
||||
@@ -0,0 +1,117 @@
|
||||
/*
|
||||
* 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.internal.schema;
|
||||
|
||||
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class TestSerDeHelper {
|
||||
|
||||
@Test
|
||||
public void testComplexSchema2Json() {
|
||||
InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(7, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(8, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(13, true,
|
||||
Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))),
|
||||
Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())),
|
||||
Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get()))
|
||||
);
|
||||
// test schema2json
|
||||
String result = SerDeHelper.toJson(internalSchema);
|
||||
InternalSchema convertedSchema = SerDeHelper.fromJson(result).get();
|
||||
Assertions.assertEquals(internalSchema, convertedSchema);
|
||||
// test schemas2json
|
||||
String results = SerDeHelper.toJson(Arrays.asList(internalSchema));
|
||||
TreeMap<Long, InternalSchema> convertedSchemas = SerDeHelper.parseSchemas(results);
|
||||
Assertions.assertEquals(1, convertedSchemas.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPrimitive2Json() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "bool", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "int", Types.IntType.get()),
|
||||
Types.Field.get(2, "long", Types.LongType.get()),
|
||||
Types.Field.get(3, "float", Types.FloatType.get()),
|
||||
Types.Field.get(4, "double", Types.DoubleType.get()),
|
||||
Types.Field.get(5, "date", Types.DateType.get()),
|
||||
Types.Field.get(6, "time", Types.TimeType.get()),
|
||||
Types.Field.get(7, "timestamp", Types.TimestampType.get()),
|
||||
Types.Field.get(8, "string", Types.StringType.get()),
|
||||
Types.Field.get(9, "uuid", Types.UUIDType.get()),
|
||||
Types.Field.get(10, "fixed", Types.FixedType.getFixed(10)),
|
||||
Types.Field.get(11, "binary", Types.BinaryType.get()),
|
||||
Types.Field.get(12, "decimal", Types.DecimalType.get(10, 2))
|
||||
}));
|
||||
InternalSchema internalSchema = new InternalSchema(record.fields());
|
||||
String result = SerDeHelper.toJson(internalSchema);
|
||||
InternalSchema convertedSchema = SerDeHelper.fromJson(result).get();
|
||||
Assertions.assertEquals(internalSchema, convertedSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSearchSchema() {
|
||||
List schemas = new ArrayList<>();
|
||||
for (int i = 0; i < 100; i++) {
|
||||
schemas.add(new InternalSchema(i * 10,
|
||||
Arrays.asList(Types.Field.get(1, true, "schema" + i * 10, Types.LongType.get()))));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(InternalSchemaUtils.searchSchema(0, schemas).getRecord().fields().get(0),
|
||||
Types.Field.get(1, true, "schema" + 0, Types.LongType.get()));
|
||||
|
||||
Assertions.assertEquals(InternalSchemaUtils.searchSchema(9, schemas).getRecord().fields().get(0),
|
||||
Types.Field.get(1, true, "schema" + 0, Types.LongType.get()));
|
||||
|
||||
Assertions.assertEquals(InternalSchemaUtils.searchSchema(99, schemas).getRecord().fields().get(0),
|
||||
Types.Field.get(1, true, "schema" + 90, Types.LongType.get()));
|
||||
|
||||
Assertions.assertEquals(InternalSchemaUtils.searchSchema(9999, schemas).getRecord().fields().get(0),
|
||||
Types.Field.get(1, true, "schema" + 990, Types.LongType.get()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInheritSchemas() {
|
||||
List schemas = new ArrayList<>();
|
||||
for (int i = 0; i < 2; i++) {
|
||||
schemas.add(new InternalSchema(i,
|
||||
Arrays.asList(Types.Field.get(1, true, "schema" + i, Types.LongType.get()))));
|
||||
}
|
||||
String oldSchemas = SerDeHelper.toJson(schemas);
|
||||
InternalSchema newSchema = new InternalSchema(3,
|
||||
Arrays.asList(Types.Field.get(1, true, "schema" + 3, Types.LongType.get())));
|
||||
|
||||
String finalResult = SerDeHelper.inheritSchemas(newSchema, oldSchemas);
|
||||
// convert back
|
||||
Assertions.assertEquals(SerDeHelper.parseSchemas(finalResult).size(), 3);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,88 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import org.apache.hudi.internal.schema.utils.SchemaChangeUtils;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class TestMergeSchema {
|
||||
|
||||
@Test
|
||||
public void testPrimitiveMerge() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "col1", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "col2", Types.IntType.get()),
|
||||
Types.Field.get(2, "col3", Types.LongType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get())}));
|
||||
|
||||
InternalSchema oldSchema = new InternalSchema(record.fields());
|
||||
// add c1 after 'col1', and c2 before 'col3'
|
||||
TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema);
|
||||
addChange.addColumns("c1", Types.BooleanType.get(), "add c1 after col1");
|
||||
addChange.addPositionChange("c1", "col1", "after");
|
||||
addChange.addColumns("c2", Types.IntType.get(), "add c2 before col3");
|
||||
addChange.addPositionChange("c2", "col3", "before");
|
||||
InternalSchema newAddSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange);
|
||||
TableChanges.ColumnDeleteChange deleteChange = TableChanges.ColumnDeleteChange.get(newAddSchema);
|
||||
deleteChange.deleteColumn("col1");
|
||||
deleteChange.deleteColumn("col3");
|
||||
InternalSchema newDeleteSchema = SchemaChangeUtils.applyTableChanges2Schema(newAddSchema, deleteChange);
|
||||
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(newDeleteSchema);
|
||||
updateChange.updateColumnType("col2", Types.LongType.get())
|
||||
.updateColumnComment("col2", "alter col2 comments")
|
||||
.renameColumn("col2", "colx").addPositionChange("col2",
|
||||
"col4", "after");
|
||||
InternalSchema updateSchema = SchemaChangeUtils.applyTableChanges2Schema(newDeleteSchema, updateChange);
|
||||
|
||||
// add col1 again
|
||||
TableChanges.ColumnAddChange addChange1 = TableChanges.ColumnAddChange.get(updateSchema);
|
||||
addChange1.addColumns("col1", Types.BooleanType.get(), "add new col1");
|
||||
InternalSchema finalSchema = SchemaChangeUtils.applyTableChanges2Schema(updateSchema, addChange1);
|
||||
// merge schema by using columnType from query schema
|
||||
InternalSchema mergeSchema = new InternalSchemaMerger(oldSchema, finalSchema, true, false).mergeSchema();
|
||||
|
||||
InternalSchema checkedSchema = new InternalSchema(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(4, true, "c1", Types.BooleanType.get(), "add c1 after col1"),
|
||||
Types.Field.get(5, true, "c2", Types.IntType.get(), "add c2 before col3"),
|
||||
Types.Field.get(3, true, "col4", Types.FloatType.get()),
|
||||
Types.Field.get(1, true, "col2", Types.LongType.get(), "alter col2 comments"),
|
||||
Types.Field.get(6, true, "col1suffix", Types.BooleanType.get(), "add new col1")
|
||||
}));
|
||||
Assertions.assertEquals(mergeSchema, checkedSchema);
|
||||
|
||||
// merge schema by using columnType from file schema
|
||||
InternalSchema mergeSchema1 = new InternalSchemaMerger(oldSchema, finalSchema, true, true).mergeSchema();
|
||||
InternalSchema checkedSchema1 = new InternalSchema(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(4, true, "c1", Types.BooleanType.get(), "add c1 after col1"),
|
||||
Types.Field.get(5, true, "c2", Types.IntType.get(), "add c2 before col3"),
|
||||
Types.Field.get(3, true, "col4", Types.FloatType.get()),
|
||||
Types.Field.get(1, true, "col2", Types.IntType.get(), "alter col2 comments"),
|
||||
Types.Field.get(6, true, "col1suffix", Types.BooleanType.get(), "add new col1")
|
||||
}));
|
||||
Assertions.assertEquals(mergeSchema1, checkedSchema1);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,229 @@
|
||||
/*
|
||||
* 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.internal.schema.action;
|
||||
|
||||
import org.apache.hudi.internal.schema.HoodieSchemaException;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
|
||||
import org.apache.hudi.internal.schema.utils.SchemaChangeUtils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class TestTableChanges {
|
||||
|
||||
@Test
|
||||
public void testPrimitiveAdd() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "col1", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "col2", Types.IntType.get()),
|
||||
Types.Field.get(2, "col3", Types.LongType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get())}));
|
||||
|
||||
Types.RecordType checkRecord = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "col1", Types.BooleanType.get()),
|
||||
Types.Field.get(4, true, "c1", Types.BooleanType.get(), "add c1 after col1"),
|
||||
Types.Field.get(1, "col2", Types.IntType.get()),
|
||||
Types.Field.get(5, true, "c2", Types.IntType.get(), "add c2 before col3"),
|
||||
Types.Field.get(2, "col3", Types.LongType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get())}));
|
||||
|
||||
InternalSchema oldSchema = new InternalSchema(record.fields());
|
||||
// add c1 after 'col1', and c2 before 'col3'
|
||||
TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema);
|
||||
addChange.addColumns("c1", Types.BooleanType.get(), "add c1 after col1");
|
||||
// check repeated add.
|
||||
Assertions.assertThrows(HoodieSchemaException.class, () -> addChange.addColumns("c1", Types.BooleanType.get(), "add c1 after col1"));
|
||||
addChange.addPositionChange("c1", "col1", "after");
|
||||
addChange.addColumns("c2", Types.IntType.get(), "add c2 before col3");
|
||||
addChange.addPositionChange("c2", "col3", "before");
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange);
|
||||
Assertions.assertEquals(newSchema.getRecord(), checkRecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNestAdd() {
|
||||
InternalSchema oldSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(7, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(8, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(13, true,
|
||||
Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))),
|
||||
Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())),
|
||||
Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get()))
|
||||
);
|
||||
|
||||
TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema);
|
||||
// add c1 first
|
||||
addChange.addColumns("c1", Types.StringType.get(), "add c1 first");
|
||||
addChange.addPositionChange("c1", "id", "before");
|
||||
//add preferences.cx before preferences.feature2
|
||||
addChange.addColumns("preferences", "cx", Types.BooleanType.get(), "add preferences.cx before preferences.feature2");
|
||||
// check repeated add.
|
||||
Assertions.assertThrows(HoodieSchemaException.class, () -> addChange.addColumns("preferences", "cx", Types.BooleanType.get(), "add preferences.cx before preferences.feature2"));
|
||||
addChange.addPositionChange("preferences.cx", "preferences.feature2", "before");
|
||||
// add locations.value.lax before locations.value.long
|
||||
addChange.addColumns("locations.value", "lax", Types.BooleanType.get(), "add locations.value.lax before locations.value.long");
|
||||
addChange.addPositionChange("locations.value.lax", "locations.value.long", "before");
|
||||
//
|
||||
// add points.element.z after points.element.y
|
||||
addChange.addColumns("points.element", "z", Types.BooleanType.get(), "add points.element.z after points.element.y");
|
||||
addChange.addPositionChange("points.element.z", "points.element.y", "after");
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange);
|
||||
InternalSchema checkedSchema = new InternalSchema(
|
||||
Types.Field.get(19, true, "c1", Types.StringType.get(), "add c1 first"),
|
||||
Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(7, false, "feature1", Types.BooleanType.get()),
|
||||
Types.Field.get(20, true, "cx", Types.BooleanType.get(), "add preferences.cx before preferences.feature2"),
|
||||
Types.Field.get(8, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()),
|
||||
Types.Field.get(21, true, "lax", Types.BooleanType.get(), "add locations.value.lax before locations.value.long"),
|
||||
Types.Field.get(12, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(13, true,
|
||||
Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()),
|
||||
Types.Field.get(15, false, "y", Types.LongType.get()),
|
||||
Types.Field.get(22, true, "z", Types.BooleanType.get(), "add points.element.z after points.element.y")))),
|
||||
Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())),
|
||||
Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get()))
|
||||
);
|
||||
Assertions.assertEquals(newSchema.getRecord(), checkedSchema.getRecord());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPrimitiveDelete() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "col1", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "col2", Types.IntType.get()),
|
||||
Types.Field.get(2, "col3", Types.LongType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get())}));
|
||||
InternalSchema oldSchema = new InternalSchema(record.fields());
|
||||
TableChanges.ColumnDeleteChange deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema);
|
||||
deleteChange.deleteColumn("col1");
|
||||
// check repeated delete.
|
||||
// deletechange can handle deleting the same column multiple times, only keep one operation.
|
||||
deleteChange.deleteColumn("col1");
|
||||
deleteChange.deleteColumn("col3");
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange);
|
||||
Types.RecordType checkRecord = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(1, "col2", Types.IntType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get())}));
|
||||
Assertions.assertEquals(newSchema.getRecord(), checkRecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNestDelete() {
|
||||
InternalSchema oldSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(5, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(9, false, "lat", Types.FloatType.get()), Types.Field.get(10, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(11, true,
|
||||
Types.RecordType.get(Types.Field.get(12, false, "x", Types.LongType.get()), Types.Field.get(13, false, "y", Types.LongType.get()))))
|
||||
);
|
||||
TableChanges.ColumnDeleteChange deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema);
|
||||
deleteChange.deleteColumn("data");
|
||||
deleteChange.deleteColumn("preferences.feature2");
|
||||
deleteChange.deleteColumn("preferences.feature2");
|
||||
deleteChange.deleteColumn("locations.value.lat");
|
||||
deleteChange.deleteColumn("points.element.y");
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange);
|
||||
InternalSchema checkedSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(5, false, "feature1",
|
||||
Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(10, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(11, true,
|
||||
Types.RecordType.get(Types.Field.get(12, false, "x", Types.LongType.get()))))
|
||||
);
|
||||
Assertions.assertEquals(newSchema.getRecord(), checkedSchema.getRecord());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPrimitiveUpdate() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "col1", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "col2", Types.IntType.get()),
|
||||
Types.Field.get(2, "col3", Types.LongType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get())}));
|
||||
InternalSchema oldSchema = new InternalSchema(record.fields());
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(oldSchema);
|
||||
updateChange.updateColumnType("col2", Types.LongType.get())
|
||||
.updateColumnComment("col2", "alter col2 comments")
|
||||
.renameColumn("col2", "colx").addPositionChange("col2", "col4", "after");
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange);
|
||||
Types.RecordType checkedRecord = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "col1", Types.BooleanType.get()),
|
||||
Types.Field.get(2, "col3", Types.LongType.get()),
|
||||
Types.Field.get(3, "col4", Types.FloatType.get()),
|
||||
Types.Field.get(1, true, "colx", Types.LongType.get(), "alter col2 comments")}));
|
||||
Assertions.assertEquals(newSchema.getRecord(), checkedRecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNestUpdate() {
|
||||
InternalSchema oldSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(5, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(9, false, "lat", Types.FloatType.get()), Types.Field.get(10, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(11, true,
|
||||
Types.RecordType.get(Types.Field.get(12, false, "x", Types.LongType.get()), Types.Field.get(13, false, "y", Types.LongType.get()))))
|
||||
);
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(oldSchema);
|
||||
updateChange
|
||||
.updateColumnNullability("id", true)
|
||||
.renameColumn("id", "idx")
|
||||
.addPositionChange("data", "points", "after");
|
||||
updateChange
|
||||
.updateColumnComment("preferences.feature1", "add feature1 comment")
|
||||
.renameColumn("preferences.feature1", "f1")
|
||||
.addPositionChange("preferences.feature1", "preferences.feature1", "first");
|
||||
updateChange.updateColumnComment("locations.value.lat", "add lat comment")
|
||||
.renameColumn("locations.value.lat", "lax")
|
||||
.addPositionChange("locations.value.lat", "locations.value.lat", "first");
|
||||
updateChange.renameColumn("points.element.x", "z")
|
||||
.addPositionChange("points.element.x", "points.element.y", "after");
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange);
|
||||
InternalSchema checkSchema = new InternalSchema(Types.Field.get(0, true, "idx", Types.IntType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(5, false, "f1",
|
||||
Types.BooleanType.get(), "add feature1 comment"), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(7, 8, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(9, false, "lax", Types.FloatType.get(), "add lat comment"), Types.Field.get(10, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(11, true,
|
||||
Types.RecordType.get(Types.Field.get(13, false, "y", Types.LongType.get()), Types.Field.get(12, false, "z", Types.LongType.get())))),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get())
|
||||
);
|
||||
Assertions.assertEquals(newSchema.getRecord(), checkSchema.getRecord());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.internal.schema.io;
|
||||
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Tests {@link FileBasedInternalSchemaStorageManager}.
|
||||
*/
|
||||
public class TestFileBasedInternalSchemaStorageManager extends HoodieCommonTestHarness {
|
||||
private HoodieActiveTimeline timeline;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPersistAndReadHistorySchemaStr() throws IOException {
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
FileBasedInternalSchemaStorageManager fm = new FileBasedInternalSchemaStorageManager(metaClient);
|
||||
InternalSchema currentSchema = getSimpleSchema();
|
||||
currentSchema.setSchemaId(0L);
|
||||
// save first schema.
|
||||
fm.persistHistorySchemaStr("0000", SerDeHelper.inheritSchemas(currentSchema, ""));
|
||||
// Simulate commit.
|
||||
simulateCommit("0000");
|
||||
metaClient.reloadActiveTimeline();
|
||||
// try to read schema
|
||||
InternalSchema readSchema = fm.getSchemaByKey("0").get();
|
||||
assertEquals(currentSchema, readSchema);
|
||||
// save history schema again
|
||||
InternalSchema secondSchema = getSimpleSchema();
|
||||
secondSchema.setSchemaId(1L);
|
||||
fm.persistHistorySchemaStr("0001", SerDeHelper.inheritSchemas(secondSchema, fm.getHistorySchemaStr()));
|
||||
// Simulate commit.
|
||||
simulateCommit("0001");
|
||||
metaClient.reloadActiveTimeline();
|
||||
// try to read schema
|
||||
assertEquals(secondSchema, fm.getSchemaByKey("1").get());
|
||||
|
||||
// test write failed and residual file clean.
|
||||
InternalSchema thirdSchema = getSimpleSchema();
|
||||
thirdSchema.setSchemaId(2L);
|
||||
fm.persistHistorySchemaStr("0002", SerDeHelper.inheritSchemas(thirdSchema, fm.getHistorySchemaStr()));
|
||||
// do not simulate commit "0002", so current save file will be residual files.
|
||||
// try 4st persist
|
||||
InternalSchema lastSchema = getSimpleSchema();
|
||||
lastSchema.setSchemaId(3L);
|
||||
fm.persistHistorySchemaStr("0004", SerDeHelper.inheritSchemas(lastSchema, fm.getHistorySchemaStr()));
|
||||
simulateCommit("0004");
|
||||
metaClient.reloadActiveTimeline();
|
||||
// now the residual file created by 3st persist should be removed.
|
||||
File f = new File(metaClient.getSchemaFolderName() + File.separator + "0002.schemacommit");
|
||||
assertTrue(!f.exists());
|
||||
assertEquals(lastSchema, fm.getSchemaByKey("3").get());
|
||||
}
|
||||
|
||||
private void simulateCommit(String commitTime) {
|
||||
if (timeline == null) {
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
}
|
||||
HoodieInstant instant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
timeline.createNewInstant(instant);
|
||||
timeline.transitionRequestedToInflight(instant, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant.getAction(), instant.getTimestamp()),
|
||||
Option.empty());
|
||||
}
|
||||
|
||||
private InternalSchema getSimpleSchema() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "bool", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "int", Types.IntType.get()),
|
||||
}));
|
||||
return new InternalSchema(record.fields());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,422 @@
|
||||
/*
|
||||
* 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.internal.schema.utils;
|
||||
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.SchemaBuilder;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
|
||||
import org.apache.hudi.internal.schema.Type;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
import org.apache.hudi.internal.schema.action.TableChanges;
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class TestAvroSchemaEvolutionUtils {
|
||||
|
||||
@Test
|
||||
public void testPrimitiveTypes() {
|
||||
Schema[] avroPrimitives = new Schema[] {
|
||||
Schema.create(Schema.Type.BOOLEAN),
|
||||
Schema.create(Schema.Type.INT),
|
||||
Schema.create(Schema.Type.LONG),
|
||||
Schema.create(Schema.Type.FLOAT),
|
||||
Schema.create(Schema.Type.DOUBLE),
|
||||
LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)),
|
||||
LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)),
|
||||
LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)),
|
||||
Schema.create(Schema.Type.STRING),
|
||||
LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16)),
|
||||
Schema.createFixed("fixed_12", null, null, 12),
|
||||
Schema.create(Schema.Type.BYTES),
|
||||
LogicalTypes.decimal(9, 4).addToSchema(Schema.createFixed("decimal_9_4", null, null, 4))};
|
||||
|
||||
Type[] primitiveTypes = new Type[] {
|
||||
Types.BooleanType.get(),
|
||||
Types.IntType.get(),
|
||||
Types.LongType.get(),
|
||||
Types.FloatType.get(),
|
||||
Types.DoubleType.get(),
|
||||
Types.DateType.get(),
|
||||
Types.TimeType.get(),
|
||||
Types.TimestampType.get(),
|
||||
Types.StringType.get(),
|
||||
Types.UUIDType.get(),
|
||||
Types.FixedType.getFixed(12),
|
||||
Types.BinaryType.get(),
|
||||
Types.DecimalType.get(9, 4)
|
||||
};
|
||||
|
||||
for (int i = 0; i < primitiveTypes.length; i++) {
|
||||
Type convertPrimitiveResult = AvroInternalSchemaConverter.convertToField(avroPrimitives[i]);
|
||||
Assertions.assertEquals(convertPrimitiveResult, primitiveTypes[i]);
|
||||
Schema convertResult = AvroInternalSchemaConverter.convert(primitiveTypes[i], "t1");
|
||||
Assertions.assertEquals(convertResult, avroPrimitives[i]);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordAndPrimitiveTypes() {
|
||||
Types.RecordType record = Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "bool", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "int", Types.IntType.get()),
|
||||
Types.Field.get(2, "long", Types.LongType.get()),
|
||||
Types.Field.get(3, "float", Types.FloatType.get()),
|
||||
Types.Field.get(4, "double", Types.DoubleType.get()),
|
||||
Types.Field.get(5, "date", Types.DateType.get()),
|
||||
Types.Field.get(6, "time", Types.TimeType.get()),
|
||||
Types.Field.get(7, "timestamp", Types.TimestampType.get()),
|
||||
Types.Field.get(8, "string", Types.StringType.get()),
|
||||
Types.Field.get(9, "uuid", Types.UUIDType.get()),
|
||||
Types.Field.get(10, "fixed", Types.FixedType.getFixed(10)),
|
||||
Types.Field.get(11, "binary", Types.BinaryType.get()),
|
||||
Types.Field.get(12, "decimal", Types.DecimalType.get(10, 2))
|
||||
}));
|
||||
|
||||
Schema schema = create("t1",
|
||||
new Schema.Field("bool", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BOOLEAN)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("int", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.INT)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("long", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("float", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.FLOAT)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("double", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.DOUBLE)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("date", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("time", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG))), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("timestamp", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("string", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.STRING)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("uuid", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16))), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("fixed", AvroInternalSchemaConverter.nullableSchema(Schema.createFixed("fixed_10", null, null, 10)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("binary", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BYTES)), null, JsonProperties.NULL_VALUE),
|
||||
new Schema.Field("decimal", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.decimal(10, 2)
|
||||
.addToSchema(Schema.createFixed("decimal_10_2", null, null, 5))), null, JsonProperties.NULL_VALUE));
|
||||
Schema convertedSchema = AvroInternalSchemaConverter.convert(record, "t1");
|
||||
Assertions.assertEquals(convertedSchema, schema);
|
||||
Types.RecordType convertedRecord = AvroInternalSchemaConverter.convert(schema).getRecord();
|
||||
Assertions.assertEquals(convertedRecord, record);
|
||||
}
|
||||
|
||||
private Schema create(String name, Schema.Field... fields) {
|
||||
return Schema.createRecord(name, null, null, false, Arrays.asList(fields));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayType() {
|
||||
Type arrayNestRecordType = Types.ArrayType.get(1, false,
|
||||
Types.RecordType.get(Arrays.asList(Types.Field.get(2, false, "a", Types.FloatType.get()),
|
||||
Types.Field.get(3, false, "b", Types.FloatType.get()))));
|
||||
|
||||
Schema schema = SchemaBuilder.array().items(create("t1",
|
||||
new Schema.Field("a", Schema.create(Schema.Type.FLOAT), null, null),
|
||||
new Schema.Field("b", Schema.create(Schema.Type.FLOAT), null, null)));
|
||||
Schema convertedSchema = AvroInternalSchemaConverter.convert(arrayNestRecordType, "t1");
|
||||
Assertions.assertEquals(convertedSchema, schema);
|
||||
Types.ArrayType convertedRecord = (Types.ArrayType) AvroInternalSchemaConverter.convertToField(schema);
|
||||
Assertions.assertEquals(convertedRecord, arrayNestRecordType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComplexConvert() {
|
||||
String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\","
|
||||
+ "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\","
|
||||
+ "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\","
|
||||
+ "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}],"
|
||||
+ "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\","
|
||||
+ "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\","
|
||||
+ "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\","
|
||||
+ "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"},"
|
||||
+ "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}},"
|
||||
+ "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}";
|
||||
Schema schema = new Schema.Parser().parse(schemaStr);
|
||||
|
||||
InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(7, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(8, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)),
|
||||
Types.Field.get(4, true, "points", Types.ArrayType.get(13, true,
|
||||
Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))),
|
||||
Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())),
|
||||
Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get()))
|
||||
);
|
||||
|
||||
Type convertRecord = AvroInternalSchemaConverter.convert(schema).getRecord();
|
||||
Assertions.assertEquals(convertRecord, internalSchema.getRecord());
|
||||
Assertions.assertEquals(schema, AvroInternalSchemaConverter.convert(internalSchema, "newTableName"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRefreshNewId() {
|
||||
Types.RecordType record = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(4, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(5, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false, "locations", Types.MapType.get(6, 7, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(8, false, "lat", Types.FloatType.get()), Types.Field.get(9, false, "long", Types.FloatType.get())), false))
|
||||
);
|
||||
AtomicInteger newId = new AtomicInteger(100);
|
||||
Types.RecordType recordWithNewId = (Types.RecordType) InternalSchemaBuilder.getBuilder().refreshNewId(record, newId);
|
||||
|
||||
Types.RecordType newRecord = Types.RecordType.get(Types.Field.get(100, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(101, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(102, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(104, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(105, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(103, false, "locations", Types.MapType.get(106, 107, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(108, false, "lat", Types.FloatType.get()), Types.Field.get(109, false, "long", Types.FloatType.get())), false))
|
||||
);
|
||||
Assertions.assertEquals(newRecord, recordWithNewId);
|
||||
}
|
||||
|
||||
/**
|
||||
* test record data type changes.
|
||||
* int => long/float/double/string
|
||||
* long => float/double/string
|
||||
* float => double/String
|
||||
* double => String/Decimal
|
||||
* Decimal => Decimal/String
|
||||
* String => date/decimal
|
||||
* date => String
|
||||
*/
|
||||
@Test
|
||||
public void testReWriteRecordWithTypeChanged() {
|
||||
Schema avroSchema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"h0_record\",\"namespace\":\"hoodie.h0\",\"fields\""
|
||||
+ ":[{\"name\":\"id\",\"type\":[\"null\",\"int\"],\"default\":null},"
|
||||
+ "{\"name\":\"comb\",\"type\":[\"null\",\"int\"],\"default\":null},"
|
||||
+ "{\"name\":\"com1\",\"type\":[\"null\",\"int\"],\"default\":null},"
|
||||
+ "{\"name\":\"col0\",\"type\":[\"null\",\"int\"],\"default\":null},"
|
||||
+ "{\"name\":\"col1\",\"type\":[\"null\",\"long\"],\"default\":null},"
|
||||
+ "{\"name\":\"col11\",\"type\":[\"null\",\"long\"],\"default\":null},"
|
||||
+ "{\"name\":\"col12\",\"type\":[\"null\",\"long\"],\"default\":null},"
|
||||
+ "{\"name\":\"col2\",\"type\":[\"null\",\"float\"],\"default\":null},"
|
||||
+ "{\"name\":\"col21\",\"type\":[\"null\",\"float\"],\"default\":null},"
|
||||
+ "{\"name\":\"col3\",\"type\":[\"null\",\"double\"],\"default\":null},"
|
||||
+ "{\"name\":\"col31\",\"type\":[\"null\",\"double\"],\"default\":null},"
|
||||
+ "{\"name\":\"col4\",\"type\":[\"null\",{\"type\":\"fixed\",\"name\":\"fixed\",\"namespace\":\"hoodie.h0.h0_record.col4\","
|
||||
+ "\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":4}],\"default\":null},"
|
||||
+ "{\"name\":\"col41\",\"type\":[\"null\",{\"type\":\"fixed\",\"name\":\"fixed\",\"namespace\":\"hoodie.h0.h0_record.col41\","
|
||||
+ "\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":4}],\"default\":null},"
|
||||
+ "{\"name\":\"col5\",\"type\":[\"null\",\"string\"],\"default\":null},"
|
||||
+ "{\"name\":\"col51\",\"type\":[\"null\",\"string\"],\"default\":null},"
|
||||
+ "{\"name\":\"col6\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null},"
|
||||
+ "{\"name\":\"col7\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}],\"default\":null},"
|
||||
+ "{\"name\":\"col8\",\"type\":[\"null\",\"boolean\"],\"default\":null},"
|
||||
+ "{\"name\":\"col9\",\"type\":[\"null\",\"bytes\"],\"default\":null},{\"name\":\"par\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}]}");
|
||||
// create a test record with avroSchema
|
||||
GenericData.Record avroRecord = new GenericData.Record(avroSchema);
|
||||
avroRecord.put("id", 1);
|
||||
avroRecord.put("comb", 100);
|
||||
avroRecord.put("com1", -100);
|
||||
avroRecord.put("col0", 256);
|
||||
avroRecord.put("col1", 1000L);
|
||||
avroRecord.put("col11", -100L);
|
||||
avroRecord.put("col12", 2000L);
|
||||
avroRecord.put("col2", -5.001f);
|
||||
avroRecord.put("col21", 5.001f);
|
||||
avroRecord.put("col3", 12.999d);
|
||||
avroRecord.put("col31", 9999.999d);
|
||||
Schema currentDecimalType = avroSchema.getField("col4").schema().getTypes().get(1);
|
||||
BigDecimal bd = new BigDecimal("123.456").setScale(((LogicalTypes.Decimal) currentDecimalType.getLogicalType()).getScale());
|
||||
avroRecord.put("col4", HoodieAvroUtils.DECIMAL_CONVERSION.toFixed(bd, currentDecimalType, currentDecimalType.getLogicalType()));
|
||||
Schema currentDecimalType1 = avroSchema.getField("col41").schema().getTypes().get(1);
|
||||
BigDecimal bd1 = new BigDecimal("7890.456").setScale(((LogicalTypes.Decimal) currentDecimalType1.getLogicalType()).getScale());
|
||||
avroRecord.put("col41", HoodieAvroUtils.DECIMAL_CONVERSION.toFixed(bd1, currentDecimalType1, currentDecimalType1.getLogicalType()));
|
||||
|
||||
avroRecord.put("col5", "2011-01-01");
|
||||
avroRecord.put("col51", "199.342");
|
||||
avroRecord.put("col6", 18987);
|
||||
avroRecord.put("col7", 1640491505000000L);
|
||||
avroRecord.put("col8", false);
|
||||
ByteBuffer bb = ByteBuffer.wrap(new byte[] {97, 48, 53});
|
||||
avroRecord.put("col9", bb);
|
||||
Assertions.assertEquals(GenericData.get().validate(avroSchema, avroRecord), true);
|
||||
InternalSchema internalSchema = AvroInternalSchemaConverter.convert(avroSchema);
|
||||
// do change type operation
|
||||
TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(internalSchema);
|
||||
updateChange
|
||||
.updateColumnType("id", Types.LongType.get())
|
||||
.updateColumnType("comb", Types.FloatType.get())
|
||||
.updateColumnType("com1", Types.DoubleType.get())
|
||||
.updateColumnType("col0", Types.StringType.get())
|
||||
.updateColumnType("col1", Types.FloatType.get())
|
||||
.updateColumnType("col11", Types.DoubleType.get())
|
||||
.updateColumnType("col12", Types.StringType.get())
|
||||
.updateColumnType("col2", Types.DoubleType.get())
|
||||
.updateColumnType("col21", Types.StringType.get())
|
||||
.updateColumnType("col3", Types.StringType.get())
|
||||
.updateColumnType("col31", Types.DecimalType.get(18, 9))
|
||||
.updateColumnType("col4", Types.DecimalType.get(18, 9))
|
||||
.updateColumnType("col41", Types.StringType.get())
|
||||
.updateColumnType("col5", Types.DateType.get())
|
||||
.updateColumnType("col51", Types.DecimalType.get(18, 9))
|
||||
.updateColumnType("col6", Types.StringType.get());
|
||||
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange);
|
||||
Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName());
|
||||
GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema);
|
||||
|
||||
Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newRecord), true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReWriteNestRecord() {
|
||||
Types.RecordType record = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(5, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())),
|
||||
Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(10, false, "lat", Types.FloatType.get()), Types.Field.get(11, false, "long", Types.FloatType.get())), false))
|
||||
);
|
||||
Schema schema = AvroInternalSchemaConverter.convert(record, "test1");
|
||||
GenericData.Record avroRecord = new GenericData.Record(schema);
|
||||
GenericData.get().validate(schema, avroRecord);
|
||||
avroRecord.put("id", 2);
|
||||
avroRecord.put("data", "xs");
|
||||
// fill record type
|
||||
GenericData.Record preferencesRecord = new GenericData.Record(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences"));
|
||||
preferencesRecord.put("feature1", false);
|
||||
preferencesRecord.put("feature2", true);
|
||||
Assertions.assertEquals(GenericData.get().validate(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences"), preferencesRecord), true);
|
||||
avroRecord.put("preferences", preferencesRecord);
|
||||
// fill mapType
|
||||
Map<String, GenericData.Record> locations = new HashMap<>();
|
||||
Schema mapSchema = AvroInternalSchemaConverter.convert(((Types.MapType)record.field("locations").type()).valueType(), "test1_locations");
|
||||
GenericData.Record locationsValue = new GenericData.Record(mapSchema);
|
||||
locationsValue.put("lat", 1.2f);
|
||||
locationsValue.put("long", 1.4f);
|
||||
GenericData.Record locationsValue1 = new GenericData.Record(mapSchema);
|
||||
locationsValue1.put("lat", 2.2f);
|
||||
locationsValue1.put("long", 2.4f);
|
||||
locations.put("key1", locationsValue);
|
||||
locations.put("key2", locationsValue1);
|
||||
avroRecord.put("locations", locations);
|
||||
|
||||
List<Double> doubles = new ArrayList<>();
|
||||
doubles.add(2.0d);
|
||||
doubles.add(3.0d);
|
||||
avroRecord.put("doubles", doubles);
|
||||
|
||||
// do check
|
||||
Assertions.assertEquals(GenericData.get().validate(schema, avroRecord), true);
|
||||
// create newSchema
|
||||
Types.RecordType newRecord = Types.RecordType.get(
|
||||
Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(5, false, "feature1", Types.BooleanType.get()),
|
||||
Types.Field.get(5, true, "featurex", Types.BooleanType.get()),
|
||||
Types.Field.get(6, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())),
|
||||
Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(),
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(10, true, "laty", Types.FloatType.get()),
|
||||
Types.Field.get(11, false, "long", Types.FloatType.get())), false)
|
||||
)
|
||||
);
|
||||
|
||||
Schema newAvroSchema = AvroInternalSchemaConverter.convert(newRecord, schema.getName());
|
||||
GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema);
|
||||
// test the correctly of rewrite
|
||||
Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newAvroRecord), true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEvolutionSchemaFromNewAvroSchema() {
|
||||
Types.RecordType oldRecord = Types.RecordType.get(
|
||||
Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(5, false, "feature1", Types.BooleanType.get()),
|
||||
Types.Field.get(6, true, "featurex", Types.BooleanType.get()),
|
||||
Types.Field.get(7, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false,"doubles", Types.ArrayType.get(8, false, Types.DoubleType.get())),
|
||||
Types.Field.get(4, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(),
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(11, false, "laty", Types.FloatType.get()),
|
||||
Types.Field.get(12, false, "long", Types.FloatType.get())), false)
|
||||
)
|
||||
);
|
||||
InternalSchema oldSchema = new InternalSchema(oldRecord.fields());
|
||||
Types.RecordType evolvedRecord = Types.RecordType.get(
|
||||
Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(5, false, "feature1", Types.BooleanType.get()),
|
||||
Types.Field.get(5, true, "featurex", Types.BooleanType.get()),
|
||||
Types.Field.get(6, true, "feature2", Types.BooleanType.get()),
|
||||
Types.Field.get(5, true, "feature3", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())),
|
||||
Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(),
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(10, false, "laty", Types.FloatType.get()),
|
||||
Types.Field.get(11, false, "long", Types.FloatType.get())), false)
|
||||
),
|
||||
Types.Field.get(0, false, "add1", Types.IntType.get()),
|
||||
Types.Field.get(2, true, "addStruct",
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(5, false, "nest1", Types.BooleanType.get()),
|
||||
Types.Field.get(5, true, "nest2", Types.BooleanType.get())))
|
||||
);
|
||||
evolvedRecord = (Types.RecordType)InternalSchemaBuilder.getBuilder().refreshNewId(evolvedRecord, new AtomicInteger(0));
|
||||
Schema evolvedAvroSchema = AvroInternalSchemaConverter.convert(evolvedRecord, "test1");
|
||||
InternalSchema result = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(evolvedAvroSchema, oldSchema);
|
||||
Types.RecordType checkedRecord = Types.RecordType.get(
|
||||
Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(5, false, "feature1", Types.BooleanType.get()),
|
||||
Types.Field.get(6, true, "featurex", Types.BooleanType.get()),
|
||||
Types.Field.get(7, true, "feature2", Types.BooleanType.get()),
|
||||
Types.Field.get(17, true, "feature3", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false,"doubles", Types.ArrayType.get(8, false, Types.DoubleType.get())),
|
||||
Types.Field.get(4, false, "locations", Types.MapType.get(9, 10, Types.StringType.get(),
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(11, false, "laty", Types.FloatType.get()),
|
||||
Types.Field.get(12, false, "long", Types.FloatType.get())), false)
|
||||
),
|
||||
Types.Field.get(13, true, "add1", Types.IntType.get()),
|
||||
Types.Field.get(14, true, "addStruct",
|
||||
Types.RecordType.get(
|
||||
Types.Field.get(15, false, "nest1", Types.BooleanType.get()),
|
||||
Types.Field.get(16, true, "nest2", Types.BooleanType.get())))
|
||||
);
|
||||
Assertions.assertEquals(result.getRecord(), checkedRecord);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,100 @@
|
||||
/*
|
||||
* 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.internal.schema.utils;
|
||||
|
||||
import org.apache.hudi.internal.schema.InternalSchema;
|
||||
import org.apache.hudi.internal.schema.InternalSchemaBuilder;
|
||||
import org.apache.hudi.internal.schema.Types;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestInternalSchemaUtils {
|
||||
@Test
|
||||
public void testPruneSchema() {
|
||||
Types.RecordType record = getSimpleRecordType();
|
||||
InternalSchema originSchema = new InternalSchema(record.fields());
|
||||
List<Integer> prunedCols = new ArrayList<>();
|
||||
prunedCols.add(4);
|
||||
prunedCols.add(3);
|
||||
prunedCols.add(0);
|
||||
prunedCols.add(2);
|
||||
InternalSchema prunedSchema = InternalSchemaUtils.pruneInternalSchemaByID(originSchema, prunedCols, null);
|
||||
InternalSchema checkedSchema = new InternalSchema(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "bool", Types.BooleanType.get()),
|
||||
Types.Field.get(2, "long", Types.LongType.get()),
|
||||
Types.Field.get(3, "float", Types.FloatType.get()),
|
||||
Types.Field.get(4, "double", Types.DoubleType.get())
|
||||
}));
|
||||
Assertions.assertEquals(prunedSchema, checkedSchema);
|
||||
|
||||
// nest schema
|
||||
Types.RecordType nestRecord = getNestRecordType();
|
||||
InternalSchema originNestSchema = new InternalSchema(nestRecord.fields());
|
||||
List<Integer> prunedNestCols = new ArrayList<>();
|
||||
prunedNestCols.add(0);
|
||||
prunedNestCols.add(1);
|
||||
prunedNestCols.add(5);
|
||||
prunedNestCols.add(11);
|
||||
InternalSchema prunedNestSchema = InternalSchemaUtils.pruneInternalSchemaByID(originNestSchema, prunedNestCols, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInternalSchemaVisitor() {
|
||||
Types.RecordType nestRecord = getNestRecordType();
|
||||
Map<String, Integer> result = InternalSchemaBuilder.getBuilder().buildNameToId(nestRecord);
|
||||
Assertions.assertEquals(result.size(), 12);
|
||||
Assertions.assertEquals(result.get("locations.value.long"), 11);
|
||||
Assertions.assertEquals(result.get("locations.value.lat"), 10);
|
||||
Assertions.assertEquals(result.get("locations.value"), 9);
|
||||
Assertions.assertEquals(result.get("locations.key"), 8);
|
||||
Assertions.assertEquals(result.get("doubles.element"), 7);
|
||||
|
||||
Types.RecordType simpleRecord = getSimpleRecordType();
|
||||
Map<String, Integer> result1 = InternalSchemaBuilder.getBuilder().buildNameToId(simpleRecord);
|
||||
Assertions.assertEquals(result1.size(), 5);
|
||||
Assertions.assertEquals(result1.get("double"), 4);
|
||||
}
|
||||
|
||||
public Types.RecordType getNestRecordType() {
|
||||
return Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()),
|
||||
Types.Field.get(1, true, "data", Types.StringType.get()),
|
||||
Types.Field.get(2, true, "preferences",
|
||||
Types.RecordType.get(Types.Field.get(5, false, "feature1",
|
||||
Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))),
|
||||
Types.Field.get(3, false,"doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())),
|
||||
Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(),
|
||||
Types.RecordType.get(Types.Field.get(10, false, "lat", Types.FloatType.get()), Types.Field.get(11, false, "long", Types.FloatType.get())), false))
|
||||
);
|
||||
}
|
||||
|
||||
public Types.RecordType getSimpleRecordType() {
|
||||
return Types.RecordType.get(Arrays.asList(new Types.Field[] {
|
||||
Types.Field.get(0, "bool", Types.BooleanType.get()),
|
||||
Types.Field.get(1, "int", Types.IntType.get()),
|
||||
Types.Field.get(2, "long", Types.LongType.get()),
|
||||
Types.Field.get(3, "float", Types.FloatType.get()),
|
||||
Types.Field.get(4, "double", Types.DoubleType.get())
|
||||
}));
|
||||
}
|
||||
}
|
||||
@@ -16,4 +16,5 @@
|
||||
# limitations under the License.
|
||||
|
||||
|
||||
org.apache.hudi.DefaultSource
|
||||
org.apache.hudi.DefaultSource
|
||||
org.apache.spark.sql.execution.datasources.parquet.SparkHoodieParquetFileFormat
|
||||
@@ -65,6 +65,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
|
||||
tableSchema: HoodieTableSchema,
|
||||
requiredSchema: HoodieTableSchema,
|
||||
filters: Array[Filter]): HoodieUnsafeRDD = {
|
||||
|
||||
val baseFileReader = createBaseFileReader(
|
||||
spark = sparkSession,
|
||||
partitionSchema = partitionSchema,
|
||||
@@ -74,7 +75,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
new HoodieFileScanRDD(sparkSession, baseFileReader, fileSplits)
|
||||
|
||||
@@ -21,12 +21,12 @@ import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceReadOptions._
|
||||
import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
|
||||
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql.execution.datasources.DataSource
|
||||
import org.apache.spark.sql.execution.streaming.{Sink, Source}
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieStreamSource
|
||||
import org.apache.spark.sql.sources._
|
||||
@@ -46,6 +46,7 @@ class DefaultSource extends RelationProvider
|
||||
with DataSourceRegister
|
||||
with StreamSinkProvider
|
||||
with StreamSourceProvider
|
||||
with SparkAdapterSupport
|
||||
with Serializable {
|
||||
|
||||
SparkSession.getActiveSession.foreach { spark =>
|
||||
@@ -108,7 +109,6 @@ class DefaultSource extends RelationProvider
|
||||
(COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) |
|
||||
(MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) =>
|
||||
new BaseFileOnlyRelation(sqlContext, metaClient, parameters, userSchema, globPaths)
|
||||
|
||||
case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) =>
|
||||
new IncrementalRelation(sqlContext, parameters, userSchema, metaClient)
|
||||
|
||||
@@ -128,6 +128,11 @@ class DefaultSource extends RelationProvider
|
||||
}
|
||||
}
|
||||
|
||||
def getValidCommits(metaClient: HoodieTableMetaClient): String = {
|
||||
metaClient
|
||||
.getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
|
||||
}
|
||||
|
||||
/**
|
||||
* This DataSource API is used for writing the DataFrame at the destination. For now, we are returning a dummy
|
||||
* relation here because Spark does not really make use of the relation returned, and just returns an empty
|
||||
|
||||
@@ -33,6 +33,9 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.StringUtils
|
||||
import org.apache.hudi.common.util.ValidationUtils.checkState
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata
|
||||
import org.apache.spark.TaskContext
|
||||
@@ -54,7 +57,7 @@ import scala.util.Try
|
||||
|
||||
trait HoodieFileSplit {}
|
||||
|
||||
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String)
|
||||
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String, internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema)
|
||||
|
||||
case class HoodieTableState(tablePath: String,
|
||||
latestCommitTimestamp: String,
|
||||
@@ -114,9 +117,9 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
|
||||
.map(HoodieSqlCommonUtils.formatQueryInstant)
|
||||
|
||||
protected lazy val tableAvroSchema: Schema = {
|
||||
protected lazy val (tableAvroSchema: Schema, internalSchema: InternalSchema) = {
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
Try(schemaUtil.getTableAvroSchema).getOrElse(
|
||||
val avroSchema = Try(schemaUtil.getTableAvroSchema).getOrElse(
|
||||
// If there is no commit in the table, we can't get the schema
|
||||
// t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead.
|
||||
userSchema match {
|
||||
@@ -124,6 +127,13 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty")
|
||||
}
|
||||
)
|
||||
// try to find internalSchema
|
||||
val internalSchemaFromMeta = try {
|
||||
schemaUtil.getTableInternalSchemaFromCommitMetadata.orElse(InternalSchema.getEmptyInternalSchema)
|
||||
} catch {
|
||||
case _ => InternalSchema.getEmptyInternalSchema
|
||||
}
|
||||
(avroSchema, internalSchemaFromMeta)
|
||||
}
|
||||
|
||||
protected val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
|
||||
@@ -155,6 +165,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
// NOTE: We're including compaction here since it's not considering a "commit" operation
|
||||
metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants
|
||||
|
||||
protected val validCommits = timeline.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
|
||||
|
||||
protected def latestInstant: Option[HoodieInstant] =
|
||||
toScalaOption(timeline.lastInstant())
|
||||
|
||||
@@ -189,8 +201,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
// filtered out upstream
|
||||
val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns)
|
||||
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns)
|
||||
val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) =
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns, internalSchema)
|
||||
|
||||
val filterExpressions = convertToExpressions(filters)
|
||||
val (partitionFilters, dataFilters) = filterExpressions.partition(isPartitionPredicate)
|
||||
@@ -198,8 +210,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
val fileSplits = collectFileSplits(partitionFilters, dataFilters)
|
||||
|
||||
val partitionSchema = StructType(Nil)
|
||||
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString)
|
||||
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString)
|
||||
val tableSchema = HoodieTableSchema(tableStructSchema, if (internalSchema.isEmptySchema) tableAvroSchema.toString else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString, internalSchema)
|
||||
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
|
||||
|
||||
// Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]]
|
||||
// Please check [[needConversion]] scala-doc for more details
|
||||
|
||||
@@ -20,6 +20,9 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.FileStatus
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{PredicateHelper, SpecificInternalRow, UnsafeProjection}
|
||||
@@ -31,7 +34,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object HoodieDataSourceHelper extends PredicateHelper {
|
||||
object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport {
|
||||
|
||||
|
||||
/**
|
||||
@@ -46,7 +49,7 @@ object HoodieDataSourceHelper extends PredicateHelper {
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
|
||||
val readParquetFile: PartitionedFile => Iterator[Any] = new ParquetFileFormat().buildReaderWithPartitionValues(
|
||||
val readParquetFile: PartitionedFile => Iterator[Any] = sparkAdapter.createHoodieParquetFileFormat().get.buildReaderWithPartitionValues(
|
||||
sparkSession = sparkSession,
|
||||
dataSchema = dataSchema,
|
||||
partitionSchema = partitionSchema,
|
||||
@@ -78,4 +81,19 @@ object HoodieDataSourceHelper extends PredicateHelper {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set internalSchema evolution parameters to configuration.
|
||||
* spark will broadcast them to each executor, we use those parameters to do schema evolution.
|
||||
*
|
||||
* @param conf hadoop conf.
|
||||
* @param internalSchema internalschema for query.
|
||||
* @param tablePath hoodie table base path.
|
||||
* @param validCommits valid commits, using give validCommits to validate all legal histroy Schema files, and return the latest one.
|
||||
*/
|
||||
def getConfigurationWithInternalSchema(conf: Configuration, internalSchema: InternalSchema, tablePath: String, validCommits: String): Configuration = {
|
||||
conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
|
||||
conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tablePath)
|
||||
conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
|
||||
conf
|
||||
}
|
||||
}
|
||||
|
||||
@@ -39,6 +39,7 @@ import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable
|
||||
import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata}
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.HoodieAvroDeserializer
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
@@ -165,9 +166,10 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
|
||||
// be stored in non-columnar formats like Avro, HFile, etc)
|
||||
private val requiredSchemaFieldOrdinals: List[Int] = collectFieldOrdinals(requiredAvroSchema, logFileReaderAvroSchema)
|
||||
|
||||
// TODO: now logScanner with internalSchema support column project, we may no need projectAvroUnsafe
|
||||
private var logScanner =
|
||||
HoodieMergeOnReadRDD.scanLog(split.logFiles, getPartitionPath(split), logFileReaderAvroSchema, tableState,
|
||||
maxCompactionMemoryInBytes, config)
|
||||
maxCompactionMemoryInBytes, config, tableSchema.internalSchema)
|
||||
|
||||
private val logRecords = logScanner.getRecords.asScala
|
||||
|
||||
@@ -305,7 +307,7 @@ private object HoodieMergeOnReadRDD {
|
||||
logSchema: Schema,
|
||||
tableState: HoodieTableState,
|
||||
maxCompactionMemoryInBytes: Long,
|
||||
hadoopConf: Configuration): HoodieMergedLogRecordScanner = {
|
||||
hadoopConf: Configuration, internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema): HoodieMergedLogRecordScanner = {
|
||||
val tablePath = tableState.tablePath
|
||||
val fs = FSUtils.getFs(tablePath, hadoopConf)
|
||||
|
||||
@@ -333,6 +335,7 @@ private object HoodieMergeOnReadRDD {
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
|
||||
.getOrElse(false))
|
||||
.withReverseReader(false)
|
||||
.withInternalSchema(internalSchema)
|
||||
.withBufferSize(
|
||||
hadoopConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
|
||||
|
||||
@@ -51,6 +51,11 @@ import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.internal.StaticSQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.{SPARK_VERSION, SparkContext}
|
||||
import org.apache.spark.SparkContext
|
||||
|
||||
import java.util.Properties
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
@@ -184,9 +189,10 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
// Create a HoodieWriteClient & issue the delete.
|
||||
val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext)
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
null, path, tblName,
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
mapAsJavaMap(addSchemaEvolutionParameters(parameters, internalSchemaOpt) - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -231,8 +237,17 @@ object HoodieSparkSqlWriter {
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
val lastestSchema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext)
|
||||
if (reconcileSchema) {
|
||||
schema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
schema = lastestSchema
|
||||
}
|
||||
if (internalSchemaOpt.isDefined) {
|
||||
schema = {
|
||||
val newSparkSchema = AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema))
|
||||
AvroConversionUtils.convertStructTypeToAvroSchema(newSparkSchema, structName, nameSpace)
|
||||
|
||||
}
|
||||
}
|
||||
validateSchemaForHoodieIsDeleted(schema)
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
@@ -264,8 +279,9 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path,
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
|
||||
tblName, mapAsJavaMap(addSchemaEvolutionParameters(parameters, internalSchemaOpt) - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
|
||||
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -315,6 +331,36 @@ object HoodieSparkSqlWriter {
|
||||
processedRecord
|
||||
}
|
||||
|
||||
def addSchemaEvolutionParameters(parameters: Map[String, String], internalSchemaOpt: Option[InternalSchema]): Map[String, String] = {
|
||||
val schemaEvolutionEnable = if (internalSchemaOpt.isDefined) "true" else "false"
|
||||
parameters ++ Map(HoodieWriteConfig.INTERNAL_SCHEMA_STRING.key() -> SerDeHelper.toJson(internalSchemaOpt.getOrElse(null)),
|
||||
HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key() -> schemaEvolutionEnable)
|
||||
}
|
||||
|
||||
/**
|
||||
* get latest internalSchema from table
|
||||
*
|
||||
* @param fs instance of FileSystem.
|
||||
* @param basePath base path.
|
||||
* @param sparkContext instance of spark context.
|
||||
* @param schema incoming record's schema.
|
||||
* @return Pair of(boolean, table schema), where first entry will be true only if schema conversion is required.
|
||||
*/
|
||||
def getLatestTableInternalSchema(fs: FileSystem, basePath: Path, sparkContext: SparkContext): Option[InternalSchema] = {
|
||||
try {
|
||||
if (FSUtils.isTableExists(basePath.toString, fs)) {
|
||||
val tableMetaClient = HoodieTableMetaClient.builder.setConf(sparkContext.hadoopConfiguration).setBasePath(basePath.toString).build()
|
||||
val tableSchemaResolver = new TableSchemaResolver(tableMetaClient)
|
||||
val internalSchemaOpt = tableSchemaResolver.getTableInternalSchemaFromCommitMetadata
|
||||
if (internalSchemaOpt.isPresent) Some(internalSchemaOpt.get()) else None
|
||||
} else {
|
||||
None
|
||||
}
|
||||
} catch {
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if schema needs upgrade (if incoming record's write schema is old while table schema got evolved).
|
||||
*
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.DataSourceOptionsHelper.allAlternatives
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
|
||||
@@ -163,6 +164,13 @@ object HoodieWriterUtils {
|
||||
diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n")
|
||||
throw new HoodieException(diffConfigs.toString.trim)
|
||||
}
|
||||
// Check schema evolution for bootstrap table.
|
||||
// now we do not support bootstrap table.
|
||||
if (params.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
&& params.getOrElse(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean) {
|
||||
throw new HoodieException(String
|
||||
.format("now schema evolution cannot support bootstrap table, pls set %s to false", HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key()))
|
||||
}
|
||||
}
|
||||
|
||||
private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieConfig, key: String): String = {
|
||||
|
||||
@@ -18,17 +18,20 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieReplaceCommitMetadata}
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieFileFormat, HoodieRecord, HoodieReplaceCommitMetadata}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
|
||||
import java.util.stream.Collectors
|
||||
|
||||
import org.apache.hadoop.fs.{GlobPattern, Path}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||
import org.apache.hudi.common.util.HoodieTimer
|
||||
import org.apache.hudi.common.util.{HoodieTimer, InternalSchemaCache}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
@@ -82,10 +85,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
|
||||
|
||||
// use schema from a file produced in the end/latest instant
|
||||
val usedSchema: StructType = {
|
||||
|
||||
val (usedSchema, internalSchema) = {
|
||||
log.info("Inferring schema..")
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val tableSchema = if (useEndInstantSchema) {
|
||||
val iSchema = if (useEndInstantSchema && !commitsToReturn.isEmpty) {
|
||||
InternalSchemaCache.searchSchemaAndCache(commitsToReturn.last.getTimestamp.toLong, metaClient, hoodieTable.getConfig.getInternalSchemaCacheEnable)
|
||||
} else {
|
||||
schemaResolver.getTableInternalSchemaFromCommitMetadata.orElse(null)
|
||||
}
|
||||
|
||||
val tableSchema = if (useEndInstantSchema && iSchema.isEmptySchema) {
|
||||
if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchemaWithoutMetadataFields() else
|
||||
schemaResolver.getTableAvroSchemaWithoutMetadataFields(commitsToReturn.last)
|
||||
} else {
|
||||
@@ -93,10 +103,15 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
if (tableSchema.getType == Schema.Type.NULL) {
|
||||
// if there is only one commit in the table and is an empty commit without schema, return empty RDD here
|
||||
StructType(Nil)
|
||||
(StructType(Nil), InternalSchema.getEmptyInternalSchema)
|
||||
} else {
|
||||
val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema)
|
||||
StructType(skeletonSchema.fields ++ dataSchema.fields)
|
||||
if (iSchema != null && !iSchema.isEmptySchema) {
|
||||
// if internalSchema is ready, dataSchema will contains skeletonSchema
|
||||
(dataSchema, iSchema)
|
||||
} else {
|
||||
(StructType(skeletonSchema.fields ++ dataSchema.fields), InternalSchema.getEmptyInternalSchema)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -161,6 +176,16 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
// unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view
|
||||
// will filter out all the files incorrectly.
|
||||
// pass internalSchema to hadoopConf, so it can be used in executors.
|
||||
val validCommits = metaClient
|
||||
.getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
|
||||
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
|
||||
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath)
|
||||
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
|
||||
val formatClassName = metaClient.getTableConfig.getBaseFileFormat match {
|
||||
case HoodieFileFormat.PARQUET => if (!internalSchema.isEmptySchema) "HoodieParquet" else "parquet"
|
||||
case HoodieFileFormat.ORC => "orc"
|
||||
}
|
||||
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class")
|
||||
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
|
||||
if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) {
|
||||
@@ -216,8 +241,8 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
|
||||
if (regularFileIdToFullPath.nonEmpty) {
|
||||
df = df.union(sqlContext.read.options(sOpts)
|
||||
.schema(usedSchema)
|
||||
.parquet(filteredRegularFullPaths.toList: _*)
|
||||
.schema(usedSchema).format(formatClassName)
|
||||
.load(filteredRegularFullPaths.toList: _*)
|
||||
.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.head.getTimestamp))
|
||||
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
|
||||
@@ -80,7 +80,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val requiredSchemaParquetReader = createBaseFileReader(
|
||||
@@ -92,7 +92,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val hoodieTableState = getTableState
|
||||
|
||||
@@ -79,7 +79,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val requiredSchemaParquetReader = createBaseFileReader(
|
||||
@@ -91,7 +91,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val tableState = getTableState
|
||||
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hudi.SparkAdapterSupport
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.sources.Filter
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
|
||||
class SparkHoodieParquetFileFormat extends ParquetFileFormat with SparkAdapterSupport {
|
||||
override def shortName(): String = "HoodieParquet"
|
||||
|
||||
override def toString: String = "HoodieParquet"
|
||||
|
||||
override def buildReaderWithPartitionValues(
|
||||
sparkSession: SparkSession,
|
||||
dataSchema: StructType,
|
||||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
sparkAdapter
|
||||
.createHoodieParquetFileFormat().get
|
||||
.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -110,7 +110,7 @@ object AlterHoodieTableAddColumnsCommand {
|
||||
HoodieWriterUtils.parametersWithWriteDefaults(hoodieCatalogTable.catalogProperties).asJava
|
||||
)
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, hoodieCatalogTable.tableType)
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, hoodieCatalogTable.tableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
@@ -118,7 +118,7 @@ object AlterHoodieTableAddColumnsCommand {
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.INSERT)
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
|
||||
client.commit(instantTime, jsc.emptyRDD)
|
||||
|
||||
@@ -39,6 +39,10 @@ class HoodieSparkSessionExtension extends (SparkSessionExtensions => Unit)
|
||||
}
|
||||
}
|
||||
|
||||
extensions.injectResolutionRule { session =>
|
||||
sparkAdapter.createResolveHudiAlterTableCommand(session)
|
||||
}
|
||||
|
||||
HoodieAnalysis.customPostHocResolutionRules().foreach { rule =>
|
||||
extensions.injectPostHocResolutionRule { session =>
|
||||
rule(session)
|
||||
|
||||
@@ -221,7 +221,7 @@ class TestHoodieSparkUtils {
|
||||
|
||||
val tableAvroSchema = new Schema.Parser().parse(avroSchemaString)
|
||||
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
val (requiredAvroSchema, requiredStructSchema, _) =
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, Array("ts"))
|
||||
|
||||
assertEquals("timestamp-millis",
|
||||
|
||||
@@ -331,7 +331,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with
|
||||
}
|
||||
|
||||
val readColumns = targetColumns ++ relation.mandatoryColumns
|
||||
val (_, projectedStructType) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
|
||||
val (_, projectedStructType, _) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
|
||||
|
||||
val row: InternalRow = rows.take(1).head
|
||||
|
||||
|
||||
@@ -102,6 +102,22 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
|
||||
assertResult(expects.map(row => Row(row: _*)).toArray.sortBy(_.toString()))(spark.sql(sql).collect().sortBy(_.toString()))
|
||||
}
|
||||
|
||||
protected def checkAnswer(array: Array[Row])(expects: Seq[Any]*): Unit = {
|
||||
assertResult(expects.map(row => Row(row: _*)).toArray)(array)
|
||||
}
|
||||
|
||||
protected def checkExceptions(sql: String)(errorMsgs: Seq[String]): Unit = {
|
||||
var hasException = false
|
||||
try {
|
||||
spark.sql(sql)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
assertResult(errorMsgs.contains(e.getMessage.split("\n")(0)))(true)
|
||||
hasException = true
|
||||
}
|
||||
assertResult(true)(hasException)
|
||||
}
|
||||
|
||||
protected def checkException(sql: String)(errorMsg: String): Unit = {
|
||||
var hasException = false
|
||||
try {
|
||||
|
||||
@@ -0,0 +1,472 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class TestSpark3DDL extends TestHoodieSqlBase {
|
||||
|
||||
def createTestResult(tableName: String): Array[Row] = {
|
||||
spark.sql(s"select * from ${tableName} order by id")
|
||||
.drop("_hoodie_commit_time", "_hoodie_commit_seqno", "_hoodie_record_key", "_hoodie_partition_path", "_hoodie_file_name").collect()
|
||||
}
|
||||
|
||||
def createAndPreparePartitionTable(spark: SparkSession, tableName: String, tablePath: String, tableType: String): Unit = {
|
||||
// try to clean tablePath
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int, comb int, col0 int, col1 bigint, col2 float, col3 double, col4 decimal(10,4), col5 string, col6 date, col7 timestamp, col8 boolean, col9 binary, par date
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'comb'
|
||||
| )
|
||||
| partitioned by (par)
|
||||
""".stripMargin)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,11,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (2,2,12,100002,102.02,1002.0002,100002.0002,'a000002','2021-12-25','2021-12-25 12:02:02',true,'a02','2021-12-25'),
|
||||
| (3,3,13,100003,103.03,1003.0003,100003.0003,'a000003','2021-12-25','2021-12-25 12:03:03',false,'a03','2021-12-25'),
|
||||
| (4,4,14,100004,104.04,1004.0004,100004.0004,'a000004','2021-12-26','2021-12-26 12:04:04',true,'a04','2021-12-26'),
|
||||
| (5,5,15,100005,105.05,1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
}
|
||||
|
||||
test("Test multi change data type") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
createAndPreparePartitionTable(spark, tableName, tablePath, tableType)
|
||||
// date -> string -> date
|
||||
spark.sql(s"alter table $tableName alter column col6 type String")
|
||||
checkAnswer(spark.sql(s"select col6 from $tableName where id = 1").collect())(
|
||||
Seq("2021-12-25")
|
||||
)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-26','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col6 type date")
|
||||
checkAnswer(spark.sql(s"select col6 from $tableName where id = 1 or id = 5 order by id").collect())(
|
||||
Seq(java.sql.Date.valueOf("2021-12-26")), // value from new file
|
||||
Seq(java.sql.Date.valueOf("2021-12-26")) // value from old file
|
||||
)
|
||||
// int -> double -> decimal
|
||||
spark.sql(s"alter table $tableName alter column col0 type double")
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,1,14.0,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col0 type decimal(16, 4)")
|
||||
checkAnswer(spark.sql(s"select col0 from $tableName where id = 1 or id = 6 order by id").collect())(
|
||||
Seq(new java.math.BigDecimal("13.0000")),
|
||||
Seq(new java.math.BigDecimal("14.0000"))
|
||||
)
|
||||
// float -> double -> decimal
|
||||
spark.sql(s"alter table $tableName alter column col2 type double")
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,100001,901.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,1,14.0,100001,601.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col2 type decimal(16, 4)")
|
||||
checkAnswer(spark.sql(s"select col0, col2 from $tableName where id = 1 or id = 6 order by id").collect())(
|
||||
Seq(new java.math.BigDecimal("13.0000"), new java.math.BigDecimal("901.0100")),
|
||||
Seq(new java.math.BigDecimal("14.0000"), new java.math.BigDecimal("601.0100"))
|
||||
)
|
||||
// long -> double -> decimal
|
||||
spark.sql(s"alter table $tableName alter column col1 type double")
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,700001.0,901.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col1 type decimal(16, 4)")
|
||||
checkAnswer(spark.sql(s"select col0, col2, col1 from $tableName where id = 1 or id = 6 order by id").collect())(
|
||||
Seq(new java.math.BigDecimal("13.0000"), new java.math.BigDecimal("901.0100"), new java.math.BigDecimal("700001.0000")),
|
||||
Seq(new java.math.BigDecimal("14.0000"), new java.math.BigDecimal("601.0100"), new java.math.BigDecimal("100001.0000"))
|
||||
)
|
||||
spark.sessionState.catalog.dropTable(TableIdentifier(tableName), true, true)
|
||||
spark.sessionState.catalog.refreshTable(TableIdentifier(tableName))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test multi change data type2") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
createAndPreparePartitionTable(spark, tableName, tablePath, tableType)
|
||||
// float -> double -> decimal -> String
|
||||
spark.sql(s"alter table $tableName alter column col2 type double")
|
||||
spark.sql(s"alter table $tableName alter column col2 type decimal(16, 4)")
|
||||
spark.sql(s"alter table $tableName alter column col2 type String")
|
||||
checkAnswer(spark.sql(s"select col2 from $tableName where id = 1").collect())(
|
||||
Seq("101.01")
|
||||
)
|
||||
// long -> double -> decimal -> string
|
||||
spark.sql(s"alter table $tableName alter column col1 type double")
|
||||
spark.sql(s"alter table $tableName alter column col1 type decimal(16, 4)")
|
||||
spark.sql(s"alter table $tableName alter column col1 type String")
|
||||
checkAnswer(spark.sql(s"select col1 from $tableName where id = 1").collect())(
|
||||
Seq("100001")
|
||||
)
|
||||
// int -> double -> decimal -> String
|
||||
spark.sql(s"alter table $tableName alter column col0 type double")
|
||||
spark.sql(s"alter table $tableName alter column col0 type decimal(16, 4)")
|
||||
spark.sql(s"alter table $tableName alter column col0 type String")
|
||||
checkAnswer(spark.sql(s"select col0 from $tableName where id = 1").collect())(
|
||||
Seq("11")
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Partition Table alter ") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
createAndPreparePartitionTable(spark, tableName, tablePath, tableType)
|
||||
|
||||
// test set properties
|
||||
spark.sql(s"alter table $tableName set tblproperties(comment='it is a hudi table', 'key1'='value1', 'key2'='value2')")
|
||||
val meta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
|
||||
assert(meta.comment.get.equals("it is a hudi table"))
|
||||
assert(Seq("key1", "key2").filter(meta.properties.contains(_)).size == 2)
|
||||
// test unset propertes
|
||||
spark.sql(s"alter table $tableName unset tblproperties(comment, 'key1', 'key2')")
|
||||
val unsetMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
|
||||
assert(Seq("key1", "key2").filter(unsetMeta.properties.contains(_)).size == 0)
|
||||
assert(unsetMeta.comment.isEmpty)
|
||||
// test forbidden operation.
|
||||
checkException(s"Alter table $tableName add columns(col_new1 int first)")("forbid adjust top-level columns position by using through first syntax")
|
||||
HoodieRecord.HOODIE_META_COLUMNS.subList(0, HoodieRecord.HOODIE_META_COLUMNS.size - 2).asScala.foreach {f =>
|
||||
checkException(s"Alter table $tableName add columns(col_new1 int after $f)")("forbid adjust the position of ordinary columns between meta columns")
|
||||
}
|
||||
Seq("id", "comb", "par").foreach { col =>
|
||||
checkException(s"alter table $tableName drop column $col")("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
checkException(s"alter table $tableName rename column $col to ${col + col}")("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
}
|
||||
// check duplicate add or rename
|
||||
// keep consistent with hive, column names insensitive
|
||||
checkExceptions(s"alter table $tableName rename column col0 to col9")(Seq("cannot rename column: col0 to a existing name",
|
||||
"Cannot rename column, because col9 already exists in root"))
|
||||
checkExceptions(s"alter table $tableName rename column col0 to COL9")(Seq("cannot rename column: col0 to a existing name", "Cannot rename column, because COL9 already exists in root"))
|
||||
checkExceptions(s"alter table $tableName add columns(col9 int first)")(Seq("cannot add column: col9 which already exist", "Cannot add column, because col9 already exists in root"))
|
||||
checkExceptions(s"alter table $tableName add columns(COL9 int first)")(Seq("cannot add column: COL9 which already exist", "Cannot add column, because COL9 already exists in root"))
|
||||
// test add comment for columns / alter columns comment
|
||||
spark.sql(s"alter table $tableName add columns(col1_new int comment 'add new columns col1_new after id' after id)")
|
||||
spark.sql(s"alter table $tableName alter column col9 comment 'col9 desc'")
|
||||
val schema = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).schema
|
||||
assert(schema.filter(p => p.name.equals("col1_new")).get(0).getComment().get == "add new columns col1_new after id")
|
||||
assert(schema.filter(p => p.name.equals("col9")).get(0).getComment().get == "col9 desc")
|
||||
// test change column type float to double
|
||||
spark.sql(s"alter table $tableName alter column col2 type double")
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 2 order by id").show(false)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,1,11,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,6,5,15,100005,105.05,1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 order by id").show(false)
|
||||
// try schedule compact
|
||||
if (tableType == "mor") spark.sql(s"schedule compaction on $tableName")
|
||||
// test change column type decimal(10,4) 为decimal(18,8)
|
||||
spark.sql(s"alter table $tableName alter column col4 type decimal(18, 8)")
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 2 order by id").show(false)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (5,6,5,15,100005,105.05,1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col4 from $tableName where id = 1 or id = 6 or id = 2 order by id").show(false)
|
||||
// test change column type float to double
|
||||
spark.sql(s"alter table $tableName alter column col2 type string")
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 2 order by id").show(false)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,1,11,100001,'101.01',1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,6,5,15,100005,'105.05',1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 order by id").show(false)
|
||||
// try schedule compact
|
||||
if (tableType == "mor") spark.sql(s"schedule compaction on $tableName")
|
||||
// if tableType is mor, check compaction
|
||||
if (tableType == "mor") {
|
||||
val compactionRows = spark.sql(s"show compaction on $tableName limit 10").collect()
|
||||
val timestamps = compactionRows.map(_.getString(0))
|
||||
assertResult(2)(timestamps.length)
|
||||
spark.sql(s"run compaction on $tableName at ${timestamps(1)}")
|
||||
spark.sql(s"run compaction on $tableName at ${timestamps(0)}")
|
||||
}
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,1,11,100001,'101.01',1001.0001,100009.0001,'a000008','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (11,3,1,11,100001,'101.01',1001.0001,100011.0001,'a000008','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,6,5,15,100005,'105.05',1005.0005,100007.0005,'a000009','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 or id = 11 order by id").show(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Chinese table ") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int, comb int, `名字` string, col9 string, `成绩` int, `身高` float, `体重` double, `上次更新时间` date, par date
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'comb'
|
||||
| )
|
||||
| partitioned by (par)
|
||||
""".stripMargin)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,'李明', '读书', 100,180.0001,99.0001,'2021-12-25', '2021-12-26')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName rename column col9 to `爱好_Best`")
|
||||
|
||||
// update current table to produce log files for mor
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,'李明', '读书', 100,180.0001,99.0001,'2021-12-26', '2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
// alter date to string
|
||||
spark.sql(s"alter table $tableName alter column `上次更新时间` type string ")
|
||||
checkAnswer(spark.sql(s"select `上次更新时间` from $tableName").collect())(
|
||||
Seq("2021-12-26")
|
||||
)
|
||||
// alter string to date
|
||||
spark.sql(s"alter table $tableName alter column `上次更新时间` type date ")
|
||||
spark.sql(s"select `上次更新时间` from $tableName").collect()
|
||||
checkAnswer(spark.sql(s"select `上次更新时间` from $tableName").collect())(
|
||||
Seq(java.sql.Date.valueOf("2021-12-26"))
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
test("Test Alter Table") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
spark.sql(s"show create table ${tableName}").show(false)
|
||||
spark.sql(s"insert into ${tableName} values (1, 'jack', 0.9, 1000)")
|
||||
spark.sql(s"update ${tableName} set price = 1.9 where id = 1")
|
||||
|
||||
spark.sql(s"alter table ${tableName} alter column id type long")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", 1.9, 1000)
|
||||
)
|
||||
// test add action, include position change
|
||||
spark.sql(s"alter table ${tableName} add columns(ext1 string comment 'add ext1' after name)")
|
||||
spark.sql(s"insert into ${tableName} values (2, 'jack', 'exx1', 0.9, 1000)")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", null, 1.9, 1000), Seq(2, "jack","exx1", 0.9, 1000)
|
||||
)
|
||||
// test rename
|
||||
spark.sql(s"alter table ${tableName} rename column price to newprice")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", null, 1.9, 1000), Seq(2, "jack","exx1", 0.9, 1000)
|
||||
)
|
||||
spark.sql(s"update ${tableName} set ext1 = 'haha' where id = 1 ")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", "haha", 1.9, 1000), Seq(2, "jack","exx1", 0.9, 1000)
|
||||
)
|
||||
// drop column newprice
|
||||
|
||||
spark.sql(s"alter table ${tableName} drop column newprice")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", "haha", 1000), Seq(2, "jack","exx1", 1000)
|
||||
)
|
||||
// add newprice back
|
||||
spark.sql(s"alter table ${tableName} add columns(newprice string comment 'add newprice back' after ext1)")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", "haha", null, 1000), Seq(2, "jack","exx1", null, 1000)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Alter Table complex") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| members map<String, struct<n:string, a:int>>,
|
||||
| user struct<name:string, age:int, score: int>,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
|
||||
spark.sql(s"alter table $tableName alter column members.value.a first")
|
||||
|
||||
spark.sql(s"insert into ${tableName} values(1, 'jack', map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStruct', 29, 100), 1000)")
|
||||
|
||||
// rename column
|
||||
spark.sql(s"alter table ${tableName} rename column user to userx")
|
||||
|
||||
checkAnswer(spark.sql(s"select ts, userx.score, id, userx.age, name from ${tableName}").collect())(
|
||||
Seq(1000, 100, 1, 29, "jack")
|
||||
)
|
||||
|
||||
// drop column
|
||||
spark.sql(s"alter table ${tableName} drop columns(name, userx.name, userx.score)")
|
||||
|
||||
spark.sql(s"select * from ${tableName}").show(false)
|
||||
|
||||
// add cols back, and adjust cols position
|
||||
spark.sql(s"alter table ${tableName} add columns(name string comment 'add name back' after userx," +
|
||||
s" userx.name string comment 'add userx.name back' first, userx.score int comment 'add userx.score back' after age)")
|
||||
|
||||
// query new columns: name, userx.name, userx.score, those field should not be readed.
|
||||
checkAnswer(spark.sql(s"select name, userx.name, userx.score from ${tableName}").collect())(Seq(null, null, null))
|
||||
|
||||
// insert again
|
||||
spark.sql(s"insert into ${tableName} values(2 , map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStructNew', 291 , 101), 'jacknew', 1000)")
|
||||
|
||||
// check again
|
||||
checkAnswer(spark.sql(s"select name, userx.name as uxname, userx.score as uxs from ${tableName} order by id").collect())(
|
||||
Seq(null, null, null),
|
||||
Seq("jacknew", "jackStructNew", 101))
|
||||
|
||||
|
||||
spark.sql(s"alter table ${tableName} alter column userx.age type long")
|
||||
|
||||
spark.sql(s"select userx.age, id, name from ${tableName}")
|
||||
checkAnswer(spark.sql(s"select userx.age, id, name from ${tableName} order by id").collect())(
|
||||
Seq(29, 1, null),
|
||||
Seq(291, 2, "jacknew"))
|
||||
// test map value type change
|
||||
spark.sql(s"alter table ${tableName} add columns(mxp map<String, int>)")
|
||||
spark.sql(s"insert into ${tableName} values(2 , map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStructNew', 291 , 101), 'jacknew', 1000, map('t1', 9))")
|
||||
spark.sql(s"alter table ${tableName} alter column mxp.value type double")
|
||||
spark.sql(s"insert into ${tableName} values(2 , map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStructNew', 291 , 101), 'jacknew', 1000, map('t1', 10))")
|
||||
spark.sql(s"select * from $tableName").show(false)
|
||||
checkAnswer(spark.sql(s"select mxp from ${tableName} order by id").collect())(
|
||||
Seq(null),
|
||||
Seq(Map("t1" -> 10.0d))
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def performClustering(writeDf: DataFrame, basePath: String, tableName: String, tableType: String): Unit = {
|
||||
writeDf.write.format("org.apache.hudi")
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType)
|
||||
.option("hoodie.upsert.shuffle.parallelism", "1")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key(), "id")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "comb")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "par")
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
.option("hoodie.schema.on.read.enable", "true")
|
||||
// option for clustering
|
||||
.option("hoodie.clustering.inline", "true")
|
||||
.option("hoodie.clustering.inline.max.commits", "1")
|
||||
.option("hoodie.clustering.plan.strategy.small.file.limit", String.valueOf(2*1024*1024L))
|
||||
.option("hoodie.clustering.plan.strategy.max.bytes.per.group", String.valueOf(10*1024*1024L))
|
||||
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(4 * 1024* 1024L))
|
||||
.option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "col1, col2")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
}
|
||||
}
|
||||
@@ -1,3 +1,4 @@
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
@@ -27,7 +28,9 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Join, LogicalPlan}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, Spark2ParsePartitionUtil, SparkParsePartitionUtil}
|
||||
import org.apache.spark.sql.hudi.SparkAdapter
|
||||
import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser
|
||||
@@ -155,4 +158,14 @@ class Spark2Adapter extends SparkAdapter {
|
||||
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
|
||||
throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark2")
|
||||
}
|
||||
|
||||
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
|
||||
new Rule[LogicalPlan] {
|
||||
override def apply(plan: LogicalPlan): LogicalPlan = plan
|
||||
}
|
||||
}
|
||||
|
||||
override def createHoodieParquetFileFormat(): Option[ParquetFileFormat] = {
|
||||
Some(new ParquetFileFormat)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@ package org.apache.spark.sql.adapter
|
||||
|
||||
import org.apache.hudi.Spark3RowSerDe
|
||||
import org.apache.hudi.client.utils.SparkRowSerDe
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.hudi.spark3.internal.ReflectUtil
|
||||
import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters}
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
@@ -27,11 +28,13 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
|
||||
import org.apache.spark.sql.connector.catalog.Table
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.hudi.SparkAdapter
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
@@ -131,4 +134,19 @@ abstract class BaseSpark3Adapter extends SparkAdapter {
|
||||
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
|
||||
throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark3.1.x")
|
||||
}
|
||||
override def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = {
|
||||
// since spark3.2.1 support datasourceV2, so we need to a new SqlParser to deal DDL statment
|
||||
if (SPARK_VERSION.startsWith("3.1")) {
|
||||
val loadClassName = "org.apache.spark.sql.parser.HoodieSpark312ExtendedSqlParser"
|
||||
Some {
|
||||
(spark: SparkSession, delegate: ParserInterface) => {
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(spark, delegate).asInstanceOf[ParserInterface]
|
||||
}
|
||||
}
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -46,6 +46,7 @@
|
||||
<configuration>
|
||||
<args>
|
||||
<arg>-nobootcp</arg>
|
||||
<arg>-target:jvm-1.8</arg>
|
||||
</args>
|
||||
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
|
||||
</configuration>
|
||||
@@ -157,7 +158,7 @@
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.12</artifactId>
|
||||
<version>${spark3.version}</version>
|
||||
<version>${spark31.version}</version>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
@@ -181,16 +182,36 @@
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
@@ -202,6 +223,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
@@ -209,7 +231,14 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
@@ -218,6 +247,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
@@ -225,6 +255,12 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
|
||||
@@ -0,0 +1,188 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet;
|
||||
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.spark.memory.MemoryMode;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.ZoneId;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class Spark312HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
|
||||
|
||||
// save the col type change info.
|
||||
private Map<Integer, Pair<DataType, DataType>> typeChangeInfos;
|
||||
|
||||
private ColumnarBatch columnarBatch;
|
||||
|
||||
private Map<Integer, WritableColumnVector> idToColumnVectors;
|
||||
|
||||
private WritableColumnVector[] columnVectors;
|
||||
|
||||
// The capacity of vectorized batch.
|
||||
private int capacity;
|
||||
|
||||
// If true, this class returns batches instead of rows.
|
||||
private boolean returnColumnarBatch;
|
||||
|
||||
// The memory mode of the columnarBatch.
|
||||
private final MemoryMode memoryMode;
|
||||
|
||||
/**
|
||||
* Batch of rows that we assemble and the current index we've returned. Every time this
|
||||
* batch is used up (batchIdx == numBatched), we populated the batch.
|
||||
*/
|
||||
private int batchIdx = 0;
|
||||
private int numBatched = 0;
|
||||
|
||||
public Spark312HoodieVectorizedParquetRecordReader(
|
||||
ZoneId convertTz,
|
||||
String datetimeRebaseMode,
|
||||
String int96RebaseMode,
|
||||
boolean useOffHeap,
|
||||
int capacity,
|
||||
Map<Integer, Pair<DataType, DataType>> typeChangeInfos) {
|
||||
super(convertTz, datetimeRebaseMode, int96RebaseMode, useOffHeap, capacity);
|
||||
memoryMode = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
|
||||
this.typeChangeInfos = typeChangeInfos;
|
||||
this.capacity = capacity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initBatch(StructType partitionColumns, InternalRow partitionValues) {
|
||||
super.initBatch(partitionColumns, partitionValues);
|
||||
if (columnVectors == null) {
|
||||
columnVectors = new WritableColumnVector[sparkSchema.length() + partitionColumns.length()];
|
||||
}
|
||||
if (idToColumnVectors == null) {
|
||||
idToColumnVectors = new HashMap<>();
|
||||
typeChangeInfos.entrySet()
|
||||
.stream()
|
||||
.forEach(f -> {
|
||||
WritableColumnVector vector =
|
||||
memoryMode == MemoryMode.OFF_HEAP ? new OffHeapColumnVector(capacity, f.getValue().getLeft()) : new OnHeapColumnVector(capacity, f.getValue().getLeft());
|
||||
idToColumnVectors.put(f.getKey(), vector);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException {
|
||||
super.initialize(inputSplit, taskAttemptContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
for (Map.Entry<Integer, WritableColumnVector> e : idToColumnVectors.entrySet()) {
|
||||
e.getValue().close();
|
||||
}
|
||||
idToColumnVectors = null;
|
||||
columnarBatch = null;
|
||||
columnVectors = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnarBatch resultBatch() {
|
||||
ColumnarBatch currentColumnBatch = super.resultBatch();
|
||||
boolean changed = false;
|
||||
for (Map.Entry<Integer, Pair<DataType, DataType>> entry : typeChangeInfos.entrySet()) {
|
||||
boolean rewrite = SparkInternalSchemaConverter
|
||||
.convertColumnVectorType((WritableColumnVector) currentColumnBatch.column(entry.getKey()),
|
||||
idToColumnVectors.get(entry.getKey()), currentColumnBatch.numRows());
|
||||
if (rewrite) {
|
||||
changed = true;
|
||||
columnVectors[entry.getKey()] = idToColumnVectors.get(entry.getKey());
|
||||
}
|
||||
}
|
||||
if (changed) {
|
||||
if (columnarBatch == null) {
|
||||
// fill other vector
|
||||
for (int i = 0; i < columnVectors.length; i++) {
|
||||
if (columnVectors[i] == null) {
|
||||
columnVectors[i] = (WritableColumnVector) currentColumnBatch.column(i);
|
||||
}
|
||||
}
|
||||
columnarBatch = new ColumnarBatch(columnVectors);
|
||||
}
|
||||
columnarBatch.setNumRows(currentColumnBatch.numRows());
|
||||
return columnarBatch;
|
||||
} else {
|
||||
return currentColumnBatch;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextBatch() throws IOException {
|
||||
boolean result = super.nextBatch();
|
||||
if (idToColumnVectors != null) {
|
||||
idToColumnVectors.entrySet().stream().forEach(e -> e.getValue().reset());
|
||||
}
|
||||
numBatched = resultBatch().numRows();
|
||||
batchIdx = 0;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void enableReturningBatches() {
|
||||
returnColumnarBatch = true;
|
||||
super.enableReturningBatches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCurrentValue() {
|
||||
if (typeChangeInfos == null || typeChangeInfos.isEmpty()) {
|
||||
return super.getCurrentValue();
|
||||
}
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch;
|
||||
}
|
||||
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch.getRow(batchIdx - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextKeyValue() throws IOException {
|
||||
resultBatch();
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return nextBatch();
|
||||
}
|
||||
|
||||
if (batchIdx >= numBatched) {
|
||||
if (!nextBatch()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
++batchIdx;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -23,6 +23,11 @@ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConver
|
||||
import org.apache.spark.sql.hudi.SparkAdapter
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_1CatalystExpressionUtils}
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_1CatalystExpressionUtils, SparkSession}
|
||||
|
||||
/**
|
||||
* Implementation of [[SparkAdapter]] for Spark 3.1.x
|
||||
@@ -37,4 +42,27 @@ class Spark3_1Adapter extends BaseSpark3Adapter {
|
||||
override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
|
||||
new HoodieSpark3_1AvroDeserializer(rootAvroType, rootCatalystType)
|
||||
|
||||
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
|
||||
if (SPARK_VERSION.startsWith("3.1")) {
|
||||
val loadClassName = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommand312"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(sparkSession).asInstanceOf[Rule[LogicalPlan]]
|
||||
} else {
|
||||
new Rule[LogicalPlan] {
|
||||
override def apply(plan: LogicalPlan): LogicalPlan = plan
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def createHoodieParquetFileFormat(): Option[ParquetFileFormat] = {
|
||||
if (SPARK_VERSION.startsWith("3.1")) {
|
||||
val loadClassName = "org.apache.spark.sql.execution.datasources.parquet.Spark312HoodieParquetFileFormat"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
Some(ctor.newInstance().asInstanceOf[ParquetFileFormat])
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.spark.sql.catalyst.plans.logical
|
||||
|
||||
import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... ADD COLUMNS command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableAddColumnsStatement(
|
||||
tableName: Seq[String],
|
||||
columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableAlterColumnStatement(
|
||||
tableName: Seq[String],
|
||||
column: Seq[String],
|
||||
dataType: Option[DataType],
|
||||
nullable: Option[Boolean],
|
||||
comment: Option[String],
|
||||
position: Option[ColumnPosition]) extends ParsedStatement
|
||||
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... RENAME COLUMN command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableRenameColumnStatement(
|
||||
tableName: Seq[String],
|
||||
column: Seq[String],
|
||||
newName: String) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... DROP COLUMNS command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableDropColumnsStatement(
|
||||
tableName: Seq[String], columnsToDrop: Seq[Seq[String]]) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableSetPropertiesStatement(
|
||||
tableName: Seq[String], properties: Map[String, String]) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableUnsetPropertiesStatement(
|
||||
tableName: Seq[String], propertyKeys: Seq[String], ifExists: Boolean) extends ParsedStatement
|
||||
@@ -0,0 +1,365 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet
|
||||
|
||||
import java.net.URI
|
||||
import java.util
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
|
||||
import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.HoodieSparkUtils
|
||||
import org.apache.hudi.common.util.InternalSchemaCache
|
||||
import org.apache.hudi.common.util.collection.Pair
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaMerger
|
||||
import org.apache.parquet.filter2.compat.FilterCompat
|
||||
import org.apache.parquet.filter2.predicate.FilterApi
|
||||
import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
|
||||
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
|
||||
|
||||
import org.apache.spark.TaskContext
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
|
||||
import org.apache.spark.sql.execution.datasources.parquet._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType}
|
||||
import org.apache.spark.util.SerializableConfiguration
|
||||
|
||||
class Spark312HoodieParquetFileFormat extends ParquetFileFormat {
|
||||
|
||||
// reference ParquetFileFormat from spark project
|
||||
override def buildReaderWithPartitionValues(
|
||||
sparkSession: SparkSession,
|
||||
dataSchema: StructType,
|
||||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
if (hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, "").isEmpty) {
|
||||
// fallback to origin parquet File read
|
||||
super.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
|
||||
} else {
|
||||
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
|
||||
hadoopConf.set(
|
||||
ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
ParquetWriteSupport.SPARK_ROW_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
SQLConf.SESSION_LOCAL_TIMEZONE.key,
|
||||
sparkSession.sessionState.conf.sessionLocalTimeZone)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
|
||||
sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.CASE_SENSITIVE.key,
|
||||
sparkSession.sessionState.conf.caseSensitiveAnalysis)
|
||||
|
||||
ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
|
||||
|
||||
// Sets flags for `ParquetToSparkSchemaConverter`
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_BINARY_AS_STRING.key,
|
||||
sparkSession.sessionState.conf.isParquetBinaryAsString)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
|
||||
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
|
||||
// for dataSource v1, we have no method to do project for spark physical plan.
|
||||
// it's safe to do cols project here.
|
||||
val internalSchemaString = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
if (querySchemaOption.isPresent && !requiredSchema.isEmpty) {
|
||||
val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get())
|
||||
hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(prunedSchema))
|
||||
}
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
// TODO: if you move this into the closure it reverts to the default values.
|
||||
// If true, enable using the custom RecordReader for parquet. This only works for
|
||||
// a subset of the types (no complex types).
|
||||
val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
|
||||
val sqlConf = sparkSession.sessionState.conf
|
||||
val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled
|
||||
val enableVectorizedReader: Boolean =
|
||||
sqlConf.parquetVectorizedReaderEnabled &&
|
||||
resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
|
||||
val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
|
||||
val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
|
||||
val capacity = sqlConf.parquetVectorizedReaderBatchSize
|
||||
val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
|
||||
// Whole stage codegen (PhysicalRDD) is able to deal with batches directly
|
||||
val returningBatch = supportBatch(sparkSession, resultSchema)
|
||||
val pushDownDate = sqlConf.parquetFilterPushDownDate
|
||||
val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
|
||||
val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
|
||||
val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
|
||||
val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
|
||||
val isCaseSensitive = sqlConf.caseSensitiveAnalysis
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
assert(file.partitionValues.numFields == partitionSchema.size)
|
||||
val filePath = new Path(new URI(file.filePath))
|
||||
val split =
|
||||
new org.apache.parquet.hadoop.ParquetInputSplit(
|
||||
filePath,
|
||||
file.start,
|
||||
file.start + file.length,
|
||||
file.length,
|
||||
Array.empty,
|
||||
null)
|
||||
val sharedConf = broadcastedHadoopConf.value.value
|
||||
// do deal with internalSchema
|
||||
val internalSchemaString = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
// querySchema must be a pruned schema.
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
val internalSchemaChangeEnabled = if (internalSchemaString.isEmpty || !querySchemaOption.isPresent) false else true
|
||||
val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
|
||||
val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
|
||||
val fileSchema = if (internalSchemaChangeEnabled) {
|
||||
val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
|
||||
InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits)
|
||||
} else {
|
||||
// this should not happened, searchSchemaAndCache will deal with correctly.
|
||||
null
|
||||
}
|
||||
|
||||
lazy val footerFileMetaData =
|
||||
ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
|
||||
val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode(
|
||||
footerFileMetaData.getKeyValueMetaData.get,
|
||||
SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ))
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
val pushed = if (enableParquetFilterPushDown) {
|
||||
val parquetSchema = footerFileMetaData.getSchema
|
||||
val parquetFilters = if (HoodieSparkUtils.gteqSpark3_1_3) {
|
||||
Spark312HoodieParquetFileFormat.createParquetFilters(
|
||||
parquetSchema,
|
||||
pushDownDate,
|
||||
pushDownTimestamp,
|
||||
pushDownDecimal,
|
||||
pushDownStringStartWith,
|
||||
pushDownInFilterThreshold,
|
||||
isCaseSensitive,
|
||||
datetimeRebaseMode)
|
||||
} else {
|
||||
Spark312HoodieParquetFileFormat.createParquetFilters(
|
||||
parquetSchema,
|
||||
pushDownDate,
|
||||
pushDownTimestamp,
|
||||
pushDownDecimal,
|
||||
pushDownStringStartWith,
|
||||
pushDownInFilterThreshold,
|
||||
isCaseSensitive)
|
||||
}
|
||||
filters.map(Spark312HoodieParquetFileFormat.rebuildFilterFromParquet(_, fileSchema, querySchemaOption.get()))
|
||||
// Collects all converted Parquet filter predicates. Notice that not all predicates can be
|
||||
// converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
|
||||
// is used here.
|
||||
.flatMap(parquetFilters.createFilter(_))
|
||||
.reduceOption(FilterApi.and)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
// PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
|
||||
// *only* if the file was created by something other than "parquet-mr", so check the actual
|
||||
// writer here for this file. We have to do this per-file, as each file in the table may
|
||||
// have different writers.
|
||||
// Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
|
||||
def isCreatedByParquetMr: Boolean =
|
||||
footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
|
||||
|
||||
val convertTz =
|
||||
if (timestampConversion && !isCreatedByParquetMr) {
|
||||
Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
val int96RebaseMode = DataSourceUtils.int96RebaseMode(
|
||||
footerFileMetaData.getKeyValueMetaData.get,
|
||||
SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ))
|
||||
|
||||
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
|
||||
// use new conf
|
||||
val hadoopAttempConf = new Configuration(broadcastedHadoopConf.value.value)
|
||||
//
|
||||
// reset request schema
|
||||
var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = new java.util.HashMap()
|
||||
if (internalSchemaChangeEnabled) {
|
||||
val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema()
|
||||
val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema)
|
||||
typeChangeInfos = SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
|
||||
hadoopAttempConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json)
|
||||
}
|
||||
val hadoopAttemptContext =
|
||||
new TaskAttemptContextImpl(hadoopAttempConf, attemptId)
|
||||
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
// Notice: This push-down is RowGroups level, not individual records.
|
||||
if (pushed.isDefined) {
|
||||
ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
|
||||
}
|
||||
val taskContext = Option(TaskContext.get())
|
||||
if (enableVectorizedReader) {
|
||||
val vectorizedReader = new Spark312HoodieVectorizedParquetRecordReader(
|
||||
convertTz.orNull,
|
||||
datetimeRebaseMode.toString,
|
||||
int96RebaseMode.toString,
|
||||
enableOffHeapColumnVector && taskContext.isDefined,
|
||||
capacity, typeChangeInfos)
|
||||
val iter = new RecordReaderIterator(vectorizedReader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
vectorizedReader.initialize(split, hadoopAttemptContext)
|
||||
logDebug(s"Appending $partitionSchema ${file.partitionValues}")
|
||||
vectorizedReader.initBatch(partitionSchema, file.partitionValues)
|
||||
if (returningBatch) {
|
||||
vectorizedReader.enableReturningBatches()
|
||||
}
|
||||
|
||||
// UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
|
||||
iter.asInstanceOf[Iterator[InternalRow]]
|
||||
} else {
|
||||
logDebug(s"Falling back to parquet-mr")
|
||||
// ParquetRecordReader returns InternalRow
|
||||
val readSupport = new ParquetReadSupport(
|
||||
convertTz,
|
||||
enableVectorizedReader = false,
|
||||
datetimeRebaseMode,
|
||||
int96RebaseMode)
|
||||
val reader = if (pushed.isDefined && enableRecordFilter) {
|
||||
val parquetFilter = FilterCompat.get(pushed.get, null)
|
||||
new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
|
||||
} else {
|
||||
new ParquetRecordReader[InternalRow](readSupport)
|
||||
}
|
||||
val iter = new RecordReaderIterator[InternalRow](reader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
reader.initialize(split, hadoopAttemptContext)
|
||||
|
||||
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
|
||||
val unsafeProjection = if (typeChangeInfos.isEmpty) {
|
||||
GenerateUnsafeProjection.generate(fullSchema, fullSchema)
|
||||
} else {
|
||||
// find type changed.
|
||||
val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata)
|
||||
} else f
|
||||
}).toAttributes ++ partitionSchema.toAttributes
|
||||
val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
Cast(attr, typeChangeInfos.get(i).getLeft)
|
||||
} else attr
|
||||
}
|
||||
GenerateUnsafeProjection.generate(castSchema, newFullSchema)
|
||||
}
|
||||
|
||||
if (partitionSchema.length == 0) {
|
||||
// There is no partition columns
|
||||
iter.map(unsafeProjection)
|
||||
} else {
|
||||
val joinedRow = new JoinedRow()
|
||||
iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object Spark312HoodieParquetFileFormat {
|
||||
|
||||
val PARQUET_FILTERS_CLASS_NAME = "org.apache.spark.sql.execution.datasources.parquet.ParquetFilters"
|
||||
|
||||
private def createParquetFilters(arg: Any*): ParquetFilters = {
|
||||
val clazz = Class.forName(PARQUET_FILTERS_CLASS_NAME, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(arg.map(_.asInstanceOf[AnyRef]): _*).asInstanceOf[ParquetFilters]
|
||||
}
|
||||
|
||||
private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = {
|
||||
if (fileSchema == null || querySchema == null) {
|
||||
oldFilter
|
||||
} else {
|
||||
oldFilter match {
|
||||
case eq: EqualTo =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute)
|
||||
case eqs: EqualNullSafe =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute)
|
||||
case gt: GreaterThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute)
|
||||
case gtr: GreaterThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute)
|
||||
case lt: LessThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute)
|
||||
case lte: LessThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute)
|
||||
case i: In =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute)
|
||||
case isn: IsNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute)
|
||||
case isnn: IsNotNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute)
|
||||
case And(left, right) =>
|
||||
And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Or(left, right) =>
|
||||
Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Not(child) =>
|
||||
Not(rebuildFilterFromParquet(child, fileSchema, querySchema))
|
||||
case ssw: StringStartsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute)
|
||||
case ses: StringEndsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute)
|
||||
case sc: StringContains =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute)
|
||||
case AlwaysTrue =>
|
||||
AlwaysTrue
|
||||
case AlwaysFalse =>
|
||||
AlwaysFalse
|
||||
case _ =>
|
||||
AlwaysTrue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,282 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import java.util.Locale
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableChange}
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.connector.catalog.CatalogV2Util.failNullType
|
||||
import org.apache.spark.sql.connector.catalog.TableChange._
|
||||
import org.apache.spark.sql.hudi.command.AlterTableCommand312
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
* Rule to mostly resolve, normalize and rewrite column names based on case sensitivity
|
||||
* for alter table column commands.
|
||||
* TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
*/
|
||||
case class ResolveHudiAlterTableCommand312(sparkSession: SparkSession) extends Rule[LogicalPlan] {
|
||||
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
|
||||
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
|
||||
case add @ HoodieAlterTableAddColumnsStatement(asTable(table), cols) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled){
|
||||
cols.foreach(c => CatalogV2Util.failNullType(c.dataType))
|
||||
val changes = cols.map { col =>
|
||||
TableChange.addColumn(
|
||||
col.name.toArray,
|
||||
col.dataType,
|
||||
col.nullable,
|
||||
col.comment.orNull,
|
||||
col.position.orNull)
|
||||
}
|
||||
val newChanges = normalizeChanges(changes, table.schema)
|
||||
AlterTableCommand312(table, newChanges, ColumnChangeID.ADD)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
|
||||
}
|
||||
case a @ HoodieAlterTableAlterColumnStatement(asTable(table), _, _, _, _, _) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled){
|
||||
a.dataType.foreach(failNullType)
|
||||
val colName = a.column.toArray
|
||||
val typeChange = a.dataType.map { newDataType =>
|
||||
TableChange.updateColumnType(colName, newDataType)
|
||||
}
|
||||
val nullabilityChange = a.nullable.map { nullable =>
|
||||
TableChange.updateColumnNullability(colName, nullable)
|
||||
}
|
||||
val commentChange = a.comment.map { newComment =>
|
||||
TableChange.updateColumnComment(colName, newComment)
|
||||
}
|
||||
val positionChange = a.position.map { newPosition =>
|
||||
TableChange.updateColumnPosition(colName, newPosition)
|
||||
}
|
||||
AlterTableCommand312(table, normalizeChanges(typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange, table.schema), ColumnChangeID.UPDATE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableAlterColumnStatement(a.tableName, a.column, a.dataType, a.nullable, a.comment, a.position)
|
||||
}
|
||||
case rename @ HoodieAlterTableRenameColumnStatement(asTable(table), col, newName) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled){
|
||||
val changes = Seq(TableChange.renameColumn(col.toArray, newName))
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.UPDATE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
|
||||
}
|
||||
case drop @ HoodieAlterTableDropColumnsStatement(asTable(table), cols) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled) {
|
||||
val changes = cols.map(col => TableChange.deleteColumn(col.toArray))
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.DELETE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
|
||||
}
|
||||
case set @ HoodieAlterTableSetPropertiesStatement(asTable(table), props) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled) {
|
||||
val changes = props.map { case (key, value) =>
|
||||
TableChange.setProperty(key, value)
|
||||
}.toSeq
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableSetPropertiesStatement(set.tableName, set.properties)
|
||||
}
|
||||
case unset @ HoodieAlterTableUnsetPropertiesStatement(asTable(table), keys, _) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled) {
|
||||
val changes = keys.map(key => TableChange.removeProperty(key))
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableUnsetPropertiesStatement(unset.tableName, unset.propertyKeys, unset.ifExists)
|
||||
}
|
||||
}
|
||||
|
||||
private def schemaEvolutionEnabled(): Boolean = sparkSession
|
||||
.sessionState.conf.getConfString(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean
|
||||
|
||||
private def isHoodieTable(table: CatalogTable): Boolean = table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"
|
||||
|
||||
def normalizeChanges(changes: Seq[TableChange], schema: StructType): Seq[TableChange] = {
|
||||
val colsToAdd = mutable.Map.empty[Seq[String], Seq[String]]
|
||||
changes.flatMap {
|
||||
case add: AddColumn =>
|
||||
def addColumn(parentSchema: StructType, parentName: String, normalizedParentName: Seq[String]): TableChange = {
|
||||
val fieldsAdded = colsToAdd.getOrElse(normalizedParentName, Nil)
|
||||
val pos = findColumnPosition(add.position(), parentName, parentSchema, fieldsAdded)
|
||||
val field = add.fieldNames().last
|
||||
colsToAdd(normalizedParentName) = fieldsAdded :+ field
|
||||
TableChange.addColumn(
|
||||
(normalizedParentName :+ field).toArray,
|
||||
add.dataType(),
|
||||
add.isNullable,
|
||||
add.comment,
|
||||
pos)
|
||||
}
|
||||
val parent = add.fieldNames().init
|
||||
if (parent.nonEmpty) {
|
||||
// Adding a nested field, need to normalize the parent column and position
|
||||
val target = schema.findNestedField(parent, includeCollections = true, conf.resolver)
|
||||
if (target.isEmpty) {
|
||||
// Leave unresolved. Throws error in CheckAnalysis
|
||||
Some(add)
|
||||
} else {
|
||||
val (normalizedName, sf) = target.get
|
||||
sf.dataType match {
|
||||
case struct: StructType =>
|
||||
Some(addColumn(struct, parent.quoted, normalizedName :+ sf.name))
|
||||
case other =>
|
||||
Some(add)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Adding to the root. Just need to normalize position
|
||||
Some(addColumn(schema, "root", Nil))
|
||||
}
|
||||
|
||||
case typeChange: UpdateColumnType =>
|
||||
// Hive style syntax provides the column type, even if it may not have changed
|
||||
val fieldOpt = schema.findNestedField(
|
||||
typeChange.fieldNames(), includeCollections = true, conf.resolver)
|
||||
|
||||
if (fieldOpt.isEmpty) {
|
||||
// We couldn't resolve the field. Leave it to CheckAnalysis
|
||||
Some(typeChange)
|
||||
} else {
|
||||
val (fieldNames, field) = fieldOpt.get
|
||||
if (field.dataType == typeChange.newDataType()) {
|
||||
// The user didn't want the field to change, so remove this change
|
||||
None
|
||||
} else {
|
||||
Some(TableChange.updateColumnType(
|
||||
(fieldNames :+ field.name).toArray, typeChange.newDataType()))
|
||||
}
|
||||
}
|
||||
case n: UpdateColumnNullability =>
|
||||
// Need to resolve column
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
n.fieldNames(),
|
||||
TableChange.updateColumnNullability(_, n.nullable())).orElse(Some(n))
|
||||
|
||||
case position: UpdateColumnPosition =>
|
||||
position.position() match {
|
||||
case after: After =>
|
||||
// Need to resolve column as well as position reference
|
||||
val fieldOpt = schema.findNestedField(
|
||||
position.fieldNames(), includeCollections = true, conf.resolver)
|
||||
|
||||
if (fieldOpt.isEmpty) {
|
||||
Some(position)
|
||||
} else {
|
||||
val (normalizedPath, field) = fieldOpt.get
|
||||
val targetCol = schema.findNestedField(
|
||||
normalizedPath :+ after.column(), includeCollections = true, conf.resolver)
|
||||
if (targetCol.isEmpty) {
|
||||
// Leave unchanged to CheckAnalysis
|
||||
Some(position)
|
||||
} else {
|
||||
Some(TableChange.updateColumnPosition(
|
||||
(normalizedPath :+ field.name).toArray,
|
||||
ColumnPosition.after(targetCol.get._2.name)))
|
||||
}
|
||||
}
|
||||
case _ =>
|
||||
// Need to resolve column
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
position.fieldNames(),
|
||||
TableChange.updateColumnPosition(_, position.position())).orElse(Some(position))
|
||||
}
|
||||
|
||||
case comment: UpdateColumnComment =>
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
comment.fieldNames(),
|
||||
TableChange.updateColumnComment(_, comment.newComment())).orElse(Some(comment))
|
||||
|
||||
case rename: RenameColumn =>
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
rename.fieldNames(),
|
||||
TableChange.renameColumn(_, rename.newName())).orElse(Some(rename))
|
||||
|
||||
case delete: DeleteColumn =>
|
||||
resolveFieldNames(schema, delete.fieldNames(), TableChange.deleteColumn)
|
||||
.orElse(Some(delete))
|
||||
|
||||
case column: ColumnChange =>
|
||||
// This is informational for future developers
|
||||
throw new UnsupportedOperationException(
|
||||
"Please add an implementation for a column change here")
|
||||
case other => Some(other)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the table change if the field can be resolved, returns None if the column is not
|
||||
* found. An error will be thrown in CheckAnalysis for columns that can't be resolved.
|
||||
*/
|
||||
private def resolveFieldNames(
|
||||
schema: StructType,
|
||||
fieldNames: Array[String],
|
||||
copy: Array[String] => TableChange): Option[TableChange] = {
|
||||
val fieldOpt = schema.findNestedField(
|
||||
fieldNames, includeCollections = true, conf.resolver)
|
||||
fieldOpt.map { case (path, field) => copy((path :+ field.name).toArray) }
|
||||
}
|
||||
|
||||
private def findColumnPosition(
|
||||
position: ColumnPosition,
|
||||
parentName: String,
|
||||
struct: StructType,
|
||||
fieldsAdded: Seq[String]): ColumnPosition = {
|
||||
position match {
|
||||
case null => null
|
||||
case after: After =>
|
||||
(struct.fieldNames ++ fieldsAdded).find(n => conf.resolver(n, after.column())) match {
|
||||
case Some(colName) =>
|
||||
ColumnPosition.after(colName)
|
||||
case None =>
|
||||
throw new AnalysisException("Couldn't find the reference column for " +
|
||||
s"$after at $parentName")
|
||||
}
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
|
||||
object asTable {
|
||||
def unapply(parts: Seq[String]): Option[CatalogTable] = {
|
||||
val identifier = parts match {
|
||||
case Seq(tblName) => TableIdentifier(tblName)
|
||||
case Seq(dbName, tblName) => TableIdentifier(tblName, Some(dbName))
|
||||
case _ =>
|
||||
throw new AnalysisException(
|
||||
s"${parts} is not a valid TableIdentifier as it has more than 2 name parts.")
|
||||
}
|
||||
Some(sparkSession.sessionState.catalog.getTableMetadata(identifier))
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,324 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi.command
|
||||
|
||||
import java.net.URI
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceOptionsHelper, DataSourceUtils}
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, Option}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.hudi.internal.schema.action.TableChanges
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.utils.{SchemaChangeUtils, SerDeHelper}
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
|
||||
import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange}
|
||||
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RemoveProperty, SetProperty}
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends RunnableCommand with Logging {
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
changeType match {
|
||||
case ColumnChangeID.ADD => applyAddAction(sparkSession)
|
||||
case ColumnChangeID.DELETE => applyDeleteAction(sparkSession)
|
||||
case ColumnChangeID.UPDATE => applyUpdateAction(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[SetProperty]).size == changes.size) =>
|
||||
applyPropertySet(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[RemoveProperty]).size == changes.size) =>
|
||||
applyPropertyUnset(sparkSession)
|
||||
case other => throw new RuntimeException(s"find unsupported alter command type: ${other}")
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
def applyAddAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val addChange = TableChanges.ColumnAddChange.get(oldSchema)
|
||||
changes.map(_.asInstanceOf[AddColumn]).foreach { addColumn =>
|
||||
val names = addColumn.fieldNames()
|
||||
val parentName = AlterTableCommand312.getParentName(names)
|
||||
// add col change
|
||||
val colType = SparkInternalSchemaConverter.buildTypeFromStructType(addColumn.dataType(), true, new AtomicInteger(0))
|
||||
addChange.addColumns(parentName, names.last, colType, addColumn.comment())
|
||||
// add position change
|
||||
addColumn.position() match {
|
||||
case after: TableChange.After =>
|
||||
addChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
addChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column add finished")
|
||||
}
|
||||
|
||||
def applyDeleteAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
|
||||
changes.map(_.asInstanceOf[DeleteColumn]).foreach { c =>
|
||||
val originalColName = c.fieldNames().mkString(".");
|
||||
AlterTableCommand312.checkSchemaChange(Seq(originalColName), table)
|
||||
deleteChange.deleteColumn(originalColName)
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange)
|
||||
// delete action should not change the getMaxColumnId field.
|
||||
newSchema.setMaxColumnId(oldSchema.getMaxColumnId)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column delete finished")
|
||||
}
|
||||
|
||||
def applyUpdateAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val updateChange = TableChanges.ColumnUpdateChange.get(oldSchema)
|
||||
changes.foreach { change =>
|
||||
change match {
|
||||
case updateType: TableChange.UpdateColumnType =>
|
||||
val newType = SparkInternalSchemaConverter.buildTypeFromStructType(updateType.newDataType(), true, new AtomicInteger(0))
|
||||
updateChange.updateColumnType(updateType.fieldNames().mkString("."), newType)
|
||||
case updateComment: TableChange.UpdateColumnComment =>
|
||||
updateChange.updateColumnComment(updateComment.fieldNames().mkString("."), updateComment.newComment())
|
||||
case updateName: TableChange.RenameColumn =>
|
||||
val originalColName = updateName.fieldNames().mkString(".")
|
||||
AlterTableCommand312.checkSchemaChange(Seq(originalColName), table)
|
||||
updateChange.renameColumn(originalColName, updateName.newName())
|
||||
case updateNullAbility: TableChange.UpdateColumnNullability =>
|
||||
updateChange.updateColumnNullability(updateNullAbility.fieldNames().mkString("."), updateNullAbility.nullable())
|
||||
case updatePosition: TableChange.UpdateColumnPosition =>
|
||||
val names = updatePosition.fieldNames()
|
||||
val parentName = AlterTableCommand312.getParentName(names)
|
||||
updatePosition.position() match {
|
||||
case after: TableChange.After =>
|
||||
updateChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
updateChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column update finished")
|
||||
}
|
||||
|
||||
// to do support unset default value to columns, and apply them to internalSchema
|
||||
def applyPropertyUnset(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val propKeys = changes.map(_.asInstanceOf[RemoveProperty]).map(_.property())
|
||||
// ignore NonExist unset
|
||||
propKeys.foreach { k =>
|
||||
if (!table.properties.contains(k) && k != TableCatalog.PROP_COMMENT) {
|
||||
logWarning(s"find non exist unset property: ${k} , ignore it")
|
||||
}
|
||||
}
|
||||
val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment
|
||||
val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) }
|
||||
val newTable = table.copy(properties = newProperties, comment = tableComment)
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
// to do support set default value to columns, and apply them to internalSchema
|
||||
def applyPropertySet(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val properties = changes.map(_.asInstanceOf[SetProperty]).map(f => f.property -> f.value).toMap
|
||||
// This overrides old properties and update the comment parameter of CatalogTable
|
||||
// with the newly added/modified comment since CatalogTable also holds comment as its
|
||||
// direct property.
|
||||
val newTable = table.copy(
|
||||
properties = table.properties ++ properties,
|
||||
comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment))
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
def getInternalSchemaAndHistorySchemaStr(sparkSession: SparkSession): (InternalSchema, String) = {
|
||||
val path = AlterTableCommand312.getTableLocation(table, sparkSession)
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
|
||||
.setConf(hadoopConf).build()
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
|
||||
val schema = schemaUtil.getTableInternalSchemaFromCommitMetadata().orElse {
|
||||
AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema)
|
||||
}
|
||||
|
||||
val historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata.orElse("")
|
||||
(schema, historySchemaStr)
|
||||
}
|
||||
}
|
||||
|
||||
object AlterTableCommand312 extends Logging {
|
||||
|
||||
/**
|
||||
* Generate an commit with new schema to change the table's schema.
|
||||
* @param internalSchema new schema after change
|
||||
* @param historySchemaStr history schemas
|
||||
* @param table The hoodie table.
|
||||
* @param sparkSession The spark session.
|
||||
*/
|
||||
def commitWithSchema(internalSchema: InternalSchema, historySchemaStr: String, table: CatalogTable, sparkSession: SparkSession): Unit = {
|
||||
val schema = AvroInternalSchemaConverter.convert(internalSchema, table.identifier.table)
|
||||
val path = getTableLocation(table, sparkSession)
|
||||
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
val client = DataSourceUtils.createHoodieClient(jsc, schema.toString,
|
||||
path, table.identifier.table, parametersWithWriteDefaults(table.storage.properties).asJava)
|
||||
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build()
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext)
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
val extraMeta = new util.HashMap[String, String]()
|
||||
extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema.setSchemaId(instantTime.toLong)))
|
||||
val schemaManager = new FileBasedInternalSchemaStorageManager(metaClient)
|
||||
schemaManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(internalSchema, historySchemaStr))
|
||||
client.commit(instantTime, jsc.emptyRDD, Option.of(extraMeta))
|
||||
val existRoTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_ro")
|
||||
val existRtTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_rt")
|
||||
try {
|
||||
sparkSession.catalog.refreshTable(table.identifier.unquotedString)
|
||||
// try to refresh ro/rt table
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_ro")
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_rt")
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
log.error(s"Exception when attempting to refresh table ${table.identifier.quotedString}", e)
|
||||
}
|
||||
// try to sync to hive
|
||||
// drop partition field before call alter table
|
||||
val fullSparkSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(internalSchema)
|
||||
val dataSparkSchema = new StructType(fullSparkSchema.fields.filter(p => !table.partitionColumnNames.exists(f => sparkSession.sessionState.conf.resolver(f, p.name))))
|
||||
alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table, dataSparkSchema)
|
||||
if (existRoTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_ro", dataSparkSchema)
|
||||
if (existRtTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_rt", dataSparkSchema)
|
||||
}
|
||||
|
||||
def alterTableDataSchema(sparkSession: SparkSession, db: String, tableName: String, dataSparkSchema: StructType): Unit = {
|
||||
sparkSession.sessionState.catalog
|
||||
.externalCatalog
|
||||
.alterTableDataSchema(db, tableName, dataSparkSchema)
|
||||
}
|
||||
|
||||
def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
|
||||
val uri = if (table.tableType == CatalogTableType.MANAGED) {
|
||||
Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
|
||||
} else {
|
||||
table.storage.locationUri
|
||||
}
|
||||
val conf = sparkSession.sessionState.newHadoopConf()
|
||||
uri.map(makePathQualified(_, conf))
|
||||
.map(removePlaceHolder)
|
||||
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
|
||||
}
|
||||
|
||||
private def removePlaceHolder(path: String): String = {
|
||||
if (path == null || path.length == 0) {
|
||||
path
|
||||
} else if (path.endsWith("-__PLACEHOLDER__")) {
|
||||
path.substring(0, path.length() - 16)
|
||||
} else {
|
||||
path
|
||||
}
|
||||
}
|
||||
|
||||
def makePathQualified(path: URI, hadoopConf: Configuration): String = {
|
||||
val hadoopPath = new Path(path)
|
||||
val fs = hadoopPath.getFileSystem(hadoopConf)
|
||||
fs.makeQualified(hadoopPath).toUri.toString
|
||||
}
|
||||
|
||||
def getParentName(names: Array[String]): String = {
|
||||
if (names.size > 1) {
|
||||
names.dropRight(1).mkString(".")
|
||||
} else ""
|
||||
}
|
||||
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION.key -> OPERATION.defaultValue,
|
||||
TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
|
||||
PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
|
||||
HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key -> HoodieWriteConfig.DEFAULT_WRITE_PAYLOAD_CLASS,
|
||||
INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
|
||||
ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue,
|
||||
INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue,
|
||||
ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue
|
||||
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
|
||||
def checkSchemaChange(colNames: Seq[String], catalogTable: CatalogTable): Unit = {
|
||||
val primaryKeys = catalogTable.storage.properties.getOrElse("primaryKey", catalogTable.properties.getOrElse("primaryKey", "keyid")).split(",").map(_.trim)
|
||||
val preCombineKey = Seq(catalogTable.storage.properties.getOrElse("preCombineField", catalogTable.properties.getOrElse("preCombineField", "ts"))).map(_.trim)
|
||||
val partitionKey = catalogTable.partitionColumnNames.map(_.trim)
|
||||
val checkNames = primaryKeys ++ preCombineKey ++ partitionKey
|
||||
colNames.foreach { col =>
|
||||
if (checkNames.contains(col)) {
|
||||
throw new UnsupportedOperationException("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.spark.sql.parser
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
|
||||
|
||||
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
class HoodieSpark312ExtendedSqlParser(session: SparkSession, delegate: ParserInterface) extends SparkSqlParser with Logging {
|
||||
override val astBuilder: SparkSqlAstBuilder = new HoodieSpark312SqlAstBuilder(session)
|
||||
}
|
||||
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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.spark.sql.parser
|
||||
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.parser.ParserUtils.withOrigin
|
||||
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.execution.SparkSqlAstBuilder
|
||||
|
||||
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
class HoodieSpark312SqlAstBuilder(sparkSession: SparkSession) extends SparkSqlAstBuilder {
|
||||
|
||||
/**
|
||||
* Parse a [[AlterTableAlterColumnStatement]] command to alter a column's property.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment'
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c FIRST
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c AFTER x
|
||||
* }}}
|
||||
*/
|
||||
override def visitAlterTableAlterColumn(ctx: AlterTableAlterColumnContext): LogicalPlan = withOrigin(ctx) {
|
||||
val alter = super.visitAlterTableAlterColumn(ctx).asInstanceOf[AlterTableAlterColumnStatement]
|
||||
HoodieAlterTableAlterColumnStatement(alter.tableName, alter.column, alter.dataType, alter.nullable, alter.comment, alter.position)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse a [[org.apache.spark.sql.catalyst.plans.logical.AlterTableAddColumnsStatement]] command.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1
|
||||
* ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
|
||||
* }}}
|
||||
*/
|
||||
override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) {
|
||||
val add = super.visitAddTableColumns(ctx).asInstanceOf[AlterTableAddColumnsStatement]
|
||||
HoodieAlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse a [[org.apache.spark.sql.catalyst.plans.logical.AlterTableRenameColumnStatement]] command.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1 RENAME COLUMN a.b.c TO x
|
||||
* }}}
|
||||
*/
|
||||
override def visitRenameTableColumn(
|
||||
ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) {
|
||||
val rename = super.visitRenameTableColumn(ctx).asInstanceOf[AlterTableRenameColumnStatement]
|
||||
HoodieAlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse a [[AlterTableDropColumnsStatement]] command.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1 DROP COLUMN a.b.c
|
||||
* ALTER TABLE table1 DROP COLUMNS a.b.c, x, y
|
||||
* }}}
|
||||
*/
|
||||
override def visitDropTableColumns(
|
||||
ctx: DropTableColumnsContext): LogicalPlan = withOrigin(ctx) {
|
||||
val drop = super.visitDropTableColumns(ctx).asInstanceOf[AlterTableDropColumnsStatement]
|
||||
HoodieAlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetPropertiesStatement]] commands.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table SET TBLPROPERTIES ('table_property' = 'property_value');
|
||||
* ALTER VIEW view SET TBLPROPERTIES ('table_property' = 'property_value');
|
||||
* }}}
|
||||
*/
|
||||
override def visitSetTableProperties(
|
||||
ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
|
||||
val set = super.visitSetTableProperties(ctx)
|
||||
set match {
|
||||
case s: AlterTableSetPropertiesStatement => HoodieAlterTableSetPropertiesStatement(s.tableName, s.properties)
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetPropertiesStatement]] commands.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
|
||||
* ALTER VIEW view UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
|
||||
* }}}
|
||||
*/
|
||||
override def visitUnsetTableProperties(
|
||||
ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
|
||||
val unset = super.visitUnsetTableProperties(ctx)
|
||||
unset match {
|
||||
case us: AlterTableUnsetPropertiesStatement => HoodieAlterTableUnsetPropertiesStatement(us.tableName, us.propertyKeys, us.ifExists)
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -80,6 +80,12 @@
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<args>
|
||||
<arg>-nobootcp</arg>
|
||||
<arg>-target:jvm-1.8</arg>
|
||||
</args>
|
||||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
@@ -175,11 +181,33 @@
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.12</artifactId>
|
||||
<version>${spark3.version}</version>
|
||||
<version>${spark32.version}</version>
|
||||
<scope>provided</scope>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-catalyst_2.12</artifactId>
|
||||
<version>${spark32.version}</version>
|
||||
<scope>provided</scope>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_2.12</artifactId>
|
||||
<version>${spark32.version}</version>
|
||||
<scope>provided</scope>
|
||||
<optional>true</optional>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>*</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
@@ -200,18 +228,49 @@
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.json4s</groupId>
|
||||
<artifactId>json4s-jackson_${scala.binary.version}</artifactId>
|
||||
<version>3.7.0-M11</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
@@ -221,6 +280,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
@@ -228,7 +288,14 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
@@ -237,6 +304,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
@@ -244,6 +312,12 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
|
||||
@@ -0,0 +1,191 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet;
|
||||
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.spark.memory.MemoryMode;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.ZoneId;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class Spark32HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
|
||||
|
||||
// save the col type change info.
|
||||
private Map<Integer, Pair<DataType, DataType>> typeChangeInfos;
|
||||
|
||||
private ColumnarBatch columnarBatch;
|
||||
|
||||
private Map<Integer, WritableColumnVector> idToColumnVectors;
|
||||
|
||||
private WritableColumnVector[] columnVectors;
|
||||
|
||||
// The capacity of vectorized batch.
|
||||
private int capacity;
|
||||
|
||||
// If true, this class returns batches instead of rows.
|
||||
private boolean returnColumnarBatch;
|
||||
|
||||
// The memory mode of the columnarBatch.
|
||||
private final MemoryMode memoryMode;
|
||||
|
||||
/**
|
||||
* Batch of rows that we assemble and the current index we've returned. Every time this
|
||||
* batch is used up (batchIdx == numBatched), we populated the batch.
|
||||
*/
|
||||
private int batchIdx = 0;
|
||||
private int numBatched = 0;
|
||||
|
||||
public Spark32HoodieVectorizedParquetRecordReader(
|
||||
ZoneId convertTz,
|
||||
String datetimeRebaseMode,
|
||||
String datetimeRebaseTz,
|
||||
String int96RebaseMode,
|
||||
String int96RebaseTz,
|
||||
boolean useOffHeap,
|
||||
int capacity,
|
||||
Map<Integer, Pair<DataType, DataType>> typeChangeInfos) {
|
||||
super(convertTz, datetimeRebaseMode, datetimeRebaseTz, int96RebaseMode, int96RebaseTz, useOffHeap, capacity);
|
||||
memoryMode = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
|
||||
this.typeChangeInfos = typeChangeInfos;
|
||||
this.capacity = capacity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initBatch(StructType partitionColumns, InternalRow partitionValues) {
|
||||
super.initBatch(partitionColumns, partitionValues);
|
||||
if (columnVectors == null) {
|
||||
columnVectors = new WritableColumnVector[sparkSchema.length() + partitionColumns.length()];
|
||||
}
|
||||
if (idToColumnVectors == null) {
|
||||
idToColumnVectors = new HashMap<>();
|
||||
typeChangeInfos.entrySet()
|
||||
.stream()
|
||||
.forEach(f -> {
|
||||
WritableColumnVector vector =
|
||||
memoryMode == MemoryMode.OFF_HEAP ? new OffHeapColumnVector(capacity, f.getValue().getLeft()) : new OnHeapColumnVector(capacity, f.getValue().getLeft());
|
||||
idToColumnVectors.put(f.getKey(), vector);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException {
|
||||
super.initialize(inputSplit, taskAttemptContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
for (Map.Entry<Integer, WritableColumnVector> e : idToColumnVectors.entrySet()) {
|
||||
e.getValue().close();
|
||||
}
|
||||
idToColumnVectors = null;
|
||||
columnarBatch = null;
|
||||
columnVectors = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnarBatch resultBatch() {
|
||||
ColumnarBatch currentColumnBatch = super.resultBatch();
|
||||
boolean changed = false;
|
||||
for (Map.Entry<Integer, Pair<DataType, DataType>> entry : typeChangeInfos.entrySet()) {
|
||||
boolean rewrite = SparkInternalSchemaConverter
|
||||
.convertColumnVectorType((WritableColumnVector) currentColumnBatch.column(entry.getKey()),
|
||||
idToColumnVectors.get(entry.getKey()), currentColumnBatch.numRows());
|
||||
if (rewrite) {
|
||||
changed = true;
|
||||
columnVectors[entry.getKey()] = idToColumnVectors.get(entry.getKey());
|
||||
}
|
||||
}
|
||||
if (changed) {
|
||||
if (columnarBatch == null) {
|
||||
// fill other vector
|
||||
for (int i = 0; i < columnVectors.length; i++) {
|
||||
if (columnVectors[i] == null) {
|
||||
columnVectors[i] = (WritableColumnVector) currentColumnBatch.column(i);
|
||||
}
|
||||
}
|
||||
columnarBatch = new ColumnarBatch(columnVectors);
|
||||
}
|
||||
columnarBatch.setNumRows(currentColumnBatch.numRows());
|
||||
return columnarBatch;
|
||||
} else {
|
||||
return currentColumnBatch;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextBatch() throws IOException {
|
||||
boolean result = super.nextBatch();
|
||||
if (idToColumnVectors != null) {
|
||||
idToColumnVectors.entrySet().stream().forEach(e -> e.getValue().reset());
|
||||
}
|
||||
numBatched = resultBatch().numRows();
|
||||
batchIdx = 0;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void enableReturningBatches() {
|
||||
returnColumnarBatch = true;
|
||||
super.enableReturningBatches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCurrentValue() {
|
||||
if (typeChangeInfos == null || typeChangeInfos.isEmpty()) {
|
||||
return super.getCurrentValue();
|
||||
}
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch;
|
||||
}
|
||||
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch.getRow(batchIdx - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextKeyValue() throws IOException {
|
||||
resultBatch();
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return nextBatch();
|
||||
}
|
||||
|
||||
if (batchIdx >= numBatched) {
|
||||
if (!nextBatch()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
++batchIdx;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,6 +22,9 @@ import org.apache.spark.sql.avro._
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_2CatalystExpressionUtils, SparkSession}
|
||||
@@ -63,4 +66,28 @@ class Spark3_2Adapter extends BaseSpark3Adapter {
|
||||
(spark: SparkSession, delegate: ParserInterface) => new HoodieSpark3_2ExtendedSqlParser(spark, delegate)
|
||||
)
|
||||
}
|
||||
|
||||
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
|
||||
if (SPARK_VERSION.startsWith("3.2")) {
|
||||
val loadClassName = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommandSpark32"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(sparkSession).asInstanceOf[Rule[LogicalPlan]]
|
||||
} else {
|
||||
new Rule[LogicalPlan] {
|
||||
override def apply(plan: LogicalPlan): LogicalPlan = plan
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def createHoodieParquetFileFormat(): Option[ParquetFileFormat] = {
|
||||
if (SPARK_VERSION.startsWith("3.2")) {
|
||||
val loadClassName = "org.apache.spark.sql.execution.datasources.parquet.Spark32HoodieParquetFileFormat"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
Some(ctor.newInstance().asInstanceOf[ParquetFileFormat])
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,349 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet
|
||||
|
||||
import java.net.URI
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.mapred.FileSplit
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
|
||||
import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.util.InternalSchemaCache
|
||||
import org.apache.hudi.common.util.collection.Pair
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaMerger
|
||||
import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
|
||||
import org.apache.parquet.filter2.compat.FilterCompat
|
||||
import org.apache.parquet.filter2.predicate.FilterApi
|
||||
import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
|
||||
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
|
||||
import org.apache.spark.TaskContext
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType}
|
||||
import org.apache.spark.util.SerializableConfiguration
|
||||
|
||||
class Spark32HoodieParquetFileFormat extends ParquetFileFormat {
|
||||
|
||||
// reference ParquetFileFormat from spark project
|
||||
override def buildReaderWithPartitionValues(
|
||||
sparkSession: SparkSession,
|
||||
dataSchema: StructType,
|
||||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
if (hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, "").isEmpty) {
|
||||
// fallback to origin parquet File read
|
||||
super.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
|
||||
} else {
|
||||
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
|
||||
hadoopConf.set(
|
||||
ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
ParquetWriteSupport.SPARK_ROW_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
SQLConf.SESSION_LOCAL_TIMEZONE.key,
|
||||
sparkSession.sessionState.conf.sessionLocalTimeZone)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
|
||||
sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.CASE_SENSITIVE.key,
|
||||
sparkSession.sessionState.conf.caseSensitiveAnalysis)
|
||||
|
||||
ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
|
||||
|
||||
// Sets flags for `ParquetToSparkSchemaConverter`
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_BINARY_AS_STRING.key,
|
||||
sparkSession.sessionState.conf.isParquetBinaryAsString)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
|
||||
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
|
||||
// for dataSource v1, we have no method to do project for spark physical plan.
|
||||
// it's safe to do cols project here.
|
||||
val internalSchemaString = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
if (querySchemaOption.isPresent && !requiredSchema.isEmpty) {
|
||||
val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get())
|
||||
hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(prunedSchema))
|
||||
}
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
// TODO: if you move this into the closure it reverts to the default values.
|
||||
// If true, enable using the custom RecordReader for parquet. This only works for
|
||||
// a subset of the types (no complex types).
|
||||
val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
|
||||
val sqlConf = sparkSession.sessionState.conf
|
||||
val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled
|
||||
val enableVectorizedReader: Boolean =
|
||||
sqlConf.parquetVectorizedReaderEnabled &&
|
||||
resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
|
||||
val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
|
||||
val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
|
||||
val capacity = sqlConf.parquetVectorizedReaderBatchSize
|
||||
val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
|
||||
// Whole stage codegen (PhysicalRDD) is able to deal with batches directly
|
||||
val returningBatch = supportBatch(sparkSession, resultSchema)
|
||||
val pushDownDate = sqlConf.parquetFilterPushDownDate
|
||||
val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
|
||||
val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
|
||||
val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
|
||||
val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
|
||||
val isCaseSensitive = sqlConf.caseSensitiveAnalysis
|
||||
val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf)
|
||||
val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead
|
||||
val int96RebaseModeInread = parquetOptions.int96RebaseModeInRead
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
assert(file.partitionValues.numFields == partitionSchema.size)
|
||||
val filePath = new Path(new URI(file.filePath))
|
||||
val split = new FileSplit(filePath, file.start, file.length, Array.empty[String])
|
||||
val sharedConf = broadcastedHadoopConf.value.value
|
||||
// do deal with internalSchema
|
||||
val internalSchemaString = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
// querySchema must be a pruned schema.
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
val internalSchemaChangeEnabled = if (internalSchemaString.isEmpty || !querySchemaOption.isPresent) false else true
|
||||
val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
|
||||
val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
|
||||
val fileSchema = if (internalSchemaChangeEnabled) {
|
||||
val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
|
||||
InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits)
|
||||
} else {
|
||||
// this should not happened, searchSchemaAndCache will deal with correctly.
|
||||
null
|
||||
}
|
||||
|
||||
lazy val footerFileMetaData =
|
||||
ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
|
||||
val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
|
||||
footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
val pushed = if (enableParquetFilterPushDown) {
|
||||
val parquetSchema = footerFileMetaData.getSchema
|
||||
val parquetFilters = new ParquetFilters(
|
||||
parquetSchema,
|
||||
pushDownDate,
|
||||
pushDownTimestamp,
|
||||
pushDownDecimal,
|
||||
pushDownStringStartWith,
|
||||
pushDownInFilterThreshold,
|
||||
isCaseSensitive,
|
||||
datetimeRebaseSpec)
|
||||
filters.map(Spark32HoodieParquetFileFormat.rebuildFilterFromParquet(_, fileSchema, querySchemaOption.get()))
|
||||
// Collects all converted Parquet filter predicates. Notice that not all predicates can be
|
||||
// converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
|
||||
// is used here.
|
||||
.flatMap(parquetFilters.createFilter(_))
|
||||
.reduceOption(FilterApi.and)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
// PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
|
||||
// *only* if the file was created by something other than "parquet-mr", so check the actual
|
||||
// writer here for this file. We have to do this per-file, as each file in the table may
|
||||
// have different writers.
|
||||
// Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
|
||||
def isCreatedByParquetMr: Boolean =
|
||||
footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
|
||||
|
||||
val convertTz =
|
||||
if (timestampConversion && !isCreatedByParquetMr) {
|
||||
Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
val int96RebaseSpec = DataSourceUtils.int96RebaseSpec(
|
||||
footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInread)
|
||||
|
||||
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
|
||||
// use new conf
|
||||
val hadoopAttempConf = new Configuration(broadcastedHadoopConf.value.value)
|
||||
//
|
||||
// reset request schema
|
||||
var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = new java.util.HashMap()
|
||||
if (internalSchemaChangeEnabled) {
|
||||
val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema()
|
||||
val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema)
|
||||
typeChangeInfos = SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
|
||||
hadoopAttempConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json)
|
||||
}
|
||||
val hadoopAttemptContext =
|
||||
new TaskAttemptContextImpl(hadoopAttempConf, attemptId)
|
||||
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
// Notice: This push-down is RowGroups level, not individual records.
|
||||
if (pushed.isDefined) {
|
||||
ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
|
||||
}
|
||||
val taskContext = Option(TaskContext.get())
|
||||
if (enableVectorizedReader) {
|
||||
val vectorizedReader = new Spark32HoodieVectorizedParquetRecordReader(
|
||||
convertTz.orNull,
|
||||
datetimeRebaseSpec.mode.toString,
|
||||
datetimeRebaseSpec.timeZone,
|
||||
int96RebaseSpec.mode.toString,
|
||||
int96RebaseSpec.timeZone,
|
||||
enableOffHeapColumnVector && taskContext.isDefined,
|
||||
capacity, typeChangeInfos)
|
||||
val iter = new RecordReaderIterator(vectorizedReader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
// taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
try {
|
||||
vectorizedReader.initialize(split, hadoopAttemptContext)
|
||||
logDebug(s"Appending $partitionSchema ${file.partitionValues}")
|
||||
vectorizedReader.initBatch(partitionSchema, file.partitionValues)
|
||||
if (returningBatch) {
|
||||
vectorizedReader.enableReturningBatches()
|
||||
}
|
||||
|
||||
// UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
|
||||
iter.asInstanceOf[Iterator[InternalRow]]
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
// SPARK-23457: In case there is an exception in initialization, close the iterator to
|
||||
// avoid leaking resources.
|
||||
iter.close()
|
||||
throw e
|
||||
}
|
||||
} else {
|
||||
logDebug(s"Falling back to parquet-mr")
|
||||
// ParquetRecordReader returns InternalRow
|
||||
val readSupport = new ParquetReadSupport(
|
||||
convertTz,
|
||||
enableVectorizedReader = false,
|
||||
datetimeRebaseSpec,
|
||||
int96RebaseSpec)
|
||||
val reader = if (pushed.isDefined && enableRecordFilter) {
|
||||
val parquetFilter = FilterCompat.get(pushed.get, null)
|
||||
new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
|
||||
} else {
|
||||
new ParquetRecordReader[InternalRow](readSupport)
|
||||
}
|
||||
val iter = new RecordReaderIterator[InternalRow](reader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
reader.initialize(split, hadoopAttemptContext)
|
||||
|
||||
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
|
||||
val unsafeProjection = if (typeChangeInfos.isEmpty) {
|
||||
GenerateUnsafeProjection.generate(fullSchema, fullSchema)
|
||||
} else {
|
||||
// find type changed.
|
||||
val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata)
|
||||
} else f
|
||||
}).toAttributes ++ partitionSchema.toAttributes
|
||||
val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
Cast(attr, typeChangeInfos.get(i).getLeft)
|
||||
} else attr
|
||||
}
|
||||
GenerateUnsafeProjection.generate(castSchema, newFullSchema)
|
||||
}
|
||||
|
||||
if (partitionSchema.length == 0) {
|
||||
// There is no partition columns
|
||||
iter.map(unsafeProjection)
|
||||
} else {
|
||||
val joinedRow = new JoinedRow()
|
||||
iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object Spark32HoodieParquetFileFormat {
|
||||
|
||||
private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = {
|
||||
if (fileSchema == null || querySchema == null) {
|
||||
oldFilter
|
||||
} else {
|
||||
oldFilter match {
|
||||
case eq: EqualTo =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute)
|
||||
case eqs: EqualNullSafe =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute)
|
||||
case gt: GreaterThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute)
|
||||
case gtr: GreaterThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute)
|
||||
case lt: LessThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute)
|
||||
case lte: LessThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute)
|
||||
case i: In =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute)
|
||||
case isn: IsNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute)
|
||||
case isnn: IsNotNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute)
|
||||
case And(left, right) =>
|
||||
And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Or(left, right) =>
|
||||
Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Not(child) =>
|
||||
Not(rebuildFilterFromParquet(child, fileSchema, querySchema))
|
||||
case ssw: StringStartsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute)
|
||||
case ses: StringEndsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute)
|
||||
case sc: StringContains =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute)
|
||||
case AlwaysTrue =>
|
||||
AlwaysTrue
|
||||
case AlwaysFalse =>
|
||||
AlwaysFalse
|
||||
case _ =>
|
||||
AlwaysTrue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.spark.sql.catalyst.analysis.ResolvedTable
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, DropColumns, LogicalPlan, RenameColumn, ReplaceColumns, SetTableProperties, UnsetTableProperties}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.hudi.catalog.HoodieInternalV2Table
|
||||
import org.apache.spark.sql.hudi.command.{AlterTableCommand => HudiAlterTableCommand}
|
||||
|
||||
/**
|
||||
* Rule to mostly resolve, normalize and rewrite column names based on case sensitivity.
|
||||
* for alter table column commands.
|
||||
*/
|
||||
class ResolveHudiAlterTableCommandSpark32(sparkSession: SparkSession) extends Rule[LogicalPlan] {
|
||||
|
||||
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
|
||||
case set @ SetTableProperties(asTable(table), _) if schemaEvolutionEnabled && set.resolved =>
|
||||
HudiAlterTableCommand(table, set.changes, ColumnChangeID.PROPERTY_CHANGE)
|
||||
case unSet @ UnsetTableProperties(asTable(table), _, _) if schemaEvolutionEnabled && unSet.resolved =>
|
||||
HudiAlterTableCommand(table, unSet.changes, ColumnChangeID.PROPERTY_CHANGE)
|
||||
case drop @ DropColumns(asTable(table), _) if schemaEvolutionEnabled && drop.resolved =>
|
||||
HudiAlterTableCommand(table, drop.changes, ColumnChangeID.DELETE)
|
||||
case add @ AddColumns(asTable(table), _) if schemaEvolutionEnabled && add.resolved =>
|
||||
HudiAlterTableCommand(table, add.changes, ColumnChangeID.ADD)
|
||||
case renameColumn @ RenameColumn(asTable(table), _, _) if schemaEvolutionEnabled && renameColumn.resolved=>
|
||||
HudiAlterTableCommand(table, renameColumn.changes, ColumnChangeID.UPDATE)
|
||||
case alter @ AlterColumn(asTable(table), _, _, _, _, _) if schemaEvolutionEnabled && alter.resolved =>
|
||||
HudiAlterTableCommand(table, alter.changes, ColumnChangeID.UPDATE)
|
||||
case replace @ ReplaceColumns(asTable(table), _) if schemaEvolutionEnabled && replace.resolved =>
|
||||
HudiAlterTableCommand(table, replace.changes, ColumnChangeID.REPLACE)
|
||||
}
|
||||
|
||||
private def schemaEvolutionEnabled(): Boolean = sparkSession
|
||||
.sessionState.conf.getConfString(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean
|
||||
|
||||
object asTable {
|
||||
def unapply(a: LogicalPlan): Option[CatalogTable] = {
|
||||
a match {
|
||||
case ResolvedTable(_, _, table: HoodieInternalV2Table, _) =>
|
||||
table.catalogTable
|
||||
case _ =>
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,347 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi.command
|
||||
|
||||
import java.net.URI
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
|
||||
import org.apache.hudi.{DataSourceOptionsHelper, DataSourceUtils}
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, Option}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.hudi.internal.schema.action.TableChanges
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.utils.{SchemaChangeUtils, SerDeHelper}
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
|
||||
import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange}
|
||||
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RemoveProperty, SetProperty}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
case class AlterTableCommand(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends HoodieLeafRunnableCommand with Logging {
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
changeType match {
|
||||
case ColumnChangeID.ADD => applyAddAction(sparkSession)
|
||||
case ColumnChangeID.DELETE => applyDeleteAction(sparkSession)
|
||||
case ColumnChangeID.UPDATE => applyUpdateAction(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[SetProperty]).size == changes.size) =>
|
||||
applyPropertySet(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[RemoveProperty]).size == changes.size) =>
|
||||
applyPropertyUnset(sparkSession)
|
||||
case ColumnChangeID.REPLACE => applyReplaceAction(sparkSession)
|
||||
case other => throw new RuntimeException(s"find unsupported alter command type: ${other}")
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
def applyReplaceAction(sparkSession: SparkSession): Unit = {
|
||||
// convert to delete first then add again
|
||||
val deleteChanges = changes.filter(p => p.isInstanceOf[DeleteColumn]).map(_.asInstanceOf[DeleteColumn])
|
||||
val addChanges = changes.filter(p => p.isInstanceOf[AddColumn]).map(_.asInstanceOf[AddColumn])
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val newSchema = applyAddAction2Schema(sparkSession, applyDeleteAction2Schema(sparkSession, oldSchema, deleteChanges), addChanges)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column replace finished")
|
||||
}
|
||||
|
||||
def applyAddAction2Schema(sparkSession: SparkSession, oldSchema: InternalSchema, addChanges: Seq[AddColumn]): InternalSchema = {
|
||||
val addChange = TableChanges.ColumnAddChange.get(oldSchema)
|
||||
addChanges.foreach { addColumn =>
|
||||
val names = addColumn.fieldNames()
|
||||
val parentName = AlterTableCommand.getParentName(names)
|
||||
// add col change
|
||||
val colType = SparkInternalSchemaConverter.buildTypeFromStructType(addColumn.dataType(), true, new AtomicInteger(0))
|
||||
addChange.addColumns(parentName, names.last, colType, addColumn.comment())
|
||||
// add position change
|
||||
addColumn.position() match {
|
||||
case after: TableChange.After =>
|
||||
addChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
addChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange)
|
||||
}
|
||||
|
||||
def applyDeleteAction2Schema(sparkSession: SparkSession, oldSchema: InternalSchema, deleteChanges: Seq[DeleteColumn]): InternalSchema = {
|
||||
val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
|
||||
deleteChanges.foreach { c =>
|
||||
val originalColName = c.fieldNames().mkString(".")
|
||||
checkSchemaChange(Seq(originalColName), table)
|
||||
deleteChange.deleteColumn(originalColName)
|
||||
}
|
||||
SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange).setSchemaId(oldSchema.getMaxColumnId)
|
||||
}
|
||||
|
||||
|
||||
def applyAddAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val newSchema = applyAddAction2Schema(sparkSession, oldSchema, changes.map(_.asInstanceOf[AddColumn]))
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column add finished")
|
||||
}
|
||||
|
||||
def applyDeleteAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val newSchema = applyDeleteAction2Schema(sparkSession, oldSchema, changes.map(_.asInstanceOf[DeleteColumn]))
|
||||
// delete action should not change the getMaxColumnId field.
|
||||
newSchema.setMaxColumnId(oldSchema.getMaxColumnId)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column delete finished")
|
||||
}
|
||||
|
||||
def applyUpdateAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val updateChange = TableChanges.ColumnUpdateChange.get(oldSchema)
|
||||
changes.foreach { change =>
|
||||
change match {
|
||||
case updateType: TableChange.UpdateColumnType =>
|
||||
val newType = SparkInternalSchemaConverter.buildTypeFromStructType(updateType.newDataType(), true, new AtomicInteger(0))
|
||||
updateChange.updateColumnType(updateType.fieldNames().mkString("."), newType)
|
||||
case updateComment: TableChange.UpdateColumnComment =>
|
||||
updateChange.updateColumnComment(updateComment.fieldNames().mkString("."), updateComment.newComment())
|
||||
case updateName: TableChange.RenameColumn =>
|
||||
val originalColName = updateName.fieldNames().mkString(".")
|
||||
checkSchemaChange(Seq(originalColName), table)
|
||||
updateChange.renameColumn(originalColName, updateName.newName())
|
||||
case updateNullAbility: TableChange.UpdateColumnNullability =>
|
||||
updateChange.updateColumnNullability(updateNullAbility.fieldNames().mkString("."), updateNullAbility.nullable())
|
||||
case updatePosition: TableChange.UpdateColumnPosition =>
|
||||
val names = updatePosition.fieldNames()
|
||||
val parentName = AlterTableCommand.getParentName(names)
|
||||
updatePosition.position() match {
|
||||
case after: TableChange.After =>
|
||||
updateChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
updateChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column update finished")
|
||||
}
|
||||
|
||||
// to do support unset default value to columns, and apply them to internalSchema
|
||||
def applyPropertyUnset(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val propKeys = changes.map(_.asInstanceOf[RemoveProperty]).map(_.property())
|
||||
// ignore NonExist unset
|
||||
propKeys.foreach { k =>
|
||||
if (!table.properties.contains(k) && k != TableCatalog.PROP_COMMENT) {
|
||||
logWarning(s"find non exist unset property: ${k} , ignore it")
|
||||
}
|
||||
}
|
||||
val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment
|
||||
val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) }
|
||||
val newTable = table.copy(properties = newProperties, comment = tableComment)
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
// to do support set default value to columns, and apply them to internalSchema
|
||||
def applyPropertySet(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val properties = changes.map(_.asInstanceOf[SetProperty]).map(f => f.property -> f.value).toMap
|
||||
// This overrides old properties and update the comment parameter of CatalogTable
|
||||
// with the newly added/modified comment since CatalogTable also holds comment as its
|
||||
// direct property.
|
||||
val newTable = table.copy(
|
||||
properties = table.properties ++ properties,
|
||||
comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment))
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
def getInternalSchemaAndHistorySchemaStr(sparkSession: SparkSession): (InternalSchema, String) = {
|
||||
val path = AlterTableCommand.getTableLocation(table, sparkSession)
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
|
||||
.setConf(hadoopConf).build()
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
|
||||
val schema = schemaUtil.getTableInternalSchemaFromCommitMetadata().orElse {
|
||||
AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema)
|
||||
}
|
||||
|
||||
val historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata.orElse("")
|
||||
(schema, historySchemaStr)
|
||||
}
|
||||
|
||||
def checkSchemaChange(colNames: Seq[String], catalogTable: CatalogTable): Unit = {
|
||||
val primaryKeys = catalogTable.storage.properties.getOrElse("primaryKey", catalogTable.properties.getOrElse("primaryKey", "keyid")).split(",").map(_.trim)
|
||||
val preCombineKey = Seq(catalogTable.storage.properties.getOrElse("preCombineField", catalogTable.properties.getOrElse("preCombineField", "ts"))).map(_.trim)
|
||||
val partitionKey = catalogTable.partitionColumnNames.map(_.trim)
|
||||
val checkNames = primaryKeys ++ preCombineKey ++ partitionKey
|
||||
colNames.foreach { col =>
|
||||
if (checkNames.contains(col)) {
|
||||
throw new UnsupportedOperationException("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object AlterTableCommand extends Logging {
|
||||
|
||||
/**
|
||||
* Generate an commit with new schema to change the table's schema.
|
||||
*
|
||||
* @param internalSchema new schema after change
|
||||
* @param historySchemaStr history schemas
|
||||
* @param table The hoodie table.
|
||||
* @param sparkSession The spark session.
|
||||
*/
|
||||
def commitWithSchema(internalSchema: InternalSchema, historySchemaStr: String, table: CatalogTable, sparkSession: SparkSession): Unit = {
|
||||
val schema = AvroInternalSchemaConverter.convert(internalSchema, table.identifier.table)
|
||||
val path = getTableLocation(table, sparkSession)
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
val client = DataSourceUtils.createHoodieClient(jsc, schema.toString,
|
||||
path, table.identifier.table, parametersWithWriteDefaults(table.storage.properties).asJava)
|
||||
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build()
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext)
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
val extraMeta = new util.HashMap[String, String]()
|
||||
extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema.setSchemaId(instantTime.toLong)))
|
||||
val schemaManager = new FileBasedInternalSchemaStorageManager(metaClient)
|
||||
schemaManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(internalSchema, historySchemaStr))
|
||||
client.commit(instantTime, jsc.emptyRDD, Option.of(extraMeta))
|
||||
val existRoTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_ro")
|
||||
val existRtTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_rt")
|
||||
try {
|
||||
sparkSession.catalog.refreshTable(table.identifier.unquotedString)
|
||||
// try to refresh ro/rt table
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_ro")
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_rt")
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
log.error(s"Exception when attempting to refresh table ${table.identifier.quotedString}", e)
|
||||
}
|
||||
// try to sync to hive
|
||||
// drop partition field before call alter table
|
||||
val fullSparkSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(internalSchema)
|
||||
val dataSparkSchema = new StructType(fullSparkSchema.fields.filter(p => !table.partitionColumnNames.exists(f => sparkSession.sessionState.conf.resolver(f, p.name))))
|
||||
alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table, dataSparkSchema)
|
||||
if (existRoTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_ro", dataSparkSchema)
|
||||
if (existRtTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_rt", dataSparkSchema)
|
||||
}
|
||||
|
||||
def alterTableDataSchema(sparkSession: SparkSession, db: String, tableName: String, dataSparkSchema: StructType): Unit = {
|
||||
sparkSession.sessionState.catalog
|
||||
.externalCatalog
|
||||
.alterTableDataSchema(db, tableName, dataSparkSchema)
|
||||
}
|
||||
|
||||
def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
|
||||
val uri = if (table.tableType == CatalogTableType.MANAGED) {
|
||||
Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
|
||||
} else {
|
||||
table.storage.locationUri
|
||||
}
|
||||
val conf = sparkSession.sessionState.newHadoopConf()
|
||||
uri.map(makePathQualified(_, conf))
|
||||
.map(removePlaceHolder)
|
||||
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
|
||||
}
|
||||
|
||||
private def removePlaceHolder(path: String): String = {
|
||||
if (path == null || path.length == 0) {
|
||||
path
|
||||
} else if (path.endsWith("-PLACEHOLDER")) {
|
||||
path.substring(0, path.length() - 16)
|
||||
} else {
|
||||
path
|
||||
}
|
||||
}
|
||||
|
||||
def makePathQualified(path: URI, hadoopConf: Configuration): String = {
|
||||
val hadoopPath = new Path(path)
|
||||
val fs = hadoopPath.getFileSystem(hadoopConf)
|
||||
fs.makeQualified(hadoopPath).toUri.toString
|
||||
}
|
||||
|
||||
def getParentName(names: Array[String]): String = {
|
||||
if (names.size > 1) {
|
||||
names.dropRight(1).mkString(".")
|
||||
} else ""
|
||||
}
|
||||
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION.key -> OPERATION.defaultValue,
|
||||
TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
|
||||
PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
|
||||
HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key -> HoodieWriteConfig.DEFAULT_WRITE_PAYLOAD_CLASS,
|
||||
INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
|
||||
ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue,
|
||||
INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue,
|
||||
ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue
|
||||
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -124,6 +124,7 @@
|
||||
<include>org.mortbay.jetty:jetty-util</include>
|
||||
|
||||
<include>org.rocksdb:rocksdbjni</include>
|
||||
<include>com.github.ben-manes.caffeine:caffeine</include>
|
||||
<include>com.beust:jcommander</include>
|
||||
<include>com.twitter:bijection-avro_${scala.binary.version}</include>
|
||||
<include>com.twitter:bijection-core_${scala.binary.version}</include>
|
||||
|
||||
@@ -95,6 +95,7 @@
|
||||
|
||||
<include>com.github.davidmoten:guava-mini</include>
|
||||
<include>com.github.davidmoten:hilbert-curve</include>
|
||||
<include>com.github.ben-manes.caffeine:caffeine</include>
|
||||
<include>com.twitter:bijection-avro_${scala.binary.version}</include>
|
||||
<include>com.twitter:bijection-core_${scala.binary.version}</include>
|
||||
<include>io.dropwizard.metrics:metrics-core</include>
|
||||
|
||||
@@ -120,6 +120,7 @@
|
||||
|
||||
<include>com.github.davidmoten:guava-mini</include>
|
||||
<include>com.github.davidmoten:hilbert-curve</include>
|
||||
<include>com.github.ben-manes.caffeine:caffeine</include>
|
||||
<include>com.twitter:bijection-avro_${scala.binary.version}</include>
|
||||
<include>com.twitter:bijection-core_${scala.binary.version}</include>
|
||||
<include>io.confluent:kafka-avro-serializer</include>
|
||||
|
||||
10
pom.xml
10
pom.xml
@@ -127,9 +127,12 @@
|
||||
<flink.runtime.artifactId>flink-runtime</flink.runtime.artifactId>
|
||||
<flink.table.runtime.artifactId>flink-table-runtime_${scala.binary.version}</flink.table.runtime.artifactId>
|
||||
<flink.table.planner.artifactId>flink-table-planner_${scala.binary.version}</flink.table.planner.artifactId>
|
||||
<spark31.version>3.1.3</spark31.version>
|
||||
<spark32.version>3.2.1</spark32.version>
|
||||
<hudi.spark.module>hudi-spark2</hudi.spark.module>
|
||||
<hudi.spark.common.module>hudi-spark2-common</hudi.spark.common.module>
|
||||
<avro.version>1.8.2</avro.version>
|
||||
<caffeine.version>2.9.1</caffeine.version>
|
||||
<scala11.version>2.11.12</scala11.version>
|
||||
<scala12.version>2.12.10</scala12.version>
|
||||
<scala.version>${scala11.version}</scala.version>
|
||||
@@ -556,6 +559,13 @@
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- caffeine -->
|
||||
<dependency>
|
||||
<groupId>com.github.ben-manes.caffeine</groupId>
|
||||
<artifactId>caffeine</artifactId>
|
||||
<version>${caffeine.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Parquet -->
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
|
||||
Reference in New Issue
Block a user