Skip to content

Add Apache Gluten/Velox support to DataFlint UI#59

Draft
menishmueli wants to merge 3 commits into
mainfrom
feature/gluten-velox-support
Draft

Add Apache Gluten/Velox support to DataFlint UI#59
menishmueli wants to merge 3 commits into
mainfrom
feature/gluten-velox-support

Conversation

@menishmueli
Copy link
Copy Markdown
Contributor

Summary

  • Add full Gluten/Velox node type support to the DataFlint SQL plan flow view: node classification, friendly display names, Velox-native metrics (aggregation/filter/sort/window time, peak memory, spill), and accelerator badges (Velox, Photon, RAPIDS, DataFusion)
  • Fix stage identification for Gluten's WholeStageCodegenTransformer by inferring codegen-to-node mapping from node ID ordering, handling AQE codegen renumbering, and propagating stages through Gluten-specific boundary nodes
  • Add Docker environment and Scala example app for running Gluten/Velox on Spark 3.5 with DataFlint, including automation script

Test plan

  • 93 UI unit tests pass (17 suites), including new GlutenStageAssignment.spec.ts with real fixture data
  • Visually verify all 16 SQL queries at localhost:4000 with the Gluten example app running in Docker
  • Verify exchange nodes split correctly between stages (write/read)
  • Verify Window nodes show partition-by and sort-by columns
  • Verify accelerator badges appear on Velox nodes and not on standard Spark nodes
  • Verify no regressions on standard (non-Gluten) Spark plans

- Add Gluten/Velox node type classification, display names, and accelerator
  badges (Velox, Photon, RAPIDS, DataFusion) in the SQL plan flow view
- Fix stage identification for Gluten's WholeStageCodegenTransformer nodes
  by inferring codegen-to-node mapping and handling AQE codegen renumbering
- Split ColumnarExchange into write/read visual nodes across stage boundaries
- Propagate stages through Gluten-specific boundary nodes (VeloxResizeBatches,
  RowToVeloxColumnar, TakeOrderedAndProjectExecTransformer, etc.)
- Show Velox native timing metrics (aggregation/filter/sort/window time,
  peak memory, spill) on plan nodes
- Strip Gluten class name prefixes from plan descriptions in parsers
- Add Docker environment and example app for running Gluten/Velox on Spark 3.5
- Add unit test for Gluten stage assignment with real fixture data
@CLAassistant
Copy link
Copy Markdown

CLAassistant commented Apr 11, 2026

CLA assistant check
Thank you for your submission! We really appreciate it. Like many open source projects, we ask that you all sign our Contributor License Agreement before we can accept your contribution.
1 out of 2 committers have signed the CLA.

✅ minskya
❌ menishmueli
You have signed the CLA already but the status is still pending? Let us recheck it.

@menishmueli
Copy link
Copy Markdown
Contributor Author

Screenshots

Status Page — GlutenVeloxExample with 16 SQL Queries

The DataFlint status page correctly identifies the Gluten/Velox application.

GroupBy Aggregation (SQL 3) — Two stages with split exchange

Shows Stage 5 (write side) and Stage 7 (read side) with the ColumnarExchange visually split. All Velox-accelerated nodes have orange Velox badges. Native metrics like Aggregation Time and Peak Memory are displayed.

Window Functions (SQL 10) — Three stages with window partition fields

Shows Stage 47, Stage 49, and Stage 52 with two split ColumnarExchange nodes. Window nodes display Partition Fields (play_name), Sort Fields, and Select Fields parsed from the physical plan.


Note: To view live screenshots, run ./docker/gluten/run-gluten-example.sh and visit http://localhost:4000

menishmueli and others added 2 commits April 13, 2026 21:33
- Add CometExchange, CometColumnarExchange, GpuColumnarExchange to
  exchange visual split, stage assignment, and shuffle metrics calculation
- Add CometHashAggregate to aggregate node parsing and naming
- Support Comet plan description format (Keys:/Functions:) in parser
- Re-add fallback plan description parsing from SQL-level planDescription
  for native engines where DataFlint custom endpoint returns empty
…support

# Conflicts:
#	spark-ui/src/reducers/PlanParsers/WindowParser.ts
#	spark-ui/src/reducers/SQLNodeStageReducer.ts
#	spark-ui/src/reducers/SqlReducer.ts
#	spark-ui/src/reducers/SqlReducerUtils.ts
Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Adds Apache Gluten/Velox (and related accelerators) support to the DataFlint UI’s SQL plan flow view, including new node classifications/labels, metrics, accelerator badges, and stage-assignment improvements, plus a runnable Docker + example app environment for validation.

Changes:

  • Extend SQL node typing/renaming and metric handling to cover Gluten/Velox operators and Velox-native metrics.
  • Improve stage identification for Gluten WholeStageCodegenTransformer and broaden exchange/stage-boundary handling.
  • Add UI accelerator badges and a Docker + Scala example app for running/validating Gluten/Velox plans.

Reviewed changes

Copilot reviewed 17 out of 17 changed files in this pull request and generated 3 comments.

Show a summary per file
File Description
spark-ui/src/reducers/SqlReducerUtils.ts Add Velox/Gluten node mappings, metric allowlists/transformers/renames, exchange/aggregate lists, and accelerator lookup.
spark-ui/src/reducers/SqlReducer.ts Add fallback plan-description parsing, Gluten codegen-id inference, and extend exchange/broadcast metric handling.
spark-ui/src/reducers/PlanParsers/ProjectParser.ts Support parsing ProjectExecTransformer.
spark-ui/src/reducers/PlanParsers/hashAggregateParser.ts Support alternate Keys: / Functions [...] formats.
spark-ui/src/reducers/PlanParsers/FilterParser.ts Support parsing FilterExecTransformer.
spark-ui/src/reducers/PlanParsers/ExchangeParser.ts Broaden exchange type parsing to include Columnar/Broadcast exchange variants.
spark-ui/src/reducers/PlanGraphUtils.ts Extend “skip-through” node list for input-row backtracking with Velox/Gluten node names.
spark-ui/src/reducers/tests/GlutenStageAssignment.spec.ts New test validating stage assignment with a real Gluten fixture.
spark-ui/src/reducers/tests/gluten-sql4-fixture.json New fixture data for Gluten stage assignment test.
spark-ui/src/components/SqlFlow/StageNode.tsx Show accelerator badge (Velox/Photon/RAPIDS/DataFusion) on nodes.
spark-ui/src/components/SqlFlow/SqlLayoutService.ts Treat Columnar/Comet/GPU exchanges as split exchange nodes in layout.
spark-plugin/example_3_5_1/src/main/scala/io/dataflint/example/GlutenVeloxExample.scala New Scala example app exercising common operators under Gluten/Velox.
spark-plugin/example_3_5_1/src/main/scala/io/dataflint/example/DataFusionCometExample.scala Comment out a hardcoded local-path example line.
docker/gluten/run-gluten-example.sh Automation script to build artifacts, download Gluten bundle, and run Docker.
docker/gluten/Dockerfile New container image to run Spark + Gluten/Velox + DataFlint example.
docker/gluten/docker-compose.yml Compose definition for running the Gluten example container.
docker/gluten/.gitignore Ignore generated jars/test data/event logs for the Docker example.
Comments suppressed due to low confidence (1)

spark-ui/src/reducers/SqlReducer.ts:824

  • In calcBroadcastExchangeDuration, the time to build / time to collect durations are currently not added to duration because they’re on a separate statement starting with unary +, so the function always returns only time to broadcast. Combine these into a single summed expression (or duration += ...) so the returned duration reflects all broadcast phases.
  if (nodeName === "BroadcastExchange" || nodeName === "ColumnarBroadcastExchange") {
    const duration = getMetricDuration("time to broadcast", metrics) ?? 0;
    +(getMetricDuration("time to build", metrics) ?? 0) +
      (getMetricDuration("time to collect", metrics) ?? 0);
    return duration;

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Comment on lines +281 to +288
for (const line of lines) {
const headerMatch = line.match(/^\((\d+)\)\s+(\S+)/);
if (headerMatch) {
if (currentName !== undefined && currentBody.length > 0) {
sections.push({ name: currentName, body: currentBody.join(" ") });
}
currentName = headerMatch[2];
currentBody = [];
Comment on lines +115 to +124
// Debug: print all node stages
for (const node of result.nodes) {
const stageInfo = node.stage
? node.stage.type === "onestage" ? `onestage:${(node.stage as any).stageId}`
: node.stage.type === "exchange" ? `exchange:w=${(node.stage as any).writeStage},r=${(node.stage as any).readStage}`
: `${node.stage.type}`
: "NONE";
console.log(` Node ${node.nodeId}: ${node.nodeName.padEnd(45)} stage=${stageInfo} wcid=${node.wholeStageCodegenId}`);
}

Comment thread docker/gluten/Dockerfile
Comment on lines +45 to +46
echo "spark.driver.extraJavaOptions=--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.misc=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED" >> /opt/spark/conf/spark-defaults.conf && \
echo "spark.executor.extraJavaOptions=--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.misc=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED" >> /opt/spark/conf/spark-defaults.conf
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants