in xtable-core/src/main/java/org/apache/xtable/iceberg/IcebergPartitionSpecExtractor.java [57:92]
public PartitionSpec toIceberg(List<InternalPartitionField> partitionFields, Schema tableSchema) {
if (partitionFields == null || partitionFields.isEmpty()) {
return PartitionSpec.unpartitioned();
}
PartitionSpec.Builder partitionSpecBuilder = PartitionSpec.builderFor(tableSchema);
for (InternalPartitionField partitioningField : partitionFields) {
String fieldPath = partitioningField.getSourceField().getPath();
switch (partitioningField.getTransformType()) {
case YEAR:
partitionSpecBuilder.year(fieldPath);
break;
case MONTH:
partitionSpecBuilder.month(fieldPath);
break;
case DAY:
partitionSpecBuilder.day(fieldPath);
break;
case HOUR:
partitionSpecBuilder.hour(fieldPath);
break;
case VALUE:
partitionSpecBuilder.identity(fieldPath);
break;
case BUCKET:
partitionSpecBuilder.bucket(
fieldPath,
(int)
partitioningField.getTransformOptions().get(InternalPartitionField.NUM_BUCKETS));
break;
default:
throw new IllegalArgumentException(
"Unsupported type: " + partitioningField.getTransformType());
}
}
return partitionSpecBuilder.build();
}