-
Notifications
You must be signed in to change notification settings - Fork 424
Usage Doc
- Include Dependencies in Your Spark Environment
- Usage
LakeSoul is currently available with Scala version 2.12 and Spark version 3.1.2.
To use spark-shell, please refer to QuickStart
To include lakesoul in your Spark scala/java project, include the maven dependency:
<dependency>
<groupId>com.dmetasoul</groupId>
<artifactId>lakesoul</artifactId>
<version>2.0.1-spark-3.1.2</version>
</dependency>
The table name in LakeSoul is a path, and the path where the data is stored is the table name.
When Dataframe.write(or writeStream) is called to write data to LakeSoulTable, a new table will automatically created using the storage path if the table does not exist.
LakeSoul manages metadata through external database, so it can process metadata efficiently, and the meta cluster can be easily scaled up in the cloud.
LakeSoulTable can be partitioned in two ways, range and hash, and they can be used at the same time.
- Range partition is a common time-based table partition. Data files of different partitions are stored in different partition paths.
- To use a hash partition, you must specify both the hash primary key fields and the hash bucket num. The hash bucket num is used to hash the hash primary key fields.
- If you specify both range partition and hash partition, each range partition will have the same hash key written to file with the same bucket id.
- When partitioning is specified, data written to LakeSoulTable must contain partitioning fields.
Depending on the specific scenario, you can choose to use a range partition, a hash partition, or both. When a hash partition is specified, the data in LakeSoulTable will be unique by the primary key, which is the hash partition field + range partition field (if any).
When a hash partition is specified, LakeSoulTable supports upsert operations, where writing to data in APPEND mode is disabled, and the lakeSoulTable.upsert()
method can be used instead.
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
import spark.implicits._
val df = Seq(("2021-01-01",1,"rice"),("2021-01-01",2,"bread")).toDF("date","id","name")
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
//create table
//spark batch
df.write
.mode("append")
.format("lakesoul")
.option("rangePartitions","date")
.option("hashPartitions","id")
.option("hashBucketNum","2")
.save(tablePath)
//spark streaming
import org.apache.spark.sql.streaming.Trigger
val readStream = spark.readStream.parquet("inputPath")
val writeStream = readStream.writeStream
.outputMode("append")
.trigger(Trigger.ProcessingTime("1 minutes"))
.format("lakesoul")
.option("rangePartitions","date")
.option("hashPartitions","id")
.option("hashBucketNum", "2")
.option("checkpointLocation", "s3a://bucket-name/checkpoint/path")
.start(tablePath)
writeStream.awaitTermination()
//for existing table, it no longer need to specify partition information when writing data
//equivalent to INSERT OVERWRITE PARTITION, if you do not specify option replaceWhere, the entire table will be overwritten
df.write
.mode("overwrite")
.format("lakesoul")
.option("replaceWhere","date='2021-01-01'")
.save(tablePath)
You can read data by Spark API or building LakeSoulTable, Spark SQL is also supported, see 8. Operate LakeSoulTable by Spark SQL
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
//by spark api
val df1 = spark.read.format("lakesoul").load(tablePath)
//by LakeSoulTableRel
val df2 = LakeSoulTable.forPath(tablePath).toDF
Upsert is supported when hash partitioning has been specified.
MergeOnRead is used by default, upsert data is written as delta files. LakeSoul provides efficient upsert and merge scan performance.
Parameter spark.dmetasoul.lakesoul.deltaFile.enabled
can be set to false
to use CopyOnWrite mode, eventually merged data will be generated after upsert, but this mode is not recommended, because it has poor performance and low concurrent.
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
import spark.implicits._
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
val lakeSoulTable = LakeSoulTable.forPath(tablePath)
val extraDF = Seq(("2021-01-01",3,"chicken")).toDF("date","id","name")
lakeSoulTable.upsert(extraDF)
In streaming, when outputMode is complete, each write will overwrite all previous data.
When outputMode is append or update, if hash partition is specified, each write is treated as an upsert, if data with the same primary key exists at read time, the latest value of the same key overrides the previous one. Update mode is available only if hash partition is specified.
Duplicate data is allowed if no hash partitioning is used.
LakeSoul supports update operations, which are performed by specifying the condition and the field Expression that needs to be updated. There are several ways to perform update, see annotations in LakeSoulTable
.
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
val lakeSoulTable = LakeSoulTable.forPath(tablePath)
import org.apache.spark.sql.functions._
//update(condition, set)
lakeSoulTable.update(col("date") > "2021-01-01", Map("date" -> lit("2021-01-02")))
LakeSoul supports delete operation to delete data that meet the conditions. Conditions can be any field, and if no condition is specified, all data in table will be deleted.
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
val lakeSoulTable = LakeSoulTable.forPath(tablePath)
//delete data that meet the condition
lakeSoulTable.delete("date='2021-01-01'")
//delete full table data
lakeSoulTable.delete()
Upsert will generates delta files, which can affect read efficiency when delta files num become too large, in this time, compaction can be performed to merge files.
When compaction is performed to the full table, you can set conditions for compaction, only range partitions that meet the conditions will perform compaction.
Conditions to trigger compaction:
- The last modification time for a range partition is before
spark.dmetasoul.lakesoul.compaction.interval
(ms), default is 12 hours - Delta file num exceeds
spark.dmetasoul.lakesoul.deltaFile.max.num
, default is 5
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
val lakeSoulTable = LakeSoulTable.forPath(tablePath)
//compaction on the specified partition
lakeSoulTable.compaction("date='2021-01-01'")
//compAction on all partitions of the table
lakeSoulTable.compaction()
//compaction on all partitions, but only partitions meet the conditions will be performed
lakeSoulTable.compaction(false)
Spark SQL is supported to read and write LakeSoulTable. To use it, you need to set spark.sql.catalog.spark_catalog
to org.apache.spark.sql.lakesoul.catalog.LakeSoulCatalog
.
Note:
- Insert into statement turns
autoMerge
on by default - Spark SQL does not support to set hash partition while creating a LakeSoulTable
- Cannot perform INSERT INTO on a hash partitioned table, use
lakeSoulTable.upsert()
instead - Some Spark SQL statements are not supported, see
org.apache.spark.sql.lakesoul.rules.LakeSoulUnsupportedOperationsCheck
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.lakesoul.catalog.LakeSoulCatalog")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
spark.range(10).createOrReplaceTempView("tmpView")
//write
spark.sql(s"insert overwrite table lakesoul.`$tablePath` partition (date='2021-01-01') select id from tmpView")
//INSERT INTO cannot be used on a hash partitioned table, use `lakeSoulTable.upsert()` instead
spark.sql(s"insert into lakesoul.`$tablePath` select * from tmpView")
//read
spark.sql(s"select * from lakesoul.`$tablePath`").show()
When hash partition is specified, the data in each range partition is partitioned according to the hash primary key and the partitioned data is ordered. Therefore, there is no need to do shuffle and sort when some operators perform on hash primary key.
LakeSoul currently supports optimization of join, intersect, and except, and more operators will be supported in the future.
Scenarios:
- Shuffle and sort are not required when data from different partitions of the same table is joined on the hash keys
- If two different tables have the same hash field type and number of fields, and the same hash bucket num, there is no need to shuffle and sort when they are joined on the hash keys
Scenarios:
- Intersect/Except on hash keys for different partitions of the same table does not require shuffle, sort, and distinct
- Intersect/Except on hash keys for different tables that have the same type and number of hash keys, and the same hash bucket num, there is no need to shuffle, sort, and distinct
In a range partition, the hash primary keys are unique, so the results of intersect or except are not repeated, so the subsequent operations do not need to deduplicate again. For example, you can directly count
the number of data, without the need for count distinc
.
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.lakesoulsoul.catalog.LakeSoulCatalog")
.getOrCreate()
import spark.implicits._
val df1 = Seq(("2021-01-01",1,1,"rice"),("2021-01-02",2,2,"bread")).toDF("date","id1","id2","name")
val df2 = Seq(("2021-01-01",1,1,2.7),("2021-01-02",2,2,1.3)).toDF("date","id3","id4","price")
val tablePath1 = "s3a://bucket-name/table/path/is/also/table/name/1"
val tablePath2 = "s3a://bucket-name/table/path/is/also/table/name/2"
df1.write
.mode("append")
.format("lakesoul")
.option("rangePartitions","date")
.option("hashPartitions","id1,id2")
.option("hashBucketNum","2")
.save(tablePath1)
df2.write
.mode("append")
.format("lakesoul")
.option("rangePartitions","date")
.option("hashPartitions","id3,id4")
.option("hashBucketNum","2")
.save(tablePath2)
//join on hash keys without shuffle and sort
//different range partitions for the same table
spark.sql(
s"""
|select t1.*,t2.* from
| (select * from lakesoul.`$tablePath1` where date='2021-01-01') t1
| join
| (select * from lakesoul.`$tablePath1` where date='2021-01-02') t2
| on t1.id1=t2.id1 and t1.id2=t2.id2
""".stripMargin)
.show()
//different tables with the same hash setting
spark.sql(
s"""
|select t1.*,t2.* from
| (select * from lakesoul.`$tablePath1` where date='2021-01-01') t1
| join
| (select * from lakesoul.`$tablePath2` where date='2021-01-01') t2
| on t1.id1=t2.id3 and t1.id2=t2.id4
""".stripMargin)
.show()
//intersect/except on hash keys without shuffle,sort and distinct
//different range partitions for the same table
spark.sql(
s"""
|select count(1) from
| (select id1,id2 from lakesoul.`$tablePath1` where date='2021-01-01'
| intersect
| select id1,id2 from lakesoul.`$tablePath1` where date='2021-01-02') t
""".stripMargin)
.show()
//different tables with the same hash setting
spark.sql(
s"""
|select count(1) from
| (select id1,id2 from lakesoul.`$tablePath1` where date='2021-01-01'
| intersect
| select id3,id4 from lakesoul.`$tablePath2` where date='2021-01-01') t
""".stripMargin)
.show()
LakeSoul supports Schema Evolution, new columns allowed to be added (partitioning fields cannot be modified). When a new column is added and the existing data is read, the new column will be NULL. You can fill the new columns by upsert operation.
Specify mergeSchema
to true
or enable autoMerge
to merge the schema when writing data. The new schema is the union of table schema and the current written data schema.
df.write
.mode("append")
.format("lakesoul")
.option("rangePartitions","date")
.option("hashPartitions","id")
.option("hashBucketNum","2")
//first way
.option("mergeSchema","true")
.save(tablePath)
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
//second way
.config("spark.dmetasoul.lakesoul.schema.autoMerge.enabled", "true")
.getOrCreate()
Drop a partition, also known as drop range partition, does not actually delete the data files. You can use the Cleanup operation to cleanup stale data.
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
val lakeSoulTable = LakeSoulTable.forPath(tablePath)
//drop the specified range partition
lakeSoulTable.dropPartition("date='2021-01-01'")
Drop table will directly deletes all the metadata and files.
import com.dmetasoul.lakesoul.tables.LakeSoulTable
import org.apache.spark.sql._
val spark = SparkSession.builder.master("local")
.config("spark.sql.extensions", "com.dmetasoul.lakesoul.sql.LakeSoulSparkSessionExtension")
.getOrCreate()
val tablePath = "s3a://bucket-name/table/path/is/also/table/name"
val lakeSoulTable = LakeSoulTable.forPath(tablePath)
//drop table
lakeSoulTable.dropTable()
English | 中文(简体) |
---|