Skip to content
Merged
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 @@ -181,9 +181,11 @@ private static DataType toFlinkPrimitiveType(PrimitiveTypeInfo hiveType) {
*/
public static List<FieldSchema> toHiveFieldSchema(TableSchema schema, boolean withOperationField) {
List<FieldSchema> columns = new ArrayList<>();
Collection<String> metaFields = withOperationField
? HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION // caution that the set may break sequence
: HoodieRecord.HOODIE_META_COLUMNS;
Collection<String> metaFields = new ArrayList<>(HoodieRecord.HOODIE_META_COLUMNS);
if (withOperationField) {
metaFields.add(HoodieRecord.OPERATION_METADATA_FIELD);
}

for (String metaField : metaFields) {
columns.add(new FieldSchema(metaField, "string", null));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -596,7 +596,7 @@ private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table,
serdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(!useRealTimeInputFormat));
serdeProperties.put("serialization.format", "1");

serdeProperties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark(catalogTable, hiveConf, properties, partitionKeys));
serdeProperties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark(catalogTable, hiveConf, properties, partitionKeys, withOperationField));

sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hudi.table.catalog;

import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.configuration.FlinkOptions;
Expand All @@ -28,6 +29,8 @@

import org.apache.avro.Schema;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.types.logical.VarCharType;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
Expand All @@ -39,7 +42,9 @@
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
Expand All @@ -49,6 +54,7 @@
import java.util.stream.Collectors;

import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR;
import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD;
import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME;

/**
Expand Down Expand Up @@ -168,8 +174,10 @@ public static Map<String, String> translateFlinkTableProperties2Spark(
CatalogTable catalogTable,
Configuration hadoopConf,
Map<String, String> properties,
List<String> partitionKeys) {
Schema schema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType());
List<String> partitionKeys,
boolean withOperationField) {
RowType rowType = supplementMetaFields((RowType) catalogTable.getSchema().toPhysicalRowDataType().getLogicalType(), withOperationField);
Schema schema = AvroSchemaConverter.convertToSchema(rowType);
MessageType messageType = TableSchemaResolver.convertAvroSchemaToParquet(schema, hadoopConf);
String sparkVersion = catalogTable.getOptions().getOrDefault(SPARK_VERSION, DEFAULT_SPARK_VERSION);
Map<String, String> sparkTableProperties = SparkDataSourceTableUtils.getSparkTableProperties(
Expand All @@ -184,6 +192,19 @@ public static Map<String, String> translateFlinkTableProperties2Spark(
e -> e.getKey().equalsIgnoreCase(FlinkOptions.TABLE_TYPE.key()) ? VALUE_MAPPING.get(e.getValue()) : e.getValue()));
}

private static RowType supplementMetaFields(RowType rowType, boolean withOperationField) {
Collection<String> metaFields = new ArrayList<>(HoodieRecord.HOODIE_META_COLUMNS);
if (withOperationField) {
metaFields.add(OPERATION_METADATA_FIELD);
}
ArrayList<RowType.RowField> rowFields = new ArrayList<>();
for (String metaField : metaFields) {
rowFields.add(new RowType.RowField(metaField, new VarCharType(10000)));
}
rowFields.addAll(rowType.getFields());
return new RowType(false, rowFields);
}

public static Map<String, String> translateSparkTableProperties2Flink(Map<String, String> options) {
if (options.containsKey(CONNECTOR.key())) {
return options;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,21 @@ public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Except
.collect(Collectors.joining(","));
assertEquals("par1:string", partitionSchema);

// validate spark schema properties
String avroSchemaStr = hiveTable.getParameters().get("spark.sql.sources.schema.part.0");
String expectedAvroSchemaStr = ""
+ "{\"type\":\"struct\",\"fields\":[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"uuid\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},"
+ "{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"age\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},"
+ "{\"name\":\"par1\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}";
assertEquals(expectedAvroSchemaStr, avroSchemaStr);

// validate catalog table
CatalogBaseTable table1 = hoodieCatalog.getTable(tablePath);
assertEquals("hudi", table1.getOptions().get(CONNECTOR.key()));
Expand Down