danny0405 commented on code in PR #5830:
URL: https://github.com/apache/hudi/pull/5830#discussion_r1033063183
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java:
##########
@@ -120,6 +121,12 @@ private FlinkOptions() {
.withDescription("The default partition name in case the dynamic
partition"
+ " column value is null/empty string");
+ public static final ConfigOption<Boolean> SCHEMA_EVOLUTION_ENABLED =
ConfigOptions
+ .key(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key())
+ .booleanType()
Review Comment:
There is no need to add the option if the key is the same with Hoodie core's.
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataProjection.java:
##########
@@ -87,4 +88,8 @@ public Object[] projectAsValues(RowData rowData) {
}
return values;
}
+
+ protected @Nullable Object rewriteVal(int pos, @Nullable Object val) {
+ return val;
Review Comment:
`rewriteVal` => `getVal `, usually we do not overwrite impl methods, but
only abstract methods, the override is not very friendly for base class
performance.
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/HoodieParquetReader.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.hudi.common.util.Option;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.table.format.cow.ParquetSplitReaderUtil;
+import org.apache.hudi.util.RowDataProjection;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Base interface for hoodie parquet readers.
+ */
+public interface HoodieParquetReader extends Closeable {
+
+ boolean reachedEnd() throws IOException;
+
+ RowData nextRecord();
+
+ static HoodieParquetReader getReader(
+ InternalSchemaManager internalSchemaManager,
+ boolean utcTimestamp,
+ boolean caseSensitive,
+ Configuration conf,
+ String[] fieldNames,
+ DataType[] fieldTypes,
+ Map<String, Object> partitionSpec,
+ int[] selectedFields,
+ int batchSize,
+ Path path,
+ long splitStart,
+ long splitLength) throws IOException {
+ Option<RowDataProjection> castProjection;
+ InternalSchema fileSchema =
internalSchemaManager.getFileSchema(path.getName());
+ if (fileSchema.isEmptySchema()) {
+ castProjection = Option.empty();
Review Comment:
Can return `HoodieParquetReader` directly here when we know `castProjection`
is empty.
##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestCastMap.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.flink.table.data.DecimalData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests for {@link CastMap}.
+ */
+public class TestCastMap {
+
+ @Test
+ public void testCastInt() {
+ CastMap castMap = new CastMap();
+ castMap.add(0, new IntType(), new BigIntType());
+ castMap.add(1, new IntType(), new FloatType());
+ castMap.add(2, new IntType(), new DoubleType());
+ castMap.add(3, new IntType(), new DecimalType());
+ castMap.add(4, new IntType(), new VarCharType());
+ int val = 1;
+ assertEquals(1L, castMap.castIfNeeded(0, val));
+ assertEquals(1.0F, castMap.castIfNeeded(1, val));
+ assertEquals(1.0, castMap.castIfNeeded(2, val));
+ assertEquals(DecimalData.fromBigDecimal(BigDecimal.ONE, 1, 0),
castMap.castIfNeeded(3, val));
+ assertEquals(BinaryStringData.fromString("1"), castMap.castIfNeeded(4,
val));
+ }
+
+ @Test
+ public void testCastLong() {
+ CastMap castMap = new CastMap();
+ castMap.add(0, new BigIntType(), new FloatType());
+ castMap.add(1, new BigIntType(), new DoubleType());
+ castMap.add(2, new BigIntType(), new DecimalType());
+ castMap.add(3, new BigIntType(), new VarCharType());
+ long val = 1L;
+ assertEquals(1.0F, castMap.castIfNeeded(0, val));
+ assertEquals(1.0, castMap.castIfNeeded(1, val));
+ assertEquals(DecimalData.fromBigDecimal(BigDecimal.ONE, 1, 0),
castMap.castIfNeeded(2, val));
+ assertEquals(BinaryStringData.fromString("1"), castMap.castIfNeeded(3,
val));
+ }
+
+ @Test
+ public void testCastFloat() {
+ CastMap castMap = new CastMap();
+ castMap.add(0, new FloatType(), new DoubleType());
+ castMap.add(1, new FloatType(), new DecimalType());
+ castMap.add(2, new FloatType(), new VarCharType());
+ float val = 1F;
+ assertEquals(1.0, castMap.castIfNeeded(0, val));
+ assertEquals(DecimalData.fromBigDecimal(BigDecimal.ONE, 1, 0),
castMap.castIfNeeded(1, val));
+ assertEquals(BinaryStringData.fromString("1.0"), castMap.castIfNeeded(2,
val));
+ }
+
+ @Test
+ public void testCastDouble() {
+ CastMap castMap = new CastMap();
+ castMap.add(0, new DoubleType(), new DecimalType());
+ castMap.add(1, new DoubleType(), new VarCharType());
+ double val = 1;
+ assertEquals(DecimalData.fromBigDecimal(BigDecimal.ONE, 1, 0),
castMap.castIfNeeded(0, val));
+ assertEquals(BinaryStringData.fromString("1.0"), castMap.castIfNeeded(1,
val));
+ }
+
+ @Test
+ public void testCastDecimal() {
+ CastMap castMap = new CastMap();
+ castMap.add(0, new DecimalType(2, 1), new DecimalType(3, 2));
+ castMap.add(1, new DecimalType(), new VarCharType());
+ DecimalData val = DecimalData.fromBigDecimal(BigDecimal.ONE, 2, 1);
+ assertEquals(DecimalData.fromBigDecimal(BigDecimal.ONE, 3, 2),
castMap.castIfNeeded(0, val));
+ assertEquals(BinaryStringData.fromString("1.0"), castMap.castIfNeeded(1,
val));
Review Comment:
For `float`, `double` and `decimal` data types, what is case when the target
data type has precision loss, do we throw exception here ? Exactly what is the
data type precedence(what kind of data type is castable here) for each of the
type ?
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.InternalSchemaCache;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.configuration.HadoopConfigurations;
+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.convert.AvroInternalSchemaConverter;
+import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
+import org.apache.hudi.util.AvroSchemaConverter;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * This class is responsible for calculating names and types of fields that
are actual at a certain point in time.
+ * If field is renamed in queried schema, its old name will be returned, which
is relevant at the provided time.
+ * If type of field is changed, its old type will be returned, and projection
will be created that will convert the old type to the queried one.
+ */
+public class InternalSchemaManager implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final InternalSchemaManager DISABLED = new
InternalSchemaManager(null, null, null, null);
+
+ private final Configuration conf;
+ private final InternalSchema querySchema;
+ private final String validCommits;
+ private final String tablePath;
+ private transient org.apache.hadoop.conf.Configuration hadoopConf;
+
+ public static InternalSchemaManager get(Configuration conf,
HoodieTableMetaClient metaClient) {
+ if (!conf.getBoolean(FlinkOptions.SCHEMA_EVOLUTION_ENABLED)) {
+ return DISABLED;
+ }
+ Option<InternalSchema> internalSchema = new
TableSchemaResolver(metaClient).getTableInternalSchemaFromCommitMetadata();
+ if (!internalSchema.isPresent() || internalSchema.get().isEmptySchema()) {
+ return DISABLED;
+ }
+ String validCommits = metaClient
+ .getCommitsAndCompactionTimeline()
+ .filterCompletedInstants()
+ .getInstants()
+ .map(HoodieInstant::getFileName)
+ .collect(Collectors.joining(","));
+ return new InternalSchemaManager(conf, internalSchema.get(), validCommits,
metaClient.getBasePathV2().toString());
+ }
+
+ public InternalSchemaManager(Configuration conf, InternalSchema querySchema,
String validCommits, String tablePath) {
+ this.conf = conf;
+ this.querySchema = querySchema;
+ this.validCommits = validCommits;
+ this.tablePath = tablePath;
+ }
+
+ public InternalSchema getQuerySchema() {
+ return querySchema != null ? querySchema :
InternalSchema.getEmptyInternalSchema();
+ }
+
+ InternalSchema getFileSchema(String fileName) {
+ InternalSchema querySchema = getQuerySchema();
+ if (querySchema.isEmptySchema()) {
+ return InternalSchema.getEmptyInternalSchema();
+ }
+ long commitInstantTime = Long.parseLong(FSUtils.getCommitTime(fileName));
+ InternalSchema fileSchemaUnmerged =
InternalSchemaCache.getInternalSchemaByVersionId(
+ commitInstantTime, tablePath, getHadoopConf(), validCommits);
+ if (querySchema.equals(fileSchemaUnmerged)) {
+ return InternalSchema.getEmptyInternalSchema();
+ }
+ return new InternalSchemaMerger(fileSchemaUnmerged, querySchema, true,
true).mergeSchema();
+ }
+
+ CastMap getCastMap(InternalSchema fileSchema, String[] queryFieldNames,
DataType[] queryFieldTypes, int[] selectedFields) {
+ assertSchemasAreNotEmpty(getQuerySchema(), fileSchema);
+
+ CastMap castMap = new CastMap();
+ Map<Integer, Integer> posProxy = getPosProxy(fileSchema, queryFieldNames);
+ if (posProxy.isEmpty()) {
+ castMap.setFileFieldTypes(queryFieldTypes);
+ return castMap;
+ }
+ List<Integer> selectedFieldList =
IntStream.of(selectedFields).boxed().collect(Collectors.toList());
+ List<DataType> fileSchemaAsDataTypes =
AvroSchemaConverter.convertToDataType(
+ AvroInternalSchemaConverter.convert(fileSchema,
"tableName")).getChildren();
+ DataType[] fileFieldTypes = new DataType[queryFieldTypes.length];
+ for (int i = 0; i < queryFieldTypes.length; i++) {
+ Integer posOfChangedType = posProxy.get(i);
+ if (posOfChangedType == null) {
+ fileFieldTypes[i] = queryFieldTypes[i];
+ } else {
+ DataType fileType = fileSchemaAsDataTypes.get(posOfChangedType);
+ fileFieldTypes[i] = fileType;
+ int selectedPos = selectedFieldList.indexOf(i);
+ if (selectedPos != -1) {
+ castMap.add(selectedPos, fileType.getLogicalType(),
queryFieldTypes[i].getLogicalType());
+ }
+ }
+ }
+ castMap.setFileFieldTypes(fileFieldTypes);
+ return castMap;
+ }
+
+ String[] getFileFieldNames(InternalSchema fileSchema, String[]
queryFieldNames) {
+ assertSchemasAreNotEmpty(getQuerySchema(), fileSchema);
+
+ Map<String, String> renamedCols =
InternalSchemaUtils.collectRenameCols(fileSchema, getQuerySchema());
+ if (renamedCols.isEmpty()) {
+ return queryFieldNames;
+ }
+ return Arrays.stream(queryFieldNames).map(name ->
renamedCols.getOrDefault(name, name)).toArray(String[]::new);
+ }
+
+ private Map<Integer, Integer> getPosProxy(InternalSchema fileSchema,
String[] queryFieldNames) {
+ Map<Integer, Pair<Type, Type>> changedCols =
InternalSchemaUtils.collectTypeChangedCols(getQuerySchema(), fileSchema);
+ HashMap<Integer, Integer> posProxy = new HashMap<>(changedCols.size());
+ List<String> fieldNameList = Arrays.asList(queryFieldNames);
+ List<Types.Field> columns = getQuerySchema().columns();
+ changedCols.forEach((posInSchema, typePair) -> {
+ String name = columns.get(posInSchema).name();
+ int posInType = fieldNameList.indexOf(name);
+ posProxy.put(posInType, posInSchema);
+ });
+ return Collections.unmodifiableMap(posProxy);
+ }
+
+ private org.apache.hadoop.conf.Configuration getHadoopConf() {
+ if (hadoopConf == null) {
+ hadoopConf = HadoopConfigurations.getHadoopConf(conf);
+ }
+ return hadoopConf;
+ }
+
+ private static void assertSchemasAreNotEmpty(InternalSchema schema1,
InternalSchema schema2) {
+ Preconditions.checkArgument(!schema1.isEmptySchema(), "InternalSchema
cannot be empty here");
Review Comment:
There is no need to bind the schema validation together, and we can give
more details exception msg for different schemas.
##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java:
##########
@@ -102,4 +108,9 @@ public <T extends SpecificRecordBase>
Option<HoodieTableMetadataWriter> getMetad
return Option.empty();
}
}
+
+ private static void setLatestInternalSchema(HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
+ Option<InternalSchema> internalSchema = new
TableSchemaResolver(metaClient).getTableInternalSchemaFromCommitMetadata();
Review Comment:
Add pre-condition check in case of null values.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]