Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ public static InternalSchema searchSchemaAndCache(long versionID, HoodieTableMet
}
}

private static TreeMap<Long, InternalSchema> getHistoricalSchemas(HoodieTableMetaClient metaClient) {
public static TreeMap<Long, InternalSchema> getHistoricalSchemas(HoodieTableMetaClient metaClient) {
TreeMap<Long, InternalSchema> result = new TreeMap<>();
FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient);
String historySchemaStr = schemasManager.getHistorySchemaStr();
Expand Down
58 changes: 58 additions & 0 deletions hudi-flink-datasource/hudi-flink/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -265,6 +265,64 @@

<!-- Test dependencies -->

<!-- Spark test -->
<!-- Spark testkit is used to prepare test data for schema evolution (with changed types, renamed columns, and so on). -->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<exclusions>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
<classifier>tests</classifier>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<exclusions>
<exclusion>
<groupId>org.eclipse.jetty.orbit</groupId>
<artifactId>javax.servlet</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>javax.servlet-api</artifactId>
</exclusion>
<exclusion>
<groupId>com.twitter</groupId>
<artifactId>chill-java</artifactId>
</exclusion>
</exclusions>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why introduces the spark dependency in flink pom ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To prepare test data. Currently, only Spark engine provides way to change schema and write new data after that.
I think when full support of schema evolution is implemented, we can remove this dependency by rewriting test to pure Flink

<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.binary.version}</artifactId>
<version>${scalatest.version}</version>
<scope>test</scope>
</dependency>

<!-- Junit 5 dependencies -->
<dependency>
<groupId>org.junit.jupiter</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,12 @@ private FlinkOptions() {
+ "The semantics is best effort because the compaction job would finally merge all changes of a record into one.\n"
+ " default false to have UPSERT semantics");

public static final ConfigOption<Boolean> SCHEMA_EVOLUTION_ENABLED = ConfigOptions
.key(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key())
.booleanType()
.defaultValue(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.defaultValue())
.withDescription(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.doc());

// ------------------------------------------------------------------------
// Metadata table Options
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,20 @@
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.InternalSchemaCache;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.OptionsResolver;
import org.apache.hudi.exception.HoodieException;
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.source.FileIndex;
import org.apache.hudi.source.IncrementalInputSplits;
import org.apache.hudi.source.StreamReadMonitoringFunction;
import org.apache.hudi.source.StreamReadOperator;
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.hudi.table.format.SchemaEvoContext;
import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat;
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
Expand Down Expand Up @@ -88,6 +92,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
Expand Down Expand Up @@ -436,7 +441,8 @@ private MergeOnReadInputFormat mergeOnReadInputFormat(
this.conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME),
this.limit == NO_LIMIT_CONSTANT ? Long.MAX_VALUE : this.limit, // ParquetInputFormat always uses the limit value
getParquetConf(this.conf, this.hadoopConf),
this.conf.getBoolean(FlinkOptions.UTC_TIMEZONE)
this.conf.getBoolean(FlinkOptions.UTC_TIMEZONE),
getSchemaEvoContext()
);
format.setFilesFilter(new LatestFileFilter(this.hadoopConf));
return format;
Expand All @@ -447,6 +453,17 @@ private Schema inferSchemaFromDdl() {
return HoodieAvroUtils.addMetadataFields(schema, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
}

private Option<SchemaEvoContext> getSchemaEvoContext() {
if (!conf.getBoolean(FlinkOptions.SCHEMA_EVOLUTION_ENABLED) || metaClient == null) {
return Option.empty();
}
TreeMap<Long, InternalSchema> schemas = InternalSchemaCache.getHistoricalSchemas(metaClient);
InternalSchema querySchema = schemas.isEmpty()
? AvroInternalSchemaConverter.convert(getTableAvroSchema())
: schemas.lastEntry().getValue();
return Option.of(new SchemaEvoContext(querySchema, metaClient));
}

@VisibleForTesting
public Schema getTableAvroSchema() {
try {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,245 @@
/*
* 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.table.format;

import org.apache.avro.Schema;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.binary.BinaryStringData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.DecimalType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.util.Preconditions;
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.convert.AvroInternalSchemaConverter;
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
import org.apache.hudi.util.AvroSchemaConverter;

import java.io.Serializable;
import java.math.BigDecimal;
import java.time.LocalDate;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import static org.apache.flink.table.types.logical.LogicalTypeRoot.BIGINT;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.DATE;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.DOUBLE;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.FLOAT;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTEGER;
import static org.apache.flink.table.types.logical.LogicalTypeRoot.VARCHAR;

/**
* CastMap is responsible for type conversion when full schema evolution enabled.
*/
public final class CastMap implements Serializable {
// Maps position (column number) to corresponding cast
private final Map<Integer, Cast> castMap = new HashMap<>();

/**
* Creates CastMap by comparing two schemes. Cast of a specific column is created if its type has changed.
*/
public static CastMap of(String tableName, InternalSchema querySchema, InternalSchema actualSchema) {
DataType queryType = internalSchemaToDataType(tableName, querySchema);
DataType actualType = internalSchemaToDataType(tableName, actualSchema);
CastMap castMap = new CastMap();
InternalSchemaUtils.collectTypeChangedCols(querySchema, actualSchema).entrySet()
.stream()
.filter(e -> !isSameType(e.getValue().getLeft(), e.getValue().getRight()))
.forEach(e -> {
int pos = e.getKey();
LogicalType target = queryType.getChildren().get(pos).getLogicalType();
LogicalType actual = actualType.getChildren().get(pos).getLogicalType();
castMap.add(pos, actual, target);
});
return castMap;
}

public Object castIfNeed(int pos, Object val) {
Cast cast = castMap.get(pos);
if (cast == null) {
return val;
}
return cast(val, cast.from(), cast.to());
}

private Object cast(Object val, LogicalType fromType, LogicalType toType) {
LogicalTypeRoot from = fromType.getTypeRoot();
LogicalTypeRoot to = toType.getTypeRoot();
switch (to) {
case BIGINT: {
// Integer => Long
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the philosophy of these mappings ?

Copy link
Copy Markdown
Contributor Author

@trushev trushev May 17, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Assume schema evolution DDL

alter table t1 alter column val type bigint

which changes type of val from int to bigint

We want to be able to read old data. To do it we need to cast val from int to long

otherwise, an exception will be thrown

java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Long
	at org.apache.flink.table.data.GenericRowData.getLong(GenericRowData.java:154)

This class is an analogue of org.apache.hudi.client.utils.SparkInternalSchemaConverter#convertColumnVectorType which converts Spark's types

if (from == INTEGER) {
return ((Number) val).longValue();
}
break;
}
case FLOAT: {
// Integer => Float
// Long => Float
if (from == INTEGER || from == BIGINT) {
return ((Number) val).floatValue();
}
break;
}
case DOUBLE: {
// Integer => Double
// Long => Double
if (from == INTEGER || from == BIGINT) {
return ((Number) val).doubleValue();
}
// Float => Double
if (from == FLOAT) {
return Double.parseDouble(val.toString());
}
break;
}
case DECIMAL: {
// Integer => Decimal
// Long => Decimal
// Double => Decimal
if (from == INTEGER || from == BIGINT || from == DOUBLE) {
return toDecimalData((Number) val, toType);
}
// Float => Decimal
if (from == FLOAT) {
return toDecimalData(Double.parseDouble(val.toString()), toType);
}
// String => Decimal
if (from == VARCHAR) {
return toDecimalData(Double.parseDouble(val.toString()), toType);
}
// Decimal => Decimal
if (from == DECIMAL) {
return toDecimalData(((DecimalData) val).toBigDecimal(), toType);
}
break;
}
case VARCHAR: {
// Integer => String
// Long => String
// Float => String
// Double => String
// Decimal => String
if (from == INTEGER
|| from == BIGINT
|| from == FLOAT
|| from == DOUBLE
|| from == DECIMAL) {
return new BinaryStringData(String.valueOf(val));
}
// Date => String
if (from == DATE) {
return new BinaryStringData(LocalDate.ofEpochDay(((Integer) val).longValue()).toString());
}
break;
}
case DATE: {
// String => Date
if (from == VARCHAR) {
return (int) LocalDate.parse(val.toString()).toEpochDay();
}
break;
}
default:
}
return val;
}

public boolean containsAnyPos(Collection<Integer> positions) {
return positions.stream().anyMatch(castMap.keySet()::contains);
}

public CastMap rearrange(List<Integer> oldIndexes, List<Integer> newIndexes) {
Preconditions.checkArgument(oldIndexes.size() == newIndexes.size());
CastMap newCastMap = new CastMap();
for (int i = 0; i < oldIndexes.size(); i++) {
Cast cast = castMap.get(oldIndexes.get(i));
if (cast != null) {
newCastMap.add(newIndexes.get(i), cast.from(), cast.to());
}
}
return newCastMap;
}

@VisibleForTesting
void add(int pos, LogicalType from, LogicalType to) {
castMap.put(pos, new Cast(from, to));
}

private DecimalData toDecimalData(Number val, LogicalType decimalType) {
BigDecimal valAsDecimal = BigDecimal.valueOf(val.doubleValue());
return toDecimalData(valAsDecimal, decimalType);
}

private DecimalData toDecimalData(BigDecimal valAsDecimal, LogicalType decimalType) {
return DecimalData.fromBigDecimal(
valAsDecimal,
((DecimalType) decimalType).getPrecision(),
((DecimalType) decimalType).getScale());
}

private static boolean isSameType(Type left, Type right) {
if (left instanceof Types.DecimalType && right instanceof Types.DecimalType) {
return left.equals(right);
}
return left.typeId().equals(right.typeId());
}

private static DataType internalSchemaToDataType(String tableName, InternalSchema internalSchema) {
Schema schema = AvroInternalSchemaConverter.convert(internalSchema, tableName);
return AvroSchemaConverter.convertToDataType(schema);
}

private static final class Cast implements Serializable {
private final LogicalType from;
private final LogicalType to;

Cast(LogicalType from, LogicalType to) {
this.from = from;
this.to = to;
}

LogicalType from() {
return from;
}

LogicalType to() {
return to;
}

@Override
public String toString() {
return from + " => " + to;
}
}

@Override
public String toString() {
return castMap.entrySet().stream()
.map(e -> e.getKey() + ": " + e.getValue())
.collect(Collectors.joining(", ", "{", "}"));
}
}
Loading