Best Practices For Bucketing in Spark SQL - by David Vrba - Towards Data Science
Best Practices For Bucketing in Spark SQL - by David Vrba - Towards Data Science
Save
There are many resources that explain the basic idea of bucketing, in
this article, we will go one step further and describe bucketing more in
detail, we will see various different aspects it can have and explain how
it works under the hood, how it evolved over time and — most
importantly — how to efficiently use it.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 1/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
a data engineer who is responsible for preparing the data and exposing
it to the data user.
What is bucketing?
Let’s start with this simple question. Bucketing in Spark is a way how to
organize data in the storage system in a particular way so it can be
leveraged in subsequent queries which can become more efficient. This
efficiency improvement is specifically related to avoiding the shuffle in
queries with joins and aggregations if the bucketing is designed well.
With bucketing, we can shuffle the data in advance and save it in this
pre-shuffled state. After reading the data back from the storage system,
Spark will be aware of this distribution and will not have to shuffle it
again.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 2/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
In Spark API there is a function bucketBy that can be used for this
purpose:
df.write
.mode(saving_mode) # append/overwrite
.option("path", output_path)
.saveAsTable(table_name)
Together with bucketBy, we can call also sortBy, this will sort each
bucket by the specified fields. Calling sortBy is optional, bucketing
will work also without the sorting. The other way around is not
working though — you can not call sortBy if you don’t call bucketBy as
well.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 3/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
2 mod 4 = 2
3 mod 4 = 3
4 mod 4 = 0
5 mod 4 = 1
6 mod 4 = 2
The computed number is the final bucket. As you can see, we just
distributed these six values into four buckets
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 4/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
To be more exact, Spark is not using a simple modulo function, but a so-
called positive modulo which makes sure the final bucket value is a
positive number and is defined as follows:
b = value mod n
if b < 0:
b = (b + n) mod n
n = 4
value = -9
b = value mod n = -9 mod 4 = -1
# be is negative so we continue:
The hash function that Spark is using is implemented with the MurMur3
hash algorithm and the function is actually exposed in the DataFrame
API (see in docs) so we can use it to compute the corresponding bucket
if we want:
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 5/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
.withColumn("hash", hash(col("id")))
Here we can see how the data would be distributed into buckets if we
use bucketing by the column id with 8 buckets. Notice that the pmod
function is called inside expr because the function is not directly
available in the PySpark API, but it is available in SQL (to see more
information about how the expr function can be used with SQL
functions, you can check my recent article about DataFrame
transformations).
Advantages of bucketing
The main goal of bucketing is to speed up queries and gain performance
improvements. There are two main areas where bucketing can help, the
first one is to avoid shuffle in queries with joins and aggregations, the
second one is to reduce the I/O with a feature called bucket pruning.
Let’s see both these optimization opportunities more in detail in the
following subsections.
Shuffle-free joins
If you are joining two tables and neither of them is particularly small,
Spark will have to make sure both tables are distributed on the cluster in
the same way (according to the joining key) and will therefore shuffle
the data (both tables will be shuffled). In the query plan, you will see an
Exchange operator in both branches of the join. Let’s see an example:
tableA.join(tableB, 'user_id')
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 6/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
If the join is planed with sort-merge join, the execution plan will look
like this:
Image by author
As you can see, each branch of the join contains an Exchange operator
that represents the shuffle (notice that Spark will not always use sort-
merge join for joining two tables — to see more details about the logic
that Spark is using for choosing a joining algorithm, see my other article
About Joins in Spark 3.0 where we discuss it in detail).
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 7/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
However, if both tables are bucketed by the joining key into the same
number of buckets, Spark will read the data on the cluster with this
specific distribution so it doesn’t require additional repartitioning and
shuffle — the Exchange operator will no longer be present in the plan:
Image by author
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 8/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
spark.conf.set("spark.sql.shuffle.partitions", n)
So if we use the default setting (200 partitions) and one of the tables
(let’s say tableA) is bucketed into, for example, 50 buckets and the other
table (tableB) is not bucketed at all, Spark will shuffle both tables and
will repartition the tables into 200 partitions.
spark.conf.set("spark.sql.shuffle.partitions", 50)
tableA.join(tableB, joining_key)
tableA
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 9/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
Both these techniques will lead to a one-side shuffle-free join, which can
be seen also from the query plan because the Exchange operator will be
in only one branch of the join and so only one table will be shuffled.
Before 3.1 the situation was actually similar to the previous case where
only one table is bucketed and the other is not, in other words, both
tables will be shuffled unless a specific condition with shuffle partitions
and the number of buckets is met in which case only one table will be
shuffled and we will get a one-side shuffle-free join. Here the condition
is similar as before — the number of shuffle partitions must be equal to
or less than the number of buckets of the bigger table. Let’s see this
more clearly on a simple example: if tableA has 50 buckets, tableB has
100, and the number of shuffle partitions is 200 (default), in that case,
both tables will be shuffled into 200 partitions. However, if the number
of shuffle partitions is set to 100 or less, only the tableA will be shuffled
into 100 partitions. Similarly, we can also repartition one of the tables to
the number of buckets of the other table in which case also only one
shuffle would happen during the execution.
In Spark 3.1.1 a new feature was implemented which can coalesce the
larger number of buckets into the smaller one if it bucket numbers are
multiples of each other. This feature is by default turned off and can be
controlled with this configuration setting
spark.sql.bucketing.coalesceBucketsInJoin.enabled. So if we turn it on and
have again tableA bucketed into 50 buckets, tableB into 100, the join will
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 10/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
Before Spark 3.0, it was possible to eliminate the Sort operator from the
join plan if each bucket was formed by exactly one file. In that case,
Spark was sure that the data is sorted after reading it on the cluster and
indeed the final plan was Sort-free. However, if there were more files
per bucket, Spark couldn’t guarantee that the data is globally sorted and
thus kept the Sort operator in the plan — the data had to be sorted
during the join execution. (See the section Bucketing from a Data
Engineer perspective below to learn how to achieve exactly one file per
bucket.)
In Spark 3.0 the situation changed and by default, the Sort is present
even if there is only one file per bucket. The reason for this change was
that listing all the files to check if there is only one per bucket was too
expensive (if there were too many files) so it was decided to turn off this
check and have the Sort in the plan all the time (for the sort-merge join).
As you can see it is a trade-off, one optimization for another. There was
also introduced a new configuration setting
spark.sql.legacy.bucketedTableScan.outputOrdering that you can set to True
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 11/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
to enforce the behavior before 3.0 and still leverage the sorted buckets
with one file.
Shuffle-free aggregations
Similarly to joins, aggregations also require correct distribution of the
data on the cluster and in general Spark will have to shuffle the data for
the following queries:
tableA
.groupBy('user_id')
.agg(count('*'))
tableA
.withColumn('n',
count('*').over(Window().partitionBy('user_id')))
If, however, the tableA is bucketed by the field user_id, both queries will
be shuffle-free.
Bucket pruning
Bucket pruning is a feature that was released in Spark 2.4 and its
purpose is to reduce I/O if we use a filter on the field by which the table
is bucketed. Let’s assume the following query:
spark.table('tableA').filter(col('user_id') == 123)
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 12/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
If the table is not bucketed, Spark will have to scan the entire table to
find this record and if the table is large, it can take many tasks that will
have to be launched and executed. On the other hand, if the table is
bucketed, Spark will know immediately to which bucket this row
belongs (Spark computes the hash function with the modulo to see
directly the bucket number) and will scan files only from the
corresponding bucket. And how does Spark know which files belong to
which bucket? Well, each file name has a specific structure and contains
information not only about the bucket to which it belongs, but also
which task produced the file as you can see from this picture:
Image by author
The bucket pruning can lead to a huge speed-up if the table is very large.
Disadvantages of bucketing
We just described the advantages that bucketing can offer. You might be
wondering whether there are also some disadvantages or simply some
situations in which it is better to avoid it. There is actually one
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 13/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
If a table is bucketed and over time it grew in size and the buckets
become large it could be more efficient to turn the bucketing off to
allow Spark to create more partitions and avoid problems with data
spill. This is useful especially if the query doesn’t do any operations that
could directly leverage the distribution provided by bucketing.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 14/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
Image by author
From this, you can see if the table is bucketed, what fields were used for
the bucketing and how many buckets the table has. Notice that we called
here show(n=100) because the show function displays by default only 20
rows, but if the schema of the table is large, the information about
bucketing will not appear in the first 20 rows, so just be aware that
depending on the table it might be needed to show more rows to see the
bucketing info.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 15/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
spark.conf.get("spark.sql.sources.bucketing.enabled")
df = spark.table(table_name)
# Spark will not use the information about bucketing:
df = spark.read.parquet(path_to_data)
Notice that in the second case where we approach the data directly from
the path, Spark will not communicate with the Hive metastore and will
not get the information about bucketing — bucketing will not be used.
Last but not least we can check the query plan and see if there are
Exchange operators in the plan in places where we want to avoid them.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 16/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
Before Spark 3.0, if the bucketing column has a different name in two
tables that we want to join and we rename the column in the DataFrame
to have the same name, the bucketing will stop working. For example,
tableA is bucketed by user_id, and tableB is bucketed by userId, the
column has the same meaning (we can join on it), but the name is
different (user_id vs userId). The bucketing will not be fully leveraged in
the following query:
tableA
.withColumnRenamed('user_id', 'userId')
.join(tableB, 'userId')
tableA
This issue was fixed in Spark 3.0 so renaming the column is no longer a
problem.
Another thing that is good to watch out for is the data types of the
joining columns — they need to be the same. Let’s assume this example:
tableA is bucketed by user_id which is of integer type, tableB is also
bucketed by user_id, but it is of long type and both tables are bucketed
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 17/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
into 50 buckets. In this situation the data types of the joining column are
different in each table, so Spark will have to cast it, will discard the
bucketing information and both tables will be shuffled:
# in both tables:
tableA.join(tableB, user_id)
It is quite unfortunate that both tables were created with a different data
type for a column that has the same meaning. Nevertheless, we can help
Spark to achieve at least one-side shuffle-free join as follows:
tableA
.withColumn('user_id', col('user_id').cast('long'))
.repartition(50, 'user_id')
.join(tableB, 'user_id')
As you can see we explicitly convert the data type to be the same in both
tables and then repartition the changed table into the same number of
partitions as is the other table. The shuffle will happen only on this side
where we repartition it, the other table will be shuffle-free. This
basically becomes equivalent to the situation in which only one table is
bucketed and the other is not.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 18/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
tableA.withColumn('x', my_udf('some_col'))
.join(tableB, 'user_id')
)
# One-side shuffle-free join:
tableA.withColumn('x', my_udf('some_col'))
.join(tableB, 'user_id')
)
# One-side shuffle-free join:
tableA.withColumn('x', my_udf('some_col'))
.join(tableB, 'user_id')
And if we want to totally avoid the shuffle, we can simply call the UDF
after the join
tableA
.join(tableB, 'user_id')
.withColumn('x', my_udf('some_col'))
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 19/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
We can fix this problem by achieving already on the cluster the same
distribution that we want to have in the file system (in the storage). If
each task has data only for one bucket, in that case, each task will write
only one file. That can be achieved by custom repartition before the
write
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 20/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
df.repartition(expr("pmod(hash(user_id), 200)"))
.write
.mode(saving_mode) # append/overwrite
.bucketBy(200, 'user_id')
.option("path", output_path)
.saveAsTable(table_name)
This will create exactly one file per bucket. As you can see we
repartition the data exactly by the same expression that is Spark using
under the hood to distribute the data among the buckets (see the
relevant section above for more details on how this works). You could
actually use here more simple df.repartition(200, ‘user_id’) with the same
result, but the advantage of the above approach is that it works also if
you want to at the same time partition the data in the filesystem by
another field as follows
df
.repartition(200,
"created_year",expr("pmod(hash(user_id), 200)"))
.write
.mode(saving_mode)
.partitionBy("created_year")
.bucketBy(200, "user_id")
.option("path", output_path)
.saveAsTable(table_name)
Here each file system partition will have exactly 200 files (one file per
bucket), so the total number of files will be the number of buckets
multiplied by the number of file system partitions. Notice that this
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 21/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
In practice, the situation is more complex and one has to face the
following challenges:
the table is continuously appended and its size grows in time and so
does the size of the buckets. In some cases, this may still be fine if
the dataset is also partitioned by some date dimension (year and
month for example) and the buckets are distributed uniformly
across these partitions. If the typical query asks always only for
recent data, for example, the last 6 months, we can design the
buckets so the reasonable size corresponds to the 6 months of data.
The total size of the bucket will grow, but it doesn’t matter, because
we will never ask for the entire bucket.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 22/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
and this will lead to data skew — the task that will process this bigger
bucket will take longer than the other.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 23/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
Future improvements
Here is listed a couple of features that are not implemented at the time
of writing this article (April 2021):
Add bucket scan info to explain (see Jira) — see the information if
the bucketing is used in the query plan
Read multiple sorted bucket files (see Jira) — leverage the sorted
buckets for the sort-merge join even if there are more files per
bucket
392 5
spark.sql.sources.bucketing.maxBuckets — maximum number of
buckets that can be used for a table. By default, it is 100 000.
spark.sql.sources.bucketing.autoBucketedScan.enabled — it will
discard bucketing information if it is not useful (based on the query
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 24/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
spark.sql.bucketing.coalesceBucketsInJoin.maxBucketRatio — the
maximum ratio of the two bucket numbers to have the coalescing
work. By default, it is 4. In other words, if one table has more than 4
times the number of buckets than the other table, the coalescing will
not take place.
Final discussion
In this report, we described bucketing from different perspectives. We
have seen some of the issues that a data engineer needs to deal with
when creating a bucketed table, like choosing a reasonable number of
buckets and controlling the number of created files. We have also
discussed the data analyst view — having bucketed tables provides
optimization opportunities. In many cases, these opportunities are
utilized by Spark out of the box, in some situations, however, extra care
needs to be taken to leverage the bucketing potential. This happens with
joins of tables where the bucketing details differ, for example, the tables
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 25/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
Query Optimization
Every Thursday, the Variable delivers the very best of Towards Data Science: from hands-on tutorials
and cutting-edge research to original features you don't want to miss. Take a look.
By signing up, you will create a Medium account if you don’t already have one. Review
our Privacy Policy for more information about our privacy practices.
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 26/27
1/3/23, 6:16 PM Best Practices for Bucketing in Spark SQL | by David Vrba | Towards Data Science
https://towardsdatascience.com/best-practices-for-bucketing-in-spark-sql-ea9f23f7dd53 27/27