├── .gitignore ├── README.md ├── dev ├── change-scala-version.sh ├── change-version-to-2.11.sh ├── change-version-to-2.12.sh └── release.sh ├── pom.xml └── src ├── main └── java │ ├── org │ └── apache │ │ └── spark │ │ └── sql │ │ └── delta │ │ ├── JavaDoc.java │ │ ├── binlog.scala │ │ ├── commands │ │ ├── BinlogSyncToDelta.scala │ │ ├── CompactTableInDelta.scala │ │ ├── DeltaCommandsFun.scala │ │ └── UpsertTableInDelta.scala │ │ ├── sources │ │ ├── MLSQLDeltaDataSource.scala │ │ ├── MLSQLDeltaSink.scala │ │ └── protocols.scala │ │ └── upsert.scala │ └── tech │ └── mlsql │ └── common │ ├── BitUtil.java │ ├── BloomFilter.java │ ├── DeltaJob.scala │ ├── JsonUtils.scala │ ├── ParquetReaderIterator.java │ ├── PathFun.scala │ ├── ScalaReflect.scala │ └── ZOrderingIndexUtils.java └── test └── java └── tech └── mlsql └── test └── delta ├── DeltaCompactionSuite.scala └── DeltaUpsertSuite.scala /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | delta-plus.iml 3 | target/ 4 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Delta Plus 2 | 3 | A library based on delta for Spark and [MLSQL](http://www.mlsql.tech). 4 | 5 | * JianShu [How delta works](https://www.jianshu.com/p/af55a2df2af8) 6 | * Medium [How delta works](https://medium.com/@williamsmith_74955/how-delta-works-4519c62aa469) 7 | * Video [是时候改变你的增量同步方案了](https://www.bilibili.com/video/av78170428/) 8 | * ZhiHu [是时候改变你的增量同步方案了](https://zhuanlan.zhihu.com/p/93744164) 9 | 10 | ## Requirements 11 | 12 | This library requires Spark 2.4+ (tested) and Delta 0.5.0(for spark 2.4.x)/0.7.0(for spark 3.0.x). 13 | 14 | ## Linking 15 | You can link against this library in your program at the following coordinates: 16 | 17 | ### Scala 2.11/Spark 2.4.3 18 | 19 | ```sql 20 | groupId: tech.mlsql 21 | artifactId: delta-plus_2.11 22 | version: 0.4.0 23 | ``` 24 | 25 | ### Scala 2.12/Spark3.0.0 26 | 27 | ```sql 28 | groupId: tech.mlsql 29 | artifactId: delta-plus_2.12 30 | version: 0.4.0 31 | ``` 32 | 33 | ## ChangeLog 34 | 35 | ### 0.4.0 36 | 1. fix issue if idCols is not the first columns, the schema will be changed. 37 | 38 | ## Limitation 39 | 40 | 1. Compaction can not be applied to delta table which will be operated by upsert/delete action. 41 | 42 | 43 | ## Binlog Replay Support 44 | 45 | To incremental sync MySQL table to Delta Lake, you should combine delta-plus with project 46 | [spark-binlog](https://github.com/allwefantasy/spark-binlog). 47 | 48 | DataFrame: 49 | 50 | ```scala 51 | val spark = SparkSession.builder() 52 | .master("local[*]") 53 | .appName("Binlog2DeltaTest") 54 | .getOrCreate() 55 | 56 | val df = spark.readStream. 57 | format("org.apache.spark.sql.mlsql.sources.MLSQLBinLogDataSource"). 58 | option("host","127.0.0.1"). 59 | option("port","3306"). 60 | option("userName","root"). 61 | option("password","123456"). 62 | option("databaseNamePattern","test"). 63 | option("tableNamePattern","mlsql_binlog"). 64 | option("bingLogNamePrefix","mysql-bin"). 65 | option("binlogIndex","10"). 66 | option("binlogFileOffset","90840"). 67 | load() 68 | 69 | val query = df.writeStream. 70 | format("org.apache.spark.sql.delta.sources.MLSQLDeltaDataSource"). 71 | option("__path__","/tmp/datahouse/{db}/{table}"). 72 | option("path","{db}/{table}"). 73 | option("mode","Append"). 74 | option("idCols","id"). 75 | option("duration","3"). 76 | option("syncType","binlog"). 77 | option("checkpointLocation", "/tmp/cpl-binlog2"). 78 | outputMode("append") 79 | .trigger(Trigger.ProcessingTime("3 seconds")) 80 | .start() 81 | 82 | query.awaitTermination() 83 | ``` 84 | 85 | MLSQL Code: 86 | 87 | ```sql 88 | set streamName="binlog"; 89 | 90 | load binlog.`` where 91 | host="127.0.0.1" 92 | and port="3306" 93 | and userName="xxxx" 94 | and password="xxxxxx" 95 | and bingLogNamePrefix="mysql-bin" 96 | and binlogIndex="4" 97 | and binlogFileOffset="4" 98 | and databaseNamePattern="mlsql_console" 99 | and tableNamePattern="script_file" 100 | as table1; 101 | 102 | save append table1 103 | as rate.`mysql_{db}.{table}` 104 | options mode="Append" 105 | and idCols="id" 106 | and duration="5" 107 | and syncType="binlog" 108 | and checkpointLocation="/tmp/cpl-binlog2"; 109 | 110 | ``` 111 | 112 | Before you run the streaming application, make sure you have fully sync the table . 113 | 114 | MLSQL Code: 115 | 116 | ```sql 117 | connect jdbc where 118 | url="jdbc:mysql://127.0.0.1:3306/mlsql_console?characterEncoding=utf8&zeroDateTimeBehavior=convertToNull&tinyInt1isBit=false" 119 | and driver="com.mysql.jdbc.Driver" 120 | and user="xxxxx" 121 | and password="xxxx" 122 | as db_cool; 123 | 124 | load jdbc.`db_cool.script_file` as script_file; 125 | 126 | run script_file as TableRepartition.`` where partitionNum="2" and partitionType="range" and partitionCols="id" 127 | as rep_script_file; 128 | 129 | save overwrite rep_script_file as delta.`mysql_mlsql_console.script_file` ; 130 | 131 | load delta.`mysql_mlsql_console.script_file` as output; 132 | ``` 133 | 134 | DataFrame Code: 135 | 136 | ```scala 137 | import org.apache.spark.sql.SparkSession 138 | val spark = SparkSession.builder() 139 | .master("local[*]") 140 | .appName("wow") 141 | .getOrCreate() 142 | 143 | val mysqlConf = Map( 144 | "url" -> "jdbc:mysql://localhost:3306/mlsql_console?characterEncoding=utf8&zeroDateTimeBehavior=convertToNull&tinyInt1isBit=false", 145 | "driver" -> "com.mysql.jdbc.Driver", 146 | "user" -> "xxxxx", 147 | "password" -> "xxxxxx", 148 | "dbtable" -> "script_file" 149 | ) 150 | 151 | import org.apache.spark.sql.functions.col 152 | var df = spark.read.format("jdbc").options(mysqlConf).load() 153 | df = df.repartitionByRange(2, col("id") ) 154 | df.write 155 | .format("org.apache.spark.sql.delta.sources.MLSQLDeltaDataSource"). 156 | mode("overwrite"). 157 | save("/tmp/datahouse/mlsql_console/script_file") 158 | spark.close() 159 | ``` 160 | 161 | 162 | ## Upsert/Delete Support 163 | 164 | DataFrame: 165 | 166 | ```scala 167 | 168 | df.writeStream. 169 | format("org.apache.spark.sql.delta.sources.MLSQLDeltaDataSource"). 170 | option("idCols","id"). // this means will execute upsert 171 | option("operation","delete"). // this means will delete data in df 172 | .mode(OutputMode.Append).save("/tmp/delta-table1") 173 | 174 | df.readStream.format("org.apache.spark.sql.delta.sources.MLSQLDeltaDataSource").load("/tmp/delta-table1") 175 | 176 | ``` 177 | 178 | when `idCols` and `operation` is not configured, then we will execute normal Append/Overwrite operation. 179 | If you have `idCols` setup, then it will execute Upsert operation. 180 | If you have `idCols`, `operation` both setup and operation equal to `delete`, then it will delete table records in df. 181 | 182 | 183 | > Notice that if the data which will be written to the delta table have duplicate records, delta-plus will throw exception 184 | by default. If you wanna do deduplicating, set `dropDuplicate` as true. 185 | 186 | MLSQL: 187 | 188 | ```sql 189 | save append table1 190 | as rate.`mysql_{db}.{table}` 191 | options mode="Append" 192 | and idCols="id" 193 | and duration="5" 194 | and syncType="binlog" 195 | and checkpointLocation="/tmp/cpl-binlog2"; 196 | ``` 197 | 198 | ## CompactionSupport 199 | 200 | DataFrame: 201 | 202 | ```scala 203 | 204 | val optimizeTableInDelta = CompactTableInDelta(log, 205 | new DeltaOptions(Map[String, String](), df.sparkSession.sessionState.conf), Seq(), Map( 206 | CompactTableInDelta.COMPACT_VERSION_OPTION -> "8", 207 | CompactTableInDelta.COMPACT_NUM_FILE_PER_DIR -> "1", 208 | CompactTableInDelta.COMPACT_RETRY_TIMES_FOR_LOCK -> "60" 209 | )) 210 | val items = optimizeTableInDelta.run(df.sparkSession) 211 | 212 | ``` 213 | 214 | MLSQL: 215 | 216 | ```sql 217 | -- compact table1 files before version 10, and make 218 | -- sure every partition only have one file 219 | !delta compact /delta/table1 10 1; 220 | ``` 221 | 222 | You can use `!delta history /delta/table1;` to get the history of the table. 223 | 224 | 225 | 226 | 227 | 228 | 229 | 230 | -------------------------------------------------------------------------------- /dev/change-scala-version.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -e 21 | 22 | VALID_VERSIONS=( 2.11 2.12 ) 23 | 24 | usage() { 25 | echo "Usage: $(basename $0) [-h|--help] 26 | where : 27 | -h| --help Display this help text 28 | valid version values : ${VALID_VERSIONS[*]} 29 | " 1>&2 30 | exit 1 31 | } 32 | 33 | if [[ ($# -ne 1) || ( $1 == "--help") || $1 == "-h" ]]; then 34 | usage 35 | fi 36 | 37 | TO_VERSION=$1 38 | 39 | check_scala_version() { 40 | for i in ${VALID_VERSIONS[*]}; do [ $i = "$1" ] && return 0; done 41 | echo "Invalid Scala version: $1. Valid versions: ${VALID_VERSIONS[*]}" 1>&2 42 | exit 1 43 | } 44 | 45 | check_scala_version "$TO_VERSION" 46 | 47 | if [ $TO_VERSION = "2.12" ]; then 48 | FROM_VERSION="2.11" 49 | else 50 | FROM_VERSION="2.12" 51 | fi 52 | 53 | sed_i() { 54 | sed -e "$1" "$2" > "$2.tmp" && mv "$2.tmp" "$2" 55 | } 56 | 57 | export -f sed_i 58 | 59 | BASEDIR=$(dirname $0)/.. 60 | find "$BASEDIR" -name 'pom.xml' -not -path '*target*' -print \ 61 | -exec bash -c "sed_i 's/\(artifactId.*\)_'$FROM_VERSION'/\1_'$TO_VERSION'/g' {}" \; 62 | 63 | # Also update in parent POM 64 | # Match any scala binary version to ensure idempotency 65 | sed_i '1,/[0-9]*\.[0-9]*[0-9]*\.[0-9]*'$TO_VERSION' 2 | 5 | 4.0.0 6 | 7 | tech.mlsql 8 | delta-plus_2.11 9 | 0.4.0 10 | Delta Plus 11 | https://github.com/allwefantasy/delta-plus.git 12 | 13 | A library based on delta for Spark and [MLSQL](http://www.mlsql.tech). 14 | 15 | 16 | 17 | Apache 2.0 License 18 | http://www.apache.org/licenses/LICENSE-2.0.html 19 | repo 20 | 21 | 22 | 23 | 24 | allwefantasy 25 | ZhuHaiLin 26 | allwefantasy@gmail.com 27 | 28 | 29 | 30 | 31 | scm:git:git@github.com:allwefantasy/delta-plus.git 32 | 33 | 34 | scm:git:git@github.com:allwefantasy/delta-plus.git 35 | 36 | https://github.com/allwefantasy/delta-plus 37 | 38 | 39 | https://github.com/allwefantasy/delta-plus/issues 40 | 41 | 42 | 43 | UTF-8 44 | 2.11.8 45 | 2.11 46 | 2.11.0-M3 47 | 48 | 2.4.3 49 | 2.4 50 | 51 | 16.0 52 | 4.5.3 53 | 54 | provided 55 | 2.6.5 56 | 0.3.6 57 | 2.0.6 58 | 59 | 60 | 61 | 62 | 63 | io.delta 64 | delta-core_${scala.binary.version} 65 | 0.5.0 66 | 67 | 68 | org.scala-lang 69 | scala-library 70 | 71 | 72 | 73 | 74 | 75 | tech.mlsql 76 | common-utils_${scala.binary.version} 77 | ${common-utils-version} 78 | 79 | 80 | 81 | net.sf.json-lib 82 | json-lib 83 | 2.4 84 | jdk15 85 | 86 | 87 | 88 | org.scalactic 89 | scalactic_${scala.binary.version} 90 | 3.0.0 91 | test 92 | 93 | 94 | org.scalatest 95 | scalatest_${scala.binary.version} 96 | 3.0.0 97 | test 98 | 99 | 100 | 101 | org.apache.spark 102 | spark-core_${scala.binary.version} 103 | ${spark.version} 104 | ${scope} 105 | 106 | 107 | org.apache.spark 108 | spark-sql_${scala.binary.version} 109 | ${spark.version} 110 | ${scope} 111 | 112 | 113 | 114 | org.apache.spark 115 | spark-mllib_${scala.binary.version} 116 | ${spark.version} 117 | ${scope} 118 | 119 | 120 | 121 | org.pegdown 122 | pegdown 123 | 1.6.0 124 | test 125 | 126 | 127 | 128 | org.apache.spark 129 | spark-graphx_${scala.binary.version} 130 | ${spark.version} 131 | ${scope} 132 | 133 | 134 | 135 | org.apache.spark 136 | spark-sql-kafka-0-10_${scala.binary.version} 137 | ${spark.version} 138 | ${scope} 139 | 140 | 141 | 142 | com.github.shyiko 143 | mysql-binlog-connector-java 144 | 0.18.1 145 | 146 | 147 | 148 | org.apache.spark 149 | spark-catalyst_${scala.binary.version} 150 | ${spark.version} 151 | tests 152 | test 153 | 154 | 155 | 156 | org.apache.spark 157 | spark-core_${scala.binary.version} 158 | ${spark.version} 159 | tests 160 | test 161 | 162 | 163 | 164 | org.apache.spark 165 | spark-sql_${scala.binary.version} 166 | ${spark.version} 167 | tests 168 | test 169 | 170 | 171 | 172 | 173 | 174 | 175 | 176 | 177 | disable-java8-doclint 178 | 179 | [1.8,) 180 | 181 | 182 | -Xdoclint:none 183 | none 184 | 185 | 186 | 187 | release-sign-artifacts 188 | 189 | 190 | performRelease 191 | true 192 | 193 | 194 | 195 | 196 | 197 | org.apache.maven.plugins 198 | maven-gpg-plugin 199 | 1.1 200 | 201 | 202 | sign-artifacts 203 | verify 204 | 205 | sign 206 | 207 | 208 | 209 | 210 | 211 | 212 | 213 | 214 | 215 | 216 | src/main/java/ 217 | 218 | 219 | src/main/resources 220 | 221 | 222 | 223 | 224 | org.apache.maven.plugins 225 | maven-surefire-plugin 226 | 3.0.0-M1 227 | 228 | 1 229 | true 230 | -Xmx4024m 231 | 232 | **/*.java 233 | **/*.scala 234 | 235 | 236 | 237 | 238 | 239 | 240 | 241 | org.scala-tools 242 | maven-scala-plugin 243 | 2.15.2 244 | 245 | 246 | 247 | -g:vars 248 | 249 | 250 | true 251 | 252 | 253 | 254 | scala-compile-first 255 | process-resources 256 | 257 | compile 258 | 259 | 260 | 261 | scala-test-compile 262 | process-test-resources 263 | 264 | testCompile 265 | 266 | 267 | 268 | 269 | 270 | 271 | org.apache.maven.plugins 272 | maven-compiler-plugin 273 | 2.3.2 274 | 275 | 276 | -g 277 | true 278 | 1.8 279 | 1.8 280 | 281 | 282 | 283 | 284 | 285 | 286 | maven-source-plugin 287 | 2.1 288 | 289 | true 290 | 291 | 292 | 293 | compile 294 | 295 | jar 296 | 297 | 298 | 299 | 300 | 301 | org.apache.maven.plugins 302 | maven-javadoc-plugin 303 | 304 | 305 | attach-javadocs 306 | 307 | jar 308 | 309 | 310 | 311 | 312 | 313 | org.sonatype.plugins 314 | nexus-staging-maven-plugin 315 | 1.6.7 316 | true 317 | 318 | sonatype-nexus-staging 319 | https://oss.sonatype.org/ 320 | true 321 | 322 | 323 | 324 | 325 | org.scalatest 326 | scalatest-maven-plugin 327 | 2.0.0 328 | 329 | streaming.core.NotToRunTag 330 | ${project.build.directory}/surefire-reports 331 | . 332 | WDF TestSuite.txt 333 | ${project.build.directory}/html/scalatest 334 | false 335 | 336 | 337 | 338 | test 339 | 340 | test 341 | 342 | 343 | 344 | 345 | 346 | 347 | 348 | 349 | sonatype-nexus-snapshots 350 | https://oss.sonatype.org/content/repositories/snapshots 351 | 352 | 353 | sonatype-nexus-staging 354 | https://oss.sonatype.org/service/local/staging/deploy/maven2/ 355 | 356 | 357 | 358 | 359 | 360 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/JavaDoc.java: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta; 2 | 3 | /** 4 | * 2019-06-20 WilliamZhu(allwefantasy@gmail.com) 5 | */ 6 | public class JavaDoc { 7 | } 8 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/binlog.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta 2 | 3 | case class TableMetaInfo(db: String, table: String, schema: String) 4 | 5 | object MLSQLMultiDeltaOptions { 6 | val META_KEY = "__meta__" 7 | val KEEP_BINLOG = "keepBinlog" 8 | val FULL_PATH_KEY = "__path__" 9 | val BINLOG_PATH = "binlogPath" 10 | } 11 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/commands/BinlogSyncToDelta.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.commands 2 | 3 | import net.sf.json.{JSONArray, JSONObject} 4 | import org.apache.spark.rdd.RDD 5 | import org.apache.spark.sql.catalyst.expressions.JsonToStructs 6 | import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions, MLSQLMultiDeltaOptions, TableMetaInfo} 7 | import org.apache.spark.sql.types.{DataType, StructType} 8 | import org.apache.spark.sql.{Column, Dataset, Row, SaveMode, functions => F} 9 | import tech.mlsql.common.utils.Md5 10 | 11 | import scala.collection.JavaConverters._ 12 | import scala.util.Try 13 | 14 | /** 15 | * 28/11/2019 WilliamZhu(allwefantasy@gmail.com) 16 | */ 17 | object BinlogSyncToDelta extends DeltaCommandsFun { 18 | def run(_ds: Dataset[Row], options: Map[String, String]): Unit = { 19 | val idCols = options(UpsertTableInDelta.ID_COLS).split(",").toSeq 20 | val newDataParallelNum = options.getOrElse(UpsertTableInDelta.NEW_DATA_PARALLEL_NUM, "8").toInt 21 | var ds = convertStreamDataFrame(_ds).asInstanceOf[Dataset[Row]] 22 | if (newDataParallelNum != ds.rdd.partitions.size) { 23 | ds = ds.repartition(newDataParallelNum) 24 | } 25 | ds.cache() 26 | try { 27 | if (options.getOrElse(MLSQLMultiDeltaOptions.KEEP_BINLOG, "false").toBoolean) { 28 | val originalLogPath = options(MLSQLMultiDeltaOptions.BINLOG_PATH) 29 | ds.write.format("org.apache.spark.sql.delta.sources.MLSQLDeltaDataSource").mode(SaveMode.Append).save(originalLogPath) 30 | } else { 31 | // do cache 32 | ds.count() 33 | } 34 | 35 | 36 | def _getInfoFromMeta(record: JSONObject, key: String) = { 37 | record.getJSONObject(MLSQLMultiDeltaOptions.META_KEY).getString(key) 38 | } 39 | 40 | def getDatabaseNameFromMeta(record: JSONObject) = { 41 | _getInfoFromMeta(record, "databaseName") 42 | } 43 | 44 | def getTableNameNameFromMeta(record: JSONObject) = { 45 | _getInfoFromMeta(record, "tableName") 46 | } 47 | 48 | def getschemaNameFromMeta(record: JSONObject) = { 49 | _getInfoFromMeta(record, "schema") 50 | } 51 | 52 | val spark = ds.sparkSession 53 | import spark.implicits._ 54 | val dataSet = ds.rdd.flatMap { row => 55 | val value = row.getString(0) 56 | val wow = JSONObject.fromObject(value) 57 | val rows = wow.remove("rows") 58 | rows.asInstanceOf[JSONArray].asScala.map { record => 59 | record.asInstanceOf[JSONObject].put(MLSQLMultiDeltaOptions.META_KEY, wow) 60 | record.asInstanceOf[JSONObject] 61 | } 62 | } 63 | 64 | val finalDataSet = dataSet.map { record => 65 | val idColKey = idCols.map { idCol => 66 | record.get(idCol).toString 67 | }.mkString("") 68 | val key = Md5.md5Hash(getDatabaseNameFromMeta(record) + "_" + getTableNameNameFromMeta(record) + "_" + idColKey) 69 | (key, record.toString) 70 | }.groupBy(_._1).map { f => f._2.map(m => JSONObject.fromObject(m._2)) }.map { records => 71 | // we get the same record operations, and sort by timestamp, get the last operation 72 | val items = records.toSeq.sortBy(record => record.getJSONObject(MLSQLMultiDeltaOptions.META_KEY).getLong("timestamp")) 73 | items.last 74 | } 75 | 76 | val tableToId = finalDataSet.map { record => 77 | TableMetaInfo(getDatabaseNameFromMeta(record), getTableNameNameFromMeta(record), getschemaNameFromMeta(record)) 78 | }.distinct().collect().zipWithIndex.toMap 79 | 80 | def saveToSink(targetRDD: RDD[JSONObject], operate: String) = { 81 | tableToId.map { case (table, index) => 82 | val tempRDD = targetRDD.filter(record => getDatabaseNameFromMeta(record) == table.db && getTableNameNameFromMeta(record) == table.table).map { record => 83 | record.remove(MLSQLMultiDeltaOptions.META_KEY) 84 | record.toString 85 | } 86 | 87 | def deserializeSchema(json: String): StructType = { 88 | Try(DataType.fromJson(json)).get match { 89 | case t: StructType => t 90 | case _ => throw new RuntimeException(s"Failed parsing StructType: $json") 91 | } 92 | } 93 | 94 | val sourceSchema = deserializeSchema(table.schema) 95 | val newColumnFromJsonStr = new Column(JsonToStructs(sourceSchema, options, F.col("value").expr, None)) 96 | val deleteDF = spark.createDataset[String](tempRDD).toDF("value").select(newColumnFromJsonStr.as("data")) 97 | .select("data.*") 98 | var path = options(MLSQLMultiDeltaOptions.FULL_PATH_KEY) 99 | 100 | val tablePath = path.replace("{db}", table.db).replace("{table}", table.table) 101 | val deltaLog = DeltaLog.forTable(spark, tablePath) 102 | 103 | val readVersion = deltaLog.snapshot.version 104 | val isInitial = readVersion < 0 105 | if (isInitial) { 106 | throw new RuntimeException(s"${tablePath} is not initialed") 107 | } 108 | 109 | val upsertTableInDelta = new UpsertTableInDelta(deleteDF, 110 | Some(SaveMode.Append), 111 | None, 112 | deltaLog, 113 | options = new DeltaOptions(Map[String, String](), ds.sparkSession.sessionState.conf), 114 | partitionColumns = Seq(), 115 | configuration = options ++ Map( 116 | UpsertTableInDelta.OPERATION_TYPE -> operate, 117 | UpsertTableInDelta.ID_COLS -> idCols.mkString(",") 118 | ) 119 | ) 120 | upsertTableInDelta.run(spark) 121 | 122 | } 123 | } 124 | 125 | // filter upsert 126 | val upsertRDD = finalDataSet.filter { record => 127 | val meta = record.getJSONObject(MLSQLMultiDeltaOptions.META_KEY) 128 | meta.getString("type") != "delete" 129 | } 130 | if (upsertRDD.count() > 0) { 131 | saveToSink(upsertRDD, UpsertTableInDelta.OPERATION_TYPE_UPSERT) 132 | } 133 | 134 | // filter delete 135 | 136 | val deleteRDD = finalDataSet.filter { record => 137 | val meta = record.getJSONObject(MLSQLMultiDeltaOptions.META_KEY) 138 | meta.getString("type") == "delete" 139 | } 140 | if (deleteRDD.count() > 0) { 141 | saveToSink(deleteRDD, UpsertTableInDelta.OPERATION_TYPE_DELETE) 142 | } 143 | } finally { 144 | ds.unpersist() 145 | } 146 | 147 | } 148 | } 149 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/commands/CompactTableInDelta.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.commands 2 | 3 | import org.apache.hadoop.fs.{FileStatus, Path} 4 | import org.apache.spark.sql.delta.actions.{Action, AddFile, RemoveFile} 5 | import org.apache.spark.sql.delta.schema.{DeltaInvariantCheckerExec, ImplicitMetadataOperation, Invariants} 6 | import org.apache.spark.sql.delta.{DeltaConcurrentModificationException, _} 7 | import org.apache.spark.sql.execution.SQLExecution 8 | import org.apache.spark.sql.execution.command.RunnableCommand 9 | import org.apache.spark.sql.execution.datasources.FileFormatWriter 10 | import org.apache.spark.sql.{Dataset, Row, SparkSession, functions => F} 11 | import tech.mlsql.common.PathFun 12 | 13 | import scala.collection.mutable.ArrayBuffer 14 | 15 | /** 16 | * 17 | * CompactTableInDelta is used to compact small files into big files. 18 | * This class requires the delta table should satisfied the following requirements: 19 | * 20 | * 1. There is at least one checkpoint have been generated. 21 | * 2. The target delta table should be written 22 | * by SaveMode.Append(Batch) or OutputMode.Append(Stream) 23 | * 3. The target delta table should not operated by upsert/delete action. 24 | * 25 | * @param deltaLog 26 | * @param options 27 | * @param partitionColumns 28 | * @param configuration 29 | */ 30 | case class CompactTableInDelta( 31 | deltaLog: DeltaLog, 32 | options: DeltaOptions, 33 | partitionColumns: Seq[String], 34 | configuration: Map[String, String] 35 | ) 36 | extends RunnableCommand 37 | with ImplicitMetadataOperation 38 | with DeltaCommand with DeltaCommandsFun { 39 | 40 | import CompactTableInDelta._ 41 | 42 | override def run(sparkSession: SparkSession): Seq[Row] = { 43 | 44 | val (items, targetVersion, commitSuccess) = _run(sparkSession) 45 | if (commitSuccess) { 46 | // cleanup deltaLog, so once optimized, we can not traval to the version 47 | // before targetVersion because the data have been deleted 48 | recordDeltaOperation(deltaLog, "delta.log.compact.cleanup") { 49 | doLogCleanup(targetVersion) 50 | } 51 | 52 | // now we can really delete all files. 53 | // Notice this is not a recovery operation. 54 | recordDeltaOperation(deltaLog, "delta.data.compact.cleanup") { 55 | doRemoveFileCleanup(items) 56 | } 57 | } else { 58 | rollback(items) 59 | } 60 | if (!commitSuccess) Seq[Row]() else { 61 | items.map(f => Row.fromSeq(Seq(f.json))) 62 | } 63 | } 64 | 65 | protected def _run(sparkSession: SparkSession): (Seq[Action], Long, Boolean) = { 66 | 67 | var compactRetryTimesForLock = configuration.get(COMPACT_RETRY_TIMES_FOR_LOCK) 68 | .map(_.toInt).getOrElse(0) 69 | 70 | 71 | var success = false 72 | 73 | // The transaction should not take too long, so we should generated 74 | // the new files firstly, and then try to start a transaction and commit, 75 | // once fails, try again until compactRetryTimesForLock times. 76 | // In the transaction, we only commit some information(AddFiles/RemoveFiles) 77 | // So it will not affect the other program to write data. 78 | val (actions, version) = optimize(sparkSession, false) 79 | 80 | while (!success && compactRetryTimesForLock > 0) { 81 | try { 82 | deltaLog.withNewTransaction { txn => 83 | txn.readWholeTable() 84 | val operation = DeltaOperations.Optimize(Seq(), Seq(), 0, false) 85 | txn.commit(actions, operation) 86 | success = true 87 | } 88 | } catch { 89 | case e@(_: java.util.ConcurrentModificationException | 90 | _: DeltaConcurrentModificationException) => 91 | logInfo(s"DeltaConcurrentModificationException throwed. " + 92 | s"tried ${compactRetryTimesForLock}") 93 | // clean data aready been written 94 | Thread.sleep(1000) 95 | compactRetryTimesForLock -= 1 96 | case e: Exception => 97 | throw e 98 | 99 | } 100 | } 101 | 102 | (actions, version, success) 103 | 104 | } 105 | 106 | protected def doLogCleanup(targetVersion: Long) = { 107 | val fs = deltaLog.fs 108 | var numDeleted = 0 109 | listExpiredDeltaLogs(targetVersion).map(_.getPath).foreach { path => 110 | // recursive = false 111 | if (fs.delete(path, false)) { 112 | numDeleted += 1 113 | } 114 | } 115 | logInfo(s"Deleted $numDeleted log files earlier than $targetVersion") 116 | } 117 | 118 | /** 119 | * Returns an iterator of expired delta logs that can be cleaned up. For a delta log to be 120 | * considered as expired, it must: 121 | * - have a checkpoint file after it 122 | * - be earlier than `targetVersion` 123 | */ 124 | private def listExpiredDeltaLogs(targetVersion: Long): Iterator[FileStatus] = { 125 | import org.apache.spark.sql.delta.util.FileNames._ 126 | 127 | val latestCheckpoint = deltaLog.lastCheckpoint 128 | if (latestCheckpoint.isEmpty) return Iterator.empty 129 | 130 | def getVersion(filePath: Path): Long = { 131 | if (isCheckpointFile(filePath)) { 132 | checkpointVersion(filePath) 133 | } else { 134 | deltaVersion(filePath) 135 | } 136 | } 137 | 138 | val files = deltaLog.store.listFrom(deltaFile(deltaLog.logPath, 0)) 139 | .filter(f => isCheckpointFile(f.getPath) || isDeltaFile(f.getPath)) 140 | .filter { f => 141 | getVersion(f.getPath) < targetVersion 142 | } 143 | files 144 | } 145 | 146 | protected def doRemoveFileCleanup(items: Seq[Action]) = { 147 | var numDeleted = 0 148 | items.filter(item => item.isInstanceOf[RemoveFile]) 149 | .map(item => item.asInstanceOf[RemoveFile]) 150 | .foreach { item => 151 | val path = new Path(deltaLog.dataPath, item.path) 152 | val pathCrc = new Path(deltaLog.dataPath, "." + item.path + ".crc") 153 | val fs = deltaLog.fs 154 | try { 155 | fs.delete(path, false) 156 | fs.delete(pathCrc, false) 157 | numDeleted += 1 158 | } catch { 159 | case e: Exception => 160 | } 161 | } 162 | logInfo(s"Deleted $numDeleted files in optimization progress") 163 | } 164 | 165 | protected def rollback(items: Seq[Action]) = { 166 | var numDeleted = 0 167 | items.filter(item => item.isInstanceOf[AddFile]) 168 | .map(item => item.asInstanceOf[AddFile]) 169 | .foreach { item => 170 | val path = new Path(deltaLog.dataPath, item.path) 171 | val pathCrc = new Path(deltaLog.dataPath, "." + item.path + ".crc") 172 | val fs = deltaLog.fs 173 | try { 174 | fs.delete(path, false) 175 | fs.delete(pathCrc, false) 176 | numDeleted += 1 177 | } catch { 178 | case e: Exception => 179 | } 180 | } 181 | logInfo(s"Deleted $numDeleted files in optimization progress") 182 | } 183 | 184 | protected def optimize(sparkSession: SparkSession, 185 | isTry: Boolean): (Seq[Action], Long) = { 186 | import sparkSession.implicits._ 187 | 188 | val metadata = deltaLog.snapshot.metadata 189 | val readVersion = deltaLog.snapshot.version 190 | if (readVersion > -1) { 191 | // For now, we only support the append mode(SaveMode/OutputMode). 192 | // So check if it satisfied this requirement. 193 | logInfo( 194 | s""" 195 | |${deltaLog.dataPath} is appendOnly? 196 | |${DeltaConfigs.IS_APPEND_ONLY.fromMetaData(metadata)} 197 | """.stripMargin) 198 | } 199 | 200 | // Validate partition predicates 201 | val replaceWhere = options.replaceWhere 202 | val partitionFilters = if (replaceWhere.isDefined) { 203 | val predicates = parsePartitionPredicates(sparkSession, replaceWhere.get) 204 | Some(predicates) 205 | } else { 206 | None 207 | } 208 | 209 | if (readVersion < 0) { 210 | // Initialize the log path 211 | DeltaErrors.notADeltaTableException("compact", new DeltaTableIdentifier(Option(deltaLog.dataPath.toString), None)) 212 | } 213 | 214 | val latestCheckpoint = deltaLog.lastCheckpoint 215 | if (latestCheckpoint.isEmpty) throw new RuntimeException( 216 | s""" 217 | |Compact delta log in ${deltaLog.dataPath.toString} should at least: 218 | |- have a checkpoint file after it 219 | |- be earlier than `targetVersion` 220 | """.stripMargin) 221 | 222 | /** 223 | * No matter the table is a partition table or not, 224 | * we can pick one version and compact all files 225 | * before it and then remove all the files compacted and 226 | * add the new compaction files. 227 | */ 228 | var version = configuration.get(COMPACT_VERSION_OPTION).map(_.toLong).getOrElse(-1L) 229 | if (version == -1) version = readVersion 230 | 231 | // check version is valid 232 | deltaLog.history.checkVersionExists(version) 233 | 234 | val newFiles = ArrayBuffer[AddFile]() 235 | val deletedFiles = ArrayBuffer[RemoveFile]() 236 | 237 | // find all files before this version 238 | val snapshot = deltaLog.getSnapshotAt(version, None) 239 | 240 | // here may cost huge memory in driver if people do not optimize their tables frequently, 241 | // we should optimize it in future 242 | val filterFiles = partitionFilters match { 243 | case None => 244 | snapshot.allFiles 245 | case Some(predicates) => 246 | DeltaLog.filterFileList( 247 | metadata.partitionColumns, snapshot.allFiles.toDF(), predicates).as[AddFile] 248 | } 249 | 250 | 251 | val filesShouldBeOptimized = filterFiles 252 | .map(addFile => PrefixAddFile(extractPathPrefix(addFile.path), addFile)) 253 | .groupBy("prefix").agg(F.collect_list("addFile").as("addFiles")).as[PrefixAddFileList] 254 | .collect().toSeq 255 | 256 | val compactNumFilePerDir = configuration.get(COMPACT_NUM_FILE_PER_DIR) 257 | .map(f => f.toInt).getOrElse(1) 258 | 259 | def writeFiles(outputPath: Path, 260 | data: Dataset[_], 261 | writeOptions: Option[DeltaOptions], 262 | isOptimize: Boolean): Seq[AddFile] = { 263 | val spark = data.sparkSession 264 | 265 | val (queryExecution, output) = normalizeData(metadata, data, metadata.partitionColumns) 266 | 267 | 268 | val committer = getCommitter(outputPath) 269 | 270 | val invariants = Invariants.getFromSchema(metadata.schema, spark) 271 | 272 | SQLExecution.withNewExecutionId(spark, queryExecution) { 273 | val outputSpec = FileFormatWriter.OutputSpec( 274 | outputPath.toString, 275 | Map.empty, 276 | output) 277 | 278 | val physicalPlan = DeltaInvariantCheckerExec(queryExecution.executedPlan, invariants) 279 | 280 | FileFormatWriter.write( 281 | sparkSession = spark, 282 | plan = physicalPlan, 283 | fileFormat = snapshot.fileFormat, // TODO doesn't support changing formats. 284 | committer = committer, 285 | outputSpec = outputSpec, 286 | hadoopConf = spark.sessionState.newHadoopConfWithOptions(metadata.configuration), 287 | partitionColumns = Seq(), // Here, we directly write into this outputPath without partition 288 | bucketSpec = None, 289 | statsTrackers = Nil, 290 | options = Map.empty) 291 | } 292 | 293 | committer.addedStatuses 294 | } 295 | 296 | filesShouldBeOptimized.foreach { fileList => 297 | val tempFiles = fileList.addFiles.map { addFile => 298 | new Path(deltaLog.dataPath, addFile.path).toString 299 | } 300 | // if the file num is smaller then we need, skip and do nothing 301 | if (tempFiles.length >= compactNumFilePerDir) { 302 | 303 | // If the code goes here, means the size of `fileList.addFiles` 304 | // is larger then 0. We can get the first element and get the file prefix 305 | // and partitionValues. In the next step when we reconstruct AddFiles, we need them. 306 | val prefix = extractPathPrefix(fileList.addFiles.head.path) 307 | val partitionValues = fileList.addFiles.head.partitionValues 308 | 309 | // Using spark datasource API directly, 310 | // is there any other ways? 311 | val df = sparkSession.read.parquet(tempFiles: _*) 312 | .repartition(compactNumFilePerDir) 313 | 314 | val filePath = if (prefix.isEmpty) deltaLog.dataPath 315 | else new Path(deltaLog.dataPath, prefix) 316 | 317 | // Path in AddFile/RemoveFile should be relative path, and if the prefix 318 | // is empty, the path will starts with "/" and we should use `stripPrefix` to remove it. 319 | newFiles ++= writeFiles(filePath, df, Some(options), false).map { addFile => 320 | addFile.copy(path = PathFun(prefix).add(addFile.path).toPath.stripPrefix("/"), partitionValues = partitionValues) 321 | } 322 | deletedFiles ++= fileList.addFiles.map(_.remove) 323 | } 324 | } 325 | 326 | logInfo(s"Add ${newFiles.size} files in optimization progress") 327 | logInfo(s"Mark remove ${deletedFiles} files in optimization progress") 328 | return (newFiles ++ deletedFiles, version) 329 | } 330 | 331 | override protected val canMergeSchema: Boolean = false 332 | override protected val canOverwriteSchema: Boolean = false 333 | 334 | 335 | } 336 | 337 | object CompactTableInDelta { 338 | val COMPACT_VERSION_OPTION = "compactVersion" 339 | val COMPACT_NUM_FILE_PER_DIR = "compactNumFilePerDir" 340 | val COMPACT_RETRY_TIMES_FOR_LOCK = "compactRetryTimesForLock" 341 | 342 | def extractPathPrefix(path: String): String = { 343 | if (!path.contains("/")) { 344 | "" 345 | } else { 346 | path.split("/").dropRight(1).mkString("/") 347 | } 348 | } 349 | } 350 | 351 | case class PrefixAddFile(prefix: String, addFile: AddFile) 352 | 353 | case class PrefixAddFileList(prefix: String, addFiles: List[AddFile]) 354 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/commands/DeltaCommandsFun.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.commands 2 | 3 | import org.apache.hadoop.fs.Path 4 | import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder 5 | import org.apache.spark.sql.catalyst.expressions.Attribute 6 | import org.apache.spark.sql.delta.actions.Metadata 7 | import org.apache.spark.sql.delta.files.DelayedCommitProtocol 8 | import org.apache.spark.sql.delta.schema.SchemaUtils 9 | import org.apache.spark.sql.execution.QueryExecution 10 | import org.apache.spark.sql.{Dataset, Row} 11 | 12 | /** 13 | * 2019-06-20 WilliamZhu(allwefantasy@gmail.com) 14 | */ 15 | trait DeltaCommandsFun { 16 | protected def normalizeData(metadata: Metadata, 17 | data: Dataset[_], 18 | partitionCols: Seq[String]): (QueryExecution, Seq[Attribute]) = { 19 | val normalizedData = SchemaUtils.normalizeColumnNames(metadata.schema, data) 20 | val cleanedData = SchemaUtils.dropNullTypeColumns(normalizedData) 21 | val queryExecution = if (cleanedData.schema != normalizedData.schema) { 22 | // For batch executions, we need to use the latest DataFrame query execution 23 | cleanedData.queryExecution 24 | } else { 25 | // For streaming workloads, we need to use the QueryExecution created from StreamExecution 26 | data.queryExecution 27 | } 28 | queryExecution -> cleanedData.queryExecution.analyzed.output 29 | } 30 | 31 | 32 | protected def getCommitter(outputPath: Path): DelayedCommitProtocol = 33 | new DelayedCommitProtocol("delta", outputPath.toString, None) 34 | 35 | protected def convertStreamDataFrame(_data: Dataset[_]) = { 36 | if (_data.isStreaming) { 37 | class ConvertStreamDataFrame[T](encoder: ExpressionEncoder[T]) { 38 | 39 | def toBatch(data: Dataset[_]): Dataset[_] = { 40 | val resolvedEncoder = encoder.resolveAndBind( 41 | data.logicalPlan.output, 42 | data.sparkSession.sessionState.analyzer) 43 | val rdd = data.queryExecution.toRdd.map(resolvedEncoder.fromRow)(encoder.clsTag) 44 | val ds = data.sparkSession.createDataset(rdd)(encoder) 45 | ds 46 | } 47 | } 48 | new ConvertStreamDataFrame[Row](_data.asInstanceOf[Dataset[Row]].exprEnc).toBatch(_data) 49 | } else _data 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/commands/UpsertTableInDelta.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.commands 2 | 3 | import java.util.UUID 4 | 5 | import org.apache.hadoop.fs.Path 6 | import org.apache.spark.sql.delta._ 7 | import org.apache.spark.sql.delta.actions._ 8 | import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, SchemaUtils} 9 | import org.apache.spark.sql.delta.sources.{BFItem, FullOuterJoinRow} 10 | import org.apache.spark.sql.execution.command.RunnableCommand 11 | import org.apache.spark.sql.execution.streaming.StreamExecution 12 | import org.apache.spark.sql.streaming.OutputMode 13 | import org.apache.spark.sql.types._ 14 | import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession, functions => F} 15 | import tech.mlsql.common.{BloomFilter, DeltaJob} 16 | 17 | case class UpsertTableInDelta(_data: Dataset[_], 18 | saveMode: Option[SaveMode], 19 | outputMode: Option[OutputMode], 20 | deltaLog: DeltaLog, 21 | options: DeltaOptions, 22 | partitionColumns: Seq[String], 23 | configuration: Map[String, String] 24 | ) extends RunnableCommand 25 | with ImplicitMetadataOperation 26 | with DeltaCommand with DeltaCommandsFun { 27 | 28 | override def run(sparkSession: SparkSession): Seq[Row] = { 29 | var res: Seq[Row] = Seq() 30 | DeltaJob.runWithTry(() => { 31 | res = _run(sparkSession) 32 | }) 33 | res 34 | } 35 | 36 | def _run(sparkSession: SparkSession): Seq[Row] = { 37 | val runId = UUID.randomUUID().toString 38 | assert(configuration.contains(UpsertTableInDelta.ID_COLS), "idCols is required ") 39 | 40 | if (outputMode.isDefined) { 41 | assert(outputMode.get == OutputMode.Append(), "append is required ") 42 | } 43 | 44 | if (saveMode.isDefined) { 45 | assert(saveMode.get == SaveMode.Append, "append is required ") 46 | } 47 | 48 | val upsertConf = new UpsertTableInDeltaConf(configuration, deltaLog, sparkSession) 49 | val upsertCommit = new UpsertCommit(deltaLog, runId, upsertConf) 50 | 51 | var actions = Seq[Action]() 52 | 53 | saveMode match { 54 | case Some(mode) => 55 | deltaLog.withNewTransaction { txn => 56 | txn.readWholeTable() 57 | if (!upsertConf.isPartialMerge) { 58 | updateMetadata(txn, _data, partitionColumns, configuration, false) 59 | } 60 | actions = upsert(txn, sparkSession, runId) 61 | val operation = DeltaOperations.Write(SaveMode.Overwrite, 62 | Option(partitionColumns), 63 | options.replaceWhere) 64 | upsertCommit.commit(txn, actions, operation) 65 | 66 | } 67 | case None => outputMode match { 68 | case Some(mode) => 69 | val queryId = sparkSession.sparkContext.getLocalProperty(StreamExecution.QUERY_ID_KEY) 70 | assert(queryId != null) 71 | 72 | if (SchemaUtils.typeExistsRecursively(_data.schema)(_.isInstanceOf[NullType])) { 73 | throw DeltaErrors.streamWriteNullTypeException 74 | } 75 | 76 | val txn = deltaLog.startTransaction() 77 | txn.readWholeTable() 78 | // Streaming sinks can't blindly overwrite schema. 79 | // See Schema Management design doc for details 80 | if (!upsertConf.isPartialMerge) { 81 | updateMetadata( 82 | txn, 83 | _data, 84 | partitionColumns, 85 | configuration = Map.empty, 86 | false) 87 | } 88 | val currentVersion = txn.txnVersion(queryId) 89 | val batchId = configuration(UpsertTableInDelta.BATCH_ID).toLong 90 | if (currentVersion >= batchId) { 91 | logInfo(s"Skipping already complete epoch $batchId, in query $queryId") 92 | } else { 93 | actions = upsert(txn, sparkSession, runId) 94 | val setTxn = SetTransaction(queryId, 95 | batchId, Some(deltaLog.clock.getTimeMillis())) :: Nil 96 | val info = DeltaOperations.StreamingUpdate(outputMode.get, queryId, batchId) 97 | upsertCommit.commit(txn, setTxn ++ actions, info) 98 | } 99 | } 100 | } 101 | 102 | if (actions.size == 0) Seq[Row]() else { 103 | actions.map(f => Row.fromSeq(Seq(f.json))) 104 | } 105 | } 106 | 107 | def upsert(txn: OptimisticTransaction, sparkSession: SparkSession, runId: String): Seq[Action] = { 108 | 109 | val upsertConf = new UpsertTableInDeltaConf(configuration, deltaLog, sparkSession) 110 | val upsertBF = new UpsertBF(upsertConf, runId) 111 | val isDelete = upsertConf.isDeleteOp 112 | 113 | // if _data is stream dataframe, we should convert it to normal 114 | // dataframe and so we can join it later 115 | var data = convertStreamDataFrame(_data) 116 | 117 | 118 | import sparkSession.implicits._ 119 | val snapshot = deltaLog.snapshot 120 | val metadata = deltaLog.snapshot.metadata 121 | 122 | /** 123 | * Firstly, we should get all partition columns from `idCols` condition. 124 | * Then we can use them to optimize file scan. 125 | */ 126 | val idCols = configuration.getOrElse(UpsertTableInDelta.ID_COLS, "") 127 | val idColsList = idCols.split(",").filterNot(_.isEmpty).toSeq 128 | val partitionColumnsInIdCols = partitionColumns.intersect(idColsList) 129 | 130 | // we should make sure the data have no duplicate otherwise throw exception 131 | if (upsertConf.isDropDuplicate()) { 132 | data = data.dropDuplicates(idColsList.toArray) 133 | } else { 134 | val tempDF = data.groupBy(idColsList.map(col => F.col(col)): _*).agg(F.count("*").as("count")) 135 | if (tempDF.filter("count > 1").count() != 0) { 136 | throw new RuntimeException("Cannot perform MERGE as multiple source rows " + 137 | "matched and attempted to update the same target row in the Delta table.") 138 | } 139 | } 140 | 141 | val sourceSchema = if (upsertConf.isInitial) data.schema else snapshot.schema 142 | 143 | 144 | if (upsertConf.isInitial && upsertConf.isPartialMerge) { 145 | throw new RuntimeException(s"Please init the table or disable ${UpsertTableInDelta.PARTIAL_MERGE}") 146 | } 147 | 148 | if (upsertConf.isInitial) { 149 | 150 | deltaLog.fs.mkdirs(deltaLog.logPath) 151 | 152 | val newFiles = if (!isDelete) { 153 | txn.writeFiles(data.repartition(1), Some(options)) 154 | } else Seq() 155 | if(upsertConf.isBloomFilterEnable){ 156 | upsertBF.generateBFForParquetFile(sourceSchema, newFiles, Seq()) 157 | } 158 | return newFiles 159 | } 160 | 161 | 162 | val partitionFilters = if (partitionColumnsInIdCols.size > 0) { 163 | val schema = data.schema 164 | 165 | def isNumber(column: String) = { 166 | schema.filter(f => f.name == column).head.dataType match { 167 | case _: LongType => true 168 | case _: IntegerType => true 169 | case _: ShortType => true 170 | case _: DoubleType => true 171 | case _ => false 172 | } 173 | } 174 | 175 | val minMaxColumns = partitionColumnsInIdCols.flatMap { column => 176 | Seq(F.lit(column), F.min(column).as(s"${column}_min"), F.max(F.max(s"${column}_max"))) 177 | }.toArray 178 | val minxMaxKeyValues = data.select(minMaxColumns: _*).collect() 179 | 180 | // build our where statement 181 | val whereStatement = minxMaxKeyValues.map { row => 182 | val column = row.getString(0) 183 | val minValue = row.get(1).toString 184 | val maxValue = row.get(2).toString 185 | 186 | if (isNumber(column)) { 187 | s"${column} >= ${minValue} and ${maxValue} >= ${column}" 188 | } else { 189 | s"""${column} >= "${minValue}" and "${maxValue}" >= ${column}""" 190 | } 191 | } 192 | logInfo(s"whereStatement: ${whereStatement.mkString(" and ")}") 193 | val predicates = parsePartitionPredicates(sparkSession, whereStatement.mkString(" and ")) 194 | Some(predicates) 195 | 196 | } else None 197 | 198 | 199 | val filterFilesDataSet = partitionFilters match { 200 | case None => 201 | snapshot.allFiles 202 | case Some(predicates) => 203 | DeltaLog.filterFileList( 204 | metadata.partitionColumns, snapshot.allFiles.toDF(), predicates).as[AddFile] 205 | } 206 | 207 | 208 | // Again, we collect all files to driver, 209 | // this may impact performance and even make the driver OOM when 210 | // the number of files are very huge. 211 | // So please make sure you have configured the partition columns or make compaction frequently 212 | 213 | val filterFiles = filterFilesDataSet.collect 214 | 215 | // filter files are affected by BF 216 | val bfPath = new Path(deltaLog.dataPath, "_bf_index_" + deltaLog.snapshot.version) 217 | val filesAreAffectedWithDeltaFormat = if (upsertConf.isBloomFilterEnable && deltaLog.fs.exists(bfPath)) { 218 | val schemaNames = data.schema.map(f => f.name) 219 | val dataBr = sparkSession.sparkContext.broadcast(data.select(idColsList.map(F.col(_)): _*).collect()) 220 | val affectedFilePaths = sparkSession.read.parquet(bfPath.toUri.getPath).as[BFItem].flatMap { bfItem => 221 | val bf = new BloomFilter(bfItem.bf) 222 | var dataInBf = false 223 | dataBr.value.foreach { row => 224 | if (!dataInBf) { 225 | val key = row.asInstanceOf[Row].toSeq.mkString("_") 226 | dataInBf = bf.mightContain(key) 227 | } 228 | 229 | } 230 | if (dataInBf) List(bfItem.fileName) else List() 231 | }.as[String].collect() 232 | filterFiles.filter(f => affectedFilePaths.contains(f.path)) 233 | } else { 234 | // filter files are affected by anti join 235 | val dataInTableWeShouldProcess = deltaLog.createDataFrame(snapshot, filterFiles, false) 236 | val dataInTableWeShouldProcessWithFileName = dataInTableWeShouldProcess. 237 | withColumn(UpsertTableInDelta.FILE_NAME, F.input_file_name()) 238 | // get all files that are affected by the new data(update) 239 | val filesAreAffected = dataInTableWeShouldProcessWithFileName.join(data, 240 | usingColumns = idColsList, 241 | joinType = "inner").select(UpsertTableInDelta.FILE_NAME). 242 | distinct().collect().map(f => f.getString(0)) 243 | 244 | val tmpFilePathSet = filesAreAffected.map(f => f.split("/").last).toSet 245 | 246 | filterFiles.filter { file => 247 | tmpFilePathSet.contains(file.path.split("/").last) 248 | } 249 | } 250 | 251 | val deletedFiles = filesAreAffectedWithDeltaFormat.map(_.remove) 252 | 253 | // we should get not changed records in affected files and write them back again 254 | val affectedRecords = deltaLog.createDataFrame(snapshot, filesAreAffectedWithDeltaFormat, false) 255 | 256 | if (upsertConf.isPartialMerge) { 257 | // new data format: {IDs... value:...} value should be JSon/StructType,so we can merge it into table 258 | // the order of fields are important 259 | //这里会导致schema被修改 260 | val newDF = affectedRecords.join(data, 261 | usingColumns = idColsList, joinType = "fullOuter").select(sourceSchema.fields.map(field=>F.col(field.name)):_*) 262 | val sourceLen = sourceSchema.fields.length 263 | val sourceSchemaSeq = sourceSchema.map(f => f.name) 264 | val targetSchemaSeq = data.schema.map(f => f.name) 265 | val targetLen = data.schema.length 266 | 267 | val targetValueName = targetSchemaSeq.filterNot(name => idColsList.contains(name)).head 268 | val targetValueIndex = targetSchemaSeq.indexOf(targetValueName) 269 | val targetValueType = data.schema.filter(f => f.name == targetValueName).head.dataType 270 | val timeZone = data.sparkSession.sessionState.conf.sessionLocalTimeZone 271 | 272 | val newRDD = newDF.rdd.map { row => 273 | //split row to two row 274 | val leftRow = Row.fromSeq((0 until sourceLen).map(row.get(_))) 275 | val rightRow = Row.fromSeq((sourceLen until (sourceLen + targetLen - idColsList.size)).map(row.get(_))) 276 | 277 | FullOuterJoinRow(leftRow, rightRow, 278 | !UpsertTableInDelta.isKeyAllNull(leftRow, idColsList, sourceSchemaSeq), 279 | !UpsertTableInDelta.isKeyAllNull(rightRow, idColsList, targetSchemaSeq)) 280 | }.map { row: FullOuterJoinRow => 281 | new UpsertMergeJsonToRow(row, sourceSchema, 0, timeZone).output 282 | } 283 | 284 | val finalNumIfKeepFileNum = if (deletedFiles.size == 0) 1 else deletedFiles.size 285 | val newTempData = sparkSession.createDataFrame(newRDD, sourceSchema).repartitionByRange(finalNumIfKeepFileNum, upsertConf.toIdCols: _*) 286 | 287 | val newFiles = if (!isDelete) { 288 | txn.writeFiles(newTempData, Some(options)) 289 | } else Seq() 290 | 291 | if (upsertConf.isBloomFilterEnable) { 292 | upsertBF.generateBFForParquetFile(sourceSchema, newFiles, deletedFiles) 293 | } 294 | logInfo(s"Update info: newFiles:${newFiles.size} deletedFiles:${deletedFiles.size}") 295 | newFiles ++ deletedFiles 296 | 297 | } else { 298 | //这里会导致schema被修改 299 | var notChangedRecords = affectedRecords.join(data, 300 | usingColumns = idColsList, joinType = "leftanti"). 301 | drop(F.col(UpsertTableInDelta.FILE_NAME)).select(sourceSchema.fields.map(field=>F.col(field.name)):_*) 302 | 303 | 304 | val newFiles = if (isDelete) { 305 | if (configuration.contains(UpsertTableInDelta.FILE_NUM)) { 306 | notChangedRecords = notChangedRecords.repartitionByRange(configuration(UpsertTableInDelta.FILE_NUM).toInt, upsertConf.toIdCols: _*) 307 | } else { 308 | if (filesAreAffectedWithDeltaFormat.length > 0) { 309 | notChangedRecords = notChangedRecords.repartitionByRange(filesAreAffectedWithDeltaFormat.length, upsertConf.toIdCols: _*) 310 | } 311 | } 312 | if (filesAreAffectedWithDeltaFormat.length > 0) { 313 | txn.writeFiles(notChangedRecords, Some(options)) 314 | } else Seq[AddFile]() 315 | } else { 316 | var newTempData = data.toDF().union(notChangedRecords) 317 | val finalNumIfKeepFileNum = if (deletedFiles.size == 0) 1 else deletedFiles.size 318 | newTempData = if (upsertConf.keepFileNum()) newTempData.repartitionByRange(finalNumIfKeepFileNum, upsertConf.toIdCols: _*) else newTempData 319 | txn.writeFiles(newTempData, Some(options)) 320 | } 321 | 322 | if (upsertConf.isBloomFilterEnable) { 323 | upsertBF.generateBFForParquetFile(sourceSchema, newFiles, deletedFiles) 324 | } 325 | logInfo(s"Update info: newFiles:${newFiles.size} deletedFiles:${deletedFiles.size} ") 326 | newFiles ++ deletedFiles 327 | } 328 | 329 | 330 | } 331 | 332 | override protected val canMergeSchema: Boolean = false 333 | override protected val canOverwriteSchema: Boolean = false 334 | 335 | } 336 | 337 | object UpsertTableInDelta { 338 | val ID_COLS = "idCols" 339 | val BATCH_ID = "batchId" 340 | val FILE_NAME = "__fileName__" 341 | val OPERATION_TYPE = "operation" 342 | val OPERATION_TYPE_UPSERT = "upsert" 343 | val OPERATION_TYPE_DELETE = "delete" 344 | val DROP_DUPLICATE = "dropDuplicate" 345 | val KEEP_FILE_NUM = "keepFileNum" 346 | 347 | val PARTIAL_MERGE = "partialMerge" 348 | 349 | val FILE_NUM = "fileNum" 350 | val BLOOM_FILTER_ENABLE = "bloomFilterEnable" 351 | 352 | val SYNC_TYPE = "syncType" 353 | val SYNC_TYPE_BINLOG = "binlog" 354 | val SYNC_TYPE_NORMAL = "normal" 355 | val NEW_DATA_PARALLEL_NUM = "newDataParallelNum" 356 | 357 | def getKey(row: Row, idColsList: Seq[String], schemaNames: Seq[String]) = { 358 | getColStrs(row, idColsList, schemaNames) 359 | } 360 | 361 | def isKeyAllNull(row: Row, idColsList: Seq[String], schemaNames: Seq[String]) = { 362 | idColsList.sorted.map(col => schemaNames.indexOf(col)).count { col => 363 | row.get(col) == null 364 | } == idColsList.length 365 | } 366 | 367 | def getColStrs(row: Row, cols: Seq[String], schemaNames: Seq[String]) = { 368 | val item = cols.sorted.map(col => schemaNames.indexOf(col)).map { col => 369 | row.get(col).toString 370 | }.mkString("_") 371 | item 372 | } 373 | } 374 | 375 | 376 | 377 | 378 | 379 | 380 | 381 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/sources/MLSQLDeltaDataSource.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.sources 2 | 3 | import org.apache.hadoop.fs.Path 4 | import org.apache.spark.sql.delta.commands.{UpsertTableInDelta, WriteIntoDelta} 5 | import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOptions} 6 | import org.apache.spark.sql.execution.streaming.Sink 7 | import org.apache.spark.sql.sources.BaseRelation 8 | import org.apache.spark.sql.streaming.OutputMode 9 | import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} 10 | 11 | /** 12 | * 2019-06-10 WilliamZhu(allwefantasy@gmail.com) 13 | */ 14 | class MLSQLDeltaDataSource extends DeltaDataSource { 15 | override def createSink( 16 | sqlContext: SQLContext, 17 | parameters: Map[String, String], 18 | partitionColumns: Seq[String], 19 | outputMode: OutputMode): Sink = { 20 | val path = parameters.getOrElse("path", { 21 | throw DeltaErrors.pathNotSpecifiedException 22 | }) 23 | if (outputMode != OutputMode.Append && outputMode != OutputMode.Complete) { 24 | throw DeltaErrors.outputModeNotSupportedException(getClass.getName, outputMode) 25 | } 26 | val deltaOptions = new DeltaOptions(parameters, sqlContext.sparkSession.sessionState.conf) 27 | new MLSQLDeltaSink(sqlContext, new Path(path), partitionColumns, outputMode, deltaOptions, parameters) 28 | } 29 | 30 | override def createRelation( 31 | sqlContext: SQLContext, 32 | mode: SaveMode, 33 | parameters: Map[String, String], 34 | data: DataFrame): BaseRelation = { 35 | val path = parameters.getOrElse("path", { 36 | throw DeltaErrors.pathNotSpecifiedException 37 | }) 38 | val partitionColumns = parameters.get(DeltaSourceUtils.PARTITIONING_COLUMNS_KEY) 39 | .map(DeltaDataSource.decodePartitioningColumns) 40 | .getOrElse(Nil) 41 | 42 | val deltaLog = DeltaLog.forTable(sqlContext.sparkSession, path) 43 | 44 | if (parameters.contains(UpsertTableInDelta.ID_COLS)) { 45 | UpsertTableInDelta(data, Option(mode), None, deltaLog, 46 | new DeltaOptions(Map[String, String](), sqlContext.sparkSession.sessionState.conf), 47 | Seq(), 48 | parameters).run(sqlContext.sparkSession) 49 | 50 | } else { 51 | WriteIntoDelta( 52 | deltaLog = deltaLog, 53 | mode = mode, 54 | new DeltaOptions(parameters, sqlContext.sparkSession.sessionState.conf), 55 | partitionColumns = partitionColumns, 56 | configuration = parameters, 57 | data = data).run(sqlContext.sparkSession) 58 | } 59 | 60 | deltaLog.createRelation() 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/sources/MLSQLDeltaSink.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.sources 2 | 3 | import org.apache.hadoop.fs.Path 4 | import org.apache.spark.sql.delta._ 5 | import org.apache.spark.sql.delta.commands.{BinlogSyncToDelta, DeltaCommandsFun, UpsertTableInDelta} 6 | import org.apache.spark.sql.streaming.OutputMode 7 | import org.apache.spark.sql.{DataFrame, SQLContext} 8 | import tech.mlsql.common.DeltaJob 9 | 10 | 11 | class MLSQLDeltaSink( 12 | sqlContext: SQLContext, 13 | path: Path, 14 | partitionColumns: Seq[String], 15 | outputMode: OutputMode, 16 | options: DeltaOptions, 17 | parameters: Map[String, String] 18 | ) extends DeltaSink( 19 | sqlContext: SQLContext, 20 | path: Path, 21 | partitionColumns: Seq[String], 22 | outputMode: OutputMode, 23 | options: DeltaOptions){ 24 | 25 | private val deltaLog = DeltaLog.forTable(sqlContext.sparkSession, path) 26 | 27 | private val sqlConf = sqlContext.sparkSession.sessionState.conf 28 | 29 | override def addBatch(batchId: Long, data: DataFrame): Unit = { 30 | if (parameters.getOrElse(UpsertTableInDelta.SYNC_TYPE, UpsertTableInDelta.SYNC_TYPE_NORMAL) == UpsertTableInDelta.SYNC_TYPE_BINLOG) { 31 | BinlogSyncToDelta.run(data, parameters) 32 | return 33 | } 34 | 35 | if (parameters.contains(UpsertTableInDelta.ID_COLS)) { 36 | UpsertTableInDelta(data, None, Option(outputMode), deltaLog, 37 | new DeltaOptions(Map[String, String](), sqlContext.sparkSession.sessionState.conf), 38 | Seq(), 39 | parameters ++ Map(UpsertTableInDelta.BATCH_ID -> batchId.toString)).run(sqlContext.sparkSession) 40 | } else { 41 | DeltaJob.runWithTry(() => { 42 | super.addBatch(batchId, data) 43 | }) 44 | } 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/sources/protocols.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta.sources 2 | 3 | import org.apache.spark.sql.Row 4 | 5 | /** 6 | * 1/11/2019 WilliamZhu(allwefantasy@gmail.com) 7 | */ 8 | case class BFItem(fileName: String, bf: String, size: Int, sizePretty: String) 9 | 10 | case class FullOuterJoinRow(left: Row, right: Row, leftPresent: Boolean, rightPresent: Boolean) 11 | -------------------------------------------------------------------------------- /src/main/java/org/apache/spark/sql/delta/upsert.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.delta 2 | 3 | import java.util.{Date, TimeZone} 4 | 5 | import org.apache.commons.lang.StringUtils 6 | import org.apache.hadoop.fs.Path 7 | import org.apache.spark.Partitioner 8 | import org.apache.spark.sql.{DataFrame, Dataset, Row, SaveMode, SparkSession,functions=>F} 9 | import org.apache.spark.sql.catalyst.util.DateTimeUtils 10 | import org.apache.spark.sql.delta.actions.{Action, AddFile, RemoveFile} 11 | import org.apache.spark.sql.delta.commands.UpsertTableInDelta 12 | import org.apache.spark.sql.delta.files.TahoeBatchFileIndex 13 | import org.apache.spark.sql.delta.sources.{BFItem, FullOuterJoinRow} 14 | import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} 15 | import org.apache.spark.sql.types.{DateType, IntegerType, LongType, StructField, StructType, TimestampType} 16 | import tech.mlsql.common.BloomFilter 17 | 18 | import scala.collection.mutable.ArrayBuffer 19 | 20 | class UpsertTableInDeltaConf(configuration: Map[String, String], @transient val deltaLog: DeltaLog, @transient val sparkSession: SparkSession) { 21 | def isDropDuplicate() = { 22 | configuration.get(UpsertTableInDelta.DROP_DUPLICATE).map(_.toBoolean).getOrElse(false) 23 | } 24 | 25 | def keepFileNum() = { 26 | configuration.get(UpsertTableInDelta.KEEP_FILE_NUM).map(_.toBoolean).getOrElse(true) 27 | } 28 | 29 | def isBloomFilterEnable = { 30 | configuration.getOrElse(UpsertTableInDelta.BLOOM_FILTER_ENABLE, "false").toBoolean 31 | } 32 | 33 | def isDeleteOp = { 34 | configuration 35 | .getOrElse(UpsertTableInDelta.OPERATION_TYPE, 36 | UpsertTableInDelta.OPERATION_TYPE_UPSERT) == UpsertTableInDelta.OPERATION_TYPE_DELETE 37 | } 38 | 39 | def isInitial = { 40 | val readVersion = deltaLog.snapshot.version 41 | val isInitial = readVersion < 0 42 | isInitial 43 | } 44 | 45 | def isPartialMerge = { 46 | configuration 47 | .getOrElse(UpsertTableInDelta.PARTIAL_MERGE, 48 | "false").toBoolean 49 | } 50 | 51 | def bfErrorRate = { 52 | configuration.getOrElse("bfErrorRate", "0.0001").toDouble 53 | } 54 | 55 | def idColsList = { 56 | val idCols = configuration.getOrElse(UpsertTableInDelta.ID_COLS, "") 57 | idCols.split(",").filterNot(_.isEmpty).toSeq 58 | } 59 | 60 | def toIdCols = { 61 | idColsList.map { col => F.col(col) } 62 | } 63 | 64 | } 65 | 66 | class UpsertCommit(deltaLog: DeltaLog, runId: String, upserConf: UpsertTableInDeltaConf) { 67 | 68 | def commit(txn: OptimisticTransaction, actions: Seq[Action], op: DeltaOperations.Operation): Long = { 69 | val currentV = deltaLog.snapshot.version.toInt 70 | 71 | def cleanTmpBFIndex(v: Long) = { 72 | try { 73 | val newBFPathFs = new Path(deltaLog.dataPath, "_bf_index_" + v + "_" + runId) 74 | deltaLog.fs.delete(newBFPathFs, true) 75 | } catch { 76 | case e1: Exception => 77 | } 78 | } 79 | 80 | def cleanOldBFIndex(v: Long) = { 81 | try { 82 | val newBFPathFs = new Path(deltaLog.dataPath, "_bf_index_" + v) 83 | deltaLog.fs.delete(newBFPathFs, true) 84 | } catch { 85 | case e1: Exception => 86 | } 87 | } 88 | 89 | def mvBFIndex(v: Long) = { 90 | try { 91 | val newBFPathFs = new Path(deltaLog.dataPath, "_bf_index_" + v + "_" + runId) 92 | val targetPath = new Path(deltaLog.dataPath, "_bf_index_" + v) 93 | deltaLog.fs.rename(newBFPathFs, targetPath) 94 | } catch { 95 | case e1: Exception => 96 | } 97 | } 98 | 99 | val newV = try { 100 | txn.commit(actions, op) 101 | } catch { 102 | case e: Exception => 103 | if (upserConf.isBloomFilterEnable) { 104 | cleanTmpBFIndex(currentV + 1) 105 | } 106 | throw e 107 | } 108 | if (newV > -1) { 109 | if (upserConf.isBloomFilterEnable) { 110 | mvBFIndex(newV) 111 | cleanOldBFIndex(newV - 1) 112 | } 113 | } 114 | newV 115 | } 116 | } 117 | 118 | class UpsertBF(upsertConf: UpsertTableInDeltaConf, runId: String) { 119 | 120 | import upsertConf.sparkSession.implicits._ 121 | 122 | def generateBFForParquetFile(sourceSchema: StructType, addFiles: Seq[AddFile], deletedFiles: Seq[RemoveFile]) = { 123 | val deltaLog = upsertConf.deltaLog 124 | val snapshot = deltaLog.snapshot 125 | val sparkSession = upsertConf.sparkSession 126 | val isInitial = upsertConf.isInitial 127 | 128 | 129 | val newBFPathFs = new Path(deltaLog.dataPath, "_bf_index_" + (deltaLog.snapshot.version.toInt + 1) + "_" + runId) 130 | val newBFPath = newBFPathFs.toUri.getPath 131 | 132 | val bfPathFs = new Path(deltaLog.dataPath, "_bf_index_" + deltaLog.snapshot.version) 133 | val bfPath = bfPathFs.toUri.getPath 134 | 135 | if (deltaLog.fs.exists(bfPathFs)) { 136 | deltaLog.fs.mkdirs(newBFPathFs) 137 | val deletePaths = deletedFiles.map(f => f.path).toSet 138 | sparkSession.read.parquet(bfPath).repartition(1).as[BFItem]. 139 | filter { f => 140 | !deletePaths.contains(f.fileName) 141 | }.write.mode(SaveMode.Append).parquet(newBFPath) 142 | } 143 | 144 | // There are 2 possible reasons that there is no _bf_index_[version] directory: 145 | // 1. No upsert operation happens before 146 | // 2. It fails to create _bf_index_[version] in previous upsert operation/version. For example, application crash happens 147 | // between commit and rename. 148 | // 149 | // When there is no _bf_index_[version], the we will back to join to find the affected files, and then 150 | // create new BF file for current version and the version uncommitted yet. 151 | // 152 | var realAddFiles = addFiles 153 | if (!deltaLog.fs.exists(bfPathFs) && deltaLog.snapshot.version > -1) { 154 | realAddFiles ++= deltaLog.snapshot.allFiles.collect() 155 | realAddFiles = realAddFiles.filterNot(addfile => deletedFiles.map(_.path).contains(addfile.path)) 156 | } 157 | 158 | val deltaPathPrefix = deltaLog.snapshot.deltaLog.dataPath.toUri.getPath 159 | 160 | def createDataFrame( 161 | addFiles: Seq[AddFile], 162 | isStreaming: Boolean = false, 163 | actionTypeOpt: Option[String] = None): DataFrame = { 164 | val actionType = actionTypeOpt.getOrElse(if (isStreaming) "streaming" else "batch") 165 | val fileIndex = new TahoeBatchFileIndex(sparkSession, actionType, addFiles, deltaLog, deltaLog.dataPath, snapshot) 166 | val relation = HadoopFsRelation( 167 | fileIndex, 168 | partitionSchema = StructType(Array[StructField]()), 169 | dataSchema = sourceSchema, 170 | bucketSpec = None, 171 | deltaLog.snapshot.fileFormat, 172 | deltaLog.snapshot.metadata.format.options)(sparkSession) 173 | 174 | Dataset.ofRows(sparkSession, LogicalRelation(relation, isStreaming = isStreaming)) 175 | } 176 | 177 | val df = if (!isInitial) { 178 | deltaLog.createDataFrame(snapshot, realAddFiles, false).withColumn(UpsertTableInDelta.FILE_NAME, F.input_file_name()) 179 | } else { 180 | createDataFrame(realAddFiles, false).withColumn(UpsertTableInDelta.FILE_NAME, F.input_file_name()) 181 | } 182 | val FILE_NAME = UpsertTableInDelta.FILE_NAME 183 | // println( 184 | // s""" 185 | // |### bf stat ### 186 | // |fileNumber: ${realAddFiles.size} 187 | // |realAddFiles: ${realAddFiles.map(f => f.path).toSeq} 188 | // |deletedFiles: ${deletedFiles.map(f => f.path).toSeq} 189 | // |mapPartitions: ${df.repartition(realAddFiles.size, F.col(FILE_NAME)).rdd.partitions.size} 190 | // |""".stripMargin) 191 | 192 | val schemaNames = df.schema.map(f => f.name) 193 | val errorRate = upsertConf.bfErrorRate 194 | val idColsList = upsertConf.idColsList 195 | val dfSchema = df.schema.map(f => f.name) 196 | val fileWithIndex = realAddFiles.zipWithIndex.map { f => (f._1.path, f._2) }.toMap 197 | val fileNum = fileWithIndex.size 198 | val rdd = df.rdd.map { row => 199 | (UpsertTableInDelta.getColStrs(row, Seq(FILE_NAME), dfSchema), row) 200 | }.partitionBy(new Partitioner() { 201 | override def numPartitions: Int = fileNum 202 | 203 | override def getPartition(key: Any): Int = fileWithIndex(StringUtils.splitByWholeSeparator(key.toString, deltaPathPrefix).last.stripPrefix("/")) 204 | }).map(f => f._2).mapPartitionsWithIndex { (index, iter) => 205 | val buffer = new ArrayBuffer[String]() 206 | var fileName: String = null 207 | var numEntries = 0 208 | while (iter.hasNext) { 209 | val row = iter.next() 210 | if (fileName == null) { 211 | fileName = row.getAs[String](FILE_NAME) 212 | } 213 | numEntries += 1 214 | buffer += UpsertTableInDelta.getKey(row, idColsList, schemaNames) 215 | } 216 | if (numEntries > 0) { 217 | val bf = new BloomFilter(numEntries, errorRate) 218 | buffer.foreach { rowId => 219 | bf.add(rowId) 220 | } 221 | // println( 222 | // s""" 223 | // |### gen bf ### 224 | // |index: ${index} 225 | // |fileName: ${StringUtils.splitByWholeSeparator(fileName, deltaPathPrefix).last.stripPrefix("/")} 226 | // |bf: ${bf.serializeToString()} 227 | // |numEntries: ${numEntries} 228 | // |errorRate: ${errorRate} 229 | // |rowIds: ${buffer.toList} 230 | // |""".stripMargin) 231 | List[BFItem](BFItem(StringUtils.splitByWholeSeparator(fileName, deltaPathPrefix).last.stripPrefix("/"), bf.serializeToString(), bf.size(), (bf.size() / 8d / 1024 / 1024) + "m")).iterator 232 | } else { 233 | // println( 234 | // s""" 235 | // |### gen bf ### 236 | // |index: ${index} 237 | // |fileName: 238 | // |bf: 239 | // |numEntries: ${numEntries} 240 | // |errorRate: ${errorRate} 241 | // |rowIds: ${buffer.toList} 242 | // |""".stripMargin) 243 | List[BFItem]().iterator 244 | } 245 | 246 | }.repartition(1).toDF().as[BFItem].write.mode(SaveMode.Append).parquet(newBFPath) 247 | } 248 | } 249 | 250 | class UpsertMergeJsonToRow(row: FullOuterJoinRow, schema: StructType, targetValueIndex: Int, defaultTimeZone: String) { 251 | val timeZone: TimeZone = DateTimeUtils.getTimeZone(defaultTimeZone) 252 | 253 | 254 | private def parseJson(jsonStr: String, callback: (String, Any) => Unit) = { 255 | import org.json4s._ 256 | import org.json4s.jackson.JsonMethods._ 257 | val obj = parse(jsonStr) 258 | obj.asInstanceOf[JObject].obj.foreach { f => 259 | val dataType = schema.filter(field => field.name == f._1).head.dataType 260 | val value = f._2 match { 261 | // case JArray(arr) => 262 | // case JObject(obj)=> 263 | case JBool(v) => v 264 | case JNull => null 265 | // case JNothing => 266 | case JDouble(v) => v 267 | case JInt(v) => 268 | dataType match { 269 | case IntegerType => 270 | v 271 | } 272 | case JLong(v) => 273 | dataType match { 274 | case TimestampType => 275 | new Date(v) 276 | case DateType => 277 | new Date(v) 278 | case LongType => 279 | v 280 | } 281 | case JString(v) => v 282 | 283 | } 284 | callback(f._1, value) 285 | } 286 | } 287 | 288 | private def merge(left: Row, right: Row) = { 289 | val tempRow = left.toSeq.toArray 290 | parseJson(right.getAs[String](targetValueIndex), (k, v) => { 291 | left.toSeq.zipWithIndex.map { wow => 292 | val index = wow._2 293 | val value = wow._1 294 | tempRow(index) = if (schema.fieldIndex(k) == index) v else value 295 | } 296 | }) 297 | Row.fromSeq(tempRow) 298 | } 299 | 300 | def output = { 301 | row match { 302 | case FullOuterJoinRow(left, right, true, true) => 303 | // upsert 304 | merge(left, right) 305 | 306 | case FullOuterJoinRow(left, right, true, false) => 307 | // no change 308 | left 309 | case FullOuterJoinRow(left, right, false, true) => 310 | // append 311 | merge(left, right) 312 | } 313 | } 314 | } 315 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/BitUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package tech.mlsql.common; 19 | 20 | /** A variety of high efficiency bit twiddling routines. 21 | * @lucene.internal 22 | */ 23 | public final class BitUtil { 24 | 25 | // magic numbers for bit interleaving 26 | private static final long MAGIC[] = { 27 | 0x5555555555555555L, 0x3333333333333333L, 28 | 0x0F0F0F0F0F0F0F0FL, 0x00FF00FF00FF00FFL, 29 | 0x0000FFFF0000FFFFL, 0x00000000FFFFFFFFL, 30 | 0xAAAAAAAAAAAAAAAAL 31 | }; 32 | // shift values for bit interleaving 33 | private static final short SHIFT[] = {1, 2, 4, 8, 16}; 34 | 35 | private BitUtil() {} // no instance 36 | 37 | // The pop methods used to rely on bit-manipulation tricks for speed but it 38 | // turns out that it is faster to use the Long.bitCount method (which is an 39 | // intrinsic since Java 6u18) in a naive loop, see LUCENE-2221 40 | 41 | /** Returns the number of set bits in an array of longs. */ 42 | public static long pop_array(long[] arr, int wordOffset, int numWords) { 43 | long popCount = 0; 44 | for (int i = wordOffset, end = wordOffset + numWords; i < end; ++i) { 45 | popCount += Long.bitCount(arr[i]); 46 | } 47 | return popCount; 48 | } 49 | 50 | /** Returns the popcount or cardinality of the two sets after an intersection. 51 | * Neither array is modified. */ 52 | public static long pop_intersect(long[] arr1, long[] arr2, int wordOffset, int numWords) { 53 | long popCount = 0; 54 | for (int i = wordOffset, end = wordOffset + numWords; i < end; ++i) { 55 | popCount += Long.bitCount(arr1[i] & arr2[i]); 56 | } 57 | return popCount; 58 | } 59 | 60 | /** Returns the popcount or cardinality of the union of two sets. 61 | * Neither array is modified. */ 62 | public static long pop_union(long[] arr1, long[] arr2, int wordOffset, int numWords) { 63 | long popCount = 0; 64 | for (int i = wordOffset, end = wordOffset + numWords; i < end; ++i) { 65 | popCount += Long.bitCount(arr1[i] | arr2[i]); 66 | } 67 | return popCount; 68 | } 69 | 70 | /** Returns the popcount or cardinality of {@code A & ~B}. 71 | * Neither array is modified. */ 72 | public static long pop_andnot(long[] arr1, long[] arr2, int wordOffset, int numWords) { 73 | long popCount = 0; 74 | for (int i = wordOffset, end = wordOffset + numWords; i < end; ++i) { 75 | popCount += Long.bitCount(arr1[i] & ~arr2[i]); 76 | } 77 | return popCount; 78 | } 79 | 80 | /** Returns the popcount or cardinality of A ^ B 81 | * Neither array is modified. */ 82 | public static long pop_xor(long[] arr1, long[] arr2, int wordOffset, int numWords) { 83 | long popCount = 0; 84 | for (int i = wordOffset, end = wordOffset + numWords; i < end; ++i) { 85 | popCount += Long.bitCount(arr1[i] ^ arr2[i]); 86 | } 87 | return popCount; 88 | } 89 | 90 | /** returns the next highest power of two, or the current value if it's already a power of two or zero*/ 91 | public static int nextHighestPowerOfTwo(int v) { 92 | v--; 93 | v |= v >> 1; 94 | v |= v >> 2; 95 | v |= v >> 4; 96 | v |= v >> 8; 97 | v |= v >> 16; 98 | v++; 99 | return v; 100 | } 101 | 102 | /** returns the next highest power of two, or the current value if it's already a power of two or zero*/ 103 | public static long nextHighestPowerOfTwo(long v) { 104 | v--; 105 | v |= v >> 1; 106 | v |= v >> 2; 107 | v |= v >> 4; 108 | v |= v >> 8; 109 | v |= v >> 16; 110 | v |= v >> 32; 111 | v++; 112 | return v; 113 | } 114 | 115 | /** 116 | * Interleaves the first 32 bits of each long value 117 | * 118 | * Adapted from: http://graphics.stanford.edu/~seander/bithacks.html#InterleaveBMN 119 | */ 120 | public static long interleave(int even, int odd) { 121 | long v1 = 0x00000000FFFFFFFFL & even; 122 | long v2 = 0x00000000FFFFFFFFL & odd; 123 | v1 = (v1 | (v1 << SHIFT[4])) & MAGIC[4]; 124 | v1 = (v1 | (v1 << SHIFT[3])) & MAGIC[3]; 125 | v1 = (v1 | (v1 << SHIFT[2])) & MAGIC[2]; 126 | v1 = (v1 | (v1 << SHIFT[1])) & MAGIC[1]; 127 | v1 = (v1 | (v1 << SHIFT[0])) & MAGIC[0]; 128 | v2 = (v2 | (v2 << SHIFT[4])) & MAGIC[4]; 129 | v2 = (v2 | (v2 << SHIFT[3])) & MAGIC[3]; 130 | v2 = (v2 | (v2 << SHIFT[2])) & MAGIC[2]; 131 | v2 = (v2 | (v2 << SHIFT[1])) & MAGIC[1]; 132 | v2 = (v2 | (v2 << SHIFT[0])) & MAGIC[0]; 133 | 134 | return (v2<<1) | v1; 135 | } 136 | 137 | /** 138 | * Extract just the even-bits value as a long from the bit-interleaved value 139 | */ 140 | public static long deinterleave(long b) { 141 | b &= MAGIC[0]; 142 | b = (b ^ (b >>> SHIFT[0])) & MAGIC[1]; 143 | b = (b ^ (b >>> SHIFT[1])) & MAGIC[2]; 144 | b = (b ^ (b >>> SHIFT[2])) & MAGIC[3]; 145 | b = (b ^ (b >>> SHIFT[3])) & MAGIC[4]; 146 | b = (b ^ (b >>> SHIFT[4])) & MAGIC[5]; 147 | return b; 148 | } 149 | 150 | /** 151 | * flip flops odd with even bits 152 | */ 153 | public static final long flipFlop(final long b) { 154 | return ((b & MAGIC[6]) >>> 1) | ((b & MAGIC[0]) << 1 ); 155 | } 156 | 157 | /** Same as {@link #zigZagEncode(long)} but on integers. */ 158 | public static int zigZagEncode(int i) { 159 | return (i >> 31) ^ (i << 1); 160 | } 161 | 162 | /** 163 | * Zig-zag 164 | * encode the provided long. Assuming the input is a signed long whose 165 | * absolute value can be stored on n bits, the returned value will 166 | * be an unsigned long that can be stored on n+1 bits. 167 | */ 168 | public static long zigZagEncode(long l) { 169 | return (l >> 63) ^ (l << 1); 170 | } 171 | 172 | /** Decode an int previously encoded with {@link #zigZagEncode(int)}. */ 173 | public static int zigZagDecode(int i) { 174 | return ((i >>> 1) ^ -(i & 1)); 175 | } 176 | 177 | /** Decode a long previously encoded with {@link #zigZagEncode(long)}. */ 178 | public static long zigZagDecode(long l) { 179 | return ((l >>> 1) ^ -(l & 1)); 180 | } 181 | 182 | /** 183 | * Returns whether the bit at given zero-based index is set. 184 | *
Example: bitIndex 66 means the third bit on the right of the second long. 185 | * 186 | * @param bits The bits stored in an array of long for efficiency. 187 | * @param numLongs The number of longs in {@code bits} to consider. 188 | * @param bitIndex The bit zero-based index. It must be greater than or equal to 0, 189 | * and strictly less than {@code numLongs * Long.SIZE}. 190 | */ 191 | public static boolean isBitSet(long[] bits, int numLongs, int bitIndex) { 192 | assert numLongs >= 0 && numLongs <= bits.length && bitIndex >= 0 && bitIndex < numLongs * Long.SIZE 193 | : "bitIndex=" + bitIndex + " numLongs=" + numLongs + " bits.length=" + bits.length; 194 | return (bits[bitIndex / Long.SIZE] & (1L << bitIndex)) != 0; // Shifts are mod 64. 195 | } 196 | 197 | /** 198 | * Counts all bits set in the provided longs. 199 | * 200 | * @param bits The bits stored in an array of long for efficiency. 201 | * @param numLongs The number of longs in {@code bits} to consider. 202 | */ 203 | public static int countBits(long[] bits, int numLongs) { 204 | assert numLongs >= 0 && numLongs <= bits.length 205 | : "numLongs=" + numLongs + " bits.length=" + bits.length; 206 | int bitCount = 0; 207 | for (int i = 0; i < numLongs; i++) { 208 | bitCount += Long.bitCount(bits[i]); 209 | } 210 | return bitCount; 211 | } 212 | 213 | /** 214 | * Counts the bits set up to the given bit zero-based index, exclusive. 215 | *
In other words, how many 1s there are up to the bit at the given index excluded. 216 | *
Example: bitIndex 66 means the third bit on the right of the second long. 217 | * 218 | * @param bits The bits stored in an array of long for efficiency. 219 | * @param numLongs The number of longs in {@code bits} to consider. 220 | * @param bitIndex The bit zero-based index, exclusive. It must be greater than or equal to 0, 221 | * and less than or equal to {@code numLongs * Long.SIZE}. 222 | */ 223 | public static int countBitsUpTo(long[] bits, int numLongs, int bitIndex) { 224 | assert numLongs >= 0 && numLongs <= bits.length && bitIndex >= 0 && bitIndex <= numLongs * Long.SIZE 225 | : "bitIndex=" + bitIndex + " numLongs=" + numLongs + " bits.length=" + bits.length; 226 | int bitCount = 0; 227 | int lastLong = bitIndex / Long.SIZE; 228 | for (int i = 0; i < lastLong; i++) { 229 | // Count the bits set for all plain longs. 230 | bitCount += Long.bitCount(bits[i]); 231 | } 232 | if (lastLong < numLongs) { 233 | // Prepare a mask with 1s on the right up to bitIndex exclusive. 234 | long mask = (1L << bitIndex) - 1L; // Shifts are mod 64. 235 | // Count the bits set only within the mask part, so up to bitIndex exclusive. 236 | bitCount += Long.bitCount(bits[lastLong] & mask); 237 | } 238 | return bitCount; 239 | } 240 | 241 | /** 242 | * Returns the index of the next bit set following the given bit zero-based index. 243 | *
For example with bits 100011: 244 | * the next bit set after index=-1 is at index=0; 245 | * the next bit set after index=0 is at index=1; 246 | * the next bit set after index=1 is at index=5; 247 | * there is no next bit set after index=5. 248 | * 249 | * @param bits The bits stored in an array of long for efficiency. 250 | * @param numLongs The number of longs in {@code bits} to consider. 251 | * @param bitIndex The bit zero-based index. It must be greater than or equal to -1, 252 | * and strictly less than {@code numLongs * Long.SIZE}. 253 | * @return The zero-based index of the next bit set after the provided {@code bitIndex}; 254 | * or -1 if none. 255 | */ 256 | public static int nextBitSet(long[] bits, int numLongs, int bitIndex) { 257 | assert numLongs >= 0 && numLongs <= bits.length && bitIndex >= -1 && bitIndex < numLongs * Long.SIZE 258 | : "bitIndex=" + bitIndex + " numLongs=" + numLongs + " bits.length=" + bits.length; 259 | int longIndex = bitIndex / Long.SIZE; 260 | // Prepare a mask with 1s on the left down to bitIndex exclusive. 261 | long mask = -(1L << (bitIndex + 1)); // Shifts are mod 64. 262 | long l = mask == -1 && bitIndex != -1 ? 0 : bits[longIndex] & mask; 263 | while (l == 0) { 264 | if (++longIndex == numLongs) { 265 | return -1; 266 | } 267 | l = bits[longIndex]; 268 | } 269 | return Long.numberOfTrailingZeros(l) + longIndex * 64; 270 | } 271 | 272 | /** 273 | * Returns the index of the previous bit set preceding the given bit zero-based index. 274 | *
For example with bits 100011: 275 | * there is no previous bit set before index=0. 276 | * the previous bit set before index=1 is at index=0; 277 | * the previous bit set before index=5 is at index=1; 278 | * the previous bit set before index=64 is at index=5; 279 | * 280 | * @param bits The bits stored in an array of long for efficiency. 281 | * @param numLongs The number of longs in {@code bits} to consider. 282 | * @param bitIndex The bit zero-based index. It must be greater than or equal to 0, 283 | * and less than or equal to {@code numLongs * Long.SIZE}. 284 | * @return The zero-based index of the previous bit set before the provided {@code bitIndex}; 285 | * or -1 if none. 286 | */ 287 | public static int previousBitSet(long[] bits, int numLongs, int bitIndex) { 288 | assert numLongs >= 0 && numLongs <= bits.length && bitIndex >= 0 && bitIndex <= numLongs * Long.SIZE 289 | : "bitIndex=" + bitIndex + " numLongs=" + numLongs + " bits.length=" + bits.length; 290 | int longIndex = bitIndex / Long.SIZE; 291 | long l; 292 | if (longIndex == numLongs) { 293 | l = 0; 294 | } else { 295 | // Prepare a mask with 1s on the right up to bitIndex exclusive. 296 | long mask = (1L << bitIndex) - 1L; // Shifts are mod 64. 297 | l = bits[longIndex] & mask; 298 | } 299 | while (l == 0) { 300 | if (longIndex-- == 0) { 301 | return -1; 302 | } 303 | l = bits[longIndex]; 304 | } 305 | return 63 - Long.numberOfLeadingZeros(l) + longIndex * 64; 306 | } 307 | } 308 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/BloomFilter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package tech.mlsql.common; 20 | 21 | import org.apache.hadoop.util.bloom.Key; 22 | import org.apache.hadoop.util.hash.Hash; 23 | 24 | import javax.xml.bind.DatatypeConverter; 25 | import java.io.*; 26 | import java.nio.charset.StandardCharsets; 27 | 28 | /** 29 | * A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}. 30 | */ 31 | public class BloomFilter { 32 | 33 | /** 34 | * Used in computing the optimal Bloom filter size. This approximately equals 0.480453. 35 | */ 36 | public static final double LOG2_SQUARED = Math.log(2) * Math.log(2); 37 | 38 | private org.apache.hadoop.util.bloom.BloomFilter filter = null; 39 | 40 | public BloomFilter(int numEntries, double errorRate) { 41 | this(numEntries, errorRate, Hash.MURMUR_HASH); 42 | } 43 | 44 | public int size() { 45 | return this.filter.getVectorSize(); 46 | } 47 | 48 | /** 49 | * Create a new Bloom filter with the given configurations. 50 | */ 51 | public BloomFilter(int numEntries, double errorRate, int hashType) { 52 | // Bit size 53 | int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED)); 54 | // Number of the hash functions 55 | int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries); 56 | // The filter 57 | this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType); 58 | } 59 | 60 | /** 61 | * Create the bloom filter from serialized string. 62 | */ 63 | public BloomFilter(String filterStr) { 64 | this.filter = new org.apache.hadoop.util.bloom.BloomFilter(); 65 | byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr); 66 | DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes)); 67 | try { 68 | this.filter.readFields(dis); 69 | dis.close(); 70 | } catch (IOException e) { 71 | throw new RuntimeException("Could not deserialize BloomFilter instance", e); 72 | } 73 | } 74 | 75 | public void add(String key) { 76 | if (key == null) { 77 | throw new NullPointerException("Key cannot by null"); 78 | } 79 | filter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); 80 | } 81 | 82 | public boolean mightContain(String key) { 83 | if (key == null) { 84 | throw new NullPointerException("Key cannot by null"); 85 | } 86 | return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); 87 | } 88 | 89 | /** 90 | * Serialize the bloom filter as a string. 91 | */ 92 | public String serializeToString() { 93 | ByteArrayOutputStream baos = new ByteArrayOutputStream(); 94 | DataOutputStream dos = new DataOutputStream(baos); 95 | try { 96 | filter.write(dos); 97 | byte[] bytes = baos.toByteArray(); 98 | dos.close(); 99 | return DatatypeConverter.printBase64Binary(bytes); 100 | } catch (IOException e) { 101 | throw new RuntimeException("Could not serialize BloomFilter instance", e); 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/DeltaJob.scala: -------------------------------------------------------------------------------- 1 | package tech.mlsql.common 2 | 3 | import org.apache.spark.sql.delta.DeltaConcurrentModificationException 4 | import org.apache.spark.sql.delta.metering.DeltaLogging 5 | 6 | /** 7 | * 28/11/2019 WilliamZhu(allwefantasy@gmail.com) 8 | */ 9 | object DeltaJob extends DeltaLogging { 10 | def runWithTry(f: () => Unit, tryTimes: Int = 3) = { 11 | val TRY_MAX_TIMES = tryTimes 12 | var count = 0L 13 | var successFlag = false 14 | var lastException: DeltaConcurrentModificationException = null 15 | while (count <= TRY_MAX_TIMES && !successFlag) { 16 | try { 17 | f() 18 | successFlag = true 19 | } catch { 20 | case e: DeltaConcurrentModificationException => 21 | count += 1 22 | lastException = e 23 | logWarning(s"try ${count} times", e) 24 | case e: Exception => throw e; 25 | } 26 | } 27 | 28 | if (!successFlag) { 29 | if (lastException != null) { 30 | throw lastException 31 | } else { 32 | throw new RuntimeException("should not happen") 33 | } 34 | 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/JsonUtils.scala: -------------------------------------------------------------------------------- 1 | package tech.mlsql.common 2 | 3 | import com.fasterxml.jackson.annotation.JsonInclude.Include 4 | import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} 5 | import com.fasterxml.jackson.module.scala.DefaultScalaModule 6 | import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper 7 | 8 | /** 9 | * 2019-06-20 WilliamZhu(allwefantasy@gmail.com) 10 | */ 11 | object JsonUtils { 12 | /** Used to convert between classes and JSON. */ 13 | val mapper = new ObjectMapper with ScalaObjectMapper 14 | mapper.setSerializationInclusion(Include.NON_NULL) 15 | mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) 16 | mapper.registerModule(DefaultScalaModule) 17 | 18 | def toJson[T: Manifest](obj: T): String = { 19 | mapper.writeValueAsString(obj) 20 | } 21 | 22 | def toPrettyJson[T: Manifest](obj: T): String = { 23 | mapper.writerWithDefaultPrettyPrinter().writeValueAsString(obj) 24 | } 25 | 26 | def fromJson[T: Manifest](json: String): T = { 27 | mapper.readValue[T](json) 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/ParquetReaderIterator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package tech.mlsql.common; 19 | 20 | import org.apache.parquet.hadoop.ParquetReader; 21 | 22 | import java.io.IOException; 23 | import java.util.Iterator; 24 | 25 | 26 | public class ParquetReaderIterator implements Iterator { 27 | 28 | // Parquet reader for an existing parquet file 29 | private final ParquetReader parquetReader; 30 | // Holds the next entry returned by the parquet reader 31 | private T next; 32 | 33 | public ParquetReaderIterator(ParquetReader parquetReader) { 34 | this.parquetReader = parquetReader; 35 | } 36 | 37 | @Override 38 | public boolean hasNext() { 39 | try { 40 | // To handle when hasNext() is called multiple times for idempotency and/or the first time 41 | if (this.next == null) { 42 | this.next = parquetReader.read(); 43 | } 44 | return this.next != null; 45 | } catch (IOException io) { 46 | throw new RuntimeException("unable to read next record from parquet file ", io); 47 | } 48 | } 49 | 50 | @Override 51 | public T next() { 52 | try { 53 | // To handle case when next() is called before hasNext() 54 | if (this.next == null) { 55 | if (!hasNext()) { 56 | throw new RuntimeException("No more records left to read from parquet file"); 57 | } 58 | } 59 | T retVal = this.next; 60 | this.next = parquetReader.read(); 61 | return retVal; 62 | } catch (IOException io) { 63 | throw new RuntimeException("unable to read next record from parquet file ", io); 64 | } 65 | } 66 | 67 | public void close() throws IOException { 68 | parquetReader.close(); 69 | } 70 | } 71 | 72 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/PathFun.scala: -------------------------------------------------------------------------------- 1 | package tech.mlsql.common 2 | 3 | import scala.collection.mutable.ArrayBuffer 4 | 5 | /** 6 | * 2019-06-20 WilliamZhu(allwefantasy@gmail.com) 7 | */ 8 | class PathFun(rootPath: String) { 9 | private val buffer = new ArrayBuffer[String]() 10 | buffer += rootPath.stripSuffix("/") 11 | 12 | def add(path: String) = { 13 | val cleanPath = path.stripPrefix("/").stripSuffix("/") 14 | if (!cleanPath.isEmpty) { 15 | buffer += cleanPath 16 | } 17 | this 18 | } 19 | 20 | def /(path: String) = { 21 | add(path) 22 | } 23 | 24 | def toPath = { 25 | buffer.mkString("/") 26 | } 27 | 28 | } 29 | 30 | object PathFun { 31 | def apply(rootPath: String): PathFun = new PathFun(rootPath) 32 | 33 | def joinPath(rootPath: String, paths: String*) = { 34 | val pf = apply(rootPath) 35 | for (arg <- paths) pf.add(arg) 36 | pf.toPath 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/ScalaReflect.scala: -------------------------------------------------------------------------------- 1 | package tech.mlsql.common 2 | 3 | import scala.reflect.ClassTag 4 | import scala.reflect.runtime.{universe => ru} 5 | 6 | /** 7 | * 2019-05-01 WilliamZhu(allwefantasy@gmail.com) 8 | */ 9 | 10 | class ScalaReflect {} 11 | 12 | object ScalaReflect { 13 | private def mirror = { 14 | ru.runtimeMirror(getClass.getClassLoader) 15 | } 16 | 17 | def fromInstance[T: ClassTag](obj: T) = { 18 | val x = mirror.reflect[T](obj) 19 | new ScalaMethodReflect(x) 20 | } 21 | 22 | def fromObject[T: ru.TypeTag]() = { 23 | new ScalaModuleReflect(ru.typeOf[T].typeSymbol.companion.asModule) 24 | } 25 | 26 | def fromObjectStr(str: String) = { 27 | val module = mirror.staticModule(str) 28 | new ScalaModuleReflect(module) 29 | } 30 | 31 | //def getClass[T: ru.TypeTag](obj: T) = ru.typeTag[T].tpe.typeSymbol.asClass 32 | 33 | 34 | } 35 | 36 | class ScalaModuleReflect(x: ru.ModuleSymbol) { 37 | 38 | private var methodName: Option[ru.MethodSymbol] = None 39 | private var fieldName: Option[ru.TermSymbol] = None 40 | 41 | def method(name: String) = { 42 | methodName = Option(x.typeSignature.member(ru.TermName(name)).asMethod) 43 | this 44 | } 45 | 46 | def field(name: String) = { 47 | fieldName = Option(x.typeSignature.member(ru.TermName(name)).asTerm) 48 | this 49 | } 50 | 51 | private def mirror = { 52 | ru.runtimeMirror(getClass.getClassLoader) 53 | } 54 | 55 | def invoke(objs: Any*) = { 56 | 57 | if (methodName.isDefined) { 58 | val instance = mirror.reflectModule(x).instance 59 | mirror.reflect(instance).reflectMethod(methodName.get.asMethod)(objs) 60 | } else if (fieldName.isDefined) { 61 | val instance = mirror.reflectModule(x).instance 62 | val fieldMirror = mirror.reflect(instance).reflectField(fieldName.get) 63 | if (objs.size > 0) { 64 | fieldMirror.set(objs.toSeq(0)) 65 | } 66 | fieldMirror.get 67 | 68 | } else { 69 | throw new IllegalArgumentException("Can not invoke `invoke` without call method or field function") 70 | } 71 | 72 | } 73 | } 74 | 75 | class ScalaMethodReflect(x: ru.InstanceMirror) { 76 | 77 | private var methodName: Option[ru.MethodSymbol] = None 78 | private var fieldName: Option[ru.TermSymbol] = None 79 | 80 | def method(name: String) = { 81 | methodName = Option(x.symbol.typeSignature.member(ru.TermName(name)).asMethod) 82 | this 83 | } 84 | 85 | def field(name: String) = { 86 | fieldName = Option(x.symbol.typeSignature.member(ru.TermName(name)).asTerm) 87 | this 88 | } 89 | 90 | def invoke(objs: Any*) = { 91 | 92 | if (methodName.isDefined) { 93 | x.reflectMethod(methodName.get.asMethod)(objs: _*) 94 | } else if (fieldName.isDefined) { 95 | val fieldMirror = x.reflectField(fieldName.get) 96 | if (objs.size > 0) { 97 | fieldMirror.set(objs.toSeq(0)) 98 | } 99 | fieldMirror.get 100 | 101 | } else { 102 | throw new IllegalArgumentException("Can not invoke `invoke` without call method or field function") 103 | } 104 | 105 | 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/main/java/tech/mlsql/common/ZOrderingIndexUtils.java: -------------------------------------------------------------------------------- 1 | package tech.mlsql.common; 2 | 3 | /** 4 | * 26/11/2019 WilliamZhu(allwefantasy@gmail.com) 5 | */ 6 | public class ZOrderingIndexUtils { 7 | 8 | // private static int splitBy3(int a) { 9 | // int x = a & 0x1fffff; //we only look at the first 21 bits 10 | // x = (x | x << 32) & 0x1f00000000ffff; // shift left 32 bits, OR with self, and 00011111000000000000000000000000000000001111111111111111 11 | // x = (x | x << 16) & 0x1f0000ff0000ff; // shift left 32 bits, OR with self, and 00011111000000000000000011111111000000000000000011111111 12 | // x = (x | x << 8) & 0x100f00f00f00f00f; // shift left 32 bits, OR with self, and 0001000000001111000000001111000000001111000000001111000000000000 13 | // x = (x | x << 4) & 0x10c30c30c30c30c3; // shift left 32 bits, OR with self, and 0001000011000011000011000011000011000011000011000011000100000000 14 | // x = (x | x << 2) & 0x1249249249249249; 15 | // return x; 16 | // } 17 | // 18 | // public static void encode2Address(int a,int b){ 19 | // 20 | // } 21 | } 22 | -------------------------------------------------------------------------------- /src/test/java/tech/mlsql/test/delta/DeltaCompactionSuite.scala: -------------------------------------------------------------------------------- 1 | package tech.mlsql.test.delta 2 | 3 | import java.io.File 4 | 5 | import org.apache.spark.sql.delta.actions.{Action, AddFile, RemoveFile} 6 | import org.apache.spark.sql.delta.commands.CompactTableInDelta 7 | import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions} 8 | import org.apache.spark.sql.execution.streaming.MemoryStream 9 | import org.apache.spark.sql.streaming._ 10 | import org.scalatest.time.SpanSugar._ 11 | 12 | // scalastyle:off: removeFile 13 | class DeltaCompactionSuite extends StreamTest { 14 | 15 | override val streamingTimeout = 1800.seconds 16 | 17 | import testImplicits._ 18 | 19 | private def withTempDirs(f: (File, File) => Unit): Unit = { 20 | withTempDir { file1 => 21 | withTempDir { file2 => 22 | f(file1, file2) 23 | } 24 | } 25 | } 26 | 27 | test("append mode") { 28 | failAfter(streamingTimeout) { 29 | withTempDirs { (outputDir, checkpointDir) => 30 | val inputData = MemoryStream[Int] 31 | val df = inputData.toDF() 32 | val query = df.writeStream 33 | .option("checkpointLocation", checkpointDir.getCanonicalPath) 34 | .outputMode(OutputMode.Append()) 35 | .format("delta") 36 | .start(outputDir.getCanonicalPath) 37 | val log = DeltaLog.forTable(spark, outputDir.getCanonicalPath) 38 | try { 39 | (1 to 15).foreach { i => 40 | inputData.addData(i) 41 | query.processAllAvailable() 42 | } 43 | val writeThread = new Thread(new Runnable { 44 | override def run(): Unit = 45 | (1 to 15).foreach { i => 46 | inputData.addData(i) 47 | Thread.sleep(1000) 48 | query.processAllAvailable() 49 | } 50 | }) 51 | writeThread.start() 52 | val optimizeTableInDelta = CompactTableInDelta(log, 53 | new DeltaOptions(Map[String, String](), df.sparkSession.sessionState.conf), Seq(), Map( 54 | CompactTableInDelta.COMPACT_VERSION_OPTION -> "8", 55 | CompactTableInDelta.COMPACT_NUM_FILE_PER_DIR -> "1", 56 | CompactTableInDelta.COMPACT_RETRY_TIMES_FOR_LOCK -> "60" 57 | )) 58 | optimizeTableInDelta.run(df.sparkSession) 59 | writeThread.join() 60 | val fileNum = new File(outputDir.getCanonicalPath).listFiles(). 61 | filter(f => f.getName.endsWith(".parquet")).length 62 | 63 | val outputDf = spark.read.format("delta").load(outputDir.getCanonicalPath) 64 | assert(outputDf.count() == 30) 65 | assert(fileNum == 22) 66 | 67 | 68 | } finally { 69 | query.stop() 70 | } 71 | } 72 | } 73 | 74 | } 75 | test("append mode with partitions") { 76 | failAfter(streamingTimeout) { 77 | withTempDirs { (outputDir, checkpointDir) => 78 | val inputData = MemoryStream[A] 79 | val df = inputData.toDF() 80 | val query = df.writeStream 81 | .option("checkpointLocation", checkpointDir.getCanonicalPath) 82 | .outputMode(OutputMode.Append()) 83 | .format("delta") 84 | .partitionBy("key") 85 | .start(outputDir.getCanonicalPath) 86 | val log = DeltaLog.forTable(spark, outputDir.getCanonicalPath) 87 | try { 88 | (1 to 15).foreach { i => 89 | val a = if (i > 3) "jack" else "william" 90 | inputData.addData(A(a, i)) 91 | query.processAllAvailable() 92 | } 93 | val writeThread = new Thread(new Runnable { 94 | override def run(): Unit = 95 | (1 to 15).foreach { i => 96 | val a = if (i > 4) "jack" else "william" 97 | inputData.addData(A(a, i)) 98 | Thread.sleep(1000) 99 | query.processAllAvailable() 100 | } 101 | }) 102 | writeThread.start() 103 | val optimizeTableInDelta = CompactTableInDelta(log, 104 | new DeltaOptions(Map[String, String](), df.sparkSession.sessionState.conf), Seq(), Map( 105 | CompactTableInDelta.COMPACT_VERSION_OPTION -> "8", 106 | CompactTableInDelta.COMPACT_NUM_FILE_PER_DIR -> "1", 107 | CompactTableInDelta.COMPACT_RETRY_TIMES_FOR_LOCK -> "60" 108 | )) 109 | val items = optimizeTableInDelta.run(df.sparkSession) 110 | writeThread.join() 111 | 112 | def recursiveListFiles(f: File): Array[File] = { 113 | val these = f.listFiles 114 | these ++ these.filter(_.isDirectory).flatMap(recursiveListFiles) 115 | } 116 | 117 | 118 | val fileNum = recursiveListFiles(new File(outputDir.getCanonicalPath)).filter { f => 119 | f.getName.endsWith(".parquet") && !f.getName.contains("checkpoint") 120 | }.length 121 | 122 | 123 | val acitons = items.map(f => Action.fromJson(f.getString(0))) 124 | val newFilesSize = acitons.filter(f => f.isInstanceOf[AddFile]).size 125 | val removeFilesSize = acitons.filter(f => f.isInstanceOf[RemoveFile]).size 126 | 127 | val outputDf = spark.read.format("delta").load(outputDir.getCanonicalPath) 128 | assert(outputDf.count() == 30) 129 | assert(fileNum == (30 - removeFilesSize + newFilesSize)) 130 | 131 | 132 | } finally { 133 | query.stop() 134 | } 135 | } 136 | } 137 | 138 | } 139 | } 140 | 141 | case class A(key: String, value: Int) 142 | -------------------------------------------------------------------------------- /src/test/java/tech/mlsql/test/delta/DeltaUpsertSuite.scala: -------------------------------------------------------------------------------- 1 | package tech.mlsql.test.delta 2 | 3 | import java.io.File 4 | 5 | import org.apache.spark.sql.delta.commands.UpsertTableInDelta 6 | import org.apache.spark.sql.delta.{ConcurrentWriteException, DeltaConcurrentModificationException} 7 | import org.apache.spark.sql.execution.streaming.MemoryStream 8 | import org.apache.spark.sql.streaming._ 9 | import org.scalatest.time.SpanSugar._ 10 | 11 | // scalastyle:off: removeFile 12 | class DeltaUpsertSuite extends StreamTest { 13 | 14 | override val streamingTimeout = 1800.seconds 15 | 16 | import testImplicits._ 17 | 18 | private def withTempDirs(f: (File, File) => Unit): Unit = { 19 | withTempDir { file1 => 20 | withTempDir { file2 => 21 | f(file1, file2) 22 | } 23 | } 24 | } 25 | 26 | val deltaFormat = "org.apache.spark.sql.delta.sources.MLSQLDeltaDataSource" 27 | test("test throw exception") { 28 | def _run = { 29 | throw new ConcurrentWriteException(None) 30 | } 31 | 32 | val TRY_MAX_TIMES = 3 33 | var count = 0L 34 | var breakFlag = false 35 | while (count <= TRY_MAX_TIMES && !breakFlag) { 36 | try { 37 | _run 38 | breakFlag = true 39 | } catch { 40 | case e: DeltaConcurrentModificationException => 41 | count += 1 42 | logWarning(s"try ${count} times", e) 43 | case e: Exception => throw e; 44 | } 45 | } 46 | } 47 | 48 | test("upsert with partitions") { 49 | failAfter(streamingTimeout) { 50 | withTempDirs { (outputDir, checkpointDir) => 51 | val inputData = MemoryStream[A2] 52 | val df = inputData.toDF() 53 | val query = df.writeStream 54 | .option("checkpointLocation", checkpointDir.getCanonicalPath) 55 | .outputMode(OutputMode.Append()) 56 | .format(deltaFormat) 57 | .option("idCols", "key,value") 58 | .partitionBy("key") 59 | .start(outputDir.getCanonicalPath) 60 | 61 | try { 62 | (1 to 6).foreach { i => 63 | val a = if (i > 3) "jack" else "william" 64 | inputData.addData(A2(a, i, a)) 65 | } 66 | query.processAllAvailable() 67 | spark.read.format(deltaFormat).load(outputDir.getCanonicalPath).createOrReplaceTempView("table1") 68 | var table1 = spark.sql( 69 | """ 70 | |select * from table1 where key="william" and value=2 71 | |""".stripMargin) 72 | assert(table1.count() == 1) 73 | assert(table1.collect().map(f => f.getAs[String]("value2")).head == "william") 74 | 75 | // try to upsert 76 | inputData.addData(A2("william", 2, "wow")) 77 | query.processAllAvailable() 78 | 79 | 80 | table1 = spark.sql( 81 | """ 82 | |select * from table1 where key="william" and value=2 83 | |""".stripMargin) 84 | assert(table1.count() == 1) 85 | assert(table1.collect().map(f => f.getAs[String]("value2")).head == "wow") 86 | 87 | table1 = spark.sql( 88 | """ 89 | |select * from table1 90 | |""".stripMargin) 91 | 92 | assert(table1.count() == 6) 93 | 94 | 95 | } finally { 96 | query.stop() 97 | } 98 | } 99 | } 100 | 101 | } 102 | 103 | test("delete with partitions") { 104 | failAfter(streamingTimeout) { 105 | withTempDirs { (outputDir, checkpointDir) => 106 | 107 | 108 | val data = (1 to 6).map { i => 109 | val a = if (i > 3) "jack" else "william" 110 | A2(a, i, a) 111 | } 112 | 113 | val initData = spark.createDataset[A2](data) 114 | initData.write.format(deltaFormat).save(outputDir.getCanonicalPath) 115 | 116 | val inputData = MemoryStream[A2] 117 | val df = inputData.toDF() 118 | val query = df.writeStream 119 | .option("checkpointLocation", checkpointDir.getCanonicalPath) 120 | .outputMode(OutputMode.Append()) 121 | .format(deltaFormat) 122 | .option(UpsertTableInDelta.ID_COLS, "key,value") 123 | .option(UpsertTableInDelta.OPERATION_TYPE, UpsertTableInDelta.OPERATION_TYPE_DELETE) 124 | .start(outputDir.getCanonicalPath) 125 | 126 | try { 127 | 128 | // try to delete 129 | inputData.addData(A2("william", 2, "wow")) 130 | query.processAllAvailable() 131 | 132 | spark.read.format(deltaFormat).load(outputDir.getCanonicalPath).createOrReplaceTempView("table1") 133 | var table1 = spark.sql( 134 | """ 135 | |select * from table1 where key="william" and value=2 136 | |""".stripMargin) 137 | assert(table1.count() == 0) 138 | 139 | table1 = spark.sql( 140 | """ 141 | |select * from table1 142 | |""".stripMargin) 143 | 144 | assert(table1.count() == 5) 145 | 146 | 147 | } finally { 148 | query.stop() 149 | } 150 | } 151 | } 152 | 153 | } 154 | } 155 | 156 | case class A2(key: String, value: Int, value2: String) 157 | --------------------------------------------------------------------------------