From b5ed4bf8df1a9ec8380c7943addd5203da2e8e12 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 20 Mar 2025 13:10:57 +0100 Subject: [PATCH 1/8] squash merge in the benchmarking scripts --- Cargo.lock | 2 +- Cargo.toml | 2 +- datafusion_ray/core.py | 44 +-- docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge | 61 ++++ docs/benchmarks/df-ray-0.1.0rc1/m7g.12x | 54 +++ docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x | 54 +++ docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge | 107 ++++++ k8s/README.md | 47 +++ k8s/bench_toolbox.py | 364 ++++++++++++++++++++ k8s/cmds.py | 280 +++++++++++++++ k8s/machine_prep.sh | 29 ++ k8s/pricing.py | 123 +++++++ k8s/pvcs.yaml.template | 31 ++ k8s/ray_cluster.yaml.template | 57 +++ k8s/ray_job.sh.template | 9 + k8s/requirements.txt.template | 6 + k8s/spark_job.yaml.template | 56 +++ k8s/spark_tpcbench.py | 145 ++++++++ tpch/make_data.py | 27 +- tpch/tpcbench.py | 39 ++- 20 files changed, 1499 insertions(+), 38 deletions(-) create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/m7g.12x create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge create mode 100644 k8s/README.md create mode 100755 k8s/bench_toolbox.py create mode 100644 k8s/cmds.py create mode 100644 k8s/machine_prep.sh create mode 100644 k8s/pricing.py create mode 100644 k8s/pvcs.yaml.template create mode 100644 k8s/ray_cluster.yaml.template create mode 100644 k8s/ray_job.sh.template create mode 100644 k8s/requirements.txt.template create mode 100644 k8s/spark_job.yaml.template create mode 100644 k8s/spark_tpcbench.py diff --git a/Cargo.lock b/Cargo.lock index e5c5ffb..287b5ae 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1604,7 +1604,7 @@ dependencies = [ [[package]] name = "datafusion_ray" -version = "0.1.0" +version = "0.1.0-rc1" dependencies = [ "anyhow", "arrow", diff --git a/Cargo.toml b/Cargo.toml index 9d9b659..651d044 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -21,7 +21,7 @@ description = "DataFusion on Ray" homepage = "https://github.com/apache/datafusion-ray" repository = "https://github.com/apache/datafusion-ray" authors = ["Apache DataFusion "] -version = "0.1.0" +version = "0.1.0-rc1" edition = "2024" readme = "README.md" license = "Apache-2.0" diff --git a/datafusion_ray/core.py b/datafusion_ray/core.py index 4832c9f..291c6dd 100644 --- a/datafusion_ray/core.py +++ b/datafusion_ray/core.py @@ -108,9 +108,9 @@ class DFRayProcessorPool: # # This is simple though and will suffice for now - def __init__(self, min_workers: int, max_workers: int): - self.min_workers = min_workers - self.max_workers = max_workers + def __init__(self, min_processors: int, max_processors: int): + self.min_processors = min_processors + self.max_processors = max_processors # a map of processor_key (a random identifier) to stage actor reference self.pool = {} @@ -137,11 +137,11 @@ def __init__(self, min_workers: int, max_workers: int): # processors available self.available = set() - for _ in range(min_workers): + for _ in range(min_processors): self._new_processor() log.info( - f"created ray processor pool (min_workers: {min_workers}, max_workers: {max_workers})" + f"created ray processor pool (min_processors: {min_processors}, max_processors: {max_processors})" ) async def start(self): @@ -159,12 +159,12 @@ async def acquire(self, need=1): have = len(self.available) total = len(self.available) + len(self.acquired) - can_make = self.max_workers - total + can_make = self.max_processors - total need_to_make = need - have if need_to_make > can_make: - raise Exception(f"Cannot allocate workers above {self.max_workers}") + raise Exception(f"Cannot allocate processors above {self.max_processors}") if need_to_make > 0: log.debug(f"creating {need_to_make} additional processors") @@ -321,11 +321,13 @@ def __str__(self): class DFRayContextSupervisor: def __init__( self, - worker_pool_min: int, - worker_pool_max: int, + processor_pool_min: int, + processor_pool_max: int, ) -> None: - log.info(f"Creating DFRayContextSupervisor worker_pool_min: {worker_pool_min}") - self.pool = DFRayProcessorPool(worker_pool_min, worker_pool_max) + log.info( + f"Creating DFRayContextSupervisor processor_pool_min: {processor_pool_min}" + ) + self.pool = DFRayProcessorPool(processor_pool_min, processor_pool_max) self.stages: dict[str, InternalStageData] = {} log.info("Created DFRayContextSupervisor") @@ -452,7 +454,7 @@ def __init__( internal_df: DFRayDataFrameInternal, supervisor, # ray.actor.ActorHandle[DFRayContextSupervisor], batch_size=8192, - partitions_per_worker: int | None = None, + partitions_per_processor: int | None = None, prefetch_buffer_size=0, ): self.df = internal_df @@ -460,7 +462,7 @@ def __init__( self._stages = None self._batches = None self.batch_size = batch_size - self.partitions_per_worker = partitions_per_worker + self.partitions_per_processor = partitions_per_processor self.prefetch_buffer_size = prefetch_buffer_size def stages(self): @@ -469,7 +471,7 @@ def stages(self): self._stages = self.df.stages( self.batch_size, self.prefetch_buffer_size, - self.partitions_per_worker, + self.partitions_per_processor, ) return self._stages @@ -541,20 +543,20 @@ def __init__( self, batch_size: int = 8192, prefetch_buffer_size: int = 0, - partitions_per_worker: int | None = None, - worker_pool_min: int = 1, - worker_pool_max: int = 100, + partitions_per_processor: int | None = None, + processor_pool_min: int = 1, + processor_pool_max: int = 100, ) -> None: self.ctx = DFRayContextInternal() self.batch_size = batch_size - self.partitions_per_worker = partitions_per_worker + self.partitions_per_processor = partitions_per_processor self.prefetch_buffer_size = prefetch_buffer_size self.supervisor = DFRayContextSupervisor.options( name="RayContextSupersisor", ).remote( - worker_pool_min, - worker_pool_max, + processor_pool_min, + processor_pool_max, ) # start up our super visor and don't check in on it until its @@ -629,7 +631,7 @@ def sql(self, query: str) -> DFRayDataFrame: df, self.supervisor, self.batch_size, - self.partitions_per_worker, + self.partitions_per_processor, self.prefetch_buffer_size, ) diff --git a/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge b/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge new file mode 100644 index 0000000..7044aca --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge @@ -0,0 +1,61 @@ +================================================================================================= +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: i4g.4xlarge +Machine On Demand Cost: 1.23552 $/hr +CPU(s): Neoverse-N1 16x +MEM: 126.8G +HD Throughput: 888.24 MB/s (from hdparm) +Data Location: /data/sf100 + +df-ray duration: 870.95s +1.04x faster +df-ray cost: $0.2989 +1.04x cheaper + +spark duration: 906.69s +spark cost: $0.3112 + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 4 +Ray Workers: 4 +Ray Worker Mem (GB): 24 +Ray Worker CPU: 3 +Ray Head Mem (GB): 4 +Ray Head CPU: 2 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 20 +Executor Overhead Mem (GB): 4 +Executor CPU: 3 +Driver Mem(GB): 4 +Driver CPU: 2 +================================================================================================= +DataFrame() ++------------+--------------------+--------------------+-----------------------+---------------+ +| tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | ++------------+--------------------+--------------------+-----------------------+---------------+ +| 1 | 107.42937994003296 | 40.58694672584534 | 0.3778011820276814 | +2.65x faster | +| 2 | 14.26254153251648 | 14.748653411865234 | 1.0340831175313665 | 1.03x slower | +| 3 | 34.457966566085815 | 23.512167930603027 | 0.6823434541765488 | +1.47x faster | +| 4 | 24.979174375534058 | 12.797027349472046 | 0.5123078592223664 | +1.95x faster | +| 5 | 57.75347018241882 | 46.364267349243164 | 0.8027962164489514 | +1.25x faster | +| 6 | 6.254350423812866 | 20.64115333557129 | 3.3002873099310186 | 3.30x slower | +| 7 | 21.208117723464966 | 67.15978837013245 | 3.1667019792060986 | 3.17x slower | +| 8 | 56.11615228652954 | 52.393213748931885 | 0.9336565607957527 | +1.07x faster | +| 9 | 108.6072256565094 | 72.31336116790771 | 0.6658245869995078 | +1.50x faster | +| 10 | 28.727190732955933 | 40.29949235916138 | 1.4028344342396717 | 1.40x slower | +| 11 | 4.468253135681152 | 13.295583248138428 | 2.9755662547331516 | 2.98x slower | +| 12 | 18.66551184654236 | 15.770261764526367 | 0.8448877209572845 | +1.18x faster | +| 13 | 38.409486532211304 | 19.281179428100586 | 0.5019900334239259 | +1.99x faster | +| 14 | 8.859760522842407 | 12.571092128753662 | 1.4188975081598003 | 1.42x slower | +| 15 | 23.616262197494507 | 26.80106806755066 | 1.1348564748910195 | 1.13x slower | +| 16 | 11.123925685882568 | 12.223425388336182 | 1.098840978760672 | 1.10x slower | +| 17 | 119.728675365448 | 82.61100387573242 | 0.6899851152915435 | +1.45x faster | +| 18 | 88.93902206420898 | 116.58043599128723 | 1.3107906213216816 | 1.31x slower | +| 19 | 13.113885879516602 | 34.65487003326416 | 2.642608785195681 | 2.64x slower | +| 20 | 16.772423267364502 | 24.642067432403564 | 1.4692013813145106 | 1.47x slower | +| 21 | 88.84716391563416 | 112.72514128684998 | 1.2687533998709224 | 1.27x slower | +| 22 | 14.35309910774231 | 8.979067087173462 | 0.6255838561255387 | +1.60x faster | +| total | 906.6930389404297 | 870.9512674808502 | 0.9605800751472101 | +1.04x faster | ++------------+--------------------+--------------------+-----------------------+---------------+ diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x new file mode 100644 index 0000000..19ffb1d --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x @@ -0,0 +1,54 @@ +========================================================================================== +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: m7g.12xlarge +CPU(s): 48x +MEM: 189G +HD Throughput: 167.53 MB/s (from hdparm) + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 2 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +========================================================================================== +DataFrame() ++------------+--------------------+--------------------+---------------------+----------------+ +| tpch_query | spark | df_ray | change | change_text | ++------------+--------------------+--------------------+---------------------+----------------+ +| 1 | 40.66942858695984 | 7.127374172210693 | 0.1752513969300272 | +5.71x faster | +| 2 | 5.27202296257019 | 4.556609869003296 | 0.8643000801312671 | +1.16x faster | +| 3 | 9.467089414596558 | 6.217605829238892 | 0.6567600195739628 | +1.52x faster | +| 4 | 6.713418006896973 | 2.6009602546691895 | 0.3874271275819732 | +2.58x faster | +| 5 | 138.02316689491272 | 14.755179166793823 | 0.10690364160407967 | +9.35x faster | +| 6 | 1.7651944160461426 | 4.644023895263672 | 2.630885217542109 | 2.63x slower | +| 7 | 7.105847120285034 | 12.337952613830566 | 1.7363098874741423 | 1.74x slower | +| 8 | 234.60386061668396 | 8.625483512878418 | 0.03676616186198009 | +27.20x faster | +| 9 | 308.04562306404114 | 15.377676725387573 | 0.04992012732539502 | +20.03x faster | +| 10 | 99.11399936676025 | 12.474991083145142 | 0.12586507620364337 | +7.95x faster | +| 11 | 2.5937740802764893 | 2.8647923469543457 | 1.104488000222813 | 1.10x slower | +| 12 | 22.38423728942871 | 4.3449485301971436 | 0.1941075085122114 | +5.15x faster | +| 13 | 31.56517219543457 | 3.571704864501953 | 0.11315334642839511 | +8.84x faster | +| 14 | 16.334258317947388 | 3.2847304344177246 | 0.20109455663552245 | +4.97x faster | +| 15 | 11.267093658447266 | 5.8551459312438965 | 0.5196678139667477 | +1.92x faster | +| 16 | 6.11682653427124 | 2.544823408126831 | 0.4160365499771397 | +2.40x faster | +| 17 | 139.6458387374878 | 17.490234375 | 0.12524708600790382 | +7.98x faster | +| 18 | 68.82194375991821 | 29.644872903823853 | 0.4307473936981271 | +2.32x faster | +| 19 | 11.364177465438843 | 5.922780275344849 | 0.5211798472311284 | +1.92x faster | +| 20 | 15.337732791900635 | 4.3357462882995605 | 0.28268495397110643 | +3.54x faster | +| 21 | 114.92279720306396 | 24.284620761871338 | 0.21131247544350482 | +4.73x faster | +| 22 | 6.929309368133545 | 1.8829975128173828 | 0.27174389434492524 | +3.68x faster | +| total | 1298.0628118515015 | 194.74525475502014 | 0.15002760496408013 | +6.67x faster | ++------------+--------------------+--------------------+---------------------+----------------+ + diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x b/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x new file mode 100644 index 0000000..01fd82d --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x @@ -0,0 +1,54 @@ +========================================================================================== +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: m7gd.12xlarge +CPU(s): 48x +MEM: 189G +HD Throughput: 2162.93 (from hdparm) + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 2 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +========================================================================================== +DataFrame() ++------------+--------------------+--------------------+---------------------+---------------+ +| tpch_query | spark | df_ray | change | change_text | ++------------+--------------------+--------------------+---------------------+---------------+ +| 1 | 40.10749578475952 | 7.07772159576416 | 0.17646879859434228 | +5.67x faster | +| 2 | 5.265030860900879 | 4.421863079071045 | 0.8398551111843696 | +1.19x faster | +| 3 | 8.993131875991821 | 4.821190357208252 | 0.5360969263754447 | +1.87x faster | +| 4 | 8.75968074798584 | 2.8997082710266113 | 0.3310289900340668 | +3.02x faster | +| 5 | 19.33255624771118 | 10.249737024307251 | 0.5301801217063955 | +1.89x faster | +| 6 | 1.754713773727417 | 4.5369789600372314 | 2.5855948861674696 | 2.59x slower | +| 7 | 6.20657753944397 | 13.625978946685791 | 2.1954094442694267 | 2.20x slower | +| 8 | 24.33410143852234 | 9.368516683578491 | 0.3849953821901789 | +2.60x faster | +| 9 | 38.12766456604004 | 13.534857511520386 | 0.35498784584817605 | +2.82x faster | +| 10 | 7.784897565841675 | 8.692375421524048 | 1.1165690168698141 | 1.12x slower | +| 11 | 2.0291600227355957 | 3.2606608867645264 | 1.6069017969162889 | 1.61x slower | +| 12 | 6.192300081253052 | 4.058101415634155 | 0.6553463757223104 | +1.53x faster | +| 13 | 8.62465763092041 | 3.507314682006836 | 0.40666132292981705 | +2.46x faster | +| 14 | 2.430983304977417 | 3.096514940261841 | 1.2737705495228016 | 1.27x slower | +| 15 | 6.798931360244751 | 5.77943229675293 | 0.8500501020714643 | +1.18x faster | +| 16 | 4.050042390823364 | 3.0604259967803955 | 0.7556528306258588 | +1.32x faster | +| 17 | 33.42715358734131 | 19.431111812591553 | 0.5812972307624187 | +1.72x faster | +| 18 | 23.402374744415283 | 28.70749068260193 | 1.2266913506054615 | 1.23x slower | +| 19 | 3.6461265087127686 | 5.904490232467651 | 1.6193870998053157 | 1.62x slower | +| 20 | 5.166107892990112 | 5.764256477355957 | 1.1157832156733452 | 1.12x slower | +| 21 | 21.18029236793518 | 19.883575677871704 | 0.9387772053597062 | +1.07x faster | +| 22 | 3.8765759468078613 | 1.483001947402954 | 0.382554596569718 | +2.61x faster | +| total | 281.4905562400818 | 183.1653048992157 | 0.650697868325625 | +1.54x faster | ++------------+--------------------+--------------------+---------------------+---------------+ + diff --git a/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge b/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge new file mode 100644 index 0000000..7aacdf9 --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge @@ -0,0 +1,107 @@ +========================================================================================== +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: r8g.8xlarge +CPU(s): 32x +MEM: 252G +HD Throughput: 167.50 (from hdparm) + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 8 +Ray Workers: 4 +Ray Worker Mem (GB): 56 +Ray Worker CPU: 6 +Ray Head Mem (GB): 8 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 44 +Executor Overhead Mem (GB): 12 +Executor CPU: 6 +Driver Mem(GB): 8 +Driver CPU: 4 +========================================================================================== +DataFrame() ++------------+--------------------+--------------------+---------------------+---------------+ +| tpch_query | spark | df_ray | change | change_text | ++------------+--------------------+--------------------+---------------------+---------------+ +| 1 | 43.84827923774719 | 10.000967502593994 | 0.22808118531558177 | +4.38x faster | +| 2 | 5.0717902183532715 | 3.8610997200012207 | 0.7612893187161155 | +1.31x faster | +| 3 | 10.256386041641235 | 8.126128196716309 | 0.7922993697510979 | +1.26x faster | +| 4 | 8.208275079727173 | 4.202239036560059 | 0.5119515361928797 | +1.95x faster | +| 5 | 36.95858907699585 | 19.0570650100708 | 0.5156329147297589 | +1.94x faster | +| 6 | 1.982734203338623 | 6.400916337966919 | 3.2283279963541 | 3.23x slower | +| 7 | 9.155696153640747 | 18.585452795028687 | 2.029933331463628 | 2.03x slower | +| 8 | 82.31400609016418 | 16.519661903381348 | 0.20069077776734895 | +4.98x faster | +| 9 | 108.94961452484131 | 21.316840410232544 | 0.19565778642908505 | +5.11x faster | +| 10 | 35.651164054870605 | 16.961328268051147 | 0.47575804935698635 | +2.10x faster | +| 11 | 2.080925941467285 | 4.405801773071289 | 2.1172314138026014 | 2.12x slower | +| 12 | 8.010701417922974 | 5.209590673446655 | 0.6503289040071856 | +1.54x faster | +| 13 | 17.210007429122925 | 5.6545422077178955 | 0.3285612880183439 | +3.04x faster | +| 14 | 2.8084030151367188 | 4.262192010879517 | 1.517656827708549 | 1.52x slower | +| 15 | 8.712156772613525 | 7.276721239089966 | 0.8352376373625622 | +1.20x faster | +| 16 | 4.409867525100708 | 4.079038619995117 | 0.9249798541061535 | +1.08x faster | +| 17 | 70.58250260353088 | 26.28227400779724 | 0.37236245582602046 | +2.69x faster | +| 18 | 44.79290056228638 | 57.51821994781494 | 1.2840923277079028 | 1.28x slower | +| 19 | 5.416426420211792 | 7.488842725753784 | 1.382616903611691 | 1.38x slower | +| 20 | 5.868547201156616 | 7.078382253646851 | 1.2061558016014238 | 1.21x slower | +| 21 | 40.45992636680603 | 36.64148950576782 | 0.9056242261436512 | +1.10x faster | +| 22 | 5.988753080368042 | 2.0949254035949707 | 0.34980994799442056 | +2.86x faster | +| total | 558.7376530170441 | 293.0237195491791 | 0.5244388273582853 | +1.91x faster | ++------------+--------------------+--------------------+---------------------+---------------+ + +========================================================================================== +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: r8g.8xlarge +CPU(s): 32x +MEM: 252G +HD Throughput: 169.07 (from hdparm) + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 8 +Ray Workers: 4 +Ray Worker Mem (GB): 56 +Ray Worker CPU: 6 +Ray Head Mem (GB): 8 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 44 +Executor Overhead Mem (GB): 12 +Executor CPU: 6 +Driver Mem(GB): 8 +Driver CPU: 4 +========================================================================================== +DataFrame() ++------------+--------------------+--------------------+---------------------+---------------+ +| tpch_query | spark | df_ray | change | change_text | ++------------+--------------------+--------------------+---------------------+---------------+ +| 1 | 44.023725271224976 | 10.511452674865723 | 0.2387679054897309 | +4.19x faster | +| 2 | 5.121235370635986 | 4.563743352890015 | 0.8911411061201159 | +1.12x faster | +| 3 | 10.236863613128662 | 11.815351963043213 | 1.1541964814193828 | 1.15x slower | +| 4 | 8.071234464645386 | 6.747247695922852 | 0.8359622961615074 | +1.20x faster | +| 5 | 36.89992165565491 | 15.730475664138794 | 0.42630105860206075 | +2.35x faster | +| 6 | 1.977816104888916 | 6.389086723327637 | 3.230374506272149 | 3.23x slower | +| 7 | 9.170192956924438 | 19.880183219909668 | 2.1679132940052352 | 2.17x slower | +| 8 | 91.3105399608612 | 12.09617304801941 | 0.1324729111579478 | +7.55x faster | +| 9 | 106.27129793167114 | 21.19954752922058 | 0.19948516619088602 | +5.01x faster | +| 10 | 25.633528232574463 | 14.167945146560669 | 0.5527114729589346 | +1.81x faster | +| 11 | 1.7444405555725098 | 4.269078493118286 | 2.4472479039087767 | 2.45x slower | +| 12 | 7.713707447052002 | 9.96561598777771 | 1.2919359537787933 | 1.29x slower | +| 13 | 17.422370433807373 | 5.676977157592773 | 0.32584413120827915 | +3.07x faster | +| 14 | 2.8041045665740967 | 3.8914337158203125 | 1.3877634101836138 | 1.39x slower | +| 15 | 8.518751859664917 | 10.617808103561401 | 1.2464042008120013 | 1.25x slower | +| 16 | 4.334302186965942 | 4.8702826499938965 | 1.1236601510249442 | 1.12x slower | +| 17 | 67.09746074676514 | 31.08197546005249 | 0.46323624044969536 | +2.16x faster | +| 18 | 70.05180883407593 | 56.567989110946655 | 0.807516466062041 | +1.24x faster | +| 19 | 5.464143991470337 | 7.1256444454193115 | 1.304073329059889 | 1.30x slower | +| 20 | 5.807567119598389 | 7.7169201374053955 | 1.3287698580983502 | 1.33x slower | +| 21 | 39.69502830505371 | 28.762243032455444 | 0.7245804893101342 | +1.38x faster | +| 22 | 5.841529130935669 | 2.004164695739746 | 0.34308905268075385 | +2.91x faster | +| total | 575.2115707397461 | 295.651340007782 | 0.5139871223862239 | +1.95x faster | ++------------+--------------------+--------------------+---------------------+---------------+ diff --git a/k8s/README.md b/k8s/README.md new file mode 100644 index 0000000..0508327 --- /dev/null +++ b/k8s/README.md @@ -0,0 +1,47 @@ +## Benchmarking on kubernetes + +This directory contains a utility `bench_toolbox.py` to facilitate benchmarking spark and datafusion-ray on k8s clusters. + +The paved path is to execute the steps on a fresh 24.04 ubuntu ami, but the tool should also work on for established k8s setups. + +If that is the case you'll want to skip the install of `https://k3s.io/` from the `k3s` subcommand and proceed. The `machine_prep.sh` script should provide clues about the environment requirements you'll need to satisfy to operate th tool. + +### Benchmarking on a fresh ubuntu 24.04 LTS ami from amazon + +- provision the machine and ssh in and download the repo + +```bash +git checkout https://github.com/robtandy/datafusion-ray +cd datafusion-ray +git checkout k8s_benchmarking +``` + +- then run the machine prep script + +````bash +cd datafusion-ray/k8s +```bash +./machine_prep.sh +```` + +Next, you'll want to choose where you'll keep your TPCH data. + +```bash +sudo mkdir /data +sudo chmod -R 777 /data +``` + +At this point, you'll have the configuration needed to operate the `bench_toolbox.py` script. So, if you first need kubernetes installed, run + +```bash +./bench_toolbox.py -v k3s --data-path /data +``` + +This will: + +- create a single machine cluster using k3s +- create the PVC for /data +- install kuberay operater +- install spark operator + + diff --git a/k8s/bench_toolbox.py b/k8s/bench_toolbox.py new file mode 100755 index 0000000..d01fdfc --- /dev/null +++ b/k8s/bench_toolbox.py @@ -0,0 +1,364 @@ +#!/usr/bin/env python3 +import click + +import cmds +import os +import pandas as pd +import glob +import json +import re +import pricing +import time +import datafusion +import ec2_metadata +import subprocess +from cmds import Runner + +runner: Runner | None = None + + +@click.group() +@click.option("--dry-run", is_flag=True) +@click.option("-v", "--verbose", is_flag=True) +def cli(dry_run: bool, verbose: bool): + global runner + runner = Runner(dry_run, verbose) + + +@cli.command(help="run spark and df ray benchmarks") +@click.option( + "--executor-cpus", + type=int, + help="how much cpu to allocate to the executor[ray worker] nodes.", + required=True, +) +@click.option( + "--executor-mem", + type=int, + help="how much memory (GiB) to allocate to the executor[ray worker] nodes.", + required=True, +) +@click.option( + "--executor-overhead-mem", + type=int, + help="how much memory (GiB) to allocate to the executor overhead. Not used on ray. Will be subtracted from executor_mem", + required=True, +) +@click.option( + "--executor-num", + type=int, + help="how many executors[ray workers] to start", + required=True, +) +@click.option( + "--driver-mem", + type=int, + help="how much memory (GiB) to allocate to the driver[head] node.", + required=True, +) +@click.option( + "--driver-cpus", + type=int, + help="how much cpu to allocate to the driver[ray head] node.", + required=True, +) +@click.option( + "--scale-factor", + type=click.Choice(["1", "10", "100", "1000"]), + help="TPCH scale factor", + required=True, +) +@click.option( + "--data-path", + type=str, + help="path(url) to the directory that holds generated TPCH data. Should be >= 300GB", + required=True, +) +@click.option( + "--output-path", + type=str, + help="path to the local directory exposed via PVC", + required=True, +) + +@click.option( + "--concurrency", + type=int, + help="DFRay only. The number of target partitions to use in planning", + required=True, +) +@click.option( + "--partitions-per-processor", + type=int, + help="how many partitions (out of [concurrency] value to host in each DFRayProcessor", + required=True, +) +@click.option( + "--processor-pool-min", + type=int, + help="minimum number of DFRayProcessrs to allocate in a pool for use by queries", + required=True, +) +@click.option( + "--df-ray-version", type=str, help="version number of DFRay to use", required=True +) +@click.option( + "--test-pypi", + is_flag=True, + help="use the test.pypi upload of DFRay", +) +@click.option( + "--arm", + is_flag=True, + help="deploy an arm image for ray cluster image", +) +@click.argument( + "system", + type=click.Choice(["spark", "df_ray"]), +) +def bench(system, **kwargs): + assert runner is not None + match system: + case "spark": + runner.run_commands(cmds.cmds["bench_spark"], kwargs) + case "df_ray": + runner.run_commands(cmds.cmds["bench_df_ray"], kwargs) + case _: + print(f"unknown system {system}") + exit(1) + + +@click.option( + "--data-path", + type=str, + help="path/url to the directory that holds generated TPCH data. Should be >= 300GB", + required=True, +) +@click.option( + "--output-path", + type=str, + help="path where outputfiles are written", + required=True, +) +@click.option( + "--data-device", + type=str, + help="path to the device in /dev/ that holds the data-path. It will be benchmarked with hdparm for throughput.", + required=True, +) +@click.option( + "--scale-factor", + type=click.Choice(["1", "10", "100", "1000"]), + help="TPCH scale factor", + required=True, +) +@cli.command(help="assemble the results into a single json") +def results(data_path, data_device, scale_factor, output_path): + df_result = json.loads( + open( + newest_file(glob.glob(os.path.join(output_path, "datafusion-ray*json"))) + ).read() + ) + spark_result = json.loads( + open(newest_file(glob.glob(os.path.join(output_path, "spark-tpch*json")))).read() + ) + print(df_result) + print(spark_result) + total_results = {"spark": spark_result, "df-ray": df_result} + + spark = [spark_result["queries"][f"{i}"] for i in range(1, 23)] + df_ray = [df_result["queries"][f"{i}"] for i in range(1, 23)] + + # add a final row with the totals + spark += [sum(spark)] + df_ray += [sum(df_ray)] + + # add another final row with costs + + # df for "dataframe" here, not "datafusion". Just using pandas for easy output + df = pd.DataFrame({"spark": spark, "df_ray": df_ray}) + df["change"] = df["df_ray"] / df["spark"] + + df["change_text"] = df["change"].apply( + lambda change: ( + f"+{(1 / change):.2f}x faster" if change < 1.0 else f" {change:.2f}x slower" + ) + ) + df["tpch_query"] = [f"{i}" for i in range(1, 23)] + ["total"] + df["sort_index"] = list(range(1, 24)) + + ts = time.time() + df.to_parquet(f"datafusion-ray-spark-comparison-{ts}.parquet") + ctx = datafusion.SessionContext() + ctx.register_parquet("results", f"datafusion-ray-spark-comparison-{ts}.parquet") + + cpu = subprocess.run( + "lscpu | grep 'Model name' |awk '{print $3}'", + shell=True, + capture_output=True, + text=True, + ).stdout.strip() + quantity = subprocess.run( + "lscpu | grep '^CPU(s):' |awk '{print $2}'", + shell=True, + capture_output=True, + text=True, + ).stdout.strip() + memory = subprocess.run( + "lsmem | grep 'Total online' |awk '{print $4}'", + shell=True, + capture_output=True, + text=True, + ).stdout.strip() + hdresults = subprocess.run( + f"sudo hdparm -t {data_device}|grep 'MB/sec'", + shell=True, + capture_output=True, + text=True, + ).stdout.strip() + + hdresult = re.search(r"([\d\.]+) MB/sec", hdresults, re.MULTILINE).group(1) + + machine = ec2_metadata.ec2_metadata.instance_type + + # if you get reserved it includes any discounts you may have associated + # with your ec2 credentials. So a public price is appropriate for sharing + hourly_cost = pricing.get_on_demand_price("us-east-1", machine) + + spark_cost = spark[-1] / 3600 * hourly_cost + df_ray_cost = df_ray[-1] / 3600 * hourly_cost + + cost_delta = df_ray_cost / spark_cost + cost_delta_text = ( + f"+{(1 / cost_delta):.2f}x cheaper" + if cost_delta < 1.0 + else f"{cost_delta:.2f}x more expensive" + ) + speed_delta_text = df["change_text"].iloc[-1] + + df_ray_cost = f"${df_ray_cost:.4f}" + df_ray_duration = f"{df_ray[-1]:.2f}s" + spark_cost = f"${spark_cost:.4f}" + spark_duration = f"{spark[-1]:.2f}s" + + print("=" * 97) + # the formatting code is terrible here, but it works for now + header = [ + "Spark and DataFusionRay TPCH 100 Benchmarks", + f"{'Machine:':<30}{machine}", + f"{'Machine On Demand Cost:':<30}{hourly_cost} $/hr", + f"{'CPU(s):':<30}{cpu} {quantity}x", + f"{'MEM:':<30}{memory}", + f"{'HD Throughput:':<30}{hdresult} MB/s (from hdparm)", + f"{'Data Location:':<30}{data_path}/sf{scale_factor}", + "", + f"{'df-ray duration:':<30}{df_ray_duration:>10} {speed_delta_text}", + f"{'df-ray cost:':<30}{df_ray_cost:>10} {cost_delta_text}", + "", + f"{'spark duration:':<30}{spark_duration:>10}", + f"{'spark cost:':<30}{spark_cost:>10}", + "", + "DataFusionRay Settings:", + f"{'concurrency:':<30}{df_result['settings']['concurrency']:>10}", + f"{'batch_size :':<30}{df_result['settings']['batch_size']:>10}", + f"{'partitions_per_processor:':<30}{df_result['settings']['partitions_per_processor']:>10}", + f"{'Ray Workers:':<30}{spark_result['spark_conf']['spark.executor.instances']:>10}", + f"{'Ray Worker Mem (GB):':<30}{int(spark_result['spark_conf']['spark.executor.memory'][:-1]) + int(spark_result['spark_conf']['spark.executor.memoryOverhead'][:-1]):>10}", + f"{'Ray Worker CPU:':<30}{spark_result['spark_conf']['spark.executor.cores']:>10}", + f"{'Ray Head Mem (GB):':<30}{int(spark_result['spark_conf']['spark.driver.memory'][:-1]):>10}", + f"{'Ray Head CPU:':<30}{spark_result['spark_conf']['spark.driver.cores']:>10}", + "", + "Spark Settings:", + f"{'Executors:':<30}{spark_result['spark_conf']['spark.executor.instances']:>10}", + f"{'Executor Mem (GB):':<30}{int(spark_result['spark_conf']['spark.executor.memory'][:-1]):>10}", + f"{'Executor Overhead Mem (GB):':<30}{int(spark_result['spark_conf']['spark.executor.memoryOverhead'][:-1]):>10}", + f"{'Executor CPU:':<30}{spark_result['spark_conf']['spark.executor.cores']:>10}", + f"{'Driver Mem(GB):':<30}{int(spark_result['spark_conf']['spark.driver.memory'][:-1]):>10}", + f"{'Driver CPU:':<30}{spark_result['spark_conf']['spark.driver.cores']:>10}", + ] + for h in header: + print(h) + + print("=" * 97) + ctx.sql( + 'select tpch_query, spark, df_ray, change as "change(=df_ray/spark)", change_text from results order by sort_index asc' + ).show(num=100) + + out_path = f"datafusion-ray-spark-comparison-{ts}.json" + open(out_path, "w").write(json.dumps(total_results)) + + +@cli.command(help="Install k3s and configure it") +@click.option( + "--data-path", + type=str, + help="path to the directory that holds generated TPCH data. Should be >= 300GB", + required=True, +) +@click.option( + "--k3s-url", + type=str, + help="url to head node of the cluster to join", +) +@click.option( + "--k3s-token", + type=str, + help="k3s token to authorize when joining the cluster", +) +def k3s(**kwargs): + assert runner is not None + if kwargs["k3s_url"]: + kwargs["k3s_url"] = f"K3S_URL={kwargs['k3s_url']}" + if kwargs["k3s_token"]: + kwargs["k3s_token"] = f"K3S_TOKEN={kwargs['k3s_token']}" + runner.run_commands(cmds.cmds["k3s_setup"], kwargs) + + +@cli.command(help="Generate TPCH data") +@click.option( + "--data-path", + type=str, + help="path to the directory that will hold the generated TPCH data. Should be >= 300GB", + required=True, +) +@click.option( + "--scale-factor", + type=click.Choice(["1", "10", "100", "1000"]), + help="TPCH scale factor", + required=True, +) +@click.option( + "--partitions", + type=int, + help="TPCH number of partitions for each table", + required=True, +) +@click.option( + "--pool-size", + type=int, + default=1, + help="number of concurrent processors to use. Watch out! too high and machine will lock up from too much memory use", +) +def generate(**kwargs): + assert runner is not None + runner.run_commands(cmds.cmds["generate"], kwargs) + + +@cli.command(help="just testing of toolbox shell commands that are harmless") +def echo(): + assert runner is not None + runner.run_commands(cmds.cmds["echo"]) + + +@cli.command() +def help(): + """Print the overall help message.""" + click.echo(cli.get_help(click.Context(cli))) + + +def newest_file(files: list[str]): + return max(files, key=os.path.getctime) + + +if __name__ == "__main__": + cli() diff --git a/k8s/cmds.py b/k8s/cmds.py new file mode 100644 index 0000000..586b63a --- /dev/null +++ b/k8s/cmds.py @@ -0,0 +1,280 @@ +from collections import namedtuple +import subprocess +import jinja2 +import click +import os +import signal +import textwrap + +Shell = namedtuple("Shell", ["cmd", "desc"]) +BackgroundShell = namedtuple("BackgroundShell", ["cmd", "desc"]) +Template = namedtuple("Template", ["path", "desc"]) +ChangeDir = namedtuple("ChangeDir", ["path", "desc"]) +Venv = namedtuple("Venv", ["cmd", "path", "desc"]) + +MY_DIR = os.path.dirname(os.path.abspath(__file__)) + +# TODO: assert that commands we require like python3, jq, curl, are all present + +cmds = { + "echo": [ + Shell("echo hello 1", "echoing first"), + Shell("echo hello 2", "echoing second"), + Shell("bad_command_garbage", "Something that will fail"), + Shell("echo hello 3", "echoing third which we wont see"), + ], + "k3s_setup": [ + Shell( + """sudo curl -sfL https://get.k3s.io | {{ k3s_url if k3s_url else "" }} {{ k3s_token if k3s_token else ""}} sh - --write-kubeconfig-mode 644""", + "Installing K3s", + ), + Shell( + "sudo chmod a+r /etc/rancher/k3s/k3s.yaml", + "Allow read access to chmod a+r /etc/rancher/k3s/k3s.yaml", + ), + Shell( + "kubectl get all", + "Check for access to cluster", + ), + Shell( + "curl https://raw.githubusercontent.com/helm/helm/main/scripts/get-helm-3 | bash", + "Installing Helm", + ), + Shell( + "helm repo add kuberay https://ray-project.github.io/kuberay-helm/", + "Adding kube ray helm repo", + ), + Shell( + "helm repo add spark-operator https://kubeflow.github.io/spark-operator", + "Adding spark operator helm repo", + ), + Shell("helm repo update", "Updating helm repos"), + Shell( + "helm --kubeconfig /etc/rancher/k3s/k3s.yaml install kuberay-operator kuberay/kuberay-operator --version 1.3.0 --wait", + "Installing kuberay-operator", + ), + Shell( + """helm install --set-json='controller.env=[{"name":"SPARK_SUBMIT_OPTS","value":"-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2"}]' spark-operator spark-operator/spark-operator""", + "Installing spark-operator", + ), + Template("pvcs.yaml.template", "rewrite pvcs.yaml.template"), + Shell("kubectl apply -f pvcs.yaml", "Apply pvcs"), + ], + "generate": [ + Shell( + "mkdir -p /data/sf{{scale_factor}}", + "make directory /data/sf{{scale_factor}}", + ), + Shell( + "python {{ MY_DIR }}/../tpch/make_data.py {{scale_factor}} {{partitions}} {{data_path}}/sf{{scale_factor}} {{pool_size}}", + "generate data", + ), + ], + "bench_spark": [ + Template("spark_job.yaml.template", "rewrite spark_job.yaml.template"), + Shell( + "cp {{ MY_DIR }}/spark_tpcbench.py {{ output_path }}", + "copy spark_tpcbench.py to data_path dir", + ), + Shell( + "cp -a {{ MY_DIR }}/../tpch/queries {{ output_path }}", + "copy tpch queries to data_path dir", + ), + Shell( + """ + wget https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar && \ + aws s3 cp aws-java-sdk-bundle-1.12.262.jar {{ data_path.replace('s3a','s3') }}/aws-java-sdk-bundle-1.12.262.jar && \ + wget https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar && \ + aws s3 cp hadoop-aws-3.3.4.jar {{ data_path.replace('s3a','s3') }}/hadoop-aws-3.3.4.jar + """, + "getting additional spark jars" + ), + Shell( + "kubectl apply -f spark_job.yaml", + "Submit spark job", + ), + Shell( + """ + while true; do + sleep 10 + STATE=$(kubectl get sparkapp/spark-tpch-bench -o json |jq -r '.status.applicationState.state') + echo "Checking on job status...got $STATE looking for COMPLETED" + if [[ $STATE != "RUNNING" ]]; then + break + fi + done + """, + "checking on job status", + ), + #Shell( + # "kubectl delete -f spark_job.yaml", + # "tear down job", + #), + ], + "bench_df_ray": [ + Template("ray_cluster.yaml.template", "rewrite ray_cluster.yaml.template"), + Shell( + "kubectl apply -f ray_cluster.yaml", + "deploying ray cluster", + ), + Shell( + "kubectl wait raycluster/datafusion-ray-cluster --for='jsonpath={.status.state}'=ready --timeout=300s", + "wait for ray cluster to be ready", + ), + Template("requirements.txt.template", "rewrite requirements.txt.template"), + Template("ray_job.sh.template", "rewrite ray_job.sh.template"), + BackgroundShell( + "kubectl port-forward svc/datafusion-ray-cluster-head-svc 8265:8265", + "port forwarding from cluster", + ), + Shell( + "cp {{ MY_DIR }}/../tpch/tpcbench.py .", + "copy tpcbench.py to .", + ), + Shell( + "cp -a {{ MY_DIR }}/../tpch/queries .", + "copy tpch queries to .", + ), + Shell( + ". ./ray_job.sh", + "running ray job", + ), + Shell( + "kubectl delete -f ray_cluster.yaml", + "tear down ray cluster", + ), + ], +} + + +class Runner: + def __init__(self, dry_run: bool = False, verbose: bool = False): + self.dry_run = dry_run + self.verbose = verbose + self.cwd = os.getcwd() + self.venv: str | None = None + self.backgrounded = [] + + def set_cwd(self, path: str): + if os.path.isabs(path): + self.cwd = path + else: + self.cwd = os.path.join(self.cwd, path) + + def activate_venv(self, path: str): + self.venv = path + + def run_commands( + self, + commands: list[dict[str, str]], + substitutions: dict[str, str] | None = None, + ): + if not substitutions: + substitutions = {} + + substitutions["MY_DIR"] = MY_DIR + + for command in commands: + match (self.dry_run, command): + case (False, Shell(cmd, desc)): + self.run_shell_command(textwrap.dedent(cmd), desc, substitutions) + + case (True, Shell(cmd, desc)): + click.secho(f"[dry run] {desc} ...") + click.secho(f" {cmd}", fg="yellow") + + case (False, BackgroundShell(cmd, desc)): + self.run_shell_command( + textwrap.dedent(cmd), desc, substitutions, background=True + ) + + case (True, BackgroundShell(cmd, desc)): + click.secho(f"[dry run] {desc} ...") + click.secho(f"[backgrounding] {cmd}", fg="yellow") + + case (False, Template(path, desc)): + click.secho(f"{desc} ...") + self.process_template(path, ".", substitutions) + + case (True, Template(path, desc)): + click.secho(f"[dry run] {desc} ...") + click.secho(f" {path} subs:{substitutions}", fg="yellow") + + case (False, ChangeDir(path, desc)): + click.secho(f"{desc} ...") + self.set_cwd(path) + + case (True, ChangeDir(path, desc)): + click.secho(f"[dry run] {desc} ...") + + case (False, Venv(cmd, path, desc)): + self.run_shell_command(cmd, desc) + self.venv = os.path.abspath(path) + + case (True, Venv(cmd, path, desc)): + click.secho(f"[dry run] {desc} ...") + + case _: + raise Exception("Unhandled case in match. Shouldn't happen") + + def run_shell_command( + self, + command: str, + desc: str, + substitutions: dict[str, str] | None = None, + background: bool = False, + ): + click.secho(f"{desc} ...") + if self.venv: + venv_path = os.path.join(self.cwd, self.venv, "bin/activate") + command = f"source {venv_path} && {command}" + if substitutions: + command = jinja2.Template(command).render(substitutions) + + if self.verbose: + back = " background" if background else "" + click.secho(f"[Running command{back}] {command}", fg="yellow") + + process = subprocess.Popen( + command, + shell=True, + cwd=self.cwd, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + executable="/bin/bash", + ) + + if background: + self.backgrounded.append(process) + return + + stdout, stderr = process.communicate() + stdout = stdout.decode() + stderr = stderr.decode() + + if process.returncode == 0: + click.secho(f" {stdout}", fg="green") + else: + click.secho(f" stdout = {stdout}", fg="red") + click.secho(f" stderr = {stderr}", fg="red") + click.secho(f"Error running command {command}") + exit(1) + + def process_template( + self, template_name: str, output_path: str, substitutions: dict[str, str] | None + ): + template_out = template_name[: template_name.index(".template")] + output_path = os.path.join(output_path, template_out) + template_path = os.path.join(MY_DIR, template_name) + + template = jinja2.Template(open(template_path).read()) + + with open(output_path, "w") as f: + f.write(template.render(substitutions)) + + def __del__(self): + for process in self.backgrounded: + try: + os.killpg(os.getpgid(process.pid), signal.SIGTERM) + except Exception as e: + print(f"Failed to kill process {process.pid}: {e}") diff --git a/k8s/machine_prep.sh b/k8s/machine_prep.sh new file mode 100644 index 0000000..6cdb95d --- /dev/null +++ b/k8s/machine_prep.sh @@ -0,0 +1,29 @@ +#!/bin/bash + +sudo apt-get update + +# install docker +sudo apt-get install -y ca-certificates curl +sudo install -m 0755 -d /etc/apt/keyrings +sudo curl -fsSL https://download.docker.com/linux/ubuntu/gpg -o /etc/apt/keyrings/docker.asc +sudo chmod a+r /etc/apt/keyrings/docker.asc + +# Add the repository to Apt sources: +echo \ + "deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/docker.asc] https://download.docker.com/linux/ubuntu \ + $(. /etc/os-release && echo "${UBUNTU_CODENAME:-$VERSION_CODENAME}") stable" | + sudo tee /etc/apt/sources.list.d/docker.list >/dev/null +sudo apt-get update + +sudo apt-get install -y docker-ce docker-ce-cli containerd.io docker-buildx-plugin docker-compose-plugin +sudo groupadd docker +sudo usermod -aG docker $USER +newgrp docker +# end docker section + +sudo apt install -y python3-virtualenv + +virtualenv -p $(which python3) venv +. ./venv/bin/activate +# install what we need to run the tool +pip install click jinja2 pandas datafusion 'ray[default]' ec2_metadata duckdb boto3 diff --git a/k8s/pricing.py b/k8s/pricing.py new file mode 100644 index 0000000..8b825b3 --- /dev/null +++ b/k8s/pricing.py @@ -0,0 +1,123 @@ +import boto3 +import json + +region_mapping = { + "ap-east-1": "Asia Pacific (Hong Kong)", + "ap-southeast-1": "Asia Pacific (Singapore)", + "ap-southeast-2": "Asia Pacific (Sydney)", + "ap-northeast-1": "Asia Pacific (Tokyo)", + "ap-northeast-2": "Asia Pacific (Seoul)", + "ap-northeast-3": "Asia Pacific (Osaka-Local)", + "ap-south-1": "Asia Pacific (Mumbai)", + "ca-central-1": "Canada (Central)", + "eu-central-1": "EU (Frankfurt)", + "eu-west-1": "EU (Ireland)", + "eu-west-2": "EU (London)", + "eu-west-3": "EU (Paris)", + "sa-east-1": "South America (Sao Paulo)", + "us-east-1": "US East (N. Virginia)", + "us-west-1": "US West (N. California)", + "us-east-2": "US East (Ohio)", + "us-west-2": "US West (Oregon)", + "cn-north-1": "China (Beijing)", + "cn-northwest-1": "China (Ningxia)", + "us-gov-west-1": "AWS GovCloud (US)", +} + + +def get_on_demand_price(region, instance_type): + """Get on-demand price for a specific instance type in a region.""" + pricing_client = boto3.client("pricing", region_name="us-east-1") + + try: + response = pricing_client.get_products( + ServiceCode="AmazonEC2", + Filters=[ + {"Type": "TERM_MATCH", "Field": "instanceType", "Value": instance_type}, + {"Type": "TERM_MATCH", "Field": "operatingSystem", "Value": "Linux"}, + {"Type": "TERM_MATCH", "Field": "preInstalledSw", "Value": "NA"}, + {"Type": "TERM_MATCH", "Field": "tenancy", "Value": "Shared"}, + {"Type": "TERM_MATCH", "Field": "capacitystatus", "Value": "Used"}, + { + "Type": "TERM_MATCH", + "Field": "location", + "Value": region_mapping[region], + }, + ], + ) + + if response["PriceList"]: + price_data = eval(response["PriceList"][0]) + terms = price_data["terms"]["OnDemand"] + price_dimensions = next(iter(terms.values()))["priceDimensions"] + price = next(iter(price_dimensions.values()))["pricePerUnit"]["USD"] + return float(price) + except Exception as e: + print( + f"Error getting on-demand price for {instance_type} in {region}: {str(e)}" + ) + return None + + +def get_reserved(region, instance_type): + """Get savings plans rates for a specific instance type in a region.""" + savingsplans_client = boto3.client("savingsplans", region_name="us-east-1") + + def request(): + results = [] + nt = None + while True: + kwargs = dict( + filters=[ + {"name": "instanceType", "values": [instance_type]}, + {"name": "region", "values": [region]}, + {"name": "tenancy", "values": ["shared"]}, + {"name": "productDescription", "values": ["Linux/UNIX"]}, + ], + serviceCodes=["AmazonEC2"], + ) + if nt: + kwargs["nextToken"] = nt + + response = savingsplans_client.describe_savings_plans_offering_rates( + **kwargs + ) + results.extend(response["searchResults"]) + if len(response["nextToken"]) > 0: + nt = response["nextToken"] + else: + break + return results + + try: + results = request() + rates = { + "1.0y": {"no": None, "partial": None, "all": None}, + "3.0y": {"no": None, "partial": None, "all": None}, + } + + for result in results: + # Skip unused box entries + if "UnusedBox" in result["usageType"]: + continue + + duration_seconds = result["savingsPlanOffering"]["durationSeconds"] + duration_years = duration_seconds / (365 * 24 * 60 * 60) + key = f"{duration_years:.1f}y" + + payment_option = ( + result["savingsPlanOffering"]["paymentOption"].lower().split()[0] + ) # 'no', 'partial', or 'all' + rate = float(result["rate"]) + + if key in rates: + rates[key][payment_option] = rate + + return ( + rates + if any(any(v is not None for v in year.values()) for year in rates.values()) + else None + ) + except Exception as e: + print(f"Error getting reserved cost for {instance_type} in {region}: {str(e)}") + return None diff --git a/k8s/pvcs.yaml.template b/k8s/pvcs.yaml.template new file mode 100644 index 0000000..8920c42 --- /dev/null +++ b/k8s/pvcs.yaml.template @@ -0,0 +1,31 @@ +apiVersion: v1 +kind: PersistentVolume +metadata: + name: ray-pv +spec: + storageClassName: local-path + capacity: + storage: 300Gi + volumeMode: Filesystem + hostPath: + path: "{{ data_path }}" + accessModes: + - ReadWriteOnce + claimRef: + name: ray-pvc + namespace: default +--- +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: ray-pvc + namespace: default +spec: + storageClassName: local-path + accessModes: + - ReadWriteOnce + volumeName: ray-pv + resources: + requests: + storage: 300Gi + diff --git a/k8s/ray_cluster.yaml.template b/k8s/ray_cluster.yaml.template new file mode 100644 index 0000000..ce1ae4a --- /dev/null +++ b/k8s/ray_cluster.yaml.template @@ -0,0 +1,57 @@ +apiVersion: ray.io/v1alpha1 +kind: RayCluster +metadata: + name: datafusion-ray-cluster +spec: + headGroupSpec: + rayStartParams: + num-cpus: "0" + template: + spec: + containers: + - name: ray-head + image: rayproject/ray:2.43.0-py312-cpu{{ "-aarch64" if arm else "" }} + imagePullPolicy: IfNotPresent + resources: + limits: + cpu: {{ driver_cpus }} + memory: {{ driver_mem }}Gi + requests: + cpu: {{ driver_cpus }} + memory: {{ driver_mem }}Gi + volumeMounts: + - mountPath: /data + name: ray-storage + volumes: + - name: ray-storage + hostPath: + path: /data + type: DirectoryOrCreate + nodeSelector: + node-role.kubernetes.io/master: "true" + workerGroupSpecs: + - replicas: {{ executor_num }} + groupName: "datafusion-ray" + rayStartParams: + num-cpus: "{{ executor_cpus }}" + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray:2.43.0-py312-cpu{{ "-aarch64" if arm else "" }} + imagePullPolicy: IfNotPresent + resources: + limits: + cpu: {{ executor_cpus }} + memory: {{ executor_mem }}Gi + requests: + cpu: {{ executor_cpus }} + memory: {{ executor_mem }}Gi + volumeMounts: + - mountPath: /data/ + name: ray-storage + volumes: + - name: ray-storage + hostPath: + path: /data + type: DirectoryOrCreate diff --git a/k8s/ray_job.sh.template b/k8s/ray_job.sh.template new file mode 100644 index 0000000..361b390 --- /dev/null +++ b/k8s/ray_job.sh.template @@ -0,0 +1,9 @@ +ray job submit --address='http://localhost:8265' \ + --runtime-env-json='{"pip":"requirements.txt", "working_dir":"./", "env_vars":{"DATAFUSION_RAY_LOG_LEVEL":"WARN", "RAY_OVERRIDE_JOB_RUNTIME_ENV":"1", "RAY_DEDUP_LOGS":"O", "RAY_COLOR_PREFIX":"1"}}' -- \ + python tpcbench.py \ + --data {{ data_path }}/sf{{ scale_factor }} \ + --concurrency {{ concurrency }} \ + --partitions-per-processor {{ partitions_per_processor }} \ + --processor-pool-min {{ processor_pool_min }} \ + --listing-tables \ + --output-path /data diff --git a/k8s/requirements.txt.template b/k8s/requirements.txt.template new file mode 100644 index 0000000..71160e2 --- /dev/null +++ b/k8s/requirements.txt.template @@ -0,0 +1,6 @@ +{% if test_pypi %} +--index-url https://test.pypi.org/simple/ +--extra-index-url https://pypi.org/simple/ +{% endif %} +datafusion-ray=={{ df_ray_version }} +duckdb diff --git a/k8s/spark_job.yaml.template b/k8s/spark_job.yaml.template new file mode 100644 index 0000000..5d57b35 --- /dev/null +++ b/k8s/spark_job.yaml.template @@ -0,0 +1,56 @@ +apiVersion: sparkoperator.k8s.io/v1beta2 +kind: SparkApplication +metadata: + name: spark-tpch-bench + namespace: default +spec: + type: Python + pythonVersion: "3" + mode: cluster + image: spark:3.5.3 + imagePullPolicy: IfNotPresent + # see https://github.com/kubeflow/spark-operator/issues/1132#issuecomment-2688276488 + deps: + packages: + - org.apache.hadoop:hadoop-aws:3.3.4 + - com.amazonaws:aws-java-sdk-bundle:1.12.782 + repositories: + - https://repo1.maven.org/maven2/ + mainApplicationFile: {{ data_path }}/spark_tpcbench.py + arguments: + - --name + - spark + - --benchmark + - tpch + - --data + - {{ data_path }}/sf{{ scale_factor }} + - --queries + - /data/queries + - --output + - /data/ + sparkVersion: 3.5.3 + volumes: + - name: spark-local-dir-spark-vol + hostPath: + path: /data + type: DirectoryOrCreate + driver: + javaOptions: "-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2" + cores: {{ driver_cpus }} + memory: "{{ driver_mem }}g" + serviceAccount: spark-operator-spark + volumeMounts: + - name: spark-local-dir-spark-vol + mountPath: /data + nodeSelector: + node-role.kubernetes.io/master: "true" + executor: + javaOptions: "-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2" + instances: {{ executor_num }} + cores: {{ executor_cpus }} + memory: "{{ executor_mem - executor_overhead_mem }}g" + memoryOverhead: "{{ executor_overhead_mem }}g" + volumeMounts: + - name: spark-local-dir-spark-vol + mountPath: /data + diff --git a/k8s/spark_tpcbench.py b/k8s/spark_tpcbench.py new file mode 100644 index 0000000..18bdf88 --- /dev/null +++ b/k8s/spark_tpcbench.py @@ -0,0 +1,145 @@ +# 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. + +import argparse +from datetime import datetime +import json +from pyspark.sql import SparkSession +import time +import sys + +import boto3 + + +def main(benchmark: str, data_path: str, query_path: str, output_path: str, name: str): + + # Initialize a SparkSession + spark = SparkSession.builder \ + .appName( f"{name} benchmark derived from {benchmark}") \ + .getOrCreate() + + spark.conf.set("spark.hadoop.fs.s3a.aws.credentials.provider", "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider") + spark.conf.set("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") + + # Register the tables + num_queries = 22 + table_names = [ + "customer", + "lineitem", + "nation", + "orders", + "part", + "partsupp", + "region", + "supplier", + ] + + for table in table_names: + path = f"{data_path}/{table}.parquet" + print(f"Registering table {table} using path {path}") + df = spark.read.parquet(path) + df.createOrReplaceTempView(table) + + conf_dict = {k: v for k, v in spark.sparkContext.getConf().getAll()} + + results = { + "engine": "spark", + "benchmark": benchmark, + "data_path": data_path, + "query_path": query_path, + "spark_conf": conf_dict, + "queries": {}, + } + + iter_start_time = time.time() + + for query in range(1, num_queries + 1): + spark.sparkContext.setJobDescription(f"{benchmark} q{query}") + + # if query == 9: + # continue + + # read text file + path = f"{query_path}/q{query}.sql" + + # if query == 72: + # # use version with sensible join order + # path = f"{query_path}/q{query}_optimized.sql" + + print(f"Reading query {query} using path {path}") + with open(path, "r") as f: + text = f.read() + # each file can contain multiple queries + queries = list( + filter(lambda x: len(x) > 0, map(lambda x: x.strip(), text.split(";"))) + ) + + start_time = time.time() + for sql in queries: + sql = sql.strip().replace("create view", "create temp view") + if len(sql) > 0: + print(f"Executing: {sql}") + df = spark.sql(sql) + rows = df.collect() + end_time = time.time() + + out_path = f"{output_path}/{name}_{benchmark}_q{query}_result.txt" + # fIXME: concat output for all queries. For example q15 has multiple + out = df._show_string(100000) + with open(out_path, "w") as f: + f.write(out) + + print(f"Query {query} took {end_time - start_time} seconds") + + results["queries"][str(query)] = end_time - start_time + print(json.dumps(results, indent=4)) + + iter_end_time = time.time() + print(f"total took {round(iter_end_time - iter_start_time,2)} seconds") + + out = json.dumps(results, indent=4) + current_time_millis = int(datetime.now().timestamp() * 1000) + results_path = f"{output_path}/{name}-{benchmark}-{current_time_millis}.json" + print(f"Writing results to {results_path}") + with open(results_path, "w") as f: + f.write(out) + + # Stop the SparkSession + spark.stop() + + +if __name__ == "__main__": + print(f"got arguments {sys.argv}") + print(f"python version {sys.version}") + print(f"python versioninfo {sys.version_info}") + + parser = argparse.ArgumentParser( + description="DataFusion benchmark derived from TPC-H / TPC-DS" + ) + parser.add_argument( + "--benchmark", required=True, help="Benchmark to run (tpch or tpcds)" + ) + parser.add_argument("--data", required=True, help="Path to data files") + parser.add_argument("--queries", required=True, help="Path to query files") + parser.add_argument("--output", required=True, help="Path to write output") + parser.add_argument( + "--name", required=True, help="Prefix for result file e.g. spark/comet/gluten" + ) + args = parser.parse_args() + print(f"parsed is {args}") + + main(args.benchmark, args.data, args.queries, args.output, args.name) diff --git a/tpch/make_data.py b/tpch/make_data.py index bec5173..4b5e8cc 100644 --- a/tpch/make_data.py +++ b/tpch/make_data.py @@ -1,23 +1,31 @@ import duckdb import sys +import os +import multiprocessing conn = duckdb.connect() -def make(scale_factor: int, output_path: str): +def make(scale_factor: int, partitions: int, output_path: str, step: int): statements = [ "install tpch", "load tpch", - f"call dbgen(sf = {scale_factor})", ] execute(statements) + print(f"step {step}") + sql = f"call dbgen(sf={scale_factor}, children={partitions}, step={step})" + conn.execute(sql) + conn.sql("show tables").show() + statements = [] + for row in conn.execute("show tables").fetchall(): table = row[0] + os.makedirs(f"{output_path}/{table}.parquet", exist_ok=True) statements.append( - f"copy {table} to '{output_path}/{table}.parquet' (format parquet, compression zstd)" + f"copy {table} to '{output_path}/{table}.parquet/part{step}.parquet' (format parquet, compression zstd)" ) execute(statements) @@ -29,4 +37,15 @@ def execute(statements): if __name__ == "__main__": - make(int(sys.argv[1]), sys.argv[2]) + # this is quick and dirty, it should be tidied up with click to process args + scale_factor = int(sys.argv[1]) + partitions = int(sys.argv[2]) + data_path = sys.argv[3] + procs = int(sys.argv[4]) + + def go(step): + make(scale_factor, partitions, data_path, step) + + steps = list(range(partitions)) + with multiprocessing.Pool(processes=procs) as pool: + pool.map(go, steps) diff --git a/tpch/tpcbench.py b/tpch/tpcbench.py index 16288fb..16c9041 100644 --- a/tpch/tpcbench.py +++ b/tpch/tpcbench.py @@ -35,10 +35,11 @@ def main( data_path: str, concurrency: int, batch_size: int, - partitions_per_worker: int | None, - worker_pool_min: int, + partitions_per_processor: int | None, + processor_pool_min: int, listing_tables: bool, validate: bool, + output_path: str, prefetch_buffer_size: int, ): # Register the tables @@ -58,9 +59,10 @@ def main( ctx = DFRayContext( batch_size=batch_size, - partitions_per_worker=partitions_per_worker, + partitions_per_processor=partitions_per_processor, prefetch_buffer_size=prefetch_buffer_size, - worker_pool_min=worker_pool_min, + processor_pool_min=processor_pool_min, + processor_pool_max=1000, ) local = LocalValidator() @@ -81,7 +83,9 @@ def main( local.register_parquet(table, path) current_time_millis = int(datetime.now().timestamp() * 1000) - results_path = f"datafusion-ray-tpch-{current_time_millis}.json" + results_path = os.path.join( + output_path, f"datafusion-ray-tpch-{current_time_millis}.json" + ) print(f"Writing results to {results_path}") results = { @@ -91,7 +95,7 @@ def main( "concurrency": concurrency, "batch_size": batch_size, "prefetch_buffer_size": prefetch_buffer_size, - "partitions_per_worker": partitions_per_worker, + "partitions_per_processor": partitions_per_processor, }, "data_path": data_path, "queries": {}, @@ -106,7 +110,6 @@ def main( statements = list( filter(lambda x: len(x) > 0, map(lambda x: x.strip(), sql.split(";"))) ) - print(f"statements = {statements}") start_time = time.time() all_batches = [] @@ -119,6 +122,12 @@ def main( calculated = "\n".join([prettify(b) for b in all_batches]) print(calculated) + out_path = os.path.join( + output_path, f"datafusion_ray_tpch_q{qnum}_result.txt" + ) + with open(out_path, "w") as f: + f.write(calculated) + if validate: all_batches = [] for sql in statements: @@ -169,8 +178,15 @@ def main( parser.add_argument( "--partitions-per-processor", type=int, - help="Max partitions per Stage Service Worker", + help="partitions per DFRayProcessor", ) + parser.add_argument( + "--output-path", + type=str, + default=".", + help="directory to write output json", + ) + parser.add_argument( "--prefetch-buffer-size", required=False, @@ -179,9 +195,9 @@ def main( help="How many batches each stage should eagerly buffer", ) parser.add_argument( - "--worker-pool-min", + "--processor-pool-min", type=int, - help="Minimum number of RayStages to keep in pool", + help="Minimum number of DFRayProcessors to keep in pool", ) args = parser.parse_args() @@ -192,8 +208,9 @@ def main( int(args.concurrency), int(args.batch_size), args.partitions_per_processor, - args.worker_pool_min, + args.processor_pool_min, args.listing_tables, args.validate, + args.output_path, args.prefetch_buffer_size, ) From 8efad6de3c309e359eb31dd440f989daa3d69c07 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 20 Mar 2025 13:11:54 +0100 Subject: [PATCH 2/8] merge in another change required before subsequent merges --- tpch/tpcbench.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tpch/tpcbench.py b/tpch/tpcbench.py index 16c9041..67234f3 100644 --- a/tpch/tpcbench.py +++ b/tpch/tpcbench.py @@ -122,9 +122,7 @@ def main( calculated = "\n".join([prettify(b) for b in all_batches]) print(calculated) - out_path = os.path.join( - output_path, f"datafusion_ray_tpch_q{qnum}_result.txt" - ) + out_path = os.path.join(output_path, f"datafusion_ray_tpch_q{qnum}_result.txt") with open(out_path, "w") as f: f.write(calculated) From beba9b27a7fb6bfbef6d78f287c8f70eed526f92 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 20 Mar 2025 13:29:50 +0100 Subject: [PATCH 3/8] update readme with more benchmarking instructions --- k8s/README.md | 43 +++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/k8s/README.md b/k8s/README.md index 0508327..bb0e2c5 100644 --- a/k8s/README.md +++ b/k8s/README.md @@ -6,6 +6,21 @@ The paved path is to execute the steps on a fresh 24.04 ubuntu ami, but the tool If that is the case you'll want to skip the install of `https://k3s.io/` from the `k3s` subcommand and proceed. The `machine_prep.sh` script should provide clues about the environment requirements you'll need to satisfy to operate th tool. +### Current status + +Help wanted! This code is rough in that it has too many steps and doesn't handle enough variety in machine/architecture to run unattended. PRs welcome to improve any and all of what is here. + +#### Known issues and quirks + +- These instructions are likely incomplete +- benchmark results for df_ray and spark do not ensure the same settings are used, they require the operator use the same versions +- the results subcommand just looks for the newest DataFusion for Ray and Spark results, and assumes those are the correct ones +- the machine_prep script does not handle errors + +### Sample results + +You can find results conducted with different versions of DataFusion for Ray in `docs/benchmarks/[dfray version]` + ### Benchmarking on a fresh ubuntu 24.04 LTS ami from amazon - provision the machine and ssh in and download the repo @@ -16,7 +31,7 @@ cd datafusion-ray git checkout k8s_benchmarking ``` -- then run the machine prep script +then run the machine prep script ````bash cd datafusion-ray/k8s @@ -44,4 +59,28 @@ This will: - install kuberay operater - install spark operator - +Next, lets generate TPCH data in the /data directory, scale factor 1 to start + +```bash +./bench_toolbox.py -v generate --data-path /data --scale-factor 1 --partitions 2 --pool-size 2 +``` + +Now we can run a benchmark with the generated data with DataFusion for Ray + +```bash +./bench_toolbox.py -v bench --executor-cpus 2 --executor-mem 10 --executor-num 2 --executor-overhead-mem 4 --driver-mem 4 --driver-cpus 2 --data-path /data --concurrency 8 --partitions-per-processor 4 --processor-pool-min 50 --df-ray-version 0.1.0rc1 --test-pypi --arm --scale-factor 1 --output-path /data df_ray +``` + +followed by spark. Make sure you use the same settings + +```bash +./bench_toolbox.py -v bench --executor-cpus 2 --executor-mem 10 --executor-num 2 --executor-overhead-mem 4 --driver-mem 4 --driver-cpus 2 --data-path /data --concurrency 8 --partitions-per-processor 4 --processor-pool-min 50 --df-ray-version 0.1.0rc1 --test-pypi --arm --scale-factor 1 --output-path /data spark +``` + +Lastly, compile the results: + +```bash +./bench_toolbox.py -v results --data-device /dev/nvme1n1 --data-path /data --scale-factor 1 --output-path /data +``` + +You should get a table of results similar to what you'd find in `docs/benchmarks` From 86f3c50956011d78fd7c6ec18152f7273d2bcb1f Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 20 Mar 2025 13:30:43 +0100 Subject: [PATCH 4/8] rename to DataFusion for Ray. black format python code --- Cargo.lock | 2 +- Cargo.toml | 2 +- README.md | 10 +-- datafusion_ray/core.py | 43 +++++++--- dev/release/README.md | 16 ++-- dev/release/check-rat-report.py | 4 +- dev/release/create-tarball.sh | 26 +++--- dev/release/generate-changelog.py | 95 ++++++++++++++------- dev/release/release-tarball.sh | 3 +- docs/README.md | 16 ++-- docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge | 4 +- docs/benchmarks/df-ray-0.1.0rc1/m7g.12x | 4 +- docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x | 4 +- docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge | 8 +- docs/contributing.md | 10 +-- examples/tips.py | 4 +- k8s/bench_toolbox.py | 28 ++++-- k8s/cmds.py | 36 +++++--- k8s/pricing.py | 47 +++++++--- k8s/spark_tpcbench.py | 38 ++++++--- src/codec.rs | 8 +- tests/test_context.py | 71 --------------- tpch/tpcbench.py | 12 ++- 23 files changed, 275 insertions(+), 216 deletions(-) delete mode 100644 tests/test_context.py diff --git a/Cargo.lock b/Cargo.lock index 287b5ae..84b9cfb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1604,7 +1604,7 @@ dependencies = [ [[package]] name = "datafusion_ray" -version = "0.1.0-rc1" +version = "0.1.0-rc2" dependencies = [ "anyhow", "arrow", diff --git a/Cargo.toml b/Cargo.toml index 651d044..065ec1b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -21,7 +21,7 @@ description = "DataFusion on Ray" homepage = "https://github.com/apache/datafusion-ray" repository = "https://github.com/apache/datafusion-ray" authors = ["Apache DataFusion "] -version = "0.1.0-rc1" +version = "0.1.0-rc2" edition = "2024" readme = "README.md" license = "Apache-2.0" diff --git a/README.md b/README.md index 540f44d..da3dd49 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,7 @@ under the License. --> -# DataFusion Ray +# DataFusion for Ray [![Apache licensed][license-badge]][license-url] [![Python Tests][actions-badge]][actions-url] @@ -32,13 +32,13 @@ ## Overview -DataFusion Ray is a distributed execution framework that enables DataFusion DataFrame and SQL queries to run on a +DataFusion for Ray is a distributed execution framework that enables DataFusion DataFrame and SQL queries to run on a Ray cluster. This integration allows users to leverage Ray's dynamic scheduling capabilities while executing queries in a distributed fashion. ## Execution Modes -DataFusion Ray supports two execution modes: +DataFusion for Ray supports two execution modes: ### Streaming Execution @@ -54,7 +54,7 @@ intermediate shuffle files that are persisted and used as input for the next sta ## Getting Started -See the [contributor guide] for instructions on building DataFusion Ray. +See the [contributor guide] for instructions on building DataFusion for Ray. Once installed, you can run queries using DataFusion's familiar API while leveraging the distributed execution capabilities of Ray. @@ -84,6 +84,6 @@ Contributions are welcome! Please open an issue or submit a pull request if you ## License -DataFusion Ray is licensed under Apache 2.0. +DataFusion for Ray is licensed under Apache 2.0. [contributor guide]: docs/contributing.md diff --git a/datafusion_ray/core.py b/datafusion_ray/core.py index 291c6dd..cff1234 100644 --- a/datafusion_ray/core.py +++ b/datafusion_ray/core.py @@ -86,7 +86,9 @@ async def wait_for(coros, name=""): # wrap the coro in a task to work with python 3.10 and 3.11+ where asyncio.wait semantics # changed to not accept any awaitable start = time.time() - done, _ = await asyncio.wait([asyncio.create_task(_ensure_coro(c)) for c in coros]) + done, _ = await asyncio.wait( + [asyncio.create_task(_ensure_coro(c)) for c in coros] + ) end = time.time() log.info(f"waiting for {name} took {end - start}s") for d in done: @@ -164,7 +166,9 @@ async def acquire(self, need=1): need_to_make = need - have if need_to_make > can_make: - raise Exception(f"Cannot allocate processors above {self.max_processors}") + raise Exception( + f"Cannot allocate processors above {self.max_processors}" + ) if need_to_make > 0: log.debug(f"creating {need_to_make} additional processors") @@ -193,9 +197,9 @@ def _new_processor(self): self.processors_ready.clear() processor_key = new_friendly_name() log.debug(f"starting processor: {processor_key}") - processor = DFRayProcessor.options(name=f"Processor : {processor_key}").remote( - processor_key - ) + processor = DFRayProcessor.options( + name=f"Processor : {processor_key}" + ).remote(processor_key) self.pool[processor_key] = processor self.processors_started.add(processor.start_up.remote()) self.available.add(processor_key) @@ -244,7 +248,9 @@ async def _wait_for_serve(self): async def all_done(self): log.info("calling processor all done") - refs = [processor.all_done.remote() for processor in self.pool.values()] + refs = [ + processor.all_done.remote() for processor in self.pool.values() + ] await wait_for(refs, "processors to be all done") log.info("all processors shutdown") @@ -287,7 +293,9 @@ async def update_plan( ) async def serve(self): - log.info(f"[{self.processor_key}] serving on {self.processor_service.addr()}") + log.info( + f"[{self.processor_key}] serving on {self.processor_service.addr()}" + ) await self.processor_service.serve() log.info(f"[{self.processor_key}] done serving") @@ -339,7 +347,9 @@ async def wait_for_ready(self): async def get_stage_addrs(self, stage_id: int): addrs = [ - sd.remote_addr for sd in self.stages.values() if sd.stage_id == stage_id + sd.remote_addr + for sd in self.stages.values() + if sd.stage_id == stage_id ] return addrs @@ -389,7 +399,10 @@ async def new_query( refs.append( isd.remote_processor.update_plan.remote( isd.stage_id, - {stage_id: val["child_addrs"] for (stage_id, val) in kid.items()}, + { + stage_id: val["child_addrs"] + for (stage_id, val) in kid.items() + }, isd.partition_group, isd.plan_bytes, ) @@ -421,7 +434,9 @@ async def sort_out_addresses(self): ] # sanity check - assert all([op == output_partitions[0] for op in output_partitions]) + assert all( + [op == output_partitions[0] for op in output_partitions] + ) output_partitions = output_partitions[0] for child_stage_isd in child_stage_datas: @@ -505,7 +520,9 @@ def collect(self) -> list[pa.RecordBatch]: ) log.debug(f"last stage addrs {last_stage_addrs}") - reader = self.df.read_final_stage(last_stage_id, last_stage_addrs[0]) + reader = self.df.read_final_stage( + last_stage_id, last_stage_addrs[0] + ) log.debug("got reader") self._batches = list(reader) return self._batches @@ -605,7 +622,9 @@ def register_csv(self, name: str, path: str): """ self.ctx.register_csv(name, path) - def register_listing_table(self, name: str, path: str, file_extention="parquet"): + def register_listing_table( + self, name: str, path: str, file_extention="parquet" + ): """ Register a directory of parquet files with the given name. The path can be a local filesystem path, absolute filesystem path, or a url. diff --git a/dev/release/README.md b/dev/release/README.md index e73858a..e19e084 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -17,7 +17,7 @@ under the License. --> -# DataFusion Ray Release Process +# DataFusion for Ray Release Process Development happens on the `main` branch, and most of the time, we depend on DataFusion using GitHub dependencies rather than using an official release from crates.io. This allows us to pick up new features and bug fixes frequently @@ -43,7 +43,7 @@ You will need a GitHub Personal Access Token. Follow [these instructions](https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/creating-a-personal-access-token) to generate one if you do not already have one. -You will need a PyPI API token. Create one at https://test.pypi.org/manage/account/#api-tokens, setting the “Scope” to +You will need a PyPI API token. Create one at , setting the “Scope” to “Entire account”. You will also need access to the [datafusion-ray](https://test.pypi.org/project/datafusion-ray/) project on testpypi. @@ -63,7 +63,7 @@ We maintain a `CHANGELOG.md` so our users know what has been changed between rel The changelog is generated using a Python script: ```bash -$ GITHUB_TOKEN= ./dev/release/generate-changelog.py 0.1.0 HEAD 0.2.0 > dev/changelog/0.2.0.md +GITHUB_TOKEN= ./dev/release/generate-changelog.py 0.1.0 HEAD 0.2.0 > dev/changelog/0.2.0.md ``` This script creates a changelog from GitHub PRs based on the labels associated with them as well as looking for @@ -91,7 +91,7 @@ git push apache 0.2.0-rc1 ./dev/release/create-tarball.sh 0.2.0 1 ``` -This will also create the email template to send to the mailing list. +This will also create the email template to send to the mailing list. Create a draft email using this content, but do not send until after completing the next step. @@ -104,7 +104,7 @@ This section assumes some familiarity with publishing Python packages to PyPi. F Pushing an `rc` tag to the release branch will cause a GitHub Workflow to run that will build the Python wheels. -Go to https://github.com/apache/datafusion-ray/actions and look for an action named "Python Release Build" +Go to and look for an action named "Python Release Build" that has run against the pushed tag. Click on the action and scroll down to the bottom of the page titled "Artifacts". Download `dist.zip`. It should @@ -207,10 +207,10 @@ git push apache 0.2.0 ### Add the release to Apache Reporter -Add the release to https://reporter.apache.org/addrelease.html?datafusion with a version name prefixed with `DATAFUSION-RAY`, +Add the release to with a version name prefixed with `DATAFUSION-RAY`, for example `DATAFUSION-RAY-0.2.0`. -The release information is used to generate a template for a board report (see example from Apache Arrow +The release information is used to generate a template for a board report (see example from Apache Arrow [here](https://github.com/apache/arrow/pull/14357)). ### Delete old RCs and Releases @@ -222,7 +222,7 @@ for more information. Release candidates should be deleted once the release is published. -Get a list of DataFusion release candidates: +Get a list of DataFusion for Ray release candidates: ```bash svn ls https://dist.apache.org/repos/dist/dev/datafusion | grep datafusion-ray diff --git a/dev/release/check-rat-report.py b/dev/release/check-rat-report.py index d3dd7c5..30a0111 100644 --- a/dev/release/check-rat-report.py +++ b/dev/release/check-rat-report.py @@ -23,7 +23,9 @@ import xml.etree.ElementTree as ET if len(sys.argv) != 3: - sys.stderr.write("Usage: %s exclude_globs.lst rat_report.xml\n" % sys.argv[0]) + sys.stderr.write( + "Usage: %s exclude_globs.lst rat_report.xml\n" % sys.argv[0] + ) sys.exit(1) exclude_globs_filename = sys.argv[1] diff --git a/dev/release/create-tarball.sh b/dev/release/create-tarball.sh index 778743c..4f2c178 100755 --- a/dev/release/create-tarball.sh +++ b/dev/release/create-tarball.sh @@ -48,9 +48,9 @@ SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" SOURCE_TOP_DIR="$(cd "${SOURCE_DIR}/../../" && pwd)" if [ "$#" -ne 2 ]; then - echo "Usage: $0 " - echo "ex. $0 4.1.0 2" - exit + echo "Usage: $0 " + echo "ex. $0 4.1.0 2" + exit fi version=$1 @@ -67,7 +67,7 @@ tarball=${distdir}/${tarname} url="https://dist.apache.org/repos/dist/dev/datafusion/${release}-rc${rc}" if [ -z "$release_hash" ]; then - echo "Cannot continue: unknown git tag: ${tag}" + echo "Cannot continue: unknown git tag: ${tag}" fi echo "Draft email for dev@datafusion.apache.org mailing list" @@ -75,10 +75,10 @@ echo "" echo "---------------------------------------------------------" cat < containing the files in git at $release_hash # the files in the tarball are prefixed with {version} (e.g. 4.0.1) mkdir -p ${distdir} -(cd "${SOURCE_TOP_DIR}" && git archive ${release_hash} --prefix ${release}/ | gzip > ${tarball}) +(cd "${SOURCE_TOP_DIR}" && git archive ${release_hash} --prefix ${release}/ | gzip >${tarball}) echo "Running rat license checker on ${tarball}" ${SOURCE_DIR}/run-rat.sh ${tarball} @@ -123,11 +122,10 @@ gpg --armor --output ${tarball}.asc --detach-sig ${tarball} # create signing with relative path of tarball # so that they can be verified with a command such as # shasum --check apache-datafusion-ray-4.1.0-rc2.tar.gz.sha512 -(cd ${distdir} && shasum -a 256 ${tarname}) > ${tarball}.sha256 -(cd ${distdir} && shasum -a 512 ${tarname}) > ${tarball}.sha512 - +(cd ${distdir} && shasum -a 256 ${tarname}) >${tarball}.sha256 +(cd ${distdir} && shasum -a 512 ${tarname}) >${tarball}.sha512 echo "Uploading to datafusion dist/dev to ${url}" svn co --depth=empty https://dist.apache.org/repos/dist/dev/datafusion ${SOURCE_TOP_DIR}/dev/dist svn add ${distdir} -svn ci -m "Apache DataFusion Ray ${version} ${rc}" ${distdir} +svn ci -m "Apache DataFusion for Ray ${version} ${rc}" ${distdir} diff --git a/dev/release/generate-changelog.py b/dev/release/generate-changelog.py index f8cf67b..9144746 100755 --- a/dev/release/generate-changelog.py +++ b/dev/release/generate-changelog.py @@ -22,20 +22,29 @@ import re import subprocess + def print_pulls(repo_name, title, pulls): - if len(pulls) > 0: + if len(pulls) > 0: print("**{}:**".format(title)) print() - for (pull, commit) in pulls: - url = "https://github.com/{}/pull/{}".format(repo_name, pull.number) - print("- {} [#{}]({}) ({})".format(pull.title, pull.number, url, commit.author.login)) + for pull, commit in pulls: + url = "https://github.com/{}/pull/{}".format( + repo_name, pull.number + ) + print( + "- {} [#{}]({}) ({})".format( + pull.title, pull.number, url, commit.author.login + ) + ) print() def generate_changelog(repo, repo_name, tag1, tag2, version): # get a list of commits between two tags - print(f"Fetching list of commits between {tag1} and {tag2}", file=sys.stderr) + print( + f"Fetching list of commits between {tag1} and {tag2}", file=sys.stderr + ) comparison = repo.compare(tag1, tag2) # get the pull requests for these commits @@ -61,29 +70,31 @@ def generate_changelog(repo, repo_name, tag1, tag2, version): # categorize the pull requests based on GitHub labels print("Categorizing pull requests", file=sys.stderr) - for (pull, commit) in all_pulls: + for pull, commit in all_pulls: # see if PR title uses Conventional Commits - cc_type = '' - cc_scope = '' - cc_breaking = '' - parts = re.findall(r'^([a-z]+)(\([a-z]+\))?(!)?:', pull.title) + cc_type = "" + cc_scope = "" + cc_breaking = "" + parts = re.findall(r"^([a-z]+)(\([a-z]+\))?(!)?:", pull.title) if len(parts) == 1: parts_tuple = parts[0] - cc_type = parts_tuple[0] # fix, feat, docs, chore - cc_scope = parts_tuple[1] # component within project - cc_breaking = parts_tuple[2] == '!' + cc_type = parts_tuple[0] # fix, feat, docs, chore + cc_scope = parts_tuple[1] # component within project + cc_breaking = parts_tuple[2] == "!" labels = [label.name for label in pull.labels] - if 'api change' in labels or cc_breaking: + if "api change" in labels or cc_breaking: breaking.append((pull, commit)) - elif 'bug' in labels or cc_type == 'fix': + elif "bug" in labels or cc_type == "fix": bugs.append((pull, commit)) - elif 'performance' in labels or cc_type == 'perf': + elif "performance" in labels or cc_type == "perf": performance.append((pull, commit)) - elif 'enhancement' in labels or cc_type == 'feat': + elif "enhancement" in labels or cc_type == "feat": enhancements.append((pull, commit)) - elif 'documentation' in labels or cc_type == 'docs' or cc_type == 'doc': + elif ( + "documentation" in labels or cc_type == "docs" or cc_type == "doc" + ): docs.append((pull, commit)) else: other.append((pull, commit)) @@ -92,7 +103,8 @@ def generate_changelog(repo, repo_name, tag1, tag2, version): print("Generating changelog content", file=sys.stderr) # ASF header - print("""\n""") +-->\n""" + ) - print(f"# Apache DataFusion Ray {version} Changelog\n") + print(f"# Apache DataFusion for Ray {version} Changelog\n") # get the number of commits - commit_count = subprocess.check_output(f"git log --pretty=oneline {tag1}..{tag2} | wc -l", shell=True, text=True).strip() + commit_count = subprocess.check_output( + f"git log --pretty=oneline {tag1}..{tag2} | wc -l", + shell=True, + text=True, + ).strip() # get number of contributors - contributor_count = subprocess.check_output(f"git shortlog -sn {tag1}..{tag2} | wc -l", shell=True, text=True).strip() + contributor_count = subprocess.check_output( + f"git shortlog -sn {tag1}..{tag2} | wc -l", shell=True, text=True + ).strip() - print(f"This release consists of {commit_count} commits from {contributor_count} contributors. " - f"See credits at the end of this changelog for more information.\n") + print( + f"This release consists of {commit_count} commits from {contributor_count} contributors. " + f"See credits at the end of this changelog for more information.\n" + ) print_pulls(repo_name, "Breaking changes", breaking) print_pulls(repo_name, "Performance related", performance) @@ -130,17 +151,24 @@ def generate_changelog(repo, repo_name, tag1, tag2, version): print_pulls(repo_name, "Other", other) # show code contributions - credits = subprocess.check_output(f"git shortlog -sn {tag1}..{tag2}", shell=True, text=True).rstrip() + credits = subprocess.check_output( + f"git shortlog -sn {tag1}..{tag2}", shell=True, text=True + ).rstrip() print("## Credits\n") - print("Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) " - "per contributor.\n") + print( + "Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) " + "per contributor.\n" + ) print("```") print(credits) print("```\n") - print("Thank you also to everyone who contributed in other ways such as filing issues, reviewing " - "PRs, and providing feedback on this release.\n") + print( + "Thank you also to everyone who contributed in other ways such as filing issues, reviewing " + "PRs, and providing feedback on this release.\n" + ) + def cli(args=None): """Process command line arguments.""" @@ -150,7 +178,9 @@ def cli(args=None): parser = argparse.ArgumentParser() parser.add_argument("tag1", help="The previous commit or tag (e.g. 0.1.0)") parser.add_argument("tag2", help="The current commit or tag (e.g. HEAD)") - parser.add_argument("version", help="The version number to include in the changelog") + parser.add_argument( + "version", help="The version number to include in the changelog" + ) args = parser.parse_args() token = os.getenv("GITHUB_TOKEN") @@ -160,5 +190,6 @@ def cli(args=None): repo = g.get_repo(project) generate_changelog(repo, project, args.tag1, args.tag2, args.version) + if __name__ == "__main__": - cli() \ No newline at end of file + cli() diff --git a/dev/release/release-tarball.sh b/dev/release/release-tarball.sh index 0225a37..98e8612 100755 --- a/dev/release/release-tarball.sh +++ b/dev/release/release-tarball.sh @@ -30,7 +30,6 @@ # # Based in part on post-01-upload.sh from apache/arrow - set -e set -u @@ -65,7 +64,7 @@ cp -r ${tmp_dir}/dev/* ${tmp_dir}/release/${release_version}/ svn add ${tmp_dir}/release/${release_version} echo "Commit release" -svn ci -m "Apache DataFusion Ray ${version}" ${tmp_dir}/release +svn ci -m "Apache DataFusion for Ray ${version}" ${tmp_dir}/release echo "Clean up" rm -rf ${tmp_dir} diff --git a/docs/README.md b/docs/README.md index 762751c..1b330ed 100644 --- a/docs/README.md +++ b/docs/README.md @@ -17,15 +17,15 @@ under the License. --> -# DataFusion Ray Design Documentation +# DataFusion for Ray Design Documentation -DataFusion Ray is a distributed SQL query engine that is powered by DataFusion and Ray. +DataFusion for Ray is a distributed SQL query engine that is powered by DataFusion and Ray. DataFusion provides a high-performance query engine that is already partition-aware, with partitions being executed -in parallel in separate threads. DataFusion Ray provides a distributed query planner that translates a DataFusion physical +in parallel in separate threads. DataFusion for Ray provides a distributed query planner that translates a DataFusion physical plan into a distributed plan. -Note that this document is dated from an early implementation of DataFusion Ray. The details around shuffle differ in the current ArrowFlight Streaming based implementation. +Note that this document is dated from an early implementation of DataFusion for Ray. The details around shuffle differ in the current ArrowFlight Streaming based implementation. However the general discussion around how to break a physical plan into discrete stages remains useful, and we retain this document here. Let's walk through an example to see how that works. We'll use [SQLBench-H](https://github.com/sql-benchmarks/sqlbench-h) @@ -123,7 +123,7 @@ GlobalLimitExec: skip=0, fetch=10 ## Partitioning & Distribution The partitioning scheme changes throughout the plan and this is the most important concept to -understand in order to understand DataFusion Ray's design. Changes in partitioning are implemented by the `RepartitionExec` +understand in order to understand DataFusion for Ray's design. Changes in partitioning are implemented by the `RepartitionExec` operator in DataFusion and are happen in the following scenarios. ### Joins @@ -155,7 +155,7 @@ Sort also has multiple approaches. - The input partitions can be collapsed down to a single partition and then sorted - Partitions can be sorted in parallel and then merged using a sort-preserving merge -DataFusion and DataFusion Ray currently choose the first approach, but there is a DataFusion PR open for implementing the second. +DataFusion and DataFusion for Ray currently choose the first approach, but there is a DataFusion PR open for implementing the second. ### Limit @@ -262,10 +262,10 @@ child plans, building up a DAG of futures. The output of each query stage needs to be persisted somewhere so that the next query stage can read it. -DataFusion Ray uses the Ray object store as a shared file system, which was proposed [here](https://github.com/datafusion-contrib/ray-sql/issues/22) and implemented [here](https://github.com/datafusion-contrib/ray-sql/pull/33). +DataFusion for Ray uses the Ray object store as a shared file system, which was proposed [here](https://github.com/datafusion-contrib/ray-sql/issues/22) and implemented [here](https://github.com/datafusion-contrib/ray-sql/pull/33). DataFusion's `RepartitionExec` uses threads and channels within a single process and is not suitable for a -distributed query engine, so DataFusion Ray rewrites the physical plan and replaces the `RepartionExec` with a pair of +distributed query engine, so DataFusion for Ray rewrites the physical plan and replaces the `RepartionExec` with a pair of operators to perform a "shuffle". These are the `ShuffleWriterExec` and `ShuffleReaderExec`. ### Shuffle Writes diff --git a/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge b/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge index 7044aca..5b06693 100644 --- a/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge +++ b/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge @@ -1,5 +1,5 @@ ================================================================================================= -Spark and DataFusionRay TPCH 100 Benchmarks +Spark and DataFusion for Ray TPCH 100 Benchmarks Machine: i4g.4xlarge Machine On Demand Cost: 1.23552 $/hr CPU(s): Neoverse-N1 16x @@ -13,7 +13,7 @@ df-ray cost: $0.2989 +1.04x cheaper spark duration: 906.69s spark cost: $0.3112 -DataFusionRay Settings: +DataFusion for Ray Settings: concurrency: 16 batch_size : 8192 partitions_per_processor: 4 diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x index 19ffb1d..7a4085e 100644 --- a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x @@ -1,11 +1,11 @@ ========================================================================================== -Spark and DataFusionRay TPCH 100 Benchmarks +Spark and DataFusion for Ray TPCH 100 Benchmarks Machine: m7g.12xlarge CPU(s): 48x MEM: 189G HD Throughput: 167.53 MB/s (from hdparm) -DataFusionRay Settings: +DataFusion for Ray Settings: concurrency: 16 batch_size : 8192 partitions_per_processor: 2 diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x b/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x index 01fd82d..3a5e37e 100644 --- a/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x @@ -1,11 +1,11 @@ ========================================================================================== -Spark and DataFusionRay TPCH 100 Benchmarks +Spark and DataFusion for Ray TPCH 100 Benchmarks Machine: m7gd.12xlarge CPU(s): 48x MEM: 189G HD Throughput: 2162.93 (from hdparm) -DataFusionRay Settings: +DataFusion for Ray Settings: concurrency: 16 batch_size : 8192 partitions_per_processor: 2 diff --git a/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge b/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge index 7aacdf9..bd9d435 100644 --- a/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge +++ b/docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge @@ -1,11 +1,11 @@ ========================================================================================== -Spark and DataFusionRay TPCH 100 Benchmarks +Spark and DataFusion for Ray TPCH 100 Benchmarks Machine: r8g.8xlarge CPU(s): 32x MEM: 252G HD Throughput: 167.50 (from hdparm) -DataFusionRay Settings: +DataFusion for Ray Settings: concurrency: 16 batch_size : 8192 partitions_per_processor: 8 @@ -53,13 +53,13 @@ DataFrame() +------------+--------------------+--------------------+---------------------+---------------+ ========================================================================================== -Spark and DataFusionRay TPCH 100 Benchmarks +Spark and DataFusion for Ray TPCH 100 Benchmarks Machine: r8g.8xlarge CPU(s): 32x MEM: 252G HD Throughput: 169.07 (from hdparm) -DataFusionRay Settings: +DataFusion for Ray Settings: concurrency: 16 batch_size : 8192 partitions_per_processor: 8 diff --git a/docs/contributing.md b/docs/contributing.md index 6dffbd0..b209c22 100644 --- a/docs/contributing.md +++ b/docs/contributing.md @@ -17,7 +17,7 @@ under the License. --> -# DataFusion Ray Contributor Guide +# DataFusion for Ray Contributor Guide ## Building @@ -83,7 +83,7 @@ RAY_COLOR_PREFIX=1 RAY_DEDUP_LOGS=0 python tips.py --data-dir=$(pwd)/../testdata RAY_COLOR_PREFIX=1 RAY_DEDUP_LOGS=0 python tpc.py --data=file:///path/to/your/tpch/directory/ --concurrency=2 --batch-size=8182 --worker-pool-min=10 --qnum 2 ``` -To execute the TPCH query #2. To execute an arbitrary query against the TPCH dataset, provide it with `--query` instead of `--qnum`. This is useful for validating plans that DataFusion Ray will create. +To execute the TPCH query #2. To execute an arbitrary query against the TPCH dataset, provide it with `--query` instead of `--qnum`. This is useful for validating plans that DataFusion for Ray will create. For example, to execute the following query: @@ -105,13 +105,13 @@ This will output a json file in the current directory with query timings. ## Logging -DataFusion Ray's logging output is determined by the `DATAFUSION_RAY_LOG_LEVEL` environment variable. The default log level is `WARN`. To change the log level, set the environment variable to one of the following values: `ERROR`, `WARN`, `INFO`, `DEBUG`, or `TRACE`. +DataFusion for Ray's logging output is determined by the `DATAFUSION_RAY_LOG_LEVEL` environment variable. The default log level is `WARN`. To change the log level, set the environment variable to one of the following values: `ERROR`, `WARN`, `INFO`, `DEBUG`, or `TRACE`. -DataFusion Ray outputs logs from both python and rust, and in order to handle this consistently, the python logger for `datafusion_ray` is routed to rust for logging. The `RUST_LOG` environment variable can be used to control other rust log output other than `datafusion_ray`. +DataFusion for Ray outputs logs from both python and rust, and in order to handle this consistently, the python logger for `datafusion_ray` is routed to rust for logging. The `RUST_LOG` environment variable can be used to control other rust log output other than `datafusion_ray`. ## Status -- DataFusion Ray can execute all TPCH queries. Tested up to SF100. +- DataFusion for Ray can execute all TPCH queries. Tested up to SF100. ## Known Issues diff --git a/examples/tips.py b/examples/tips.py index 7537f5f..bd1d396 100644 --- a/examples/tips.py +++ b/examples/tips.py @@ -36,7 +36,9 @@ def go(data_dir: str): if __name__ == "__main__": ray.init(namespace="tips", runtime_env=df_ray_runtime_env) parser = argparse.ArgumentParser() - parser.add_argument("--data-dir", required=True, help="path to tips.parquet files") + parser.add_argument( + "--data-dir", required=True, help="path to tips.parquet files" + ) args = parser.parse_args() go(args.data_dir) diff --git a/k8s/bench_toolbox.py b/k8s/bench_toolbox.py index d01fdfc..54fd974 100755 --- a/k8s/bench_toolbox.py +++ b/k8s/bench_toolbox.py @@ -80,7 +80,6 @@ def cli(dry_run: bool, verbose: bool): help="path to the local directory exposed via PVC", required=True, ) - @click.option( "--concurrency", type=int, @@ -100,7 +99,10 @@ def cli(dry_run: bool, verbose: bool): required=True, ) @click.option( - "--df-ray-version", type=str, help="version number of DFRay to use", required=True + "--df-ray-version", + type=str, + help="version number of DFRay to use", + required=True, ) @click.option( "--test-pypi", @@ -156,11 +158,17 @@ def bench(system, **kwargs): def results(data_path, data_device, scale_factor, output_path): df_result = json.loads( open( - newest_file(glob.glob(os.path.join(output_path, "datafusion-ray*json"))) + newest_file( + glob.glob(os.path.join(output_path, "datafusion-ray*json")) + ) ).read() ) spark_result = json.loads( - open(newest_file(glob.glob(os.path.join(output_path, "spark-tpch*json")))).read() + open( + newest_file( + glob.glob(os.path.join(output_path, "spark-tpch*json")) + ) + ).read() ) print(df_result) print(spark_result) @@ -181,7 +189,9 @@ def results(data_path, data_device, scale_factor, output_path): df["change_text"] = df["change"].apply( lambda change: ( - f"+{(1 / change):.2f}x faster" if change < 1.0 else f" {change:.2f}x slower" + f"+{(1 / change):.2f}x faster" + if change < 1.0 + else f" {change:.2f}x slower" ) ) df["tpch_query"] = [f"{i}" for i in range(1, 23)] + ["total"] @@ -190,7 +200,9 @@ def results(data_path, data_device, scale_factor, output_path): ts = time.time() df.to_parquet(f"datafusion-ray-spark-comparison-{ts}.parquet") ctx = datafusion.SessionContext() - ctx.register_parquet("results", f"datafusion-ray-spark-comparison-{ts}.parquet") + ctx.register_parquet( + "results", f"datafusion-ray-spark-comparison-{ts}.parquet" + ) cpu = subprocess.run( "lscpu | grep 'Model name' |awk '{print $3}'", @@ -244,7 +256,7 @@ def results(data_path, data_device, scale_factor, output_path): print("=" * 97) # the formatting code is terrible here, but it works for now header = [ - "Spark and DataFusionRay TPCH 100 Benchmarks", + "Spark and DataFusion for Ray TPCH 100 Benchmarks", f"{'Machine:':<30}{machine}", f"{'Machine On Demand Cost:':<30}{hourly_cost} $/hr", f"{'CPU(s):':<30}{cpu} {quantity}x", @@ -258,7 +270,7 @@ def results(data_path, data_device, scale_factor, output_path): f"{'spark duration:':<30}{spark_duration:>10}", f"{'spark cost:':<30}{spark_cost:>10}", "", - "DataFusionRay Settings:", + "DataFusion for Ray Settings:", f"{'concurrency:':<30}{df_result['settings']['concurrency']:>10}", f"{'batch_size :':<30}{df_result['settings']['batch_size']:>10}", f"{'partitions_per_processor:':<30}{df_result['settings']['partitions_per_processor']:>10}", diff --git a/k8s/cmds.py b/k8s/cmds.py index 586b63a..f5c6180 100644 --- a/k8s/cmds.py +++ b/k8s/cmds.py @@ -87,7 +87,7 @@ wget https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar && \ aws s3 cp hadoop-aws-3.3.4.jar {{ data_path.replace('s3a','s3') }}/hadoop-aws-3.3.4.jar """, - "getting additional spark jars" + "getting additional spark jars", ), Shell( "kubectl apply -f spark_job.yaml", @@ -106,13 +106,15 @@ """, "checking on job status", ), - #Shell( + # Shell( # "kubectl delete -f spark_job.yaml", # "tear down job", - #), + # ), ], "bench_df_ray": [ - Template("ray_cluster.yaml.template", "rewrite ray_cluster.yaml.template"), + Template( + "ray_cluster.yaml.template", "rewrite ray_cluster.yaml.template" + ), Shell( "kubectl apply -f ray_cluster.yaml", "deploying ray cluster", @@ -121,7 +123,9 @@ "kubectl wait raycluster/datafusion-ray-cluster --for='jsonpath={.status.state}'=ready --timeout=300s", "wait for ray cluster to be ready", ), - Template("requirements.txt.template", "rewrite requirements.txt.template"), + Template( + "requirements.txt.template", "rewrite requirements.txt.template" + ), Template("ray_job.sh.template", "rewrite ray_job.sh.template"), BackgroundShell( "kubectl port-forward svc/datafusion-ray-cluster-head-svc 8265:8265", @@ -177,7 +181,9 @@ def run_commands( for command in commands: match (self.dry_run, command): case (False, Shell(cmd, desc)): - self.run_shell_command(textwrap.dedent(cmd), desc, substitutions) + self.run_shell_command( + textwrap.dedent(cmd), desc, substitutions + ) case (True, Shell(cmd, desc)): click.secho(f"[dry run] {desc} ...") @@ -185,7 +191,10 @@ def run_commands( case (False, BackgroundShell(cmd, desc)): self.run_shell_command( - textwrap.dedent(cmd), desc, substitutions, background=True + textwrap.dedent(cmd), + desc, + substitutions, + background=True, ) case (True, BackgroundShell(cmd, desc)): @@ -198,7 +207,9 @@ def run_commands( case (True, Template(path, desc)): click.secho(f"[dry run] {desc} ...") - click.secho(f" {path} subs:{substitutions}", fg="yellow") + click.secho( + f" {path} subs:{substitutions}", fg="yellow" + ) case (False, ChangeDir(path, desc)): click.secho(f"{desc} ...") @@ -215,7 +226,9 @@ def run_commands( click.secho(f"[dry run] {desc} ...") case _: - raise Exception("Unhandled case in match. Shouldn't happen") + raise Exception( + "Unhandled case in match. Shouldn't happen" + ) def run_shell_command( self, @@ -261,7 +274,10 @@ def run_shell_command( exit(1) def process_template( - self, template_name: str, output_path: str, substitutions: dict[str, str] | None + self, + template_name: str, + output_path: str, + substitutions: dict[str, str] | None, ): template_out = template_name[: template_name.index(".template")] output_path = os.path.join(output_path, template_out) diff --git a/k8s/pricing.py b/k8s/pricing.py index 8b825b3..9a85e12 100644 --- a/k8s/pricing.py +++ b/k8s/pricing.py @@ -33,11 +33,27 @@ def get_on_demand_price(region, instance_type): response = pricing_client.get_products( ServiceCode="AmazonEC2", Filters=[ - {"Type": "TERM_MATCH", "Field": "instanceType", "Value": instance_type}, - {"Type": "TERM_MATCH", "Field": "operatingSystem", "Value": "Linux"}, - {"Type": "TERM_MATCH", "Field": "preInstalledSw", "Value": "NA"}, + { + "Type": "TERM_MATCH", + "Field": "instanceType", + "Value": instance_type, + }, + { + "Type": "TERM_MATCH", + "Field": "operatingSystem", + "Value": "Linux", + }, + { + "Type": "TERM_MATCH", + "Field": "preInstalledSw", + "Value": "NA", + }, {"Type": "TERM_MATCH", "Field": "tenancy", "Value": "Shared"}, - {"Type": "TERM_MATCH", "Field": "capacitystatus", "Value": "Used"}, + { + "Type": "TERM_MATCH", + "Field": "capacitystatus", + "Value": "Used", + }, { "Type": "TERM_MATCH", "Field": "location", @@ -50,7 +66,9 @@ def get_on_demand_price(region, instance_type): price_data = eval(response["PriceList"][0]) terms = price_data["terms"]["OnDemand"] price_dimensions = next(iter(terms.values()))["priceDimensions"] - price = next(iter(price_dimensions.values()))["pricePerUnit"]["USD"] + price = next(iter(price_dimensions.values()))["pricePerUnit"][ + "USD" + ] return float(price) except Exception as e: print( @@ -79,8 +97,10 @@ def request(): if nt: kwargs["nextToken"] = nt - response = savingsplans_client.describe_savings_plans_offering_rates( - **kwargs + response = ( + savingsplans_client.describe_savings_plans_offering_rates( + **kwargs + ) ) results.extend(response["searchResults"]) if len(response["nextToken"]) > 0: @@ -106,7 +126,9 @@ def request(): key = f"{duration_years:.1f}y" payment_option = ( - result["savingsPlanOffering"]["paymentOption"].lower().split()[0] + result["savingsPlanOffering"]["paymentOption"] + .lower() + .split()[0] ) # 'no', 'partial', or 'all' rate = float(result["rate"]) @@ -115,9 +137,14 @@ def request(): return ( rates - if any(any(v is not None for v in year.values()) for year in rates.values()) + if any( + any(v is not None for v in year.values()) + for year in rates.values() + ) else None ) except Exception as e: - print(f"Error getting reserved cost for {instance_type} in {region}: {str(e)}") + print( + f"Error getting reserved cost for {instance_type} in {region}: {str(e)}" + ) return None diff --git a/k8s/spark_tpcbench.py b/k8s/spark_tpcbench.py index 18bdf88..7ffef3d 100644 --- a/k8s/spark_tpcbench.py +++ b/k8s/spark_tpcbench.py @@ -25,17 +25,28 @@ import boto3 -def main(benchmark: str, data_path: str, query_path: str, output_path: str, name: str): +def main( + benchmark: str, + data_path: str, + query_path: str, + output_path: str, + name: str, +): # Initialize a SparkSession - spark = SparkSession.builder \ - .appName( f"{name} benchmark derived from {benchmark}") \ - .getOrCreate() + spark = SparkSession.builder.appName( + f"{name} benchmark derived from {benchmark}" + ).getOrCreate() - spark.conf.set("spark.hadoop.fs.s3a.aws.credentials.provider", "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider") - spark.conf.set("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") + spark.conf.set( + "spark.hadoop.fs.s3a.aws.credentials.provider", + "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider", + ) + spark.conf.set( + "spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem" + ) - # Register the tables + # Register the tables num_queries = 22 table_names = [ "customer", @@ -85,7 +96,10 @@ def main(benchmark: str, data_path: str, query_path: str, output_path: str, name text = f.read() # each file can contain multiple queries queries = list( - filter(lambda x: len(x) > 0, map(lambda x: x.strip(), text.split(";"))) + filter( + lambda x: len(x) > 0, + map(lambda x: x.strip(), text.split(";")), + ) ) start_time = time.time() @@ -113,7 +127,9 @@ def main(benchmark: str, data_path: str, query_path: str, output_path: str, name out = json.dumps(results, indent=4) current_time_millis = int(datetime.now().timestamp() * 1000) - results_path = f"{output_path}/{name}-{benchmark}-{current_time_millis}.json" + results_path = ( + f"{output_path}/{name}-{benchmark}-{current_time_millis}.json" + ) print(f"Writing results to {results_path}") with open(results_path, "w") as f: f.write(out) @@ -137,7 +153,9 @@ def main(benchmark: str, data_path: str, query_path: str, output_path: str, name parser.add_argument("--queries", required=True, help="Path to query files") parser.add_argument("--output", required=True, help="Path to write output") parser.add_argument( - "--name", required=True, help="Prefix for result file e.g. spark/comet/gluten" + "--name", + required=True, + help="Prefix for result file e.g. spark/comet/gluten", ) args = parser.parse_args() print(f"parsed is {args}") diff --git a/src/codec.rs b/src/codec.rs index b817e13..d10f10e 100644 --- a/src/codec.rs +++ b/src/codec.rs @@ -17,8 +17,8 @@ use datafusion::{ physical_plan::ExecutionPlan, }; use datafusion_proto::physical_plan::{ - from_proto::parse_protobuf_partitioning, to_proto::serialize_partitioning, - DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, + DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, from_proto::parse_protobuf_partitioning, + to_proto::serialize_partitioning, }; use datafusion_proto::protobuf; @@ -27,7 +27,7 @@ use prost::Message; use crate::stage_reader::DFRayStageReaderExec; #[derive(Debug)] -/// Physical Extension Codec for for DataFusion Ray plans +/// Physical Extension Codec for for DataFusion for Ray plans pub struct RayCodec {} impl PhysicalExtensionCodec for RayCodec { @@ -154,7 +154,7 @@ mod test { use crate::stage_reader::DFRayStageReaderExec; use arrow::datatypes::DataType; use datafusion::{ - physical_plan::{display::DisplayableExecutionPlan, displayable, Partitioning}, + physical_plan::{Partitioning, display::DisplayableExecutionPlan, displayable}, prelude::SessionContext, }; use datafusion_proto::physical_plan::AsExecutionPlan; diff --git a/tests/test_context.py b/tests/test_context.py deleted file mode 100644 index 602f761..0000000 --- a/tests/test_context.py +++ /dev/null @@ -1,71 +0,0 @@ -# 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. - -from datafusion_ray.context import DatafusionRayContext -from datafusion import SessionContext, SessionConfig, RuntimeConfig, col, lit, functions as F -import pytest - -@pytest.fixture -def df_ctx(): - """Fixture to create a DataFusion context.""" - # used fixed partition count so that tests are deterministic on different environments - config = SessionConfig().with_target_partitions(4) - return SessionContext(config=config) - -@pytest.fixture -def ctx(df_ctx): - """Fixture to create a Datafusion Ray context.""" - return DatafusionRayContext(df_ctx) - -def test_basic_query_succeed(df_ctx, ctx): - df_ctx.register_csv("tips", "examples/tips.csv", has_header=True) - record_batches = ctx.sql("SELECT * FROM tips") - assert len(record_batches) <= 4 - num_rows = sum(batch.num_rows for batch in record_batches) - assert num_rows == 244 - -def test_aggregate_csv(df_ctx, ctx): - df_ctx.register_csv("tips", "examples/tips.csv", has_header=True) - record_batches = ctx.sql("select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker") - assert len(record_batches) <= 4 - num_rows = sum(batch.num_rows for batch in record_batches) - assert num_rows == 4 - -def test_aggregate_parquet(df_ctx, ctx): - df_ctx.register_parquet("tips", "examples/tips.parquet") - record_batches = ctx.sql("select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker") - assert len(record_batches) <= 4 - num_rows = sum(batch.num_rows for batch in record_batches) - assert num_rows == 4 - -def test_aggregate_parquet_dataframe(df_ctx, ctx): - df = df_ctx.read_parquet(f"examples/tips.parquet") - df = ( - df.aggregate( - [col("sex"), col("smoker"), col("day"), col("time")], - [F.avg(col("tip") / col("total_bill")).alias("tip_pct")], - ) - .filter(col("day") != lit("Dinner")) - .aggregate([col("sex"), col("smoker")], [F.avg(col("tip_pct")).alias("avg_pct")]) - ) - ray_results = ctx.plan(df.execution_plan()) - df_ctx.create_dataframe([ray_results]).show() - - -def test_no_result_query(df_ctx, ctx): - df_ctx.register_csv("tips", "examples/tips.csv", has_header=True) - ctx.sql("CREATE VIEW tips_view AS SELECT * FROM tips") diff --git a/tpch/tpcbench.py b/tpch/tpcbench.py index 67234f3..9f886bd 100644 --- a/tpch/tpcbench.py +++ b/tpch/tpcbench.py @@ -108,7 +108,9 @@ def main( sql = tpch_query(qnum) statements = list( - filter(lambda x: len(x) > 0, map(lambda x: x.strip(), sql.split(";"))) + filter( + lambda x: len(x) > 0, map(lambda x: x.strip(), sql.split(";")) + ) ) start_time = time.time() @@ -122,7 +124,9 @@ def main( calculated = "\n".join([prettify(b) for b in all_batches]) print(calculated) - out_path = os.path.join(output_path, f"datafusion_ray_tpch_q{qnum}_result.txt") + out_path = os.path.join( + output_path, f"datafusion_ray_tpch_q{qnum}_result.txt" + ) with open(out_path, "w") as f: f.write(calculated) @@ -161,7 +165,9 @@ def main( parser.add_argument( "--concurrency", required=True, help="Number of concurrent tasks" ) - parser.add_argument("--qnum", type=int, default=-1, help="TPCH query number, 1-22") + parser.add_argument( + "--qnum", type=int, default=-1, help="TPCH query number, 1-22" + ) parser.add_argument("--listing-tables", action="store_true") parser.add_argument("--validate", action="store_true") parser.add_argument( From 0cc9466138d889930b13ef7c12574d3d51f231db Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 20 Mar 2025 13:31:39 +0100 Subject: [PATCH 5/8] updates to better support s3 and local for benchmarking --- docs/benchmarks/df-ray-0.1.0rc1/m7g.12x | 63 +++++++++++++++++++++++++ k8s/bench_toolbox.py | 37 ++++++--------- k8s/cmds.py | 54 +++++++++------------ k8s/spark_job.yaml.template | 6 ++- k8s/spark_tpcbench.py | 40 ++++------------ 5 files changed, 115 insertions(+), 85 deletions(-) diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x index 7a4085e..4a8b9cf 100644 --- a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12x @@ -1,3 +1,66 @@ +================================================================================= +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: m7g.12xlarge +Machine On Demand Cost: 1.9584 $/hr +CPU(s): Neoverse-V1 48x +MEM: 189G +HD Throughput: 166.46 MB/s (from hdparm) +Data Location: /data/sf100 + +df-ray duration: 204.97s +4.75x faster +df-ray cost: $0.1115 +4.75x cheaper + +spark duration: 974.60s +spark cost: $0.5302 + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 1 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +================================================================================= +DataFrame() ++------------+------------+------------+-----------------------+----------------+ +| tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | ++------------+------------+------------+-----------------------+----------------+ +| 1 | 41.0527 | 7.3332 | 0.1786 | +5.60x faster | +| 2 | 5.2886 | 3.3632 | 0.6359 | +1.57x faster | +| 3 | 8.8360 | 5.8260 | 0.6593 | +1.52x faster | +| 4 | 6.7364 | 2.8624 | 0.4249 | +2.35x faster | +| 5 | 113.1327 | 9.7685 | 0.0863 | +11.58x faster | +| 6 | 4.0279 | 5.0924 | 1.2643 | 1.26x slower | +| 7 | 10.3482 | 15.0313 | 1.4526 | 1.45x slower | +| 8 | 197.1830 | 10.5564 | 0.0535 | +18.68x faster | +| 9 | 244.0854 | 17.0069 | 0.0697 | +14.35x faster | +| 10 | 25.6980 | 10.1537 | 0.3951 | +2.53x faster | +| 11 | 2.0117 | 3.2634 | 1.6222 | 1.62x slower | +| 12 | 20.2309 | 4.5518 | 0.2250 | +4.44x faster | +| 13 | 8.1399 | 3.5019 | 0.4302 | +2.32x faster | +| 14 | 2.5995 | 3.3954 | 1.3062 | 1.31x slower | +| 15 | 6.7179 | 6.2090 | 0.9243 | +1.08x faster | +| 16 | 4.2510 | 3.1190 | 0.7337 | +1.36x faster | +| 17 | 119.6635 | 21.6696 | 0.1811 | +5.52x faster | +| 18 | 61.9433 | 32.2979 | 0.5214 | +1.92x faster | +| 19 | 4.1546 | 5.9170 | 1.4242 | 1.42x slower | +| 20 | 5.0565 | 5.5918 | 1.1059 | 1.11x slower | +| 21 | 78.7263 | 26.4413 | 0.3359 | +2.98x faster | +| 22 | 4.7114 | 2.0223 | 0.4292 | +2.33x faster | +| total | 974.5953 | 204.9744 | 0.2103 | +4.75x faster | ++------------+------------+------------+-----------------------+----------------+ + + ========================================================================================== Spark and DataFusion for Ray TPCH 100 Benchmarks Machine: m7g.12xlarge diff --git a/k8s/bench_toolbox.py b/k8s/bench_toolbox.py index 54fd974..83a8980 100755 --- a/k8s/bench_toolbox.py +++ b/k8s/bench_toolbox.py @@ -80,6 +80,7 @@ def cli(dry_run: bool, verbose: bool): help="path to the local directory exposed via PVC", required=True, ) + @click.option( "--concurrency", type=int, @@ -99,10 +100,7 @@ def cli(dry_run: bool, verbose: bool): required=True, ) @click.option( - "--df-ray-version", - type=str, - help="version number of DFRay to use", - required=True, + "--df-ray-version", type=str, help="version number of DFRay to use", required=True ) @click.option( "--test-pypi", @@ -158,17 +156,11 @@ def bench(system, **kwargs): def results(data_path, data_device, scale_factor, output_path): df_result = json.loads( open( - newest_file( - glob.glob(os.path.join(output_path, "datafusion-ray*json")) - ) + newest_file(glob.glob(os.path.join(output_path, "datafusion-ray*json"))) ).read() ) spark_result = json.loads( - open( - newest_file( - glob.glob(os.path.join(output_path, "spark-tpch*json")) - ) - ).read() + open(newest_file(glob.glob(os.path.join(output_path, "spark-tpch*json")))).read() ) print(df_result) print(spark_result) @@ -189,20 +181,21 @@ def results(data_path, data_device, scale_factor, output_path): df["change_text"] = df["change"].apply( lambda change: ( - f"+{(1 / change):.2f}x faster" - if change < 1.0 - else f" {change:.2f}x slower" + f"+{(1 / change):.2f}x faster" if change < 1.0 else f" {change:.2f}x slower" ) ) df["tpch_query"] = [f"{i}" for i in range(1, 23)] + ["total"] df["sort_index"] = list(range(1, 24)) + df["spark"] = df["spark"].apply(lambda s: f"{s:>10.4f}") + df["df_ray"] = df["df_ray"].apply(lambda s: f"{s:>10.4f}") + df["change"] = df["change"].apply(lambda s: f"{s:>21.4f}") + df["change_text"] = df["change_text"].apply(lambda s: f"{s:>14}") + ts = time.time() df.to_parquet(f"datafusion-ray-spark-comparison-{ts}.parquet") ctx = datafusion.SessionContext() - ctx.register_parquet( - "results", f"datafusion-ray-spark-comparison-{ts}.parquet" - ) + ctx.register_parquet("results", f"datafusion-ray-spark-comparison-{ts}.parquet") cpu = subprocess.run( "lscpu | grep 'Model name' |awk '{print $3}'", @@ -253,10 +246,10 @@ def results(data_path, data_device, scale_factor, output_path): spark_cost = f"${spark_cost:.4f}" spark_duration = f"{spark[-1]:.2f}s" - print("=" * 97) + print("=" * 81) # the formatting code is terrible here, but it works for now header = [ - "Spark and DataFusion for Ray TPCH 100 Benchmarks", + "Spark and DataFusionRay TPCH 100 Benchmarks", f"{'Machine:':<30}{machine}", f"{'Machine On Demand Cost:':<30}{hourly_cost} $/hr", f"{'CPU(s):':<30}{cpu} {quantity}x", @@ -270,7 +263,7 @@ def results(data_path, data_device, scale_factor, output_path): f"{'spark duration:':<30}{spark_duration:>10}", f"{'spark cost:':<30}{spark_cost:>10}", "", - "DataFusion for Ray Settings:", + "DataFusionRay Settings:", f"{'concurrency:':<30}{df_result['settings']['concurrency']:>10}", f"{'batch_size :':<30}{df_result['settings']['batch_size']:>10}", f"{'partitions_per_processor:':<30}{df_result['settings']['partitions_per_processor']:>10}", @@ -291,7 +284,7 @@ def results(data_path, data_device, scale_factor, output_path): for h in header: print(h) - print("=" * 97) + print("=" * 81) ctx.sql( 'select tpch_query, spark, df_ray, change as "change(=df_ray/spark)", change_text from results order by sort_index asc' ).show(num=100) diff --git a/k8s/cmds.py b/k8s/cmds.py index f5c6180..5793583 100644 --- a/k8s/cmds.py +++ b/k8s/cmds.py @@ -25,7 +25,7 @@ ], "k3s_setup": [ Shell( - """sudo curl -sfL https://get.k3s.io | {{ k3s_url if k3s_url else "" }} {{ k3s_token if k3s_token else ""}} sh - --write-kubeconfig-mode 644""", + """sudo curl -sfL https://get.k3s.io | {{ k3s_url if k3s_url else "" }} {{ k3s_token if k3s_token else ""}} sh -s -""", "Installing K3s", ), Shell( @@ -41,11 +41,11 @@ "Installing Helm", ), Shell( - "helm repo add kuberay https://ray-project.github.io/kuberay-helm/", + "helm --kubeconfig /etc/rancher/k3s/k3s.yaml repo add kuberay https://ray-project.github.io/kuberay-helm/", "Adding kube ray helm repo", ), Shell( - "helm repo add spark-operator https://kubeflow.github.io/spark-operator", + "helm --kubeconfig /etc/rancher/k3s/k3s.yaml repo add spark-operator https://kubeflow.github.io/spark-operator", "Adding spark operator helm repo", ), Shell("helm repo update", "Updating helm repos"), @@ -54,7 +54,7 @@ "Installing kuberay-operator", ), Shell( - """helm install --set-json='controller.env=[{"name":"SPARK_SUBMIT_OPTS","value":"-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2"}]' spark-operator spark-operator/spark-operator""", + """helm --kubeconfig /etc/rancher/k3s/k3s.yaml install --set-json='controller.env=[{"name":"SPARK_SUBMIT_OPTS","value":"-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2"}]' spark-operator spark-operator/spark-operator""", "Installing spark-operator", ), Template("pvcs.yaml.template", "rewrite pvcs.yaml.template"), @@ -82,12 +82,20 @@ ), Shell( """ + {% if data_path.startswith("s3") %} wget https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar && \ - aws s3 cp aws-java-sdk-bundle-1.12.262.jar {{ data_path.replace('s3a','s3') }}/aws-java-sdk-bundle-1.12.262.jar && \ wget https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar && \ + aws s3 cp aws-java-sdk-bundle-1.12.262.jar {{ data_path.replace('s3a','s3') }}/aws-java-sdk-bundle-1.12.262.jar && \ aws s3 cp hadoop-aws-3.3.4.jar {{ data_path.replace('s3a','s3') }}/hadoop-aws-3.3.4.jar + {% else %} + wget https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar && \ + wget https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar && \ + mv aws-java-sdk-bundle-1.12.262.jar {{ data_path }}/aws-java-sdk-bundle-1.12.262.jar && \ + mv hadoop-aws-3.3.4.jar {{ data_path }}/hadoop-aws-3.3.4.jar + + {% endif %} """, - "getting additional spark jars", + "getting additional spark jars" ), Shell( "kubectl apply -f spark_job.yaml", @@ -106,15 +114,13 @@ """, "checking on job status", ), - # Shell( + #Shell( # "kubectl delete -f spark_job.yaml", # "tear down job", - # ), + #), ], "bench_df_ray": [ - Template( - "ray_cluster.yaml.template", "rewrite ray_cluster.yaml.template" - ), + Template("ray_cluster.yaml.template", "rewrite ray_cluster.yaml.template"), Shell( "kubectl apply -f ray_cluster.yaml", "deploying ray cluster", @@ -123,9 +129,7 @@ "kubectl wait raycluster/datafusion-ray-cluster --for='jsonpath={.status.state}'=ready --timeout=300s", "wait for ray cluster to be ready", ), - Template( - "requirements.txt.template", "rewrite requirements.txt.template" - ), + Template("requirements.txt.template", "rewrite requirements.txt.template"), Template("ray_job.sh.template", "rewrite ray_job.sh.template"), BackgroundShell( "kubectl port-forward svc/datafusion-ray-cluster-head-svc 8265:8265", @@ -181,9 +185,7 @@ def run_commands( for command in commands: match (self.dry_run, command): case (False, Shell(cmd, desc)): - self.run_shell_command( - textwrap.dedent(cmd), desc, substitutions - ) + self.run_shell_command(textwrap.dedent(cmd), desc, substitutions) case (True, Shell(cmd, desc)): click.secho(f"[dry run] {desc} ...") @@ -191,10 +193,7 @@ def run_commands( case (False, BackgroundShell(cmd, desc)): self.run_shell_command( - textwrap.dedent(cmd), - desc, - substitutions, - background=True, + textwrap.dedent(cmd), desc, substitutions, background=True ) case (True, BackgroundShell(cmd, desc)): @@ -207,9 +206,7 @@ def run_commands( case (True, Template(path, desc)): click.secho(f"[dry run] {desc} ...") - click.secho( - f" {path} subs:{substitutions}", fg="yellow" - ) + click.secho(f" {path} subs:{substitutions}", fg="yellow") case (False, ChangeDir(path, desc)): click.secho(f"{desc} ...") @@ -226,9 +223,7 @@ def run_commands( click.secho(f"[dry run] {desc} ...") case _: - raise Exception( - "Unhandled case in match. Shouldn't happen" - ) + raise Exception("Unhandled case in match. Shouldn't happen") def run_shell_command( self, @@ -274,10 +269,7 @@ def run_shell_command( exit(1) def process_template( - self, - template_name: str, - output_path: str, - substitutions: dict[str, str] | None, + self, template_name: str, output_path: str, substitutions: dict[str, str] | None ): template_out = template_name[: template_name.index(".template")] output_path = os.path.join(output_path, template_out) diff --git a/k8s/spark_job.yaml.template b/k8s/spark_job.yaml.template index 5d57b35..a00f7df 100644 --- a/k8s/spark_job.yaml.template +++ b/k8s/spark_job.yaml.template @@ -11,12 +11,14 @@ spec: imagePullPolicy: IfNotPresent # see https://github.com/kubeflow/spark-operator/issues/1132#issuecomment-2688276488 deps: +{% if data_path.startswith("s3") %} packages: - org.apache.hadoop:hadoop-aws:3.3.4 - com.amazonaws:aws-java-sdk-bundle:1.12.782 repositories: - https://repo1.maven.org/maven2/ - mainApplicationFile: {{ data_path }}/spark_tpcbench.py +{% endif %} + mainApplicationFile: {{ "local://" if data_path.startswith("/") else "" }}{{ data_path }}/spark_tpcbench.py arguments: - --name - spark @@ -27,7 +29,7 @@ spec: - --queries - /data/queries - --output - - /data/ + - {{ output_path }} sparkVersion: 3.5.3 volumes: - name: spark-local-dir-spark-vol diff --git a/k8s/spark_tpcbench.py b/k8s/spark_tpcbench.py index 7ffef3d..ed4b978 100644 --- a/k8s/spark_tpcbench.py +++ b/k8s/spark_tpcbench.py @@ -22,31 +22,18 @@ import time import sys -import boto3 - -def main( - benchmark: str, - data_path: str, - query_path: str, - output_path: str, - name: str, -): +def main(benchmark: str, data_path: str, query_path: str, output_path: str, name: str): # Initialize a SparkSession - spark = SparkSession.builder.appName( - f"{name} benchmark derived from {benchmark}" - ).getOrCreate() + spark = SparkSession.builder \ + .appName( f"{name} benchmark derived from {benchmark}") \ + .getOrCreate() - spark.conf.set( - "spark.hadoop.fs.s3a.aws.credentials.provider", - "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider", - ) - spark.conf.set( - "spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem" - ) + spark.conf.set("spark.hadoop.fs.s3a.aws.credentials.provider", "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider") + spark.conf.set("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") - # Register the tables + # Register the tables num_queries = 22 table_names = [ "customer", @@ -96,10 +83,7 @@ def main( text = f.read() # each file can contain multiple queries queries = list( - filter( - lambda x: len(x) > 0, - map(lambda x: x.strip(), text.split(";")), - ) + filter(lambda x: len(x) > 0, map(lambda x: x.strip(), text.split(";"))) ) start_time = time.time() @@ -127,9 +111,7 @@ def main( out = json.dumps(results, indent=4) current_time_millis = int(datetime.now().timestamp() * 1000) - results_path = ( - f"{output_path}/{name}-{benchmark}-{current_time_millis}.json" - ) + results_path = f"{output_path}/{name}-{benchmark}-{current_time_millis}.json" print(f"Writing results to {results_path}") with open(results_path, "w") as f: f.write(out) @@ -153,9 +135,7 @@ def main( parser.add_argument("--queries", required=True, help="Path to query files") parser.add_argument("--output", required=True, help="Path to write output") parser.add_argument( - "--name", - required=True, - help="Prefix for result file e.g. spark/comet/gluten", + "--name", required=True, help="Prefix for result file e.g. spark/comet/gluten" ) args = parser.parse_args() print(f"parsed is {args}") From b771189bd2ada1cbe2fa092d02713da6dba594b6 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Thu, 20 Mar 2025 13:32:11 +0100 Subject: [PATCH 6/8] add more benchmark results --- .../benchmarks/df-ray-0.1.0rc1/i4g.4xlarge_s3 | 61 +++++++++ docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge | 117 ++++++++++++++++++ .../df-ray-0.1.0rc1/m7g.12xlarge_s3 | 61 +++++++++ docs/benchmarks/df-ray-0.1.0rc1/m7gd.12xlarge | 117 ++++++++++++++++++ k8s/cmds.py | 10 +- 5 files changed, 361 insertions(+), 5 deletions(-) create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge_s3 create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge_s3 create mode 100644 docs/benchmarks/df-ray-0.1.0rc1/m7gd.12xlarge diff --git a/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge_s3 b/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge_s3 new file mode 100644 index 0000000..3e74b16 --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge_s3 @@ -0,0 +1,61 @@ +================================================================================ +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: 1x i4g.4xlarge +Per Machine On Demand Cost: 1.23552 $/hr +CPU TOTAL: 16x Neoverse-N1x +MEM TOTAL: 126.8GB +HD Throughput: 921.58 MB/s (from hdparm) +Data Location: s3://rob-tandy-tmp//sf100 + +df-ray duration: 1424.35s +2.44x faster +df-ray cost: $0.4888 +2.44x cheaper + +spark duration: 3473.58s +spark cost: $1.1921 + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 4 +Ray Workers: 4 +Ray Worker Mem (GB): 24 +Ray Worker CPU: 3 +Ray Head Mem (GB): 4 +Ray Head CPU: 2 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 20 +Executor Overhead Mem (GB): 4 +Executor CPU: 3 +Driver Mem(GB): 4 +Driver CPU: 2 +================================================================================ +DataFrame() ++------------+------------+------------+-----------------------+---------------+ +| tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | ++------------+------------+------------+-----------------------+---------------+ +| 1 | 223.7065 | 53.2348 | 0.2380 | +4.20x faster | +| 2 | 39.5393 | 25.1823 | 0.6369 | +1.57x faster | +| 3 | 186.6593 | 58.9956 | 0.3161 | +3.16x faster | +| 4 | 126.7456 | 27.6134 | 0.2179 | +4.59x faster | +| 5 | 218.8699 | 102.4856 | 0.4682 | +2.14x faster | +| 6 | 96.1340 | 33.1685 | 0.3450 | +2.90x faster | +| 7 | 210.9092 | 98.8647 | 0.4688 | +2.13x faster | +| 8 | 187.0856 | 70.3633 | 0.3761 | +2.66x faster | +| 9 | 203.7582 | 105.3517 | 0.5170 | +1.93x faster | +| 10 | 110.8606 | 72.4830 | 0.6538 | +1.53x faster | +| 11 | 20.4039 | 19.8943 | 0.9750 | +1.03x faster | +| 12 | 96.4592 | 49.0558 | 0.5086 | +1.97x faster | +| 13 | 61.5240 | 20.0987 | 0.3267 | +3.06x faster | +| 14 | 143.0664 | 26.4152 | 0.1846 | +5.42x faster | +| 15 | 307.8785 | 71.2565 | 0.2314 | +4.32x faster | +| 16 | 20.2879 | 14.6983 | 0.7245 | +1.38x faster | +| 17 | 301.2155 | 148.0474 | 0.4915 | +2.03x faster | +| 18 | 242.4097 | 144.3459 | 0.5955 | +1.68x faster | +| 19 | 89.4152 | 92.0230 | 1.0292 | 1.03x slower | +| 20 | 123.9332 | 39.6039 | 0.3196 | +3.13x faster | +| 21 | 428.6912 | 140.1514 | 0.3269 | +3.06x faster | +| 22 | 34.0295 | 11.0184 | 0.3238 | +3.09x faster | +| total | 3473.5824 | 1424.3519 | 0.4101 | +2.44x faster | ++------------+------------+------------+-----------------------+---------------+ diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge new file mode 100644 index 0000000..4a8b9cf --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge @@ -0,0 +1,117 @@ +================================================================================= +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: m7g.12xlarge +Machine On Demand Cost: 1.9584 $/hr +CPU(s): Neoverse-V1 48x +MEM: 189G +HD Throughput: 166.46 MB/s (from hdparm) +Data Location: /data/sf100 + +df-ray duration: 204.97s +4.75x faster +df-ray cost: $0.1115 +4.75x cheaper + +spark duration: 974.60s +spark cost: $0.5302 + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 1 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +================================================================================= +DataFrame() ++------------+------------+------------+-----------------------+----------------+ +| tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | ++------------+------------+------------+-----------------------+----------------+ +| 1 | 41.0527 | 7.3332 | 0.1786 | +5.60x faster | +| 2 | 5.2886 | 3.3632 | 0.6359 | +1.57x faster | +| 3 | 8.8360 | 5.8260 | 0.6593 | +1.52x faster | +| 4 | 6.7364 | 2.8624 | 0.4249 | +2.35x faster | +| 5 | 113.1327 | 9.7685 | 0.0863 | +11.58x faster | +| 6 | 4.0279 | 5.0924 | 1.2643 | 1.26x slower | +| 7 | 10.3482 | 15.0313 | 1.4526 | 1.45x slower | +| 8 | 197.1830 | 10.5564 | 0.0535 | +18.68x faster | +| 9 | 244.0854 | 17.0069 | 0.0697 | +14.35x faster | +| 10 | 25.6980 | 10.1537 | 0.3951 | +2.53x faster | +| 11 | 2.0117 | 3.2634 | 1.6222 | 1.62x slower | +| 12 | 20.2309 | 4.5518 | 0.2250 | +4.44x faster | +| 13 | 8.1399 | 3.5019 | 0.4302 | +2.32x faster | +| 14 | 2.5995 | 3.3954 | 1.3062 | 1.31x slower | +| 15 | 6.7179 | 6.2090 | 0.9243 | +1.08x faster | +| 16 | 4.2510 | 3.1190 | 0.7337 | +1.36x faster | +| 17 | 119.6635 | 21.6696 | 0.1811 | +5.52x faster | +| 18 | 61.9433 | 32.2979 | 0.5214 | +1.92x faster | +| 19 | 4.1546 | 5.9170 | 1.4242 | 1.42x slower | +| 20 | 5.0565 | 5.5918 | 1.1059 | 1.11x slower | +| 21 | 78.7263 | 26.4413 | 0.3359 | +2.98x faster | +| 22 | 4.7114 | 2.0223 | 0.4292 | +2.33x faster | +| total | 974.5953 | 204.9744 | 0.2103 | +4.75x faster | ++------------+------------+------------+-----------------------+----------------+ + + +========================================================================================== +Spark and DataFusion for Ray TPCH 100 Benchmarks +Machine: m7g.12xlarge +CPU(s): 48x +MEM: 189G +HD Throughput: 167.53 MB/s (from hdparm) + +DataFusion for Ray Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 2 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +========================================================================================== +DataFrame() ++------------+--------------------+--------------------+---------------------+----------------+ +| tpch_query | spark | df_ray | change | change_text | ++------------+--------------------+--------------------+---------------------+----------------+ +| 1 | 40.66942858695984 | 7.127374172210693 | 0.1752513969300272 | +5.71x faster | +| 2 | 5.27202296257019 | 4.556609869003296 | 0.8643000801312671 | +1.16x faster | +| 3 | 9.467089414596558 | 6.217605829238892 | 0.6567600195739628 | +1.52x faster | +| 4 | 6.713418006896973 | 2.6009602546691895 | 0.3874271275819732 | +2.58x faster | +| 5 | 138.02316689491272 | 14.755179166793823 | 0.10690364160407967 | +9.35x faster | +| 6 | 1.7651944160461426 | 4.644023895263672 | 2.630885217542109 | 2.63x slower | +| 7 | 7.105847120285034 | 12.337952613830566 | 1.7363098874741423 | 1.74x slower | +| 8 | 234.60386061668396 | 8.625483512878418 | 0.03676616186198009 | +27.20x faster | +| 9 | 308.04562306404114 | 15.377676725387573 | 0.04992012732539502 | +20.03x faster | +| 10 | 99.11399936676025 | 12.474991083145142 | 0.12586507620364337 | +7.95x faster | +| 11 | 2.5937740802764893 | 2.8647923469543457 | 1.104488000222813 | 1.10x slower | +| 12 | 22.38423728942871 | 4.3449485301971436 | 0.1941075085122114 | +5.15x faster | +| 13 | 31.56517219543457 | 3.571704864501953 | 0.11315334642839511 | +8.84x faster | +| 14 | 16.334258317947388 | 3.2847304344177246 | 0.20109455663552245 | +4.97x faster | +| 15 | 11.267093658447266 | 5.8551459312438965 | 0.5196678139667477 | +1.92x faster | +| 16 | 6.11682653427124 | 2.544823408126831 | 0.4160365499771397 | +2.40x faster | +| 17 | 139.6458387374878 | 17.490234375 | 0.12524708600790382 | +7.98x faster | +| 18 | 68.82194375991821 | 29.644872903823853 | 0.4307473936981271 | +2.32x faster | +| 19 | 11.364177465438843 | 5.922780275344849 | 0.5211798472311284 | +1.92x faster | +| 20 | 15.337732791900635 | 4.3357462882995605 | 0.28268495397110643 | +3.54x faster | +| 21 | 114.92279720306396 | 24.284620761871338 | 0.21131247544350482 | +4.73x faster | +| 22 | 6.929309368133545 | 1.8829975128173828 | 0.27174389434492524 | +3.68x faster | +| total | 1298.0628118515015 | 194.74525475502014 | 0.15002760496408013 | +6.67x faster | ++------------+--------------------+--------------------+---------------------+----------------+ + diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge_s3 b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge_s3 new file mode 100644 index 0000000..558d336 --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge_s3 @@ -0,0 +1,61 @@ +================================================================================= +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: m7g.12xlarge +Machine On Demand Cost: 1.9584 $/hr +CPU(s): Neoverse-V1 48x +MEM: 189G +HD Throughput: 167.24 MB/s (from hdparm) +Data Location: s3://rob-tandy-tmp/sf100 + +df-ray duration: 480.91s +3.06x faster +df-ray cost: $0.2616 +3.06x cheaper + +spark duration: 1469.31s +spark cost: $0.7993 + +DataFusionRay Settings: +concurrency: 32 +batch_size : 8192 +partitions_per_processor: 1 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +================================================================================= +DataFrame() ++------------+------------+------------+-----------------------+----------------+ +| tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | ++------------+------------+------------+-----------------------+----------------+ +| 1 | 58.4442 | 20.6952 | 0.3541 | +2.82x faster | +| 2 | 13.5071 | 14.9597 | 1.1075 | 1.11x slower | +| 3 | 61.9178 | 34.7799 | 0.5617 | +1.78x faster | +| 4 | 43.8635 | 15.1339 | 0.3450 | +2.90x faster | +| 5 | 96.1693 | 23.3548 | 0.2429 | +4.12x faster | +| 6 | 33.6715 | 10.1410 | 0.3012 | +3.32x faster | +| 7 | 67.6622 | 25.9438 | 0.3834 | +2.61x faster | +| 8 | 202.4597 | 25.1409 | 0.1242 | +8.05x faster | +| 9 | 243.0195 | 29.2583 | 0.1204 | +8.31x faster | +| 10 | 40.6520 | 27.6224 | 0.6795 | +1.47x faster | +| 11 | 7.3265 | 8.2168 | 1.1215 | 1.12x slower | +| 12 | 34.8475 | 21.6464 | 0.6212 | +1.61x faster | +| 13 | 14.5846 | 7.1863 | 0.4927 | +2.03x faster | +| 14 | 50.6580 | 11.1040 | 0.2192 | +4.56x faster | +| 15 | 102.5048 | 20.0342 | 0.1954 | +5.12x faster | +| 16 | 7.2824 | 5.2953 | 0.7271 | +1.38x faster | +| 17 | 111.3701 | 34.5333 | 0.3101 | +3.23x faster | +| 18 | 74.6563 | 37.2044 | 0.4983 | +2.01x faster | +| 19 | 30.9803 | 32.1251 | 1.0370 | 1.04x slower | +| 20 | 39.0791 | 17.3910 | 0.4450 | +2.25x faster | +| 21 | 124.4177 | 52.3758 | 0.4210 | +2.38x faster | +| 22 | 10.2409 | 6.7682 | 0.6609 | +1.51x faster | +| total | 1469.3147 | 480.9106 | 0.3273 | +3.06x faster | ++------------+------------+------------+-----------------------+----------------+ diff --git a/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12xlarge b/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12xlarge new file mode 100644 index 0000000..53b0f7a --- /dev/null +++ b/docs/benchmarks/df-ray-0.1.0rc1/m7gd.12xlarge @@ -0,0 +1,117 @@ +================================================================================= +Spark and DataFusionRay TPCH 100 Benchmarks +Machine: m7gd.12xlarge +Machine On Demand Cost: 2.5628 $/hr +CPU(s): Neoverse-V1 48x +MEM: 189G +HD Throughput: 2153.86 MB/s (from hdparm) +Data Location: /data/sf100 + +df-ray duration: 213.06s +1.29x faster +df-ray cost: $0.1517 +1.29x cheaper + +spark duration: 275.49s +spark cost: $0.1961 + +DataFusionRay Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 2 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +================================================================================= +DataFrame() ++------------+------------+------------+-----------------------+----------------+ +| tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | ++------------+------------+------------+-----------------------+----------------+ +| 1 | 39.8715 | 7.2273 | 0.1813 | +5.52x faster | +| 2 | 5.1250 | 3.3663 | 0.6568 | +1.52x faster | +| 3 | 9.4450 | 7.9609 | 0.8429 | +1.19x faster | +| 4 | 10.7174 | 2.9885 | 0.2789 | +3.59x faster | +| 5 | 17.1354 | 16.6721 | 0.9730 | +1.03x faster | +| 6 | 1.6761 | 5.0164 | 2.9928 | 2.99x slower | +| 7 | 6.1539 | 16.4512 | 2.6733 | 2.67x slower | +| 8 | 22.3847 | 12.0402 | 0.5379 | +1.86x faster | +| 9 | 37.1457 | 16.9252 | 0.4556 | +2.19x faster | +| 10 | 7.6453 | 9.7099 | 1.2701 | 1.27x slower | +| 11 | 1.7255 | 2.7529 | 1.5954 | 1.60x slower | +| 12 | 6.2536 | 4.5053 | 0.7204 | +1.39x faster | +| 13 | 8.1595 | 4.9620 | 0.6081 | +1.64x faster | +| 14 | 2.4955 | 3.3134 | 1.3277 | 1.33x slower | +| 15 | 6.9314 | 6.7785 | 0.9779 | +1.02x faster | +| 16 | 3.8568 | 2.3385 | 0.6063 | +1.65x faster | +| 17 | 32.9444 | 17.6122 | 0.5346 | +1.87x faster | +| 18 | 23.3267 | 33.7229 | 1.4457 | 1.45x slower | +| 19 | 3.5956 | 7.6568 | 2.1295 | 2.13x slower | +| 20 | 4.9011 | 5.8706 | 1.1978 | 1.20x slower | +| 21 | 20.1347 | 23.3859 | 1.1615 | 1.16x slower | +| 22 | 3.8665 | 1.8024 | 0.4662 | +2.15x faster | +| total | 275.4912 | 213.0596 | 0.7734 | +1.29x faster | ++------------+------------+------------+-----------------------+----------------+ + + +========================================================================================== +Spark and DataFusion for Ray TPCH 100 Benchmarks +Machine: m7gd.12xlarge +CPU(s): 48x +MEM: 189G +HD Throughput: 2162.93 (from hdparm) + +DataFusion for Ray Settings: +concurrency: 16 +batch_size : 8192 +partitions_per_processor: 2 +Ray Workers: 4 +Ray Worker Mem (GB): 40 +Ray Worker CPU: 10 +Ray Head Mem (GB): 10 +Ray Head CPU: 4 + +Spark Settings: +Executors: 4 +Executor Mem (GB): 32 +Executor Overhead Mem (GB): 8 +Executor CPU: 10 +Driver Mem(GB): 10 +Driver CPU: 4 +========================================================================================== +DataFrame() ++------------+--------------------+--------------------+---------------------+---------------+ +| tpch_query | spark | df_ray | change | change_text | ++------------+--------------------+--------------------+---------------------+---------------+ +| 1 | 40.10749578475952 | 7.07772159576416 | 0.17646879859434228 | +5.67x faster | +| 2 | 5.265030860900879 | 4.421863079071045 | 0.8398551111843696 | +1.19x faster | +| 3 | 8.993131875991821 | 4.821190357208252 | 0.5360969263754447 | +1.87x faster | +| 4 | 8.75968074798584 | 2.8997082710266113 | 0.3310289900340668 | +3.02x faster | +| 5 | 19.33255624771118 | 10.249737024307251 | 0.5301801217063955 | +1.89x faster | +| 6 | 1.754713773727417 | 4.5369789600372314 | 2.5855948861674696 | 2.59x slower | +| 7 | 6.20657753944397 | 13.625978946685791 | 2.1954094442694267 | 2.20x slower | +| 8 | 24.33410143852234 | 9.368516683578491 | 0.3849953821901789 | +2.60x faster | +| 9 | 38.12766456604004 | 13.534857511520386 | 0.35498784584817605 | +2.82x faster | +| 10 | 7.784897565841675 | 8.692375421524048 | 1.1165690168698141 | 1.12x slower | +| 11 | 2.0291600227355957 | 3.2606608867645264 | 1.6069017969162889 | 1.61x slower | +| 12 | 6.192300081253052 | 4.058101415634155 | 0.6553463757223104 | +1.53x faster | +| 13 | 8.62465763092041 | 3.507314682006836 | 0.40666132292981705 | +2.46x faster | +| 14 | 2.430983304977417 | 3.096514940261841 | 1.2737705495228016 | 1.27x slower | +| 15 | 6.798931360244751 | 5.77943229675293 | 0.8500501020714643 | +1.18x faster | +| 16 | 4.050042390823364 | 3.0604259967803955 | 0.7556528306258588 | +1.32x faster | +| 17 | 33.42715358734131 | 19.431111812591553 | 0.5812972307624187 | +1.72x faster | +| 18 | 23.402374744415283 | 28.70749068260193 | 1.2266913506054615 | 1.23x slower | +| 19 | 3.6461265087127686 | 5.904490232467651 | 1.6193870998053157 | 1.62x slower | +| 20 | 5.166107892990112 | 5.764256477355957 | 1.1157832156733452 | 1.12x slower | +| 21 | 21.18029236793518 | 19.883575677871704 | 0.9387772053597062 | +1.07x faster | +| 22 | 3.8765759468078613 | 1.483001947402954 | 0.382554596569718 | +2.61x faster | +| total | 281.4905562400818 | 183.1653048992157 | 0.650697868325625 | +1.54x faster | ++------------+--------------------+--------------------+---------------------+---------------+ + diff --git a/k8s/cmds.py b/k8s/cmds.py index 5793583..c50eb89 100644 --- a/k8s/cmds.py +++ b/k8s/cmds.py @@ -95,7 +95,7 @@ {% endif %} """, - "getting additional spark jars" + "getting additional spark jars", ), Shell( "kubectl apply -f spark_job.yaml", @@ -114,10 +114,10 @@ """, "checking on job status", ), - #Shell( - # "kubectl delete -f spark_job.yaml", - # "tear down job", - #), + Shell( + "kubectl delete -f spark_job.yaml", + "tear down job", + ), ], "bench_df_ray": [ Template("ray_cluster.yaml.template", "rewrite ray_cluster.yaml.template"), From 49577293522b89c49cbee509149b6f8f34f65103 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Wed, 2 Apr 2025 12:11:42 -0400 Subject: [PATCH 7/8] revert broken change in CI TPCH testing --- tpch/make_data.py | 27 ++++----------------------- 1 file changed, 4 insertions(+), 23 deletions(-) diff --git a/tpch/make_data.py b/tpch/make_data.py index 4b5e8cc..bec5173 100644 --- a/tpch/make_data.py +++ b/tpch/make_data.py @@ -1,31 +1,23 @@ import duckdb import sys -import os -import multiprocessing conn = duckdb.connect() -def make(scale_factor: int, partitions: int, output_path: str, step: int): +def make(scale_factor: int, output_path: str): statements = [ "install tpch", "load tpch", + f"call dbgen(sf = {scale_factor})", ] execute(statements) - print(f"step {step}") - sql = f"call dbgen(sf={scale_factor}, children={partitions}, step={step})" - conn.execute(sql) - conn.sql("show tables").show() - statements = [] - for row in conn.execute("show tables").fetchall(): table = row[0] - os.makedirs(f"{output_path}/{table}.parquet", exist_ok=True) statements.append( - f"copy {table} to '{output_path}/{table}.parquet/part{step}.parquet' (format parquet, compression zstd)" + f"copy {table} to '{output_path}/{table}.parquet' (format parquet, compression zstd)" ) execute(statements) @@ -37,15 +29,4 @@ def execute(statements): if __name__ == "__main__": - # this is quick and dirty, it should be tidied up with click to process args - scale_factor = int(sys.argv[1]) - partitions = int(sys.argv[2]) - data_path = sys.argv[3] - procs = int(sys.argv[4]) - - def go(step): - make(scale_factor, partitions, data_path, step) - - steps = list(range(partitions)) - with multiprocessing.Pool(processes=procs) as pool: - pool.map(go, steps) + make(int(sys.argv[1]), sys.argv[2]) From c2ff7811cce45d46a3ee73f1f7427483c87f5a95 Mon Sep 17 00:00:00 2001 From: Rob Tandy Date: Wed, 2 Apr 2025 12:58:47 -0400 Subject: [PATCH 8/8] additional fix for CI --- .github/workflows/main.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 2e9c5df..e26fef0 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -131,5 +131,5 @@ jobs: --concurrency 3 \ --partitions-per-processor 2 \ --batch-size=8192 \ - --worker-pool-min=20 \ + --processor-pool-min=20 \ --validate