Skip to content

Commit c8b61f5

Browse files
Yxangpan3793
authored andcommitted
Spark 3.4: Support distribute by any predefined transform
1 parent c025ef3 commit c8b61f5

File tree

10 files changed

+407
-4
lines changed

10 files changed

+407
-4
lines changed

clickhouse-core/src/main/scala/xenon/clickhouse/spec/NodeSpec.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -97,4 +97,6 @@ case class ClusterSpec(
9797
override def toString: String = s"cluster: $name, shards: [${shards.mkString(", ")}]"
9898

9999
@JsonIgnore @transient override lazy val nodes: Array[NodeSpec] = shards.sorted.flatMap(_.nodes)
100+
101+
def totalWeight: Int = shards.map(_.weight).sum
100102
}
Lines changed: 117 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,117 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* https://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
15+
package org.apache.spark.sql.clickhouse.cluster
16+
17+
import org.apache.spark.SparkConf
18+
import org.apache.spark.sql.Row
19+
20+
class ClusterShardByTransformSuite extends SparkClickHouseClusterTest {
21+
override protected def sparkConf: SparkConf = {
22+
val _conf = super.sparkConf
23+
.set("spark.clickhouse.write.distributed.convertLocal", "true")
24+
_conf
25+
}
26+
27+
def runTest(func_name: String, func_args: Array[String]): Unit = {
28+
val func_expr = s"$func_name(${func_args.mkString(",")})"
29+
val cluster = "single_replica"
30+
val db = s"db_${func_name}_shard_transform"
31+
val tbl_dist = s"tbl_${func_name}_shard"
32+
val tbl_local = s"${tbl_dist}_local"
33+
34+
try {
35+
runClickHouseSQL(s"CREATE DATABASE IF NOT EXISTS $db ON CLUSTER $cluster")
36+
37+
spark.sql(
38+
s"""CREATE TABLE $db.$tbl_local (
39+
| create_time TIMESTAMP NOT NULL,
40+
| create_date DATE NOT NULL,
41+
| value STRING NOT NULL
42+
|) USING ClickHouse
43+
|TBLPROPERTIES (
44+
| cluster = '$cluster',
45+
| engine = 'MergeTree()',
46+
| order_by = 'create_time'
47+
|)
48+
|""".stripMargin
49+
)
50+
51+
runClickHouseSQL(
52+
s"""CREATE TABLE $db.$tbl_dist ON CLUSTER $cluster
53+
|AS $db.$tbl_local
54+
|ENGINE = Distributed($cluster, '$db', '$tbl_local', $func_expr)
55+
|""".stripMargin
56+
)
57+
spark.sql(
58+
s"""INSERT INTO `$db`.`$tbl_dist`
59+
|VALUES
60+
| (timestamp'2021-01-01 10:10:10', date'2021-01-01', '1'),
61+
| (timestamp'2022-02-02 11:10:10', date'2022-02-02', '2'),
62+
| (timestamp'2023-03-03 12:10:10', date'2023-03-03', '3'),
63+
| (timestamp'2024-04-04 13:10:10', date'2024-04-04', '4')
64+
| AS tab(create_time, create_date, value)
65+
|""".stripMargin
66+
)
67+
// check that data is indeed written
68+
checkAnswer(
69+
spark.table(s"$db.$tbl_dist").select("value").orderBy("create_time"),
70+
Seq(Row("1"), Row("2"), Row("3"), Row("4"))
71+
)
72+
73+
// check same data is sharded in the same server comparing native sharding
74+
runClickHouseSQL(
75+
s"""INSERT INTO `$db`.`$tbl_dist`
76+
|VALUES
77+
| (timestamp'2021-01-01 10:10:10', date'2021-01-01', '1'),
78+
| (timestamp'2022-02-02 11:10:10', date'2022-02-02', '2'),
79+
| (timestamp'2023-03-03 12:10:10', date'2023-03-03', '3'),
80+
| (timestamp'2024-04-04 13:10:10', date'2024-04-04', '4')
81+
|""".stripMargin
82+
)
83+
checkAnswer(
84+
spark.table(s"$db.$tbl_local")
85+
.groupBy("value").count().filter("count != 2"),
86+
Seq.empty
87+
)
88+
89+
} finally {
90+
runClickHouseSQL(s"DROP TABLE IF EXISTS $db.$tbl_dist ON CLUSTER $cluster")
91+
runClickHouseSQL(s"DROP TABLE IF EXISTS $db.$tbl_local ON CLUSTER $cluster")
92+
runClickHouseSQL(s"DROP DATABASE IF EXISTS $db ON CLUSTER $cluster")
93+
}
94+
}
95+
96+
Seq(
97+
// wait for SPARK-44180 to be fixed, then add implicit cast test cases
98+
("toYear", Array("create_date")),
99+
// ("toYear", Array("create_time")),
100+
("toYYYYMM", Array("create_date")),
101+
// ("toYYYYMM", Array("create_time")),
102+
("toYYYYMMDD", Array("create_date")),
103+
// ("toYYYYMMDD", Array("create_time")),
104+
("toHour", Array("create_time")),
105+
("xxHash64", Array("value")),
106+
("murmurHash2_64", Array("value")),
107+
("murmurHash2_32", Array("value")),
108+
("murmurHash3_64", Array("value")),
109+
("murmurHash3_32", Array("value")),
110+
("cityHash64", Array("value")),
111+
("modulo", Array("toYYYYMM(create_date)", "10"))
112+
).foreach {
113+
case (func_name: String, func_args: Array[String]) =>
114+
test(s"shard by $func_name(${func_args.mkString(",")})")(runTest(func_name, func_args))
115+
}
116+
117+
}

spark-3.4/clickhouse-spark/src/main/scala/org/apache/spark/sql/clickhouse/ExprUtils.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ object ExprUtils extends SQLConfHelper with Serializable {
5858
functionRegistry: FunctionRegistry
5959
): Array[V2SortOrder] =
6060
toSparkSplits(
61-
shardingKeyIgnoreRand,
61+
shardingKeyIgnoreRand.map(k => ExprUtils.toSplitWithModulo(k, cluster.get.totalWeight)),
6262
partitionKey,
6363
functionRegistry
6464
).map(Expressions.sort(_, SortDirection.ASCENDING)) ++:
@@ -216,4 +216,7 @@ object ExprUtils extends SQLConfHelper with Serializable {
216216
case bucket: BucketTransform => throw CHClientException(s"Bucket transform not support yet: $bucket")
217217
case other: Transform => throw CHClientException(s"Unsupported transform: $other")
218218
}
219+
220+
def toSplitWithModulo(shardingKey: Expr, weight: Int): FuncExpr =
221+
FuncExpr("modulo", List(shardingKey, StringLiteral(weight.toString)))
219222
}
Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* https://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
15+
package xenon.clickhouse.func
16+
17+
import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction}
18+
import org.apache.spark.sql.types._
19+
20+
import java.time.LocalDate
21+
import java.time.format.DateTimeFormatter
22+
23+
object Days extends UnboundFunction with ScalarFunction[Int] with ClickhouseEquivFunction {
24+
25+
override def name: String = "clickhouse_days"
26+
27+
override def canonicalName: String = s"clickhouse.$name"
28+
29+
override def toString: String = name
30+
31+
override val ckFuncNames: Array[String] = Array("toYYYYMMDD")
32+
33+
override def description: String = s"$name: (date: Date) => shard_num: int"
34+
35+
override def bind(inputType: StructType): BoundFunction = inputType.fields match {
36+
case Array(StructField(_, DateType, _, _)) => this
37+
// case Array(StructField(_, TimestampType, _, _)) | Array(StructField(_, TimestampNTZType, _, _)) => this
38+
case _ => throw new UnsupportedOperationException(s"Expect 1 DATE argument. $description")
39+
}
40+
41+
override def inputTypes: Array[DataType] = Array(DateType)
42+
43+
override def resultType: DataType = IntegerType
44+
45+
override def isResultNullable: Boolean = false
46+
47+
def invoke(days: Int): Int = {
48+
val date = LocalDate.ofEpochDay(days)
49+
val formatter = DateTimeFormatter.ofPattern("yyyyMMdd")
50+
date.format(formatter).toInt
51+
}
52+
}

spark-3.4/clickhouse-spark/src/main/scala/xenon/clickhouse/func/FunctionRegistry.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,12 @@ object StaticFunctionRegistry extends FunctionRegistry {
5353
"clickhouse_murmurHash2_64" -> MurmurHash2_64,
5454
"clickhouse_murmurHash3_32" -> MurmurHash3_32,
5555
"clickhouse_murmurHash3_64" -> MurmurHash3_64,
56-
"clickhouse_cityHash64" -> CityHash64
56+
"clickhouse_cityHash64" -> CityHash64,
57+
"clickhouse_years" -> Years,
58+
"clickhouse_months" -> Months,
59+
"clickhouse_days" -> Days,
60+
"clickhouse_hours" -> Hours,
61+
"sharding_mod" -> Mod
5762
)
5863

5964
override def list: Array[String] = functions.keys.toArray
Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,51 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* https://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
15+
package xenon.clickhouse.func
16+
17+
import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction}
18+
import org.apache.spark.sql.types._
19+
20+
import java.sql.Timestamp
21+
import java.text.SimpleDateFormat
22+
23+
object Hours extends UnboundFunction with ScalarFunction[Int] with ClickhouseEquivFunction {
24+
25+
override def name: String = "clickhouse_hours"
26+
27+
override def canonicalName: String = s"clickhouse.$name"
28+
29+
override def toString: String = name
30+
31+
override val ckFuncNames: Array[String] = Array("toHour", "HOUR")
32+
33+
override def description: String = s"$name: (time: timestamp) => shard_num: int"
34+
35+
override def bind(inputType: StructType): BoundFunction = inputType.fields match {
36+
case Array(StructField(_, TimestampType, _, _)) | Array(StructField(_, TimestampNTZType, _, _)) => this
37+
case _ => throw new UnsupportedOperationException(s"Expect 1 TIMESTAMP argument. $description")
38+
}
39+
40+
override def inputTypes: Array[DataType] = Array(TimestampType)
41+
42+
override def resultType: DataType = IntegerType
43+
44+
override def isResultNullable: Boolean = false
45+
46+
def invoke(time: Long): Int = {
47+
val ts = new Timestamp(time / 1000)
48+
val formatter: SimpleDateFormat = new SimpleDateFormat("hh")
49+
formatter.format(ts).toInt
50+
}
51+
}
Lines changed: 63 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,63 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* https://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
15+
package xenon.clickhouse.func
16+
17+
import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction}
18+
import org.apache.spark.sql.types._
19+
20+
object Mod extends UnboundFunction with ScalarFunction[Long] with ClickhouseEquivFunction {
21+
22+
override def name: String = "sharding_mod"
23+
24+
override def canonicalName: String = s"clickhouse.$name"
25+
26+
override def toString: String = name
27+
28+
// remainder is not a Clickhouse function, but modulo will be parsed to remainder in the connector.
29+
// Added remainder as a synonym.
30+
override val ckFuncNames: Array[String] = Array("modulo", "remainder")
31+
32+
override def description: String = s"$name: (a: long, b: long) => mod: long"
33+
34+
override def bind(inputType: StructType): BoundFunction = inputType.fields match {
35+
case Array(a, b) if
36+
(a match {
37+
case StructField(_, LongType, _, _) => true
38+
case StructField(_, IntegerType, _, _) => true
39+
case StructField(_, ShortType, _, _) => true
40+
case StructField(_, ByteType, _, _) => true
41+
case StructField(_, StringType, _, _) => true
42+
case _ => false
43+
}) &&
44+
(b match {
45+
case StructField(_, LongType, _, _) => true
46+
case StructField(_, IntegerType, _, _) => true
47+
case StructField(_, ShortType, _, _) => true
48+
case StructField(_, ByteType, _, _) => true
49+
case StructField(_, StringType, _, _) => true
50+
case _ => false
51+
}) =>
52+
this
53+
case _ => throw new UnsupportedOperationException(s"Expect 2 integer arguments. $description")
54+
}
55+
56+
override def inputTypes: Array[DataType] = Array(LongType, LongType)
57+
58+
override def resultType: DataType = LongType
59+
60+
override def isResultNullable: Boolean = false
61+
62+
def invoke(a: Long, b: Long): Long = a % b
63+
}
Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* https://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
15+
package xenon.clickhouse.func
16+
17+
import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction}
18+
import org.apache.spark.sql.types._
19+
20+
import java.time.LocalDate
21+
import java.time.format.DateTimeFormatter
22+
23+
object Months extends UnboundFunction with ScalarFunction[Int] with ClickhouseEquivFunction {
24+
25+
override def name: String = "clickhouse_months"
26+
27+
override def canonicalName: String = s"clickhouse.$name"
28+
29+
override def toString: String = name
30+
31+
override val ckFuncNames: Array[String] = Array("toYYYYMM")
32+
33+
override def description: String = s"$name: (date: Date) => shard_num: int"
34+
35+
override def bind(inputType: StructType): BoundFunction = inputType.fields match {
36+
case Array(StructField(_, DateType, _, _)) => this
37+
// case Array(StructField(_, TimestampType, _, _)) | Array(StructField(_, TimestampNTZType, _, _)) => this
38+
case _ => throw new UnsupportedOperationException(s"Expect 1 DATE argument. $description")
39+
}
40+
41+
override def inputTypes: Array[DataType] = Array(DateType)
42+
43+
override def resultType: DataType = IntegerType
44+
45+
override def isResultNullable: Boolean = false
46+
47+
def invoke(days: Int): Int = {
48+
val date = LocalDate.ofEpochDay(days)
49+
val formatter = DateTimeFormatter.ofPattern("yyyyMM")
50+
date.format(formatter).toInt
51+
}
52+
}

0 commit comments

Comments
 (0)