@@ -72,6 +72,47 @@ object DatasetBenchmark {
72
72
benchmark
73
73
}
74
74
75
+ def backToBackMapPrimitive (spark : SparkSession , numRows : Long , numChains : Int ): Benchmark = {
76
+ import spark .implicits ._
77
+
78
+ val df = spark.range(1 , numRows).select($" id" .as(" l" ))
79
+ val benchmark = new Benchmark (" back-to-back map for primitive" , numRows)
80
+ val func = (d : Long ) => d+ 1
81
+
82
+ val rdd = spark.sparkContext.range(1 , numRows).map(l => l.toLong)
83
+ benchmark.addCase(" RDD" ) { iter =>
84
+ var res = rdd
85
+ var i = 0
86
+ while (i < numChains) {
87
+ res = rdd.map(func)
88
+ i += 1
89
+ }
90
+ res.foreach(_ => Unit )
91
+ }
92
+
93
+ benchmark.addCase(" DataFrame" ) { iter =>
94
+ var res = df
95
+ var i = 0
96
+ while (i < numChains) {
97
+ res = res.select($" l" + 1 as " l" )
98
+ i += 1
99
+ }
100
+ res.queryExecution.toRdd.foreach(_ => Unit )
101
+ }
102
+
103
+ benchmark.addCase(" Dataset" ) { iter =>
104
+ var res = df.as[Long ]
105
+ var i = 0
106
+ while (i < numChains) {
107
+ res = res.map(func)
108
+ i += 1
109
+ }
110
+ res.queryExecution.toRdd.foreach(_ => Unit )
111
+ }
112
+
113
+ benchmark
114
+ }
115
+
75
116
def backToBackFilter (spark : SparkSession , numRows : Long , numChains : Int ): Benchmark = {
76
117
import spark .implicits ._
77
118
@@ -166,20 +207,32 @@ object DatasetBenchmark {
166
207
val numChains = 10
167
208
168
209
val benchmark = backToBackMap(spark, numRows, numChains)
169
- val benchmark2 = backToBackFilter(spark, numRows, numChains)
170
- val benchmark3 = aggregate(spark, numRows)
210
+ val benchmark2 = backToBackMapPrimitive(spark, numRows, numChains)
211
+ val benchmark3 = backToBackFilter(spark, numRows, numChains)
212
+ val benchmark4 = aggregate(spark, numRows)
171
213
172
214
/*
173
- Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11.4
174
- Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz
175
- back-to-back map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
176
- -------------------------------------------------------------------------------------------
177
- RDD 1935 / 2105 51.7 19.3 1.0X
178
- DataFrame 756 / 799 132.3 7.6 2.6X
179
- Dataset 7359 / 7506 13.6 73 .6 0.3X
215
+ Java HotSpot(TM) 64-Bit Server VM 1.8.0_91-b14 on Mac OS X 10.11.5
216
+ Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz
217
+ back-to-back map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
218
+ ------------------------------------------------------------------------------------------------
219
+ RDD 5144 / 5330 19.4 51.4 1.0X
220
+ DataFrame 6878 / 7133 14.5 68.8 0.7X
221
+ Dataset 13760 / 14076 7.3 137 .6 0.4X
180
222
*/
181
223
benchmark.run()
182
224
225
+ /*
226
+ Java HotSpot(TM) 64-Bit Server VM 1.8.0_91-b14 on Mac OS X 10.11.5
227
+ Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz
228
+ back-to-back map for primitive: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
229
+ ------------------------------------------------------------------------------------------------
230
+ RDD 1963 / 2172 50.9 19.6 1.0X
231
+ DataFrame 1563 / 1746 64.0 15.6 1.3X
232
+ Dataset 5640 / 5810 17.7 56.4 0.3X
233
+ */
234
+ benchmark2.run()
235
+
183
236
/*
184
237
Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11.4
185
238
Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz
@@ -189,7 +242,7 @@ object DatasetBenchmark {
189
242
DataFrame 103 / 127 967.4 1.0 19.1X
190
243
Dataset 4343 / 4477 23.0 43.4 0.5X
191
244
*/
192
- benchmark2 .run()
245
+ benchmark3 .run()
193
246
194
247
/*
195
248
Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11.4
@@ -201,6 +254,6 @@ object DatasetBenchmark {
201
254
Dataset sum using Aggregator 4111 / 4282 24.3 41.1 0.5X
202
255
Dataset complex Aggregator 8782 / 9036 11.4 87.8 0.2X
203
256
*/
204
- benchmark3 .run()
257
+ benchmark4 .run()
205
258
}
206
259
}
0 commit comments