Skip to content

Commit 7709768

Browse files
authored
Merge pull request #614 from Iskander14yo/add-engine/apache-auron
add Apache Auron
2 parents 7160e21 + b7349b4 commit 7709768

File tree

8 files changed

+267
-3
lines changed

8 files changed

+267
-3
lines changed

spark-auron/README.md

Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,37 @@
1+
This README includes info on configuring Apache Auron (formerly Blaze) for ClickBench. For additional details, please refer to [Auron's docs](https://auron.apache.org/), [spark-accelerators README](../spark/README-accelerators.md) and [issue](https://github.com/apache/auron/issues/1287) discussing the results.
2+
3+
### Run
4+
5+
As usual, benchmark can be run via `./benchmark.sh`. Additionally, users can provide machine spec like `./benchmark.sh c6a.8xlarge` so script saves it in relevant file.
6+
7+
## Notes
8+
9+
### Debug
10+
11+
- To find all unsupported queries from `log.txt`:
12+
```
13+
>>> grep -o 'expressions.*' log.txt | grep -v 'toprettystring' | grep -o ' .*' | sort | uniq -c
14+
15+
45 cast(EventTime#4L as timestamp)
16+
12 cast(none#0L as timestamp)
17+
153 date_add(1970-01-01, EventDate#5)
18+
72 date_add(1970-01-01, none#0)
19+
24 date_add(1970-01-01, none#1)
20+
15 date_trunc(class org.apache.spark.sql.auron.NativeExprWrapper() dataType:StringType), ...)
21+
15 minute(class org.apache.spark.sql.auron.NativeExprWrapper() dataType:TimestampType), ...)
22+
9 regexp_replace(Referer#14, ^https?://(?:www.)?([^/]+)/.*$, $1, 1)
23+
6 regexp_replace(none#0, ^https?://(?:www.)?([^/]+)/.*$, $1, 1)
24+
```
25+
26+
### Links
27+
28+
- Refer to Auron's [`pom.xml`](https://github.com/apache/auron/blob/v5.0.0/pom.xml) for exact _version compatibility_ between Auron, Spark, Scala, and Java.
29+
- Download _pre-built JARs_ from the [Auron archives](https://auron.apache.org/archives).
30+
- View an example _Auron configuration_ in the [benchmarks documentation](https://auron.apache.org/documents/benchmarks.html#benchmark-configuration).
31+
32+
### Configuration
33+
34+
- As of version 5.0, Spark 3.5.5 is chosen since it's used for the `spark-3.5` shim (see `pom.xml`) and TPC-DS testing.
35+
- Apache Auron was previously named [Blaze](https://github.com/apache/auron/issues/1168). This change occurred after version 5.0, so previous naming references (links, settings) still remain. These will be updated in the next version.
36+
- In version 5.0, Auron generates extensive INFO logs (~55MB file after ~40 queries), which may impact system performance. This behavior will be manageable in next version and will require setting `spark.auron.native.log.level`.
37+
- Auron's memory configuration follows the example from the [benchmark page](https://auron.apache.org/documents/benchmarks.html#benchmark-configuration).

spark-auron/benchmark.sh

Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,90 @@
1+
#!/bin/bash
2+
3+
# Note: Keep in sync with spark-*/benchmark.sh (see README-accelerators.md for details)
4+
#
5+
# Highlights:
6+
# - pyspark==3.5.6 version is used (latest stable for Auron 5.0.0)
7+
# - Auron installation is added
8+
# - auto-save results
9+
10+
# Install
11+
12+
sudo apt-get update -y
13+
sudo apt-get install -y python3-pip python3-venv openjdk-17-jdk
14+
15+
export JAVA_HOME="/usr/lib/jvm/java-17-openjdk-$(dpkg --print-architecture)/"
16+
export PATH=$JAVA_HOME/bin:$PATH
17+
18+
python3 -m venv myenv
19+
source myenv/bin/activate
20+
pip install pyspark==3.5.5 psutil
21+
22+
# Load the data
23+
24+
wget --continue --progress=dot:giga 'https://datasets.clickhouse.com/hits_compatible/hits.parquet'
25+
26+
# Install Auron
27+
28+
AURON_JAR_URL='https://github.com/apache/auron/releases/download/v5.0.0/blaze-engine-spark-3.5-release-5.0.0-SNAPSHOT.jar'
29+
30+
wget --continue --progress=dot:giga $AURON_JAR_URL -O auron.jar
31+
32+
# Run the queries
33+
34+
./run.sh 2>&1 | tee log.txt
35+
36+
# Print results to stdout as required
37+
cat log.txt | grep -P '^Time:\s+([\d\.]+)|Failure!' | sed -r -e 's/Time: //; s/^Failure!$/null/' |
38+
awk '{ if (i % 3 == 0) { printf "[" }; printf $1; if (i % 3 != 2) { printf "," } else { print "]," }; ++i; }'
39+
40+
DATA_SIZE=$(du -b hits.parquet | cut -f1)
41+
42+
echo "Data size: $DATA_SIZE"
43+
echo "Load time: 0"
44+
45+
# Save results as JSON
46+
47+
MACHINE="${1:-c6a.4xlarge}" # Use first argument as machine name, default to c6a.4xlarge
48+
AURON_VERSION=$(echo $AURON_JAR_URL | grep -Po "\d.\d.\d" | head -n 1)
49+
SPARK_VERSION=$(pip freeze | grep '^pyspark==' | cut -d '=' -f3)
50+
51+
mkdir -p results
52+
53+
(
54+
cat << EOF
55+
{
56+
"system": "Spark (Auron)",
57+
"date": "$(date +%Y-%m-%d)",
58+
"machine": "${MACHINE}",
59+
"cluster_size": 1,
60+
"proprietary": "no",
61+
"tuned": "no",
62+
"comment": "Using Auron ${AURON_VERSION} with Spark ${SPARK_VERSION}",
63+
"tags": ["Java", "Rust", "column-oriented", "Spark derivative", "DataFusion", "Parquet"],
64+
"load_time": 0,
65+
"data_size": ${DATA_SIZE},
66+
"result": [
67+
EOF
68+
69+
cat log.txt | grep -P '^Time:\s+([\d\.]+)|Failure!' | sed -r -e 's/Time: //; s/^Failure!$/null/' |
70+
awk -v total=$(grep -cP '^Time:\s+[\d\.]+|Failure!' log.txt) '
71+
{
72+
if (i % 3 == 0) printf "\t\t[";
73+
if ($1 == "null") printf "null";
74+
else printf "%.3f", $1;
75+
if (i % 3 != 2) printf ", ";
76+
else {
77+
if (i < total - 1) printf "],\n";
78+
else printf "]";
79+
}
80+
i++;
81+
}'
82+
83+
cat << EOF
84+
85+
]
86+
}
87+
EOF
88+
) > "results/${MACHINE}.json"
89+
90+
echo "Results have been saved to results/${MACHINE}.json"

spark-auron/queries.sql

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,43 @@
1+
SELECT COUNT(*) FROM hits;
2+
SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0;
3+
SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits;
4+
SELECT AVG(UserID) FROM hits;
5+
SELECT COUNT(DISTINCT UserID) FROM hits;
6+
SELECT COUNT(DISTINCT SearchPhrase) FROM hits;
7+
SELECT MIN(EventDate), MAX(EventDate) FROM hits;
8+
SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC;
9+
SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10;
10+
SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10;
11+
SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;
12+
SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;
13+
SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
14+
SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;
15+
SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;
16+
SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10;
17+
SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10;
18+
SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10;
19+
SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10;
20+
SELECT UserID FROM hits WHERE UserID = 435090932899640449;
21+
SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%';
22+
SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
23+
SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
24+
SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10;
25+
SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10;
26+
SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10;
27+
SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10;
28+
SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25;
29+
SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25;
30+
SELECT SUM(ResolutionWidth), SUM(ResolutionWidth + 1), SUM(ResolutionWidth + 2), SUM(ResolutionWidth + 3), SUM(ResolutionWidth + 4), SUM(ResolutionWidth + 5), SUM(ResolutionWidth + 6), SUM(ResolutionWidth + 7), SUM(ResolutionWidth + 8), SUM(ResolutionWidth + 9), SUM(ResolutionWidth + 10), SUM(ResolutionWidth + 11), SUM(ResolutionWidth + 12), SUM(ResolutionWidth + 13), SUM(ResolutionWidth + 14), SUM(ResolutionWidth + 15), SUM(ResolutionWidth + 16), SUM(ResolutionWidth + 17), SUM(ResolutionWidth + 18), SUM(ResolutionWidth + 19), SUM(ResolutionWidth + 20), SUM(ResolutionWidth + 21), SUM(ResolutionWidth + 22), SUM(ResolutionWidth + 23), SUM(ResolutionWidth + 24), SUM(ResolutionWidth + 25), SUM(ResolutionWidth + 26), SUM(ResolutionWidth + 27), SUM(ResolutionWidth + 28), SUM(ResolutionWidth + 29), SUM(ResolutionWidth + 30), SUM(ResolutionWidth + 31), SUM(ResolutionWidth + 32), SUM(ResolutionWidth + 33), SUM(ResolutionWidth + 34), SUM(ResolutionWidth + 35), SUM(ResolutionWidth + 36), SUM(ResolutionWidth + 37), SUM(ResolutionWidth + 38), SUM(ResolutionWidth + 39), SUM(ResolutionWidth + 40), SUM(ResolutionWidth + 41), SUM(ResolutionWidth + 42), SUM(ResolutionWidth + 43), SUM(ResolutionWidth + 44), SUM(ResolutionWidth + 45), SUM(ResolutionWidth + 46), SUM(ResolutionWidth + 47), SUM(ResolutionWidth + 48), SUM(ResolutionWidth + 49), SUM(ResolutionWidth + 50), SUM(ResolutionWidth + 51), SUM(ResolutionWidth + 52), SUM(ResolutionWidth + 53), SUM(ResolutionWidth + 54), SUM(ResolutionWidth + 55), SUM(ResolutionWidth + 56), SUM(ResolutionWidth + 57), SUM(ResolutionWidth + 58), SUM(ResolutionWidth + 59), SUM(ResolutionWidth + 60), SUM(ResolutionWidth + 61), SUM(ResolutionWidth + 62), SUM(ResolutionWidth + 63), SUM(ResolutionWidth + 64), SUM(ResolutionWidth + 65), SUM(ResolutionWidth + 66), SUM(ResolutionWidth + 67), SUM(ResolutionWidth + 68), SUM(ResolutionWidth + 69), SUM(ResolutionWidth + 70), SUM(ResolutionWidth + 71), SUM(ResolutionWidth + 72), SUM(ResolutionWidth + 73), SUM(ResolutionWidth + 74), SUM(ResolutionWidth + 75), SUM(ResolutionWidth + 76), SUM(ResolutionWidth + 77), SUM(ResolutionWidth + 78), SUM(ResolutionWidth + 79), SUM(ResolutionWidth + 80), SUM(ResolutionWidth + 81), SUM(ResolutionWidth + 82), SUM(ResolutionWidth + 83), SUM(ResolutionWidth + 84), SUM(ResolutionWidth + 85), SUM(ResolutionWidth + 86), SUM(ResolutionWidth + 87), SUM(ResolutionWidth + 88), SUM(ResolutionWidth + 89) FROM hits;
31+
SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;
32+
SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;
33+
SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;
34+
SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10;
35+
SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10;
36+
SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;
37+
SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;
38+
SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;
39+
SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000;
40+
SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END AS Src, URL AS Dst, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000;
41+
SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100;
42+
SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000;
43+
SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000;

spark-auron/query.py

Lines changed: 66 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,66 @@
1+
#!/usr/bin/env python3
2+
3+
"""
4+
Note: Keep in sync with spark-*/query.py (see README-accelerators.md for details)
5+
6+
Highlights:
7+
- memory is split between heap (for Spark) and memoryOverhead (for Auron)
8+
- Auron configuration is added to `SparkSession`
9+
"""
10+
11+
from pyspark.sql import SparkSession
12+
import pyspark.sql.functions as F
13+
14+
import psutil
15+
import re
16+
import sys
17+
import timeit
18+
19+
20+
query = sys.stdin.read()
21+
# Replace \1 to $1 because spark recognizes only this pattern style (in query 28)
22+
query = re.sub(r"""(REGEXP_REPLACE\(.*?,\s*('[^']*')\s*,\s*)('1')""", r"\1'$1'", query)
23+
print(query)
24+
25+
# Calculate available memory to configurate SparkSession
26+
ram = int(round(psutil.virtual_memory().available / (1024 ** 3) * 0.7))
27+
exec_memory = ram // 3
28+
exec_overhead = ram - exec_memory
29+
print(f"SparkSession will be set with {exec_memory} GB of memory and {exec_overhead} GB of memory overhead")
30+
31+
builder = (
32+
SparkSession
33+
.builder
34+
.appName("ClickBench")
35+
.config("spark.driver", "local[*]") # To ensure using all cores
36+
.config("spark.driver.memory", f"{exec_memory}g") # Set amount of memory SparkSession can use
37+
.config("spark.sql.parquet.binaryAsString", True) # Treat binary as string to get correct length calculations and text results
38+
39+
# Additional Auron configuration
40+
.config("spark.jars", "auron.jar")
41+
.config("spark.driver.extraClassPath", "auron.jar")
42+
.config("spark.auron.enable", True)
43+
.config("spark.sql.extensions", "org.apache.spark.sql.blaze.BlazeSparkSessionExtension")
44+
.config("spark.shuffle.manager", "org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager")
45+
.config("spark.memory.offHeap.enabled", "false")
46+
.config("spark.driver.memoryOverhead", exec_overhead * 1024)
47+
)
48+
49+
spark = builder.getOrCreate()
50+
51+
df = spark.read.parquet("hits.parquet")
52+
# Do casting before creating the view so no need to change to unreadable integer dates in SQL
53+
df = df.withColumn("EventTime", F.col("EventTime").cast("timestamp"))
54+
df = df.withColumn("EventDate", F.date_add(F.lit("1970-01-01"), F.col("EventDate")))
55+
df.createOrReplaceTempView("hits")
56+
57+
for try_num in range(3):
58+
try:
59+
start = timeit.default_timer()
60+
result = spark.sql(query)
61+
result.show(100) # some queries should return more than 20 rows which is the default show limit
62+
end = timeit.default_timer()
63+
print("Time: ", end - start)
64+
except Exception as e:
65+
print(e)
66+
print("Failure!")

spark-auron/run.sh

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,10 @@
1+
#!/bin/bash
2+
3+
# Note: Keep in sync with spark-*/run.sh (see README-accelerators.md for details)
4+
5+
cat queries.sql | while read query; do
6+
sync
7+
echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null
8+
9+
./query.py <<< "${query}"
10+
done

spark-auron/template.json

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,11 @@
1+
{
2+
"system": "Spark (Auron)",
3+
"proprietary": "no",
4+
"tuned": "no",
5+
"tags": [
6+
"Java",
7+
"Rust",
8+
"column-oriented",
9+
"Spark derivative"
10+
]
11+
}

spark-comet/README.md

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,9 @@ This README includes info on configuring Apache Comet for ClickBench. For additi
44

55
As usual, benchmark can be run via `./benchmark.sh`. Additionally, users can provide machine spec like `./benchmark.sh c6a.8xlarge` so script saves it in relevant file.
66

7-
### Notes
7+
## Notes
8+
9+
### Debug
810

911
- To find all unsupported queries from `log.txt` (requires running bench in debug mode):
1012
```bash
@@ -15,6 +17,9 @@ As usual, benchmark can be run via `./benchmark.sh`. Additionally, users can pro
1517
123 +- HashAggregate [COMET: distinct aggregates are not supported]
1618
...
1719
```
20+
21+
### Links
22+
1823
- Check [here](https://datafusion.apache.org/comet/user-guide/installation.html#supported-spark-versions) for _version compatibility_ between Spark and Comet.
1924
- Check [here](https://datafusion.apache.org/comet/user-guide/installation.html#using-a-published-jar-file) for _links to Comet jar_.
2025
- Check [here](https://datafusion.apache.org/comet/user-guide/installation.html#run-spark-shell-with-comet-enabled) for _basic Comet configuration_.

spark-gluten/README.md

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,10 +1,12 @@
1-
This README includes info on configuring Apache Gluten for ClickBench. For additional details, please refer to [Gluten's docs](https://apache.github.io/incubator-gluten/get-started/Velox.html) and [spark-accelerators README](../spark/README-accelerators.md).
1+
This README includes info on configuring Apache Gluten for ClickBench. For additional details, please refer to [Gluten's docs](https://apache.github.io/incubator-gluten/get-started/Velox.html), [spark-accelerators README](../spark/README-accelerators.md) and [discussion](https://github.com/apache/incubator-gluten/discussions/10465).
22

33
### Run
44

55
As usual, benchmark can be run via `./benchmark.sh`. Additionally, users can provide machine spec like `./benchmark.sh c6a.8xlarge` so script saves it in relevant file.
66

7-
### Notes
7+
## Notes
8+
9+
### Links
810

911
- Check [here](https://gluten.apache.org/downloads/) for _pre-built jars_.
1012
- Check [here](https://gluten.apache.org/#3-how-to-use) and [here](https://apache.github.io/incubator-gluten/get-started/Velox.html#submit-the-spark-sql-job) for _examples of Gluten configuration_.

0 commit comments

Comments
 (0)