forked from apache/datafusion-comet
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathCometScanRule.scala
More file actions
821 lines (721 loc) · 32.2 KB
/
CometScanRule.scala
File metadata and controls
821 lines (721 loc) · 32.2 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.comet.rules
import java.net.URI
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
import scala.jdk.CollectionConverters._
import org.apache.hadoop.conf.Configuration
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpression, Expression, GenericInternalRow, PlanExpression}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.util.{sideBySide, ArrayBasedMapData, GenericArrayData, MetadataColumnHelper}
import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues
import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec}
import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.InSubqueryExec
import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport}
import org.apache.comet.CometConf._
import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, withInfo, withInfos}
import org.apache.comet.DataTypeSupport.isComplexType
import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection}
import org.apache.comet.objectstore.NativeConfig
import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet}
import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported}
import org.apache.comet.serde.operator.CometNativeScan
import org.apache.comet.shims.CometTypeShim
/**
* Spark physical optimizer rule for replacing Spark scans with Comet scans.
*/
case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with CometTypeShim {
import CometScanRule._
private lazy val showTransformations = CometConf.COMET_EXPLAIN_TRANSFORMATIONS.get()
override def apply(plan: SparkPlan): SparkPlan = {
val newPlan = _apply(plan)
if (showTransformations && !newPlan.fastEquals(plan)) {
logInfo(s"""
|=== Applying Rule $ruleName ===
|${sideBySide(plan.treeString, newPlan.treeString).mkString("\n")}
|""".stripMargin)
}
newPlan
}
private def _apply(plan: SparkPlan): SparkPlan = {
if (!isCometLoaded(conf)) return plan
def isSupportedScanNode(plan: SparkPlan): Boolean = plan match {
case _: FileSourceScanExec => true
case _: BatchScanExec => true
case _ => false
}
def hasMetadataCol(plan: SparkPlan): Boolean = {
plan.expressions.exists(_.exists {
case a: Attribute =>
a.isMetadataCol
case _ => false
})
}
def isIcebergMetadataTable(scanExec: BatchScanExec): Boolean = {
// List of Iceberg metadata tables:
// https://iceberg.apache.org/docs/latest/spark-queries/#inspecting-tables
val metadataTableSuffix = Set(
"history",
"metadata_log_entries",
"snapshots",
"entries",
"files",
"manifests",
"partitions",
"position_deletes",
"all_data_files",
"all_delete_files",
"all_entries",
"all_manifests")
metadataTableSuffix.exists(suffix => scanExec.table.name().endsWith(suffix))
}
def transformScan(plan: SparkPlan): SparkPlan = plan match {
case scan if !CometConf.COMET_NATIVE_SCAN_ENABLED.get(conf) =>
withInfo(scan, "Comet Scan is not enabled")
case scan if hasMetadataCol(scan) =>
withInfo(scan, "Metadata column is not supported")
// data source V1
case scanExec: FileSourceScanExec =>
transformV1Scan(scanExec)
// data source V2
case scanExec: BatchScanExec =>
if (isIcebergMetadataTable(scanExec)) {
withInfo(scanExec, "Iceberg Metadata tables are not supported")
} else {
transformV2Scan(scanExec)
}
}
plan.transform {
case scan if isSupportedScanNode(scan) => transformScan(scan)
}
}
private def transformV1Scan(scanExec: FileSourceScanExec): SparkPlan = {
if (COMET_DPP_FALLBACK_ENABLED.get() &&
scanExec.partitionFilters.exists(isDynamicPruningFilter)) {
return withInfo(scanExec, "Dynamic Partition Pruning is not supported")
}
scanExec.relation match {
case r: HadoopFsRelation =>
if (!CometScanExec.isFileFormatSupported(r.fileFormat)) {
return withInfo(scanExec, s"Unsupported file format ${r.fileFormat}")
}
val hadoopConf = r.sparkSession.sessionState.newHadoopConfWithOptions(r.options)
// TODO is this restriction valid for all native scan types?
val possibleDefaultValues = getExistenceDefaultValues(scanExec.requiredSchema)
if (possibleDefaultValues.exists(d => {
d != null && (d.isInstanceOf[ArrayBasedMapData] || d
.isInstanceOf[GenericInternalRow] || d.isInstanceOf[GenericArrayData])
})) {
// Spark already converted these to Java-native types, so we can't check SQL types.
// ArrayBasedMapData, GenericInternalRow, GenericArrayData correspond to maps, structs,
// and arrays respectively.
withInfo(
scanExec,
"Full native scan disabled because default values for nested types are not supported")
return scanExec
}
COMET_NATIVE_SCAN_IMPL.get() match {
case SCAN_AUTO =>
// TODO add support for native_datafusion in the future
nativeIcebergCompatScan(session, scanExec, r, hadoopConf)
.getOrElse(scanExec)
case SCAN_NATIVE_DATAFUSION =>
nativeDataFusionScan(session, scanExec, r, hadoopConf).getOrElse(scanExec)
case SCAN_NATIVE_ICEBERG_COMPAT =>
nativeIcebergCompatScan(session, scanExec, r, hadoopConf).getOrElse(scanExec)
case SCAN_NATIVE_COMET =>
nativeCometScan(session, scanExec, r, hadoopConf).getOrElse(scanExec)
}
case _ =>
withInfo(scanExec, s"Unsupported relation ${scanExec.relation}")
}
}
private def nativeDataFusionScan(
session: SparkSession,
scanExec: FileSourceScanExec,
r: HadoopFsRelation,
hadoopConf: Configuration): Option[SparkPlan] = {
if (!CometNativeScan.isSupported(scanExec)) {
return None
}
if (encryptionEnabled(hadoopConf) && !isEncryptionConfigSupported(hadoopConf)) {
withInfo(scanExec, s"$SCAN_NATIVE_DATAFUSION does not support encryption")
return None
}
if (!isSchemaSupported(scanExec, SCAN_NATIVE_DATAFUSION, r)) {
return None
}
Some(CometScanExec(scanExec, session, SCAN_NATIVE_DATAFUSION))
}
private def nativeIcebergCompatScan(
session: SparkSession,
scanExec: FileSourceScanExec,
r: HadoopFsRelation,
hadoopConf: Configuration): Option[SparkPlan] = {
if (encryptionEnabled(hadoopConf) && !isEncryptionConfigSupported(hadoopConf)) {
withInfo(scanExec, s"$SCAN_NATIVE_ICEBERG_COMPAT does not support encryption")
return None
}
if (!isSchemaSupported(scanExec, SCAN_NATIVE_ICEBERG_COMPAT, r)) {
return None
}
Some(CometScanExec(scanExec, session, SCAN_NATIVE_ICEBERG_COMPAT))
}
private def nativeCometScan(
session: SparkSession,
scanExec: FileSourceScanExec,
r: HadoopFsRelation,
hadoopConf: Configuration): Option[SparkPlan] = {
if (!isSchemaSupported(scanExec, SCAN_NATIVE_COMET, r)) {
return None
}
Some(CometScanExec(scanExec, session, SCAN_NATIVE_COMET))
}
private def transformV2Scan(scanExec: BatchScanExec): SparkPlan = {
scanExec.scan match {
case scan: ParquetScan =>
val fallbackReasons = new ListBuffer[String]()
val schemaSupported =
CometBatchScanExec.isSchemaSupported(scan.readDataSchema, fallbackReasons)
if (!schemaSupported) {
fallbackReasons += s"Schema ${scan.readDataSchema} is not supported"
}
val partitionSchemaSupported =
CometBatchScanExec.isSchemaSupported(scan.readPartitionSchema, fallbackReasons)
if (!partitionSchemaSupported) {
fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported"
}
if (scan.pushedAggregate.nonEmpty) {
fallbackReasons += "Comet does not support pushed aggregate"
}
if (schemaSupported && partitionSchemaSupported && scan.pushedAggregate.isEmpty) {
val cometScan = CometParquetScan(session, scanExec.scan.asInstanceOf[ParquetScan])
CometBatchScanExec(
scanExec.copy(scan = cometScan),
runtimeFilters = scanExec.runtimeFilters)
} else {
withInfos(scanExec, fallbackReasons.toSet)
}
case scan: CSVScan if COMET_CSV_V2_NATIVE_ENABLED.get() =>
val fallbackReasons = new ListBuffer[String]()
val schemaSupported =
CometBatchScanExec.isSchemaSupported(scan.readDataSchema, fallbackReasons)
if (!schemaSupported) {
fallbackReasons += s"Schema ${scan.readDataSchema} is not supported"
}
val partitionSchemaSupported =
CometBatchScanExec.isSchemaSupported(scan.readPartitionSchema, fallbackReasons)
if (!partitionSchemaSupported) {
fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported"
}
val corruptedRecordsColumnName =
SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD)
val containsCorruptedRecordsColumn =
!scan.readDataSchema.fieldNames.contains(corruptedRecordsColumnName)
if (!containsCorruptedRecordsColumn) {
fallbackReasons += "Comet doesn't support the processing of corrupted records"
}
val isInferSchemaEnabled = scan.options.getBoolean("inferSchema", false)
if (isInferSchemaEnabled) {
fallbackReasons += "Comet doesn't support inferSchema=true option"
}
val delimiter =
Option(scan.options.get("delimiter"))
.orElse(Option(scan.options.get("sep")))
.getOrElse(",")
val isSingleCharacterDelimiter = delimiter.length == 1
if (!isSingleCharacterDelimiter) {
fallbackReasons +=
s"Comet supports only single-character delimiters, but got: '$delimiter'"
}
if (schemaSupported && partitionSchemaSupported && containsCorruptedRecordsColumn
&& !isInferSchemaEnabled && isSingleCharacterDelimiter) {
CometBatchScanExec(
scanExec.clone().asInstanceOf[BatchScanExec],
runtimeFilters = scanExec.runtimeFilters)
} else {
withInfos(scanExec, fallbackReasons.toSet)
}
// Iceberg scan - patched version implementing SupportsComet interface
case s: SupportsComet if !COMET_ICEBERG_NATIVE_ENABLED.get() =>
val fallbackReasons = new ListBuffer[String]()
if (!s.isCometEnabled) {
fallbackReasons += "Comet extension is not enabled for " +
s"${scanExec.scan.getClass.getSimpleName}: not enabled on data source side"
}
val schemaSupported =
CometBatchScanExec.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons)
if (!schemaSupported) {
fallbackReasons += "Comet extension is not enabled for " +
s"${scanExec.scan.getClass.getSimpleName}: Schema not supported"
}
if (s.isCometEnabled && schemaSupported) {
// When reading from Iceberg, we automatically enable type promotion
SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true")
CometBatchScanExec(
scanExec.clone().asInstanceOf[BatchScanExec],
runtimeFilters = scanExec.runtimeFilters)
} else {
withInfos(scanExec, fallbackReasons.toSet)
}
// Iceberg scan - detected by class name (works with unpatched Iceberg)
case _
if scanExec.scan.getClass.getName ==
"org.apache.iceberg.spark.source.SparkBatchQueryScan" =>
val fallbackReasons = new ListBuffer[String]()
// Native Iceberg scan requires both configs to be enabled
if (!COMET_ICEBERG_NATIVE_ENABLED.get()) {
fallbackReasons += "Native Iceberg scan disabled because " +
s"${COMET_ICEBERG_NATIVE_ENABLED.key} is not enabled"
return withInfos(scanExec, fallbackReasons.toSet)
}
if (!COMET_EXEC_ENABLED.get()) {
fallbackReasons += "Native Iceberg scan disabled because " +
s"${COMET_EXEC_ENABLED.key} is not enabled"
return withInfos(scanExec, fallbackReasons.toSet)
}
val typeChecker = CometScanTypeChecker(SCAN_NATIVE_DATAFUSION)
val schemaSupported =
typeChecker.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons)
if (!schemaSupported) {
fallbackReasons += "Comet extension is not enabled for " +
s"${scanExec.scan.getClass.getSimpleName}: Schema not supported"
}
// Extract all Iceberg metadata once using reflection.
// If any required reflection fails, this returns None, and we fall back to Spark.
// First get metadataLocation and catalogProperties which are needed by the factory.
val tableOpt = IcebergReflection.getTable(scanExec.scan)
val metadataLocationOpt = tableOpt.flatMap { table =>
IcebergReflection.getMetadataLocation(table)
}
val metadataOpt = metadataLocationOpt.flatMap { metadataLocation =>
try {
val session = org.apache.spark.sql.SparkSession.active
val hadoopConf = session.sessionState.newHadoopConf()
// For REST catalogs, the metadata file may not exist on disk since metadata
// is fetched via HTTP. Check if file exists; if not, use table location instead.
val metadataUri = new java.net.URI(metadataLocation)
val metadataFile = new java.io.File(metadataUri.getPath)
val effectiveLocation =
if (!metadataFile.exists() && metadataUri.getScheme == "file") {
// Metadata file doesn't exist (REST catalog with InMemoryFileIO or similar)
// Use table location instead for FileIO initialization
tableOpt
.flatMap { table =>
try {
val locationMethod = table.getClass.getMethod("location")
val tableLocation = locationMethod.invoke(table).asInstanceOf[String]
Some(tableLocation)
} catch {
case _: Exception =>
Some(metadataLocation)
}
}
.getOrElse(metadataLocation)
} else {
metadataLocation
}
val effectiveUri = new java.net.URI(effectiveLocation)
val hadoopS3Options = NativeConfig.extractObjectStoreOptions(hadoopConf, effectiveUri)
val catalogProperties =
org.apache.comet.serde.operator.CometIcebergNativeScan
.hadoopToIcebergS3Properties(hadoopS3Options)
val result = CometIcebergNativeScanMetadata
.extract(scanExec.scan, effectiveLocation, catalogProperties)
result
} catch {
case e: Exception =>
logError(
s"Failed to extract catalog properties from Iceberg scan: ${e.getMessage}",
e)
None
}
}
// If metadata extraction failed, fall back to Spark
val metadata = metadataOpt match {
case Some(m) => m
case None =>
fallbackReasons += "Failed to extract Iceberg metadata via reflection"
return withInfos(scanExec, fallbackReasons.toSet)
}
// Now perform all validation using the pre-extracted metadata
// Check if table uses a FileIO implementation compatible with iceberg-rust
val fileIOCompatible = IcebergReflection.getFileIO(metadata.table) match {
case Some(_) =>
// InMemoryFileIO is now supported with table location fallback for REST catalogs
true
case None =>
fallbackReasons += "Could not check FileIO compatibility"
false
}
// Check Iceberg table format version
val formatVersionSupported = IcebergReflection.getFormatVersion(metadata.table) match {
case Some(formatVersion) =>
if (formatVersion > 2) {
fallbackReasons += "Iceberg table format version " +
s"$formatVersion is not supported. " +
"Comet only supports Iceberg table format V1 and V2"
false
} else {
true
}
case None =>
fallbackReasons += "Could not verify Iceberg table format version"
false
}
// Check if all files are Parquet format and use supported filesystem schemes
val (allParquetFiles, unsupportedSchemes) =
IcebergReflection.validateFileFormatsAndSchemes(metadata.tasks)
val allSupportedFilesystems = if (unsupportedSchemes.isEmpty) {
true
} else {
fallbackReasons += "Iceberg scan contains files with unsupported filesystem " +
s"schemes: ${unsupportedSchemes.mkString(", ")}. " +
"Comet only supports: file, s3, s3a, gs, gcs, oss, abfss, abfs, wasbs, wasb"
false
}
if (!allParquetFiles) {
fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " +
"Comet only supports Parquet files in Iceberg tables"
}
// Partition values are deserialized via iceberg-rust's Literal::try_from_json()
// which has incomplete type support (binary/fixed unimplemented, decimals limited)
val partitionTypesSupported = (for {
partitionSpec <- IcebergReflection.getPartitionSpec(metadata.table)
} yield {
val unsupportedTypes =
IcebergReflection.validatePartitionTypes(partitionSpec, metadata.scanSchema)
if (unsupportedTypes.nonEmpty) {
unsupportedTypes.foreach { case (fieldName, typeStr, reason) =>
fallbackReasons +=
s"Partition column '$fieldName' with type $typeStr is not yet supported by " +
s"iceberg-rust: $reason"
}
false
} else {
true
}
}).getOrElse {
// Fall back to Spark if reflection fails - cannot verify safety
val msg =
"Iceberg reflection failure: Could not verify partition types compatibility"
logError(msg)
fallbackReasons += msg
false
}
// Get filter expressions for complex predicates check
val filterExpressionsOpt = IcebergReflection.getFilterExpressions(scanExec.scan)
// IS NULL/NOT NULL on complex types fail because iceberg-rust's accessor creation
// only handles primitive fields. Nested field filters work because Iceberg Java
// pre-binds them to field IDs. Element/key access filters don't push down to FileScanTasks.
val complexTypePredicatesSupported = filterExpressionsOpt
.map { filters =>
// Empty filters can't trigger accessor issues
if (filters.isEmpty) {
true
} else {
val readSchema = scanExec.scan.readSchema()
// Identify complex type columns that would trigger accessor creation failures
val complexColumns = readSchema
.filter(field => isComplexType(field.dataType))
.map(_.name)
.toSet
// Detect IS NULL/NOT NULL on complex columns (pattern: is_null(ref(name="col")))
// Nested field filters use different patterns and don't trigger this issue
val hasComplexNullCheck = filters.asScala.exists { expr =>
val exprStr = expr.toString
val isNullCheck = exprStr.contains("is_null") || exprStr.contains("not_null")
if (isNullCheck) {
complexColumns.exists { colName =>
exprStr.contains(s"""ref(name="$colName")""")
}
} else {
false
}
}
if (hasComplexNullCheck) {
fallbackReasons += "IS NULL / IS NOT NULL predicates on complex type columns " +
"(struct/array/map) are not yet supported by iceberg-rust " +
"(nested field filters like address.city = 'NYC' are supported)"
false
} else {
true
}
}
}
.getOrElse {
// Fall back to Spark if reflection fails - cannot verify safety
val msg =
"Iceberg reflection failure: Could not check for complex type predicates"
logError(msg)
fallbackReasons += msg
false
}
// Check for unsupported transform functions in residual expressions
// iceberg-rust can only handle identity transforms in residuals; all other transforms
// (truncate, bucket, year, month, day, hour) must fall back to Spark
val transformFunctionsSupported =
try {
IcebergReflection.findNonIdentityTransformInResiduals(metadata.tasks) match {
case Some(transformType) =>
// Found unsupported transform
fallbackReasons +=
s"Iceberg transform function '$transformType' in residual expression " +
"is not yet supported by iceberg-rust. " +
"Only identity transforms are supported."
false
case None =>
// No unsupported transforms found - safe to use native execution
true
}
} catch {
case e: Exception =>
// Reflection failure - cannot verify safety, must fall back
fallbackReasons += "Iceberg reflection failure: Could not check for " +
s"transform functions in residuals: ${e.getMessage}"
false
}
// Check for unsupported struct types in delete files
val deleteFileTypesSupported = {
var hasUnsupportedDeletes = false
try {
val deleteFiles = IcebergReflection.getDeleteFiles(metadata.tasks)
if (!deleteFiles.isEmpty) {
deleteFiles.asScala.foreach { deleteFile =>
val equalityFieldIds = IcebergReflection.getEqualityFieldIds(deleteFile)
if (!equalityFieldIds.isEmpty) {
// Look up field types
equalityFieldIds.asScala.foreach { fieldId =>
val fieldInfo = IcebergReflection.getFieldInfo(
metadata.scanSchema,
fieldId.asInstanceOf[Int])
fieldInfo match {
case Some((fieldName, fieldType)) =>
if (fieldType.contains("struct")) {
hasUnsupportedDeletes = true
fallbackReasons +=
s"Equality delete on unsupported column type '$fieldName' " +
s"($fieldType) is not yet supported by iceberg-rust. " +
"Struct types in equality deletes " +
"require datum conversion support that is not yet implemented."
}
case None =>
}
}
}
}
}
} catch {
case e: Exception =>
// Reflection failure means we cannot verify safety - must fall back
hasUnsupportedDeletes = true
fallbackReasons += "Iceberg reflection failure: Could not verify delete file " +
s"types for safety: ${e.getMessage}"
}
!hasUnsupportedDeletes
}
// Check that all DPP subqueries use InSubqueryExec which we know how to handle.
// Future Spark versions might introduce new subquery types we haven't tested.
val dppSubqueriesSupported = {
val unsupportedSubqueries = scanExec.runtimeFilters.collect {
case DynamicPruningExpression(e) if !e.isInstanceOf[InSubqueryExec] =>
e.getClass.getSimpleName
}
if (unsupportedSubqueries.nonEmpty) {
fallbackReasons +=
s"Unsupported DPP subquery types: ${unsupportedSubqueries.mkString(", ")}. " +
"CometIcebergNativeScanExec only supports InSubqueryExec for DPP"
false
} else {
true
}
}
if (schemaSupported && fileIOCompatible && formatVersionSupported && allParquetFiles &&
allSupportedFilesystems && partitionTypesSupported &&
complexTypePredicatesSupported && transformFunctionsSupported &&
deleteFileTypesSupported && dppSubqueriesSupported) {
CometBatchScanExec(
scanExec.clone().asInstanceOf[BatchScanExec],
runtimeFilters = scanExec.runtimeFilters,
nativeIcebergScanMetadata = Some(metadata))
} else {
withInfos(scanExec, fallbackReasons.toSet)
}
case other =>
withInfo(
scanExec,
s"Unsupported scan: ${other.getClass.getName}. " +
"Comet Scan only supports Parquet and Iceberg Parquet file formats")
}
}
private def selectScan(
scanExec: FileSourceScanExec,
partitionSchema: StructType,
hadoopConf: Configuration): String = {
val fallbackReasons = new ListBuffer[String]()
// native_iceberg_compat only supports local filesystem and S3
if (scanExec.relation.inputFiles
.forall(path => path.startsWith("file://") || path.startsWith("s3a://"))) {
val filePath = scanExec.relation.inputFiles.headOption
if (filePath.exists(_.startsWith("s3a://"))) {
validateObjectStoreConfig(filePath.get, hadoopConf, fallbackReasons)
}
} else {
fallbackReasons += s"$SCAN_NATIVE_ICEBERG_COMPAT only supports local filesystem and S3"
}
val typeChecker = CometScanTypeChecker(SCAN_NATIVE_ICEBERG_COMPAT)
val schemaSupported =
typeChecker.isSchemaSupported(scanExec.requiredSchema, fallbackReasons)
val partitionSchemaSupported =
typeChecker.isSchemaSupported(partitionSchema, fallbackReasons)
val cometExecEnabled = COMET_EXEC_ENABLED.get()
if (!cometExecEnabled) {
fallbackReasons += s"$SCAN_NATIVE_ICEBERG_COMPAT requires ${COMET_EXEC_ENABLED.key}=true"
}
if (cometExecEnabled && schemaSupported && partitionSchemaSupported &&
fallbackReasons.isEmpty) {
logInfo(s"Auto scan mode selecting $SCAN_NATIVE_ICEBERG_COMPAT")
SCAN_NATIVE_ICEBERG_COMPAT
} else {
logInfo(
s"Auto scan mode falling back to $SCAN_NATIVE_COMET due to " +
s"${fallbackReasons.mkString(", ")}")
SCAN_NATIVE_COMET
}
}
private def isDynamicPruningFilter(e: Expression): Boolean =
e.exists(_.isInstanceOf[PlanExpression[_]])
private def isSchemaSupported(
scanExec: FileSourceScanExec,
scanImpl: String,
r: HadoopFsRelation): Boolean = {
val fallbackReasons = new ListBuffer[String]()
val typeChecker = CometScanTypeChecker(scanImpl)
val schemaSupported =
typeChecker.isSchemaSupported(scanExec.requiredSchema, fallbackReasons)
if (!schemaSupported) {
withInfo(
scanExec,
s"Unsupported schema ${scanExec.requiredSchema} " +
s"for $scanImpl: ${fallbackReasons.mkString(", ")}")
return false
}
val partitionSchemaSupported =
typeChecker.isSchemaSupported(r.partitionSchema, fallbackReasons)
if (!partitionSchemaSupported) {
withInfo(
scanExec,
s"Unsupported partitioning schema ${scanExec.requiredSchema} " +
s"for $scanImpl: ${fallbackReasons
.mkString(", ")}")
return false
}
true
}
}
case class CometScanTypeChecker(scanImpl: String) extends DataTypeSupport with CometTypeShim {
// this class is intended to be used with a specific scan impl
assert(scanImpl != CometConf.SCAN_AUTO)
override def isTypeSupported(
dt: DataType,
name: String,
fallbackReasons: ListBuffer[String]): Boolean = {
dt match {
case ShortType
if scanImpl != CometConf.SCAN_NATIVE_COMET &&
CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.get() =>
fallbackReasons += s"$scanImpl scan may not handle unsigned UINT_8 correctly for $dt. " +
s"Set ${CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key}=false to allow " +
"native execution if your data does not contain unsigned small integers. " +
CometConf.COMPAT_GUIDE
false
case _: StructType | _: ArrayType | _: MapType if scanImpl == CometConf.SCAN_NATIVE_COMET =>
false
case dt if isStringCollationType(dt) =>
// we don't need specific support for collation in scans, but this
// is a convenient place to force the whole query to fall back to Spark for now
false
case s: StructType if s.fields.isEmpty =>
false
case _ =>
super.isTypeSupported(dt, name, fallbackReasons)
}
}
}
object CometScanRule extends Logging {
/**
* Validating object store configs can cause requests to be made to S3 APIs (such as when
* resolving the region for a bucket). We use a cache to reduce the number of S3 calls.
*
* The key is the config map converted to a string. The value is the reason that the config is
* not valid, or None if the config is valid.
*/
val configValidityMap = new mutable.HashMap[String, Option[String]]()
/**
* We do not expect to see a large number of unique configs within the lifetime of a Spark
* session, but we reset the cache once it reaches a fixed size to prevent it growing
* indefinitely.
*/
val configValidityMapMaxSize = 1024
def validateObjectStoreConfig(
filePath: String,
hadoopConf: Configuration,
fallbackReasons: mutable.ListBuffer[String]): Unit = {
val objectStoreConfigMap =
NativeConfig.extractObjectStoreOptions(hadoopConf, URI.create(filePath))
val cacheKey = objectStoreConfigMap
.map { case (k, v) =>
s"$k=$v"
}
.toList
.sorted
.mkString("\n")
if (configValidityMap.size >= configValidityMapMaxSize) {
logWarning("Resetting S3 object store validity cache")
configValidityMap.clear()
}
configValidityMap.get(cacheKey) match {
case Some(Some(reason)) =>
fallbackReasons += reason
case Some(None) =>
// previously validated
case _ =>
try {
val objectStoreOptions = objectStoreConfigMap.asJava
Native.validateObjectStoreConfig(filePath, objectStoreOptions)
} catch {
case e: CometNativeException =>
val reason = "Object store config not supported by " +
s"$SCAN_NATIVE_ICEBERG_COMPAT: ${e.getMessage}"
fallbackReasons += reason
configValidityMap.put(cacheKey, Some(reason))
}
}
}
}