Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-5088]Fix bug:Failed to synchronize the hive metadata of the Flink table #7056

Merged
merged 3 commits into from
Nov 7, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -24,8 +24,10 @@
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.configuration.FlinkOptions;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.catalog.CatalogBaseTable;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
Expand All @@ -42,6 +44,7 @@
import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;

import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -177,10 +180,19 @@ private static DataType toFlinkPrimitiveType(PrimitiveTypeInfo hiveType) {

/**
* Create Hive field schemas from Flink table schema including the hoodie metadata fields.
*
* @param table
*/
public static List<FieldSchema> toHiveFieldSchema(TableSchema schema) {
public static List<FieldSchema> toHiveFieldSchema(CatalogBaseTable table) {
TableSchema schema = table.getSchema();
Configuration configuration = Configuration.fromMap(table.getOptions());
Boolean changelogEnable = configuration.getBoolean(FlinkOptions.CHANGELOG_ENABLED);
Collection<String> hoodieMetaColumns = HoodieRecord.HOODIE_META_COLUMNS;
if (changelogEnable) {
hoodieMetaColumns = HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION;
Copy link
Contributor

Choose a reason for hiding this comment

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

In current master, we do not add _hoodie_operation field for hive table now, how the hive table was created locally ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

specifying changlog.enabled=true and hive_sync.skip_ro_suffix = true, and Hive sync automatically synchronizes metadata

Copy link
Contributor

Choose a reason for hiding this comment

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

The _hoodie_operation field is useless for hive table, we should not sync it.

Copy link
Contributor

Choose a reason for hiding this comment

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

What version of hudi did you use for the streaming writer ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

What version of hudi did you use for the streaming writer ?

0.12.1

Copy link
Contributor

Choose a reason for hiding this comment

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

Can you past the error stack trace here ? The writer does not expect to sync the _hoodie_operation meta field now.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

image

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This should be because the operation field exists in the parquet file and is not filtered out.

}
List<FieldSchema> columns = new ArrayList<>();
for (String metaField : HoodieRecord.HOODIE_META_COLUMNS) {
for (String metaField : hoodieMetaColumns) {
columns.add(new FieldSchema(metaField, "string", null));
}
columns.addAll(createHiveColumns(schema));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -546,7 +546,7 @@ private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table,
// because since Hive 3.x, there is validation when altering table,
// when the metadata fields are synced through the hive sync tool,
// a compatability issue would be reported.
List<FieldSchema> allColumns = HiveSchemaUtils.toHiveFieldSchema(table.getSchema());
List<FieldSchema> allColumns = HiveSchemaUtils.toHiveFieldSchema(table);

// Table columns and partition keys
CatalogTable catalogTable = (CatalogTable) table;
Expand Down Expand Up @@ -799,7 +799,7 @@ public void dropPartition(
try (HoodieFlinkWriteClient<?> writeClient = createWriteClient(tablePath, table)) {
boolean hiveStylePartitioning = Boolean.parseBoolean(table.getOptions().get(FlinkOptions.HIVE_STYLE_PARTITIONING.key()));
writeClient.deletePartitions(
Collections.singletonList(HoodieCatalogUtil.inferPartitionPath(hiveStylePartitioning, partitionSpec)),
Collections.singletonList(HoodieCatalogUtil.inferPartitionPath(hiveStylePartitioning, partitionSpec)),
HoodieActiveTimeline.createNewInstantTime())
Copy link
Contributor

Choose a reason for hiding this comment

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

Unnecessary change.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is the format of the previous code. I corrected it.

.forEach(writeStatus -> {
if (writeStatus.hasErrors()) {
Expand Down