diff --git a/.github/workflows/build-and-test.yml b/.github/workflows/build-and-test.yml index 86edb521..0ec0b1e0 100644 --- a/.github/workflows/build-and-test.yml +++ b/.github/workflows/build-and-test.yml @@ -13,16 +13,7 @@ # name: "Build and Test" -on: - push: - branches: - - "branch-*" - - "main" - pull_request: - branches: - - "branch-*" - - "main" - workflow_dispatch: +on: [push] jobs: run-tests: diff --git a/.github/workflows/cloud.yml b/.github/workflows/cloud.yml index 2487fcd1..db74c52d 100644 --- a/.github/workflows/cloud.yml +++ b/.github/workflows/cloud.yml @@ -28,6 +28,8 @@ on: jobs: run-tests-with-clickhouse-cloud: runs-on: ubuntu-22.04 + # Only run on main repository where secrets are available + if: github.repository == 'ClickHouse/spark-clickhouse-connector' strategy: max-parallel: 1 fail-fast: false @@ -44,6 +46,31 @@ jobs: distribution: zulu java-version: 8 cache: gradle + - name: Wake up ClickHouse Cloud instance + env: + CLICKHOUSE_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }} + run: | + echo "Waking up ClickHouse Cloud instance..." + max_attempts=3 + attempt=1 + + while [ $attempt -le $max_attempts ]; do + echo "Attempt $attempt of $max_attempts" + if curl -sS "https://${CLICKHOUSE_CLOUD_HOST}:8443/?query=SELECT+1" \ + --user "default:${CLICKHOUSE_PASSWORD}" \ + --max-time 60 > /dev/null; then + echo "Instance is awake!" + break + else + if [ $attempt -eq $max_attempts ]; then + echo "Failed to wake instance after $max_attempts attempts" + exit 1 + fi + echo "Retrying in 10 seconds..." + sleep 10 + ((attempt++)) + fi + done - run: >- ./gradlew clean cloudTest --no-daemon --refresh-dependencies -Dspark_binary_version=${{ matrix.spark }} diff --git a/build.gradle b/build.gradle index 8f96e97e..171988f6 100644 --- a/build.gradle +++ b/build.gradle @@ -90,7 +90,16 @@ allprojects { version = getProjectVersion() repositories { - maven { url = "$mavenCentralMirror" } + maven { + url = "$mavenCentralMirror" + } + + maven { + url = "$mavenSnapshotsRepo" + mavenContent { + snapshotsOnly() + } + } } } @@ -218,7 +227,7 @@ project(':clickhouse-core') { api "com.fasterxml.jackson.datatype:jackson-datatype-jsr310:$jackson_version" api "com.fasterxml.jackson.module:jackson-module-scala_$scala_binary_version:$jackson_version" - api("com.clickhouse:clickhouse-jdbc:$clickhouse_jdbc_version:all") { transitive = false } + api("com.clickhouse:client-v2:${clickhouse_client_v2_version}:all") { transitive = false } compileOnly "jakarta.annotation:jakarta.annotation-api:$jakarta_annotation_api_version" @@ -239,6 +248,7 @@ project(":clickhouse-core-it") { testImplementation(testFixtures(project(":clickhouse-core"))) testImplementation("com.clickhouse:clickhouse-jdbc:$clickhouse_jdbc_version:all") { transitive = false } + testImplementation "org.slf4j:slf4j-log4j12:$slf4j_version" } diff --git a/clickhouse-core/src/main/scala/com/clickhouse/spark/client/NodeClient.scala b/clickhouse-core/src/main/scala/com/clickhouse/spark/client/NodeClient.scala index 52d98fab..0220b685 100644 --- a/clickhouse-core/src/main/scala/com/clickhouse/spark/client/NodeClient.scala +++ b/clickhouse-core/src/main/scala/com/clickhouse/spark/client/NodeClient.scala @@ -14,10 +14,16 @@ package com.clickhouse.spark.client -import com.clickhouse.spark.Logging import com.clickhouse.client._ -import com.clickhouse.client.config.ClickHouseClientOption -import com.clickhouse.data.{ClickHouseCompression, ClickHouseFormat} +import com.clickhouse.client.api.{Client, ServerException} +import com.clickhouse.client.api.enums.Protocol +import com.clickhouse.client.api.insert.{InsertResponse, InsertSettings} +import com.clickhouse.client.api.query.{QueryResponse, QuerySettings} +import com.clickhouse.data.ClickHouseFormat +import com.clickhouse.shaded.org.apache.commons.io.IOUtils +import com.clickhouse.spark.Logging + +import java.util.concurrent.TimeUnit import com.clickhouse.spark.exception.{CHClientException, CHException, CHServerException} import com.clickhouse.spark.format.{ JSONCompactEachRowWithNamesAndTypesSimpleOutput, @@ -30,7 +36,8 @@ import com.clickhouse.spark.spec.NodeSpec import com.fasterxml.jackson.databind.JsonNode import com.fasterxml.jackson.databind.node.ObjectNode -import java.io.InputStream +import java.io.{ByteArrayInputStream, InputStream} +import java.time.temporal.ChronoUnit import java.util.UUID import scala.util.{Failure, Success, Try} @@ -40,7 +47,7 @@ object NodeClient { class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { // TODO: add configurable timeout - private val timeout: Int = 30000 + private val timeout: Int = 60000 private lazy val userAgent: String = { val title = getClass.getPackage.getImplementationTitle @@ -78,24 +85,26 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { private def shouldInferRuntime(): Boolean = nodeSpec.infer_runtime_env.equalsIgnoreCase("true") || nodeSpec.infer_runtime_env == "1" - private val node: ClickHouseNode = ClickHouseNode.builder() - .options(nodeSpec.options) - .host(nodeSpec.host) - .port(nodeSpec.protocol, nodeSpec.port) - .database(nodeSpec.database) - .credentials(ClickHouseCredentials.fromUserAndPassword(nodeSpec.username, nodeSpec.password)) - .build() + private def createClickHouseURL(nodeSpec: NodeSpec): String = { + val ssl: Boolean = nodeSpec.options.getOrDefault("ssl", "false").toBoolean + if (ssl) { + s"https://${nodeSpec.host}:${nodeSpec.port}" + } else { + s"http://${nodeSpec.host}:${nodeSpec.port}" + } + } - private val client: ClickHouseClient = ClickHouseClient.builder() - .option(ClickHouseClientOption.FORMAT, ClickHouseFormat.RowBinary) - .option( - ClickHouseClientOption.PRODUCT_NAME, - userAgent - ) - .nodeSelector(ClickHouseNodeSelector.of(node.getProtocol)) + private val client = new Client.Builder() + .setUsername(nodeSpec.username) + .setPassword(nodeSpec.password) + .setDefaultDatabase(nodeSpec.database) + .setOptions(nodeSpec.options) + .setClientName(userAgent) + .addEndpoint(createClickHouseURL(nodeSpec)) .build() - override def close(): Unit = client.close() + override def close(): Unit = + client.close() private def nextQueryId(): String = UUID.randomUUID.toString @@ -119,7 +128,6 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { database: String, table: String, inputFormat: String, - inputCompressionType: ClickHouseCompression = ClickHouseCompression.NONE, data: InputStream, settings: Map[String, String] = Map.empty ): Either[CHException, SimpleOutput[ObjectNode]] = @@ -127,7 +135,6 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { database, table, inputFormat, - inputCompressionType, data, "JSONEachRow", JSONEachRowSimpleOutput.deserialize, @@ -149,24 +156,32 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { database: String, table: String, inputFormat: String, - inputCompressionType: ClickHouseCompression, data: InputStream, outputFormat: String, deserializer: InputStream => SimpleOutput[OUT], settings: Map[String, String] ): Either[CHException, SimpleOutput[OUT]] = { + def readAllBytes(inputStream: InputStream): Array[Byte] = + IOUtils.toByteArray(inputStream) val queryId = nextQueryId() val sql = s"INSERT INTO `$database`.`$table` FORMAT $inputFormat" onExecuteQuery(queryId, sql) - val req = client.write(node) - .query(sql, queryId) - .decompressClientRequest(inputCompressionType) - .format(ClickHouseFormat.valueOf(outputFormat)) - settings.foreach { case (k, v) => req.set(k, v) } - Try(req.data(data).executeAndWait()) match { - case Success(resp) => Right(deserializer(resp.getInputStream)) - case Failure(ex: ClickHouseException) => - Left(CHServerException(ex.getErrorCode, ex.getMessage, Some(nodeSpec), Some(ex))) + val insertSettings: InsertSettings = new InsertSettings(); + settings.foreach { case (k, v) => insertSettings.setOption(k, v) } + insertSettings.setDatabase(database) + // TODO: check what type of compression is supported by the client v2 + insertSettings.compressClientRequest(true) + val payload: Array[Byte] = readAllBytes(data) + val is: InputStream = new ByteArrayInputStream("".getBytes()) + Try(client.insert( + table, + new ByteArrayInputStream(payload), + ClickHouseFormat.valueOf(inputFormat), + insertSettings + ).get()) match { + case Success(resp: InsertResponse) => Right(deserializer(is)) + case Failure(se: ServerException) => + Left(CHServerException(se.getCode, se.getMessage, Some(nodeSpec), Some(se))) case Failure(ex) => Left(CHClientException(ex.getMessage, Some(nodeSpec), Some(ex))) } } @@ -179,16 +194,15 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { ): Either[CHException, SimpleOutput[OUT]] = { val queryId = nextQueryId() onExecuteQuery(queryId, sql) - val req = client.read(node) - .query(sql, queryId).asInstanceOf[ClickHouseRequest[_]] - .format(ClickHouseFormat.valueOf(outputFormat)).asInstanceOf[ClickHouseRequest[_]] - .option(ClickHouseClientOption.CONNECTION_TIMEOUT, timeout).asInstanceOf[ClickHouseRequest[_]] - settings.foreach { case (k, v) => req.set(k, v).asInstanceOf[ClickHouseRequest[_]] } - Try(req.executeAndWait()) match { - case Success(resp) => Right(deserializer(resp.getInputStream)) - case Failure(ex: ClickHouseException) => - Left(CHServerException(ex.getErrorCode, ex.getMessage, Some(nodeSpec), Some(ex))) - case Failure(ex) => Left(CHClientException(ex.getMessage, Some(nodeSpec), Some(ex))) + val querySettings: QuerySettings = new QuerySettings() + val clickHouseFormat = ClickHouseFormat.valueOf(outputFormat) + querySettings.setFormat(clickHouseFormat) + querySettings.setQueryId(queryId) + settings.foreach { case (k, v) => querySettings.setOption(k, v) } + Try(client.query(sql, querySettings).get(timeout, TimeUnit.MILLISECONDS)) match { + case Success(response: QueryResponse) => Right(deserializer(response.getInputStream)) + case Failure(se: ServerException) => Left(CHServerException(se.getCode, se.getMessage, Some(nodeSpec), Some(se))) + case Failure(ex: Exception) => Left(CHClientException(ex.getMessage, Some(nodeSpec), Some(ex))) } } @@ -203,28 +217,26 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { } // ////////////////////////////////////////////////////////////////////////////// - // ///////////////////////// ret ClickHouseResponse ///////////////////////////// + // ///////////////////////// ret QueryResponse ///////////////////////////// // ////////////////////////////////////////////////////////////////////////////// def queryAndCheck( sql: String, outputFormat: String, - outputCompressionType: ClickHouseCompression, settings: Map[String, String] = Map.empty - ): ClickHouseResponse = { + ): QueryResponse = { val queryId = nextQueryId() onExecuteQuery(queryId, sql) - val req = client.read(node) - .query(sql, queryId).asInstanceOf[ClickHouseRequest[_]] - .compressServerResponse(outputCompressionType).asInstanceOf[ClickHouseRequest[_]] - .format(ClickHouseFormat.valueOf(outputFormat)).asInstanceOf[ClickHouseRequest[_]] - .option(ClickHouseClientOption.CONNECTION_TIMEOUT, timeout).asInstanceOf[ClickHouseRequest[_]] - settings.foreach { case (k, v) => req.set(k, v).asInstanceOf[ClickHouseRequest[_]] } - Try(req.executeAndWait()) match { - case Success(resp) => resp - case Failure(ex: ClickHouseException) => - throw CHServerException(ex.getErrorCode, ex.getMessage, Some(nodeSpec), Some(ex)) - case Failure(ex) => throw CHClientException(ex.getMessage, Some(nodeSpec), Some(ex)) + val querySettings: QuerySettings = new QuerySettings() + val clickHouseFormat = ClickHouseFormat.valueOf(outputFormat) + querySettings.setFormat(clickHouseFormat) + querySettings.setQueryId(queryId) + settings.foreach { case (k, v) => querySettings.setOption(k, v) } + + Try(client.query(sql, querySettings).get(timeout, TimeUnit.MILLISECONDS)) match { + case Success(response: QueryResponse) => response + case Failure(se: ServerException) => throw CHServerException(se.getCode, se.getMessage, Some(nodeSpec), Some(se)) + case Failure(ex: Exception) => throw CHClientException(ex.getMessage, Some(nodeSpec), Some(ex)) } } @@ -238,5 +250,5 @@ class NodeClient(val nodeSpec: NodeSpec) extends AutoCloseable with Logging { |""".stripMargin ) def ping(timeout: Int = timeout) = - client.ping(node, timeout) + client.ping(timeout) } diff --git a/gradle.properties b/gradle.properties index ef0863bc..dd9fbbe9 100644 --- a/gradle.properties +++ b/gradle.properties @@ -13,7 +13,7 @@ # mavenCentralMirror=https://repo1.maven.org/maven2/ -mavenSnapshotsRepo=https://s01.oss.sonatype.org/content/repositories/snapshots/ +mavenSnapshotsRepo=https://central.sonatype.com/repository/maven-snapshots/ mavenReleasesRepo=https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/ systemProp.scala_binary_version=2.12 @@ -23,7 +23,8 @@ systemProp.known_spark_binary_versions=3.3,3.4,3.5 group=com.clickhouse.spark -clickhouse_jdbc_version=0.6.3 +clickhouse_jdbc_version=0.9.4 +clickhouse_client_v2_version=0.9.4 spark_33_version=3.3.4 spark_34_version=3.4.2 diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala index 6f5686fe..30799fe2 100644 --- a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala @@ -38,6 +38,8 @@ class TPCDSClusterSuite extends SparkClickHouseClusterTest { spark.sql("CREATE DATABASE tpcds_sf1_cluster WITH DBPROPERTIES (cluster = 'single_replica')") TPCDSTestUtils.tablePrimaryKeys.foreach { case (table, primaryKeys) => + println(s"before table ${table} ${primaryKeys}") + val start: Long = System.currentTimeMillis() spark.sql( s""" |CREATE TABLE tpcds_sf1_cluster.$table @@ -51,6 +53,7 @@ class TPCDSClusterSuite extends SparkClickHouseClusterTest { |SELECT * FROM tpcds.sf1.$table; |""".stripMargin ) + println(s"time took table ${table} ${System.currentTimeMillis() - start}") } TPCDSTestUtils.tablePrimaryKeys.keys.foreach { table => diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala new file mode 100644 index 00000000..721ba948 --- /dev/null +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.ClickHouseSingleMixIn +import org.apache.spark.SparkConf + +class ClickHouseSingleArrowWriterSuite extends ClickHouseArrowWriterSuite with ClickHouseSingleMixIn + +abstract class ClickHouseArrowWriterSuite extends ClickHouseWriterTestBase { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.write.format", "arrow") + .set("spark.clickhouse.read.format", "json") + +} diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala new file mode 100644 index 00000000..decfe0af --- /dev/null +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.{ClickHouseCloudMixIn, ClickHouseSingleMixIn} +import org.apache.spark.SparkConf +import org.scalatest.tags.Cloud + +@Cloud +class ClickHouseCloudBinaryReaderSuite extends ClickHouseBinaryReaderSuite with ClickHouseCloudMixIn + +class ClickHouseSingleBinaryReaderSuite extends ClickHouseBinaryReaderSuite with ClickHouseSingleMixIn + +/** + * Test suite for ClickHouse Binary Reader. + * Uses binary format for reading data from ClickHouse. + * All test cases are inherited from ClickHouseReaderTestBase. + */ +abstract class ClickHouseBinaryReaderSuite extends ClickHouseReaderTestBase { + + // Override to use binary format for reading + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.read.format", "binary") + .set("spark.clickhouse.write.format", "arrow") + + // All tests are inherited from ClickHouseReaderTestBase + // Additional binary-specific tests can be added here if needed +} diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala new file mode 100644 index 00000000..c62d5564 --- /dev/null +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.{ClickHouseCloudMixIn, ClickHouseSingleMixIn} +import org.scalatest.tags.Cloud + +@Cloud +class ClickHouseCloudJsonReaderSuite extends ClickHouseJsonReaderSuite with ClickHouseCloudMixIn + +class ClickHouseSingleJsonReaderSuite extends ClickHouseJsonReaderSuite with ClickHouseSingleMixIn + +/** + * Test suite for ClickHouse JSON Reader. + * Uses JSON format for reading data from ClickHouse (default in SparkClickHouseSingleTest). + * All test cases are inherited from ClickHouseReaderTestBase. + */ +abstract class ClickHouseJsonReaderSuite extends ClickHouseReaderTestBase { + // Uses JSON format (configured in SparkClickHouseSingleTest) + // All tests are inherited from ClickHouseReaderTestBase + // Additional JSON-specific tests can be added here if needed +} diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala new file mode 100644 index 00000000..3532b140 --- /dev/null +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.ClickHouseSingleMixIn +import org.apache.spark.SparkConf + +class ClickHouseSingleJsonWriterSuite extends ClickHouseJsonWriterSuite with ClickHouseSingleMixIn + +abstract class ClickHouseJsonWriterSuite extends ClickHouseWriterTestBase { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.write.format", "json") + .set("spark.clickhouse.read.format", "json") + +} diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala new file mode 100644 index 00000000..73e9119f --- /dev/null +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala @@ -0,0 +1,1331 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import org.apache.spark.sql.Row + +/** + * Shared test cases for both JSON and Binary readers. + * Subclasses only need to configure the read format. + * + * Tests are organized by ClickHouse data type with both regular and nullable variants. + * Each type includes comprehensive coverage of edge cases and null handling. + */ +trait ClickHouseReaderTestBase extends SparkClickHouseSingleTest { + + // ============================================================================ + // ArrayType Tests + // ============================================================================ + + test("decode ArrayType - Array of integers") { + withKVTable("test_db", "test_array_int", valueColDef = "Array(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_int VALUES + |(1, [1, 2, 3]), + |(2, []), + |(3, [100, 200, 300, 400]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_int ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(1).getSeq[Int](1) == Seq()) + assert(result(2).getSeq[Int](1) == Seq(100, 200, 300, 400)) + } + } + test("decode ArrayType - Array of strings") { + withKVTable("test_db", "test_array_string", valueColDef = "Array(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_string VALUES + |(1, ['hello', 'world']), + |(2, []), + |(3, ['a', 'b', 'c']) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_string ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[String](1) == Seq("hello", "world")) + assert(result(1).getSeq[String](1) == Seq()) + assert(result(2).getSeq[String](1) == Seq("a", "b", "c")) + } + } + test("decode ArrayType - Array with nullable elements") { + withKVTable("test_db", "test_array_nullable", valueColDef = "Array(Nullable(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_nullable VALUES + |(1, [1, NULL, 3]), + |(2, [NULL, NULL]), + |(3, [100, 200]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_nullable ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify arrays can be read + assert(result(0).getSeq[Any](1) != null) + assert(result(1).getSeq[Any](1) != null) + assert(result(2).getSeq[Any](1) != null) + } + } + test("decode ArrayType - empty arrays") { + withKVTable("test_db", "test_empty_array", valueColDef = "Array(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_empty_array VALUES + |(1, []), + |(2, [1, 2, 3]), + |(3, []) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_empty_array ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1).isEmpty) + assert(result(1).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(2).getSeq[Int](1).isEmpty) + } + } + test("decode ArrayType - Nested arrays") { + withKVTable("test_db", "test_nested_array", valueColDef = "Array(Array(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_nested_array VALUES + |(1, [[1, 2], [3, 4]]), + |(2, [[], [5]]), + |(3, [[10, 20, 30]]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_nested_array ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify nested arrays can be read + assert(result(0).get(1) != null) + assert(result(1).get(1) != null) + assert(result(2).get(1) != null) + } + } + test("decode BinaryType - FixedString") { + // FixedString is read as String by default in the connector + withKVTable("test_db", "test_fixedstring", valueColDef = "FixedString(5)") { + runClickHouseSQL( + """INSERT INTO test_db.test_fixedstring VALUES + |(1, 'hello'), + |(2, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_fixedstring ORDER BY key") + val result = df.collect() + assert(result.length == 2) + // FixedString should be readable + assert(result(0).get(1) != null) + assert(result(1).get(1) != null) + } + } + test("decode BinaryType - FixedString nullable with null values") { + withKVTable("test_db", "test_fixedstring_null", valueColDef = "Nullable(FixedString(5))") { + runClickHouseSQL( + """INSERT INTO test_db.test_fixedstring_null VALUES + |(1, 'hello'), + |(2, NULL), + |(3, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_fixedstring_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).get(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).get(1) != null) + } + } + + // ============================================================================ + // BooleanType Tests + // ============================================================================ + + test("decode BooleanType - true and false values") { + // ClickHouse Bool is stored as UInt8 (0 or 1) + // JSON format reads as Boolean, Binary format reads as Short + withKVTable("test_db", "test_bool", valueColDef = "Bool") { + runClickHouseSQL( + """INSERT INTO test_db.test_bool VALUES + |(1, true), + |(2, false), + |(3, 1), + |(4, 0) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_bool ORDER BY key") + val result = df.collect() + assert(result.length == 4) + // Check the value - handle both Boolean (JSON) and Short (Binary) formats + val v0 = result(0).get(1) + val v1 = result(1).get(1) + v0 match { + case b: Boolean => + assert(b == true) + assert(result(1).getBoolean(1) == false) + assert(result(2).getBoolean(1) == true) + assert(result(3).getBoolean(1) == false) + case s: Short => + assert(s == 1) + assert(result(1).getShort(1) == 0) + assert(result(2).getShort(1) == 1) + assert(result(3).getShort(1) == 0) + case _ => fail(s"Unexpected type: ${v0.getClass}") + } + } + } + test("decode BooleanType - nullable with null values") { + withKVTable("test_db", "test_bool_null", valueColDef = "Nullable(Bool)") { + runClickHouseSQL( + """INSERT INTO test_db.test_bool_null VALUES + |(1, true), + |(2, NULL), + |(3, false) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_bool_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(1).isNullAt(1)) + // Check the value - handle both Boolean (JSON) and Short (Binary) formats + val v0 = result(0).get(1) + v0 match { + case b: Boolean => + assert(b == true) + assert(result(2).getBoolean(1) == false) + case s: Short => + assert(s == 1) + assert(result(2).getShort(1) == 0) + case _ => fail(s"Unexpected type: ${v0.getClass}") + } + } + } + + // ============================================================================ + // ByteType Tests + // ============================================================================ + + test("decode ByteType - min and max values") { + withKVTable("test_db", "test_byte", valueColDef = "Int8") { + runClickHouseSQL( + """INSERT INTO test_db.test_byte VALUES + |(1, -128), + |(2, 0), + |(3, 127) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_byte ORDER BY key") + checkAnswer( + df, + Row(1, -128.toByte) :: Row(2, 0.toByte) :: Row(3, 127.toByte) :: Nil + ) + } + } + test("decode ByteType - nullable with null values") { + withKVTable("test_db", "test_byte_null", valueColDef = "Nullable(Int8)") { + runClickHouseSQL( + """INSERT INTO test_db.test_byte_null VALUES + |(1, -128), + |(2, NULL), + |(3, 127) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_byte_null ORDER BY key") + checkAnswer( + df, + Row(1, -128.toByte) :: Row(2, null) :: Row(3, 127.toByte) :: Nil + ) + } + } + test("decode DateTime32 - 32-bit timestamp") { + withKVTable("test_db", "test_datetime32", valueColDef = "DateTime32") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime32 VALUES + |(1, '2024-01-01 12:00:00'), + |(2, '2024-06-15 18:30:45') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime32 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + } + } + test("decode DateTime32 - nullable with null values") { + withKVTable("test_db", "test_datetime32_null", valueColDef = "Nullable(DateTime32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime32_null VALUES + |(1, '2024-01-01 12:00:00'), + |(2, NULL), + |(3, '2024-06-15 18:30:45') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode DateType - Date") { + withKVTable("test_db", "test_date", valueColDef = "Date") { + runClickHouseSQL( + """INSERT INTO test_db.test_date VALUES + |(1, '2024-01-01'), + |(2, '2024-06-15'), + |(3, '2024-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - Date32") { + withKVTable("test_db", "test_date32", valueColDef = "Date32") { + runClickHouseSQL( + """INSERT INTO test_db.test_date32 VALUES + |(1, '1900-01-01'), + |(2, '2024-06-15'), + |(3, '2100-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date32 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - Date32 nullable with null values") { + withKVTable("test_db", "test_date32_null", valueColDef = "Nullable(Date32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_date32_null VALUES + |(1, '1900-01-01'), + |(2, NULL), + |(3, '2100-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - nullable with null values") { + withKVTable("test_db", "test_date_null", valueColDef = "Nullable(Date)") { + runClickHouseSQL( + """INSERT INTO test_db.test_date_null VALUES + |(1, '2024-01-01'), + |(2, NULL), + |(3, '2024-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + test("decode DecimalType - Decimal128") { + // Decimal128(20) means scale=20, max precision=38 total digits + // Use values with max 18 digits before decimal to stay within 38 total + withKVTable("test_db", "test_decimal128", valueColDef = "Decimal128(20)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal128 VALUES + |(1, 123456789012345.12345678901234567890), + |(2, -999999999999999.99999999999999999999), + |(3, 0.00000000000000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal128 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Decimal128(20) means 20 decimal places, total precision up to 38 digits + assert(math.abs(result(0).getDecimal(1).doubleValue() - 123456789012345.12345678901234567890) < 0.01) + assert(math.abs(result(1).getDecimal(1).doubleValue() - -999999999999999.99999999999999999999) < 0.01) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal128 nullable with null values") { + withKVTable("test_db", "test_decimal128_null", valueColDef = "Nullable(Decimal128(20))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal128_null VALUES + |(1, 123456789012345.12345678901234567890), + |(2, NULL), + |(3, 0.00000000000000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal32") { + withKVTable("test_db", "test_decimal32", valueColDef = "Decimal32(4)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal32 VALUES + |(1, 12345.6789), + |(2, -9999.9999), + |(3, 0.0001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal32 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).doubleValue() == 12345.6789) + assert(result(1).getDecimal(1).doubleValue() == -9999.9999) + assert(result(2).getDecimal(1).doubleValue() == 0.0001) + } + } + test("decode DecimalType - Decimal32 nullable with null values") { + withKVTable("test_db", "test_decimal32_null", valueColDef = "Nullable(Decimal32(4))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal32_null VALUES + |(1, 12345.6789), + |(2, NULL), + |(3, 0.0001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal64") { + // Decimal64(10) means scale=10, max precision=18 total digits + // Use values with max 8 digits before decimal to stay within 18 total + withKVTable("test_db", "test_decimal64", valueColDef = "Decimal64(10)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal64 VALUES + |(1, 1234567.0123456789), + |(2, -9999999.9999999999), + |(3, 0.0000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDecimal(1).doubleValue() - 1234567.0123456789) < 0.0001) + assert(math.abs(result(1).getDecimal(1).doubleValue() - -9999999.9999999999) < 0.0001) + assert(math.abs(result(2).getDecimal(1).doubleValue() - 0.0000000001) < 0.0000000001) + } + } + test("decode DecimalType - Decimal64 nullable with null values") { + withKVTable("test_db", "test_decimal64_null", valueColDef = "Nullable(Decimal64(10))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal64_null VALUES + |(1, 1234567.0123456789), + |(2, NULL), + |(3, 0.0000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DoubleType - nullable with null values") { + withKVTable("test_db", "test_double_null", valueColDef = "Nullable(Float64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_double_null VALUES + |(1, 1.23), + |(2, NULL), + |(3, -4.56) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_double_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 1.23) < 0.0001) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getDouble(1) - -4.56) < 0.0001) + } + } + test("decode DoubleType - regular values") { + withKVTable("test_db", "test_double", valueColDef = "Float64") { + runClickHouseSQL( + """INSERT INTO test_db.test_double VALUES + |(1, -3.141592653589793), + |(2, 0.0), + |(3, 3.141592653589793) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_double ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - -3.141592653589793) < 0.000001) + assert(result(1).getDouble(1) == 0.0) + assert(math.abs(result(2).getDouble(1) - 3.141592653589793) < 0.000001) + } + } + test("decode Enum16 - large enum") { + withKVTable("test_db", "test_enum16", valueColDef = "Enum16('small' = 1, 'medium' = 100, 'large' = 1000)") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum16 VALUES + |(1, 'small'), + |(2, 'medium'), + |(3, 'large') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum16 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "small") + assert(result(1).getString(1) == "medium") + assert(result(2).getString(1) == "large") + } + } + test("decode Enum16 - nullable with null values") { + withKVTable( + "test_db", + "test_enum16_null", + valueColDef = "Nullable(Enum16('small' = 1, 'medium' = 100, 'large' = 1000))" + ) { + runClickHouseSQL( + """INSERT INTO test_db.test_enum16_null VALUES + |(1, 'small'), + |(2, NULL), + |(3, 'large') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum16_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "small") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "large") + } + } + test("decode Enum8 - nullable with null values") { + withKVTable("test_db", "test_enum8_null", valueColDef = "Nullable(Enum8('red' = 1, 'green' = 2, 'blue' = 3))") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum8_null VALUES + |(1, 'red'), + |(2, NULL), + |(3, 'blue') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum8_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "red") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "blue") + } + } + test("decode Enum8 - small enum") { + withKVTable("test_db", "test_enum8", valueColDef = "Enum8('red' = 1, 'green' = 2, 'blue' = 3)") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum8 VALUES + |(1, 'red'), + |(2, 'green'), + |(3, 'blue') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum8 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "red") + assert(result(1).getString(1) == "green") + assert(result(2).getString(1) == "blue") + } + } + test("decode FloatType - nullable with null values") { + withKVTable("test_db", "test_float_null", valueColDef = "Nullable(Float32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_float_null VALUES + |(1, 1.5), + |(2, NULL), + |(3, -2.5) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_float_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 1.5f) < 0.01f) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getFloat(1) - -2.5f) < 0.01f) + } + } + test("decode FloatType - regular values") { + withKVTable("test_db", "test_float", valueColDef = "Float32") { + runClickHouseSQL( + """INSERT INTO test_db.test_float VALUES + |(1, -3.14), + |(2, 0.0), + |(3, 3.14) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_float ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - -3.14f) < 0.01f) + assert(result(1).getFloat(1) == 0.0f) + assert(math.abs(result(2).getFloat(1) - 3.14f) < 0.01f) + } + } + test("decode Int128 - large integers as Decimal") { + withKVTable("test_db", "test_int128", valueColDef = "Int128") { + runClickHouseSQL( + """INSERT INTO test_db.test_int128 VALUES + |(1, 0), + |(2, 123456789012345678901234567890), + |(3, -123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int128 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int128 - nullable with null values") { + withKVTable("test_db", "test_int128_null", valueColDef = "Nullable(Int128)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int128_null VALUES + |(1, 0), + |(2, NULL), + |(3, -123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int256 - nullable with null values") { + withKVTable("test_db", "test_int256_null", valueColDef = "Nullable(Int256)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int256_null VALUES + |(1, 0), + |(2, NULL), + |(3, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int256_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int256 - very large integers as Decimal") { + withKVTable("test_db", "test_int256", valueColDef = "Int256") { + runClickHouseSQL( + """INSERT INTO test_db.test_int256 VALUES + |(1, 0), + |(2, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int256 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode IntegerType - min and max values") { + withKVTable("test_db", "test_int", valueColDef = "Int32") { + runClickHouseSQL( + """INSERT INTO test_db.test_int VALUES + |(1, -2147483648), + |(2, 0), + |(3, 2147483647) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int ORDER BY key") + checkAnswer( + df, + Row(1, -2147483648) :: Row(2, 0) :: Row(3, 2147483647) :: Nil + ) + } + } + test("decode IntegerType - nullable with null values") { + withKVTable("test_db", "test_int_null", valueColDef = "Nullable(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int_null VALUES + |(1, -2147483648), + |(2, NULL), + |(3, 2147483647) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int_null ORDER BY key") + checkAnswer( + df, + Row(1, -2147483648) :: Row(2, null) :: Row(3, 2147483647) :: Nil + ) + } + } + test("decode IPv4 - IP addresses") { + withKVTable("test_db", "test_ipv4", valueColDef = "IPv4") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv4 VALUES + |(1, '127.0.0.1'), + |(2, '192.168.1.1'), + |(3, '8.8.8.8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv4 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "127.0.0.1") + assert(result(1).getString(1) == "192.168.1.1") + assert(result(2).getString(1) == "8.8.8.8") + } + } + test("decode IPv4 - nullable with null values") { + withKVTable("test_db", "test_ipv4_null", valueColDef = "Nullable(IPv4)") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv4_null VALUES + |(1, '127.0.0.1'), + |(2, NULL), + |(3, '8.8.8.8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv4_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "127.0.0.1") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "8.8.8.8") + } + } + test("decode IPv6 - IPv6 addresses") { + withKVTable("test_db", "test_ipv6", valueColDef = "IPv6") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv6 VALUES + |(1, '::1'), + |(2, '2001:0db8:85a3:0000:0000:8a2e:0370:7334'), + |(3, 'fe80::1') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv6 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) != null) + assert(result(1).getString(1) != null) + assert(result(2).getString(1) != null) + } + } + test("decode IPv6 - nullable with null values") { + withKVTable("test_db", "test_ipv6_null", valueColDef = "Nullable(IPv6)") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv6_null VALUES + |(1, '::1'), + |(2, NULL), + |(3, 'fe80::1') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv6_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) != null) + } + } + test("decode JSON - nullable with null values") { + withKVTable("test_db", "test_json_null", valueColDef = "Nullable(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_json_null VALUES + |(1, '{"name": "Alice", "age": 30}'), + |(2, NULL), + |(3, '{"name": "Charlie", "age": 35}') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_json_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1).contains("Alice")) + assert(result(1).isNullAt(1)) + assert(result(2).getString(1).contains("Charlie")) + } + } + test("decode JSON - semi-structured data") { + withKVTable("test_db", "test_json", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_json VALUES + |(1, '{"name": "Alice", "age": 30}'), + |(2, '{"name": "Bob", "age": 25}'), + |(3, '{"name": "Charlie", "age": 35}') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_json ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1).contains("Alice")) + assert(result(1).getString(1).contains("Bob")) + assert(result(2).getString(1).contains("Charlie")) + } + } + test("decode LongType - min and max values") { + withKVTable("test_db", "test_long", valueColDef = "Int64") { + runClickHouseSQL( + """INSERT INTO test_db.test_long VALUES + |(1, -9223372036854775808), + |(2, 0), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long ORDER BY key") + checkAnswer( + df, + Row(1, -9223372036854775808L) :: Row(2, 0L) :: Row(3, 9223372036854775807L) :: Nil + ) + } + } + test("decode LongType - nullable with null values") { + withKVTable("test_db", "test_long_null", valueColDef = "Nullable(Int64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_long_null VALUES + |(1, -9223372036854775808), + |(2, NULL), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long_null ORDER BY key") + checkAnswer( + df, + Row(1, -9223372036854775808L) :: Row(2, null) :: Row(3, 9223372036854775807L) :: Nil + ) + } + } + test("decode LongType - UInt32 nullable with null values") { + withKVTable("test_db", "test_uint32_null", valueColDef = "Nullable(UInt32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint32_null VALUES + |(1, 0), + |(2, NULL), + |(3, 4294967295) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint32_null ORDER BY key") + checkAnswer( + df, + Row(1, 0L) :: Row(2, null) :: Row(3, 4294967295L) :: Nil + ) + } + } + test("decode LongType - UInt32 values") { + withKVTable("test_db", "test_uint32", valueColDef = "UInt32") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint32 VALUES + |(1, 0), + |(2, 2147483648), + |(3, 4294967295) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint32 ORDER BY key") + checkAnswer( + df, + Row(1, 0L) :: Row(2, 2147483648L) :: Row(3, 4294967295L) :: Nil + ) + } + } + test("decode MapType - Map of String to Int") { + withKVTable("test_db", "test_map", valueColDef = "Map(String, Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_map VALUES + |(1, {'a': 1, 'b': 2}), + |(2, {}), + |(3, {'x': 100, 'y': 200, 'z': 300}) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_map ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1) == Map("a" -> 1, "b" -> 2)) + assert(result(1).getMap[String, Int](1) == Map()) + assert(result(2).getMap[String, Int](1) == Map("x" -> 100, "y" -> 200, "z" -> 300)) + } + } + test("decode MapType - Map with nullable values") { + withKVTable("test_db", "test_map_nullable", valueColDef = "Map(String, Nullable(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_map_nullable VALUES + |(1, {'a': 1, 'b': NULL}), + |(2, {'x': NULL}), + |(3, {'p': 100, 'q': 200}) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_map_nullable ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify maps can be read + assert(result(0).getMap[String, Any](1) != null) + assert(result(1).getMap[String, Any](1) != null) + assert(result(2).getMap[String, Any](1) != null) + } + } + test("decode ShortType - min and max values") { + withKVTable("test_db", "test_short", valueColDef = "Int16") { + runClickHouseSQL( + """INSERT INTO test_db.test_short VALUES + |(1, -32768), + |(2, 0), + |(3, 32767) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_short ORDER BY key") + checkAnswer( + df, + Row(1, -32768.toShort) :: Row(2, 0.toShort) :: Row(3, 32767.toShort) :: Nil + ) + } + } + test("decode ShortType - nullable with null values") { + withKVTable("test_db", "test_short_null", valueColDef = "Nullable(Int16)") { + runClickHouseSQL( + """INSERT INTO test_db.test_short_null VALUES + |(1, -32768), + |(2, NULL), + |(3, 32767) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_short_null ORDER BY key") + checkAnswer( + df, + Row(1, -32768.toShort) :: Row(2, null) :: Row(3, 32767.toShort) :: Nil + ) + } + } + test("decode ShortType - UInt8 nullable with null values") { + withKVTable("test_db", "test_uint8_null", valueColDef = "Nullable(UInt8)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint8_null VALUES + |(1, 0), + |(2, NULL), + |(3, 255) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint8_null ORDER BY key") + checkAnswer( + df, + Row(1, 0.toShort) :: Row(2, null) :: Row(3, 255.toShort) :: Nil + ) + } + } + test("decode ShortType - UInt8 values") { + withKVTable("test_db", "test_uint8", valueColDef = "UInt8") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint8 VALUES + |(1, 0), + |(2, 128), + |(3, 255) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint8 ORDER BY key") + checkAnswer( + df, + Row(1, 0.toShort) :: Row(2, 128.toShort) :: Row(3, 255.toShort) :: Nil + ) + } + } + test("decode StringType - empty strings") { + withKVTable("test_db", "test_empty_string", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_empty_string VALUES + |(1, ''), + |(2, 'not empty'), + |(3, '') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_empty_string ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "") + assert(result(1).getString(1) == "not empty") + assert(result(2).getString(1) == "") + } + } + test("decode StringType - nullable with null values") { + withKVTable("test_db", "test_string_null", valueColDef = "Nullable(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_string_null VALUES + |(1, 'hello'), + |(2, NULL), + |(3, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_string_null ORDER BY key") + checkAnswer( + df, + Row(1, "hello") :: Row(2, null) :: Row(3, "world") :: Nil + ) + } + } + test("decode StringType - regular strings") { + withKVTable("test_db", "test_string", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_string VALUES + |(1, 'hello'), + |(2, ''), + |(3, 'world with spaces'), + |(4, 'special chars: !@#$%^&*()') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_string ORDER BY key") + checkAnswer( + df, + Row(1, "hello") :: Row(2, "") :: Row(3, "world with spaces") :: Row(4, "special chars: !@#$%^&*()") :: Nil + ) + } + } + test("decode StringType - UUID") { + withKVTable("test_db", "test_uuid", valueColDef = "UUID") { + runClickHouseSQL( + """INSERT INTO test_db.test_uuid VALUES + |(1, '550e8400-e29b-41d4-a716-446655440000'), + |(2, '6ba7b810-9dad-11d1-80b4-00c04fd430c8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uuid ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getString(1) == "550e8400-e29b-41d4-a716-446655440000") + assert(result(1).getString(1) == "6ba7b810-9dad-11d1-80b4-00c04fd430c8") + } + } + test("decode StringType - UUID nullable with null values") { + withKVTable("test_db", "test_uuid_null", valueColDef = "Nullable(UUID)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uuid_null VALUES + |(1, '550e8400-e29b-41d4-a716-446655440000'), + |(2, NULL), + |(3, '6ba7b810-9dad-11d1-80b4-00c04fd430c8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uuid_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "550e8400-e29b-41d4-a716-446655440000") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "6ba7b810-9dad-11d1-80b4-00c04fd430c8") + } + } + test("decode StringType - very long strings") { + val longString = "a" * 10000 + withKVTable("test_db", "test_long_string", valueColDef = "String") { + runClickHouseSQL( + s"""INSERT INTO test_db.test_long_string VALUES + |(1, '$longString') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long_string ORDER BY key") + val result = df.collect() + assert(result.length == 1) + assert(result(0).getString(1).length == 10000) + } + } + test("decode TimestampType - DateTime") { + withKVTable("test_db", "test_datetime", valueColDef = "DateTime") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime VALUES + |(1, '2024-01-01 00:00:00'), + |(2, '2024-06-15 12:30:45'), + |(3, '2024-12-31 23:59:59') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - DateTime64") { + withKVTable("test_db", "test_datetime64", valueColDef = "DateTime64(3)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime64 VALUES + |(1, '2024-01-01 00:00:00.123'), + |(2, '2024-06-15 12:30:45.456'), + |(3, '2024-12-31 23:59:59.999') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - DateTime64 nullable with null values") { + withKVTable("test_db", "test_datetime64_null", valueColDef = "Nullable(DateTime64(3))") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime64_null VALUES + |(1, '2024-01-01 00:00:00.123'), + |(2, NULL), + |(3, '2024-12-31 23:59:59.999') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - nullable with null values") { + withKVTable("test_db", "test_datetime_null", valueColDef = "Nullable(DateTime)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime_null VALUES + |(1, '2024-01-01 00:00:00'), + |(2, NULL), + |(3, '2024-12-31 23:59:59') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode UInt128 - large unsigned integers as Decimal") { + withKVTable("test_db", "test_uint128", valueColDef = "UInt128") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint128 VALUES + |(1, 0), + |(2, 123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint128 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode UInt128 - nullable with null values") { + withKVTable("test_db", "test_uint128_null", valueColDef = "Nullable(UInt128)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint128_null VALUES + |(1, 0), + |(2, NULL), + |(3, 123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode UInt16 - nullable with null values") { + withKVTable("test_db", "test_uint16_null", valueColDef = "Nullable(UInt16)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint16_null VALUES + |(1, 0), + |(2, NULL), + |(3, 65535) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint16_null ORDER BY key") + checkAnswer( + df, + Row(1, 0) :: Row(2, null) :: Row(3, 65535) :: Nil + ) + } + } + test("decode UInt16 - unsigned 16-bit integers") { + withKVTable("test_db", "test_uint16", valueColDef = "UInt16") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint16 VALUES + |(1, 0), + |(2, 32768), + |(3, 65535) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint16 ORDER BY key") + checkAnswer( + df, + Row(1, 0) :: Row(2, 32768) :: Row(3, 65535) :: Nil + ) + } + } + test("decode UInt256 - nullable with null values") { + withKVTable("test_db", "test_uint256_null", valueColDef = "Nullable(UInt256)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint256_null VALUES + |(1, 0), + |(2, NULL), + |(3, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint256_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode UInt256 - very large unsigned integers as Decimal") { + withKVTable("test_db", "test_uint256", valueColDef = "UInt256") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint256 VALUES + |(1, 0), + |(2, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint256 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode UInt64 - nullable with null values") { + withKVTable("test_db", "test_uint64_null", valueColDef = "Nullable(UInt64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint64_null VALUES + |(1, 0), + |(2, NULL), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getLong(1) == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getLong(1) == 9223372036854775807L) + } + } + test("decode UInt64 - unsigned 64-bit integers") { + withKVTable("test_db", "test_uint64", valueColDef = "UInt64") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint64 VALUES + |(1, 0), + |(2, 1234567890), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getLong(1) == 0L) + assert(result(1).getLong(1) == 1234567890L) + // Max value that fits in signed Long + assert(result(2).getLong(1) == 9223372036854775807L) + } + } + +} diff --git a/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala new file mode 100644 index 00000000..28267dc2 --- /dev/null +++ b/spark-3.3/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala @@ -0,0 +1,758 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.types._ + +/** + * Shared test cases for both JSON and Binary writers. + * Subclasses only need to configure the write format. + */ +trait ClickHouseWriterTestBase extends SparkClickHouseSingleTest { + + test("write ArrayType - array of integers") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_array_int", schema) { + val data = Seq( + Row(1, Seq(1, 2, 3)), + Row(2, Seq(10, 20, 30)), + Row(3, Seq(100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_array_int") + + val result = spark.table("test_db.test_write_array_int").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(1).getSeq[Int](1) == Seq(10, 20, 30)) + assert(result(2).getSeq[Int](1) == Seq(100)) + } + } + + test("write ArrayType - empty arrays") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_empty_array", schema) { + val data = Seq( + Row(1, Seq()), + Row(2, Seq(1, 2, 3)), + Row(3, Seq()) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_array") + + val result = spark.table("test_db.test_write_empty_array").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1).isEmpty) + assert(result(1).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(2).getSeq[Int](1).isEmpty) + } + } + + test("write ArrayType - nested arrays") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField( + "value", + ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = false), + nullable = false + ) + )) + + withTable("test_db", "test_write_nested_array", schema) { + val data = Seq( + Row(1, Seq(Seq(1, 2), Seq(3, 4))), + Row(2, Seq(Seq(10, 20, 30))), + Row(3, Seq(Seq(), Seq(100))) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_nested_array") + + val result = spark.table("test_db.test_write_nested_array").orderBy("id").collect() + assert(result.length == 3) + // Convert to List for Scala 2.12/2.13 compatibility + val row0 = result(0).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + val row1 = result(1).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + val row2 = result(2).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + assert(row0 == Seq(Seq(1, 2), Seq(3, 4))) + assert(row1 == Seq(Seq(10, 20, 30))) + assert(row2(0).isEmpty) + assert(row2(1) == Seq(100)) + } + } + + test("write ArrayType - with nullable elements") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = true), nullable = false) + )) + + withTable("test_db", "test_write_array_nullable", schema) { + val data = Seq( + Row(1, Seq(1, null, 3)), + Row(2, Seq(null, null)), + Row(3, Seq(10, 20, 30)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_array_nullable") + + val result = spark.table("test_db.test_write_array_nullable").orderBy("id").collect() + assert(result.length == 3) + val arr1 = result(0).getSeq[Any](1) + assert(arr1.length == 3) + assert(arr1(0) == 1) + assert(arr1(1) == null) + assert(arr1(2) == 3) + } + } + + test("write BooleanType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", BooleanType, nullable = true) + )) + + withTable("test_db", "test_write_bool_null", schema) { + val data = Seq( + Row(1, true), + Row(2, null), + Row(3, false) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_bool_null") + + val result = spark.table("test_db.test_write_bool_null").orderBy("id").collect() + assert(result.length == 3) + // Boolean is stored as UInt8 in ClickHouse, reads back as Short + assert(result(0).getShort(1) == 1) + assert(result(1).isNullAt(1)) + assert(result(2).getShort(1) == 0) + } + } + + // NOTE: ClickHouse stores Boolean as UInt8, so it reads back as Short (0 or 1) + test("write BooleanType - true and false values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", BooleanType, nullable = false) + )) + + withTable("test_db", "test_write_bool", schema) { + val data = Seq( + Row(1, true), + Row(2, false) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_bool") + + val result = spark.table("test_db.test_write_bool").orderBy("id").collect() + assert(result.length == 2) + // Boolean is stored as UInt8 in ClickHouse, reads back as Short + assert(result(0).getShort(1) == 1) + assert(result(1).getShort(1) == 0) + } + } + + test("write ByteType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ByteType, nullable = false) + )) + + withTable("test_db", "test_write_byte", schema) { + val data = Seq( + Row(1, Byte.MinValue), + Row(2, 0.toByte), + Row(3, Byte.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_byte") + + val result = spark.table("test_db.test_write_byte").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getByte(1) == Byte.MinValue) + assert(result(1).getByte(1) == 0.toByte) + assert(result(2).getByte(1) == Byte.MaxValue) + } + } + + test("write ByteType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ByteType, nullable = true) + )) + + withTable("test_db", "test_write_byte_null", schema) { + val data = Seq( + Row(1, Byte.MinValue), + Row(2, null), + Row(3, Byte.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_byte_null") + + val result = spark.table("test_db.test_write_byte_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getByte(1) == Byte.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getByte(1) == Byte.MaxValue) + } + } + + test("write DateType - dates") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DateType, nullable = false) + )) + + withTable("test_db", "test_write_date", schema) { + val data = Seq( + Row(1, java.sql.Date.valueOf("2024-01-01")), + Row(2, java.sql.Date.valueOf("2024-06-15")), + Row(3, java.sql.Date.valueOf("2024-12-31")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_date") + + val result = spark.table("test_db.test_write_date").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + + test("write DateType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DateType, nullable = true) + )) + + withTable("test_db", "test_write_date_null", schema) { + val data = Seq( + Row(1, java.sql.Date.valueOf("2024-01-01")), + Row(2, null), + Row(3, java.sql.Date.valueOf("2024-12-31")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_date_null") + + val result = spark.table("test_db.test_write_date_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + + test("write DecimalType - Decimal(10,2)") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(10, 2), nullable = false) + )) + + withTable("test_db", "test_write_decimal", schema) { + val data = Seq( + Row(1, BigDecimal("12345.67")), + Row(2, BigDecimal("-9999.99")), + Row(3, BigDecimal("0.01")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal") + + val result = spark.table("test_db.test_write_decimal").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) == BigDecimal("12345.67").underlying()) + assert(result(1).getDecimal(1) == BigDecimal("-9999.99").underlying()) + assert(result(2).getDecimal(1) == BigDecimal("0.01").underlying()) + } + } + + test("write DecimalType - Decimal(18,4)") { + // Note: High-precision decimals (>15-17 significant digits) may lose precision in JSON/Arrow formats. + // This appears to be related to the serialization/deserialization path, possibly due to intermediate + // double conversions in the format parsers. This test uses tolerance-based assertions to account + // for this observed behavior. Binary format (RowBinaryWithNamesAndTypes) preserves full precision. + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(18, 4), nullable = false) + )) + + withTable("test_db", "test_write_decimal_18_4", schema) { + val data = Seq( + Row(1, BigDecimal("12345678901234.5678")), + Row(2, BigDecimal("-9999999999999.9999")), + Row(3, BigDecimal("0.0001")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal_18_4") + + val result = spark.table("test_db.test_write_decimal_18_4").orderBy("id").collect() + assert(result.length == 3) + // Use tolerance for high-precision values (18 significant digits) + val tolerance = BigDecimal("0.001") + assert((BigDecimal(result(0).getDecimal(1)) - BigDecimal("12345678901234.5678")).abs < tolerance) + assert((BigDecimal(result(1).getDecimal(1)) - BigDecimal("-9999999999999.9999")).abs < tolerance) + // Small values should be exact + assert(result(2).getDecimal(1) == BigDecimal("0.0001").underlying()) + } + } + + test("write DecimalType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(10, 2), nullable = true) + )) + + withTable("test_db", "test_write_decimal_null", schema) { + val data = Seq( + Row(1, BigDecimal("12345.67")), + Row(2, null), + Row(3, BigDecimal("-9999.99")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal_null") + + val result = spark.table("test_db.test_write_decimal_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) == BigDecimal("12345.67").underlying()) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) == BigDecimal("-9999.99").underlying()) + } + } + + test("write DoubleType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DoubleType, nullable = true) + )) + + withTable("test_db", "test_write_double_null", schema) { + val data = Seq( + Row(1, 3.14159), + Row(2, null), + Row(3, -2.71828) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_double_null") + + val result = spark.table("test_db.test_write_double_null").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 3.14159) < 0.00001) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getDouble(1) - -2.71828) < 0.00001) + } + } + + test("write DoubleType - regular values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DoubleType, nullable = false) + )) + + withTable("test_db", "test_write_double", schema) { + val data = Seq( + Row(1, 3.14159), + Row(2, -2.71828), + Row(3, 0.0) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_double") + + val result = spark.table("test_db.test_write_double").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 3.14159) < 0.00001) + assert(math.abs(result(1).getDouble(1) - -2.71828) < 0.00001) + assert(result(2).getDouble(1) == 0.0) + } + } + + test("write FloatType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", FloatType, nullable = true) + )) + + withTable("test_db", "test_write_float_null", schema) { + val data = Seq( + Row(1, 3.14f), + Row(2, null), + Row(3, -2.718f) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_float_null") + + val result = spark.table("test_db.test_write_float_null").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 3.14f) < 0.001f) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getFloat(1) - -2.718f) < 0.001f) + } + } + + test("write FloatType - regular values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", FloatType, nullable = false) + )) + + withTable("test_db", "test_write_float", schema) { + val data = Seq( + Row(1, 3.14f), + Row(2, -2.718f), + Row(3, 0.0f) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_float") + + val result = spark.table("test_db.test_write_float").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 3.14f) < 0.001f) + assert(math.abs(result(1).getFloat(1) - -2.718f) < 0.001f) + assert(result(2).getFloat(1) == 0.0f) + } + } + + test("write IntegerType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", IntegerType, nullable = false) + )) + + withTable("test_db", "test_write_int", schema) { + val data = Seq( + Row(1, Int.MinValue), + Row(2, 0), + Row(3, Int.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_int") + + val result = spark.table("test_db.test_write_int").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getInt(1) == Int.MinValue) + assert(result(1).getInt(1) == 0) + assert(result(2).getInt(1) == Int.MaxValue) + } + } + + test("write IntegerType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", IntegerType, nullable = true) + )) + + withTable("test_db", "test_write_int_null", schema) { + val data = Seq( + Row(1, Int.MinValue), + Row(2, null), + Row(3, Int.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_int_null") + + val result = spark.table("test_db.test_write_int_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getInt(1) == Int.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getInt(1) == Int.MaxValue) + } + } + + test("write LongType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", LongType, nullable = false) + )) + + withTable("test_db", "test_write_long", schema) { + val data = Seq( + Row(1, Long.MinValue), + Row(2, 0L), + Row(3, Long.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_long") + + val result = spark.table("test_db.test_write_long").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getLong(1) == Long.MinValue) + assert(result(1).getLong(1) == 0L) + assert(result(2).getLong(1) == Long.MaxValue) + } + } + + test("write LongType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", LongType, nullable = true) + )) + + withTable("test_db", "test_write_long_null", schema) { + val data = Seq( + Row(1, Long.MinValue), + Row(2, null), + Row(3, Long.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_long_null") + + val result = spark.table("test_db.test_write_long_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getLong(1) == Long.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getLong(1) == Long.MaxValue) + } + } + + test("write MapType - empty maps") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_empty_map", schema) { + val data = Seq( + Row(1, Map[String, Int]()), + Row(2, Map("a" -> 1)), + Row(3, Map[String, Int]()) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_map") + + val result = spark.table("test_db.test_write_empty_map").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1).isEmpty) + assert(result(1).getMap[String, Int](1) == Map("a" -> 1)) + assert(result(2).getMap[String, Int](1).isEmpty) + } + } + + test("write MapType - map of string to int") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_map", schema) { + val data = Seq( + Row(1, Map("a" -> 1, "b" -> 2)), + Row(2, Map("x" -> 10, "y" -> 20)), + Row(3, Map("foo" -> 100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_map") + + val result = spark.table("test_db.test_write_map").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1) == Map("a" -> 1, "b" -> 2)) + assert(result(1).getMap[String, Int](1) == Map("x" -> 10, "y" -> 20)) + assert(result(2).getMap[String, Int](1) == Map("foo" -> 100)) + } + } + + test("write MapType - with nullable values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = true), nullable = false) + )) + + withTable("test_db", "test_write_map_nullable", schema) { + val data = Seq( + Row(1, Map("a" -> 1, "b" -> null)), + Row(2, Map("x" -> null, "y" -> 20)), + Row(3, Map("foo" -> 100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_map_nullable") + + val result = spark.table("test_db.test_write_map_nullable").orderBy("id").collect() + assert(result.length == 3) + val map1 = result(0).getMap[String, Any](1) + assert(map1("a") == 1) + assert(map1("b") == null) + } + } + + test("write ShortType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ShortType, nullable = false) + )) + + withTable("test_db", "test_write_short", schema) { + val data = Seq( + Row(1, Short.MinValue), + Row(2, 0.toShort), + Row(3, Short.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_short") + + val result = spark.table("test_db.test_write_short").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getShort(1) == Short.MinValue) + assert(result(1).getShort(1) == 0.toShort) + assert(result(2).getShort(1) == Short.MaxValue) + } + } + + test("write ShortType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ShortType, nullable = true) + )) + + withTable("test_db", "test_write_short_null", schema) { + val data = Seq( + Row(1, Short.MinValue), + Row(2, null), + Row(3, Short.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_short_null") + + val result = spark.table("test_db.test_write_short_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getShort(1) == Short.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getShort(1) == Short.MaxValue) + } + } + + test("write StringType - empty strings") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = false) + )) + + withTable("test_db", "test_write_empty_string", schema) { + val data = Seq( + Row(1, ""), + Row(2, "not empty"), + Row(3, "") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_string") + + val result = spark.table("test_db.test_write_empty_string").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "") + assert(result(1).getString(1) == "not empty") + assert(result(2).getString(1) == "") + } + } + + test("write StringType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = true) + )) + + withTable("test_db", "test_write_string_null", schema) { + val data = Seq( + Row(1, "hello"), + Row(2, null), + Row(3, "world") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_string_null") + + val result = spark.table("test_db.test_write_string_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "hello") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "world") + } + } + + test("write StringType - regular strings") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = false) + )) + + withTable("test_db", "test_write_string", schema) { + val data = Seq( + Row(1, "hello"), + Row(2, "world"), + Row(3, "test") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_string") + + val result = spark.table("test_db.test_write_string").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "hello") + assert(result(1).getString(1) == "world") + assert(result(2).getString(1) == "test") + } + } + + test("write TimestampType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", TimestampType, nullable = true) + )) + + withTable("test_db", "test_write_timestamp_null", schema) { + val data = Seq( + Row(1, java.sql.Timestamp.valueOf("2024-01-01 12:00:00")), + Row(2, null), + Row(3, java.sql.Timestamp.valueOf("2024-12-31 23:59:59")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_timestamp_null") + + val result = spark.table("test_db.test_write_timestamp_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + + test("write TimestampType - timestamps") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", TimestampType, nullable = false) + )) + + withTable("test_db", "test_write_timestamp", schema) { + val data = Seq( + Row(1, java.sql.Timestamp.valueOf("2024-01-01 12:00:00")), + Row(2, java.sql.Timestamp.valueOf("2024-06-15 18:30:45")), + Row(3, java.sql.Timestamp.valueOf("2024-12-31 23:59:59")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_timestamp") + + val result = spark.table("test_db.test_write_timestamp").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + +} diff --git a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala index 4c6d71f9..1e680e9c 100644 --- a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala +++ b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala @@ -14,17 +14,18 @@ package com.clickhouse.spark.read -import com.clickhouse.client.ClickHouseResponse +import com.clickhouse.spark.{ClickHouseHelper, Logging, TaskMetric} +import com.clickhouse.spark.client.{NodeClient, NodesClient} import com.clickhouse.data.ClickHouseCompression +import com.clickhouse.spark.format.StreamOutput import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} import org.apache.spark.sql.clickhouse.ClickHouseSQLConf._ import org.apache.spark.sql.connector.metric.CustomTaskMetric import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.types._ import com.clickhouse.spark.Metrics.{BLOCKS_READ, BYTES_READ} -import com.clickhouse.spark.client.{NodeClient, NodesClient} -import com.clickhouse.spark.format.StreamOutput -import com.clickhouse.spark.{ClickHouseHelper, Logging, TaskMetric} +import com.clickhouse.client.ClickHouseResponse +import com.clickhouse.client.api.query.QueryResponse abstract class ClickHouseReader[Record]( scanJob: ScanJobDescription, @@ -40,7 +41,7 @@ abstract class ClickHouseReader[Record]( val database: String = part.table.database val table: String = part.table.name - val codec: ClickHouseCompression = scanJob.readOptions.compressionCodec +// val codec: ClickHouseCompression = scanJob.readOptions.compressionCodec val readSchema: StructType = scanJob.readSchema private lazy val nodesClient = NodesClient(part.candidateNodes) @@ -67,11 +68,12 @@ abstract class ClickHouseReader[Record]( def format: String - lazy val resp: ClickHouseResponse = nodeClient.queryAndCheck(scanQuery, format, codec) + // , codec + lazy val resp: QueryResponse = nodeClient.queryAndCheck(scanQuery, format) - def totalBlocksRead: Long = resp.getSummary.getStatistics.getBlocks + def totalBlocksRead: Long = 0L // resp.getSummary.getStatistics.getBlocks - def totalBytesRead: Long = resp.getSummary.getReadBytes + def totalBytesRead: Long = resp.getReadBytes // resp.getSummary.getReadBytes override def currentMetricsValues: Array[CustomTaskMetric] = Array( TaskMetric(BLOCKS_READ, totalBlocksRead), diff --git a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala index 5e90c4eb..dd1f9127 100644 --- a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala +++ b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala @@ -14,6 +14,10 @@ package com.clickhouse.spark.read.format +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader +import com.clickhouse.client.api.data_formats.{ClickHouseBinaryFormatReader, RowBinaryWithNamesAndTypesFormatReader} +import com.clickhouse.client.api.query.{GenericRecord, Records} + import java.util.Collections import com.clickhouse.data.value.{ ClickHouseArrayValue, @@ -34,120 +38,118 @@ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import java.time.ZoneOffset +import java.io.InputStream +import java.time.{LocalDate, ZoneOffset, ZonedDateTime} +import java.util import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ class ClickHouseBinaryReader( scanJob: ScanJobDescription, part: ClickHouseInputPartition -) extends ClickHouseReader[ClickHouseRecord](scanJob, part) { +) extends ClickHouseReader[GenericRecord](scanJob, part) { override val format: String = "RowBinaryWithNamesAndTypes" - lazy val streamOutput: Iterator[ClickHouseRecord] = resp.records().asScala.iterator + lazy val streamOutput: Iterator[GenericRecord] = { + val inputString: InputStream = resp.getInputStream + val cbfr: ClickHouseBinaryFormatReader = new RowBinaryWithNamesAndTypesFormatReader( + inputString, + resp.getSettings, + new BinaryStreamReader.DefaultByteBufferAllocator + ) + val r = new Records(resp, cbfr) + r.asScala.iterator + } - override def decode(record: ClickHouseRecord): InternalRow = { - val values: Array[Any] = new Array[Any](record.size) + override def decode(record: GenericRecord): InternalRow = { + val size = record.getSchema.getColumns.size() + val values: Array[Any] = new Array[Any](size) if (readSchema.nonEmpty) { var i: Int = 0 - while (i < record.size) { - values(i) = decodeValue(record.getValue(i), readSchema.fields(i)) + while (i < size) { + val v: Object = record.getObject(i + 1) + values(i) = decodeValue(v, readSchema.fields(i)) i = i + 1 } } new GenericInternalRow(values) } - private def decodeValue(value: ClickHouseValue, structField: StructField): Any = { - if (value == null || value.isNullOrEmpty && value.isNullable) { + private def decodeValue(value: Object, structField: StructField): Any = { + if (value == null) { // should we check `structField.nullable`? return null } structField.dataType match { - case BooleanType => value.asBoolean - case ByteType => value.asByte - case ShortType => value.asShort - case IntegerType => value.asInteger - case LongType => value.asLong - case FloatType => value.asFloat - case DoubleType => value.asDouble - case d: DecimalType => Decimal(value.asBigDecimal(d.scale)) + case BooleanType => value.asInstanceOf[Boolean] + case ByteType => value.asInstanceOf[Byte] + case ShortType => value.asInstanceOf[Short] +// case IntegerType if value.getClass.toString.equals("class java.lang.Long") => + case IntegerType if value.isInstanceOf[java.lang.Long] => + val v: Integer = Integer.valueOf(value.asInstanceOf[Long].toInt) + v.intValue() + case IntegerType => + value.asInstanceOf[Integer].intValue() + case LongType if value.isInstanceOf[java.math.BigInteger] => + value.asInstanceOf[java.math.BigInteger].longValue() + case LongType => + value.asInstanceOf[Long] + case FloatType => value.asInstanceOf[Float] + case DoubleType => value.asInstanceOf[Double] + case d: DecimalType => + // Java client returns BigInteger for Int256/UInt256, BigDecimal for Decimal types + val dec: BigDecimal = value match { + case bi: java.math.BigInteger => BigDecimal(bi) + case bd: java.math.BigDecimal => BigDecimal(bd) + } + Decimal(dec.setScale(d.scale)) case TimestampType => - var _instant = value.asZonedDateTime.withZoneSameInstant(ZoneOffset.UTC) + var _instant = value.asInstanceOf[ZonedDateTime].withZoneSameInstant(ZoneOffset.UTC) TimeUnit.SECONDS.toMicros(_instant.toEpochSecond) + TimeUnit.NANOSECONDS.toMicros(_instant.getNano()) - case StringType if value.isInstanceOf[ClickHouseStringValue] => UTF8String.fromBytes(value.asBinary) - case StringType => UTF8String.fromString(value.asString) - case DateType => value.asDate.toEpochDay.toInt - case BinaryType => value.asBinary + case StringType => + val strValue = value match { + case uuid: java.util.UUID => uuid.toString + case inet: java.net.InetAddress => inet.getHostAddress + case s: String => s + case enumValue: BinaryStreamReader.EnumValue => enumValue.toString + case _ => value.toString + } + UTF8String.fromString(strValue) + case DateType => + val localDate = value match { + case ld: LocalDate => ld + case zdt: ZonedDateTime => zdt.toLocalDate + case _ => value.asInstanceOf[LocalDate] + } + localDate.toEpochDay.toInt + case BinaryType => value.asInstanceOf[String].getBytes case ArrayType(_dataType, _nullable) => - val arrayValue = value.asInstanceOf[ClickHouseArraySequence] + // Java client returns BinaryStreamReader.ArrayValue for arrays + val arrayVal = value.asInstanceOf[BinaryStreamReader.ArrayValue] + val arrayValue = arrayVal.getArrayOfObjects().toSeq.asInstanceOf[Seq[Object]] val convertedArray = Array.tabulate(arrayValue.length) { i => decodeValue( - arrayValue.getValue(i, createClickHouseValue(null, _dataType)), + arrayValue(i), StructField("element", _dataType, _nullable) ) } new GenericArrayData(convertedArray) case MapType(_keyType, _valueType, _valueNullable) => - val convertedMap = value.asMap().asScala.map { case (rawKey, rawValue) => - val decodedKey = decodeValue(createClickHouseValue(rawKey, _keyType), StructField("key", _keyType, false)) - val decodedValue = - decodeValue(createClickHouseValue(rawValue, _valueType), StructField("value", _valueType, _valueNullable)) - (decodedKey, decodedValue) - } + // Java client returns util.Map (LinkedHashMap or EmptyMap) + val javaMap = value.asInstanceOf[util.Map[Object, Object]] + val convertedMap = + javaMap.asScala.map { case (rawKey, rawValue) => + val decodedKey = decodeValue(rawKey, StructField("key", _keyType, false)) + val decodedValue = + decodeValue(rawValue, StructField("value", _valueType, _valueNullable)) + (decodedKey, decodedValue) + } ArrayBasedMapData(convertedMap) - case _ => throw CHClientException(s"Unsupported catalyst type ${structField.name}[${structField.dataType}]") } } - private def createClickHouseValue(rawValue: Any, dataType: DataType): ClickHouseValue = { - val isNull = rawValue == null - - dataType match { - case StringType => - if (isNull) ClickHouseStringValue.ofNull() - else ClickHouseStringValue.of(rawValue.toString) - - case IntegerType => - if (isNull) ClickHouseIntegerValue.ofNull() - else ClickHouseIntegerValue.of(rawValue.asInstanceOf[Int]) - - case LongType => - if (isNull) ClickHouseLongValue.ofNull() - else ClickHouseLongValue.of(rawValue.asInstanceOf[Long]) - - case DoubleType => - if (isNull) ClickHouseDoubleValue.ofNull() - else ClickHouseDoubleValue.of(rawValue.asInstanceOf[Double]) - - case FloatType => - if (isNull) ClickHouseFloatValue.ofNull() - else ClickHouseFloatValue.of(rawValue.asInstanceOf[Float]) - - case BooleanType => - if (isNull) ClickHouseBoolValue.ofNull() - else ClickHouseBoolValue.of(rawValue.asInstanceOf[Boolean]) - - case _: ArrayType => - if (isNull) ClickHouseArrayValue.ofEmpty() - else ClickHouseArrayValue.of(rawValue.asInstanceOf[Array[Object]]) - - case _: MapType => - if (isNull) ClickHouseMapValue.ofEmpty(classOf[Object], classOf[Object]) - else ClickHouseMapValue.of( - rawValue.asInstanceOf[java.util.Map[Object, Object]], - classOf[Object], - classOf[Object] - ) - - case _ => - if (isNull) ClickHouseStringValue.ofNull() - else ClickHouseStringValue.of(rawValue.toString) - } - } - } diff --git a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala index 2671155f..a13062fb 100644 --- a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala +++ b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala @@ -85,10 +85,15 @@ class ClickHouseJsonReader( TimeUnit.SECONDS.toMicros(_instant.toEpochSecond) + TimeUnit.NANOSECONDS.toMicros(_instant.getNano()) case StringType => UTF8String.fromString(jsonNode.asText) case DateType => LocalDate.parse(jsonNode.asText, dateFmt).toEpochDay.toInt - case BinaryType => jsonNode.binaryValue + case BinaryType if jsonNode.isTextual => + // ClickHouse JSON format returns FixedString as plain text, not Base64 + jsonNode.asText.getBytes("UTF-8") + case BinaryType => + // True binary data is Base64 encoded in JSON format + jsonNode.binaryValue case ArrayType(_dataType, _nullable) => val _structField = StructField(s"${structField.name}__array_element__", _dataType, _nullable) - new GenericArrayData(jsonNode.asScala.map(decodeValue(_, _structField))) + new GenericArrayData(jsonNode.asScala.map(decodeValue(_, _structField)).toArray) case MapType(StringType, _valueType, _valueNullable) => val mapData = jsonNode.fields.asScala.map { entry => val _structField = StructField(s"${structField.name}__map_value__", _valueType, _valueNullable) diff --git a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala index 13953a2a..6b715a0b 100644 --- a/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala +++ b/spark-3.3/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala @@ -225,7 +225,8 @@ abstract class ClickHouseWriter(writeJob: WriteJobDescription) writeJob.writeOptions.retryInterval ) { var startWriteTime = System.currentTimeMillis - client.syncInsertOutputJSONEachRow(database, table, format, codec, new ByteArrayInputStream(data)) match { + // , codec + client.syncInsertOutputJSONEachRow(database, table, format, new ByteArrayInputStream(data)) match { case Right(_) => writeTime = System.currentTimeMillis - startWriteTime _totalWriteTime.add(writeTime) diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala index 6f5686fe..30799fe2 100644 --- a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala @@ -38,6 +38,8 @@ class TPCDSClusterSuite extends SparkClickHouseClusterTest { spark.sql("CREATE DATABASE tpcds_sf1_cluster WITH DBPROPERTIES (cluster = 'single_replica')") TPCDSTestUtils.tablePrimaryKeys.foreach { case (table, primaryKeys) => + println(s"before table ${table} ${primaryKeys}") + val start: Long = System.currentTimeMillis() spark.sql( s""" |CREATE TABLE tpcds_sf1_cluster.$table @@ -51,6 +53,7 @@ class TPCDSClusterSuite extends SparkClickHouseClusterTest { |SELECT * FROM tpcds.sf1.$table; |""".stripMargin ) + println(s"time took table ${table} ${System.currentTimeMillis() - start}") } TPCDSTestUtils.tablePrimaryKeys.keys.foreach { table => diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala new file mode 100644 index 00000000..721ba948 --- /dev/null +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.ClickHouseSingleMixIn +import org.apache.spark.SparkConf + +class ClickHouseSingleArrowWriterSuite extends ClickHouseArrowWriterSuite with ClickHouseSingleMixIn + +abstract class ClickHouseArrowWriterSuite extends ClickHouseWriterTestBase { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.write.format", "arrow") + .set("spark.clickhouse.read.format", "json") + +} diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala new file mode 100644 index 00000000..decfe0af --- /dev/null +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.{ClickHouseCloudMixIn, ClickHouseSingleMixIn} +import org.apache.spark.SparkConf +import org.scalatest.tags.Cloud + +@Cloud +class ClickHouseCloudBinaryReaderSuite extends ClickHouseBinaryReaderSuite with ClickHouseCloudMixIn + +class ClickHouseSingleBinaryReaderSuite extends ClickHouseBinaryReaderSuite with ClickHouseSingleMixIn + +/** + * Test suite for ClickHouse Binary Reader. + * Uses binary format for reading data from ClickHouse. + * All test cases are inherited from ClickHouseReaderTestBase. + */ +abstract class ClickHouseBinaryReaderSuite extends ClickHouseReaderTestBase { + + // Override to use binary format for reading + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.read.format", "binary") + .set("spark.clickhouse.write.format", "arrow") + + // All tests are inherited from ClickHouseReaderTestBase + // Additional binary-specific tests can be added here if needed +} diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala new file mode 100644 index 00000000..c62d5564 --- /dev/null +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.{ClickHouseCloudMixIn, ClickHouseSingleMixIn} +import org.scalatest.tags.Cloud + +@Cloud +class ClickHouseCloudJsonReaderSuite extends ClickHouseJsonReaderSuite with ClickHouseCloudMixIn + +class ClickHouseSingleJsonReaderSuite extends ClickHouseJsonReaderSuite with ClickHouseSingleMixIn + +/** + * Test suite for ClickHouse JSON Reader. + * Uses JSON format for reading data from ClickHouse (default in SparkClickHouseSingleTest). + * All test cases are inherited from ClickHouseReaderTestBase. + */ +abstract class ClickHouseJsonReaderSuite extends ClickHouseReaderTestBase { + // Uses JSON format (configured in SparkClickHouseSingleTest) + // All tests are inherited from ClickHouseReaderTestBase + // Additional JSON-specific tests can be added here if needed +} diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala new file mode 100644 index 00000000..3532b140 --- /dev/null +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.ClickHouseSingleMixIn +import org.apache.spark.SparkConf + +class ClickHouseSingleJsonWriterSuite extends ClickHouseJsonWriterSuite with ClickHouseSingleMixIn + +abstract class ClickHouseJsonWriterSuite extends ClickHouseWriterTestBase { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.write.format", "json") + .set("spark.clickhouse.read.format", "json") + +} diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala new file mode 100644 index 00000000..73e9119f --- /dev/null +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala @@ -0,0 +1,1331 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import org.apache.spark.sql.Row + +/** + * Shared test cases for both JSON and Binary readers. + * Subclasses only need to configure the read format. + * + * Tests are organized by ClickHouse data type with both regular and nullable variants. + * Each type includes comprehensive coverage of edge cases and null handling. + */ +trait ClickHouseReaderTestBase extends SparkClickHouseSingleTest { + + // ============================================================================ + // ArrayType Tests + // ============================================================================ + + test("decode ArrayType - Array of integers") { + withKVTable("test_db", "test_array_int", valueColDef = "Array(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_int VALUES + |(1, [1, 2, 3]), + |(2, []), + |(3, [100, 200, 300, 400]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_int ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(1).getSeq[Int](1) == Seq()) + assert(result(2).getSeq[Int](1) == Seq(100, 200, 300, 400)) + } + } + test("decode ArrayType - Array of strings") { + withKVTable("test_db", "test_array_string", valueColDef = "Array(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_string VALUES + |(1, ['hello', 'world']), + |(2, []), + |(3, ['a', 'b', 'c']) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_string ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[String](1) == Seq("hello", "world")) + assert(result(1).getSeq[String](1) == Seq()) + assert(result(2).getSeq[String](1) == Seq("a", "b", "c")) + } + } + test("decode ArrayType - Array with nullable elements") { + withKVTable("test_db", "test_array_nullable", valueColDef = "Array(Nullable(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_nullable VALUES + |(1, [1, NULL, 3]), + |(2, [NULL, NULL]), + |(3, [100, 200]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_nullable ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify arrays can be read + assert(result(0).getSeq[Any](1) != null) + assert(result(1).getSeq[Any](1) != null) + assert(result(2).getSeq[Any](1) != null) + } + } + test("decode ArrayType - empty arrays") { + withKVTable("test_db", "test_empty_array", valueColDef = "Array(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_empty_array VALUES + |(1, []), + |(2, [1, 2, 3]), + |(3, []) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_empty_array ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1).isEmpty) + assert(result(1).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(2).getSeq[Int](1).isEmpty) + } + } + test("decode ArrayType - Nested arrays") { + withKVTable("test_db", "test_nested_array", valueColDef = "Array(Array(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_nested_array VALUES + |(1, [[1, 2], [3, 4]]), + |(2, [[], [5]]), + |(3, [[10, 20, 30]]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_nested_array ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify nested arrays can be read + assert(result(0).get(1) != null) + assert(result(1).get(1) != null) + assert(result(2).get(1) != null) + } + } + test("decode BinaryType - FixedString") { + // FixedString is read as String by default in the connector + withKVTable("test_db", "test_fixedstring", valueColDef = "FixedString(5)") { + runClickHouseSQL( + """INSERT INTO test_db.test_fixedstring VALUES + |(1, 'hello'), + |(2, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_fixedstring ORDER BY key") + val result = df.collect() + assert(result.length == 2) + // FixedString should be readable + assert(result(0).get(1) != null) + assert(result(1).get(1) != null) + } + } + test("decode BinaryType - FixedString nullable with null values") { + withKVTable("test_db", "test_fixedstring_null", valueColDef = "Nullable(FixedString(5))") { + runClickHouseSQL( + """INSERT INTO test_db.test_fixedstring_null VALUES + |(1, 'hello'), + |(2, NULL), + |(3, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_fixedstring_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).get(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).get(1) != null) + } + } + + // ============================================================================ + // BooleanType Tests + // ============================================================================ + + test("decode BooleanType - true and false values") { + // ClickHouse Bool is stored as UInt8 (0 or 1) + // JSON format reads as Boolean, Binary format reads as Short + withKVTable("test_db", "test_bool", valueColDef = "Bool") { + runClickHouseSQL( + """INSERT INTO test_db.test_bool VALUES + |(1, true), + |(2, false), + |(3, 1), + |(4, 0) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_bool ORDER BY key") + val result = df.collect() + assert(result.length == 4) + // Check the value - handle both Boolean (JSON) and Short (Binary) formats + val v0 = result(0).get(1) + val v1 = result(1).get(1) + v0 match { + case b: Boolean => + assert(b == true) + assert(result(1).getBoolean(1) == false) + assert(result(2).getBoolean(1) == true) + assert(result(3).getBoolean(1) == false) + case s: Short => + assert(s == 1) + assert(result(1).getShort(1) == 0) + assert(result(2).getShort(1) == 1) + assert(result(3).getShort(1) == 0) + case _ => fail(s"Unexpected type: ${v0.getClass}") + } + } + } + test("decode BooleanType - nullable with null values") { + withKVTable("test_db", "test_bool_null", valueColDef = "Nullable(Bool)") { + runClickHouseSQL( + """INSERT INTO test_db.test_bool_null VALUES + |(1, true), + |(2, NULL), + |(3, false) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_bool_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(1).isNullAt(1)) + // Check the value - handle both Boolean (JSON) and Short (Binary) formats + val v0 = result(0).get(1) + v0 match { + case b: Boolean => + assert(b == true) + assert(result(2).getBoolean(1) == false) + case s: Short => + assert(s == 1) + assert(result(2).getShort(1) == 0) + case _ => fail(s"Unexpected type: ${v0.getClass}") + } + } + } + + // ============================================================================ + // ByteType Tests + // ============================================================================ + + test("decode ByteType - min and max values") { + withKVTable("test_db", "test_byte", valueColDef = "Int8") { + runClickHouseSQL( + """INSERT INTO test_db.test_byte VALUES + |(1, -128), + |(2, 0), + |(3, 127) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_byte ORDER BY key") + checkAnswer( + df, + Row(1, -128.toByte) :: Row(2, 0.toByte) :: Row(3, 127.toByte) :: Nil + ) + } + } + test("decode ByteType - nullable with null values") { + withKVTable("test_db", "test_byte_null", valueColDef = "Nullable(Int8)") { + runClickHouseSQL( + """INSERT INTO test_db.test_byte_null VALUES + |(1, -128), + |(2, NULL), + |(3, 127) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_byte_null ORDER BY key") + checkAnswer( + df, + Row(1, -128.toByte) :: Row(2, null) :: Row(3, 127.toByte) :: Nil + ) + } + } + test("decode DateTime32 - 32-bit timestamp") { + withKVTable("test_db", "test_datetime32", valueColDef = "DateTime32") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime32 VALUES + |(1, '2024-01-01 12:00:00'), + |(2, '2024-06-15 18:30:45') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime32 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + } + } + test("decode DateTime32 - nullable with null values") { + withKVTable("test_db", "test_datetime32_null", valueColDef = "Nullable(DateTime32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime32_null VALUES + |(1, '2024-01-01 12:00:00'), + |(2, NULL), + |(3, '2024-06-15 18:30:45') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode DateType - Date") { + withKVTable("test_db", "test_date", valueColDef = "Date") { + runClickHouseSQL( + """INSERT INTO test_db.test_date VALUES + |(1, '2024-01-01'), + |(2, '2024-06-15'), + |(3, '2024-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - Date32") { + withKVTable("test_db", "test_date32", valueColDef = "Date32") { + runClickHouseSQL( + """INSERT INTO test_db.test_date32 VALUES + |(1, '1900-01-01'), + |(2, '2024-06-15'), + |(3, '2100-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date32 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - Date32 nullable with null values") { + withKVTable("test_db", "test_date32_null", valueColDef = "Nullable(Date32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_date32_null VALUES + |(1, '1900-01-01'), + |(2, NULL), + |(3, '2100-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - nullable with null values") { + withKVTable("test_db", "test_date_null", valueColDef = "Nullable(Date)") { + runClickHouseSQL( + """INSERT INTO test_db.test_date_null VALUES + |(1, '2024-01-01'), + |(2, NULL), + |(3, '2024-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + test("decode DecimalType - Decimal128") { + // Decimal128(20) means scale=20, max precision=38 total digits + // Use values with max 18 digits before decimal to stay within 38 total + withKVTable("test_db", "test_decimal128", valueColDef = "Decimal128(20)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal128 VALUES + |(1, 123456789012345.12345678901234567890), + |(2, -999999999999999.99999999999999999999), + |(3, 0.00000000000000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal128 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Decimal128(20) means 20 decimal places, total precision up to 38 digits + assert(math.abs(result(0).getDecimal(1).doubleValue() - 123456789012345.12345678901234567890) < 0.01) + assert(math.abs(result(1).getDecimal(1).doubleValue() - -999999999999999.99999999999999999999) < 0.01) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal128 nullable with null values") { + withKVTable("test_db", "test_decimal128_null", valueColDef = "Nullable(Decimal128(20))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal128_null VALUES + |(1, 123456789012345.12345678901234567890), + |(2, NULL), + |(3, 0.00000000000000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal32") { + withKVTable("test_db", "test_decimal32", valueColDef = "Decimal32(4)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal32 VALUES + |(1, 12345.6789), + |(2, -9999.9999), + |(3, 0.0001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal32 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).doubleValue() == 12345.6789) + assert(result(1).getDecimal(1).doubleValue() == -9999.9999) + assert(result(2).getDecimal(1).doubleValue() == 0.0001) + } + } + test("decode DecimalType - Decimal32 nullable with null values") { + withKVTable("test_db", "test_decimal32_null", valueColDef = "Nullable(Decimal32(4))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal32_null VALUES + |(1, 12345.6789), + |(2, NULL), + |(3, 0.0001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal64") { + // Decimal64(10) means scale=10, max precision=18 total digits + // Use values with max 8 digits before decimal to stay within 18 total + withKVTable("test_db", "test_decimal64", valueColDef = "Decimal64(10)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal64 VALUES + |(1, 1234567.0123456789), + |(2, -9999999.9999999999), + |(3, 0.0000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDecimal(1).doubleValue() - 1234567.0123456789) < 0.0001) + assert(math.abs(result(1).getDecimal(1).doubleValue() - -9999999.9999999999) < 0.0001) + assert(math.abs(result(2).getDecimal(1).doubleValue() - 0.0000000001) < 0.0000000001) + } + } + test("decode DecimalType - Decimal64 nullable with null values") { + withKVTable("test_db", "test_decimal64_null", valueColDef = "Nullable(Decimal64(10))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal64_null VALUES + |(1, 1234567.0123456789), + |(2, NULL), + |(3, 0.0000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DoubleType - nullable with null values") { + withKVTable("test_db", "test_double_null", valueColDef = "Nullable(Float64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_double_null VALUES + |(1, 1.23), + |(2, NULL), + |(3, -4.56) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_double_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 1.23) < 0.0001) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getDouble(1) - -4.56) < 0.0001) + } + } + test("decode DoubleType - regular values") { + withKVTable("test_db", "test_double", valueColDef = "Float64") { + runClickHouseSQL( + """INSERT INTO test_db.test_double VALUES + |(1, -3.141592653589793), + |(2, 0.0), + |(3, 3.141592653589793) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_double ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - -3.141592653589793) < 0.000001) + assert(result(1).getDouble(1) == 0.0) + assert(math.abs(result(2).getDouble(1) - 3.141592653589793) < 0.000001) + } + } + test("decode Enum16 - large enum") { + withKVTable("test_db", "test_enum16", valueColDef = "Enum16('small' = 1, 'medium' = 100, 'large' = 1000)") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum16 VALUES + |(1, 'small'), + |(2, 'medium'), + |(3, 'large') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum16 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "small") + assert(result(1).getString(1) == "medium") + assert(result(2).getString(1) == "large") + } + } + test("decode Enum16 - nullable with null values") { + withKVTable( + "test_db", + "test_enum16_null", + valueColDef = "Nullable(Enum16('small' = 1, 'medium' = 100, 'large' = 1000))" + ) { + runClickHouseSQL( + """INSERT INTO test_db.test_enum16_null VALUES + |(1, 'small'), + |(2, NULL), + |(3, 'large') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum16_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "small") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "large") + } + } + test("decode Enum8 - nullable with null values") { + withKVTable("test_db", "test_enum8_null", valueColDef = "Nullable(Enum8('red' = 1, 'green' = 2, 'blue' = 3))") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum8_null VALUES + |(1, 'red'), + |(2, NULL), + |(3, 'blue') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum8_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "red") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "blue") + } + } + test("decode Enum8 - small enum") { + withKVTable("test_db", "test_enum8", valueColDef = "Enum8('red' = 1, 'green' = 2, 'blue' = 3)") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum8 VALUES + |(1, 'red'), + |(2, 'green'), + |(3, 'blue') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum8 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "red") + assert(result(1).getString(1) == "green") + assert(result(2).getString(1) == "blue") + } + } + test("decode FloatType - nullable with null values") { + withKVTable("test_db", "test_float_null", valueColDef = "Nullable(Float32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_float_null VALUES + |(1, 1.5), + |(2, NULL), + |(3, -2.5) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_float_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 1.5f) < 0.01f) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getFloat(1) - -2.5f) < 0.01f) + } + } + test("decode FloatType - regular values") { + withKVTable("test_db", "test_float", valueColDef = "Float32") { + runClickHouseSQL( + """INSERT INTO test_db.test_float VALUES + |(1, -3.14), + |(2, 0.0), + |(3, 3.14) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_float ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - -3.14f) < 0.01f) + assert(result(1).getFloat(1) == 0.0f) + assert(math.abs(result(2).getFloat(1) - 3.14f) < 0.01f) + } + } + test("decode Int128 - large integers as Decimal") { + withKVTable("test_db", "test_int128", valueColDef = "Int128") { + runClickHouseSQL( + """INSERT INTO test_db.test_int128 VALUES + |(1, 0), + |(2, 123456789012345678901234567890), + |(3, -123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int128 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int128 - nullable with null values") { + withKVTable("test_db", "test_int128_null", valueColDef = "Nullable(Int128)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int128_null VALUES + |(1, 0), + |(2, NULL), + |(3, -123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int256 - nullable with null values") { + withKVTable("test_db", "test_int256_null", valueColDef = "Nullable(Int256)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int256_null VALUES + |(1, 0), + |(2, NULL), + |(3, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int256_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int256 - very large integers as Decimal") { + withKVTable("test_db", "test_int256", valueColDef = "Int256") { + runClickHouseSQL( + """INSERT INTO test_db.test_int256 VALUES + |(1, 0), + |(2, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int256 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode IntegerType - min and max values") { + withKVTable("test_db", "test_int", valueColDef = "Int32") { + runClickHouseSQL( + """INSERT INTO test_db.test_int VALUES + |(1, -2147483648), + |(2, 0), + |(3, 2147483647) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int ORDER BY key") + checkAnswer( + df, + Row(1, -2147483648) :: Row(2, 0) :: Row(3, 2147483647) :: Nil + ) + } + } + test("decode IntegerType - nullable with null values") { + withKVTable("test_db", "test_int_null", valueColDef = "Nullable(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int_null VALUES + |(1, -2147483648), + |(2, NULL), + |(3, 2147483647) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int_null ORDER BY key") + checkAnswer( + df, + Row(1, -2147483648) :: Row(2, null) :: Row(3, 2147483647) :: Nil + ) + } + } + test("decode IPv4 - IP addresses") { + withKVTable("test_db", "test_ipv4", valueColDef = "IPv4") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv4 VALUES + |(1, '127.0.0.1'), + |(2, '192.168.1.1'), + |(3, '8.8.8.8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv4 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "127.0.0.1") + assert(result(1).getString(1) == "192.168.1.1") + assert(result(2).getString(1) == "8.8.8.8") + } + } + test("decode IPv4 - nullable with null values") { + withKVTable("test_db", "test_ipv4_null", valueColDef = "Nullable(IPv4)") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv4_null VALUES + |(1, '127.0.0.1'), + |(2, NULL), + |(3, '8.8.8.8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv4_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "127.0.0.1") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "8.8.8.8") + } + } + test("decode IPv6 - IPv6 addresses") { + withKVTable("test_db", "test_ipv6", valueColDef = "IPv6") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv6 VALUES + |(1, '::1'), + |(2, '2001:0db8:85a3:0000:0000:8a2e:0370:7334'), + |(3, 'fe80::1') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv6 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) != null) + assert(result(1).getString(1) != null) + assert(result(2).getString(1) != null) + } + } + test("decode IPv6 - nullable with null values") { + withKVTable("test_db", "test_ipv6_null", valueColDef = "Nullable(IPv6)") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv6_null VALUES + |(1, '::1'), + |(2, NULL), + |(3, 'fe80::1') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv6_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) != null) + } + } + test("decode JSON - nullable with null values") { + withKVTable("test_db", "test_json_null", valueColDef = "Nullable(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_json_null VALUES + |(1, '{"name": "Alice", "age": 30}'), + |(2, NULL), + |(3, '{"name": "Charlie", "age": 35}') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_json_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1).contains("Alice")) + assert(result(1).isNullAt(1)) + assert(result(2).getString(1).contains("Charlie")) + } + } + test("decode JSON - semi-structured data") { + withKVTable("test_db", "test_json", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_json VALUES + |(1, '{"name": "Alice", "age": 30}'), + |(2, '{"name": "Bob", "age": 25}'), + |(3, '{"name": "Charlie", "age": 35}') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_json ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1).contains("Alice")) + assert(result(1).getString(1).contains("Bob")) + assert(result(2).getString(1).contains("Charlie")) + } + } + test("decode LongType - min and max values") { + withKVTable("test_db", "test_long", valueColDef = "Int64") { + runClickHouseSQL( + """INSERT INTO test_db.test_long VALUES + |(1, -9223372036854775808), + |(2, 0), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long ORDER BY key") + checkAnswer( + df, + Row(1, -9223372036854775808L) :: Row(2, 0L) :: Row(3, 9223372036854775807L) :: Nil + ) + } + } + test("decode LongType - nullable with null values") { + withKVTable("test_db", "test_long_null", valueColDef = "Nullable(Int64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_long_null VALUES + |(1, -9223372036854775808), + |(2, NULL), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long_null ORDER BY key") + checkAnswer( + df, + Row(1, -9223372036854775808L) :: Row(2, null) :: Row(3, 9223372036854775807L) :: Nil + ) + } + } + test("decode LongType - UInt32 nullable with null values") { + withKVTable("test_db", "test_uint32_null", valueColDef = "Nullable(UInt32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint32_null VALUES + |(1, 0), + |(2, NULL), + |(3, 4294967295) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint32_null ORDER BY key") + checkAnswer( + df, + Row(1, 0L) :: Row(2, null) :: Row(3, 4294967295L) :: Nil + ) + } + } + test("decode LongType - UInt32 values") { + withKVTable("test_db", "test_uint32", valueColDef = "UInt32") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint32 VALUES + |(1, 0), + |(2, 2147483648), + |(3, 4294967295) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint32 ORDER BY key") + checkAnswer( + df, + Row(1, 0L) :: Row(2, 2147483648L) :: Row(3, 4294967295L) :: Nil + ) + } + } + test("decode MapType - Map of String to Int") { + withKVTable("test_db", "test_map", valueColDef = "Map(String, Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_map VALUES + |(1, {'a': 1, 'b': 2}), + |(2, {}), + |(3, {'x': 100, 'y': 200, 'z': 300}) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_map ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1) == Map("a" -> 1, "b" -> 2)) + assert(result(1).getMap[String, Int](1) == Map()) + assert(result(2).getMap[String, Int](1) == Map("x" -> 100, "y" -> 200, "z" -> 300)) + } + } + test("decode MapType - Map with nullable values") { + withKVTable("test_db", "test_map_nullable", valueColDef = "Map(String, Nullable(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_map_nullable VALUES + |(1, {'a': 1, 'b': NULL}), + |(2, {'x': NULL}), + |(3, {'p': 100, 'q': 200}) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_map_nullable ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify maps can be read + assert(result(0).getMap[String, Any](1) != null) + assert(result(1).getMap[String, Any](1) != null) + assert(result(2).getMap[String, Any](1) != null) + } + } + test("decode ShortType - min and max values") { + withKVTable("test_db", "test_short", valueColDef = "Int16") { + runClickHouseSQL( + """INSERT INTO test_db.test_short VALUES + |(1, -32768), + |(2, 0), + |(3, 32767) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_short ORDER BY key") + checkAnswer( + df, + Row(1, -32768.toShort) :: Row(2, 0.toShort) :: Row(3, 32767.toShort) :: Nil + ) + } + } + test("decode ShortType - nullable with null values") { + withKVTable("test_db", "test_short_null", valueColDef = "Nullable(Int16)") { + runClickHouseSQL( + """INSERT INTO test_db.test_short_null VALUES + |(1, -32768), + |(2, NULL), + |(3, 32767) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_short_null ORDER BY key") + checkAnswer( + df, + Row(1, -32768.toShort) :: Row(2, null) :: Row(3, 32767.toShort) :: Nil + ) + } + } + test("decode ShortType - UInt8 nullable with null values") { + withKVTable("test_db", "test_uint8_null", valueColDef = "Nullable(UInt8)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint8_null VALUES + |(1, 0), + |(2, NULL), + |(3, 255) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint8_null ORDER BY key") + checkAnswer( + df, + Row(1, 0.toShort) :: Row(2, null) :: Row(3, 255.toShort) :: Nil + ) + } + } + test("decode ShortType - UInt8 values") { + withKVTable("test_db", "test_uint8", valueColDef = "UInt8") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint8 VALUES + |(1, 0), + |(2, 128), + |(3, 255) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint8 ORDER BY key") + checkAnswer( + df, + Row(1, 0.toShort) :: Row(2, 128.toShort) :: Row(3, 255.toShort) :: Nil + ) + } + } + test("decode StringType - empty strings") { + withKVTable("test_db", "test_empty_string", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_empty_string VALUES + |(1, ''), + |(2, 'not empty'), + |(3, '') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_empty_string ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "") + assert(result(1).getString(1) == "not empty") + assert(result(2).getString(1) == "") + } + } + test("decode StringType - nullable with null values") { + withKVTable("test_db", "test_string_null", valueColDef = "Nullable(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_string_null VALUES + |(1, 'hello'), + |(2, NULL), + |(3, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_string_null ORDER BY key") + checkAnswer( + df, + Row(1, "hello") :: Row(2, null) :: Row(3, "world") :: Nil + ) + } + } + test("decode StringType - regular strings") { + withKVTable("test_db", "test_string", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_string VALUES + |(1, 'hello'), + |(2, ''), + |(3, 'world with spaces'), + |(4, 'special chars: !@#$%^&*()') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_string ORDER BY key") + checkAnswer( + df, + Row(1, "hello") :: Row(2, "") :: Row(3, "world with spaces") :: Row(4, "special chars: !@#$%^&*()") :: Nil + ) + } + } + test("decode StringType - UUID") { + withKVTable("test_db", "test_uuid", valueColDef = "UUID") { + runClickHouseSQL( + """INSERT INTO test_db.test_uuid VALUES + |(1, '550e8400-e29b-41d4-a716-446655440000'), + |(2, '6ba7b810-9dad-11d1-80b4-00c04fd430c8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uuid ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getString(1) == "550e8400-e29b-41d4-a716-446655440000") + assert(result(1).getString(1) == "6ba7b810-9dad-11d1-80b4-00c04fd430c8") + } + } + test("decode StringType - UUID nullable with null values") { + withKVTable("test_db", "test_uuid_null", valueColDef = "Nullable(UUID)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uuid_null VALUES + |(1, '550e8400-e29b-41d4-a716-446655440000'), + |(2, NULL), + |(3, '6ba7b810-9dad-11d1-80b4-00c04fd430c8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uuid_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "550e8400-e29b-41d4-a716-446655440000") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "6ba7b810-9dad-11d1-80b4-00c04fd430c8") + } + } + test("decode StringType - very long strings") { + val longString = "a" * 10000 + withKVTable("test_db", "test_long_string", valueColDef = "String") { + runClickHouseSQL( + s"""INSERT INTO test_db.test_long_string VALUES + |(1, '$longString') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long_string ORDER BY key") + val result = df.collect() + assert(result.length == 1) + assert(result(0).getString(1).length == 10000) + } + } + test("decode TimestampType - DateTime") { + withKVTable("test_db", "test_datetime", valueColDef = "DateTime") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime VALUES + |(1, '2024-01-01 00:00:00'), + |(2, '2024-06-15 12:30:45'), + |(3, '2024-12-31 23:59:59') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - DateTime64") { + withKVTable("test_db", "test_datetime64", valueColDef = "DateTime64(3)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime64 VALUES + |(1, '2024-01-01 00:00:00.123'), + |(2, '2024-06-15 12:30:45.456'), + |(3, '2024-12-31 23:59:59.999') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - DateTime64 nullable with null values") { + withKVTable("test_db", "test_datetime64_null", valueColDef = "Nullable(DateTime64(3))") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime64_null VALUES + |(1, '2024-01-01 00:00:00.123'), + |(2, NULL), + |(3, '2024-12-31 23:59:59.999') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - nullable with null values") { + withKVTable("test_db", "test_datetime_null", valueColDef = "Nullable(DateTime)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime_null VALUES + |(1, '2024-01-01 00:00:00'), + |(2, NULL), + |(3, '2024-12-31 23:59:59') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode UInt128 - large unsigned integers as Decimal") { + withKVTable("test_db", "test_uint128", valueColDef = "UInt128") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint128 VALUES + |(1, 0), + |(2, 123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint128 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode UInt128 - nullable with null values") { + withKVTable("test_db", "test_uint128_null", valueColDef = "Nullable(UInt128)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint128_null VALUES + |(1, 0), + |(2, NULL), + |(3, 123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode UInt16 - nullable with null values") { + withKVTable("test_db", "test_uint16_null", valueColDef = "Nullable(UInt16)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint16_null VALUES + |(1, 0), + |(2, NULL), + |(3, 65535) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint16_null ORDER BY key") + checkAnswer( + df, + Row(1, 0) :: Row(2, null) :: Row(3, 65535) :: Nil + ) + } + } + test("decode UInt16 - unsigned 16-bit integers") { + withKVTable("test_db", "test_uint16", valueColDef = "UInt16") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint16 VALUES + |(1, 0), + |(2, 32768), + |(3, 65535) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint16 ORDER BY key") + checkAnswer( + df, + Row(1, 0) :: Row(2, 32768) :: Row(3, 65535) :: Nil + ) + } + } + test("decode UInt256 - nullable with null values") { + withKVTable("test_db", "test_uint256_null", valueColDef = "Nullable(UInt256)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint256_null VALUES + |(1, 0), + |(2, NULL), + |(3, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint256_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode UInt256 - very large unsigned integers as Decimal") { + withKVTable("test_db", "test_uint256", valueColDef = "UInt256") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint256 VALUES + |(1, 0), + |(2, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint256 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode UInt64 - nullable with null values") { + withKVTable("test_db", "test_uint64_null", valueColDef = "Nullable(UInt64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint64_null VALUES + |(1, 0), + |(2, NULL), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getLong(1) == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getLong(1) == 9223372036854775807L) + } + } + test("decode UInt64 - unsigned 64-bit integers") { + withKVTable("test_db", "test_uint64", valueColDef = "UInt64") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint64 VALUES + |(1, 0), + |(2, 1234567890), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getLong(1) == 0L) + assert(result(1).getLong(1) == 1234567890L) + // Max value that fits in signed Long + assert(result(2).getLong(1) == 9223372036854775807L) + } + } + +} diff --git a/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala new file mode 100644 index 00000000..28267dc2 --- /dev/null +++ b/spark-3.4/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala @@ -0,0 +1,758 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.types._ + +/** + * Shared test cases for both JSON and Binary writers. + * Subclasses only need to configure the write format. + */ +trait ClickHouseWriterTestBase extends SparkClickHouseSingleTest { + + test("write ArrayType - array of integers") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_array_int", schema) { + val data = Seq( + Row(1, Seq(1, 2, 3)), + Row(2, Seq(10, 20, 30)), + Row(3, Seq(100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_array_int") + + val result = spark.table("test_db.test_write_array_int").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(1).getSeq[Int](1) == Seq(10, 20, 30)) + assert(result(2).getSeq[Int](1) == Seq(100)) + } + } + + test("write ArrayType - empty arrays") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_empty_array", schema) { + val data = Seq( + Row(1, Seq()), + Row(2, Seq(1, 2, 3)), + Row(3, Seq()) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_array") + + val result = spark.table("test_db.test_write_empty_array").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1).isEmpty) + assert(result(1).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(2).getSeq[Int](1).isEmpty) + } + } + + test("write ArrayType - nested arrays") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField( + "value", + ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = false), + nullable = false + ) + )) + + withTable("test_db", "test_write_nested_array", schema) { + val data = Seq( + Row(1, Seq(Seq(1, 2), Seq(3, 4))), + Row(2, Seq(Seq(10, 20, 30))), + Row(3, Seq(Seq(), Seq(100))) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_nested_array") + + val result = spark.table("test_db.test_write_nested_array").orderBy("id").collect() + assert(result.length == 3) + // Convert to List for Scala 2.12/2.13 compatibility + val row0 = result(0).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + val row1 = result(1).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + val row2 = result(2).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + assert(row0 == Seq(Seq(1, 2), Seq(3, 4))) + assert(row1 == Seq(Seq(10, 20, 30))) + assert(row2(0).isEmpty) + assert(row2(1) == Seq(100)) + } + } + + test("write ArrayType - with nullable elements") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = true), nullable = false) + )) + + withTable("test_db", "test_write_array_nullable", schema) { + val data = Seq( + Row(1, Seq(1, null, 3)), + Row(2, Seq(null, null)), + Row(3, Seq(10, 20, 30)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_array_nullable") + + val result = spark.table("test_db.test_write_array_nullable").orderBy("id").collect() + assert(result.length == 3) + val arr1 = result(0).getSeq[Any](1) + assert(arr1.length == 3) + assert(arr1(0) == 1) + assert(arr1(1) == null) + assert(arr1(2) == 3) + } + } + + test("write BooleanType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", BooleanType, nullable = true) + )) + + withTable("test_db", "test_write_bool_null", schema) { + val data = Seq( + Row(1, true), + Row(2, null), + Row(3, false) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_bool_null") + + val result = spark.table("test_db.test_write_bool_null").orderBy("id").collect() + assert(result.length == 3) + // Boolean is stored as UInt8 in ClickHouse, reads back as Short + assert(result(0).getShort(1) == 1) + assert(result(1).isNullAt(1)) + assert(result(2).getShort(1) == 0) + } + } + + // NOTE: ClickHouse stores Boolean as UInt8, so it reads back as Short (0 or 1) + test("write BooleanType - true and false values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", BooleanType, nullable = false) + )) + + withTable("test_db", "test_write_bool", schema) { + val data = Seq( + Row(1, true), + Row(2, false) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_bool") + + val result = spark.table("test_db.test_write_bool").orderBy("id").collect() + assert(result.length == 2) + // Boolean is stored as UInt8 in ClickHouse, reads back as Short + assert(result(0).getShort(1) == 1) + assert(result(1).getShort(1) == 0) + } + } + + test("write ByteType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ByteType, nullable = false) + )) + + withTable("test_db", "test_write_byte", schema) { + val data = Seq( + Row(1, Byte.MinValue), + Row(2, 0.toByte), + Row(3, Byte.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_byte") + + val result = spark.table("test_db.test_write_byte").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getByte(1) == Byte.MinValue) + assert(result(1).getByte(1) == 0.toByte) + assert(result(2).getByte(1) == Byte.MaxValue) + } + } + + test("write ByteType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ByteType, nullable = true) + )) + + withTable("test_db", "test_write_byte_null", schema) { + val data = Seq( + Row(1, Byte.MinValue), + Row(2, null), + Row(3, Byte.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_byte_null") + + val result = spark.table("test_db.test_write_byte_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getByte(1) == Byte.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getByte(1) == Byte.MaxValue) + } + } + + test("write DateType - dates") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DateType, nullable = false) + )) + + withTable("test_db", "test_write_date", schema) { + val data = Seq( + Row(1, java.sql.Date.valueOf("2024-01-01")), + Row(2, java.sql.Date.valueOf("2024-06-15")), + Row(3, java.sql.Date.valueOf("2024-12-31")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_date") + + val result = spark.table("test_db.test_write_date").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + + test("write DateType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DateType, nullable = true) + )) + + withTable("test_db", "test_write_date_null", schema) { + val data = Seq( + Row(1, java.sql.Date.valueOf("2024-01-01")), + Row(2, null), + Row(3, java.sql.Date.valueOf("2024-12-31")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_date_null") + + val result = spark.table("test_db.test_write_date_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + + test("write DecimalType - Decimal(10,2)") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(10, 2), nullable = false) + )) + + withTable("test_db", "test_write_decimal", schema) { + val data = Seq( + Row(1, BigDecimal("12345.67")), + Row(2, BigDecimal("-9999.99")), + Row(3, BigDecimal("0.01")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal") + + val result = spark.table("test_db.test_write_decimal").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) == BigDecimal("12345.67").underlying()) + assert(result(1).getDecimal(1) == BigDecimal("-9999.99").underlying()) + assert(result(2).getDecimal(1) == BigDecimal("0.01").underlying()) + } + } + + test("write DecimalType - Decimal(18,4)") { + // Note: High-precision decimals (>15-17 significant digits) may lose precision in JSON/Arrow formats. + // This appears to be related to the serialization/deserialization path, possibly due to intermediate + // double conversions in the format parsers. This test uses tolerance-based assertions to account + // for this observed behavior. Binary format (RowBinaryWithNamesAndTypes) preserves full precision. + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(18, 4), nullable = false) + )) + + withTable("test_db", "test_write_decimal_18_4", schema) { + val data = Seq( + Row(1, BigDecimal("12345678901234.5678")), + Row(2, BigDecimal("-9999999999999.9999")), + Row(3, BigDecimal("0.0001")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal_18_4") + + val result = spark.table("test_db.test_write_decimal_18_4").orderBy("id").collect() + assert(result.length == 3) + // Use tolerance for high-precision values (18 significant digits) + val tolerance = BigDecimal("0.001") + assert((BigDecimal(result(0).getDecimal(1)) - BigDecimal("12345678901234.5678")).abs < tolerance) + assert((BigDecimal(result(1).getDecimal(1)) - BigDecimal("-9999999999999.9999")).abs < tolerance) + // Small values should be exact + assert(result(2).getDecimal(1) == BigDecimal("0.0001").underlying()) + } + } + + test("write DecimalType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(10, 2), nullable = true) + )) + + withTable("test_db", "test_write_decimal_null", schema) { + val data = Seq( + Row(1, BigDecimal("12345.67")), + Row(2, null), + Row(3, BigDecimal("-9999.99")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal_null") + + val result = spark.table("test_db.test_write_decimal_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) == BigDecimal("12345.67").underlying()) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) == BigDecimal("-9999.99").underlying()) + } + } + + test("write DoubleType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DoubleType, nullable = true) + )) + + withTable("test_db", "test_write_double_null", schema) { + val data = Seq( + Row(1, 3.14159), + Row(2, null), + Row(3, -2.71828) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_double_null") + + val result = spark.table("test_db.test_write_double_null").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 3.14159) < 0.00001) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getDouble(1) - -2.71828) < 0.00001) + } + } + + test("write DoubleType - regular values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DoubleType, nullable = false) + )) + + withTable("test_db", "test_write_double", schema) { + val data = Seq( + Row(1, 3.14159), + Row(2, -2.71828), + Row(3, 0.0) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_double") + + val result = spark.table("test_db.test_write_double").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 3.14159) < 0.00001) + assert(math.abs(result(1).getDouble(1) - -2.71828) < 0.00001) + assert(result(2).getDouble(1) == 0.0) + } + } + + test("write FloatType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", FloatType, nullable = true) + )) + + withTable("test_db", "test_write_float_null", schema) { + val data = Seq( + Row(1, 3.14f), + Row(2, null), + Row(3, -2.718f) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_float_null") + + val result = spark.table("test_db.test_write_float_null").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 3.14f) < 0.001f) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getFloat(1) - -2.718f) < 0.001f) + } + } + + test("write FloatType - regular values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", FloatType, nullable = false) + )) + + withTable("test_db", "test_write_float", schema) { + val data = Seq( + Row(1, 3.14f), + Row(2, -2.718f), + Row(3, 0.0f) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_float") + + val result = spark.table("test_db.test_write_float").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 3.14f) < 0.001f) + assert(math.abs(result(1).getFloat(1) - -2.718f) < 0.001f) + assert(result(2).getFloat(1) == 0.0f) + } + } + + test("write IntegerType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", IntegerType, nullable = false) + )) + + withTable("test_db", "test_write_int", schema) { + val data = Seq( + Row(1, Int.MinValue), + Row(2, 0), + Row(3, Int.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_int") + + val result = spark.table("test_db.test_write_int").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getInt(1) == Int.MinValue) + assert(result(1).getInt(1) == 0) + assert(result(2).getInt(1) == Int.MaxValue) + } + } + + test("write IntegerType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", IntegerType, nullable = true) + )) + + withTable("test_db", "test_write_int_null", schema) { + val data = Seq( + Row(1, Int.MinValue), + Row(2, null), + Row(3, Int.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_int_null") + + val result = spark.table("test_db.test_write_int_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getInt(1) == Int.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getInt(1) == Int.MaxValue) + } + } + + test("write LongType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", LongType, nullable = false) + )) + + withTable("test_db", "test_write_long", schema) { + val data = Seq( + Row(1, Long.MinValue), + Row(2, 0L), + Row(3, Long.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_long") + + val result = spark.table("test_db.test_write_long").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getLong(1) == Long.MinValue) + assert(result(1).getLong(1) == 0L) + assert(result(2).getLong(1) == Long.MaxValue) + } + } + + test("write LongType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", LongType, nullable = true) + )) + + withTable("test_db", "test_write_long_null", schema) { + val data = Seq( + Row(1, Long.MinValue), + Row(2, null), + Row(3, Long.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_long_null") + + val result = spark.table("test_db.test_write_long_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getLong(1) == Long.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getLong(1) == Long.MaxValue) + } + } + + test("write MapType - empty maps") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_empty_map", schema) { + val data = Seq( + Row(1, Map[String, Int]()), + Row(2, Map("a" -> 1)), + Row(3, Map[String, Int]()) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_map") + + val result = spark.table("test_db.test_write_empty_map").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1).isEmpty) + assert(result(1).getMap[String, Int](1) == Map("a" -> 1)) + assert(result(2).getMap[String, Int](1).isEmpty) + } + } + + test("write MapType - map of string to int") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_map", schema) { + val data = Seq( + Row(1, Map("a" -> 1, "b" -> 2)), + Row(2, Map("x" -> 10, "y" -> 20)), + Row(3, Map("foo" -> 100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_map") + + val result = spark.table("test_db.test_write_map").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1) == Map("a" -> 1, "b" -> 2)) + assert(result(1).getMap[String, Int](1) == Map("x" -> 10, "y" -> 20)) + assert(result(2).getMap[String, Int](1) == Map("foo" -> 100)) + } + } + + test("write MapType - with nullable values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = true), nullable = false) + )) + + withTable("test_db", "test_write_map_nullable", schema) { + val data = Seq( + Row(1, Map("a" -> 1, "b" -> null)), + Row(2, Map("x" -> null, "y" -> 20)), + Row(3, Map("foo" -> 100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_map_nullable") + + val result = spark.table("test_db.test_write_map_nullable").orderBy("id").collect() + assert(result.length == 3) + val map1 = result(0).getMap[String, Any](1) + assert(map1("a") == 1) + assert(map1("b") == null) + } + } + + test("write ShortType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ShortType, nullable = false) + )) + + withTable("test_db", "test_write_short", schema) { + val data = Seq( + Row(1, Short.MinValue), + Row(2, 0.toShort), + Row(3, Short.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_short") + + val result = spark.table("test_db.test_write_short").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getShort(1) == Short.MinValue) + assert(result(1).getShort(1) == 0.toShort) + assert(result(2).getShort(1) == Short.MaxValue) + } + } + + test("write ShortType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ShortType, nullable = true) + )) + + withTable("test_db", "test_write_short_null", schema) { + val data = Seq( + Row(1, Short.MinValue), + Row(2, null), + Row(3, Short.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_short_null") + + val result = spark.table("test_db.test_write_short_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getShort(1) == Short.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getShort(1) == Short.MaxValue) + } + } + + test("write StringType - empty strings") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = false) + )) + + withTable("test_db", "test_write_empty_string", schema) { + val data = Seq( + Row(1, ""), + Row(2, "not empty"), + Row(3, "") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_string") + + val result = spark.table("test_db.test_write_empty_string").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "") + assert(result(1).getString(1) == "not empty") + assert(result(2).getString(1) == "") + } + } + + test("write StringType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = true) + )) + + withTable("test_db", "test_write_string_null", schema) { + val data = Seq( + Row(1, "hello"), + Row(2, null), + Row(3, "world") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_string_null") + + val result = spark.table("test_db.test_write_string_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "hello") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "world") + } + } + + test("write StringType - regular strings") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = false) + )) + + withTable("test_db", "test_write_string", schema) { + val data = Seq( + Row(1, "hello"), + Row(2, "world"), + Row(3, "test") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_string") + + val result = spark.table("test_db.test_write_string").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "hello") + assert(result(1).getString(1) == "world") + assert(result(2).getString(1) == "test") + } + } + + test("write TimestampType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", TimestampType, nullable = true) + )) + + withTable("test_db", "test_write_timestamp_null", schema) { + val data = Seq( + Row(1, java.sql.Timestamp.valueOf("2024-01-01 12:00:00")), + Row(2, null), + Row(3, java.sql.Timestamp.valueOf("2024-12-31 23:59:59")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_timestamp_null") + + val result = spark.table("test_db.test_write_timestamp_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + + test("write TimestampType - timestamps") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", TimestampType, nullable = false) + )) + + withTable("test_db", "test_write_timestamp", schema) { + val data = Seq( + Row(1, java.sql.Timestamp.valueOf("2024-01-01 12:00:00")), + Row(2, java.sql.Timestamp.valueOf("2024-06-15 18:30:45")), + Row(3, java.sql.Timestamp.valueOf("2024-12-31 23:59:59")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_timestamp") + + val result = spark.table("test_db.test_write_timestamp").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + +} diff --git a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala index 4c6d71f9..1e680e9c 100644 --- a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala +++ b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala @@ -14,17 +14,18 @@ package com.clickhouse.spark.read -import com.clickhouse.client.ClickHouseResponse +import com.clickhouse.spark.{ClickHouseHelper, Logging, TaskMetric} +import com.clickhouse.spark.client.{NodeClient, NodesClient} import com.clickhouse.data.ClickHouseCompression +import com.clickhouse.spark.format.StreamOutput import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} import org.apache.spark.sql.clickhouse.ClickHouseSQLConf._ import org.apache.spark.sql.connector.metric.CustomTaskMetric import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.types._ import com.clickhouse.spark.Metrics.{BLOCKS_READ, BYTES_READ} -import com.clickhouse.spark.client.{NodeClient, NodesClient} -import com.clickhouse.spark.format.StreamOutput -import com.clickhouse.spark.{ClickHouseHelper, Logging, TaskMetric} +import com.clickhouse.client.ClickHouseResponse +import com.clickhouse.client.api.query.QueryResponse abstract class ClickHouseReader[Record]( scanJob: ScanJobDescription, @@ -40,7 +41,7 @@ abstract class ClickHouseReader[Record]( val database: String = part.table.database val table: String = part.table.name - val codec: ClickHouseCompression = scanJob.readOptions.compressionCodec +// val codec: ClickHouseCompression = scanJob.readOptions.compressionCodec val readSchema: StructType = scanJob.readSchema private lazy val nodesClient = NodesClient(part.candidateNodes) @@ -67,11 +68,12 @@ abstract class ClickHouseReader[Record]( def format: String - lazy val resp: ClickHouseResponse = nodeClient.queryAndCheck(scanQuery, format, codec) + // , codec + lazy val resp: QueryResponse = nodeClient.queryAndCheck(scanQuery, format) - def totalBlocksRead: Long = resp.getSummary.getStatistics.getBlocks + def totalBlocksRead: Long = 0L // resp.getSummary.getStatistics.getBlocks - def totalBytesRead: Long = resp.getSummary.getReadBytes + def totalBytesRead: Long = resp.getReadBytes // resp.getSummary.getReadBytes override def currentMetricsValues: Array[CustomTaskMetric] = Array( TaskMetric(BLOCKS_READ, totalBlocksRead), diff --git a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala index 5e90c4eb..dd1f9127 100644 --- a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala +++ b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala @@ -14,6 +14,10 @@ package com.clickhouse.spark.read.format +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader +import com.clickhouse.client.api.data_formats.{ClickHouseBinaryFormatReader, RowBinaryWithNamesAndTypesFormatReader} +import com.clickhouse.client.api.query.{GenericRecord, Records} + import java.util.Collections import com.clickhouse.data.value.{ ClickHouseArrayValue, @@ -34,120 +38,118 @@ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import java.time.ZoneOffset +import java.io.InputStream +import java.time.{LocalDate, ZoneOffset, ZonedDateTime} +import java.util import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ class ClickHouseBinaryReader( scanJob: ScanJobDescription, part: ClickHouseInputPartition -) extends ClickHouseReader[ClickHouseRecord](scanJob, part) { +) extends ClickHouseReader[GenericRecord](scanJob, part) { override val format: String = "RowBinaryWithNamesAndTypes" - lazy val streamOutput: Iterator[ClickHouseRecord] = resp.records().asScala.iterator + lazy val streamOutput: Iterator[GenericRecord] = { + val inputString: InputStream = resp.getInputStream + val cbfr: ClickHouseBinaryFormatReader = new RowBinaryWithNamesAndTypesFormatReader( + inputString, + resp.getSettings, + new BinaryStreamReader.DefaultByteBufferAllocator + ) + val r = new Records(resp, cbfr) + r.asScala.iterator + } - override def decode(record: ClickHouseRecord): InternalRow = { - val values: Array[Any] = new Array[Any](record.size) + override def decode(record: GenericRecord): InternalRow = { + val size = record.getSchema.getColumns.size() + val values: Array[Any] = new Array[Any](size) if (readSchema.nonEmpty) { var i: Int = 0 - while (i < record.size) { - values(i) = decodeValue(record.getValue(i), readSchema.fields(i)) + while (i < size) { + val v: Object = record.getObject(i + 1) + values(i) = decodeValue(v, readSchema.fields(i)) i = i + 1 } } new GenericInternalRow(values) } - private def decodeValue(value: ClickHouseValue, structField: StructField): Any = { - if (value == null || value.isNullOrEmpty && value.isNullable) { + private def decodeValue(value: Object, structField: StructField): Any = { + if (value == null) { // should we check `structField.nullable`? return null } structField.dataType match { - case BooleanType => value.asBoolean - case ByteType => value.asByte - case ShortType => value.asShort - case IntegerType => value.asInteger - case LongType => value.asLong - case FloatType => value.asFloat - case DoubleType => value.asDouble - case d: DecimalType => Decimal(value.asBigDecimal(d.scale)) + case BooleanType => value.asInstanceOf[Boolean] + case ByteType => value.asInstanceOf[Byte] + case ShortType => value.asInstanceOf[Short] +// case IntegerType if value.getClass.toString.equals("class java.lang.Long") => + case IntegerType if value.isInstanceOf[java.lang.Long] => + val v: Integer = Integer.valueOf(value.asInstanceOf[Long].toInt) + v.intValue() + case IntegerType => + value.asInstanceOf[Integer].intValue() + case LongType if value.isInstanceOf[java.math.BigInteger] => + value.asInstanceOf[java.math.BigInteger].longValue() + case LongType => + value.asInstanceOf[Long] + case FloatType => value.asInstanceOf[Float] + case DoubleType => value.asInstanceOf[Double] + case d: DecimalType => + // Java client returns BigInteger for Int256/UInt256, BigDecimal for Decimal types + val dec: BigDecimal = value match { + case bi: java.math.BigInteger => BigDecimal(bi) + case bd: java.math.BigDecimal => BigDecimal(bd) + } + Decimal(dec.setScale(d.scale)) case TimestampType => - var _instant = value.asZonedDateTime.withZoneSameInstant(ZoneOffset.UTC) + var _instant = value.asInstanceOf[ZonedDateTime].withZoneSameInstant(ZoneOffset.UTC) TimeUnit.SECONDS.toMicros(_instant.toEpochSecond) + TimeUnit.NANOSECONDS.toMicros(_instant.getNano()) - case StringType if value.isInstanceOf[ClickHouseStringValue] => UTF8String.fromBytes(value.asBinary) - case StringType => UTF8String.fromString(value.asString) - case DateType => value.asDate.toEpochDay.toInt - case BinaryType => value.asBinary + case StringType => + val strValue = value match { + case uuid: java.util.UUID => uuid.toString + case inet: java.net.InetAddress => inet.getHostAddress + case s: String => s + case enumValue: BinaryStreamReader.EnumValue => enumValue.toString + case _ => value.toString + } + UTF8String.fromString(strValue) + case DateType => + val localDate = value match { + case ld: LocalDate => ld + case zdt: ZonedDateTime => zdt.toLocalDate + case _ => value.asInstanceOf[LocalDate] + } + localDate.toEpochDay.toInt + case BinaryType => value.asInstanceOf[String].getBytes case ArrayType(_dataType, _nullable) => - val arrayValue = value.asInstanceOf[ClickHouseArraySequence] + // Java client returns BinaryStreamReader.ArrayValue for arrays + val arrayVal = value.asInstanceOf[BinaryStreamReader.ArrayValue] + val arrayValue = arrayVal.getArrayOfObjects().toSeq.asInstanceOf[Seq[Object]] val convertedArray = Array.tabulate(arrayValue.length) { i => decodeValue( - arrayValue.getValue(i, createClickHouseValue(null, _dataType)), + arrayValue(i), StructField("element", _dataType, _nullable) ) } new GenericArrayData(convertedArray) case MapType(_keyType, _valueType, _valueNullable) => - val convertedMap = value.asMap().asScala.map { case (rawKey, rawValue) => - val decodedKey = decodeValue(createClickHouseValue(rawKey, _keyType), StructField("key", _keyType, false)) - val decodedValue = - decodeValue(createClickHouseValue(rawValue, _valueType), StructField("value", _valueType, _valueNullable)) - (decodedKey, decodedValue) - } + // Java client returns util.Map (LinkedHashMap or EmptyMap) + val javaMap = value.asInstanceOf[util.Map[Object, Object]] + val convertedMap = + javaMap.asScala.map { case (rawKey, rawValue) => + val decodedKey = decodeValue(rawKey, StructField("key", _keyType, false)) + val decodedValue = + decodeValue(rawValue, StructField("value", _valueType, _valueNullable)) + (decodedKey, decodedValue) + } ArrayBasedMapData(convertedMap) - case _ => throw CHClientException(s"Unsupported catalyst type ${structField.name}[${structField.dataType}]") } } - private def createClickHouseValue(rawValue: Any, dataType: DataType): ClickHouseValue = { - val isNull = rawValue == null - - dataType match { - case StringType => - if (isNull) ClickHouseStringValue.ofNull() - else ClickHouseStringValue.of(rawValue.toString) - - case IntegerType => - if (isNull) ClickHouseIntegerValue.ofNull() - else ClickHouseIntegerValue.of(rawValue.asInstanceOf[Int]) - - case LongType => - if (isNull) ClickHouseLongValue.ofNull() - else ClickHouseLongValue.of(rawValue.asInstanceOf[Long]) - - case DoubleType => - if (isNull) ClickHouseDoubleValue.ofNull() - else ClickHouseDoubleValue.of(rawValue.asInstanceOf[Double]) - - case FloatType => - if (isNull) ClickHouseFloatValue.ofNull() - else ClickHouseFloatValue.of(rawValue.asInstanceOf[Float]) - - case BooleanType => - if (isNull) ClickHouseBoolValue.ofNull() - else ClickHouseBoolValue.of(rawValue.asInstanceOf[Boolean]) - - case _: ArrayType => - if (isNull) ClickHouseArrayValue.ofEmpty() - else ClickHouseArrayValue.of(rawValue.asInstanceOf[Array[Object]]) - - case _: MapType => - if (isNull) ClickHouseMapValue.ofEmpty(classOf[Object], classOf[Object]) - else ClickHouseMapValue.of( - rawValue.asInstanceOf[java.util.Map[Object, Object]], - classOf[Object], - classOf[Object] - ) - - case _ => - if (isNull) ClickHouseStringValue.ofNull() - else ClickHouseStringValue.of(rawValue.toString) - } - } - } diff --git a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala index 2671155f..a13062fb 100644 --- a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala +++ b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala @@ -85,10 +85,15 @@ class ClickHouseJsonReader( TimeUnit.SECONDS.toMicros(_instant.toEpochSecond) + TimeUnit.NANOSECONDS.toMicros(_instant.getNano()) case StringType => UTF8String.fromString(jsonNode.asText) case DateType => LocalDate.parse(jsonNode.asText, dateFmt).toEpochDay.toInt - case BinaryType => jsonNode.binaryValue + case BinaryType if jsonNode.isTextual => + // ClickHouse JSON format returns FixedString as plain text, not Base64 + jsonNode.asText.getBytes("UTF-8") + case BinaryType => + // True binary data is Base64 encoded in JSON format + jsonNode.binaryValue case ArrayType(_dataType, _nullable) => val _structField = StructField(s"${structField.name}__array_element__", _dataType, _nullable) - new GenericArrayData(jsonNode.asScala.map(decodeValue(_, _structField))) + new GenericArrayData(jsonNode.asScala.map(decodeValue(_, _structField)).toArray) case MapType(StringType, _valueType, _valueNullable) => val mapData = jsonNode.fields.asScala.map { entry => val _structField = StructField(s"${structField.name}__map_value__", _valueType, _valueNullable) diff --git a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala index bedd827c..9fc13f5b 100644 --- a/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala +++ b/spark-3.4/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala @@ -251,7 +251,8 @@ abstract class ClickHouseWriter(writeJob: WriteJobDescription) writeJob.writeOptions.retryInterval ) { var startWriteTime = System.currentTimeMillis - client.syncInsertOutputJSONEachRow(database, table, format, codec, new ByteArrayInputStream(data)) match { + // , codec + client.syncInsertOutputJSONEachRow(database, table, format, new ByteArrayInputStream(data)) match { case Right(_) => writeTime = System.currentTimeMillis - startWriteTime _totalWriteTime.add(writeTime) diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/SparkClickHouseClusterTest.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/SparkClickHouseClusterTest.scala index c8de2044..bc91abe4 100644 --- a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/SparkClickHouseClusterTest.scala +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/SparkClickHouseClusterTest.scala @@ -127,7 +127,7 @@ trait SparkClickHouseClusterTest extends SparkTest with ClickHouseClusterMixIn { |) |""".stripMargin ) - + Thread.sleep(3000) if (writeData) { val tblSchema = spark.table(s"$db.$tbl_dist").schema val dataDF = spark.createDataFrame(Seq( diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala index 6f5686fe..a50506d1 100644 --- a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/cluster/TPCDSClusterSuite.scala @@ -38,6 +38,8 @@ class TPCDSClusterSuite extends SparkClickHouseClusterTest { spark.sql("CREATE DATABASE tpcds_sf1_cluster WITH DBPROPERTIES (cluster = 'single_replica')") TPCDSTestUtils.tablePrimaryKeys.foreach { case (table, primaryKeys) => + println(s"before table ${table} ${primaryKeys}") + val start: Long = System.currentTimeMillis() spark.sql( s""" |CREATE TABLE tpcds_sf1_cluster.$table @@ -51,9 +53,11 @@ class TPCDSClusterSuite extends SparkClickHouseClusterTest { |SELECT * FROM tpcds.sf1.$table; |""".stripMargin ) + println(s"time took table ${table} ${System.currentTimeMillis() - start}") } TPCDSTestUtils.tablePrimaryKeys.keys.foreach { table => + println(s"table ${table}") assert(spark.table(s"tpcds.sf1.$table").count === spark.table(s"tpcds_sf1_cluster.$table").count) } } diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala new file mode 100644 index 00000000..721ba948 --- /dev/null +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseArrowWriterSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.ClickHouseSingleMixIn +import org.apache.spark.SparkConf + +class ClickHouseSingleArrowWriterSuite extends ClickHouseArrowWriterSuite with ClickHouseSingleMixIn + +abstract class ClickHouseArrowWriterSuite extends ClickHouseWriterTestBase { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.write.format", "arrow") + .set("spark.clickhouse.read.format", "json") + +} diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala new file mode 100644 index 00000000..decfe0af --- /dev/null +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseBinaryReaderSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.{ClickHouseCloudMixIn, ClickHouseSingleMixIn} +import org.apache.spark.SparkConf +import org.scalatest.tags.Cloud + +@Cloud +class ClickHouseCloudBinaryReaderSuite extends ClickHouseBinaryReaderSuite with ClickHouseCloudMixIn + +class ClickHouseSingleBinaryReaderSuite extends ClickHouseBinaryReaderSuite with ClickHouseSingleMixIn + +/** + * Test suite for ClickHouse Binary Reader. + * Uses binary format for reading data from ClickHouse. + * All test cases are inherited from ClickHouseReaderTestBase. + */ +abstract class ClickHouseBinaryReaderSuite extends ClickHouseReaderTestBase { + + // Override to use binary format for reading + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.read.format", "binary") + .set("spark.clickhouse.write.format", "arrow") + + // All tests are inherited from ClickHouseReaderTestBase + // Additional binary-specific tests can be added here if needed +} diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala new file mode 100644 index 00000000..c62d5564 --- /dev/null +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonReaderSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.{ClickHouseCloudMixIn, ClickHouseSingleMixIn} +import org.scalatest.tags.Cloud + +@Cloud +class ClickHouseCloudJsonReaderSuite extends ClickHouseJsonReaderSuite with ClickHouseCloudMixIn + +class ClickHouseSingleJsonReaderSuite extends ClickHouseJsonReaderSuite with ClickHouseSingleMixIn + +/** + * Test suite for ClickHouse JSON Reader. + * Uses JSON format for reading data from ClickHouse (default in SparkClickHouseSingleTest). + * All test cases are inherited from ClickHouseReaderTestBase. + */ +abstract class ClickHouseJsonReaderSuite extends ClickHouseReaderTestBase { + // Uses JSON format (configured in SparkClickHouseSingleTest) + // All tests are inherited from ClickHouseReaderTestBase + // Additional JSON-specific tests can be added here if needed +} diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala new file mode 100644 index 00000000..3532b140 --- /dev/null +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseJsonWriterSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import com.clickhouse.spark.base.ClickHouseSingleMixIn +import org.apache.spark.SparkConf + +class ClickHouseSingleJsonWriterSuite extends ClickHouseJsonWriterSuite with ClickHouseSingleMixIn + +abstract class ClickHouseJsonWriterSuite extends ClickHouseWriterTestBase { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.clickhouse.write.format", "json") + .set("spark.clickhouse.read.format", "json") + +} diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala new file mode 100644 index 00000000..73e9119f --- /dev/null +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseReaderTestBase.scala @@ -0,0 +1,1331 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import org.apache.spark.sql.Row + +/** + * Shared test cases for both JSON and Binary readers. + * Subclasses only need to configure the read format. + * + * Tests are organized by ClickHouse data type with both regular and nullable variants. + * Each type includes comprehensive coverage of edge cases and null handling. + */ +trait ClickHouseReaderTestBase extends SparkClickHouseSingleTest { + + // ============================================================================ + // ArrayType Tests + // ============================================================================ + + test("decode ArrayType - Array of integers") { + withKVTable("test_db", "test_array_int", valueColDef = "Array(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_int VALUES + |(1, [1, 2, 3]), + |(2, []), + |(3, [100, 200, 300, 400]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_int ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(1).getSeq[Int](1) == Seq()) + assert(result(2).getSeq[Int](1) == Seq(100, 200, 300, 400)) + } + } + test("decode ArrayType - Array of strings") { + withKVTable("test_db", "test_array_string", valueColDef = "Array(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_string VALUES + |(1, ['hello', 'world']), + |(2, []), + |(3, ['a', 'b', 'c']) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_string ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[String](1) == Seq("hello", "world")) + assert(result(1).getSeq[String](1) == Seq()) + assert(result(2).getSeq[String](1) == Seq("a", "b", "c")) + } + } + test("decode ArrayType - Array with nullable elements") { + withKVTable("test_db", "test_array_nullable", valueColDef = "Array(Nullable(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_array_nullable VALUES + |(1, [1, NULL, 3]), + |(2, [NULL, NULL]), + |(3, [100, 200]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_array_nullable ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify arrays can be read + assert(result(0).getSeq[Any](1) != null) + assert(result(1).getSeq[Any](1) != null) + assert(result(2).getSeq[Any](1) != null) + } + } + test("decode ArrayType - empty arrays") { + withKVTable("test_db", "test_empty_array", valueColDef = "Array(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_empty_array VALUES + |(1, []), + |(2, [1, 2, 3]), + |(3, []) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_empty_array ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1).isEmpty) + assert(result(1).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(2).getSeq[Int](1).isEmpty) + } + } + test("decode ArrayType - Nested arrays") { + withKVTable("test_db", "test_nested_array", valueColDef = "Array(Array(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_nested_array VALUES + |(1, [[1, 2], [3, 4]]), + |(2, [[], [5]]), + |(3, [[10, 20, 30]]) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_nested_array ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify nested arrays can be read + assert(result(0).get(1) != null) + assert(result(1).get(1) != null) + assert(result(2).get(1) != null) + } + } + test("decode BinaryType - FixedString") { + // FixedString is read as String by default in the connector + withKVTable("test_db", "test_fixedstring", valueColDef = "FixedString(5)") { + runClickHouseSQL( + """INSERT INTO test_db.test_fixedstring VALUES + |(1, 'hello'), + |(2, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_fixedstring ORDER BY key") + val result = df.collect() + assert(result.length == 2) + // FixedString should be readable + assert(result(0).get(1) != null) + assert(result(1).get(1) != null) + } + } + test("decode BinaryType - FixedString nullable with null values") { + withKVTable("test_db", "test_fixedstring_null", valueColDef = "Nullable(FixedString(5))") { + runClickHouseSQL( + """INSERT INTO test_db.test_fixedstring_null VALUES + |(1, 'hello'), + |(2, NULL), + |(3, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_fixedstring_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).get(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).get(1) != null) + } + } + + // ============================================================================ + // BooleanType Tests + // ============================================================================ + + test("decode BooleanType - true and false values") { + // ClickHouse Bool is stored as UInt8 (0 or 1) + // JSON format reads as Boolean, Binary format reads as Short + withKVTable("test_db", "test_bool", valueColDef = "Bool") { + runClickHouseSQL( + """INSERT INTO test_db.test_bool VALUES + |(1, true), + |(2, false), + |(3, 1), + |(4, 0) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_bool ORDER BY key") + val result = df.collect() + assert(result.length == 4) + // Check the value - handle both Boolean (JSON) and Short (Binary) formats + val v0 = result(0).get(1) + val v1 = result(1).get(1) + v0 match { + case b: Boolean => + assert(b == true) + assert(result(1).getBoolean(1) == false) + assert(result(2).getBoolean(1) == true) + assert(result(3).getBoolean(1) == false) + case s: Short => + assert(s == 1) + assert(result(1).getShort(1) == 0) + assert(result(2).getShort(1) == 1) + assert(result(3).getShort(1) == 0) + case _ => fail(s"Unexpected type: ${v0.getClass}") + } + } + } + test("decode BooleanType - nullable with null values") { + withKVTable("test_db", "test_bool_null", valueColDef = "Nullable(Bool)") { + runClickHouseSQL( + """INSERT INTO test_db.test_bool_null VALUES + |(1, true), + |(2, NULL), + |(3, false) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_bool_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(1).isNullAt(1)) + // Check the value - handle both Boolean (JSON) and Short (Binary) formats + val v0 = result(0).get(1) + v0 match { + case b: Boolean => + assert(b == true) + assert(result(2).getBoolean(1) == false) + case s: Short => + assert(s == 1) + assert(result(2).getShort(1) == 0) + case _ => fail(s"Unexpected type: ${v0.getClass}") + } + } + } + + // ============================================================================ + // ByteType Tests + // ============================================================================ + + test("decode ByteType - min and max values") { + withKVTable("test_db", "test_byte", valueColDef = "Int8") { + runClickHouseSQL( + """INSERT INTO test_db.test_byte VALUES + |(1, -128), + |(2, 0), + |(3, 127) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_byte ORDER BY key") + checkAnswer( + df, + Row(1, -128.toByte) :: Row(2, 0.toByte) :: Row(3, 127.toByte) :: Nil + ) + } + } + test("decode ByteType - nullable with null values") { + withKVTable("test_db", "test_byte_null", valueColDef = "Nullable(Int8)") { + runClickHouseSQL( + """INSERT INTO test_db.test_byte_null VALUES + |(1, -128), + |(2, NULL), + |(3, 127) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_byte_null ORDER BY key") + checkAnswer( + df, + Row(1, -128.toByte) :: Row(2, null) :: Row(3, 127.toByte) :: Nil + ) + } + } + test("decode DateTime32 - 32-bit timestamp") { + withKVTable("test_db", "test_datetime32", valueColDef = "DateTime32") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime32 VALUES + |(1, '2024-01-01 12:00:00'), + |(2, '2024-06-15 18:30:45') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime32 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + } + } + test("decode DateTime32 - nullable with null values") { + withKVTable("test_db", "test_datetime32_null", valueColDef = "Nullable(DateTime32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime32_null VALUES + |(1, '2024-01-01 12:00:00'), + |(2, NULL), + |(3, '2024-06-15 18:30:45') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode DateType - Date") { + withKVTable("test_db", "test_date", valueColDef = "Date") { + runClickHouseSQL( + """INSERT INTO test_db.test_date VALUES + |(1, '2024-01-01'), + |(2, '2024-06-15'), + |(3, '2024-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - Date32") { + withKVTable("test_db", "test_date32", valueColDef = "Date32") { + runClickHouseSQL( + """INSERT INTO test_db.test_date32 VALUES + |(1, '1900-01-01'), + |(2, '2024-06-15'), + |(3, '2100-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date32 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - Date32 nullable with null values") { + withKVTable("test_db", "test_date32_null", valueColDef = "Nullable(Date32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_date32_null VALUES + |(1, '1900-01-01'), + |(2, NULL), + |(3, '2100-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + test("decode DateType - nullable with null values") { + withKVTable("test_db", "test_date_null", valueColDef = "Nullable(Date)") { + runClickHouseSQL( + """INSERT INTO test_db.test_date_null VALUES + |(1, '2024-01-01'), + |(2, NULL), + |(3, '2024-12-31') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_date_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + test("decode DecimalType - Decimal128") { + // Decimal128(20) means scale=20, max precision=38 total digits + // Use values with max 18 digits before decimal to stay within 38 total + withKVTable("test_db", "test_decimal128", valueColDef = "Decimal128(20)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal128 VALUES + |(1, 123456789012345.12345678901234567890), + |(2, -999999999999999.99999999999999999999), + |(3, 0.00000000000000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal128 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Decimal128(20) means 20 decimal places, total precision up to 38 digits + assert(math.abs(result(0).getDecimal(1).doubleValue() - 123456789012345.12345678901234567890) < 0.01) + assert(math.abs(result(1).getDecimal(1).doubleValue() - -999999999999999.99999999999999999999) < 0.01) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal128 nullable with null values") { + withKVTable("test_db", "test_decimal128_null", valueColDef = "Nullable(Decimal128(20))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal128_null VALUES + |(1, 123456789012345.12345678901234567890), + |(2, NULL), + |(3, 0.00000000000000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal32") { + withKVTable("test_db", "test_decimal32", valueColDef = "Decimal32(4)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal32 VALUES + |(1, 12345.6789), + |(2, -9999.9999), + |(3, 0.0001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal32 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).doubleValue() == 12345.6789) + assert(result(1).getDecimal(1).doubleValue() == -9999.9999) + assert(result(2).getDecimal(1).doubleValue() == 0.0001) + } + } + test("decode DecimalType - Decimal32 nullable with null values") { + withKVTable("test_db", "test_decimal32_null", valueColDef = "Nullable(Decimal32(4))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal32_null VALUES + |(1, 12345.6789), + |(2, NULL), + |(3, 0.0001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal32_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DecimalType - Decimal64") { + // Decimal64(10) means scale=10, max precision=18 total digits + // Use values with max 8 digits before decimal to stay within 18 total + withKVTable("test_db", "test_decimal64", valueColDef = "Decimal64(10)") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal64 VALUES + |(1, 1234567.0123456789), + |(2, -9999999.9999999999), + |(3, 0.0000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDecimal(1).doubleValue() - 1234567.0123456789) < 0.0001) + assert(math.abs(result(1).getDecimal(1).doubleValue() - -9999999.9999999999) < 0.0001) + assert(math.abs(result(2).getDecimal(1).doubleValue() - 0.0000000001) < 0.0000000001) + } + } + test("decode DecimalType - Decimal64 nullable with null values") { + withKVTable("test_db", "test_decimal64_null", valueColDef = "Nullable(Decimal64(10))") { + runClickHouseSQL( + """INSERT INTO test_db.test_decimal64_null VALUES + |(1, 1234567.0123456789), + |(2, NULL), + |(3, 0.0000000001) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_decimal64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode DoubleType - nullable with null values") { + withKVTable("test_db", "test_double_null", valueColDef = "Nullable(Float64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_double_null VALUES + |(1, 1.23), + |(2, NULL), + |(3, -4.56) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_double_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 1.23) < 0.0001) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getDouble(1) - -4.56) < 0.0001) + } + } + test("decode DoubleType - regular values") { + withKVTable("test_db", "test_double", valueColDef = "Float64") { + runClickHouseSQL( + """INSERT INTO test_db.test_double VALUES + |(1, -3.141592653589793), + |(2, 0.0), + |(3, 3.141592653589793) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_double ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - -3.141592653589793) < 0.000001) + assert(result(1).getDouble(1) == 0.0) + assert(math.abs(result(2).getDouble(1) - 3.141592653589793) < 0.000001) + } + } + test("decode Enum16 - large enum") { + withKVTable("test_db", "test_enum16", valueColDef = "Enum16('small' = 1, 'medium' = 100, 'large' = 1000)") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum16 VALUES + |(1, 'small'), + |(2, 'medium'), + |(3, 'large') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum16 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "small") + assert(result(1).getString(1) == "medium") + assert(result(2).getString(1) == "large") + } + } + test("decode Enum16 - nullable with null values") { + withKVTable( + "test_db", + "test_enum16_null", + valueColDef = "Nullable(Enum16('small' = 1, 'medium' = 100, 'large' = 1000))" + ) { + runClickHouseSQL( + """INSERT INTO test_db.test_enum16_null VALUES + |(1, 'small'), + |(2, NULL), + |(3, 'large') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum16_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "small") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "large") + } + } + test("decode Enum8 - nullable with null values") { + withKVTable("test_db", "test_enum8_null", valueColDef = "Nullable(Enum8('red' = 1, 'green' = 2, 'blue' = 3))") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum8_null VALUES + |(1, 'red'), + |(2, NULL), + |(3, 'blue') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum8_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "red") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "blue") + } + } + test("decode Enum8 - small enum") { + withKVTable("test_db", "test_enum8", valueColDef = "Enum8('red' = 1, 'green' = 2, 'blue' = 3)") { + runClickHouseSQL( + """INSERT INTO test_db.test_enum8 VALUES + |(1, 'red'), + |(2, 'green'), + |(3, 'blue') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_enum8 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "red") + assert(result(1).getString(1) == "green") + assert(result(2).getString(1) == "blue") + } + } + test("decode FloatType - nullable with null values") { + withKVTable("test_db", "test_float_null", valueColDef = "Nullable(Float32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_float_null VALUES + |(1, 1.5), + |(2, NULL), + |(3, -2.5) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_float_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 1.5f) < 0.01f) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getFloat(1) - -2.5f) < 0.01f) + } + } + test("decode FloatType - regular values") { + withKVTable("test_db", "test_float", valueColDef = "Float32") { + runClickHouseSQL( + """INSERT INTO test_db.test_float VALUES + |(1, -3.14), + |(2, 0.0), + |(3, 3.14) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_float ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - -3.14f) < 0.01f) + assert(result(1).getFloat(1) == 0.0f) + assert(math.abs(result(2).getFloat(1) - 3.14f) < 0.01f) + } + } + test("decode Int128 - large integers as Decimal") { + withKVTable("test_db", "test_int128", valueColDef = "Int128") { + runClickHouseSQL( + """INSERT INTO test_db.test_int128 VALUES + |(1, 0), + |(2, 123456789012345678901234567890), + |(3, -123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int128 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int128 - nullable with null values") { + withKVTable("test_db", "test_int128_null", valueColDef = "Nullable(Int128)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int128_null VALUES + |(1, 0), + |(2, NULL), + |(3, -123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int256 - nullable with null values") { + withKVTable("test_db", "test_int256_null", valueColDef = "Nullable(Int256)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int256_null VALUES + |(1, 0), + |(2, NULL), + |(3, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int256_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode Int256 - very large integers as Decimal") { + withKVTable("test_db", "test_int256", valueColDef = "Int256") { + runClickHouseSQL( + """INSERT INTO test_db.test_int256 VALUES + |(1, 0), + |(2, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int256 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode IntegerType - min and max values") { + withKVTable("test_db", "test_int", valueColDef = "Int32") { + runClickHouseSQL( + """INSERT INTO test_db.test_int VALUES + |(1, -2147483648), + |(2, 0), + |(3, 2147483647) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int ORDER BY key") + checkAnswer( + df, + Row(1, -2147483648) :: Row(2, 0) :: Row(3, 2147483647) :: Nil + ) + } + } + test("decode IntegerType - nullable with null values") { + withKVTable("test_db", "test_int_null", valueColDef = "Nullable(Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_int_null VALUES + |(1, -2147483648), + |(2, NULL), + |(3, 2147483647) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_int_null ORDER BY key") + checkAnswer( + df, + Row(1, -2147483648) :: Row(2, null) :: Row(3, 2147483647) :: Nil + ) + } + } + test("decode IPv4 - IP addresses") { + withKVTable("test_db", "test_ipv4", valueColDef = "IPv4") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv4 VALUES + |(1, '127.0.0.1'), + |(2, '192.168.1.1'), + |(3, '8.8.8.8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv4 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "127.0.0.1") + assert(result(1).getString(1) == "192.168.1.1") + assert(result(2).getString(1) == "8.8.8.8") + } + } + test("decode IPv4 - nullable with null values") { + withKVTable("test_db", "test_ipv4_null", valueColDef = "Nullable(IPv4)") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv4_null VALUES + |(1, '127.0.0.1'), + |(2, NULL), + |(3, '8.8.8.8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv4_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "127.0.0.1") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "8.8.8.8") + } + } + test("decode IPv6 - IPv6 addresses") { + withKVTable("test_db", "test_ipv6", valueColDef = "IPv6") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv6 VALUES + |(1, '::1'), + |(2, '2001:0db8:85a3:0000:0000:8a2e:0370:7334'), + |(3, 'fe80::1') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv6 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) != null) + assert(result(1).getString(1) != null) + assert(result(2).getString(1) != null) + } + } + test("decode IPv6 - nullable with null values") { + withKVTable("test_db", "test_ipv6_null", valueColDef = "Nullable(IPv6)") { + runClickHouseSQL( + """INSERT INTO test_db.test_ipv6_null VALUES + |(1, '::1'), + |(2, NULL), + |(3, 'fe80::1') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_ipv6_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) != null) + } + } + test("decode JSON - nullable with null values") { + withKVTable("test_db", "test_json_null", valueColDef = "Nullable(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_json_null VALUES + |(1, '{"name": "Alice", "age": 30}'), + |(2, NULL), + |(3, '{"name": "Charlie", "age": 35}') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_json_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1).contains("Alice")) + assert(result(1).isNullAt(1)) + assert(result(2).getString(1).contains("Charlie")) + } + } + test("decode JSON - semi-structured data") { + withKVTable("test_db", "test_json", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_json VALUES + |(1, '{"name": "Alice", "age": 30}'), + |(2, '{"name": "Bob", "age": 25}'), + |(3, '{"name": "Charlie", "age": 35}') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_json ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1).contains("Alice")) + assert(result(1).getString(1).contains("Bob")) + assert(result(2).getString(1).contains("Charlie")) + } + } + test("decode LongType - min and max values") { + withKVTable("test_db", "test_long", valueColDef = "Int64") { + runClickHouseSQL( + """INSERT INTO test_db.test_long VALUES + |(1, -9223372036854775808), + |(2, 0), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long ORDER BY key") + checkAnswer( + df, + Row(1, -9223372036854775808L) :: Row(2, 0L) :: Row(3, 9223372036854775807L) :: Nil + ) + } + } + test("decode LongType - nullable with null values") { + withKVTable("test_db", "test_long_null", valueColDef = "Nullable(Int64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_long_null VALUES + |(1, -9223372036854775808), + |(2, NULL), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long_null ORDER BY key") + checkAnswer( + df, + Row(1, -9223372036854775808L) :: Row(2, null) :: Row(3, 9223372036854775807L) :: Nil + ) + } + } + test("decode LongType - UInt32 nullable with null values") { + withKVTable("test_db", "test_uint32_null", valueColDef = "Nullable(UInt32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint32_null VALUES + |(1, 0), + |(2, NULL), + |(3, 4294967295) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint32_null ORDER BY key") + checkAnswer( + df, + Row(1, 0L) :: Row(2, null) :: Row(3, 4294967295L) :: Nil + ) + } + } + test("decode LongType - UInt32 values") { + withKVTable("test_db", "test_uint32", valueColDef = "UInt32") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint32 VALUES + |(1, 0), + |(2, 2147483648), + |(3, 4294967295) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint32 ORDER BY key") + checkAnswer( + df, + Row(1, 0L) :: Row(2, 2147483648L) :: Row(3, 4294967295L) :: Nil + ) + } + } + test("decode MapType - Map of String to Int") { + withKVTable("test_db", "test_map", valueColDef = "Map(String, Int32)") { + runClickHouseSQL( + """INSERT INTO test_db.test_map VALUES + |(1, {'a': 1, 'b': 2}), + |(2, {}), + |(3, {'x': 100, 'y': 200, 'z': 300}) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_map ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1) == Map("a" -> 1, "b" -> 2)) + assert(result(1).getMap[String, Int](1) == Map()) + assert(result(2).getMap[String, Int](1) == Map("x" -> 100, "y" -> 200, "z" -> 300)) + } + } + test("decode MapType - Map with nullable values") { + withKVTable("test_db", "test_map_nullable", valueColDef = "Map(String, Nullable(Int32))") { + runClickHouseSQL( + """INSERT INTO test_db.test_map_nullable VALUES + |(1, {'a': 1, 'b': NULL}), + |(2, {'x': NULL}), + |(3, {'p': 100, 'q': 200}) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_map_nullable ORDER BY key") + val result = df.collect() + assert(result.length == 3) + // Verify maps can be read + assert(result(0).getMap[String, Any](1) != null) + assert(result(1).getMap[String, Any](1) != null) + assert(result(2).getMap[String, Any](1) != null) + } + } + test("decode ShortType - min and max values") { + withKVTable("test_db", "test_short", valueColDef = "Int16") { + runClickHouseSQL( + """INSERT INTO test_db.test_short VALUES + |(1, -32768), + |(2, 0), + |(3, 32767) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_short ORDER BY key") + checkAnswer( + df, + Row(1, -32768.toShort) :: Row(2, 0.toShort) :: Row(3, 32767.toShort) :: Nil + ) + } + } + test("decode ShortType - nullable with null values") { + withKVTable("test_db", "test_short_null", valueColDef = "Nullable(Int16)") { + runClickHouseSQL( + """INSERT INTO test_db.test_short_null VALUES + |(1, -32768), + |(2, NULL), + |(3, 32767) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_short_null ORDER BY key") + checkAnswer( + df, + Row(1, -32768.toShort) :: Row(2, null) :: Row(3, 32767.toShort) :: Nil + ) + } + } + test("decode ShortType - UInt8 nullable with null values") { + withKVTable("test_db", "test_uint8_null", valueColDef = "Nullable(UInt8)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint8_null VALUES + |(1, 0), + |(2, NULL), + |(3, 255) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint8_null ORDER BY key") + checkAnswer( + df, + Row(1, 0.toShort) :: Row(2, null) :: Row(3, 255.toShort) :: Nil + ) + } + } + test("decode ShortType - UInt8 values") { + withKVTable("test_db", "test_uint8", valueColDef = "UInt8") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint8 VALUES + |(1, 0), + |(2, 128), + |(3, 255) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint8 ORDER BY key") + checkAnswer( + df, + Row(1, 0.toShort) :: Row(2, 128.toShort) :: Row(3, 255.toShort) :: Nil + ) + } + } + test("decode StringType - empty strings") { + withKVTable("test_db", "test_empty_string", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_empty_string VALUES + |(1, ''), + |(2, 'not empty'), + |(3, '') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_empty_string ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "") + assert(result(1).getString(1) == "not empty") + assert(result(2).getString(1) == "") + } + } + test("decode StringType - nullable with null values") { + withKVTable("test_db", "test_string_null", valueColDef = "Nullable(String)") { + runClickHouseSQL( + """INSERT INTO test_db.test_string_null VALUES + |(1, 'hello'), + |(2, NULL), + |(3, 'world') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_string_null ORDER BY key") + checkAnswer( + df, + Row(1, "hello") :: Row(2, null) :: Row(3, "world") :: Nil + ) + } + } + test("decode StringType - regular strings") { + withKVTable("test_db", "test_string", valueColDef = "String") { + runClickHouseSQL( + """INSERT INTO test_db.test_string VALUES + |(1, 'hello'), + |(2, ''), + |(3, 'world with spaces'), + |(4, 'special chars: !@#$%^&*()') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_string ORDER BY key") + checkAnswer( + df, + Row(1, "hello") :: Row(2, "") :: Row(3, "world with spaces") :: Row(4, "special chars: !@#$%^&*()") :: Nil + ) + } + } + test("decode StringType - UUID") { + withKVTable("test_db", "test_uuid", valueColDef = "UUID") { + runClickHouseSQL( + """INSERT INTO test_db.test_uuid VALUES + |(1, '550e8400-e29b-41d4-a716-446655440000'), + |(2, '6ba7b810-9dad-11d1-80b4-00c04fd430c8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uuid ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getString(1) == "550e8400-e29b-41d4-a716-446655440000") + assert(result(1).getString(1) == "6ba7b810-9dad-11d1-80b4-00c04fd430c8") + } + } + test("decode StringType - UUID nullable with null values") { + withKVTable("test_db", "test_uuid_null", valueColDef = "Nullable(UUID)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uuid_null VALUES + |(1, '550e8400-e29b-41d4-a716-446655440000'), + |(2, NULL), + |(3, '6ba7b810-9dad-11d1-80b4-00c04fd430c8') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uuid_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getString(1) == "550e8400-e29b-41d4-a716-446655440000") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "6ba7b810-9dad-11d1-80b4-00c04fd430c8") + } + } + test("decode StringType - very long strings") { + val longString = "a" * 10000 + withKVTable("test_db", "test_long_string", valueColDef = "String") { + runClickHouseSQL( + s"""INSERT INTO test_db.test_long_string VALUES + |(1, '$longString') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_long_string ORDER BY key") + val result = df.collect() + assert(result.length == 1) + assert(result(0).getString(1).length == 10000) + } + } + test("decode TimestampType - DateTime") { + withKVTable("test_db", "test_datetime", valueColDef = "DateTime") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime VALUES + |(1, '2024-01-01 00:00:00'), + |(2, '2024-06-15 12:30:45'), + |(3, '2024-12-31 23:59:59') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - DateTime64") { + withKVTable("test_db", "test_datetime64", valueColDef = "DateTime64(3)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime64 VALUES + |(1, '2024-01-01 00:00:00.123'), + |(2, '2024-06-15 12:30:45.456'), + |(3, '2024-12-31 23:59:59.999') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - DateTime64 nullable with null values") { + withKVTable("test_db", "test_datetime64_null", valueColDef = "Nullable(DateTime64(3))") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime64_null VALUES + |(1, '2024-01-01 00:00:00.123'), + |(2, NULL), + |(3, '2024-12-31 23:59:59.999') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode TimestampType - nullable with null values") { + withKVTable("test_db", "test_datetime_null", valueColDef = "Nullable(DateTime)") { + runClickHouseSQL( + """INSERT INTO test_db.test_datetime_null VALUES + |(1, '2024-01-01 00:00:00'), + |(2, NULL), + |(3, '2024-12-31 23:59:59') + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_datetime_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + test("decode UInt128 - large unsigned integers as Decimal") { + withKVTable("test_db", "test_uint128", valueColDef = "UInt128") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint128 VALUES + |(1, 0), + |(2, 123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint128 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode UInt128 - nullable with null values") { + withKVTable("test_db", "test_uint128_null", valueColDef = "Nullable(UInt128)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint128_null VALUES + |(1, 0), + |(2, NULL), + |(3, 123456789012345678901234567890) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint128_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode UInt16 - nullable with null values") { + withKVTable("test_db", "test_uint16_null", valueColDef = "Nullable(UInt16)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint16_null VALUES + |(1, 0), + |(2, NULL), + |(3, 65535) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint16_null ORDER BY key") + checkAnswer( + df, + Row(1, 0) :: Row(2, null) :: Row(3, 65535) :: Nil + ) + } + } + test("decode UInt16 - unsigned 16-bit integers") { + withKVTable("test_db", "test_uint16", valueColDef = "UInt16") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint16 VALUES + |(1, 0), + |(2, 32768), + |(3, 65535) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint16 ORDER BY key") + checkAnswer( + df, + Row(1, 0) :: Row(2, 32768) :: Row(3, 65535) :: Nil + ) + } + } + test("decode UInt256 - nullable with null values") { + withKVTable("test_db", "test_uint256_null", valueColDef = "Nullable(UInt256)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint256_null VALUES + |(1, 0), + |(2, NULL), + |(3, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint256_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) != null) + } + } + test("decode UInt256 - very large unsigned integers as Decimal") { + withKVTable("test_db", "test_uint256", valueColDef = "UInt256") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint256 VALUES + |(1, 0), + |(2, 12345678901234567890123456789012345678) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint256 ORDER BY key") + val result = df.collect() + assert(result.length == 2) + assert(result(0).getDecimal(1).toBigInteger.longValue == 0L) + assert(result(1).getDecimal(1) != null) + } + } + test("decode UInt64 - nullable with null values") { + withKVTable("test_db", "test_uint64_null", valueColDef = "Nullable(UInt64)") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint64_null VALUES + |(1, 0), + |(2, NULL), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint64_null ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getLong(1) == 0L) + assert(result(1).isNullAt(1)) + assert(result(2).getLong(1) == 9223372036854775807L) + } + } + test("decode UInt64 - unsigned 64-bit integers") { + withKVTable("test_db", "test_uint64", valueColDef = "UInt64") { + runClickHouseSQL( + """INSERT INTO test_db.test_uint64 VALUES + |(1, 0), + |(2, 1234567890), + |(3, 9223372036854775807) + |""".stripMargin + ) + + val df = spark.sql("SELECT key, value FROM test_db.test_uint64 ORDER BY key") + val result = df.collect() + assert(result.length == 3) + assert(result(0).getLong(1) == 0L) + assert(result(1).getLong(1) == 1234567890L) + // Max value that fits in signed Long + assert(result(2).getLong(1) == 9223372036854775807L) + } + } + +} diff --git a/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala new file mode 100644 index 00000000..28267dc2 --- /dev/null +++ b/spark-3.5/clickhouse-spark-it/src/test/scala/org/apache/spark/sql/clickhouse/single/ClickHouseWriterTestBase.scala @@ -0,0 +1,758 @@ +/* + * Licensed 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 + * + * https://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 org.apache.spark.sql.clickhouse.single + +import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.types._ + +/** + * Shared test cases for both JSON and Binary writers. + * Subclasses only need to configure the write format. + */ +trait ClickHouseWriterTestBase extends SparkClickHouseSingleTest { + + test("write ArrayType - array of integers") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_array_int", schema) { + val data = Seq( + Row(1, Seq(1, 2, 3)), + Row(2, Seq(10, 20, 30)), + Row(3, Seq(100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_array_int") + + val result = spark.table("test_db.test_write_array_int").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(1).getSeq[Int](1) == Seq(10, 20, 30)) + assert(result(2).getSeq[Int](1) == Seq(100)) + } + } + + test("write ArrayType - empty arrays") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_empty_array", schema) { + val data = Seq( + Row(1, Seq()), + Row(2, Seq(1, 2, 3)), + Row(3, Seq()) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_array") + + val result = spark.table("test_db.test_write_empty_array").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getSeq[Int](1).isEmpty) + assert(result(1).getSeq[Int](1) == Seq(1, 2, 3)) + assert(result(2).getSeq[Int](1).isEmpty) + } + } + + test("write ArrayType - nested arrays") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField( + "value", + ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = false), + nullable = false + ) + )) + + withTable("test_db", "test_write_nested_array", schema) { + val data = Seq( + Row(1, Seq(Seq(1, 2), Seq(3, 4))), + Row(2, Seq(Seq(10, 20, 30))), + Row(3, Seq(Seq(), Seq(100))) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_nested_array") + + val result = spark.table("test_db.test_write_nested_array").orderBy("id").collect() + assert(result.length == 3) + // Convert to List for Scala 2.12/2.13 compatibility + val row0 = result(0).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + val row1 = result(1).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + val row2 = result(2).getAs[scala.collection.Seq[scala.collection.Seq[Int]]](1).map(_.toList).toList + assert(row0 == Seq(Seq(1, 2), Seq(3, 4))) + assert(row1 == Seq(Seq(10, 20, 30))) + assert(row2(0).isEmpty) + assert(row2(1) == Seq(100)) + } + } + + test("write ArrayType - with nullable elements") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ArrayType(IntegerType, containsNull = true), nullable = false) + )) + + withTable("test_db", "test_write_array_nullable", schema) { + val data = Seq( + Row(1, Seq(1, null, 3)), + Row(2, Seq(null, null)), + Row(3, Seq(10, 20, 30)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_array_nullable") + + val result = spark.table("test_db.test_write_array_nullable").orderBy("id").collect() + assert(result.length == 3) + val arr1 = result(0).getSeq[Any](1) + assert(arr1.length == 3) + assert(arr1(0) == 1) + assert(arr1(1) == null) + assert(arr1(2) == 3) + } + } + + test("write BooleanType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", BooleanType, nullable = true) + )) + + withTable("test_db", "test_write_bool_null", schema) { + val data = Seq( + Row(1, true), + Row(2, null), + Row(3, false) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_bool_null") + + val result = spark.table("test_db.test_write_bool_null").orderBy("id").collect() + assert(result.length == 3) + // Boolean is stored as UInt8 in ClickHouse, reads back as Short + assert(result(0).getShort(1) == 1) + assert(result(1).isNullAt(1)) + assert(result(2).getShort(1) == 0) + } + } + + // NOTE: ClickHouse stores Boolean as UInt8, so it reads back as Short (0 or 1) + test("write BooleanType - true and false values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", BooleanType, nullable = false) + )) + + withTable("test_db", "test_write_bool", schema) { + val data = Seq( + Row(1, true), + Row(2, false) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_bool") + + val result = spark.table("test_db.test_write_bool").orderBy("id").collect() + assert(result.length == 2) + // Boolean is stored as UInt8 in ClickHouse, reads back as Short + assert(result(0).getShort(1) == 1) + assert(result(1).getShort(1) == 0) + } + } + + test("write ByteType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ByteType, nullable = false) + )) + + withTable("test_db", "test_write_byte", schema) { + val data = Seq( + Row(1, Byte.MinValue), + Row(2, 0.toByte), + Row(3, Byte.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_byte") + + val result = spark.table("test_db.test_write_byte").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getByte(1) == Byte.MinValue) + assert(result(1).getByte(1) == 0.toByte) + assert(result(2).getByte(1) == Byte.MaxValue) + } + } + + test("write ByteType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ByteType, nullable = true) + )) + + withTable("test_db", "test_write_byte_null", schema) { + val data = Seq( + Row(1, Byte.MinValue), + Row(2, null), + Row(3, Byte.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_byte_null") + + val result = spark.table("test_db.test_write_byte_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getByte(1) == Byte.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getByte(1) == Byte.MaxValue) + } + } + + test("write DateType - dates") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DateType, nullable = false) + )) + + withTable("test_db", "test_write_date", schema) { + val data = Seq( + Row(1, java.sql.Date.valueOf("2024-01-01")), + Row(2, java.sql.Date.valueOf("2024-06-15")), + Row(3, java.sql.Date.valueOf("2024-12-31")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_date") + + val result = spark.table("test_db.test_write_date").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).getDate(1) != null) + assert(result(2).getDate(1) != null) + } + } + + test("write DateType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DateType, nullable = true) + )) + + withTable("test_db", "test_write_date_null", schema) { + val data = Seq( + Row(1, java.sql.Date.valueOf("2024-01-01")), + Row(2, null), + Row(3, java.sql.Date.valueOf("2024-12-31")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_date_null") + + val result = spark.table("test_db.test_write_date_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDate(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getDate(1) != null) + } + } + + test("write DecimalType - Decimal(10,2)") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(10, 2), nullable = false) + )) + + withTable("test_db", "test_write_decimal", schema) { + val data = Seq( + Row(1, BigDecimal("12345.67")), + Row(2, BigDecimal("-9999.99")), + Row(3, BigDecimal("0.01")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal") + + val result = spark.table("test_db.test_write_decimal").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) == BigDecimal("12345.67").underlying()) + assert(result(1).getDecimal(1) == BigDecimal("-9999.99").underlying()) + assert(result(2).getDecimal(1) == BigDecimal("0.01").underlying()) + } + } + + test("write DecimalType - Decimal(18,4)") { + // Note: High-precision decimals (>15-17 significant digits) may lose precision in JSON/Arrow formats. + // This appears to be related to the serialization/deserialization path, possibly due to intermediate + // double conversions in the format parsers. This test uses tolerance-based assertions to account + // for this observed behavior. Binary format (RowBinaryWithNamesAndTypes) preserves full precision. + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(18, 4), nullable = false) + )) + + withTable("test_db", "test_write_decimal_18_4", schema) { + val data = Seq( + Row(1, BigDecimal("12345678901234.5678")), + Row(2, BigDecimal("-9999999999999.9999")), + Row(3, BigDecimal("0.0001")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal_18_4") + + val result = spark.table("test_db.test_write_decimal_18_4").orderBy("id").collect() + assert(result.length == 3) + // Use tolerance for high-precision values (18 significant digits) + val tolerance = BigDecimal("0.001") + assert((BigDecimal(result(0).getDecimal(1)) - BigDecimal("12345678901234.5678")).abs < tolerance) + assert((BigDecimal(result(1).getDecimal(1)) - BigDecimal("-9999999999999.9999")).abs < tolerance) + // Small values should be exact + assert(result(2).getDecimal(1) == BigDecimal("0.0001").underlying()) + } + } + + test("write DecimalType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DecimalType(10, 2), nullable = true) + )) + + withTable("test_db", "test_write_decimal_null", schema) { + val data = Seq( + Row(1, BigDecimal("12345.67")), + Row(2, null), + Row(3, BigDecimal("-9999.99")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_decimal_null") + + val result = spark.table("test_db.test_write_decimal_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getDecimal(1) == BigDecimal("12345.67").underlying()) + assert(result(1).isNullAt(1)) + assert(result(2).getDecimal(1) == BigDecimal("-9999.99").underlying()) + } + } + + test("write DoubleType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DoubleType, nullable = true) + )) + + withTable("test_db", "test_write_double_null", schema) { + val data = Seq( + Row(1, 3.14159), + Row(2, null), + Row(3, -2.71828) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_double_null") + + val result = spark.table("test_db.test_write_double_null").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 3.14159) < 0.00001) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getDouble(1) - -2.71828) < 0.00001) + } + } + + test("write DoubleType - regular values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", DoubleType, nullable = false) + )) + + withTable("test_db", "test_write_double", schema) { + val data = Seq( + Row(1, 3.14159), + Row(2, -2.71828), + Row(3, 0.0) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_double") + + val result = spark.table("test_db.test_write_double").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getDouble(1) - 3.14159) < 0.00001) + assert(math.abs(result(1).getDouble(1) - -2.71828) < 0.00001) + assert(result(2).getDouble(1) == 0.0) + } + } + + test("write FloatType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", FloatType, nullable = true) + )) + + withTable("test_db", "test_write_float_null", schema) { + val data = Seq( + Row(1, 3.14f), + Row(2, null), + Row(3, -2.718f) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_float_null") + + val result = spark.table("test_db.test_write_float_null").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 3.14f) < 0.001f) + assert(result(1).isNullAt(1)) + assert(math.abs(result(2).getFloat(1) - -2.718f) < 0.001f) + } + } + + test("write FloatType - regular values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", FloatType, nullable = false) + )) + + withTable("test_db", "test_write_float", schema) { + val data = Seq( + Row(1, 3.14f), + Row(2, -2.718f), + Row(3, 0.0f) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_float") + + val result = spark.table("test_db.test_write_float").orderBy("id").collect() + assert(result.length == 3) + assert(math.abs(result(0).getFloat(1) - 3.14f) < 0.001f) + assert(math.abs(result(1).getFloat(1) - -2.718f) < 0.001f) + assert(result(2).getFloat(1) == 0.0f) + } + } + + test("write IntegerType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", IntegerType, nullable = false) + )) + + withTable("test_db", "test_write_int", schema) { + val data = Seq( + Row(1, Int.MinValue), + Row(2, 0), + Row(3, Int.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_int") + + val result = spark.table("test_db.test_write_int").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getInt(1) == Int.MinValue) + assert(result(1).getInt(1) == 0) + assert(result(2).getInt(1) == Int.MaxValue) + } + } + + test("write IntegerType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", IntegerType, nullable = true) + )) + + withTable("test_db", "test_write_int_null", schema) { + val data = Seq( + Row(1, Int.MinValue), + Row(2, null), + Row(3, Int.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_int_null") + + val result = spark.table("test_db.test_write_int_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getInt(1) == Int.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getInt(1) == Int.MaxValue) + } + } + + test("write LongType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", LongType, nullable = false) + )) + + withTable("test_db", "test_write_long", schema) { + val data = Seq( + Row(1, Long.MinValue), + Row(2, 0L), + Row(3, Long.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_long") + + val result = spark.table("test_db.test_write_long").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getLong(1) == Long.MinValue) + assert(result(1).getLong(1) == 0L) + assert(result(2).getLong(1) == Long.MaxValue) + } + } + + test("write LongType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", LongType, nullable = true) + )) + + withTable("test_db", "test_write_long_null", schema) { + val data = Seq( + Row(1, Long.MinValue), + Row(2, null), + Row(3, Long.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_long_null") + + val result = spark.table("test_db.test_write_long_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getLong(1) == Long.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getLong(1) == Long.MaxValue) + } + } + + test("write MapType - empty maps") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_empty_map", schema) { + val data = Seq( + Row(1, Map[String, Int]()), + Row(2, Map("a" -> 1)), + Row(3, Map[String, Int]()) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_map") + + val result = spark.table("test_db.test_write_empty_map").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1).isEmpty) + assert(result(1).getMap[String, Int](1) == Map("a" -> 1)) + assert(result(2).getMap[String, Int](1).isEmpty) + } + } + + test("write MapType - map of string to int") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = false), nullable = false) + )) + + withTable("test_db", "test_write_map", schema) { + val data = Seq( + Row(1, Map("a" -> 1, "b" -> 2)), + Row(2, Map("x" -> 10, "y" -> 20)), + Row(3, Map("foo" -> 100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_map") + + val result = spark.table("test_db.test_write_map").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getMap[String, Int](1) == Map("a" -> 1, "b" -> 2)) + assert(result(1).getMap[String, Int](1) == Map("x" -> 10, "y" -> 20)) + assert(result(2).getMap[String, Int](1) == Map("foo" -> 100)) + } + } + + test("write MapType - with nullable values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", MapType(StringType, IntegerType, valueContainsNull = true), nullable = false) + )) + + withTable("test_db", "test_write_map_nullable", schema) { + val data = Seq( + Row(1, Map("a" -> 1, "b" -> null)), + Row(2, Map("x" -> null, "y" -> 20)), + Row(3, Map("foo" -> 100)) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_map_nullable") + + val result = spark.table("test_db.test_write_map_nullable").orderBy("id").collect() + assert(result.length == 3) + val map1 = result(0).getMap[String, Any](1) + assert(map1("a") == 1) + assert(map1("b") == null) + } + } + + test("write ShortType - min and max values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ShortType, nullable = false) + )) + + withTable("test_db", "test_write_short", schema) { + val data = Seq( + Row(1, Short.MinValue), + Row(2, 0.toShort), + Row(3, Short.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_short") + + val result = spark.table("test_db.test_write_short").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getShort(1) == Short.MinValue) + assert(result(1).getShort(1) == 0.toShort) + assert(result(2).getShort(1) == Short.MaxValue) + } + } + + test("write ShortType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", ShortType, nullable = true) + )) + + withTable("test_db", "test_write_short_null", schema) { + val data = Seq( + Row(1, Short.MinValue), + Row(2, null), + Row(3, Short.MaxValue) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_short_null") + + val result = spark.table("test_db.test_write_short_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getShort(1) == Short.MinValue) + assert(result(1).isNullAt(1)) + assert(result(2).getShort(1) == Short.MaxValue) + } + } + + test("write StringType - empty strings") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = false) + )) + + withTable("test_db", "test_write_empty_string", schema) { + val data = Seq( + Row(1, ""), + Row(2, "not empty"), + Row(3, "") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_empty_string") + + val result = spark.table("test_db.test_write_empty_string").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "") + assert(result(1).getString(1) == "not empty") + assert(result(2).getString(1) == "") + } + } + + test("write StringType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = true) + )) + + withTable("test_db", "test_write_string_null", schema) { + val data = Seq( + Row(1, "hello"), + Row(2, null), + Row(3, "world") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_string_null") + + val result = spark.table("test_db.test_write_string_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "hello") + assert(result(1).isNullAt(1)) + assert(result(2).getString(1) == "world") + } + } + + test("write StringType - regular strings") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", StringType, nullable = false) + )) + + withTable("test_db", "test_write_string", schema) { + val data = Seq( + Row(1, "hello"), + Row(2, "world"), + Row(3, "test") + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_string") + + val result = spark.table("test_db.test_write_string").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getString(1) == "hello") + assert(result(1).getString(1) == "world") + assert(result(2).getString(1) == "test") + } + } + + test("write TimestampType - nullable with null values") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", TimestampType, nullable = true) + )) + + withTable("test_db", "test_write_timestamp_null", schema) { + val data = Seq( + Row(1, java.sql.Timestamp.valueOf("2024-01-01 12:00:00")), + Row(2, null), + Row(3, java.sql.Timestamp.valueOf("2024-12-31 23:59:59")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_timestamp_null") + + val result = spark.table("test_db.test_write_timestamp_null").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).isNullAt(1)) + assert(result(2).getTimestamp(1) != null) + } + } + + test("write TimestampType - timestamps") { + val schema = StructType(Seq( + StructField("id", IntegerType, nullable = false), + StructField("value", TimestampType, nullable = false) + )) + + withTable("test_db", "test_write_timestamp", schema) { + val data = Seq( + Row(1, java.sql.Timestamp.valueOf("2024-01-01 12:00:00")), + Row(2, java.sql.Timestamp.valueOf("2024-06-15 18:30:45")), + Row(3, java.sql.Timestamp.valueOf("2024-12-31 23:59:59")) + ) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df.write.mode(SaveMode.Append).saveAsTable("test_db.test_write_timestamp") + + val result = spark.table("test_db.test_write_timestamp").orderBy("id").collect() + assert(result.length == 3) + assert(result(0).getTimestamp(1) != null) + assert(result(1).getTimestamp(1) != null) + assert(result(2).getTimestamp(1) != null) + } + } + +} diff --git a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala index 18f76870..53246f1b 100644 --- a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala +++ b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/ClickHouseReader.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.types._ import com.clickhouse.spark.Metrics.{BLOCKS_READ, BYTES_READ} import com.clickhouse.client.ClickHouseResponse +import com.clickhouse.client.api.query.QueryResponse abstract class ClickHouseReader[Record]( scanJob: ScanJobDescription, @@ -40,7 +41,6 @@ abstract class ClickHouseReader[Record]( val database: String = part.table.database val table: String = part.table.name - val codec: ClickHouseCompression = scanJob.readOptions.compressionCodec val readSchema: StructType = scanJob.readSchema private lazy val nodesClient = NodesClient(part.candidateNodes) @@ -67,11 +67,11 @@ abstract class ClickHouseReader[Record]( def format: String - lazy val resp: ClickHouseResponse = nodeClient.queryAndCheck(scanQuery, format, codec) + lazy val resp: QueryResponse = nodeClient.queryAndCheck(scanQuery, format) - def totalBlocksRead: Long = resp.getSummary.getStatistics.getBlocks + def totalBlocksRead: Long = 0L // resp.getSummary.getStatistics.getBlocks - def totalBytesRead: Long = resp.getSummary.getReadBytes + def totalBytesRead: Long = resp.getReadBytes // resp.getSummary.getReadBytes override def currentMetricsValues: Array[CustomTaskMetric] = Array( TaskMetric(BLOCKS_READ, totalBlocksRead), diff --git a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala index 5e90c4eb..dd1f9127 100644 --- a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala +++ b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseBinaryReader.scala @@ -14,6 +14,10 @@ package com.clickhouse.spark.read.format +import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader +import com.clickhouse.client.api.data_formats.{ClickHouseBinaryFormatReader, RowBinaryWithNamesAndTypesFormatReader} +import com.clickhouse.client.api.query.{GenericRecord, Records} + import java.util.Collections import com.clickhouse.data.value.{ ClickHouseArrayValue, @@ -34,120 +38,118 @@ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import java.time.ZoneOffset +import java.io.InputStream +import java.time.{LocalDate, ZoneOffset, ZonedDateTime} +import java.util import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ class ClickHouseBinaryReader( scanJob: ScanJobDescription, part: ClickHouseInputPartition -) extends ClickHouseReader[ClickHouseRecord](scanJob, part) { +) extends ClickHouseReader[GenericRecord](scanJob, part) { override val format: String = "RowBinaryWithNamesAndTypes" - lazy val streamOutput: Iterator[ClickHouseRecord] = resp.records().asScala.iterator + lazy val streamOutput: Iterator[GenericRecord] = { + val inputString: InputStream = resp.getInputStream + val cbfr: ClickHouseBinaryFormatReader = new RowBinaryWithNamesAndTypesFormatReader( + inputString, + resp.getSettings, + new BinaryStreamReader.DefaultByteBufferAllocator + ) + val r = new Records(resp, cbfr) + r.asScala.iterator + } - override def decode(record: ClickHouseRecord): InternalRow = { - val values: Array[Any] = new Array[Any](record.size) + override def decode(record: GenericRecord): InternalRow = { + val size = record.getSchema.getColumns.size() + val values: Array[Any] = new Array[Any](size) if (readSchema.nonEmpty) { var i: Int = 0 - while (i < record.size) { - values(i) = decodeValue(record.getValue(i), readSchema.fields(i)) + while (i < size) { + val v: Object = record.getObject(i + 1) + values(i) = decodeValue(v, readSchema.fields(i)) i = i + 1 } } new GenericInternalRow(values) } - private def decodeValue(value: ClickHouseValue, structField: StructField): Any = { - if (value == null || value.isNullOrEmpty && value.isNullable) { + private def decodeValue(value: Object, structField: StructField): Any = { + if (value == null) { // should we check `structField.nullable`? return null } structField.dataType match { - case BooleanType => value.asBoolean - case ByteType => value.asByte - case ShortType => value.asShort - case IntegerType => value.asInteger - case LongType => value.asLong - case FloatType => value.asFloat - case DoubleType => value.asDouble - case d: DecimalType => Decimal(value.asBigDecimal(d.scale)) + case BooleanType => value.asInstanceOf[Boolean] + case ByteType => value.asInstanceOf[Byte] + case ShortType => value.asInstanceOf[Short] +// case IntegerType if value.getClass.toString.equals("class java.lang.Long") => + case IntegerType if value.isInstanceOf[java.lang.Long] => + val v: Integer = Integer.valueOf(value.asInstanceOf[Long].toInt) + v.intValue() + case IntegerType => + value.asInstanceOf[Integer].intValue() + case LongType if value.isInstanceOf[java.math.BigInteger] => + value.asInstanceOf[java.math.BigInteger].longValue() + case LongType => + value.asInstanceOf[Long] + case FloatType => value.asInstanceOf[Float] + case DoubleType => value.asInstanceOf[Double] + case d: DecimalType => + // Java client returns BigInteger for Int256/UInt256, BigDecimal for Decimal types + val dec: BigDecimal = value match { + case bi: java.math.BigInteger => BigDecimal(bi) + case bd: java.math.BigDecimal => BigDecimal(bd) + } + Decimal(dec.setScale(d.scale)) case TimestampType => - var _instant = value.asZonedDateTime.withZoneSameInstant(ZoneOffset.UTC) + var _instant = value.asInstanceOf[ZonedDateTime].withZoneSameInstant(ZoneOffset.UTC) TimeUnit.SECONDS.toMicros(_instant.toEpochSecond) + TimeUnit.NANOSECONDS.toMicros(_instant.getNano()) - case StringType if value.isInstanceOf[ClickHouseStringValue] => UTF8String.fromBytes(value.asBinary) - case StringType => UTF8String.fromString(value.asString) - case DateType => value.asDate.toEpochDay.toInt - case BinaryType => value.asBinary + case StringType => + val strValue = value match { + case uuid: java.util.UUID => uuid.toString + case inet: java.net.InetAddress => inet.getHostAddress + case s: String => s + case enumValue: BinaryStreamReader.EnumValue => enumValue.toString + case _ => value.toString + } + UTF8String.fromString(strValue) + case DateType => + val localDate = value match { + case ld: LocalDate => ld + case zdt: ZonedDateTime => zdt.toLocalDate + case _ => value.asInstanceOf[LocalDate] + } + localDate.toEpochDay.toInt + case BinaryType => value.asInstanceOf[String].getBytes case ArrayType(_dataType, _nullable) => - val arrayValue = value.asInstanceOf[ClickHouseArraySequence] + // Java client returns BinaryStreamReader.ArrayValue for arrays + val arrayVal = value.asInstanceOf[BinaryStreamReader.ArrayValue] + val arrayValue = arrayVal.getArrayOfObjects().toSeq.asInstanceOf[Seq[Object]] val convertedArray = Array.tabulate(arrayValue.length) { i => decodeValue( - arrayValue.getValue(i, createClickHouseValue(null, _dataType)), + arrayValue(i), StructField("element", _dataType, _nullable) ) } new GenericArrayData(convertedArray) case MapType(_keyType, _valueType, _valueNullable) => - val convertedMap = value.asMap().asScala.map { case (rawKey, rawValue) => - val decodedKey = decodeValue(createClickHouseValue(rawKey, _keyType), StructField("key", _keyType, false)) - val decodedValue = - decodeValue(createClickHouseValue(rawValue, _valueType), StructField("value", _valueType, _valueNullable)) - (decodedKey, decodedValue) - } + // Java client returns util.Map (LinkedHashMap or EmptyMap) + val javaMap = value.asInstanceOf[util.Map[Object, Object]] + val convertedMap = + javaMap.asScala.map { case (rawKey, rawValue) => + val decodedKey = decodeValue(rawKey, StructField("key", _keyType, false)) + val decodedValue = + decodeValue(rawValue, StructField("value", _valueType, _valueNullable)) + (decodedKey, decodedValue) + } ArrayBasedMapData(convertedMap) - case _ => throw CHClientException(s"Unsupported catalyst type ${structField.name}[${structField.dataType}]") } } - private def createClickHouseValue(rawValue: Any, dataType: DataType): ClickHouseValue = { - val isNull = rawValue == null - - dataType match { - case StringType => - if (isNull) ClickHouseStringValue.ofNull() - else ClickHouseStringValue.of(rawValue.toString) - - case IntegerType => - if (isNull) ClickHouseIntegerValue.ofNull() - else ClickHouseIntegerValue.of(rawValue.asInstanceOf[Int]) - - case LongType => - if (isNull) ClickHouseLongValue.ofNull() - else ClickHouseLongValue.of(rawValue.asInstanceOf[Long]) - - case DoubleType => - if (isNull) ClickHouseDoubleValue.ofNull() - else ClickHouseDoubleValue.of(rawValue.asInstanceOf[Double]) - - case FloatType => - if (isNull) ClickHouseFloatValue.ofNull() - else ClickHouseFloatValue.of(rawValue.asInstanceOf[Float]) - - case BooleanType => - if (isNull) ClickHouseBoolValue.ofNull() - else ClickHouseBoolValue.of(rawValue.asInstanceOf[Boolean]) - - case _: ArrayType => - if (isNull) ClickHouseArrayValue.ofEmpty() - else ClickHouseArrayValue.of(rawValue.asInstanceOf[Array[Object]]) - - case _: MapType => - if (isNull) ClickHouseMapValue.ofEmpty(classOf[Object], classOf[Object]) - else ClickHouseMapValue.of( - rawValue.asInstanceOf[java.util.Map[Object, Object]], - classOf[Object], - classOf[Object] - ) - - case _ => - if (isNull) ClickHouseStringValue.ofNull() - else ClickHouseStringValue.of(rawValue.toString) - } - } - } diff --git a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala index 8c044ad4..f5a99695 100644 --- a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala +++ b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/read/format/ClickHouseJsonReader.scala @@ -85,10 +85,15 @@ class ClickHouseJsonReader( TimeUnit.SECONDS.toMicros(_instant.toEpochSecond) + TimeUnit.NANOSECONDS.toMicros(_instant.getNano()) case StringType => UTF8String.fromString(jsonNode.asText) case DateType => LocalDate.parse(jsonNode.asText, dateFmt).toEpochDay.toInt - case BinaryType => jsonNode.binaryValue + case BinaryType if jsonNode.isTextual => + // ClickHouse JSON format returns FixedString as plain text, not Base64 + jsonNode.asText.getBytes("UTF-8") + case BinaryType => + // True binary data is Base64 encoded in JSON format + jsonNode.binaryValue case ArrayType(_dataType, _nullable) => val _structField = StructField(s"${structField.name}__array_element__", _dataType, _nullable) - new GenericArrayData(jsonNode.asScala.map(decodeValue(_, _structField))) + new GenericArrayData(jsonNode.asScala.map(decodeValue(_, _structField)).toArray) case MapType(StringType, _valueType, _valueNullable) => val mapData = jsonNode.fields.asScala.map { entry => val _structField = StructField(s"${structField.name}__map_value__", _valueType, _valueNullable) diff --git a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala index 6f9b267b..9d17feea 100644 --- a/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala +++ b/spark-3.5/clickhouse-spark/src/main/scala/com/clickhouse/spark/write/ClickHouseWriter.scala @@ -253,7 +253,8 @@ abstract class ClickHouseWriter(writeJob: WriteJobDescription) writeJob.writeOptions.retryInterval ) { var startWriteTime = System.currentTimeMillis - client.syncInsertOutputJSONEachRow(database, table, format, codec, new ByteArrayInputStream(data)) match { + // codec, + client.syncInsertOutputJSONEachRow(database, table, format, new ByteArrayInputStream(data)) match { case Right(_) => writeTime = System.currentTimeMillis - startWriteTime _totalWriteTime.add(writeTime)