Skip to main content
Skip to main content
Edit this page

Spark connector

This connector leverages ClickHouse-specific optimizations, such as advanced partitioning and predicate pushdown, to improve query performance and data handling. The connector is based on ClickHouse's official JDBC connector, and manages its own catalog.

Before Spark 3.0, Spark lacked a built-in catalog concept, so users typically relied on external catalog systems such as Hive Metastore or AWS Glue. With these external solutions, users had to register their data source tables manually before accessing them in Spark. However, since Spark 3.0 introduced the catalog concept, Spark can now automatically discover tables by registering catalog plugins.

Spark's default catalog is spark_catalog, and tables are identified by {catalog name}.{database}.{table}. With the new catalog feature, it is now possible to add and work with multiple catalogs in a single Spark application.

Requirements

  • Java 8 or 17 (Java 17+ required for Spark 4.0)
  • Scala 2.12 or 2.13 (Spark 4.0 only supports Scala 2.13)
  • Apache Spark 3.3, 3.4, 3.5, or 4.0

Compatibility matrix

VersionCompatible Spark VersionsClickHouse JDBC version
mainSpark 3.3, 3.4, 3.5, 4.00.9.4
0.9.0Spark 3.3, 3.4, 3.5, 4.00.9.4
0.8.1Spark 3.3, 3.4, 3.50.6.3
0.7.3Spark 3.3, 3.40.4.6
0.6.0Spark 3.30.3.2-patch11
0.5.0Spark 3.2, 3.30.3.2-patch11
0.4.0Spark 3.2, 3.3Not depend on
0.3.0Spark 3.2, 3.3Not depend on
0.2.1Spark 3.2Not depend on
0.1.2Spark 3.2Not depend on

Installation & setup

For integrating ClickHouse with Spark, there are multiple installation options to suit different project setups. You can add the ClickHouse Spark connector as a dependency directly in your project's build file (such as in pom.xml for Maven or build.sbt for SBT). Alternatively, you can put the required JAR files in your $SPARK_HOME/jars/ folder, or pass them directly as a Spark option using the --jars flag in the spark-submit command. Both approaches ensure the ClickHouse connector is available in your Spark environment.

Import as a Dependency

<dependency>
  <groupId>com.clickhouse.spark</groupId>
  <artifactId>clickhouse-spark-runtime-{{ spark_binary_version }}_{{ scala_binary_version }}</artifactId>
  <version>{{ stable_version }}</version>
</dependency>
<dependency>
  <groupId>com.clickhouse</groupId>
  <artifactId>clickhouse-jdbc</artifactId>
  <classifier>all</classifier>
  <version>{{ clickhouse_jdbc_version }}</version>
  <exclusions>
    <exclusion>
      <groupId>*</groupId>
      <artifactId>*</artifactId>
    </exclusion>
  </exclusions>
</dependency>

Add the following repository if you want to use SNAPSHOT version.

<repositories>
  <repository>
    <id>sonatype-oss-snapshots</id>
    <name>Sonatype OSS Snapshots Repository</name>
    <url>https://s01.oss.sonatype.org/content/repositories/snapshots</url>
  </repository>
</repositories>

Download the library

The name pattern of the binary JAR is:

clickhouse-spark-runtime-${spark_binary_version}_${scala_binary_version}-${version}.jar

You can find all available released JAR files in the Maven Central Repository and all daily build SNAPSHOT JAR files in the Sonatype OSS Snapshots Repository.

References

It's essential to include the clickhouse-jdbc JAR with the "all" classifier, as the connector relies on clickhouse-http and clickhouse-client — both of which are bundled in clickhouse-jdbc:all. Alternatively, you can add clickhouse-client JAR and clickhouse-http individually if you prefer not to use the full JDBC package.

In any case, ensure that the package versions are compatible according to the Compatibility Matrix.

Register the catalog (required)

In order to access your ClickHouse tables, you must configure a new Spark catalog with the following configs:

PropertyValueDefault ValueRequired
spark.sql.catalog.<catalog_name>com.clickhouse.spark.ClickHouseCatalogN/AYes
spark.sql.catalog.<catalog_name>.host<clickhouse_host>localhostNo
spark.sql.catalog.<catalog_name>.protocolhttphttpNo
spark.sql.catalog.<catalog_name>.http_port<clickhouse_port>8123No
spark.sql.catalog.<catalog_name>.user<clickhouse_username>defaultNo
spark.sql.catalog.<catalog_name>.password<clickhouse_password>(empty string)No
spark.sql.catalog.<catalog_name>.database<database>defaultNo
spark.<catalog_name>.write.formatjsonarrowNo

These settings could be set via one of the following:

  • Edit/Create spark-defaults.conf.
  • Pass the configuration to your spark-submit command (or to your spark-shell/spark-sql CLI commands).
  • Add the configuration when initiating your context.
References

When working with a ClickHouse cluster, you need to set a unique catalog name for each instance. For example:

spark.sql.catalog.clickhouse1                com.clickhouse.spark.ClickHouseCatalog
spark.sql.catalog.clickhouse1.host           10.0.0.1
spark.sql.catalog.clickhouse1.protocol       https
spark.sql.catalog.clickhouse1.http_port      8443
spark.sql.catalog.clickhouse1.user           default
spark.sql.catalog.clickhouse1.password
spark.sql.catalog.clickhouse1.database       default
spark.sql.catalog.clickhouse1.option.ssl     true

spark.sql.catalog.clickhouse2                com.clickhouse.spark.ClickHouseCatalog
spark.sql.catalog.clickhouse2.host           10.0.0.2
spark.sql.catalog.clickhouse2.protocol       https
spark.sql.catalog.clickhouse2.http_port      8443
spark.sql.catalog.clickhouse2.user           default
spark.sql.catalog.clickhouse2.password
spark.sql.catalog.clickhouse2.database       default
spark.sql.catalog.clickhouse2.option.ssl     true

That way, you would be able to access clickhouse1 table <ck_db>.<ck_table> from Spark SQL by clickhouse1.<ck_db>.<ck_table>, and access clickhouse2 table <ck_db>.<ck_table> by clickhouse2.<ck_db>.<ck_table>.

Using the TableProvider API (Format-based Access)

In addition to the catalog-based approach, the ClickHouse Spark connector supports a format-based access pattern via the TableProvider API.

Format-based Read Example

from pyspark.sql import SparkSession

spark = SparkSession.builder.getOrCreate()

# Read from ClickHouse using format API
df = spark.read \
    .format("clickhouse") \
    .option("host", "your-clickhouse-host") \
    .option("protocol", "https") \
    .option("http_port", "8443") \
    .option("database", "default") \
    .option("table", "your_table") \
    .option("user", "default") \
    .option("password", "your_password") \
    .option("ssl", "true") \
    .load()

df.show()

Format-based Write Example

# Write to ClickHouse using format API
df.write \
    .format("clickhouse") \
    .option("host", "your-clickhouse-host") \
    .option("protocol", "https") \
    .option("http_port", "8443") \
    .option("database", "default") \
    .option("table", "your_table") \
    .option("user", "default") \
    .option("password", "your_password") \
    .option("ssl", "true") \
    .mode("append") \
    .save()

TableProvider Features

The TableProvider API provides several powerful features:

Automatic Table Creation

When writing to a non-existent table, the connector automatically creates the table with an appropriate schema. The connector provides intelligent defaults:

  • Engine: Defaults to MergeTree() if not specified. You can specify a different engine using the engine option (e.g., ReplacingMergeTree(), SummingMergeTree(), etc.)
  • ORDER BY: Required - You must explicitly specify the order_by option when creating a new table. The connector validates that all specified columns exist in the schema.
  • Nullable Key Support: Automatically adds settings.allow_nullable_key=1 if ORDER BY contains nullable columns
# Table will be created automatically with explicit ORDER BY (required)
df.write \
    .format("clickhouse") \
    .option("host", "your-host") \
    .option("database", "default") \
    .option("table", "new_table") \
    .option("order_by", "id") \
    .mode("append") \
    .save()

# Specify table creation options with custom engine
df.write \
    .format("clickhouse") \
    .option("host", "your-host") \
    .option("database", "default") \
    .option("table", "new_table") \
    .option("order_by", "id, timestamp") \
    .option("engine", "ReplacingMergeTree()") \
    .option("settings.allow_nullable_key", "1") \
    .mode("append") \
    .save()
References

ORDER BY Required: The order_by option is required when creating a new table via the TableProvider API. You must explicitly specify which column(s) to use for the ORDER BY clause. The connector validates that all specified columns exist in the schema and will throw an error if any columns are missing.

Engine Selection: The default engine is MergeTree(), but you can specify any ClickHouse table engine using the engine option (e.g., ReplacingMergeTree(), SummingMergeTree(), AggregatingMergeTree(), etc.).

TableProvider Options

When using the format-based API, the following options are available:

Connection Options

OptionDescriptionDefault ValueRequired
hostClickHouse server hostnamelocalhostYes
protocolConnection protocol (http or https)httpNo
http_portHTTP/HTTPS port8123No
databaseDatabase namedefaultYes
tableTable nameN/AYes
userUsername for authenticationdefaultNo
passwordPassword for authentication(empty string)No
sslEnable SSL connectionfalseNo
ssl_modeSSL mode (NONE, STRICT, etc.)STRICTNo
timezoneTimezone for date/time operationsserverNo

Table Creation Options

These options are used when the table doesn't exist and needs to be created:

OptionDescriptionDefault ValueRequired
order_byColumn(s) to use for ORDER BY clause. Comma-separated for multiple columnsN/AYes
engineClickHouse table engine (e.g., MergeTree(), ReplacingMergeTree(), SummingMergeTree(), etc.)MergeTree()No
settings.allow_nullable_keyEnable nullable keys in ORDER BY (for ClickHouse Cloud)Auto-detected**No
settings.<key>Any ClickHouse table settingN/ANo
clusterCluster name for Distributed tablesN/ANo

* The order_by option is required when creating a new table. All specified columns must exist in the schema.
** Automatically set to 1 if ORDER BY contains nullable columns and not explicitly provided.

Tip

Best Practice: For ClickHouse Cloud, explicitly set settings.allow_nullable_key=1 if your ORDER BY columns might be nullable, as ClickHouse Cloud requires this setting.

Writing Modes

The TableProvider API supports the following Spark write modes:

  • append: Add data to existing table
  • overwrite: Replace all data in the table (truncates table)
# Overwrite mode (truncates table first)
df.write \
    .format("clickhouse") \
    .option("host", "your-host") \
    .option("database", "default") \
    .option("table", "my_table") \
    .mode("overwrite") \
    .save()

Databricks Integration

The ClickHouse Spark connector works seamlessly with Databricks, including support for Databricks Unity Catalog environments. This section covers setup and usage specific to Databricks.

Installation on Databricks

Option 1: Upload JAR via Databricks UI

  1. Build or download the runtime JAR:

    clickhouse-spark-runtime-{{ spark_binary_version }}_{{ scala_binary_version }}-{{ stable_version }}.jar
    
  2. Navigate to your Databricks workspace:

    • Go to Compute → Select your cluster
    • Click the Libraries tab
    • Click Install New
    • Select UploadJAR
    • Upload the runtime JAR file
    • Click Install
  3. Restart the cluster to load the library

Option 2: Install via Databricks CLI

# Upload JAR to DBFS
databricks fs cp clickhouse-spark-runtime-{{ spark_binary_version }}_{{ scala_binary_version }}-{{ stable_version }}.jar \
  dbfs:/FileStore/jars/

# Install on cluster
databricks libraries install \
  --cluster-id <your-cluster-id> \
  --jar dbfs:/FileStore/jars/clickhouse-spark-runtime-{{ spark_binary_version }}_{{ scala_binary_version }}-{{ stable_version }}.jar

Option 3: Maven Coordinates (Recommended)

In your cluster configuration, add the Maven coordinates:

com.clickhouse.spark:clickhouse-spark-runtime-{{ spark_binary_version }}_{{ scala_binary_version }}:{{ stable_version }}
Note

The Spark 4.0 runtime JAR includes shaded Jackson dependencies to prevent conflicts with Databricks Hudi's Jackson version. This is built into the standard JAR, so no special classifier, variant, or separate Databricks jar is needed - just use the standard clickhouse-spark-runtime-4.0_2.13 artifact.

Databricks Notebook Usage

Using Format-based API (Recommended for Databricks)

The format-based API (TableProvider) is the recommended approach for Databricks, especially with Unity Catalog. It provides better compatibility and avoids catalog conflicts:

# Read from ClickHouse using TableProvider API
df = spark.read \
    .format("clickhouse") \
    .option("host", "your-clickhouse-cloud-host.clickhouse.cloud") \
    .option("protocol", "https") \
    .option("http_port", "8443") \
    .option("database", "default") \
    .option("table", "events") \
    .option("user", "default") \
    .option("password", dbutils.secrets.get(scope="clickhouse", key="password")) \
    .option("ssl", "true") \
    .load()

# Schema is automatically inferred
df.display()

Databricks-Specific Considerations

Important: In Databricks environments, only the TableProvider API (format-based access) is supported. The Catalog API is not available in Databricks.

Use the format-based API as shown in the Format-based Read Example and Format-based Write Example sections above.

ClickHouse Cloud settings

When connecting to ClickHouse Cloud, make sure to enable SSL and set the appropriate SSL mode. For example:

spark.sql.catalog.clickhouse.option.ssl        true
spark.sql.catalog.clickhouse.option.ssl_mode   NONE

Read data

public static void main(String[] args) {
        // Create a Spark session
        SparkSession spark = SparkSession.builder()
                .appName("example")
                .master("local[*]")
                .config("spark.sql.catalog.clickhouse", "com.clickhouse.spark.ClickHouseCatalog")
                .config("spark.sql.catalog.clickhouse.host", "127.0.0.1")
                .config("spark.sql.catalog.clickhouse.protocol", "http")
                .config("spark.sql.catalog.clickhouse.http_port", "8123")
                .config("spark.sql.catalog.clickhouse.user", "default")
                .config("spark.sql.catalog.clickhouse.password", "123456")
                .config("spark.sql.catalog.clickhouse.database", "default")
                .config("spark.clickhouse.write.format", "json")
                .getOrCreate();

        Dataset<Row> df = spark.sql("select * from clickhouse.default.example_table");

        df.show();

        spark.stop();
    }

Write data

 public static void main(String[] args) throws AnalysisException {

        // Create a Spark session
        SparkSession spark = SparkSession.builder()
                .appName("example")
                .master("local[*]")
                .config("spark.sql.catalog.clickhouse", "com.clickhouse.spark.ClickHouseCatalog")
                .config("spark.sql.catalog.clickhouse.host", "127.0.0.1")
                .config("spark.sql.catalog.clickhouse.protocol", "http")
                .config("spark.sql.catalog.clickhouse.http_port", "8123")
                .config("spark.sql.catalog.clickhouse.user", "default")
                .config("spark.sql.catalog.clickhouse.password", "123456")
                .config("spark.sql.catalog.clickhouse.database", "default")
                .config("spark.clickhouse.write.format", "json")
                .getOrCreate();

        // Define the schema for the DataFrame
        StructType schema = new StructType(new StructField[]{
                DataTypes.createStructField("id", DataTypes.IntegerType, false),
                DataTypes.createStructField("name", DataTypes.StringType, false),
        });

        List<Row> data = Arrays.asList(
                RowFactory.create(1, "Alice"),
                RowFactory.create(2, "Bob")
        );

        // Create a DataFrame
        Dataset<Row> df = spark.createDataFrame(data, schema);

        df.writeTo("clickhouse.default.example_table").append();

        spark.stop();
    }

DDL operations

You can perform DDL operations on your ClickHouse instance using Spark SQL, with all changes immediately persisted in ClickHouse. Spark SQL allows you to write queries exactly as you would in ClickHouse, so you can directly execute commands such as CREATE TABLE, TRUNCATE, and more - without modification, for instance:

Note

When using Spark SQL, only one statement can be executed at a time.

USE clickhouse; 

CREATE TABLE test_db.tbl_sql (
  create_time TIMESTAMP NOT NULL,
  m           INT       NOT NULL COMMENT 'part key',
  id          BIGINT    NOT NULL COMMENT 'sort key',
  value       STRING
) USING ClickHouse
PARTITIONED BY (m)
TBLPROPERTIES (
  engine = 'MergeTree()',
  order_by = 'id',
  settings.index_granularity = 8192
);

The above examples demonstrate Spark SQL queries, which you can run within your application using any API—Java, Scala, PySpark, or shell.

Working with VariantType

Note

VariantType support is available in Spark 4.0+ and requires ClickHouse 25.3+ with experimental JSON/Variant types enabled.

The connector supports Spark's VariantType for working with semi-structured data. VariantType maps to ClickHouse's JSON and Variant types, allowing you to store and query flexible schema data efficiently.

ClickHouse Type Mapping

ClickHouse TypeSpark TypeDescription
JSONVariantTypeStores JSON objects only (must start with {)
Variant(T1, T2, ...)VariantTypeStores multiple types including primitives, arrays, and JSON

Reading VariantType Data

When reading from ClickHouse, JSON and Variant columns are automatically mapped to Spark's VariantType:

// Read JSON column as VariantType
val df = spark.sql("SELECT id, data FROM clickhouse.default.json_table")

// Access variant data
df.show()

// Convert variant to JSON string for inspection
import org.apache.spark.sql.functions._
df.select(
  col("id"),
  to_json(col("data")).as("data_json")
).show()

Writing VariantType Data

You can write VariantType data to ClickHouse using either JSON or Variant column types:

import org.apache.spark.sql.functions._

// Create DataFrame with JSON data
val jsonData = Seq(
  (1, """{"name": "Alice", "age": 30}"""),
  (2, """{"name": "Bob", "age": 25}"""),
  (3, """{"name": "Charlie", "city": "NYC"}""")
).toDF("id", "json_string")

// Parse JSON strings to VariantType
val variantDF = jsonData.select(
  col("id"),
  parse_json(col("json_string")).as("data")
)

// Write to ClickHouse with JSON type (JSON objects only)
variantDF.writeTo("clickhouse.default.user_data").create()

// Or specify Variant with multiple types
spark.sql("""
  CREATE TABLE clickhouse.default.mixed_data (
    id INT,
    data VARIANT
  ) USING clickhouse
  TBLPROPERTIES (
    'clickhouse.column.data.variant_types' = 'String, Int64, Bool, JSON',
    'engine' = 'MergeTree()',
    'order_by' = 'id'
  )
""")

Configuring Variant Types

When creating tables with VariantType columns, you can specify which ClickHouse types to use:

JSON Type (Default)

If no variant_types property is specified, the column defaults to ClickHouse's JSON type, which only accepts JSON objects:

CREATE TABLE clickhouse.default.json_table (
  id INT,
  data VARIANT
) USING clickhouse
TBLPROPERTIES (
  'engine' = 'MergeTree()',
  'order_by' = 'id'
)

This creates: CREATE TABLE json_table (id Int32, data JSON) ENGINE = MergeTree() ORDER BY id

Variant Type with Multiple Types

To support primitives, arrays, and JSON objects, specify the types in the variant_types property:

CREATE TABLE clickhouse.default.flexible_data (
  id INT,
  data VARIANT
) USING clickhouse
TBLPROPERTIES (
  'clickhouse.column.data.variant_types' = 'String, Int64, Float64, Bool, Array(String), JSON',
  'engine' = 'MergeTree()',
  'order_by' = 'id'
)

This creates: CREATE TABLE flexible_data (id Int32, data Variant(String, Int64, Float64, Bool, Array(String), JSON)) ENGINE = MergeTree() ORDER BY id

Supported Variant Types

The following ClickHouse types can be used in Variant():

  • Primitives: String, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64, Bool
  • Arrays: Array(T) where T is any supported type, including nested arrays
  • JSON: JSON for storing JSON objects

Read Format Configuration

By default, JSON and Variant columns are read as VariantType. You can override this behavior to read them as strings:

// Read JSON/Variant as strings instead of VariantType
spark.conf.set("spark.clickhouse.read.jsonAs", "string")

val df = spark.sql("SELECT id, data FROM clickhouse.default.json_table")
// data column will be StringType containing JSON strings

Write Format Support

VariantType write support varies by format:

FormatSupportNotes
JSON✅ FullSupports both JSON and Variant types. Recommended for VariantType data
Arrow⚠️ PartialSupports writing to ClickHouse JSON type. Does not support ClickHouse Variant type.

Configure the write format:

spark.conf.set("spark.clickhouse.write.format", "json")  // Recommended for Variant types
Tip

If you need to write to a ClickHouse Variant type, use JSON format. Arrow format only supports writing to JSON type.

Best Practices

  1. Use JSON type for JSON-only data: If you only store JSON objects, use the default JSON type (no variant_types property)
  2. Specify types explicitly: When using Variant(), explicitly list all types you plan to store
  3. Enable experimental features: Ensure ClickHouse has allow_experimental_json_type = 1 enabled
  4. Use JSON format for writes: JSON format is recommended for VariantType data for better compatibility
  5. Consider query patterns: JSON/Variant types support ClickHouse's JSON path queries for efficient filtering

Example: Complete Workflow

import org.apache.spark.sql.functions._

// Enable experimental JSON type in ClickHouse
spark.sql("SET allow_experimental_json_type = 1")

// Create table with Variant column
spark.sql("""
  CREATE TABLE clickhouse.default.events (
    event_id BIGINT,
    event_time TIMESTAMP,
    event_data VARIANT
  ) USING clickhouse
  TBLPROPERTIES (
    'clickhouse.column.event_data.variant_types' = 'String, Int64, Bool, JSON',
    'engine' = 'MergeTree()',
    'order_by' = 'event_time'
  )
""")

// Prepare data with mixed types
val events = Seq(
  (1L, "2024-01-01 10:00:00", """{"action": "login", "user_id": 123}"""),
  (2L, "2024-01-01 10:05:00", """{"action": "purchase", "amount": 99.99}"""),
  (3L, "2024-01-01 10:10:00", """{"action": "logout", "duration": 600}""")
).toDF("event_id", "event_time", "json_data")

// Convert to VariantType and write
val variantEvents = events.select(
  col("event_id"),
  to_timestamp(col("event_time")).as("event_time"),
  parse_json(col("json_data")).as("event_data")
)

variantEvents.writeTo("clickhouse.default.events").append()

// Read and query
val result = spark.sql("""
  SELECT event_id, event_time, event_data
  FROM clickhouse.default.events
  WHERE event_time >= '2024-01-01'
  ORDER BY event_time
""")

result.show(false)

Configurations

The following are the adjustable configurations available in the connector.

Note

Using Configurations: These are Spark-level configuration options that apply to both Catalog API and TableProvider API. They can be set in two ways:

  1. Global Spark configuration (applies to all operations):

    spark.conf.set("spark.clickhouse.write.batchSize", "20000")
    spark.conf.set("spark.clickhouse.write.compression.codec", "lz4")
    
  2. Per-operation override (TableProvider API only - can override global settings):

    df.write \
        .format("clickhouse") \
        .option("host", "your-host") \
        .option("database", "default") \
        .option("table", "my_table") \
        .option("spark.clickhouse.write.batchSize", "20000") \
        .option("spark.clickhouse.write.compression.codec", "lz4") \
        .mode("append") \
        .save()
    

Alternatively, set them in spark-defaults.conf or when creating the Spark session.


KeyDefaultDescriptionSince
spark.clickhouse.ignoreUnsupportedTransformfalseClickHouse supports using complex expressions as sharding keys or partition values, e.g. cityHash64(col_1, col_2), which are currently not supported by Spark. If true, ignore the unsupported expressions, otherwise fail fast w/ an exception. Note, when spark.clickhouse.write.distributed.convertLocal is enabled, ignore unsupported sharding keys may corrupt the data.0.4.0
spark.clickhouse.read.compression.codeclz4The codec used to decompress data for reading. Supported codecs: none, lz4.0.5.0
spark.clickhouse.read.distributed.convertLocaltrueWhen reading Distributed table, read local table instead of itself. If true, ignore spark.clickhouse.read.distributed.useClusterNodes.0.1.0
spark.clickhouse.read.fixedStringAsbinaryRead ClickHouse FixedString type as the specified Spark data type. Supported types: binary, string0.8.0
spark.clickhouse.read.formatjsonSerialize format for reading. Supported formats: json, binary0.6.0
spark.clickhouse.read.runtimeFilter.enabledfalseEnable runtime filter for reading.0.8.0
spark.clickhouse.read.splitByPartitionIdtrueIf true, construct input partition filter by virtual column _partition_id, instead of partition value. There are known issues with assembling SQL predicates by partition value. This feature requires ClickHouse Server v21.6+0.4.0
spark.clickhouse.useNullableQuerySchemafalseIf true, mark all the fields of the query schema as nullable when executing CREATE/REPLACE TABLE ... AS SELECT ... on creating the table. Note, this configuration requires SPARK-43390(available in Spark 3.5), w/o this patch, it always acts as true.0.8.0
spark.clickhouse.write.batchSize10000The number of records per batch on writing to ClickHouse.0.1.0
spark.clickhouse.write.compression.codeclz4The codec used to compress data for writing. Supported codecs: none, lz4.0.3.0
spark.clickhouse.write.distributed.convertLocalfalseWhen writing Distributed table, write local table instead of itself. If true, ignore spark.clickhouse.write.distributed.useClusterNodes.0.1.0
spark.clickhouse.write.distributed.useClusterNodestrueWrite to all nodes of cluster when writing Distributed table.0.1.0
spark.clickhouse.write.formatarrowSerialize format for writing. Supported formats: json, arrow0.4.0
spark.clickhouse.write.localSortByKeytrueIf true, do local sort by sort keys before writing.0.3.0
spark.clickhouse.write.localSortByPartitionvalue of spark.clickhouse.write.repartitionByPartitionIf true, do local sort by partition before writing. If not set, it equals to spark.clickhouse.write.repartitionByPartition.0.3.0
spark.clickhouse.write.maxRetry3The maximum number of write we will retry for a single batch write failed with retryable codes.0.1.0
spark.clickhouse.write.repartitionByPartitiontrueWhether to repartition data by ClickHouse partition keys to meet the distributions of ClickHouse table before writing.0.3.0
spark.clickhouse.write.repartitionNum0Repartition data to meet the distributions of ClickHouse table is required before writing, use this conf to specific the repartition number, value less than 1 mean no requirement.0.1.0
spark.clickhouse.write.repartitionStrictlyfalseIf true, Spark will strictly distribute incoming records across partitions to satisfy the required distribution before passing the records to the data source table on write. Otherwise, Spark may apply certain optimizations to speed up the query but break the distribution requirement. Note, this configuration requires SPARK-37523(available in Spark 3.4), w/o this patch, it always acts as true.0.3.0
spark.clickhouse.write.retryInterval10sThe interval in seconds between write retry.0.1.0
spark.clickhouse.write.retryableErrorCodes241The retryable error codes returned by ClickHouse server when write failing.0.1.0

Supported data types

This section outlines the mapping of data types between Spark and ClickHouse. The tables below provide quick references for converting data types when reading from ClickHouse into Spark and when inserting data from Spark into ClickHouse.

Reading data from ClickHouse into Spark

ClickHouse Data TypeSpark Data TypeSupportedIs PrimitiveNotes
NothingNullTypeYes
BoolBooleanTypeYes
UInt8, Int16ShortTypeYes
Int8ByteTypeYes
UInt16,Int32IntegerTypeYes
UInt32,Int64, UInt64LongTypeYes
Int128,UInt128, Int256, UInt256DecimalType(38, 0)Yes
Float32FloatTypeYes
Float64DoubleTypeYes
String, UUID, Enum8, Enum16, IPv4, IPv6StringTypeYes
FixedStringBinaryType, StringTypeYesControlled by configuration READ_FIXED_STRING_AS
DecimalDecimalTypeYesPrecision and scale up to Decimal128
Decimal32DecimalType(9, scale)Yes
Decimal64DecimalType(18, scale)Yes
Decimal128DecimalType(38, scale)Yes
Date, Date32DateTypeYes
DateTime, DateTime32, DateTime64TimestampTypeYes
ArrayArrayTypeNoArray element type is also converted
MapMapTypeNoKeys are limited to StringType
IntervalYearYearMonthIntervalType(Year)Yes
IntervalMonthYearMonthIntervalType(Month)Yes
IntervalDay, IntervalHour, IntervalMinute, IntervalSecondDayTimeIntervalTypeNoSpecific interval type is used
JSON, VariantVariantTypeNoRequires Spark 4.0+ and ClickHouse 25.3+. Can be read as StringType with spark.clickhouse.read.jsonAs=string
Object
Nested
TupleStructTypeNoSupports both named and unnamed tuples. Named tuples map to struct fields by name, unnamed tuples use _1, _2, etc. Supports nested structs and nullable fields
Point
Polygon
MultiPolygon
Ring
IntervalQuarter
IntervalWeek
Decimal256
AggregateFunction
SimpleAggregateFunction

Inserting data from Spark into ClickHouse

Spark Data TypeClickHouse Data TypeSupportedIs PrimitiveNotes
BooleanTypeBoolYesMapped to Bool type (not UInt8) since version 0.9.0
ByteTypeInt8Yes
ShortTypeInt16Yes
IntegerTypeInt32Yes
LongTypeInt64Yes
FloatTypeFloat32Yes
DoubleTypeFloat64Yes
StringTypeStringYes
VarcharTypeStringYes
CharTypeStringYes
DecimalTypeDecimal(p, s)YesPrecision and scale up to Decimal128
DateTypeDateYes
TimestampTypeDateTimeYes
ArrayType (list, tuple, or array)ArrayNoArray element type is also converted
MapTypeMapNoKeys are limited to StringType
StructTypeTupleNoConverted to named Tuple with field names.
VariantTypeJSON or VariantNoRequires Spark 4.0+ and ClickHouse 25.3+. Defaults to JSON type. Use clickhouse.column.<name>.variant_types property to specify Variant with multiple types.
Object
Nested

Contributing and support

If you'd like to contribute to the project or report any issues, we welcome your input! Visit our GitHub repository to open an issue, suggest improvements, or submit a pull request. Contributions are welcome! Please check the contribution guidelines in the repository before starting. Thank you for helping improve our ClickHouse Spark connector!