This is an automated email from the ASF dual-hosted git repository.

felixybw pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new a508e82089 [GLUTEN-11417][VL] Enhance qualification tool to support 
lakehouse format detection (#11436)
a508e82089 is described below

commit a508e8208932fb07261e4b0de89cf52fe0560e2f
Author: rlizzy <[email protected]>
AuthorDate: Wed Feb 11 21:03:47 2026 -0800

    [GLUTEN-11417][VL] Enhance qualification tool to support lakehouse format 
detection (#11436)
    
    Add support for detecting and evaluating lakehouse table formats (Iceberg,
    Delta Lake, Hudi, Paimon) in the qualification tool. The tool previously
    only recognized raw file formats and reported negative improvements for
    lakehouse workloads.
    
    New classes:
    - LakehouseFormatDetector: Central detector using multi-signal approach
    - ProviderClassExtractor: Extracts provider class from node description
    - LocationPatternMatcher: Fallback detection via location path patterns
    
    Detection strategy (in order of reliability):
    1. Provider class extraction from nodeDesc (most reliable)
    2. Location path pattern matching (for Delta/_delta_log, Hudi/.hoodie)
    3. Node name patterns (least reliable, fallback only)
    
    Key design decisions:
    - Support is conditional based on detected format, NOT blanket operator
    - BatchScanExec is NOT added to supported operators list
    - Underlying file format (Parquet/ORC) determines actual support
    - Same schema type checks apply to lakehouse scans
    
    Closes #11417
    
    Signed-off-by: ruo <[email protected]>
    Co-authored-by: Claude Opus 4.5 <[email protected]>
---
 .../gluten/qt/support/LakehouseFormatDetector.java | 132 ++++++++
 .../gluten/qt/support/LocationPatternMatcher.java  |  98 ++++++
 .../gluten/qt/support/NodeSupportVisitor.java      |  44 ++-
 .../gluten/qt/support/ProviderClassExtractor.java  |  87 +++++
 .../qt/support/LakehouseFormatDetectorTest.java    | 305 +++++++++++++++++
 .../support/NodeSupportVisitorLakehouseTest.java   | 366 +++++++++++++++++++++
 6 files changed, 1030 insertions(+), 2 deletions(-)

diff --git 
a/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/LakehouseFormatDetector.java
 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/LakehouseFormatDetector.java
new file mode 100644
index 0000000000..3f34d3b18f
--- /dev/null
+++ 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/LakehouseFormatDetector.java
@@ -0,0 +1,132 @@
+/*
+ * 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.gluten.qt.support;
+
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Detects lakehouse table formats (Iceberg, Delta Lake, Hudi, Paimon) from 
Spark execution plan
+ * scan nodes. This class uses a multi-signal detection approach combining:
+ *
+ * <ul>
+ *   <li>Provider class extraction from node description
+ *   <li>Location path pattern matching (fallback for Delta/Hudi)
+ *   <li>Node name pattern matching (secondary signal)
+ * </ul>
+ *
+ * <p>Detection is performed in order of reliability: provider class > 
location pattern > node name.
+ */
+public class LakehouseFormatDetector {
+
+  private static final Pattern FORMAT_PATTERN = Pattern.compile("Format: 
([^,\\]]+)");
+
+  private final ProviderClassExtractor providerExtractor;
+  private final LocationPatternMatcher locationMatcher;
+
+  public LakehouseFormatDetector() {
+    this.providerExtractor = new ProviderClassExtractor();
+    this.locationMatcher = new LocationPatternMatcher();
+  }
+
+  /**
+   * Detects the lakehouse format from a scan node.
+   *
+   * @param nodeName the operator name (e.g., "BatchScanExec", "FileScan 
parquet")
+   * @param nodeDesc the full node description containing provider class, 
location, format, etc.
+   * @return the detected format result, or empty if not a lakehouse scan
+   */
+  public Optional<LakehouseFormat> detect(String nodeName, String nodeDesc) {
+    // First try provider class extraction (most reliable)
+    Optional<LakehouseFormat> fromProvider = 
providerExtractor.extract(nodeDesc);
+    if (fromProvider.isPresent()) {
+      return fromProvider;
+    }
+
+    // Then try location pattern matching (for Delta/Hudi without provider 
class)
+    Optional<LakehouseFormat> fromLocation = locationMatcher.match(nodeDesc);
+    if (fromLocation.isPresent()) {
+      return fromLocation;
+    }
+
+    // Finally try node name patterns (least reliable, but catches some cases)
+    return detectFromNodeName(nodeName);
+  }
+
+  /**
+   * Detects lakehouse format from node name patterns. This is the least 
reliable method and should
+   * only be used as a fallback.
+   */
+  private Optional<LakehouseFormat> detectFromNodeName(String nodeName) {
+    if (nodeName == null) {
+      return Optional.empty();
+    }
+
+    String lowerName = nodeName.toLowerCase();
+
+    // Check for explicit lakehouse scan names
+    if (lowerName.contains("icebergscan") || lowerName.contains("iceberg")) {
+      return Optional.of(LakehouseFormat.ICEBERG);
+    }
+    if (lowerName.contains("deltascan") || lowerName.contains("tahoefile")) {
+      return Optional.of(LakehouseFormat.DELTA);
+    }
+    if (lowerName.contains("hudifilescan")
+        || lowerName.contains("hoodiefilescan")
+        || lowerName.contains("hoodie")) {
+      return Optional.of(LakehouseFormat.HUDI);
+    }
+    if (lowerName.contains("paimonscan") || lowerName.contains("paimon")) {
+      return Optional.of(LakehouseFormat.PAIMON);
+    }
+
+    return Optional.empty();
+  }
+
+  /**
+   * Extracts the underlying file format from node description.
+   *
+   * @param nodeDesc the node description
+   * @return the file format (e.g., "Parquet", "ORC"), or empty string if not 
found
+   */
+  public String extractFileFormat(String nodeDesc) {
+    if (nodeDesc == null) {
+      return "";
+    }
+    Matcher matcher = FORMAT_PATTERN.matcher(nodeDesc);
+    return matcher.find() ? matcher.group(1).trim() : "";
+  }
+
+  /** Represents the detected lakehouse table format. */
+  public enum LakehouseFormat {
+    ICEBERG("Iceberg"),
+    DELTA("Delta"),
+    HUDI("Hudi"),
+    PAIMON("Paimon");
+
+    private final String displayName;
+
+    LakehouseFormat(String displayName) {
+      this.displayName = displayName;
+    }
+
+    public String getDisplayName() {
+      return displayName;
+    }
+  }
+}
diff --git 
a/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/LocationPatternMatcher.java
 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/LocationPatternMatcher.java
new file mode 100644
index 0000000000..112c4148bc
--- /dev/null
+++ 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/LocationPatternMatcher.java
@@ -0,0 +1,98 @@
+/*
+ * 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.gluten.qt.support;
+
+import org.apache.gluten.qt.support.LakehouseFormatDetector.LakehouseFormat;
+
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Detects lakehouse table formats by matching location path patterns in node 
descriptions. This is
+ * a fallback detection method when provider class information is not 
available.
+ *
+ * <p>This is particularly useful for:
+ *
+ * <ul>
+ *   <li>Delta Lake: Often appears as {@code FileScan parquet} with {@code 
_delta_log} in the
+ *       location path
+ *   <li>Hudi: May have {@code .hoodie} marker directory in the location path
+ * </ul>
+ *
+ * <p>Note: Iceberg and Paimon typically include provider class information in 
the node description,
+ * so location-based detection is less reliable for these formats.
+ */
+public class LocationPatternMatcher {
+
+  // Extract location path from node description
+  private static final Pattern LOCATION_PATTERN = Pattern.compile("Location: 
([^,\\]]+)");
+
+  // Delta Lake marker - _delta_log directory indicates a Delta table
+  private static final Pattern DELTA_LOCATION = Pattern.compile("_delta_log");
+
+  // Hudi marker - .hoodie directory indicates a Hudi table
+  private static final Pattern HUDI_LOCATION = Pattern.compile("\\.hoodie");
+
+  /**
+   * Matches the node description against known location patterns for 
lakehouse formats.
+   *
+   * @param nodeDesc the full node description
+   * @return the detected lakehouse format, or empty if no location pattern 
matches
+   */
+  public Optional<LakehouseFormat> match(String nodeDesc) {
+    if (nodeDesc == null || nodeDesc.isEmpty()) {
+      return Optional.empty();
+    }
+
+    // Extract location path first
+    String location = extractLocation(nodeDesc);
+    if (location.isEmpty()) {
+      // If no Location field, check the entire description for patterns
+      // This handles cases where the path might be in a different format
+      location = nodeDesc;
+    }
+
+    // Check for Delta Lake pattern
+    if (matchesPattern(DELTA_LOCATION, location)) {
+      return Optional.of(LakehouseFormat.DELTA);
+    }
+
+    // Check for Hudi pattern
+    if (matchesPattern(HUDI_LOCATION, location)) {
+      return Optional.of(LakehouseFormat.HUDI);
+    }
+
+    return Optional.empty();
+  }
+
+  /**
+   * Extracts the location path from the node description.
+   *
+   * @param nodeDesc the node description
+   * @return the extracted location path, or empty string if not found
+   */
+  private String extractLocation(String nodeDesc) {
+    Matcher matcher = LOCATION_PATTERN.matcher(nodeDesc);
+    return matcher.find() ? matcher.group(1) : "";
+  }
+
+  private boolean matchesPattern(Pattern pattern, String text) {
+    Matcher matcher = pattern.matcher(text);
+    return matcher.find();
+  }
+}
diff --git 
a/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/NodeSupportVisitor.java
 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/NodeSupportVisitor.java
index bcb922558c..0aa10b3954 100644
--- 
a/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/NodeSupportVisitor.java
+++ 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/NodeSupportVisitor.java
@@ -17,6 +17,7 @@
 package org.apache.gluten.qt.support;
 
 import org.apache.gluten.qt.graph.SparkPlanGraphNodeInternal;
+import org.apache.gluten.qt.support.LakehouseFormatDetector.LakehouseFormat;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
@@ -52,6 +53,9 @@ public class NodeSupportVisitor extends GraphVisitor {
   private static final Pattern SIZE_PATTERN = 
Pattern.compile("([\\d.]+)\\s*([a-zA-Z]+)");
   private static final Pattern FORMAT_PATTERN = Pattern.compile("Format: 
([^,]+)");
   private static final Pattern SCHEMA_PATTERN = Pattern.compile("ReadSchema: 
struct<(.*)");
+
+  private final LakehouseFormatDetector lakehouseDetector = new 
LakehouseFormatDetector();
+
   private static final ImmutableSet<String> FULLY_SUPPORTED_OPERATORS =
       ImmutableSet.of(
           "AdaptiveSparkPlan",
@@ -193,10 +197,46 @@ public class NodeSupportVisitor extends GraphVisitor {
           ? new Supported()
           : new NotSupported(NODE_NOT_SUPPORTED, "BHJ Not Supported");
     } else if (name.contains("Scan")) {
-      Matcher formatMatcher = FORMAT_PATTERN.matcher(desc);
+      // Try to detect lakehouse format first
+      Optional<LakehouseFormat> lakehouseFormat = 
lakehouseDetector.detect(name, desc);
+
       Matcher schemaMatcher = SCHEMA_PATTERN.matcher(desc);
-      String format = formatMatcher.find() ? formatMatcher.group(1) : "";
       String schema = schemaMatcher.find() ? schemaMatcher.group(1) : "";
+
+      if (lakehouseFormat.isPresent()) {
+        // Lakehouse format detected - check underlying file format
+        String fileFormat = lakehouseDetector.extractFileFormat(desc);
+        String formatPrefix = lakehouseFormat.get().getDisplayName() + " Scan";
+
+        if (fileFormat.isEmpty()) {
+          return new NotSupported(
+              NODE_NOT_SUPPORTED, formatPrefix + " with format UNKNOWN not 
supported");
+        }
+
+        // Lakehouse formats use Parquet/ORC underneath - apply same type 
checks
+        if (fileFormat.equals("Parquet") && checkTypeSupport(schema, 
PARQUET_UNSUPPORTED_TYPES)) {
+          return new Supported();
+        } else if (fileFormat.equals("ORC") && checkTypeSupport(schema, 
ORC_UNSUPPORTED_TYPES)) {
+          return new Supported();
+        } else if (fileFormat.equals("Parquet")) {
+          return new NotSupported(
+              NODE_NOT_SUPPORTED,
+              formatPrefix + " with " + schemaConditions(schema, 
PARQUET_UNSUPPORTED_TYPES));
+        } else if (fileFormat.equals("ORC")) {
+          return new NotSupported(
+              NODE_NOT_SUPPORTED,
+              formatPrefix + " with " + schemaConditions(schema, 
ORC_UNSUPPORTED_TYPES));
+        } else {
+          // Unsupported underlying format (e.g., Avro for Iceberg)
+          return new NotSupported(
+              NODE_NOT_SUPPORTED, formatPrefix + " (" + fileFormat + ") not 
supported");
+        }
+      }
+
+      // Not a lakehouse format - fall back to raw format detection
+      Matcher formatMatcher = FORMAT_PATTERN.matcher(desc);
+      String format = formatMatcher.find() ? formatMatcher.group(1) : "";
+
       if (format.equals("Parquet") && checkTypeSupport(schema, 
PARQUET_UNSUPPORTED_TYPES)) {
         return new Supported();
       } else if (format.equals("ORC") && checkTypeSupport(schema, 
ORC_UNSUPPORTED_TYPES)) {
diff --git 
a/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/ProviderClassExtractor.java
 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/ProviderClassExtractor.java
new file mode 100644
index 0000000000..77e2d49156
--- /dev/null
+++ 
b/tools/qualification-tool/src/main/java/org/apache/gluten/qt/support/ProviderClassExtractor.java
@@ -0,0 +1,87 @@
+/*
+ * 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.gluten.qt.support;
+
+import org.apache.gluten.qt.support.LakehouseFormatDetector.LakehouseFormat;
+
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Extracts and identifies provider/source class from Spark execution plan 
node descriptions. This
+ * is the most reliable method for detecting lakehouse table formats as it 
directly identifies the
+ * data source provider class.
+ *
+ * <p>Supported provider patterns:
+ *
+ * <ul>
+ *   <li>Iceberg: {@code org.apache.iceberg.spark.source.*}
+ *   <li>Delta Lake: {@code io.delta.*} or {@code 
com.databricks.sql.transaction.tahoe.*}
+ *   <li>Hudi: {@code org.apache.hudi.*}
+ *   <li>Paimon: {@code org.apache.paimon.*}
+ * </ul>
+ */
+public class ProviderClassExtractor {
+
+  // Provider class patterns - order matters for matching priority
+  private static final Pattern ICEBERG_PROVIDER =
+      Pattern.compile("org\\.apache\\.iceberg\\.spark\\.source\\.[\\w.]+");
+
+  private static final Pattern DELTA_PROVIDER =
+      
Pattern.compile("(io\\.delta\\.|com\\.databricks\\.sql\\.transaction\\.tahoe\\.)[\\w.]+");
+
+  private static final Pattern HUDI_PROVIDER = 
Pattern.compile("org\\.apache\\.hudi\\.[\\w.]+");
+
+  private static final Pattern PAIMON_PROVIDER = 
Pattern.compile("org\\.apache\\.paimon\\.[\\w.]+");
+
+  /**
+   * Extracts the lakehouse format from the node description by matching 
provider class patterns.
+   *
+   * @param nodeDesc the full node description
+   * @return the detected lakehouse format, or empty if no provider class 
matches
+   */
+  public Optional<LakehouseFormat> extract(String nodeDesc) {
+    if (nodeDesc == null || nodeDesc.isEmpty()) {
+      return Optional.empty();
+    }
+
+    // Check patterns in order of specificity
+    if (matchesPattern(ICEBERG_PROVIDER, nodeDesc)) {
+      return Optional.of(LakehouseFormat.ICEBERG);
+    }
+
+    if (matchesPattern(DELTA_PROVIDER, nodeDesc)) {
+      return Optional.of(LakehouseFormat.DELTA);
+    }
+
+    if (matchesPattern(HUDI_PROVIDER, nodeDesc)) {
+      return Optional.of(LakehouseFormat.HUDI);
+    }
+
+    if (matchesPattern(PAIMON_PROVIDER, nodeDesc)) {
+      return Optional.of(LakehouseFormat.PAIMON);
+    }
+
+    return Optional.empty();
+  }
+
+  private boolean matchesPattern(Pattern pattern, String text) {
+    Matcher matcher = pattern.matcher(text);
+    return matcher.find();
+  }
+}
diff --git 
a/tools/qualification-tool/src/test/java/org/apache/gluten/qt/support/LakehouseFormatDetectorTest.java
 
b/tools/qualification-tool/src/test/java/org/apache/gluten/qt/support/LakehouseFormatDetectorTest.java
new file mode 100644
index 0000000000..bf76a612b7
--- /dev/null
+++ 
b/tools/qualification-tool/src/test/java/org/apache/gluten/qt/support/LakehouseFormatDetectorTest.java
@@ -0,0 +1,305 @@
+/*
+ * 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.gluten.qt.support;
+
+import org.apache.gluten.qt.support.LakehouseFormatDetector.LakehouseFormat;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.junit.Assert.*;
+
+/** Unit tests for {@link LakehouseFormatDetector}. */
+public class LakehouseFormatDetectorTest {
+
+  private LakehouseFormatDetector detector;
+
+  @Before
+  public void setUp() {
+    detector = new LakehouseFormatDetector();
+  }
+
+  // ==================== Iceberg Detection Tests ====================
+
+  @Test
+  public void testDetectIcebergFromProviderClass() {
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] IcebergScan\n"
+            + "org.apache.iceberg.spark.source.SparkBatchQueryScan@12345\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("BatchScanExec", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.ICEBERG, result.get());
+  }
+
+  @Test
+  public void testDetectIcebergFromNodeName() {
+    String nodeDesc = "Format: Parquet, ReadSchema: struct<col1:string>";
+
+    Optional<LakehouseFormat> result = detector.detect("IcebergScan", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.ICEBERG, result.get());
+  }
+
+  // ==================== Delta Lake Detection Tests ====================
+
+  @Test
+  public void testDetectDeltaFromProviderClass() {
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "io.delta.tables.DeltaTable\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.DELTA, result.get());
+  }
+
+  @Test
+  public void testDetectDeltaFromTahoeProviderClass() {
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "com.databricks.sql.transaction.tahoe.DeltaLog\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.DELTA, result.get());
+  }
+
+  @Test
+  public void testDetectDeltaFromLocationPath() {
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: Parquet,\n"
+            + "Location: 
InMemoryFileIndex[dbfs:/user/hive/warehouse/sales/_delta_log],\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.DELTA, result.get());
+  }
+
+  @Test
+  public void testDetectDeltaFromNodeName() {
+    String nodeDesc = "Format: Parquet, ReadSchema: struct<col1:string>";
+
+    Optional<LakehouseFormat> result = detector.detect("DeltaScan", nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.DELTA, result.get());
+  }
+
+  // ==================== Hudi Detection Tests ====================
+
+  @Test
+  public void testDetectHudiFromProviderClass() {
+    String nodeDesc =
+        "HoodieFileScan [col1#0, col2#1]\n"
+            + "org.apache.hudi.HoodieSparkUtils\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("HoodieFileScan", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.HUDI, result.get());
+  }
+
+  @Test
+  public void testDetectHudiFromLocationPath() {
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "Location: s3://bucket/table/.hoodie/metadata,\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.HUDI, result.get());
+  }
+
+  @Test
+  public void testDetectHudiFromNodeName() {
+    String nodeDesc = "Format: Parquet, ReadSchema: struct<col1:string>";
+
+    Optional<LakehouseFormat> result = detector.detect("HudiFileScan", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.HUDI, result.get());
+  }
+
+  // ==================== Paimon Detection Tests ====================
+
+  @Test
+  public void testDetectPaimonFromProviderClass() {
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] PaimonScan\n"
+            + "org.apache.paimon.spark.SparkSource\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("BatchScanExec", 
nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.PAIMON, result.get());
+  }
+
+  @Test
+  public void testDetectPaimonFromNodeName() {
+    String nodeDesc = "Format: Parquet, ReadSchema: struct<col1:string>";
+
+    Optional<LakehouseFormat> result = detector.detect("PaimonScan", nodeDesc);
+
+    assertTrue(result.isPresent());
+    assertEquals(LakehouseFormat.PAIMON, result.get());
+  }
+
+  // ==================== Raw Format Tests (No Lakehouse) ====================
+
+  @Test
+  public void testRawParquetScan() {
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: Parquet,\n"
+            + "Location: s3://bucket/raw_parquet_table,\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
nodeDesc);
+
+    assertFalse(result.isPresent());
+  }
+
+  @Test
+  public void testRawOrcScan() {
+    String nodeDesc =
+        "FileScan orc [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: ORC,\n"
+            + "Location: hdfs://cluster/raw_orc_table,\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan orc", 
nodeDesc);
+
+    assertFalse(result.isPresent());
+  }
+
+  // ==================== False Positive Prevention Tests ====================
+
+  @Test
+  public void testTableNamedDeltaSalesNotDetectedAsDelta() {
+    // A raw Parquet table named "delta_sales" should NOT be detected as Delta
+    String nodeDesc =
+        "FileScan parquet delta_sales [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: Parquet,\n"
+            + "Location: s3://bucket/delta_sales,\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
nodeDesc);
+
+    // Should NOT detect as Delta since there's no provider class or _delta_log
+    assertFalse(result.isPresent());
+  }
+
+  @Test
+  public void testGenericBatchScanExecNotDetected() {
+    // Generic V2 scan (e.g., Kafka) should NOT be detected as lakehouse
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1]\n"
+            + "org.apache.spark.sql.kafka010.KafkaSourceProvider\n"
+            + "Format: kafka, ReadSchema: struct<key:binary,value:binary>";
+
+    Optional<LakehouseFormat> result = detector.detect("BatchScanExec", 
nodeDesc);
+
+    assertFalse(result.isPresent());
+  }
+
+  // ==================== File Format Extraction Tests ====================
+
+  @Test
+  public void testExtractParquetFormat() {
+    String nodeDesc = "Format: Parquet, ReadSchema: struct<col1:string>";
+
+    String format = detector.extractFileFormat(nodeDesc);
+
+    assertEquals("Parquet", format);
+  }
+
+  @Test
+  public void testExtractOrcFormat() {
+    String nodeDesc = "Format: ORC, ReadSchema: struct<col1:string>";
+
+    String format = detector.extractFileFormat(nodeDesc);
+
+    assertEquals("ORC", format);
+  }
+
+  @Test
+  public void testExtractAvroFormat() {
+    String nodeDesc = "Format: Avro, ReadSchema: struct<col1:string>";
+
+    String format = detector.extractFileFormat(nodeDesc);
+
+    assertEquals("Avro", format);
+  }
+
+  @Test
+  public void testExtractFormatMissing() {
+    String nodeDesc = "ReadSchema: struct<col1:string>";
+
+    String format = detector.extractFileFormat(nodeDesc);
+
+    assertEquals("", format);
+  }
+
+  @Test
+  public void testExtractFormatNullDesc() {
+    String format = detector.extractFileFormat(null);
+
+    assertEquals("", format);
+  }
+
+  // ==================== Edge Cases ====================
+
+  @Test
+  public void testNullNodeName() {
+    String nodeDesc = "Format: Parquet";
+
+    Optional<LakehouseFormat> result = detector.detect(null, nodeDesc);
+
+    assertFalse(result.isPresent());
+  }
+
+  @Test
+  public void testNullNodeDesc() {
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", 
null);
+
+    assertFalse(result.isPresent());
+  }
+
+  @Test
+  public void testEmptyNodeDesc() {
+    Optional<LakehouseFormat> result = detector.detect("FileScan parquet", "");
+
+    assertFalse(result.isPresent());
+  }
+}
diff --git 
a/tools/qualification-tool/src/test/java/org/apache/gluten/qt/support/NodeSupportVisitorLakehouseTest.java
 
b/tools/qualification-tool/src/test/java/org/apache/gluten/qt/support/NodeSupportVisitorLakehouseTest.java
new file mode 100644
index 0000000000..34ac222db5
--- /dev/null
+++ 
b/tools/qualification-tool/src/test/java/org/apache/gluten/qt/support/NodeSupportVisitorLakehouseTest.java
@@ -0,0 +1,366 @@
+/*
+ * 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.gluten.qt.support;
+
+import org.apache.gluten.qt.graph.MetricInternal;
+import org.apache.gluten.qt.graph.SparkPlanGraphInternal;
+import org.apache.gluten.qt.graph.SparkPlanGraphNodeInternal;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * Integration tests for {@link NodeSupportVisitor} with lakehouse format scan 
nodes. These tests
+ * verify that the visitor correctly identifies and evaluates support for 
Iceberg, Delta Lake, Hudi,
+ * and Paimon scan nodes.
+ */
+public class NodeSupportVisitorLakehouseTest {
+
+  // ==================== Iceberg Integration Tests ====================
+
+  @Test
+  public void testIcebergParquetScanSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] IcebergScan\n"
+            + "org.apache.iceberg.spark.source.SparkBatchQueryScan@12345\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Iceberg Parquet scan should be supported", support instanceof 
Supported);
+  }
+
+  @Test
+  public void testIcebergOrcScanSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] IcebergScan\n"
+            + "org.apache.iceberg.spark.source.SparkBatchQueryScan@12345\n"
+            + "Format: ORC, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Iceberg ORC scan should be supported", support instanceof 
Supported);
+  }
+
+  @Test
+  public void testIcebergAvroScanNotSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] IcebergScan\n"
+            + "org.apache.iceberg.spark.source.SparkBatchQueryScan@12345\n"
+            + "Format: Avro, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Iceberg Avro scan should not be supported", support instanceof 
NotSupported);
+    assertTrue(
+        "Should mention Iceberg and Avro",
+        ((NotSupported) support)
+                .getCategoryReason(NotSupportedCategory.NODE_NOT_SUPPORTED)
+                .contains("Iceberg")
+            && ((NotSupported) support)
+                .getCategoryReason(NotSupportedCategory.NODE_NOT_SUPPORTED)
+                .contains("Avro"));
+  }
+
+  @Test
+  public void testIcebergParquetWithUnsupportedTypeNotSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] IcebergScan\n"
+            + "org.apache.iceberg.spark.source.SparkBatchQueryScan@12345\n"
+            + "Format: Parquet, ReadSchema: 
struct<col1:string,col2:map<string,int>>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue(
+        "Iceberg Parquet with map type should not be supported", support 
instanceof NotSupported);
+    assertTrue(
+        "Should mention map type",
+        ((NotSupported) support)
+            .getCategoryReason(NotSupportedCategory.NODE_NOT_SUPPORTED)
+            .contains("map"));
+  }
+
+  // ==================== Delta Lake Integration Tests ====================
+
+  @Test
+  public void testDeltaLakeScanWithProviderSupported() {
+    String nodeName = "FileScan parquet";
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "io.delta.tables.DeltaTable\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Delta Lake scan with provider should be supported", support 
instanceof Supported);
+  }
+
+  @Test
+  public void testDeltaLakeScanWithLocationSupported() {
+    String nodeName = "FileScan parquet";
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: Parquet,\n"
+            + "Location: 
InMemoryFileIndex[dbfs:/user/hive/warehouse/sales/_delta_log],\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue(
+        "Delta Lake scan with _delta_log location should be supported",
+        support instanceof Supported);
+  }
+
+  @Test
+  public void testDeltaLakeScanWithUnsupportedTypeNotSupported() {
+    String nodeName = "FileScan parquet";
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "io.delta.tables.DeltaTable\n"
+            + "Format: Parquet, ReadSchema: 
struct<col1:string,nested:struct<a:int,b:int>>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue(
+        "Delta Lake scan with struct type should not be supported",
+        support instanceof NotSupported);
+    assertTrue(
+        "Should mention struct type",
+        ((NotSupported) support)
+            .getCategoryReason(NotSupportedCategory.NODE_NOT_SUPPORTED)
+            .contains("struct"));
+  }
+
+  // ==================== Hudi Integration Tests ====================
+
+  @Test
+  public void testHudiScanWithProviderSupported() {
+    String nodeName = "HoodieFileScan";
+    String nodeDesc =
+        "HoodieFileScan [col1#0, col2#1]\n"
+            + "org.apache.hudi.HoodieSparkUtils\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Hudi scan with provider should be supported", support 
instanceof Supported);
+  }
+
+  @Test
+  public void testHudiScanWithLocationSupported() {
+    String nodeName = "FileScan parquet";
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "Location: s3://bucket/table/.hoodie/metadata,\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Hudi scan with .hoodie location should be supported", support 
instanceof Supported);
+  }
+
+  // ==================== Paimon Integration Tests ====================
+
+  @Test
+  public void testPaimonParquetScanSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] PaimonScan\n"
+            + "org.apache.paimon.spark.SparkSource\n"
+            + "Format: Parquet, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Paimon Parquet scan should be supported", support instanceof 
Supported);
+  }
+
+  @Test
+  public void testPaimonOrcScanSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] PaimonScan\n"
+            + "org.apache.paimon.spark.SparkSource\n"
+            + "Format: ORC, ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Paimon ORC scan should be supported", support instanceof 
Supported);
+  }
+
+  // ==================== Raw Format Tests (No Lakehouse) ====================
+
+  @Test
+  public void testRawParquetScanSupported() {
+    String nodeName = "FileScan parquet";
+    String nodeDesc =
+        "FileScan parquet [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: Parquet,\n"
+            + "Location: s3://bucket/raw_parquet_table,\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Raw Parquet scan should be supported", support instanceof 
Supported);
+  }
+
+  @Test
+  public void testRawOrcScanSupported() {
+    String nodeName = "Scan orc";
+    String nodeDesc =
+        "FileScan orc [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: ORC,\n"
+            + "Location: hdfs://cluster/raw_orc_table,\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue("Raw ORC scan should be supported", support instanceof 
Supported);
+  }
+
+  // ==================== False Positive Prevention Tests ====================
+
+  @Test
+  public void testTableNamedDeltaSalesNotMisclassified() {
+    // A raw Parquet table with "delta" in the name should NOT be treated as 
Delta Lake
+    String nodeName = "FileScan parquet delta_sales";
+    String nodeDesc =
+        "FileScan parquet delta_sales [col1#0, col2#1]\n"
+            + "Batched: true, DataFilters: [], Format: Parquet,\n"
+            + "Location: s3://bucket/delta_sales,\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    // Should be supported as raw Parquet (not misclassified as Delta)
+    assertTrue(
+        "Table named delta_sales should be supported as raw Parquet", support 
instanceof Supported);
+  }
+
+  @Test
+  public void testGenericBatchScanExecNotMisclassified() {
+    // Generic V2 scan (e.g., Kafka) should NOT be treated as lakehouse
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1]\n"
+            + "org.apache.spark.sql.kafka010.KafkaSourceProvider\n"
+            + "Format: kafka, ReadSchema: struct<key:binary,value:binary>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    // Should NOT be supported (not a lakehouse format, unknown format)
+    assertTrue("Kafka BatchScanExec should not be supported", support 
instanceof NotSupported);
+  }
+
+  // ==================== Missing Format Field Tests ====================
+
+  @Test
+  public void testIcebergScanWithMissingFormatNotSupported() {
+    String nodeName = "BatchScanExec";
+    String nodeDesc =
+        "BatchScanExec [col1#0, col2#1] IcebergScan\n"
+            + "org.apache.iceberg.spark.source.SparkBatchQueryScan@12345\n"
+            + "ReadSchema: struct<col1:string,col2:int>";
+
+    ExecutionDescription execDesc = createSingleNodeExecution(1L, nodeName, 
nodeDesc);
+    NodeSupportVisitor visitor = new NodeSupportVisitor(execDesc);
+    ExecutionDescription result = visitor.visitAndTag();
+
+    GlutenSupport support = result.getNodeIdToGluttenSupportMap().get(1L);
+    assertTrue(
+        "Iceberg scan with missing format should not be supported",
+        support instanceof NotSupported);
+    assertTrue(
+        "Should mention UNKNOWN format",
+        ((NotSupported) support)
+            .getCategoryReason(NotSupportedCategory.NODE_NOT_SUPPORTED)
+            .contains("UNKNOWN"));
+  }
+
+  // ==================== Helper Methods ====================
+
+  /** Creates an ExecutionDescription with a single scan node for testing. */
+  private ExecutionDescription createSingleNodeExecution(
+      long nodeId, String nodeName, String nodeDesc) {
+    SparkPlanGraphNodeInternal node =
+        new SparkPlanGraphNodeInternal(nodeId, nodeName, nodeDesc, 
Collections.emptyList());
+
+    List<SparkPlanGraphNodeInternal> nodeList = List.of(node);
+    SparkPlanGraphInternal graph =
+        new SparkPlanGraphInternal(
+            nodeList, // nodes
+            Collections.emptyList(), // edges
+            nodeList // allNodes
+            );
+
+    List<MetricInternal> metrics = Collections.emptyList();
+
+    return new ExecutionDescription(graph, metrics);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to