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

HIVE-28727: Iceberg: Null partitions make major compactions fail #5622

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

okumin
Copy link
Contributor

@okumin okumin commented Jan 25, 2025

What changes were proposed in this pull request?

Let IcebergTableUtil list NULL partitions so that Hive can successfully compact a table that includes such partitions.

Why are the changes needed?

OPTIMIZE TABLE ... REWRITE DATA or ALTER TABLE ... COMPACT 'major' fail on the current master branch when a null partition is included.

Does this PR introduce any user-facing change?

At least, such compaction works on 4.0.1. So, I think we can say this is not a user-facing change. I suspect HIVE-28256 introduced the issue though my confidence is far from 100%

Is the change a dependency upgrade?

No.

How was this patch tested?

I updated an existing qtest.

{"company_id":100,"dept_id":1,"team_id":null} 1 0 3
{"company_id":100,"dept_id":2,"team_id":null} 1 0 3
{"company_id":null,"dept_id":null,"team_id":null} 1 0 1
{"company_id":null,"dept_id":null,"team_id":null} 1 0 2
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I wonder why two {"company_id":null,"dept_id":null,"team_id":null} remain. I think it is minor and out of scope of HIVE-28727

@okumin okumin marked this pull request as ready for review January 26, 2025 02:16
@okumin
Copy link
Contributor Author

okumin commented Jan 26, 2025

@difin Could you please have a look? You are one of the most familiar engineers with Apache Icebergs' compaction.

@okumin okumin changed the title [WIP] HIVE-28727: Iceberg: Null partitions make major compactions fail HIVE-28727: Iceberg: Null partitions make major compactions fail Jan 26, 2025
} else {
throw new RuntimeException(String.format("Partition field's %s value is null", fieldName));
}
String fieldName = targetKeyType.fields().get(i).name();
Copy link
Member

@deniskuzZ deniskuzZ Feb 1, 2025

Choose a reason for hiding this comment

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

Could we simplify this with

StructProjection projection = StructProjection.create(sourceKeyType, targetKeyType)
    .wrap(sourceKey);
PartitionData keyTemplate = new PartitionData(targetKeyType);
return keyTemplate.copyFor(projection);

Copy link
Member

Choose a reason for hiding this comment

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

above PartitionData toPartitionData(StructLike key, Types.StructType keyType) could be replaced with

PartitionData keyTemplate = new PartitionData(keyType);
return keyTemplate.copyFor(key);

Copy link
Member

@deniskuzZ deniskuzZ Feb 1, 2025

Choose a reason for hiding this comment

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

since you are doing refactor here, could we also leverage IcebergTableUtil#toPartitionData in HiveIcebergStorageHandler#getPartishSummary? see

try (Closeable toClose = partitionStatsRecords) {
  PartitionStats partitionStats = Iterables.tryFind(partitionStatsRecords, stats -> {
    PartitionSpec spec = table.specs().get(stats.specId());
    
    PartitionData data  = IcebergTableUtil.toPartitionData(stats.partition(), 
        partitionType, spec.partitionType());
    return spec.partitionToPath(data).equals(partish.getPartition().getName());

  }).orNull();

Copy link
Member

@deniskuzZ deniskuzZ Feb 3, 2025

Choose a reason for hiding this comment

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

hi @okumin, WDYT, does it make sense? Tried locally and test with your updates passed.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants