Skip to content

Commit 3f53ee6

Browse files
committed
restore identitypartitionexpt for truncate, etc
1 parent 19f421e commit 3f53ee6

4 files changed

Lines changed: 90 additions & 43 deletions

File tree

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java

Lines changed: 6 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import java.util.Optional;
3030
import java.util.function.BiConsumer;
3131
import java.util.function.Consumer;
32+
import java.util.function.Function;
3233
import java.util.stream.Collectors;
3334
import java.util.stream.Stream;
3435
import org.apache.commons.collections4.CollectionUtils;
@@ -592,15 +593,15 @@ private Expression buildDeletePredicate(List<String> partNames) throws MetaExcep
592593
return Expressions.alwaysTrue();
593594
}
594595

595-
Map<String, List<PartitionField>> partitionFieldsBySourceColumn =
596-
IcebergTableUtil.partitionFieldsBySourceColumn(icebergTable, true);
596+
Map<String, PartitionField> partitionFields = icebergTable.spec().fields().stream()
597+
.collect(Collectors.toMap(PartitionField::name, Function.identity()));
597598
Expression predicate = Expressions.alwaysFalse();
598599

599600
for (String partName : partNames) {
600601
try {
601-
Map<String, String> partSpec = Warehouse.makeSpecFromName(partName);
602-
Expression partitionExpr = IcebergTableUtil.generateExpressionFromPartitionSpec(
603-
icebergTable, partSpec, partitionFieldsBySourceColumn);
602+
Map<String, String> partitionSpec = Warehouse.makeSpecFromName(partName);
603+
Expression partitionExpr = IcebergTableUtil.generateExprFromIdentityPartitionSpec(
604+
icebergTable, partitionSpec, partitionFields);
604605

605606
predicate = Expressions.or(predicate, partitionExpr);
606607
} catch (Exception e) {
@@ -612,10 +613,6 @@ private Expression buildDeletePredicate(List<String> partNames) throws MetaExcep
612613
return predicate;
613614
}
614615

615-
@Override public boolean createHMSTableInHook() {
616-
return createHMSTableInHook;
617-
}
618-
619616
private void alterTableProperties(org.apache.hadoop.hive.metastore.api.Table hmsTable,
620617
Map<String, String> contextProperties) {
621618
Map<String, String> hmsTableParameters = hmsTable.getParameters();

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2033,7 +2033,7 @@ public boolean canUseTruncate(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
20332033
return false;
20342034
}
20352035

2036-
Expression partitionExpr = IcebergTableUtil.generateExpressionFromPartitionSpec(
2036+
Expression partitionExpr = IcebergTableUtil.generateExprFromIdentityPartitionSpec(
20372037
table, partitionSpec, true);
20382038

20392039
FindFiles.Builder builder = new FindFiles.Builder(table).withRecordsMatching(partitionExpr);

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -172,7 +172,7 @@ public static void appendFiles(URI fromURI, String format, Table icebergTbl, boo
172172
if (isOverwrite) {
173173
DeleteFiles delete = transaction.newDelete();
174174
if (partitionSpec != null) {
175-
Expression partitionExpr = IcebergTableUtil.generateExpressionFromPartitionSpec(
175+
Expression partitionExpr = IcebergTableUtil.generateExprFromIdentityPartitionSpec(
176176
icebergTbl, partitionSpec, true);
177177
delete.deleteFromRowFilter(partitionExpr);
178178
} else {

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java

Lines changed: 82 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -105,6 +105,7 @@
105105
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
106106
import org.apache.iceberg.transforms.Transform;
107107
import org.apache.iceberg.types.Conversions;
108+
import org.apache.iceberg.types.Type;
108109
import org.apache.iceberg.types.Types;
109110
import org.apache.iceberg.util.ByteBuffers;
110111
import org.apache.iceberg.util.Pair;
@@ -440,64 +441,113 @@ public static PartitionData toPartitionData(StructLike key, Types.StructType key
440441

441442
public static PartitionData toPartitionData(StructLike sourceKey, Types.StructType sourceKeyType,
442443
Types.StructType targetKeyType) {
443-
StructProjection projection = StructProjection.create(sourceKeyType, targetKeyType).wrap(sourceKey);
444+
StructProjection projection = StructProjection.create(sourceKeyType, targetKeyType)
445+
.wrap(sourceKey);
444446
return toPartitionData(projection, targetKeyType);
445447
}
446448

447-
public static Expression generateExpressionFromPartitionSpec(Table table, Map<String, String> partitionSpec,
449+
public static Expression generateExprFromIdentityPartitionSpec(Table table, Map<String, String> partitionSpec,
448450
boolean latestSpecOnly) throws SemanticException {
449451

450-
return generateExpressionFromPartitionSpec(
451-
table, partitionSpec, partitionFieldsBySourceColumn(table, latestSpecOnly));
452+
Map<String, PartitionField> partitionFields = getPartitionFields(table, latestSpecOnly).stream()
453+
.collect(Collectors.toMap(PartitionField::name, Function.identity()));
454+
455+
return generateExprFromIdentityPartitionSpec(table, partitionSpec, partitionFields);
456+
}
457+
458+
public static Expression generateExprFromIdentityPartitionSpec(Table table, Map<String, String> partitionSpec,
459+
Map<String, PartitionField> partitionFields) throws SemanticException {
460+
461+
return buildPartitionExpression(
462+
partitionSpec,
463+
(column, value) ->
464+
buildIdentityPartitionPredicate(table, column, value, partitionFields.get(column)),
465+
partitionFields::containsKey
466+
);
452467
}
453468

454-
public static Map<String, List<PartitionField>> partitionFieldsBySourceColumn(Table table, boolean latestSpecOnly) {
469+
public static Expression generateExprFromPartitionSpec(Table table, Map<String, String> partitionSpec,
470+
boolean latestSpecOnly) throws SemanticException {
471+
455472
// Group partition fields by source column name to handle partition evolution
456473
// where the same source column may have multiple transforms across different specs
457-
return getPartitionFields(table, latestSpecOnly).stream()
458-
.collect(Collectors.groupingBy(
459-
partitionField -> table.schema().findColumnName(partitionField.sourceId()))
460-
);
474+
Map<String, List<PartitionField>> partitionFieldsBySourceColumn =
475+
getPartitionFields(table, latestSpecOnly).stream()
476+
.collect(Collectors.groupingBy(
477+
pf -> table.schema().findColumnName(pf.sourceId()))
478+
);
479+
480+
return buildPartitionExpression(
481+
partitionSpec,
482+
(column, value) ->
483+
buildTransformPartitionPredicate(table, value, partitionFieldsBySourceColumn.get(column)),
484+
partitionFieldsBySourceColumn::containsKey
485+
);
486+
}
487+
488+
@FunctionalInterface
489+
private interface PartitionPredicateBuilder {
490+
Expression build(String partitionColumn, String partitionValue) throws SemanticException;
461491
}
462492

463-
public static Expression generateExpressionFromPartitionSpec(Table table, Map<String, String> partitionSpec,
464-
Map<String, List<PartitionField>> partitionFieldsBySourceColumn) throws SemanticException {
493+
private static Expression buildPartitionExpression(
494+
Map<String, String> partitionSpec,
495+
PartitionPredicateBuilder predicateBuilder,
496+
Predicate<String> fieldValidator) throws SemanticException {
465497

466498
Expression predicate = Expressions.alwaysTrue();
467499

468500
for (Map.Entry<String, String> entry : partitionSpec.entrySet()) {
469501
String partitionColumn = entry.getKey();
470-
List<PartitionField> partitionFields = partitionFieldsBySourceColumn.get(partitionColumn);
471502

472-
if (partitionFields == null) {
503+
// Validate field exists
504+
if (!fieldValidator.test(partitionColumn)) {
473505
throw new SemanticException(String.format("No partition column by the name: %s", partitionColumn));
474506
}
507+
Expression columnPredicate = predicateBuilder.build(partitionColumn, entry.getValue());
508+
predicate = Expressions.and(predicate, columnPredicate);
509+
}
475510

476-
// When there are multiple partition fields for the same source column (due to partition evolution),
477-
// create an OR expression that matches any of the transforms
478-
Types.NestedField sourceField = table.schema().findField(
479-
partitionFields.getFirst().sourceId());
480-
Object sourceValue = Conversions.fromPartitionString(sourceField.type(), entry.getValue());
511+
return predicate;
512+
}
481513

482-
Expression columnPredicate = Expressions.alwaysFalse();
514+
private static Expression buildIdentityPartitionPredicate(Table table, String partitionColumn,
515+
String partitionValue, PartitionField partitionField) throws SemanticException {
483516

484-
for (PartitionField partitionField : partitionFields) {
485-
// Apply the transform to the source value
486-
@SuppressWarnings("unchecked")
487-
Transform<Object, Object> transform = (Transform<Object, Object>) partitionField.transform();
488-
Object transformedValue = transform.bind(sourceField.type()).apply(sourceValue);
517+
if (!partitionField.transform().isIdentity()) {
518+
throw new SemanticException(String.format("Partition transforms are not supported here: %s", partitionColumn));
519+
}
520+
Type columnType = table.schema().findField(partitionField.sourceId()).type();
521+
Object columnValue = Conversions.fromPartitionString(columnType, partitionValue);
489522

490-
TransformSpec transformSpec = TransformSpec.fromString(transform.toString().toUpperCase(), sourceField.name());
491-
UnboundTerm<Object> term = SchemaUtils.toTerm(transformSpec);
523+
return Expressions.equal(partitionColumn, columnValue);
524+
}
492525

493-
columnPredicate = Expressions.or(
494-
columnPredicate, Expressions.equal(term, transformedValue));
495-
}
526+
private static Expression buildTransformPartitionPredicate(Table table, String partitionValue,
527+
List<PartitionField> partitionFields) {
496528

497-
predicate = Expressions.and(predicate, columnPredicate);
529+
// Get source field type from first partition field (all share same source)
530+
Types.NestedField sourceField = table.schema().findField(
531+
partitionFields.getFirst().sourceId());
532+
Object sourceValue = Conversions.fromPartitionString(sourceField.type(), partitionValue);
533+
534+
Expression columnPredicate = Expressions.alwaysFalse();
535+
536+
// Create OR expression for each transform on this source column
537+
for (PartitionField partitionField : partitionFields) {
538+
// Apply the transform to the source value
539+
@SuppressWarnings("unchecked")
540+
Transform<Object, Object> transform = (Transform<Object, Object>) partitionField.transform();
541+
Object transformedValue = transform.bind(sourceField.type()).apply(sourceValue);
542+
543+
TransformSpec transformSpec = TransformSpec.fromString(transform.toString().toUpperCase(), sourceField.name());
544+
UnboundTerm<Object> term = SchemaUtils.toTerm(transformSpec);
545+
546+
columnPredicate = Expressions.or(
547+
columnPredicate, Expressions.equal(term, transformedValue));
498548
}
499549

500-
return predicate;
550+
return columnPredicate;
501551
}
502552

503553
public static List<PartitionField> getPartitionFields(Table table, boolean latestSpecOnly) {
@@ -575,7 +625,7 @@ private static List<String> getPartitionNames(Configuration conf,
575625
Comparator<Map.Entry<String, Integer>> specIdComparator) throws SemanticException {
576626
Table icebergTable = getTable(conf, table.getTTable());
577627

578-
Expression partitionExpr = IcebergTableUtil.generateExpressionFromPartitionSpec(
628+
Expression partitionExpr = IcebergTableUtil.generateExprFromPartitionSpec(
579629
icebergTable, partitionSpec, latestSpecOnly);
580630

581631
int latestSpecId = icebergTable.spec().specId();

0 commit comments

Comments
 (0)