18
18
19
19
package org .apache .xtable .iceberg ;
20
20
21
+ import static org .apache .xtable .iceberg .IcebergPartitionValueConverter .BUCKET ;
22
+
21
23
import java .util .ArrayList ;
22
24
import java .util .Collections ;
23
25
import java .util .List ;
26
+ import java .util .Map ;
27
+ import java .util .regex .Matcher ;
28
+ import java .util .regex .Pattern ;
24
29
25
30
import lombok .AccessLevel ;
26
31
import lombok .NoArgsConstructor ;
32
37
import org .apache .iceberg .types .Types ;
33
38
34
39
import org .apache .xtable .exception .NotSupportedException ;
40
+ import org .apache .xtable .exception .PartitionSpecException ;
35
41
import org .apache .xtable .model .schema .InternalField ;
36
42
import org .apache .xtable .model .schema .InternalPartitionField ;
37
43
import org .apache .xtable .model .schema .InternalSchema ;
41
47
/** Partition spec builder and extractor for Iceberg. */
42
48
@ NoArgsConstructor (access = AccessLevel .PRIVATE )
43
49
public class IcebergPartitionSpecExtractor {
50
+ private static final Pattern NUM_BUCKETS_MATCHER = Pattern .compile ("bucket\\ [(\\ d+)\\ ]" );
44
51
private static final IcebergPartitionSpecExtractor INSTANCE = new IcebergPartitionSpecExtractor ();
45
52
46
53
public static IcebergPartitionSpecExtractor getInstance () {
@@ -70,6 +77,12 @@ public PartitionSpec toIceberg(List<InternalPartitionField> partitionFields, Sch
70
77
case VALUE :
71
78
partitionSpecBuilder .identity (fieldPath );
72
79
break ;
80
+ case BUCKET :
81
+ partitionSpecBuilder .bucket (
82
+ fieldPath ,
83
+ (int )
84
+ partitioningField .getTransformOptions ().get (InternalPartitionField .NUM_BUCKETS ));
85
+ break ;
73
86
default :
74
87
throw new IllegalArgumentException (
75
88
"Unsupported type: " + partitioningField .getTransformType ());
@@ -99,13 +112,27 @@ PartitionTransformType fromIcebergTransform(Transform<?, ?> transform) {
99
112
throw new NotSupportedException (transformName );
100
113
}
101
114
102
- if (transformName .startsWith ("bucket" )) {
103
- throw new NotSupportedException ( transformName ) ;
115
+ if (transformName .startsWith (BUCKET )) {
116
+ return PartitionTransformType . BUCKET ;
104
117
}
105
118
106
119
throw new NotSupportedException (transform .toString ());
107
120
}
108
121
122
+ private Map <String , Object > getPartitionTransformOptions (Transform <?, ?> transform ) {
123
+ if (transform .toString ().startsWith (BUCKET )) {
124
+ Matcher matcher = NUM_BUCKETS_MATCHER .matcher (transform .toString ());
125
+ if (matcher .matches ()) {
126
+ return Collections .singletonMap (
127
+ InternalPartitionField .NUM_BUCKETS , Integer .parseInt (matcher .group (1 )));
128
+ } else {
129
+ throw new PartitionSpecException (
130
+ "Cannot parse number of buckets from partition transform: " + transform );
131
+ }
132
+ }
133
+ return Collections .emptyMap ();
134
+ }
135
+
109
136
/**
110
137
* Generates internal representation of the Iceberg partition spec.
111
138
*
@@ -121,6 +148,10 @@ public List<InternalPartitionField> fromIceberg(
121
148
122
149
List <InternalPartitionField > irPartitionFields = new ArrayList <>(iceSpec .fields ().size ());
123
150
for (PartitionField iceField : iceSpec .fields ()) {
151
+ // skip void transform
152
+ if (iceField .transform ().isVoid ()) {
153
+ continue ;
154
+ }
124
155
// fetch the ice field from the schema to properly handle hidden partition fields
125
156
int sourceColumnId = iceField .sourceId ();
126
157
Types .NestedField iceSchemaField = iceSchema .findField (sourceColumnId );
@@ -131,6 +162,7 @@ public List<InternalPartitionField> fromIceberg(
131
162
InternalPartitionField .builder ()
132
163
.sourceField (irField )
133
164
.transformType (fromIcebergTransform (iceField .transform ()))
165
+ .transformOptions (getPartitionTransformOptions (iceField .transform ()))
134
166
.build ();
135
167
irPartitionFields .add (irPartitionField );
136
168
}
0 commit comments