forked from apache/incubator-gluten
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[GLUTEN-5454][CH] Support delete/update/optimize/vacuum API for the M…
…ergeTree + Delta (apache#5460) Support delete/update/optimize/vacuum API for the MergeTree + Delta: Similar to the DeltaTable API in Delta, add a ClickhouseTable to support delete/update/optimize/vacuum for the MergeTree + Delta by the api way: ``` // create a ClickhouseTable, the dataPath is the delta root path val clickhouseTable = ClickhouseTable.forPath(spark, dataPath) // optimize clickhouseTable.optimize().executeCompaction() // vacuum clickhouseTable.vacuum(0.0) // update clickhouseTable.updateExpr("l_orderkey = 10086", Map("l_returnflag" -> "'X'")) // delete clickhouseTable.delete("mod(l_orderkey, 3) = 2") ``` Close apache#5454.
- Loading branch information
Showing
13 changed files
with
364 additions
and
80 deletions.
There are no files selected for viewing
71 changes: 71 additions & 0 deletions
71
backends-clickhouse/src/main/delta-20/io/delta/tables/ClickhouseTable.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package io.delta.tables | ||
|
||
import org.apache.spark.sql.{Dataset, Row, SparkSession} | ||
import org.apache.spark.sql.delta.{DeltaErrors, DeltaTableIdentifier, DeltaTableUtils} | ||
import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 | ||
|
||
import org.apache.hadoop.fs.Path | ||
|
||
class ClickhouseTable( | ||
@transient private val _df: Dataset[Row], | ||
@transient private val table: ClickHouseTableV2) | ||
extends DeltaTable(_df, table) { | ||
|
||
override def optimize(): DeltaOptimizeBuilder = { | ||
DeltaOptimizeBuilder( | ||
sparkSession, | ||
table.tableIdentifier.getOrElse(s"clickhouse.`${deltaLog.dataPath.toString}`")) | ||
} | ||
} | ||
|
||
object ClickhouseTable { | ||
|
||
/** | ||
* Create a DeltaTable for the data at the given `path`. | ||
* | ||
* Note: This uses the active SparkSession in the current thread to read the table data. Hence, | ||
* this throws error if active SparkSession has not been set, that is, | ||
* `SparkSession.getActiveSession()` is empty. | ||
* | ||
* @since 0.3.0 | ||
*/ | ||
def forPath(path: String): DeltaTable = { | ||
val sparkSession = SparkSession.getActiveSession.getOrElse { | ||
throw DeltaErrors.activeSparkSessionNotFound() | ||
} | ||
forPath(sparkSession, path) | ||
} | ||
|
||
/** | ||
* Create a DeltaTable for the data at the given `path` using the given SparkSession. | ||
* | ||
* @since 0.3.0 | ||
*/ | ||
def forPath(sparkSession: SparkSession, path: String): DeltaTable = { | ||
val hdpPath = new Path(path) | ||
if (DeltaTableUtils.isDeltaTable(sparkSession, hdpPath)) { | ||
new ClickhouseTable( | ||
sparkSession.read.format("clickhouse").load(path), | ||
new ClickHouseTableV2(spark = sparkSession, path = hdpPath) | ||
) | ||
} else { | ||
throw DeltaErrors.notADeltaTableException(DeltaTableIdentifier(path = Some(path))) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
137 changes: 137 additions & 0 deletions
137
backends-clickhouse/src/main/delta-22/io/delta/tables/ClickhouseTable.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,137 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package io.delta.tables | ||
|
||
import org.apache.spark.sql.{Dataset, Row, SparkSession} | ||
import org.apache.spark.sql.delta.{DeltaErrors, DeltaTableIdentifier, DeltaTableUtils} | ||
import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 | ||
|
||
import org.apache.hadoop.fs.Path | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
class ClickhouseTable( | ||
@transient private val _df: Dataset[Row], | ||
@transient private val table: ClickHouseTableV2) | ||
extends DeltaTable(_df, table) { | ||
|
||
override def optimize(): DeltaOptimizeBuilder = { | ||
DeltaOptimizeBuilder( | ||
sparkSession, | ||
table.tableIdentifier.getOrElse(s"clickhouse.`${deltaLog.dataPath.toString}`"), | ||
table.options) | ||
} | ||
} | ||
|
||
object ClickhouseTable { | ||
|
||
/** | ||
* Instantiate a [[DeltaTable]] object representing the data at the given path, If the given path | ||
* is invalid (i.e. either no table exists or an existing table is not a Delta table), it throws a | ||
* `not a Delta table` error. | ||
* | ||
* Note: This uses the active SparkSession in the current thread to read the table data. Hence, | ||
* this throws error if active SparkSession has not been set, that is, | ||
* `SparkSession.getActiveSession()` is empty. | ||
* | ||
* @since 0.3.0 | ||
*/ | ||
def forPath(path: String): DeltaTable = { | ||
val sparkSession = SparkSession.getActiveSession.getOrElse { | ||
throw DeltaErrors.activeSparkSessionNotFound() | ||
} | ||
forPath(sparkSession, path) | ||
} | ||
|
||
/** | ||
* Instantiate a [[DeltaTable]] object representing the data at the given path, If the given path | ||
* is invalid (i.e. either no table exists or an existing table is not a Delta table), it throws a | ||
* `not a Delta table` error. | ||
* | ||
* @since 0.3.0 | ||
*/ | ||
def forPath(sparkSession: SparkSession, path: String): DeltaTable = { | ||
forPath(sparkSession, path, Map.empty[String, String]) | ||
} | ||
|
||
/** | ||
* Instantiate a [[DeltaTable]] object representing the data at the given path, If the given path | ||
* is invalid (i.e. either no table exists or an existing table is not a Delta table), it throws a | ||
* `not a Delta table` error. | ||
* | ||
* @param hadoopConf | ||
* Hadoop configuration starting with "fs." or "dfs." will be picked up by `DeltaTable` to | ||
* access the file system when executing queries. Other configurations will not be allowed. | ||
* | ||
* {{{ | ||
* val hadoopConf = Map( | ||
* "fs.s3a.access.key" -> "<access-key>", | ||
* "fs.s3a.secret.key" -> "<secret-key>" | ||
* ) | ||
* DeltaTable.forPath(spark, "/path/to/table", hadoopConf) | ||
* }}} | ||
* @since 2.2.0 | ||
*/ | ||
def forPath( | ||
sparkSession: SparkSession, | ||
path: String, | ||
hadoopConf: scala.collection.Map[String, String]): DeltaTable = { | ||
// We only pass hadoopConf so that we won't pass any unsafe options to Delta. | ||
val badOptions = hadoopConf.filterKeys { | ||
k => !DeltaTableUtils.validDeltaTableHadoopPrefixes.exists(k.startsWith) | ||
}.toMap | ||
if (!badOptions.isEmpty) { | ||
throw DeltaErrors.unsupportedDeltaTableForPathHadoopConf(badOptions) | ||
} | ||
val fileSystemOptions: Map[String, String] = hadoopConf.toMap | ||
val hdpPath = new Path(path) | ||
if (DeltaTableUtils.isDeltaTable(sparkSession, hdpPath, fileSystemOptions)) { | ||
new ClickhouseTable( | ||
sparkSession.read.format("clickhouse").options(fileSystemOptions).load(path), | ||
new ClickHouseTableV2(spark = sparkSession, path = hdpPath, options = fileSystemOptions) | ||
) | ||
} else { | ||
throw DeltaErrors.notADeltaTableException(DeltaTableIdentifier(path = Some(path))) | ||
} | ||
} | ||
|
||
/** | ||
* Java friendly API to instantiate a [[DeltaTable]] object representing the data at the given | ||
* path, If the given path is invalid (i.e. either no table exists or an existing table is not a | ||
* Delta table), it throws a `not a Delta table` error. | ||
* | ||
* @param hadoopConf | ||
* Hadoop configuration starting with "fs." or "dfs." will be picked up by `DeltaTable` to | ||
* access the file system when executing queries. Other configurations will be ignored. | ||
* | ||
* {{{ | ||
* val hadoopConf = Map( | ||
* "fs.s3a.access.key" -> "<access-key>", | ||
* "fs.s3a.secret.key", "<secret-key>" | ||
* ) | ||
* DeltaTable.forPath(spark, "/path/to/table", hadoopConf) | ||
* }}} | ||
* @since 2.2.0 | ||
*/ | ||
def forPath( | ||
sparkSession: SparkSession, | ||
path: String, | ||
hadoopConf: java.util.Map[String, String]): DeltaTable = { | ||
val fsOptions = hadoopConf.asScala.toMap | ||
forPath(sparkSession, path, fsOptions) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
43 changes: 0 additions & 43 deletions
43
backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/commands/CommandUtils.scala
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.