25
25
import static org .apache .iceberg .TableProperties .DELETE_ORC_WRITE_BATCH_SIZE ;
26
26
import static org .apache .iceberg .TableProperties .ORC_BLOCK_SIZE_BYTES ;
27
27
import static org .apache .iceberg .TableProperties .ORC_BLOCK_SIZE_BYTES_DEFAULT ;
28
+ import static org .apache .iceberg .TableProperties .ORC_BLOOM_FILTER_COLUMNS ;
29
+ import static org .apache .iceberg .TableProperties .ORC_BLOOM_FILTER_FPP ;
28
30
import static org .apache .iceberg .TableProperties .ORC_COMPRESSION ;
29
31
import static org .apache .iceberg .TableProperties .ORC_COMPRESSION_DEFAULT ;
30
32
import static org .apache .iceberg .TableProperties .ORC_COMPRESSION_STRATEGY ;
@@ -202,6 +204,8 @@ public <D> FileAppender<D> build() {
202
204
OrcConf .COMPRESS .setString (conf , context .compressionKind ().name ());
203
205
OrcConf .COMPRESSION_STRATEGY .setString (conf , context .compressionStrategy ().name ());
204
206
OrcConf .OVERWRITE_OUTPUT_FILE .setBoolean (conf , overwrite );
207
+ OrcConf .BLOOM_FILTER_COLUMNS .setString (conf , context .bloomFilterColumns ());
208
+ OrcConf .BLOOM_FILTER_FPP .setDouble (conf , context .bloomFilterFpp ());
205
209
206
210
return new OrcFileAppender <>(
207
211
schema ,
@@ -220,6 +224,10 @@ private static class Context {
220
224
private final CompressionKind compressionKind ;
221
225
private final CompressionStrategy compressionStrategy ;
222
226
227
+ private final String bloomFilterColumns ;
228
+
229
+ private final double bloomFilterFpp ;
230
+
223
231
public long stripeSize () {
224
232
return stripeSize ;
225
233
}
@@ -240,17 +248,29 @@ public CompressionStrategy compressionStrategy() {
240
248
return compressionStrategy ;
241
249
}
242
250
251
+ public String bloomFilterColumns () {
252
+ return bloomFilterColumns ;
253
+ }
254
+
255
+ public double bloomFilterFpp () {
256
+ return bloomFilterFpp ;
257
+ }
258
+
243
259
private Context (
244
260
long stripeSize ,
245
261
long blockSize ,
246
262
int vectorizedRowBatchSize ,
247
263
CompressionKind compressionKind ,
248
- CompressionStrategy compressionStrategy ) {
264
+ CompressionStrategy compressionStrategy ,
265
+ String bloomFilterColumns ,
266
+ double bloomFilterFpp ) {
249
267
this .stripeSize = stripeSize ;
250
268
this .blockSize = blockSize ;
251
269
this .vectorizedRowBatchSize = vectorizedRowBatchSize ;
252
270
this .compressionKind = compressionKind ;
253
271
this .compressionStrategy = compressionStrategy ;
272
+ this .bloomFilterColumns = bloomFilterColumns ;
273
+ this .bloomFilterFpp = bloomFilterFpp ;
254
274
}
255
275
256
276
static Context dataContext (Map <String , String > config ) {
@@ -285,9 +305,24 @@ static Context dataContext(Map<String, String> config) {
285
305
strategyAsString =
286
306
PropertyUtil .propertyAsString (config , ORC_COMPRESSION_STRATEGY , strategyAsString );
287
307
CompressionStrategy compressionStrategy = toCompressionStrategy (strategyAsString );
308
+ String bloomFilterColumns =
309
+ PropertyUtil .propertyAsString (config , OrcConf .BLOOM_FILTER_COLUMNS .getAttribute (), "" );
310
+ bloomFilterColumns =
311
+ PropertyUtil .propertyAsString (config , ORC_BLOOM_FILTER_COLUMNS , bloomFilterColumns );
312
+
313
+ double bloomFilterFpp =
314
+ PropertyUtil .propertyAsDouble (config , OrcConf .BLOOM_FILTER_FPP .getAttribute (), 0.05 );
315
+ bloomFilterFpp =
316
+ PropertyUtil .propertyAsDouble (config , ORC_BLOOM_FILTER_FPP , bloomFilterFpp );
288
317
289
318
return new Context (
290
- stripeSize , blockSize , vectorizedRowBatchSize , compressionKind , compressionStrategy );
319
+ stripeSize ,
320
+ blockSize ,
321
+ vectorizedRowBatchSize ,
322
+ compressionKind ,
323
+ compressionStrategy ,
324
+ bloomFilterColumns ,
325
+ bloomFilterFpp );
291
326
}
292
327
293
328
static Context deleteContext (Map <String , String > config ) {
@@ -316,9 +351,21 @@ static Context deleteContext(Map<String, String> config) {
316
351
strategyAsString != null
317
352
? toCompressionStrategy (strategyAsString )
318
353
: dataContext .compressionStrategy ();
354
+ String bloomFilterColumns =
355
+ PropertyUtil .propertyAsString (config , OrcConf .BLOOM_FILTER_COLUMNS .getAttribute (), "" );
356
+ double bloomFilterFpp =
357
+ PropertyUtil .propertyAsDouble (config , OrcConf .BLOOM_FILTER_FPP .getAttribute (), 0.05 );
358
+ bloomFilterFpp =
359
+ PropertyUtil .propertyAsDouble (config , ORC_BLOOM_FILTER_FPP , bloomFilterFpp );
319
360
320
361
return new Context (
321
- stripeSize , blockSize , vectorizedRowBatchSize , compressionKind , compressionStrategy );
362
+ stripeSize ,
363
+ blockSize ,
364
+ vectorizedRowBatchSize ,
365
+ compressionKind ,
366
+ compressionStrategy ,
367
+ bloomFilterColumns ,
368
+ bloomFilterFpp );
322
369
}
323
370
324
371
private static CompressionKind toCompressionKind (String codecAsString ) {
0 commit comments