Skip to content

Commit a79db08

Browse files
authored
Merge pull request #557 from Iskander14yo/add-engine/apache-comet
add Apache Comet
2 parents 2ab5bf9 + 4e19e8d commit a79db08

File tree

10 files changed

+343
-2
lines changed

10 files changed

+343
-2
lines changed

spark-comet/README.md

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
This README includes info on configuring Comet for ClickBench. For additional details, please refer to [Comet's docs](https://datafusion.apache.org/comet/user-guide/overview.html), [spark-accelerators README](../spark/README-accelerators.md) and [issue](https://github.com/apache/datafusion-comet/issues/2035) 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+
### Tips
8+
- To find all unsupported queries from `log.txt` (requires running bench in debug mode):
9+
```bash
10+
>>> grep -P "\[COMET:" log.txt | sed -e 's/^[ \t]*//' | sort | uniq -c
11+
12+
78 +- GlobalLimit [COMET: GlobalLimit is not supported]
13+
18 +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
14+
123 +- HashAggregate [COMET: distinct aggregates are not supported]
15+
...
16+
```
17+
- Check [here](https://datafusion.apache.org/comet/user-guide/installation.html#supported-spark-versions) for _version compatibility_ between Spark and Comet.
18+
- Check [here](https://datafusion.apache.org/comet/user-guide/installation.html#using-a-published-jar-file) for _links to Comet jar_.
19+
- Check [here](https://datafusion.apache.org/comet/user-guide/installation.html#run-spark-shell-with-comet-enabled) for _basic Comet configuration_.
20+
21+
### Configuration
22+
- Comet requires a __dedicated memory pool__, which can be allocated either through memoryOverhead or off-heap memory. The [documentation recommends](https://datafusion.apache.org/comet/user-guide/tuning.html#configuring-comet-memory-in-off-heap-mode) using off-heap memory, which is the approach used in ClickBench.
23+
Therefore, we need to split available memory between heap (for Spark) and off-heap (for Comet). For both TPC-H and TPC-DS benchmarks, Comet documentation [suggests](https://datafusion.apache.org/comet/contributor-guide/benchmarking.html) a 50/50 split. This allocation appears to be chosen to support Spark execution as well — it happens when Comet can't handle certain operators, which is also relevant for ClickBench.
24+
- `spark.driver.extraClassPath` is set to prevent Comet from failing on certain queries (check [docs](https://datafusion.apache.org/comet/user-guide/installation.html#additional-configuration) for details).
25+
- `spark.comet.regexp.allowIncompatible=True` allows Comet to use incompatible regular expressions instead of falling back to Spark (check [docs](https://datafusion.apache.org/comet/user-guide/compatibility.html#regular-expressions) for details).
26+
- `spark.comet.scan.allowIncompatible=True` allows Comet to use different parquet readers to prevent query failures (check [issue](https://github.com/apache/datafusion-comet/issues/2035#issuecomment-3090666597) for details). <br>
27+
What happens here: `hits.parquet` contains `short` type columns, so Comet uses the `native_comet` reader for compatibility. However, this reader [doesn't work](https://github.com/apache/datafusion-comet/issues/2038) for some queries (e.g., query 40), causing Comet to fail. As a workaround, we can allow Comet to produce results incompatible with Spark, enabling it to use [other readers](https://datafusion.apache.org/comet/user-guide/compatibility.html#parquet-scans) and successfully execute these queries.

spark-comet/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+
# Current differences:
6+
# - pyspark==3.5.6 version is used (latest stable for Comet 0.9.0)
7+
# - Comet 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.6 psutil
21+
22+
# Load the data
23+
24+
wget --continue --progress=dot:giga 'https://datasets.clickhouse.com/hits_compatible/hits.parquet'
25+
26+
# Install Comet
27+
28+
COMET_JAR_URL='https://repo1.maven.org/maven2/org/apache/datafusion/comet-spark-spark3.5_2.12/0.9.0/comet-spark-spark3.5_2.12-0.9.0.jar'
29+
30+
wget --continue --progress=dot:giga $COMET_JAR_URL -O comet.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+
COMET_VERSION=$(echo $COMET_JAR_URL | grep -Po ".{5}(?=.jar)")
49+
SPARK_VERSION=$(pip freeze | grep '^pyspark==' | cut -d '=' -f3)
50+
51+
mkdir -p results
52+
53+
(
54+
cat << EOF
55+
{
56+
"system": "Spark (Comet)",
57+
"date": "$(date +%Y-%m-%d)",
58+
"machine": "${MACHINE}",
59+
"cluster_size": 1,
60+
"proprietary": "no",
61+
"tuned": "no",
62+
"comment": "Using Comet ${COMET_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-comet/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-comet/query.py

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

spark-comet/results/c6a.4xlarge.json

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,57 @@
1+
{
2+
"system": "Spark (Comet)",
3+
"date": "2025-07-17",
4+
"machine": "c6a.4xlarge",
5+
"cluster_size": 1,
6+
"proprietary": "no",
7+
"tuned": "no",
8+
"comment": "Using Comet 0.9.0 with Spark 3.5.6",
9+
"tags": ["Java", "Rust", "column-oriented", "Spark derivative", "DataFusion", "Parquet"],
10+
"load_time": 0,
11+
"data_size": 14779976446,
12+
"result": [
13+
[4.472, 2.103, 1.843],
14+
[3.578, 1.073, 0.934],
15+
[6.644, 1.412, 1.175],
16+
[4.696, 2.004, 1.786],
17+
[5.631, 2.619, 2.494],
18+
[6.264, 2.870, 2.715],
19+
[5.0163, 2.275, 2.116],
20+
[3.756, 1.121, 0.928],
21+
[6.773, 2.983, 2.714],
22+
[13.568, 4.030, 3.534],
23+
[5.836, 2.355, 2.186],
24+
[5.648, 1.566, 1.407],
25+
[6.183, 3.122, 2.780],
26+
[10.864, 5.143, 4.993],
27+
[5.930, 2.934, 2.704],
28+
[5.991, 3.144, 2.790],
29+
[8.383, 4.513, 4.088],
30+
[7.609, 3.542, 3.492],
31+
[16.285, 12.08, 11.858],
32+
[4.626, 2.276, 2.057],
33+
[21.794, 3.180, 3.069],
34+
[25.802, 3.617, 3.358],
35+
[47.731, 5.476, 5.387],
36+
[112.794, 20.449, 20.447],
37+
[8.229, 2.295, 2.258],
38+
[5.093, 2.193, 2.009],
39+
[8.189, 2.316, 2.309],
40+
[22.221, 3.847, 3.526],
41+
[24.937, 17.818, 17.447],
42+
[6.512, 3.486, 3.336],
43+
[11.685, 2.552, 2.231],
44+
[21.547, 3.102, 2.872],
45+
[18.465, 8.194, 7.955],
46+
[24.087, 8.077, 7.767],
47+
[24.110, 8.102, 7.702],
48+
[6.394, 3.394, 3.071],
49+
[32.970, 4.560, 4.352],
50+
[26.427, 4.248, 4.169],
51+
[37.740, 4.908, 4.685],
52+
[null, null, null],
53+
[13.638, 1.867, 1.490],
54+
[25.261, 2.015, 1.763],
55+
[8.614, 1.839, 1.526]
56+
]
57+
}

spark-comet/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

0 commit comments

Comments
 (0)