|
| 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 | +package org.apache.spark.gluten.delta |
| 18 | + |
| 19 | +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite} |
| 20 | + |
| 21 | +import org.apache.spark.SparkConf |
| 22 | +import org.apache.spark.sql.delta.files.TahoeFileIndex |
| 23 | +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper |
| 24 | + |
| 25 | +// Some sqls' line length exceeds 100 |
| 26 | +// scalastyle:off line.size.limit |
| 27 | + |
| 28 | +class GlutenDeltaParquetDeletionVectorSuite |
| 29 | + extends GlutenClickHouseTPCHAbstractSuite |
| 30 | + with AdaptiveSparkPlanHelper { |
| 31 | + |
| 32 | + override protected val needCopyParquetToTablePath = true |
| 33 | + |
| 34 | + override protected val tablesPath: String = basePath + "/tpch-data" |
| 35 | + override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" |
| 36 | + override protected val queriesResults: String = rootPath + "mergetree-queries-output" |
| 37 | + |
| 38 | + // import org.apache.gluten.backendsapi.clickhouse.CHConfig._ |
| 39 | + |
| 40 | + /** Run Gluten + ClickHouse Backend with SortShuffleManager */ |
| 41 | + override protected def sparkConf: SparkConf = { |
| 42 | + super.sparkConf |
| 43 | + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") |
| 44 | + .set("spark.io.compression.codec", "LZ4") |
| 45 | + .set("spark.sql.shuffle.partitions", "5") |
| 46 | + .set("spark.sql.autoBroadcastJoinThreshold", "10MB") |
| 47 | + .set("spark.sql.adaptive.enabled", "true") |
| 48 | + .set("spark.sql.files.maxPartitionBytes", "20000000") |
| 49 | + .set("spark.sql.storeAssignmentPolicy", "legacy") |
| 50 | + // .setCHConfig("use_local_format", true) |
| 51 | + .set("spark.databricks.delta.retentionDurationCheck.enabled", "false") |
| 52 | + } |
| 53 | + |
| 54 | + override protected def createTPCHNotNullTables(): Unit = { |
| 55 | + createNotNullTPCHTablesInParquet(tablesPath) |
| 56 | + } |
| 57 | + |
| 58 | + private val q1SchemaString: String = |
| 59 | + s""" l_orderkey bigint, |
| 60 | + | l_partkey bigint, |
| 61 | + | l_suppkey bigint, |
| 62 | + | l_linenumber bigint, |
| 63 | + | l_quantity double, |
| 64 | + | l_extendedprice double, |
| 65 | + | l_discount double, |
| 66 | + | l_tax double, |
| 67 | + | l_returnflag string, |
| 68 | + | l_linestatus string, |
| 69 | + | l_shipdate date, |
| 70 | + | l_commitdate date, |
| 71 | + | l_receiptdate date, |
| 72 | + | l_shipinstruct string, |
| 73 | + | l_shipmode string, |
| 74 | + | l_comment string""".stripMargin |
| 75 | + |
| 76 | + test("test parquet table delete with the delta DV") { |
| 77 | + spark.sql(s""" |
| 78 | + |set spark.gluten.enabled=false; |
| 79 | + |""".stripMargin) |
| 80 | + spark.sql(s""" |
| 81 | + |DROP TABLE IF EXISTS lineitem_delta_parquet_delete_dv; |
| 82 | + |""".stripMargin) |
| 83 | + |
| 84 | + spark.sql(s""" |
| 85 | + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_delete_dv |
| 86 | + |($q1SchemaString) |
| 87 | + |USING delta |
| 88 | + |TBLPROPERTIES (delta.enableDeletionVectors='true') |
| 89 | + |LOCATION '$basePath/lineitem_delta_parquet_delete_dv' |
| 90 | + |""".stripMargin) |
| 91 | + |
| 92 | + spark.sql(s""" |
| 93 | + | insert into table lineitem_delta_parquet_delete_dv |
| 94 | + | select /*+ REPARTITION(6) */ * from lineitem |
| 95 | + |""".stripMargin) |
| 96 | + |
| 97 | + val df1 = spark.sql(s""" |
| 98 | + | delete from lineitem_delta_parquet_delete_dv |
| 99 | + | where mod(l_orderkey, 3) = 1 and l_orderkey < 100 |
| 100 | + |""".stripMargin) |
| 101 | + spark.sql(s""" |
| 102 | + |set spark.gluten.enabled=true; |
| 103 | + |""".stripMargin) |
| 104 | + |
| 105 | + val df = spark.sql(s""" |
| 106 | + | select sum(l_linenumber) from lineitem_delta_parquet_delete_dv |
| 107 | + |""".stripMargin) |
| 108 | + val result = df.collect() |
| 109 | + assert( |
| 110 | + result.apply(0).get(0) === 1802335 |
| 111 | + ) |
| 112 | + val scanExec = collect(df.queryExecution.executedPlan) { |
| 113 | + case f: FileSourceScanExecTransformer => f |
| 114 | + } |
| 115 | + val parquetScan = scanExec.head |
| 116 | + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] |
| 117 | + val addFiles = fileIndex.matchingFiles(Nil, Nil) |
| 118 | + assert(addFiles.size === 6) |
| 119 | + |
| 120 | + spark.sql(s""" |
| 121 | + |set spark.gluten.enabled=false; |
| 122 | + |""".stripMargin) |
| 123 | + spark.sql(s""" |
| 124 | + | delete from lineitem_delta_parquet_delete_dv where mod(l_orderkey, 3) = 2 |
| 125 | + |""".stripMargin) |
| 126 | + spark.sql(s""" |
| 127 | + |set spark.gluten.enabled=true; |
| 128 | + |""".stripMargin) |
| 129 | + |
| 130 | + val df3 = spark.sql(s""" |
| 131 | + | select sum(l_linenumber) from lineitem_delta_parquet_delete_dv |
| 132 | + |""".stripMargin) |
| 133 | + assert( |
| 134 | + df3.collect().apply(0).get(0) === 1200560 |
| 135 | + ) |
| 136 | + } |
| 137 | + |
| 138 | + test("test parquet partition table delete with the delta DV") { |
| 139 | + withSQLConf(("spark.sql.sources.partitionOverwriteMode", "dynamic")) { |
| 140 | + spark.sql(s""" |
| 141 | + |set spark.gluten.enabled=false; |
| 142 | + |""".stripMargin) |
| 143 | + spark.sql(s""" |
| 144 | + |DROP TABLE IF EXISTS lineitem_delta_partition_parquet_delete_dv; |
| 145 | + |""".stripMargin) |
| 146 | + |
| 147 | + spark.sql(s""" |
| 148 | + |CREATE TABLE IF NOT EXISTS lineitem_delta_partition_parquet_delete_dv |
| 149 | + |($q1SchemaString) |
| 150 | + |USING delta |
| 151 | + |PARTITIONED BY (l_returnflag) |
| 152 | + |TBLPROPERTIES (delta.enableDeletionVectors='true') |
| 153 | + |LOCATION '$basePath/lineitem_delta_partition_parquet_delete_dv' |
| 154 | + |""".stripMargin) |
| 155 | + |
| 156 | + spark.sql(s""" |
| 157 | + | insert into table lineitem_delta_partition_parquet_delete_dv |
| 158 | + | select /*+ REPARTITION(6) */ * from lineitem |
| 159 | + |""".stripMargin) |
| 160 | + |
| 161 | + val df1 = spark.sql(s""" |
| 162 | + | delete from lineitem_delta_partition_parquet_delete_dv |
| 163 | + | where mod(l_orderkey, 3) = 1 |
| 164 | + |""".stripMargin) |
| 165 | + spark.sql(s""" |
| 166 | + |set spark.gluten.enabled=true; |
| 167 | + |""".stripMargin) |
| 168 | + |
| 169 | + val df = |
| 170 | + spark.sql(s""" |
| 171 | + | select sum(l_linenumber) from lineitem_delta_partition_parquet_delete_dv |
| 172 | + |""".stripMargin) |
| 173 | + val result = df.collect() |
| 174 | + assert( |
| 175 | + result.apply(0).get(0) === 1201486 |
| 176 | + ) |
| 177 | + val scanExec = collect(df.queryExecution.executedPlan) { |
| 178 | + case f: FileSourceScanExecTransformer => f |
| 179 | + } |
| 180 | + assert(scanExec.nonEmpty) |
| 181 | + } |
| 182 | + } |
| 183 | +} |
| 184 | +// scalastyle:off line.size.limit |
0 commit comments