diff --git a/java/common/src/main/resources/org/apache/tsfile/i18n/messages.properties b/java/common/src/main/resources/org/apache/tsfile/i18n/messages.properties
index 98909f7a6..d59705312 100644
--- a/java/common/src/main/resources/org/apache/tsfile/i18n/messages.properties
+++ b/java/common/src/main/resources/org/apache/tsfile/i18n/messages.properties
@@ -1548,3 +1548,72 @@ log.tools.csv_read_error = Error reading CSV file: %1$s
# CsvSourceReader — error closing CSV reader
log.tools.csv_close_reader_error = Error closing CSV reader
+
+# --- spark connector ERROR messages ---
+
+error.spark.write_table_required = Writing TsFile table model requires option "table"
+error.spark.write_tag_columns_required = Writing TsFile table model requires option "tagColumns"
+error.spark.null_tag_policy_unsupported = Only nullTagPolicy=error is supported
+error.spark.unsupported_option = Unsupported TsFile connector option: %1$s
+error.spark.model_unsupported = Only TsFile table model is supported, but option "model" was %1$s
+error.spark.multiple_load_paths_unsupported = Multiple Spark load paths are not supported yet; pass a directory or glob path instead
+error.spark.path_required = TsFile connector requires a path
+error.spark.positive_option_required = %1$s must be positive
+error.spark.timestamp_as_invalid = timestampAs must be either long or timestamp
+error.spark.timestamp_precision_invalid = timestampPrecision must be one of ms, us, or ns
+error.spark.column_list_empty = Column list option must not be empty
+error.spark.no_tsfile_files = No TsFile files found under path: %1$s
+error.spark.multiple_tables_requires_table = TsFile %1$s contains multiple tables; specify option "table"
+error.spark.table_not_found_in_file = TsFile file %1$s does not contain table %2$s
+error.spark.read_table_metadata_failed = Failed to read TsFile table metadata from %1$s
+error.spark.incompatible_table_name = Incompatible TsFile table schema in %1$s: expected table %2$s but found %3$s
+error.spark.incompatible_column_count = Incompatible TsFile table schema in %1$s: column count differs
+error.spark.incompatible_table_schema_column = Incompatible TsFile table schema in %1$s at column %2$s: expected %3$s/%4$s/%5$s but found %6$s/%7$s/%8$s
+error.spark.incompatible_tag_column_count = Incompatible TsFile table schema in %1$s: TAG column count differs
+error.spark.incompatible_tag_column = Incompatible TsFile table schema in %1$s at TAG column %2$s: expected %3$s/%4$s but found %5$s/%6$s
+error.spark.incompatible_field_column_type = Incompatible TsFile table schema in %1$s: FIELD column %2$s has type %4$s but expected %3$s
+error.spark.path_not_exist = TsFile input path does not exist: %1$s
+error.spark.discover_path_failed = Failed to discover TsFile input path: %1$s
+error.spark.input_not_tsfile = Input path is not a .tsfile file: %1$s
+error.spark.local_paths_only = Only local TsFile paths are supported: %1$s
+error.spark.unknown_projected_column = Unknown projected TsFile table column: %1$s
+error.spark.time_tag_projection_requires_field = Time/TAG-only projection requires at least one FIELD column in the TsFile table
+error.spark.unsupported_time_filter_literal = Unsupported time filter literal: %1$s
+error.spark.duplicate_tsfile_column = Duplicate TsFile table column after lower-case normalization: %1$s
+error.spark.duplicate_spark_schema_column = Duplicate Spark schema column after lower-case normalization: %1$s
+error.spark.tag_column_must_string_tsfile = TAG column %1$s.%2$s must be TSDataType.STRING, but was %3$s
+error.spark.unsupported_column_category = Column category %1$s is not supported for %2$s.%3$s
+error.spark.unknown_tsfile_column = Unknown TsFile table column: %1$s
+error.spark.time_column_not_null = timeColumn must not be null
+error.spark.tag_column_not_null = TAG column must not be null: %1$s
+error.spark.unsupported_tsfile_type_write = Unsupported TsFile data type for write: %1$s
+error.spark.write_tablet_failed = Failed to write TsFile tablet
+error.spark.time_column_missing = Time column does not exist: %1$s
+error.spark.time_column_type_invalid = Time column must be LongType or TimestampType
+error.spark.duplicate_tag_columns = Duplicate tagColumns after lower-case normalization
+error.spark.time_column_in_tag_columns = timeColumn must not be listed in tagColumns
+error.spark.duplicate_field_columns = Duplicate fieldColumns after lower-case normalization
+error.spark.time_column_in_field_columns = timeColumn must not be listed in fieldColumns
+error.spark.column_both_tag_field = Column cannot be both TAG and FIELD: %1$s
+error.spark.field_required_write = At least one FIELD column is required for TsFile table writes
+error.spark.tag_column_missing = TAG column does not exist: %1$s
+error.spark.tag_column_must_string_spark = TAG column must be StringType: %1$s
+error.spark.field_column_missing = FIELD column does not exist: %1$s
+error.spark.duplicate_dataframe_column = Duplicate DataFrame column after lower-case normalization: %1$s
+error.spark.local_output_paths_only = Only local output paths are supported: %1$s
+error.spark.output_exists_append = TsFile output file already exists during append: %1$s
+error.spark.commit_failed = Failed to commit TsFile Spark write
+error.spark.delete_empty_cleanup_failed = Failed to delete empty path during TsFile write cleanup: %1$s
+error.spark.delete_cleanup_failed = Failed to delete path during TsFile write cleanup: %1$s
+error.spark.query_table_failed = Failed to query TsFile table data from %1$s
+error.spark.unsupported_tsfile_type = Unsupported TsFile data type: %1$s
+error.spark.unsupported_tsfile_type_connector = Unsupported TsFile data type for Spark connector: %1$s
+error.spark.unsupported_spark_field_type = Unsupported Spark SQL type for TsFile FIELD: %1$s
+error.spark.unsupported_timestamp_precision = Unsupported timestamp precision: %1$s
+error.spark.encoding_invalid = Unsupported encoding: %1$s
+error.spark.encoding_not_supported = Encoding %1$s is not supported for type %2$s
+error.spark.compression_invalid = Unsupported compression: %1$s
+error.spark.duplicate_external_schema_column = Duplicate external Spark schema column after lower-case normalization: %1$s
+error.spark.external_schema_column_missing = External Spark schema column does not exist in TsFile table metadata: %1$s
+error.spark.external_schema_type_mismatch = External Spark schema column %1$s has type %2$s, but TsFile table metadata has type %3$s
+error.spark.catalog_alter_unsupported = TsFile table catalog does not support ALTER TABLE
diff --git a/java/common/src/main/resources/org/apache/tsfile/i18n/messages_zh.properties b/java/common/src/main/resources/org/apache/tsfile/i18n/messages_zh.properties
index a1a437cfb..9c66b50b0 100644
--- a/java/common/src/main/resources/org/apache/tsfile/i18n/messages_zh.properties
+++ b/java/common/src/main/resources/org/apache/tsfile/i18n/messages_zh.properties
@@ -1548,3 +1548,72 @@ log.tools.csv_read_error = 读取 CSV 文件出错: %1$s
# CsvSourceReader — error closing CSV reader
log.tools.csv_close_reader_error = 关闭 CSV reader 出错
+
+# --- spark connector ERROR messages ---
+
+error.spark.write_table_required = 写入 TsFile table model 需要指定 option "table"
+error.spark.write_tag_columns_required = 写入 TsFile table model 需要指定 option "tagColumns"
+error.spark.null_tag_policy_unsupported = 仅支持 nullTagPolicy=error
+error.spark.unsupported_option = 不支持的 TsFile connector option: %1$s
+error.spark.model_unsupported = 仅支持 TsFile table model,但 option "model" 为 %1$s
+error.spark.multiple_load_paths_unsupported = 暂不支持多个 Spark load path;请改为传入目录或 glob path
+error.spark.path_required = TsFile connector 需要 path
+error.spark.positive_option_required = %1$s 必须为正数
+error.spark.timestamp_as_invalid = timestampAs 必须为 long 或 timestamp
+error.spark.timestamp_precision_invalid = timestampPrecision 必须为 ms、us 或 ns
+error.spark.column_list_empty = 列表 option 不能为空
+error.spark.no_tsfile_files = 路径下未找到 TsFile 文件: %1$s
+error.spark.multiple_tables_requires_table = TsFile %1$s 包含多个 table;请指定 option "table"
+error.spark.table_not_found_in_file = TsFile 文件 %1$s 不包含 table %2$s
+error.spark.read_table_metadata_failed = 读取 %1$s 的 TsFile table metadata 失败
+error.spark.incompatible_table_name = %1$s 中的 TsFile table schema 不兼容: 期望 table %2$s,但实际为 %3$s
+error.spark.incompatible_column_count = %1$s 中的 TsFile table schema 不兼容: column 数量不同
+error.spark.incompatible_table_schema_column = %1$s 中的 TsFile table schema 在 column %2$s 处不兼容: 期望 %3$s/%4$s/%5$s,但实际为 %6$s/%7$s/%8$s
+error.spark.incompatible_tag_column_count = %1$s 中的 TsFile table schema 不兼容: TAG column 数量不同
+error.spark.incompatible_tag_column = %1$s 中的 TsFile table schema 在 TAG column %2$s 处不兼容: 期望 %3$s/%4$s,但实际为 %5$s/%6$s
+error.spark.incompatible_field_column_type = %1$s 中的 TsFile table schema 不兼容: FIELD column %2$s 的 type 为 %4$s,但期望为 %3$s
+error.spark.path_not_exist = TsFile 输入路径不存在: %1$s
+error.spark.discover_path_failed = 发现 TsFile 输入路径失败: %1$s
+error.spark.input_not_tsfile = 输入路径不是 .tsfile 文件: %1$s
+error.spark.local_paths_only = 仅支持本地 TsFile 路径: %1$s
+error.spark.unknown_projected_column = 未知的 projected TsFile table column: %1$s
+error.spark.time_tag_projection_requires_field = Time/TAG-only projection 要求 TsFile table 中至少有一个 FIELD column
+error.spark.unsupported_time_filter_literal = 不支持的 time filter literal: %1$s
+error.spark.duplicate_tsfile_column = lower-case 归一化后 TsFile table column 重复: %1$s
+error.spark.duplicate_spark_schema_column = lower-case 归一化后 Spark schema column 重复: %1$s
+error.spark.tag_column_must_string_tsfile = TAG column %1$s.%2$s 必须为 TSDataType.STRING,但实际为 %3$s
+error.spark.unsupported_column_category = %2$s.%3$s 不支持 Column category %1$s
+error.spark.unknown_tsfile_column = 未知的 TsFile table column: %1$s
+error.spark.time_column_not_null = timeColumn 不能为 null
+error.spark.tag_column_not_null = TAG column 不能为 null: %1$s
+error.spark.unsupported_tsfile_type_write = 写入不支持的 TsFile data type: %1$s
+error.spark.write_tablet_failed = 写入 TsFile tablet 失败
+error.spark.time_column_missing = Time column 不存在: %1$s
+error.spark.time_column_type_invalid = Time column 必须为 LongType 或 TimestampType
+error.spark.duplicate_tag_columns = lower-case 归一化后 tagColumns 重复
+error.spark.time_column_in_tag_columns = timeColumn 不能出现在 tagColumns 中
+error.spark.duplicate_field_columns = lower-case 归一化后 fieldColumns 重复
+error.spark.time_column_in_field_columns = timeColumn 不能出现在 fieldColumns 中
+error.spark.column_both_tag_field = 同一列不能同时为 TAG 和 FIELD: %1$s
+error.spark.field_required_write = TsFile table 写入至少需要一个 FIELD column
+error.spark.tag_column_missing = TAG column 不存在: %1$s
+error.spark.tag_column_must_string_spark = TAG column 必须为 StringType: %1$s
+error.spark.field_column_missing = FIELD column 不存在: %1$s
+error.spark.duplicate_dataframe_column = lower-case 归一化后 DataFrame column 重复: %1$s
+error.spark.local_output_paths_only = 仅支持本地输出路径: %1$s
+error.spark.output_exists_append = append 时 TsFile 输出文件已存在: %1$s
+error.spark.commit_failed = 提交 TsFile Spark 写入失败
+error.spark.delete_empty_cleanup_failed = TsFile 写入清理期间删除空路径失败: %1$s
+error.spark.delete_cleanup_failed = TsFile 写入清理期间删除路径失败: %1$s
+error.spark.query_table_failed = 查询 %1$s 中的 TsFile table data 失败
+error.spark.unsupported_tsfile_type = 不支持的 TsFile data type: %1$s
+error.spark.unsupported_tsfile_type_connector = Spark connector 不支持的 TsFile data type: %1$s
+error.spark.unsupported_spark_field_type = TsFile FIELD 不支持的 Spark SQL type: %1$s
+error.spark.unsupported_timestamp_precision = 不支持的 timestamp precision: %1$s
+error.spark.encoding_invalid = 不支持的 encoding: %1$s
+error.spark.encoding_not_supported = Encoding %1$s 不支持 type %2$s
+error.spark.compression_invalid = 不支持的 compression: %1$s
+error.spark.duplicate_external_schema_column = lower-case 归一化后 external Spark schema column 重复: %1$s
+error.spark.external_schema_column_missing = TsFile table metadata 中不存在 external Spark schema column: %1$s
+error.spark.external_schema_type_mismatch = External Spark schema column %1$s 的 type 为 %2$s,但 TsFile table metadata 中的 type 为 %3$s
+error.spark.catalog_alter_unsupported = TsFile table catalog 不支持 ALTER TABLE
diff --git a/java/pom.xml b/java/pom.xml
index 3106d99bb..32851ed0a 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -33,6 +33,7 @@
common
tsfile
+ spark-tsfile
examples
tools
diff --git a/java/spark-tsfile/README.md b/java/spark-tsfile/README.md
new file mode 100644
index 000000000..51c7b34ff
--- /dev/null
+++ b/java/spark-tsfile/README.md
@@ -0,0 +1,169 @@
+
+
+# TsFile Spark Connector
+
+This module provides a Spark 3.x DataSource V2 connector for TsFile table model
+files. The short name is `tsfile`.
+
+## Read
+
+```scala
+val df = spark.read
+ .format("tsfile")
+ .option("model", "table")
+ .option("table", "weather")
+ .load("/data/tsfile/weather")
+
+df.select("time", "city", "temperature")
+ .where("time >= 1700000000000 and city = 'beijing'")
+ .show()
+```
+
+If every input TsFile contains exactly one table, `table` can be omitted and the
+connector will infer it from the TsFile metadata. If an input file contains
+multiple tables, `table` must be provided.
+
+The input path can be a single `.tsfile`, a directory containing `.tsfile` files,
+or a glob path. The initial connector discovers paths through Hadoop APIs but
+opens TsFile data with the local Java reader, so only local file paths are
+supported.
+
+## Write
+
+```scala
+val rows = Seq(
+ (1700000000000L, "beijing", 20, 0.32d),
+ (1700000001000L, "shanghai", 21, 0.35d)
+).toDF("time", "city", "temperature", "humidity")
+
+rows.write
+ .format("tsfile")
+ .option("model", "table")
+ .option("table", "weather")
+ .option("timeColumn", "time")
+ .option("tagColumns", "city")
+ .option("compression", "lz4")
+ .mode("append")
+ .save("/data/tsfile/weather")
+```
+
+Each Spark task writes a separate `part-*.tsfile`. Append mode adds new TsFile
+files to the output directory; it does not append rows to an existing TsFile.
+
+TAG columns must be non-null strings. FIELD columns default to all columns that
+are not the time column or TAG columns, and null FIELD values are written as
+sparse TsFile values.
+
+## SQL Temporary View
+
+```scala
+spark.read
+ .format("tsfile")
+ .option("table", "weather")
+ .load("/data/tsfile/weather/*.tsfile")
+ .createOrReplaceTempView("weather_tsfile")
+
+spark.sql(
+ """
+ |select time, city, temperature
+ |from weather_tsfile
+ |where city = 'beijing'
+ |order by time
+ |""".stripMargin)
+ .show()
+```
+
+## SQL CTAS Write
+
+SQL CTAS uses the connector's Spark DataSource V2 catalog, configured for the
+Spark session.
+
+```scala
+spark.conf.set(
+ "spark.sql.catalog.tsfile_catalog",
+ "org.apache.tsfile.spark.TsFileTableCatalog")
+
+spark.sql(
+ """
+ |CREATE TABLE tsfile_catalog.weather_copy
+ |USING tsfile
+ |TBLPROPERTIES (
+ | 'path' = '/data/tsfile/weather-copy',
+ | 'model' = 'table',
+ | 'table' = 'weather',
+ | 'tagColumns' = 'city'
+ |)
+ |AS SELECT time, city, temperature, humidity
+ |FROM source_weather
+ |""".stripMargin)
+```
+
+## Options
+
+| Option | Default | Description |
+| --- | --- | --- |
+| `model` | `table` | Must be `table`. |
+| `table` | none | Table name. Required for writes and for multi-table reads. |
+| `timeColumn` | `time` | Spark time column name. |
+| `tagColumns` | none | Comma-separated TAG columns. Required for writes. |
+| `fieldColumns` | inferred | Comma-separated FIELD columns for writes. |
+| `timestampAs` | `long` | Use `long` or `timestamp` for TsFile `TIMESTAMP` fields and the Spark time column. |
+| `timestampPrecision` | `ms` | Raw TsFile timestamp precision: `ms`, `us`, or `ns`. |
+| `mergeSchema` | `false` | When `true`, multi-file reads union compatible FIELD columns. TAG columns must keep the same order and type. |
+| `pushdown` | `true` | Enables supported time and TAG equality predicate pushdown. |
+| `compression` | default TsFile setting | Compression codec for written FIELD columns. |
+| `encoding` | default TsFile setting | Encoding for written FIELD columns. |
+| `nullTagPolicy` | `error` | Only `error` is supported. |
+| `maxRowsPerTablet` | `1024` | Maximum rows buffered in each TsFile `Tablet` before flushing. |
+
+Table and column names are normalized to lower case to match TsFile table model
+metadata behavior.
+
+## Initial Scope
+
+This module is the initial Spark 3.x DataSource V2 connector for TsFile table
+model files. It intentionally keeps the first production surface narrow:
+
+- Batch read and batch write are supported; streaming read/write are not.
+- Writes are append-only and create new `part-*.tsfile` files. Overwrite and
+ truncate semantics are not supported in this initial connector.
+- Only TsFile table model is supported. Tree model files are outside this
+ module's scope.
+- Input discovery supports a single `.tsfile`, a directory of `.tsfile` files,
+ and glob paths, but actual TsFile reading and writing is local-file only in
+ this initial version. Non-`file` Hadoop paths should be handled in a follow-up
+ change.
+- `mergeSchema=true` supports FIELD column union for multi-file reads. TAG
+ columns must keep the same order and type, and same-name FIELD columns must
+ keep the same TsFile type.
+- User-provided read schemas are validated against TsFile table metadata and
+ may be used as read projections. They cannot change the stored column types.
+- Predicate pushdown is limited to `time =`, `time >`, `time >=`, `time <`,
+ `time <=`, AND-combined time ranges, and string equality on TAG columns.
+ Unsupported predicates are returned to Spark as residual filters.
+- Unsupported table categories and types fail fast: `ATTRIBUTE`, `TIME`,
+ `VECTOR`, `UNKNOWN`, and `OBJECT` are not part of the first connector scope.
+- TAG columns must be non-null strings. FIELD columns may be null and are
+ written/read as sparse TsFile values.
+
+Follow-up issues should track non-local filesystem support, a broader predicate
+pushdown matrix, streaming semantics, and expanded type or category support.
diff --git a/java/spark-tsfile/pom.xml b/java/spark-tsfile/pom.xml
new file mode 100644
index 000000000..cca15b472
--- /dev/null
+++ b/java/spark-tsfile/pom.xml
@@ -0,0 +1,125 @@
+
+
+
+ 4.0.0
+
+ org.apache.tsfile
+ tsfile-java
+ 2.3.2-SNAPSHOT
+
+ spark-tsfile
+ TsFile: Java: Spark Connector
+ Spark SQL/DataFrame connector for TsFile table model
+
+ 3.5.8
+ 2.12
+
+
+
+ org.apache.tsfile
+ tsfile
+ 2.3.2-SNAPSHOT
+
+
+ org.antlr
+ antlr4-runtime
+
+
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ ${spark.version}
+ provided
+
+
+ org.apache.parquet
+ parquet-hadoop
+
+
+ commons-logging
+ commons-logging
+
+
+ org.lz4
+ lz4-java
+
+
+ org.antlr
+ antlr4-runtime
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+
+
+
+
+ org.antlr
+ antlr4-runtime
+ 4.9.3
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ 2.15.2
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ 2.15.2
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ 2.15.2
+ provided
+
+
+ junit
+ junit
+ test
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ ${argLine} --add-exports=java.base/sun.nio.ch=ALL-UNNAMED --add-exports=java.base/sun.security.action=ALL-UNNAMED
+
+
+
+
+
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileSparkException.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileSparkException.java
new file mode 100644
index 000000000..633b3cdef
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileSparkException.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+public class TsFileSparkException extends RuntimeException {
+
+ public TsFileSparkException(String message) {
+ super(message);
+ }
+
+ public TsFileSparkException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTable.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTable.java
new file mode 100644
index 000000000..4170c4052
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTable.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.catalog.SupportsRead;
+import org.apache.spark.sql.connector.catalog.SupportsWrite;
+import org.apache.spark.sql.connector.catalog.TableCapability;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class TsFileTable implements SupportsRead, SupportsWrite {
+
+ private static final Set CAPABILITIES;
+
+ static {
+ Set capabilities = new HashSet<>();
+ capabilities.add(TableCapability.BATCH_READ);
+ capabilities.add(TableCapability.BATCH_WRITE);
+ capabilities.add(TableCapability.ACCEPT_ANY_SCHEMA);
+ CAPABILITIES = Collections.unmodifiableSet(capabilities);
+ }
+
+ private final StructType schema;
+ private final Map properties;
+
+ public TsFileTable(StructType schema, Map properties) {
+ this.schema = schema;
+ this.properties = Collections.unmodifiableMap(new HashMap<>(properties));
+ }
+
+ @Override
+ public String name() {
+ String path = properties.get("path");
+ return path == null ? "tsfile" : "tsfile:" + path;
+ }
+
+ @Override
+ public StructType schema() {
+ return schema;
+ }
+
+ @Override
+ public Map properties() {
+ return properties;
+ }
+
+ @Override
+ public Set capabilities() {
+ return CAPABILITIES;
+ }
+
+ @Override
+ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) {
+ return new TsFileTableScanBuilder(TsFileTableOptions.forRead(mergedOptions(options)), schema);
+ }
+
+ @Override
+ public WriteBuilder newWriteBuilder(LogicalWriteInfo info) {
+ return new TsFileTableWriteBuilder(info, properties);
+ }
+
+ private CaseInsensitiveStringMap mergedOptions(CaseInsensitiveStringMap options) {
+ Map merged = new HashMap<>(properties);
+ merged.putAll(options.asCaseSensitiveMap());
+ return new CaseInsensitiveStringMap(merged);
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableBatchWrite.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableBatchWrite.java
new file mode 100644
index 000000000..cbcba9570
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableBatchWrite.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.connector.write.BatchWrite;
+import org.apache.spark.sql.connector.write.DataWriterFactory;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class TsFileTableBatchWrite implements BatchWrite {
+
+ private final TsFileTableWriteContext context;
+ private final String queryId;
+
+ public TsFileTableBatchWrite(TsFileTableWriteContext context, String queryId) {
+ this.context = context;
+ this.queryId = queryId;
+ }
+
+ @Override
+ public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+ return new TsFileTableDataWriterFactory(context, queryId);
+ }
+
+ @Override
+ public void commit(WriterCommitMessage[] messages) {
+ try {
+ Path outputPath = context.outputPath();
+ Files.createDirectories(outputPath);
+ List tsfileMessages = validMessages(messages);
+ validateFinalFilesDoNotExist(tsfileMessages);
+ validateNoDuplicateFinalFiles(tsfileMessages);
+ Path temporaryPath = outputPath.resolve("_temporary");
+ Path queryTemporaryPath = temporaryPath.resolve(TsFileTableDataWriterFactory.safeId(queryId));
+ for (TsFileTableWriterCommitMessage tsfileMessage : tsfileMessages) {
+ Files.move(Path.of(tsfileMessage.tempFile()), Path.of(tsfileMessage.finalFile()));
+ }
+ deleteRecursively(queryTemporaryPath);
+ deleteIfEmpty(temporaryPath);
+ } catch (FileAlreadyExistsException e) {
+ throw new TsFileSparkException(Messages.format("error.spark.output_exists_append", e), e);
+ } catch (IOException e) {
+ throw new TsFileSparkException(Messages.get("error.spark.commit_failed"), e);
+ }
+ }
+
+ @Override
+ public void abort(WriterCommitMessage[] messages) {
+ if (messages != null) {
+ for (WriterCommitMessage message : messages) {
+ if (message instanceof TsFileTableWriterCommitMessage) {
+ TsFileTableWriterCommitMessage tsfileMessage = (TsFileTableWriterCommitMessage) message;
+ deleteIfPresent(tsfileMessage.tempFile());
+ }
+ }
+ }
+ Path temporaryPath = context.outputPath().resolve("_temporary");
+ deleteRecursively(temporaryPath.resolve(TsFileTableDataWriterFactory.safeId(queryId)));
+ deleteIfEmpty(temporaryPath);
+ }
+
+ private static List validMessages(
+ WriterCommitMessage[] messages) {
+ List tsfileMessages = new ArrayList<>();
+ if (messages == null) {
+ return tsfileMessages;
+ }
+ for (WriterCommitMessage message : messages) {
+ if (!(message instanceof TsFileTableWriterCommitMessage)) {
+ continue;
+ }
+ TsFileTableWriterCommitMessage tsfileMessage = (TsFileTableWriterCommitMessage) message;
+ if (tsfileMessage.tempFile() != null) {
+ tsfileMessages.add(tsfileMessage);
+ }
+ }
+ return tsfileMessages;
+ }
+
+ private static void validateFinalFilesDoNotExist(List messages)
+ throws IOException {
+ for (TsFileTableWriterCommitMessage message : messages) {
+ Path finalFile = Path.of(message.finalFile());
+ if (Files.exists(finalFile)) {
+ throw new FileAlreadyExistsException(finalFile.toString());
+ }
+ }
+ }
+
+ private static void validateNoDuplicateFinalFiles(List messages)
+ throws IOException {
+ Set finalFiles = new HashSet<>();
+ for (TsFileTableWriterCommitMessage message : messages) {
+ Path finalFile = Path.of(message.finalFile());
+ if (!finalFiles.add(finalFile)) {
+ throw new FileAlreadyExistsException(finalFile.toString());
+ }
+ }
+ }
+
+ private static void deleteIfPresent(String path) {
+ if (path != null) {
+ deleteRecursively(Path.of(path));
+ }
+ }
+
+ private static void deleteIfEmpty(Path path) {
+ if (path == null || !Files.isDirectory(path)) {
+ return;
+ }
+ try (DirectoryStream stream = Files.newDirectoryStream(path)) {
+ if (!stream.iterator().hasNext()) {
+ Files.deleteIfExists(path);
+ }
+ } catch (IOException e) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.delete_empty_cleanup_failed", path), e);
+ }
+ }
+
+ private static void deleteRecursively(Path path) {
+ if (path == null || !Files.exists(path)) {
+ return;
+ }
+ try {
+ if (Files.isDirectory(path)) {
+ try (DirectoryStream stream = Files.newDirectoryStream(path)) {
+ for (Path child : stream) {
+ deleteRecursively(child);
+ }
+ }
+ }
+ Files.deleteIfExists(path);
+ } catch (IOException e) {
+ throw new TsFileSparkException(Messages.format("error.spark.delete_cleanup_failed", path), e);
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableCatalog.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableCatalog.java
new file mode 100644
index 000000000..032bf36bd
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableCatalog.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.Table;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.catalog.TableChange;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TsFileTableCatalog implements TableCatalog {
+
+ private final Map tables = new HashMap<>();
+ private String name;
+
+ @Override
+ public void initialize(String name, CaseInsensitiveStringMap options) {
+ this.name = name;
+ }
+
+ @Override
+ public String name() {
+ return name;
+ }
+
+ @Override
+ public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException {
+ List identifiers = new ArrayList<>();
+ for (StoredTable table : tables.values()) {
+ if (sameNamespace(namespace, table.identifier.namespace())) {
+ identifiers.add(table.identifier);
+ }
+ }
+ return identifiers.toArray(new Identifier[0]);
+ }
+
+ @Override
+ public Table loadTable(Identifier ident) throws NoSuchTableException {
+ StoredTable storedTable = tables.get(key(ident));
+ if (storedTable == null) {
+ throw new NoSuchTableException(ident);
+ }
+ return storedTable.table();
+ }
+
+ @Override
+ public Table createTable(
+ Identifier ident, StructType schema, Transform[] partitions, Map properties)
+ throws TableAlreadyExistsException, NoSuchNamespaceException {
+ String key = key(ident);
+ if (tables.containsKey(key)) {
+ throw new TableAlreadyExistsException(ident);
+ }
+ Map connectorOptions = connectorOptions(properties);
+ StoredTable storedTable = new StoredTable(ident, schema, connectorOptions);
+ tables.put(key, storedTable);
+ return storedTable.table();
+ }
+
+ @Override
+ public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchTableException {
+ throw new UnsupportedOperationException(Messages.get("error.spark.catalog_alter_unsupported"));
+ }
+
+ @Override
+ public boolean dropTable(Identifier ident) {
+ return tables.remove(key(ident)) != null;
+ }
+
+ @Override
+ public void renameTable(Identifier oldIdent, Identifier newIdent)
+ throws NoSuchTableException, TableAlreadyExistsException {
+ StoredTable storedTable = tables.remove(key(oldIdent));
+ if (storedTable == null) {
+ throw new NoSuchTableException(oldIdent);
+ }
+ String newKey = key(newIdent);
+ if (tables.containsKey(newKey)) {
+ tables.put(key(oldIdent), storedTable);
+ throw new TableAlreadyExistsException(newIdent);
+ }
+ tables.put(newKey, storedTable.withIdentifier(newIdent));
+ }
+
+ private static Map connectorOptions(Map properties) {
+ Map options = new HashMap<>();
+ for (Map.Entry entry : properties.entrySet()) {
+ String key = entry.getKey();
+ if (key.startsWith(TableCatalog.OPTION_PREFIX)) {
+ key = key.substring(TableCatalog.OPTION_PREFIX.length());
+ }
+ if (TsFileTableOptions.isKnownOption(key)) {
+ options.put(key, entry.getValue());
+ }
+ }
+ return options;
+ }
+
+ private static String key(Identifier ident) {
+ return String.join("\u0001", ident.namespace()) + "\u0002" + ident.name();
+ }
+
+ private static boolean sameNamespace(String[] left, String[] right) {
+ if (left.length != right.length) {
+ return false;
+ }
+ for (int i = 0; i < left.length; i++) {
+ if (!left[i].equals(right[i])) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private static class StoredTable {
+ private final Identifier identifier;
+ private final StructType schema;
+ private final Map properties;
+
+ private StoredTable(Identifier identifier, StructType schema, Map properties) {
+ this.identifier = identifier;
+ this.schema = schema;
+ this.properties = properties;
+ }
+
+ private StoredTable withIdentifier(Identifier identifier) {
+ return new StoredTable(identifier, schema, properties);
+ }
+
+ private Table table() {
+ return new TsFileTable(schema, properties);
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableDataWriterFactory.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableDataWriterFactory.java
new file mode 100644
index 000000000..106e5aa87
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableDataWriterFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.write.DataWriter;
+import org.apache.spark.sql.connector.write.DataWriterFactory;
+
+public class TsFileTableDataWriterFactory implements DataWriterFactory {
+
+ private final TsFileTableWriteContext context;
+ private final String queryId;
+
+ public TsFileTableDataWriterFactory(TsFileTableWriteContext context, String queryId) {
+ this.context = context;
+ this.queryId = queryId;
+ }
+
+ @Override
+ public DataWriter createWriter(int partitionId, long taskId) {
+ return new TsFileTablePartitionWriter(context, safeId(queryId), partitionId, taskId);
+ }
+
+ public static String safeId(String queryId) {
+ return queryId == null ? "query" : queryId.replaceAll("[^A-Za-z0-9._-]", "_");
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableFilterTranslator.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableFilterTranslator.java
new file mode 100644
index 000000000..52eaf82b3
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableFilterTranslator.java
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.sources.And;
+import org.apache.spark.sql.sources.EqualTo;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.GreaterThan;
+import org.apache.spark.sql.sources.GreaterThanOrEqual;
+import org.apache.spark.sql.sources.LessThan;
+import org.apache.spark.sql.sources.LessThanOrEqual;
+
+import java.math.BigInteger;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TsFileTableFilterTranslator {
+
+ private static final BigInteger BIG_LONG_MIN = BigInteger.valueOf(Long.MIN_VALUE);
+ private static final BigInteger BIG_LONG_MAX = BigInteger.valueOf(Long.MAX_VALUE);
+ private static final BigInteger BIG_ONE = BigInteger.ONE;
+ private static final BigInteger BIG_NS_PER_MICRO = BigInteger.valueOf(1_000L);
+
+ private final TsFileTableSchema tableSchema;
+ private final TsFileTableOptions options;
+ private long startTime = Long.MIN_VALUE;
+ private long endTime = Long.MAX_VALUE;
+ private final Map tagEqualities = new HashMap<>();
+ private final List pushedFilters = new ArrayList<>();
+ private final List residualFilters = new ArrayList<>();
+
+ public TsFileTableFilterTranslator(TsFileTableSchema tableSchema, TsFileTableOptions options) {
+ this.tableSchema = tableSchema;
+ this.options = options;
+ }
+
+ public Filter[] pushFilters(Filter[] filters) {
+ pushedFilters.clear();
+ residualFilters.clear();
+ startTime = Long.MIN_VALUE;
+ endTime = Long.MAX_VALUE;
+ tagEqualities.clear();
+ if (!options.pushdown()) {
+ Collections.addAll(residualFilters, filters);
+ return residualFilters.toArray(new Filter[0]);
+ }
+ for (Filter filter : filters) {
+ translate(filter);
+ }
+ return residualFilters.toArray(new Filter[0]);
+ }
+
+ private void translate(Filter filter) {
+ if (filter instanceof And) {
+ translate(((And) filter).left());
+ translate(((And) filter).right());
+ return;
+ }
+ if (filter instanceof EqualTo) {
+ EqualTo equalTo = (EqualTo) filter;
+ String attribute = normalizeAttribute(equalTo.attribute());
+ if (isTimeColumn(attribute)) {
+ pushTimeFilter(filter, equalTo.value(), TimeComparison.EQUAL);
+ return;
+ }
+ if (isTagColumn(attribute) && equalTo.value() instanceof String) {
+ String previous = tagEqualities.putIfAbsent(attribute, (String) equalTo.value());
+ if (previous == null || previous.equals(equalTo.value())) {
+ pushedFilters.add(filter);
+ } else {
+ residualFilters.add(filter);
+ }
+ return;
+ }
+ } else if (filter instanceof GreaterThan) {
+ GreaterThan greaterThan = (GreaterThan) filter;
+ if (isTimeColumn(normalizeAttribute(greaterThan.attribute()))) {
+ pushTimeFilter(filter, greaterThan.value(), TimeComparison.GREATER_THAN);
+ return;
+ }
+ } else if (filter instanceof GreaterThanOrEqual) {
+ GreaterThanOrEqual greaterThanOrEqual = (GreaterThanOrEqual) filter;
+ if (isTimeColumn(normalizeAttribute(greaterThanOrEqual.attribute()))) {
+ pushTimeFilter(filter, greaterThanOrEqual.value(), TimeComparison.GREATER_THAN_OR_EQUAL);
+ return;
+ }
+ } else if (filter instanceof LessThan) {
+ LessThan lessThan = (LessThan) filter;
+ if (isTimeColumn(normalizeAttribute(lessThan.attribute()))) {
+ pushTimeFilter(filter, lessThan.value(), TimeComparison.LESS_THAN);
+ return;
+ }
+ } else if (filter instanceof LessThanOrEqual) {
+ LessThanOrEqual lessThanOrEqual = (LessThanOrEqual) filter;
+ if (isTimeColumn(normalizeAttribute(lessThanOrEqual.attribute()))) {
+ pushTimeFilter(filter, lessThanOrEqual.value(), TimeComparison.LESS_THAN_OR_EQUAL);
+ return;
+ }
+ }
+ residualFilters.add(filter);
+ }
+
+ private boolean isTimeColumn(String attribute) {
+ return TsFileTableOptions.normalizeName(options.timeColumn()).equals(attribute);
+ }
+
+ private boolean isTagColumn(String attribute) {
+ TsFileTableSchema.ColumnInfo column = tableSchema.column(attribute);
+ return column != null && column.category() == ColumnCategory.TAG;
+ }
+
+ private String normalizeAttribute(String attribute) {
+ return TsFileTableOptions.normalizeName(attribute);
+ }
+
+ private void pushTimeFilter(Filter filter, Object value, TimeComparison comparison) {
+ RawTimeRange range = toRawTimeRange(value, comparison);
+ if (range == null) {
+ residualFilters.add(filter);
+ return;
+ }
+ startTime = Math.max(startTime, range.startTime);
+ endTime = Math.min(endTime, range.endTime);
+ pushedFilters.add(filter);
+ }
+
+ private RawTimeRange toRawTimeRange(Object value, TimeComparison comparison) {
+ if (value instanceof Number) {
+ return rawLongRange(((Number) value).longValue(), comparison);
+ }
+ if (value instanceof Timestamp) {
+ return timestampRange(timestampMicros((Timestamp) value), comparison);
+ }
+ return null;
+ }
+
+ private RawTimeRange timestampRange(long micros, TimeComparison comparison) {
+ switch (options.timestampPrecision()) {
+ case MS:
+ return timestampMillisRange(micros, comparison);
+ case US:
+ return rawLongRange(micros, comparison);
+ case NS:
+ return timestampNanosRange(micros, comparison);
+ default:
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.unsupported_timestamp_precision", options.timestampPrecision()));
+ }
+ }
+
+ private RawTimeRange timestampMillisRange(long micros, TimeComparison comparison) {
+ switch (comparison) {
+ case EQUAL:
+ if (Math.floorMod(micros, 1_000L) != 0) {
+ return RawTimeRange.empty();
+ }
+ return RawTimeRange.point(Math.floorDiv(micros, 1_000L));
+ case GREATER_THAN:
+ return rawLongRange(Math.floorDiv(micros, 1_000L), TimeComparison.GREATER_THAN);
+ case GREATER_THAN_OR_EQUAL:
+ return rawLongRange(ceilDiv(micros, 1_000L), TimeComparison.GREATER_THAN_OR_EQUAL);
+ case LESS_THAN:
+ return rawLongRange(ceilDiv(micros, 1_000L), TimeComparison.LESS_THAN);
+ case LESS_THAN_OR_EQUAL:
+ return rawLongRange(Math.floorDiv(micros, 1_000L), TimeComparison.LESS_THAN_OR_EQUAL);
+ default:
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.unsupported_timestamp_precision", options.timestampPrecision()));
+ }
+ }
+
+ private RawTimeRange timestampNanosRange(long micros, TimeComparison comparison) {
+ BigInteger value = BigInteger.valueOf(micros);
+ switch (comparison) {
+ case EQUAL:
+ return RawTimeRange.between(
+ value.multiply(BIG_NS_PER_MICRO),
+ value.add(BIG_ONE).multiply(BIG_NS_PER_MICRO).subtract(BIG_ONE));
+ case GREATER_THAN:
+ return RawTimeRange.atLeast(value.add(BIG_ONE).multiply(BIG_NS_PER_MICRO));
+ case GREATER_THAN_OR_EQUAL:
+ return RawTimeRange.atLeast(value.multiply(BIG_NS_PER_MICRO));
+ case LESS_THAN:
+ return RawTimeRange.atMost(value.multiply(BIG_NS_PER_MICRO).subtract(BIG_ONE));
+ case LESS_THAN_OR_EQUAL:
+ return RawTimeRange.atMost(value.add(BIG_ONE).multiply(BIG_NS_PER_MICRO).subtract(BIG_ONE));
+ default:
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.unsupported_timestamp_precision", options.timestampPrecision()));
+ }
+ }
+
+ private RawTimeRange rawLongRange(long rawTime, TimeComparison comparison) {
+ switch (comparison) {
+ case EQUAL:
+ return RawTimeRange.point(rawTime);
+ case GREATER_THAN:
+ return rawTime == Long.MAX_VALUE ? RawTimeRange.empty() : RawTimeRange.atLeast(rawTime + 1);
+ case GREATER_THAN_OR_EQUAL:
+ return RawTimeRange.atLeast(rawTime);
+ case LESS_THAN:
+ return rawTime == Long.MIN_VALUE ? RawTimeRange.empty() : RawTimeRange.atMost(rawTime - 1);
+ case LESS_THAN_OR_EQUAL:
+ return RawTimeRange.atMost(rawTime);
+ default:
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_time_filter_literal", rawTime));
+ }
+ }
+
+ private long timestampMicros(Timestamp timestamp) {
+ Instant instant = timestamp.toInstant();
+ return Math.addExact(
+ Math.multiplyExact(instant.getEpochSecond(), 1_000_000L), instant.getNano() / 1_000L);
+ }
+
+ private long ceilDiv(long value, long divisor) {
+ long floor = Math.floorDiv(value, divisor);
+ return Math.floorMod(value, divisor) == 0 ? floor : floor + 1;
+ }
+
+ public Filter[] pushedFilters() {
+ return pushedFilters.toArray(new Filter[0]);
+ }
+
+ public long startTime() {
+ return startTime;
+ }
+
+ public long endTime() {
+ return endTime;
+ }
+
+ public Map tagEqualities() {
+ return Collections.unmodifiableMap(tagEqualities);
+ }
+
+ private enum TimeComparison {
+ EQUAL,
+ GREATER_THAN,
+ GREATER_THAN_OR_EQUAL,
+ LESS_THAN,
+ LESS_THAN_OR_EQUAL
+ }
+
+ private static class RawTimeRange {
+ private final long startTime;
+ private final long endTime;
+
+ private RawTimeRange(long startTime, long endTime) {
+ this.startTime = startTime;
+ this.endTime = endTime;
+ }
+
+ private static RawTimeRange point(long value) {
+ return new RawTimeRange(value, value);
+ }
+
+ private static RawTimeRange atLeast(long value) {
+ return new RawTimeRange(value, Long.MAX_VALUE);
+ }
+
+ private static RawTimeRange atLeast(BigInteger value) {
+ if (value.compareTo(BIG_LONG_MAX) > 0) {
+ return empty();
+ }
+ return new RawTimeRange(clampLower(value), Long.MAX_VALUE);
+ }
+
+ private static RawTimeRange atMost(long value) {
+ return new RawTimeRange(Long.MIN_VALUE, value);
+ }
+
+ private static RawTimeRange atMost(BigInteger value) {
+ if (value.compareTo(BIG_LONG_MIN) < 0) {
+ return empty();
+ }
+ return new RawTimeRange(Long.MIN_VALUE, clampUpper(value));
+ }
+
+ private static RawTimeRange between(BigInteger startTime, BigInteger endTime) {
+ if (startTime.compareTo(BIG_LONG_MAX) > 0 || endTime.compareTo(BIG_LONG_MIN) < 0) {
+ return empty();
+ }
+ return new RawTimeRange(clampLower(startTime), clampUpper(endTime));
+ }
+
+ private static RawTimeRange empty() {
+ return new RawTimeRange(1L, 0L);
+ }
+
+ private static long clampLower(BigInteger value) {
+ return value.compareTo(BIG_LONG_MIN) < 0 ? Long.MIN_VALUE : value.longValue();
+ }
+
+ private static long clampUpper(BigInteger value) {
+ return value.compareTo(BIG_LONG_MAX) > 0 ? Long.MAX_VALUE : value.longValue();
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableInputPartition.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableInputPartition.java
new file mode 100644
index 000000000..7aa181f63
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableInputPartition.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.read.InputPartition;
+
+public class TsFileTableInputPartition implements InputPartition {
+
+ private final String file;
+ private final TsFileTableReadContext context;
+
+ public TsFileTableInputPartition(String file, TsFileTableReadContext context) {
+ this.file = file;
+ this.context = context;
+ }
+
+ public String file() {
+ return file;
+ }
+
+ public TsFileTableReadContext context() {
+ return context;
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableOptions.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableOptions.java
new file mode 100644
index 000000000..67b20379b
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableOptions.java
@@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TsFileTableOptions implements Serializable {
+
+ private static final Set KNOWN_OPTIONS =
+ new HashSet<>(
+ Arrays.asList(
+ "path",
+ "paths",
+ "model",
+ "table",
+ "timecolumn",
+ "tagcolumns",
+ "fieldcolumns",
+ "timestampas",
+ "timestampprecision",
+ "mergeschema",
+ "pushdown",
+ "compression",
+ "encoding",
+ "nulltagpolicy",
+ "maxrowspertablet"));
+
+ private final Map rawOptions;
+ private final String path;
+ private final String table;
+ private final String timeColumn;
+ private final TimestampAs timestampAs;
+ private final TimestampPrecision timestampPrecision;
+ private final boolean mergeSchema;
+ private final boolean pushdown;
+ private final List tagColumns;
+ private final List fieldColumns;
+ private final String compression;
+ private final String encoding;
+ private final String nullTagPolicy;
+ private final int maxRowsPerTablet;
+
+ private TsFileTableOptions(
+ Map rawOptions,
+ String path,
+ String table,
+ String timeColumn,
+ TimestampAs timestampAs,
+ TimestampPrecision timestampPrecision,
+ boolean mergeSchema,
+ boolean pushdown,
+ List tagColumns,
+ List fieldColumns,
+ String compression,
+ String encoding,
+ String nullTagPolicy,
+ int maxRowsPerTablet) {
+ this.rawOptions = rawOptions;
+ this.path = path;
+ this.table = normalizeName(table);
+ this.timeColumn = timeColumn;
+ this.timestampAs = timestampAs;
+ this.timestampPrecision = timestampPrecision;
+ this.mergeSchema = mergeSchema;
+ this.pushdown = pushdown;
+ this.tagColumns = tagColumns;
+ this.fieldColumns = fieldColumns;
+ this.compression = compression;
+ this.encoding = encoding;
+ this.nullTagPolicy = nullTagPolicy;
+ this.maxRowsPerTablet = maxRowsPerTablet;
+ }
+
+ public static TsFileTableOptions forRead(CaseInsensitiveStringMap options) {
+ validateKnownOptions(options);
+ validateModel(options);
+ String path = path(options);
+ String timeColumn = option(options, "timeColumn", "time");
+ TimestampAs timestampAs = timestampAs(option(options, "timestampAs", "long"));
+ TimestampPrecision timestampPrecision =
+ timestampPrecision(option(options, "timestampPrecision", "ms"));
+ boolean mergeSchema = options.getBoolean("mergeSchema", false);
+ return new TsFileTableOptions(
+ options.asCaseSensitiveMap(),
+ path,
+ blankToNull(options.get("table")),
+ timeColumn,
+ timestampAs,
+ timestampPrecision,
+ mergeSchema,
+ options.getBoolean("pushdown", true),
+ Collections.emptyList(),
+ Collections.emptyList(),
+ blankToNull(options.get("compression")),
+ blankToNull(options.get("encoding")),
+ option(options, "nullTagPolicy", "error"),
+ positiveInt(options, "maxRowsPerTablet", 1024));
+ }
+
+ public static TsFileTableOptions forWrite(CaseInsensitiveStringMap options) {
+ validateKnownOptions(options);
+ validateModel(options);
+ String path = path(options);
+ String table = blankToNull(options.get("table"));
+ if (table == null) {
+ throw new TsFileSparkException(Messages.get("error.spark.write_table_required"));
+ }
+ List tagColumns = parseColumns(options.get("tagColumns"));
+ if (tagColumns.isEmpty()) {
+ throw new TsFileSparkException(Messages.get("error.spark.write_tag_columns_required"));
+ }
+ String nullTagPolicy = option(options, "nullTagPolicy", "error").toLowerCase(Locale.ROOT);
+ if (!"error".equals(nullTagPolicy)) {
+ throw new TsFileSparkException(Messages.get("error.spark.null_tag_policy_unsupported"));
+ }
+ return new TsFileTableOptions(
+ options.asCaseSensitiveMap(),
+ path,
+ table,
+ option(options, "timeColumn", "time"),
+ timestampAs(option(options, "timestampAs", "long")),
+ timestampPrecision(option(options, "timestampPrecision", "ms")),
+ false,
+ options.getBoolean("pushdown", true),
+ tagColumns,
+ parseColumns(options.get("fieldColumns")),
+ blankToNull(options.get("compression")),
+ blankToNull(options.get("encoding")),
+ nullTagPolicy,
+ positiveInt(options, "maxRowsPerTablet", 1024));
+ }
+
+ private static void validateKnownOptions(CaseInsensitiveStringMap options) {
+ for (String key : options.keySet()) {
+ if (!isKnownOption(key)) {
+ throw new TsFileSparkException(Messages.format("error.spark.unsupported_option", key));
+ }
+ }
+ }
+
+ public static boolean isKnownOption(String key) {
+ return KNOWN_OPTIONS.contains(key.toLowerCase(Locale.ROOT));
+ }
+
+ private static void validateModel(CaseInsensitiveStringMap options) {
+ String model = option(options, "model", "table").toLowerCase(Locale.ROOT);
+ if (!"table".equals(model)) {
+ throw new TsFileSparkException(Messages.format("error.spark.model_unsupported", model));
+ }
+ }
+
+ private static String path(CaseInsensitiveStringMap options) {
+ if (blankToNull(options.get("paths")) != null) {
+ throw new TsFileSparkException(Messages.get("error.spark.multiple_load_paths_unsupported"));
+ }
+ String path = blankToNull(options.get("path"));
+ if (path == null) {
+ throw new TsFileSparkException(Messages.get("error.spark.path_required"));
+ }
+ return path;
+ }
+
+ private static String option(
+ CaseInsensitiveStringMap options, String optionName, String defaultValue) {
+ String value = blankToNull(options.get(optionName));
+ return value == null ? defaultValue : value;
+ }
+
+ private static int positiveInt(
+ CaseInsensitiveStringMap options, String optionName, int defaultValue) {
+ int value = options.getInt(optionName, defaultValue);
+ if (value <= 0) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.positive_option_required", optionName));
+ }
+ return value;
+ }
+
+ private static TimestampAs timestampAs(String value) {
+ switch (value.toLowerCase(Locale.ROOT)) {
+ case "long":
+ return TimestampAs.LONG;
+ case "timestamp":
+ return TimestampAs.TIMESTAMP;
+ default:
+ throw new TsFileSparkException(Messages.get("error.spark.timestamp_as_invalid"));
+ }
+ }
+
+ private static TimestampPrecision timestampPrecision(String value) {
+ switch (value.toLowerCase(Locale.ROOT)) {
+ case "ms":
+ return TimestampPrecision.MS;
+ case "us":
+ return TimestampPrecision.US;
+ case "ns":
+ return TimestampPrecision.NS;
+ default:
+ throw new TsFileSparkException(Messages.get("error.spark.timestamp_precision_invalid"));
+ }
+ }
+
+ private static List parseColumns(String value) {
+ if (blankToNull(value) == null) {
+ return Collections.emptyList();
+ }
+ List columns =
+ Arrays.stream(value.split(","))
+ .map(String::trim)
+ .filter(column -> !column.isEmpty())
+ .collect(Collectors.toCollection(ArrayList::new));
+ if (columns.isEmpty()) {
+ throw new TsFileSparkException(Messages.get("error.spark.column_list_empty"));
+ }
+ return Collections.unmodifiableList(columns);
+ }
+
+ public static String normalizeName(String name) {
+ return name == null ? null : name.toLowerCase(Locale.ROOT);
+ }
+
+ public static String blankToNull(String value) {
+ return value == null || value.trim().isEmpty() ? null : value.trim();
+ }
+
+ public Map rawOptions() {
+ return rawOptions;
+ }
+
+ public String path() {
+ return path;
+ }
+
+ public String table() {
+ return table;
+ }
+
+ public String timeColumn() {
+ return timeColumn;
+ }
+
+ public TimestampAs timestampAs() {
+ return timestampAs;
+ }
+
+ public TimestampPrecision timestampPrecision() {
+ return timestampPrecision;
+ }
+
+ public boolean mergeSchema() {
+ return mergeSchema;
+ }
+
+ public boolean pushdown() {
+ return pushdown;
+ }
+
+ public List tagColumns() {
+ return tagColumns;
+ }
+
+ public List fieldColumns() {
+ return fieldColumns;
+ }
+
+ public String compression() {
+ return compression;
+ }
+
+ public String encoding() {
+ return encoding;
+ }
+
+ public String nullTagPolicy() {
+ return nullTagPolicy;
+ }
+
+ public int maxRowsPerTablet() {
+ return maxRowsPerTablet;
+ }
+
+ public enum TimestampAs {
+ LONG,
+ TIMESTAMP
+ }
+
+ public enum TimestampPrecision {
+ MS,
+ US,
+ NS
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionReader.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionReader.java
new file mode 100644
index 000000000..0aba3d8f1
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionReader.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.exception.read.ReadProcessException;
+import org.apache.tsfile.exception.write.NoMeasurementException;
+import org.apache.tsfile.exception.write.NoTableException;
+import org.apache.tsfile.i18n.Messages;
+import org.apache.tsfile.read.filter.basic.Filter;
+import org.apache.tsfile.read.filter.factory.TagFilterBuilder;
+import org.apache.tsfile.read.query.dataset.ResultSet;
+import org.apache.tsfile.read.v4.DeviceTableModelReader;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TsFileTablePartitionReader implements PartitionReader {
+
+ private final String file;
+ private final TsFileTableReadContext context;
+ private final Map resultColumnIndex = new HashMap<>();
+ private DeviceTableModelReader reader;
+ private ResultSet resultSet;
+ private InternalRow current;
+ private boolean initialized;
+
+ public TsFileTablePartitionReader(String file, TsFileTableReadContext context) {
+ this.file = file;
+ this.context = context;
+ }
+
+ @Override
+ public boolean next() throws IOException {
+ initialize();
+ if (resultSet == null || !resultSet.next()) {
+ current = null;
+ return false;
+ }
+ current = buildRow();
+ return true;
+ }
+
+ private void initialize() throws IOException {
+ if (initialized) {
+ return;
+ }
+ initialized = true;
+ if (context.isEmptyTimeRange()) {
+ return;
+ }
+ List queryColumns = context.queryColumns(file);
+ for (int i = 0; i < queryColumns.size(); i++) {
+ resultColumnIndex.put(queryColumns.get(i), i + 2);
+ }
+ reader = new DeviceTableModelReader(new File(file));
+ try {
+ resultSet =
+ reader.query(
+ context.tableSchema().tableName(),
+ queryColumns,
+ context.startTime(),
+ context.endTime(),
+ tagFilter());
+ } catch (NoMeasurementException | NoTableException | ReadProcessException e) {
+ throw new IOException(Messages.format("error.spark.query_table_failed", file), e);
+ }
+ }
+
+ private Filter tagFilter() {
+ if (context.tagEqualities().isEmpty()) {
+ return null;
+ }
+ TagFilterBuilder builder = new TagFilterBuilder(context.tableSchema().toTableSchema());
+ Filter filter = null;
+ for (Map.Entry entry : context.tagEqualities().entrySet()) {
+ Filter current = builder.eq(entry.getKey(), entry.getValue());
+ filter = filter == null ? current : builder.and(filter, current);
+ }
+ return filter;
+ }
+
+ private InternalRow buildRow() {
+ StructType readSchema = context.readSchema();
+ Object[] values = new Object[readSchema.size()];
+ StructField[] fields = readSchema.fields();
+ String normalizedTime = TsFileTableOptions.normalizeName(context.options().timeColumn());
+ for (int i = 0; i < fields.length; i++) {
+ String normalizedName = TsFileTableOptions.normalizeName(fields[i].name());
+ if (normalizedTime.equals(normalizedName)) {
+ values[i] = convertTime(resultSet.getLong(1), fields[i].dataType());
+ } else {
+ TsFileTableSchema.ColumnInfo column = context.tableSchema().column(normalizedName);
+ Integer resultIndex = resultColumnIndex.get(normalizedName);
+ values[i] =
+ resultIndex == null ? null : convertColumn(column, resultIndex, fields[i].dataType());
+ }
+ }
+ return new GenericInternalRow(values);
+ }
+
+ private Object convertTime(long rawTime, DataType sparkType) {
+ if (sparkType.sameType(DataTypes.TimestampType)) {
+ return TsFileTableTypeConverter.rawToTimestampMicros(
+ rawTime, context.options().timestampPrecision());
+ }
+ return rawTime;
+ }
+
+ private Object convertColumn(
+ TsFileTableSchema.ColumnInfo column, int resultIndex, DataType sparkType) {
+ if (resultSet.isNull(resultIndex)) {
+ return null;
+ }
+ TSDataType type = column.type();
+ switch (type) {
+ case BOOLEAN:
+ return resultSet.getBoolean(resultIndex);
+ case INT32:
+ return resultSet.getInt(resultIndex);
+ case INT64:
+ return resultSet.getLong(resultIndex);
+ case FLOAT:
+ return resultSet.getFloat(resultIndex);
+ case DOUBLE:
+ return resultSet.getDouble(resultIndex);
+ case TEXT:
+ case STRING:
+ return UTF8String.fromString(resultSet.getString(resultIndex));
+ case DATE:
+ LocalDate date = resultSet.getDate(resultIndex);
+ return TsFileTableTypeConverter.toSparkDate(date);
+ case TIMESTAMP:
+ long rawTimestamp = resultSet.getLong(resultIndex);
+ if (sparkType.sameType(DataTypes.TimestampType)) {
+ return TsFileTableTypeConverter.rawToTimestampMicros(
+ rawTimestamp, context.options().timestampPrecision());
+ }
+ return rawTimestamp;
+ case BLOB:
+ return resultSet.getBinary(resultIndex);
+ case VECTOR:
+ case UNKNOWN:
+ case OBJECT:
+ default:
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_tsfile_type", type));
+ }
+ }
+
+ @Override
+ public InternalRow get() {
+ return current;
+ }
+
+ @Override
+ public void close() {
+ if (resultSet != null) {
+ resultSet.close();
+ }
+ if (reader != null) {
+ reader.close();
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionReaderFactory.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionReaderFactory.java
new file mode 100644
index 000000000..d29054ace
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionReaderFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+
+public class TsFileTablePartitionReaderFactory implements PartitionReaderFactory {
+
+ @Override
+ public PartitionReader createReader(InputPartition partition) {
+ TsFileTableInputPartition inputPartition = (TsFileTableInputPartition) partition;
+ return new TsFileTablePartitionReader(inputPartition.file(), inputPartition.context());
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionWriter.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionWriter.java
new file mode 100644
index 000000000..cf2386dff
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTablePartitionWriter.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.exception.write.WriteProcessException;
+import org.apache.tsfile.i18n.Messages;
+import org.apache.tsfile.utils.Binary;
+import org.apache.tsfile.write.TsFileWriter;
+import org.apache.tsfile.write.record.Tablet;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.write.DataWriter;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+public class TsFileTablePartitionWriter implements DataWriter {
+
+ private final TsFileTableWriteContext context;
+ private final String queryId;
+ private final int partitionId;
+ private final long taskId;
+ private Path tempFile;
+ private Path finalFile;
+ private TsFileWriter writer;
+ private Tablet tablet;
+ private boolean closed;
+
+ public TsFileTablePartitionWriter(
+ TsFileTableWriteContext context, String queryId, int partitionId, long taskId) {
+ this.context = context;
+ this.queryId = queryId;
+ this.partitionId = partitionId;
+ this.taskId = taskId;
+ }
+
+ @Override
+ public void write(InternalRow record) throws IOException {
+ initializeWriterIfNeeded();
+ int rowIndex = tablet.getRowSize();
+ tablet.addTimestamp(rowIndex, readTime(record));
+ for (TsFileTableWriteContext.WriteColumn column : context.columns()) {
+ addColumnValue(record, rowIndex, column);
+ }
+ if (tablet.getRowSize() >= context.options().maxRowsPerTablet()) {
+ flushTablet();
+ }
+ }
+
+ private void initializeWriterIfNeeded() throws IOException {
+ if (writer != null) {
+ return;
+ }
+ Path outputPath = context.outputPath();
+ Path tempDir = outputPath.resolve("_temporary").resolve(queryId);
+ Files.createDirectories(tempDir);
+ String fileName = String.format("part-%s-%05d-%020d.tsfile", queryId, partitionId, taskId);
+ tempFile = tempDir.resolve(fileName);
+ finalFile = outputPath.resolve(fileName);
+ writer = new TsFileWriter(tempFile.toFile());
+ writer.registerTableSchema(context.tableSchema());
+ tablet =
+ new Tablet(
+ context.options().table(),
+ context.columnNames(),
+ context.dataTypes(),
+ context.categories(),
+ context.options().maxRowsPerTablet());
+ }
+
+ private long readTime(InternalRow record) {
+ if (record.isNullAt(context.timeColumnIndex())) {
+ throw new TsFileSparkException(Messages.get("error.spark.time_column_not_null"));
+ }
+ if (context.timeColumnType().sameType(DataTypes.TimestampType)) {
+ return TsFileTableTypeConverter.timestampMicrosToRaw(
+ record.getLong(context.timeColumnIndex()), context.options().timestampPrecision());
+ }
+ return record.getLong(context.timeColumnIndex());
+ }
+
+ private void addColumnValue(
+ InternalRow record, int rowIndex, TsFileTableWriteContext.WriteColumn column) {
+ if (column.category() == ColumnCategory.TAG && record.isNullAt(column.inputIndex())) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.tag_column_not_null", column.name()));
+ }
+ Object value = record.isNullAt(column.inputIndex()) ? null : readValue(record, column);
+ tablet.addValue(column.name(), rowIndex, value);
+ }
+
+ private Object readValue(InternalRow record, TsFileTableWriteContext.WriteColumn column) {
+ TSDataType type = column.tsType();
+ int index = column.inputIndex();
+ switch (type) {
+ case BOOLEAN:
+ return record.getBoolean(index);
+ case INT32:
+ return record.getInt(index);
+ case INT64:
+ return record.getLong(index);
+ case FLOAT:
+ return record.getFloat(index);
+ case DOUBLE:
+ return record.getDouble(index);
+ case TEXT:
+ case STRING:
+ UTF8String string = record.getUTF8String(index);
+ return string == null ? null : string.toString();
+ case DATE:
+ return TsFileTableTypeConverter.fromSparkDate(record.getInt(index));
+ case TIMESTAMP:
+ return TsFileTableTypeConverter.timestampMicrosToRaw(
+ record.getLong(index), context.options().timestampPrecision());
+ case BLOB:
+ return new Binary(record.getBinary(index));
+ case VECTOR:
+ case UNKNOWN:
+ case OBJECT:
+ default:
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_tsfile_type_write", type));
+ }
+ }
+
+ private void flushTablet() throws IOException {
+ if (tablet == null || tablet.getRowSize() == 0) {
+ return;
+ }
+ try {
+ writer.writeTable(tablet);
+ tablet.reset();
+ } catch (WriteProcessException e) {
+ throw new IOException(Messages.get("error.spark.write_tablet_failed"), e);
+ }
+ }
+
+ @Override
+ public WriterCommitMessage commit() throws IOException {
+ if (writer == null) {
+ return new TsFileTableWriterCommitMessage(null, null);
+ }
+ flushTablet();
+ closeWriter();
+ return new TsFileTableWriterCommitMessage(tempFile.toString(), finalFile.toString());
+ }
+
+ @Override
+ public void abort() throws IOException {
+ closeWriter();
+ if (tempFile != null) {
+ Files.deleteIfExists(tempFile);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ closeWriter();
+ }
+
+ private void closeWriter() throws IOException {
+ if (closed || writer == null) {
+ return;
+ }
+ closed = true;
+ writer.close();
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableProvider.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableProvider.java
new file mode 100644
index 000000000..9b83aa24f
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableProvider.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.catalog.Table;
+import org.apache.spark.sql.connector.catalog.TableProvider;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import java.util.Map;
+
+public class TsFileTableProvider implements TableProvider, DataSourceRegister {
+
+ @Override
+ public StructType inferSchema(CaseInsensitiveStringMap options) {
+ return TsFileTableSchemaInferer.infer(TsFileTableOptions.forRead(options))
+ .tableSchema()
+ .sparkSchema();
+ }
+
+ @Override
+ public Table getTable(
+ StructType schema, Transform[] partitioning, Map properties) {
+ return new TsFileTable(schema, properties);
+ }
+
+ @Override
+ public boolean supportsExternalMetadata() {
+ return true;
+ }
+
+ @Override
+ public String shortName() {
+ return "tsfile";
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableReadContext.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableReadContext.java
new file mode 100644
index 000000000..95db81033
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableReadContext.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+
+public class TsFileTableReadContext implements Serializable {
+
+ private final TsFileTableOptions options;
+ private final List files;
+ private final TsFileTableSchema tableSchema;
+ private final Map schemasByFile;
+ private final StructType readSchema;
+ private final List queryColumns;
+ private final String hiddenFieldColumn;
+ private final long startTime;
+ private final long endTime;
+ private final Map tagEqualities;
+
+ public TsFileTableReadContext(
+ TsFileTableOptions options,
+ List files,
+ TsFileTableSchema tableSchema,
+ Map schemasByFile,
+ StructType readSchema,
+ long startTime,
+ long endTime,
+ Map tagEqualities) {
+ this.options = options;
+ this.files = Collections.unmodifiableList(new ArrayList<>(files));
+ this.tableSchema = tableSchema;
+ this.schemasByFile = Collections.unmodifiableMap(new HashMap<>(schemasByFile));
+ this.readSchema = readSchema;
+ QueryColumns selected = selectQueryColumns(readSchema, tableSchema, options.timeColumn());
+ this.queryColumns = selected.queryColumns;
+ this.hiddenFieldColumn = selected.hiddenFieldColumn;
+ this.startTime = startTime;
+ this.endTime = endTime;
+ this.tagEqualities = tagEqualities;
+ }
+
+ public TsFileTableReadContext(
+ TsFileTableOptions options,
+ List files,
+ TsFileTableSchema tableSchema,
+ StructType readSchema,
+ long startTime,
+ long endTime,
+ Map tagEqualities) {
+ this(
+ options,
+ files,
+ tableSchema,
+ schemaByFile(files, tableSchema),
+ readSchema,
+ startTime,
+ endTime,
+ tagEqualities);
+ }
+
+ private static Map schemaByFile(
+ List files, TsFileTableSchema tableSchema) {
+ Map schemas = new HashMap<>();
+ for (String file : files) {
+ schemas.put(file, tableSchema);
+ }
+ return schemas;
+ }
+
+ private static QueryColumns selectQueryColumns(
+ StructType readSchema, TsFileTableSchema tableSchema, String timeColumn) {
+ LinkedHashSet queryColumns = new LinkedHashSet<>();
+ boolean hasFieldColumn = false;
+ String normalizedTime = TsFileTableOptions.normalizeName(timeColumn);
+ for (StructField field : readSchema.fields()) {
+ String normalizedName = TsFileTableOptions.normalizeName(field.name());
+ if (normalizedTime.equals(normalizedName)) {
+ continue;
+ }
+ TsFileTableSchema.ColumnInfo column = tableSchema.column(normalizedName);
+ if (column == null) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unknown_projected_column", field.name()));
+ }
+ queryColumns.add(column.name());
+ hasFieldColumn = hasFieldColumn || column.category() == ColumnCategory.FIELD;
+ }
+ String hiddenFieldColumn = null;
+ if (!hasFieldColumn) {
+ TsFileTableSchema.ColumnInfo hidden = tableSchema.firstFieldColumn();
+ if (hidden == null) {
+ throw new TsFileSparkException(
+ Messages.get("error.spark.time_tag_projection_requires_field"));
+ }
+ hiddenFieldColumn = hidden.name();
+ queryColumns.add(hidden.name());
+ }
+ return new QueryColumns(new ArrayList<>(queryColumns), hiddenFieldColumn);
+ }
+
+ public TsFileTableOptions options() {
+ return options;
+ }
+
+ public List files() {
+ return files;
+ }
+
+ public TsFileTableSchema tableSchema() {
+ return tableSchema;
+ }
+
+ public StructType readSchema() {
+ return readSchema;
+ }
+
+ public List queryColumns() {
+ return queryColumns;
+ }
+
+ public List queryColumns(String file) {
+ TsFileTableSchema fileSchema = schemasByFile.get(file);
+ if (fileSchema == null) {
+ fileSchema = tableSchema;
+ }
+ List fileQueryColumns = new ArrayList<>();
+ boolean hasFieldColumn = false;
+ for (String columnName : queryColumns) {
+ TsFileTableSchema.ColumnInfo fileColumn = fileSchema.column(columnName);
+ if (fileColumn == null) {
+ continue;
+ }
+ fileQueryColumns.add(fileColumn.name());
+ hasFieldColumn = hasFieldColumn || fileColumn.category() == ColumnCategory.FIELD;
+ }
+ if (!hasFieldColumn) {
+ TsFileTableSchema.ColumnInfo hidden = fileSchema.firstFieldColumn();
+ if (hidden == null) {
+ throw new TsFileSparkException(
+ Messages.get("error.spark.time_tag_projection_requires_field"));
+ }
+ if (!fileQueryColumns.contains(hidden.name())) {
+ fileQueryColumns.add(hidden.name());
+ }
+ }
+ return Collections.unmodifiableList(fileQueryColumns);
+ }
+
+ public String hiddenFieldColumn() {
+ return hiddenFieldColumn;
+ }
+
+ public long startTime() {
+ return startTime;
+ }
+
+ public long endTime() {
+ return endTime;
+ }
+
+ public Map tagEqualities() {
+ return tagEqualities;
+ }
+
+ public boolean isEmptyTimeRange() {
+ return startTime > endTime;
+ }
+
+ private static class QueryColumns {
+ private final List queryColumns;
+ private final String hiddenFieldColumn;
+
+ private QueryColumns(List queryColumns, String hiddenFieldColumn) {
+ this.queryColumns = Collections.unmodifiableList(queryColumns);
+ this.hiddenFieldColumn = hiddenFieldColumn;
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableScan.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableScan.java
new file mode 100644
index 000000000..1ef5c6e13
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableScan.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.read.Batch;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+public class TsFileTableScan implements Scan, Batch {
+
+ private final TsFileTableReadContext context;
+
+ public TsFileTableScan(TsFileTableReadContext context) {
+ this.context = context;
+ }
+
+ @Override
+ public StructType readSchema() {
+ return context.readSchema();
+ }
+
+ @Override
+ public String description() {
+ return "TsFile table scan: table=" + context.tableSchema().tableName();
+ }
+
+ @Override
+ public Batch toBatch() {
+ return this;
+ }
+
+ @Override
+ public InputPartition[] planInputPartitions() {
+ InputPartition[] partitions = new InputPartition[context.files().size()];
+ for (int i = 0; i < context.files().size(); i++) {
+ partitions[i] = new TsFileTableInputPartition(context.files().get(i), context);
+ }
+ return partitions;
+ }
+
+ @Override
+ public PartitionReaderFactory createReaderFactory() {
+ return new TsFileTablePartitionReaderFactory();
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableScanBuilder.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableScanBuilder.java
new file mode 100644
index 000000000..02bbbc452
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableScanBuilder.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.connector.read.SupportsPushDownFilters;
+import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class TsFileTableScanBuilder
+ implements ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns {
+
+ private final TsFileTableOptions options;
+ private final TsFileTableSchemaInferer.InferenceResult inferenceResult;
+ private final TsFileTableFilterTranslator filterTranslator;
+ private StructType readSchema;
+
+ public TsFileTableScanBuilder(TsFileTableOptions options) {
+ this(options, null);
+ }
+
+ public TsFileTableScanBuilder(TsFileTableOptions options, StructType externalSchema) {
+ this.options = options;
+ this.inferenceResult = TsFileTableSchemaInferer.infer(options);
+ validateExternalSchema(externalSchema, inferenceResult.tableSchema());
+ this.filterTranslator = new TsFileTableFilterTranslator(inferenceResult.tableSchema(), options);
+ this.readSchema =
+ externalSchema == null ? inferenceResult.tableSchema().sparkSchema() : externalSchema;
+ }
+
+ private static void validateExternalSchema(
+ StructType externalSchema, TsFileTableSchema tableSchema) {
+ if (externalSchema == null) {
+ return;
+ }
+ Set normalizedNames = new HashSet<>();
+ for (StructField field : externalSchema.fields()) {
+ String normalizedName = TsFileTableOptions.normalizeName(field.name());
+ if (!normalizedNames.add(normalizedName)) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.duplicate_external_schema_column", field.name()));
+ }
+ StructField expectedField = findExpectedField(tableSchema.sparkSchema(), normalizedName);
+ if (expectedField == null) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.external_schema_column_missing", field.name()));
+ }
+ if (!field.dataType().sameType(expectedField.dataType())) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.external_schema_type_mismatch",
+ field.name(),
+ field.dataType(),
+ expectedField.dataType()));
+ }
+ }
+ }
+
+ private static StructField findExpectedField(StructType expectedSchema, String normalizedName) {
+ for (StructField field : expectedSchema.fields()) {
+ if (normalizedName.equals(TsFileTableOptions.normalizeName(field.name()))) {
+ return field;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Filter[] pushFilters(Filter[] filters) {
+ return filterTranslator.pushFilters(filters);
+ }
+
+ @Override
+ public Filter[] pushedFilters() {
+ return filterTranslator.pushedFilters();
+ }
+
+ @Override
+ public void pruneColumns(StructType requiredSchema) {
+ this.readSchema = requiredSchema;
+ }
+
+ @Override
+ public Scan build() {
+ TsFileTableReadContext context =
+ new TsFileTableReadContext(
+ options,
+ inferenceResult.files(),
+ inferenceResult.tableSchema(),
+ inferenceResult.schemasByFile(),
+ readSchema,
+ filterTranslator.startTime(),
+ filterTranslator.endTime(),
+ filterTranslator.tagEqualities());
+ return new TsFileTableScan(context);
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableSchema.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableSchema.java
new file mode 100644
index 000000000..48ceb347e
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableSchema.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.file.metadata.TableSchema;
+import org.apache.tsfile.i18n.Messages;
+import org.apache.tsfile.write.schema.IMeasurementSchema;
+import org.apache.tsfile.write.schema.MeasurementSchema;
+
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class TsFileTableSchema implements Serializable {
+
+ private final String tableName;
+ private final String timeColumn;
+ private final List columns;
+ private final StructType sparkSchema;
+ private final Map columnByName;
+
+ public TsFileTableSchema(
+ String tableName,
+ String timeColumn,
+ List columns,
+ TsFileTableOptions.TimestampAs timestampAs) {
+ this.tableName = TsFileTableOptions.normalizeName(tableName);
+ this.timeColumn = timeColumn;
+ this.columns = Collections.unmodifiableList(new ArrayList<>(columns));
+ this.columnByName = buildColumnMap(columns);
+ this.sparkSchema = buildSparkSchema(timeColumn, columns, timestampAs);
+ }
+
+ private static Map buildColumnMap(List columns) {
+ Map map = new HashMap<>();
+ for (ColumnInfo column : columns) {
+ ColumnInfo previous = map.put(column.name(), column);
+ if (previous != null) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.duplicate_tsfile_column", column.name()));
+ }
+ }
+ return Collections.unmodifiableMap(map);
+ }
+
+ private static StructType buildSparkSchema(
+ String timeColumn, List columns, TsFileTableOptions.TimestampAs timestampAs) {
+ List fields = new ArrayList<>();
+ fields.add(
+ DataTypes.createStructField(
+ timeColumn,
+ timestampAs == TsFileTableOptions.TimestampAs.TIMESTAMP
+ ? DataTypes.TimestampType
+ : DataTypes.LongType,
+ false,
+ Metadata.empty()));
+ Set normalizedSparkNames = new HashSet<>();
+ normalizedSparkNames.add(TsFileTableOptions.normalizeName(timeColumn));
+ for (ColumnInfo column : columns) {
+ if (!normalizedSparkNames.add(column.name())) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.duplicate_spark_schema_column", column.name()));
+ }
+ fields.add(
+ DataTypes.createStructField(
+ column.name(),
+ TsFileTableTypeConverter.toSparkType(column.type(), timestampAs),
+ column.category() == ColumnCategory.FIELD,
+ Metadata.empty()));
+ }
+ return DataTypes.createStructType(fields);
+ }
+
+ public static TsFileTableSchema fromTableSchema(
+ TableSchema tableSchema, String timeColumn, TsFileTableOptions.TimestampAs timestampAs) {
+ List measurementSchemas = tableSchema.getColumnSchemas();
+ List categories = tableSchema.getColumnTypes();
+ List columns = new ArrayList<>();
+ for (int i = 0; i < measurementSchemas.size(); i++) {
+ String columnName =
+ TsFileTableOptions.normalizeName(measurementSchemas.get(i).getMeasurementName());
+ TSDataType type = measurementSchemas.get(i).getType();
+ ColumnCategory category = categories.get(i);
+ validateColumn(tableSchema.getTableName(), columnName, type, category);
+ columns.add(new ColumnInfo(columnName, type, category));
+ }
+ return new TsFileTableSchema(tableSchema.getTableName(), timeColumn, columns, timestampAs);
+ }
+
+ private static void validateColumn(
+ String tableName, String columnName, TSDataType type, ColumnCategory category) {
+ if (category == ColumnCategory.TAG) {
+ if (type != TSDataType.STRING) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.tag_column_must_string_tsfile", tableName, columnName, type));
+ }
+ return;
+ }
+ if (category != ColumnCategory.FIELD) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.unsupported_column_category", category, tableName, columnName));
+ }
+ TsFileTableTypeConverter.toSparkType(type, TsFileTableOptions.TimestampAs.LONG);
+ }
+
+ public TableSchema toTableSchema() {
+ List measurementSchemas = new ArrayList<>();
+ List categories = new ArrayList<>();
+ for (ColumnInfo column : columns) {
+ measurementSchemas.add(new MeasurementSchema(column.name(), column.type()));
+ categories.add(column.category());
+ }
+ return new TableSchema(tableName, measurementSchemas, categories);
+ }
+
+ public String tableName() {
+ return tableName;
+ }
+
+ public String timeColumn() {
+ return timeColumn;
+ }
+
+ public List columns() {
+ return columns;
+ }
+
+ public StructType sparkSchema() {
+ return sparkSchema;
+ }
+
+ public ColumnInfo column(String normalizedName) {
+ return columnByName.get(normalizedName);
+ }
+
+ public ColumnInfo firstFieldColumn() {
+ for (ColumnInfo column : columns) {
+ if (column.category() == ColumnCategory.FIELD) {
+ return column;
+ }
+ }
+ return null;
+ }
+
+ public List columnNames(List normalizedNames) {
+ List names = new ArrayList<>(normalizedNames.size());
+ for (String name : normalizedNames) {
+ ColumnInfo column = column(name);
+ if (column == null) {
+ throw new TsFileSparkException(Messages.format("error.spark.unknown_tsfile_column", name));
+ }
+ names.add(column.name());
+ }
+ return names;
+ }
+
+ public static class ColumnInfo implements Serializable {
+ private final String name;
+ private final TSDataType type;
+ private final ColumnCategory category;
+
+ public ColumnInfo(String name, TSDataType type, ColumnCategory category) {
+ this.name = TsFileTableOptions.normalizeName(name);
+ this.type = type;
+ this.category = category;
+ }
+
+ public String name() {
+ return name;
+ }
+
+ public TSDataType type() {
+ return type;
+ }
+
+ public ColumnCategory category() {
+ return category;
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableSchemaInferer.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableSchemaInferer.java
new file mode 100644
index 000000000..eb6422ec3
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableSchemaInferer.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.file.metadata.TableSchema;
+import org.apache.tsfile.i18n.Messages;
+import org.apache.tsfile.read.TsFileSequenceReader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public final class TsFileTableSchemaInferer {
+
+ private TsFileTableSchemaInferer() {}
+
+ public static InferenceResult infer(TsFileTableOptions options) {
+ List files = discoverTsFiles(options.path());
+ if (files.isEmpty()) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.no_tsfile_files", options.path()));
+ }
+
+ String selectedTable = options.table();
+ boolean inferTable = selectedTable == null;
+ TsFileTableSchema selectedSchema = null;
+ Map schemasByFile = new LinkedHashMap<>();
+ for (String file : files) {
+ Map schemaMap = readTableSchemas(file);
+ if (inferTable) {
+ if (schemaMap.size() != 1) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.multiple_tables_requires_table", file));
+ }
+ if (selectedTable == null) {
+ selectedTable = schemaMap.keySet().iterator().next();
+ }
+ }
+ TableSchema current = schemaMap.get(selectedTable);
+ if (current == null) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.table_not_found_in_file", file, selectedTable));
+ }
+ TsFileTableSchema currentSchema =
+ TsFileTableSchema.fromTableSchema(current, options.timeColumn(), options.timestampAs());
+ schemasByFile.put(file, currentSchema);
+ if (selectedSchema == null) {
+ selectedSchema = currentSchema;
+ } else if (options.mergeSchema()) {
+ selectedSchema =
+ mergeCompatible(selectedSchema, currentSchema, file, options.timestampAs());
+ } else {
+ validateCompatible(selectedSchema, currentSchema, file);
+ }
+ }
+
+ return new InferenceResult(files, schemasByFile, selectedSchema);
+ }
+
+ private static Map readTableSchemas(String file) {
+ try (TsFileSequenceReader reader = new TsFileSequenceReader(file)) {
+ return reader.getTableSchemaMap();
+ } catch (IOException e) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.read_table_metadata_failed", file), e);
+ }
+ }
+
+ private static void validateCompatible(
+ TsFileTableSchema expected, TsFileTableSchema actual, String actualFile) {
+ if (!expected.tableName().equals(actual.tableName())) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.incompatible_table_name",
+ actualFile,
+ expected.tableName(),
+ actual.tableName()));
+ }
+ if (expected.columns().size() != actual.columns().size()) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.incompatible_column_count", actualFile));
+ }
+ for (int i = 0; i < expected.columns().size(); i++) {
+ TsFileTableSchema.ColumnInfo expectedColumn = expected.columns().get(i);
+ TsFileTableSchema.ColumnInfo actualColumn = actual.columns().get(i);
+ if (!expectedColumn.name().equals(actualColumn.name())
+ || expectedColumn.type() != actualColumn.type()
+ || expectedColumn.category() != actualColumn.category()) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.incompatible_table_schema_column",
+ actualFile,
+ i,
+ expectedColumn.name(),
+ expectedColumn.type(),
+ expectedColumn.category(),
+ actualColumn.name(),
+ actualColumn.type(),
+ actualColumn.category()));
+ }
+ }
+ }
+
+ private static TsFileTableSchema mergeCompatible(
+ TsFileTableSchema expected,
+ TsFileTableSchema actual,
+ String actualFile,
+ TsFileTableOptions.TimestampAs timestampAs) {
+ if (!expected.tableName().equals(actual.tableName())) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.incompatible_table_name",
+ actualFile,
+ expected.tableName(),
+ actual.tableName()));
+ }
+ List expectedTags = columns(expected, ColumnCategory.TAG);
+ List actualTags = columns(actual, ColumnCategory.TAG);
+ if (expectedTags.size() != actualTags.size()) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.incompatible_tag_column_count", actualFile));
+ }
+ for (int i = 0; i < expectedTags.size(); i++) {
+ TsFileTableSchema.ColumnInfo expectedTag = expectedTags.get(i);
+ TsFileTableSchema.ColumnInfo actualTag = actualTags.get(i);
+ if (!expectedTag.name().equals(actualTag.name()) || expectedTag.type() != actualTag.type()) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.incompatible_tag_column",
+ actualFile,
+ i,
+ expectedTag.name(),
+ expectedTag.type(),
+ actualTag.name(),
+ actualTag.type()));
+ }
+ }
+
+ Map fields = new LinkedHashMap<>();
+ addFields(fields, expected, actualFile);
+ addFields(fields, actual, actualFile);
+
+ List mergedColumns = new ArrayList<>(expectedTags);
+ mergedColumns.addAll(fields.values());
+ return new TsFileTableSchema(
+ expected.tableName(), expected.timeColumn(), mergedColumns, timestampAs);
+ }
+
+ private static List columns(
+ TsFileTableSchema schema, ColumnCategory category) {
+ List columns = new ArrayList<>();
+ for (TsFileTableSchema.ColumnInfo column : schema.columns()) {
+ if (column.category() == category) {
+ columns.add(column);
+ }
+ }
+ return columns;
+ }
+
+ private static void addFields(
+ Map fields,
+ TsFileTableSchema schema,
+ String actualFile) {
+ for (TsFileTableSchema.ColumnInfo column : schema.columns()) {
+ if (column.category() != ColumnCategory.FIELD) {
+ continue;
+ }
+ TsFileTableSchema.ColumnInfo existing = fields.get(column.name());
+ if (existing == null) {
+ fields.put(column.name(), column);
+ } else if (existing.type() != column.type()) {
+ throw new TsFileSparkException(
+ Messages.format(
+ "error.spark.incompatible_field_column_type",
+ actualFile,
+ column.name(),
+ existing.type(),
+ column.type()));
+ }
+ }
+ }
+
+ private static List discoverTsFiles(String inputPath) {
+ try {
+ Configuration conf = new Configuration();
+ Path path = new Path(inputPath);
+ FileSystem fs = path.getFileSystem(conf);
+ List statuses = new ArrayList<>();
+ if (containsGlob(inputPath)) {
+ FileStatus[] globStatuses = fs.globStatus(path);
+ if (globStatuses != null) {
+ Collections.addAll(statuses, globStatuses);
+ }
+ } else if (fs.exists(path)) {
+ statuses.add(fs.getFileStatus(path));
+ } else {
+ throw new TsFileSparkException(Messages.format("error.spark.path_not_exist", inputPath));
+ }
+ List files = new ArrayList<>();
+ for (FileStatus status : statuses) {
+ collectTsFiles(fs, status, files);
+ }
+ return files.stream().distinct().sorted().collect(Collectors.toList());
+ } catch (IOException e) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.discover_path_failed", inputPath), e);
+ }
+ }
+
+ private static void collectTsFiles(FileSystem fs, FileStatus status, List files)
+ throws IOException {
+ if (status.isDirectory()) {
+ FileStatus[] children = fs.listStatus(status.getPath());
+ List sortedChildren = new ArrayList<>();
+ Collections.addAll(sortedChildren, children);
+ sortedChildren.sort(Comparator.comparing(child -> child.getPath().toString()));
+ for (FileStatus child : sortedChildren) {
+ if (isHidden(child.getPath())) {
+ continue;
+ }
+ if (child.isFile() && child.getPath().getName().endsWith(".tsfile")) {
+ files.add(toLocalFile(child.getPath()));
+ }
+ }
+ } else if (status.isFile()) {
+ if (!status.getPath().getName().endsWith(".tsfile")) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.input_not_tsfile", status.getPath()));
+ }
+ files.add(toLocalFile(status.getPath()));
+ }
+ }
+
+ private static boolean containsGlob(String path) {
+ return path.indexOf('*') >= 0 || path.indexOf('?') >= 0 || path.indexOf('[') >= 0;
+ }
+
+ private static boolean isHidden(Path path) {
+ String name = path.getName();
+ return name.startsWith("_") || name.startsWith(".");
+ }
+
+ private static String toLocalFile(Path path) {
+ URI uri = path.toUri();
+ String scheme = uri.getScheme();
+ if (scheme == null) {
+ return path.toString();
+ }
+ if ("file".equalsIgnoreCase(scheme)) {
+ return Paths.get(uri).toString();
+ }
+ throw new TsFileSparkException(Messages.format("error.spark.local_paths_only", path));
+ }
+
+ public static class InferenceResult {
+ private final List files;
+ private final Map schemasByFile;
+ private final TsFileTableSchema tableSchema;
+
+ public InferenceResult(
+ List files,
+ Map schemasByFile,
+ TsFileTableSchema tableSchema) {
+ this.files = Collections.unmodifiableList(new ArrayList<>(files));
+ this.schemasByFile = Collections.unmodifiableMap(new LinkedHashMap<>(schemasByFile));
+ this.tableSchema = tableSchema;
+ }
+
+ public List files() {
+ return files;
+ }
+
+ public TsFileTableSchema tableSchema() {
+ return tableSchema;
+ }
+
+ public Map schemasByFile() {
+ return schemasByFile;
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableTypeConverter.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableTypeConverter.java
new file mode 100644
index 000000000..59cb6a006
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableTypeConverter.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.file.metadata.enums.CompressionType;
+import org.apache.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.tsfile.i18n.Messages;
+
+import org.apache.spark.sql.types.BinaryType;
+import org.apache.spark.sql.types.BooleanType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.DateType;
+import org.apache.spark.sql.types.DoubleType;
+import org.apache.spark.sql.types.FloatType;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.LongType;
+import org.apache.spark.sql.types.StringType;
+import org.apache.spark.sql.types.TimestampType;
+
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Locale;
+
+public final class TsFileTableTypeConverter {
+
+ private TsFileTableTypeConverter() {}
+
+ public static DataType toSparkType(TSDataType type, TsFileTableOptions.TimestampAs timestampAs) {
+ switch (type) {
+ case BOOLEAN:
+ return DataTypes.BooleanType;
+ case INT32:
+ return DataTypes.IntegerType;
+ case INT64:
+ return DataTypes.LongType;
+ case FLOAT:
+ return DataTypes.FloatType;
+ case DOUBLE:
+ return DataTypes.DoubleType;
+ case TEXT:
+ case STRING:
+ return DataTypes.StringType;
+ case DATE:
+ return DataTypes.DateType;
+ case TIMESTAMP:
+ return timestampAs == TsFileTableOptions.TimestampAs.TIMESTAMP
+ ? DataTypes.TimestampType
+ : DataTypes.LongType;
+ case BLOB:
+ return DataTypes.BinaryType;
+ case VECTOR:
+ case UNKNOWN:
+ case OBJECT:
+ default:
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_tsfile_type_connector", type));
+ }
+ }
+
+ public static TSDataType toTsFileFieldType(
+ DataType sparkType, TsFileTableOptions.TimestampAs timestampAs) {
+ if (sparkType instanceof BooleanType) {
+ return TSDataType.BOOLEAN;
+ }
+ if (sparkType instanceof IntegerType) {
+ return TSDataType.INT32;
+ }
+ if (sparkType instanceof LongType) {
+ return TSDataType.INT64;
+ }
+ if (sparkType instanceof FloatType) {
+ return TSDataType.FLOAT;
+ }
+ if (sparkType instanceof DoubleType) {
+ return TSDataType.DOUBLE;
+ }
+ if (sparkType instanceof StringType) {
+ return TSDataType.STRING;
+ }
+ if (sparkType instanceof DateType) {
+ return TSDataType.DATE;
+ }
+ if (sparkType instanceof TimestampType) {
+ return TSDataType.TIMESTAMP;
+ }
+ if (sparkType instanceof BinaryType) {
+ return TSDataType.BLOB;
+ }
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_spark_field_type", sparkType));
+ }
+
+ public static long timestampMicrosToRaw(
+ long micros, TsFileTableOptions.TimestampPrecision precision) {
+ switch (precision) {
+ case MS:
+ return Math.floorDiv(micros, 1_000L);
+ case US:
+ return micros;
+ case NS:
+ return Math.multiplyExact(micros, 1_000L);
+ default:
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_timestamp_precision", precision));
+ }
+ }
+
+ public static long rawToTimestampMicros(
+ long raw, TsFileTableOptions.TimestampPrecision precision) {
+ switch (precision) {
+ case MS:
+ return Math.multiplyExact(raw, 1_000L);
+ case US:
+ return raw;
+ case NS:
+ return Math.floorDiv(raw, 1_000L);
+ default:
+ throw new TsFileSparkException(
+ Messages.format("error.spark.unsupported_timestamp_precision", precision));
+ }
+ }
+
+ public static int toSparkDate(LocalDate date) {
+ return Math.toIntExact(date.toEpochDay());
+ }
+
+ public static LocalDate fromSparkDate(int days) {
+ return LocalDate.ofEpochDay(days);
+ }
+
+ public static LocalDate millisToDate(long millis) {
+ return Instant.ofEpochMilli(millis).atZone(ZoneOffset.UTC).toLocalDate();
+ }
+
+ public static TSEncoding parseEncoding(String encoding, TSDataType type) {
+ if (encoding == null) {
+ return null;
+ }
+ TSEncoding parsed;
+ try {
+ parsed = TSEncoding.valueOf(encoding.toUpperCase(Locale.ROOT));
+ } catch (IllegalArgumentException e) {
+ throw new TsFileSparkException(Messages.format("error.spark.encoding_invalid", encoding), e);
+ }
+ if (!TSEncoding.isSupported(type, parsed)) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.encoding_not_supported", parsed, type));
+ }
+ return parsed;
+ }
+
+ public static CompressionType parseCompression(String compression) {
+ if (compression == null) {
+ return null;
+ }
+ try {
+ return CompressionType.valueOf(compression.toUpperCase(Locale.ROOT));
+ } catch (IllegalArgumentException e) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.compression_invalid", compression), e);
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWrite.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWrite.java
new file mode 100644
index 000000000..f4fd767bf
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWrite.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.write.BatchWrite;
+import org.apache.spark.sql.connector.write.Write;
+
+public class TsFileTableWrite implements Write {
+
+ private final TsFileTableWriteContext context;
+ private final String queryId;
+
+ public TsFileTableWrite(TsFileTableWriteContext context, String queryId) {
+ this.context = context;
+ this.queryId = queryId;
+ }
+
+ @Override
+ public BatchWrite toBatch() {
+ return new TsFileTableBatchWrite(context, queryId);
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriteBuilder.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriteBuilder.java
new file mode 100644
index 000000000..61ef86c80
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriteBuilder.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TsFileTableWriteBuilder implements WriteBuilder {
+
+ private final LogicalWriteInfo info;
+ private final Map tableProperties;
+
+ public TsFileTableWriteBuilder(LogicalWriteInfo info) {
+ this(info, Collections.emptyMap());
+ }
+
+ public TsFileTableWriteBuilder(LogicalWriteInfo info, Map tableProperties) {
+ this.info = info;
+ this.tableProperties = tableProperties;
+ }
+
+ @Override
+ public Write build() {
+ TsFileTableOptions options = TsFileTableOptions.forWrite(mergedOptions());
+ TsFileTableWriteContext context = TsFileTableWriteContext.build(options, info.schema());
+ return new TsFileTableWrite(context, info.queryId());
+ }
+
+ private CaseInsensitiveStringMap mergedOptions() {
+ Map merged = new HashMap<>(tableProperties);
+ merged.putAll(info.options().asCaseSensitiveMap());
+ return new CaseInsensitiveStringMap(merged);
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriteContext.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriteContext.java
new file mode 100644
index 000000000..ee68e182e
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriteContext.java
@@ -0,0 +1,320 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.file.metadata.TableSchema;
+import org.apache.tsfile.file.metadata.enums.CompressionType;
+import org.apache.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.tsfile.i18n.Messages;
+import org.apache.tsfile.write.schema.IMeasurementSchema;
+import org.apache.tsfile.write.schema.MeasurementSchema;
+
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class TsFileTableWriteContext implements Serializable {
+
+ private final TsFileTableOptions options;
+ private final String outputPath;
+ private final int timeColumnIndex;
+ private final DataType timeColumnType;
+ private final List columns;
+ private transient TableSchema tableSchema;
+
+ private TsFileTableWriteContext(
+ TsFileTableOptions options,
+ String outputPath,
+ int timeColumnIndex,
+ DataType timeColumnType,
+ List columns,
+ TableSchema tableSchema) {
+ this.options = options;
+ this.outputPath = outputPath;
+ this.timeColumnIndex = timeColumnIndex;
+ this.timeColumnType = timeColumnType;
+ this.columns = Collections.unmodifiableList(new ArrayList<>(columns));
+ this.tableSchema = tableSchema;
+ }
+
+ public static TsFileTableWriteContext build(TsFileTableOptions options, StructType schema) {
+ Map indexByName = buildIndex(schema);
+ String normalizedTime = TsFileTableOptions.normalizeName(options.timeColumn());
+ Integer timeColumnIndex = indexByName.get(normalizedTime);
+ if (timeColumnIndex == null) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.time_column_missing", options.timeColumn()));
+ }
+ DataType timeColumnType = schema.fields()[timeColumnIndex].dataType();
+ if (!timeColumnType.sameType(DataTypes.LongType)
+ && !timeColumnType.sameType(DataTypes.TimestampType)) {
+ throw new TsFileSparkException(Messages.get("error.spark.time_column_type_invalid"));
+ }
+
+ List normalizedTags = normalizeColumns(options.tagColumns());
+ Set tagSet = new LinkedHashSet<>(normalizedTags);
+ if (tagSet.size() != normalizedTags.size()) {
+ throw new TsFileSparkException(Messages.get("error.spark.duplicate_tag_columns"));
+ }
+ if (tagSet.contains(normalizedTime)) {
+ throw new TsFileSparkException(Messages.get("error.spark.time_column_in_tag_columns"));
+ }
+
+ List normalizedFields =
+ options.fieldColumns().isEmpty()
+ ? inferFieldColumns(schema, normalizedTime, tagSet)
+ : normalizeColumns(options.fieldColumns());
+ Set fieldSet = new LinkedHashSet<>(normalizedFields);
+ if (fieldSet.size() != normalizedFields.size()) {
+ throw new TsFileSparkException(Messages.get("error.spark.duplicate_field_columns"));
+ }
+ if (fieldSet.contains(normalizedTime)) {
+ throw new TsFileSparkException(Messages.get("error.spark.time_column_in_field_columns"));
+ }
+ for (String tag : tagSet) {
+ if (fieldSet.contains(tag)) {
+ throw new TsFileSparkException(Messages.format("error.spark.column_both_tag_field", tag));
+ }
+ }
+ if (fieldSet.isEmpty()) {
+ throw new TsFileSparkException(Messages.get("error.spark.field_required_write"));
+ }
+
+ List columns = new ArrayList<>();
+ for (String tag : normalizedTags) {
+ Integer index = indexByName.get(tag);
+ if (index == null) {
+ throw new TsFileSparkException(Messages.format("error.spark.tag_column_missing", tag));
+ }
+ DataType type = schema.fields()[index].dataType();
+ if (!type.sameType(DataTypes.StringType)) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.tag_column_must_string_spark", tag));
+ }
+ columns.add(new WriteColumn(tag, index, type, TSDataType.STRING, ColumnCategory.TAG));
+ }
+ for (String field : normalizedFields) {
+ Integer index = indexByName.get(field);
+ if (index == null) {
+ throw new TsFileSparkException(Messages.format("error.spark.field_column_missing", field));
+ }
+ DataType sparkType = schema.fields()[index].dataType();
+ TSDataType tsType =
+ TsFileTableTypeConverter.toTsFileFieldType(sparkType, options.timestampAs());
+ columns.add(new WriteColumn(field, index, sparkType, tsType, ColumnCategory.FIELD));
+ }
+
+ return new TsFileTableWriteContext(
+ options,
+ outputPath(options.path()),
+ timeColumnIndex,
+ timeColumnType,
+ columns,
+ buildTableSchema(options, columns));
+ }
+
+ private static Map buildIndex(StructType schema) {
+ Map indexByName = new HashMap<>();
+ StructField[] fields = schema.fields();
+ for (int i = 0; i < fields.length; i++) {
+ String normalized = TsFileTableOptions.normalizeName(fields[i].name());
+ Integer previous = indexByName.put(normalized, i);
+ if (previous != null) {
+ throw new TsFileSparkException(
+ Messages.format("error.spark.duplicate_dataframe_column", normalized));
+ }
+ }
+ return indexByName;
+ }
+
+ private static List normalizeColumns(List columns) {
+ List normalized = new ArrayList<>();
+ for (String column : columns) {
+ normalized.add(TsFileTableOptions.normalizeName(column));
+ }
+ return normalized;
+ }
+
+ private static List inferFieldColumns(
+ StructType schema, String normalizedTime, Set tagSet) {
+ List fields = new ArrayList<>();
+ for (StructField field : schema.fields()) {
+ String normalized = TsFileTableOptions.normalizeName(field.name());
+ if (!normalized.equals(normalizedTime) && !tagSet.contains(normalized)) {
+ fields.add(normalized);
+ }
+ }
+ return fields;
+ }
+
+ private static TableSchema buildTableSchema(
+ TsFileTableOptions options, List columns) {
+ List measurementSchemas = new ArrayList<>();
+ List categories = new ArrayList<>();
+ CompressionType compression = TsFileTableTypeConverter.parseCompression(options.compression());
+ for (WriteColumn column : columns) {
+ TSEncoding encoding = null;
+ if (column.category() == ColumnCategory.FIELD) {
+ encoding = TsFileTableTypeConverter.parseEncoding(options.encoding(), column.tsType());
+ }
+ if (encoding == null && compression == null) {
+ measurementSchemas.add(new MeasurementSchema(column.name(), column.tsType()));
+ } else if (encoding == null) {
+ measurementSchemas.add(
+ new MeasurementSchema(
+ column.name(),
+ column.tsType(),
+ org.apache.tsfile.common.conf.TSFileDescriptor.getInstance()
+ .getConfig()
+ .getValueEncoder(column.tsType()),
+ compression));
+ } else if (compression == null) {
+ measurementSchemas.add(new MeasurementSchema(column.name(), column.tsType(), encoding));
+ } else {
+ measurementSchemas.add(
+ new MeasurementSchema(column.name(), column.tsType(), encoding, compression));
+ }
+ categories.add(column.category());
+ }
+ return new TableSchema(options.table(), measurementSchemas, categories);
+ }
+
+ private static String outputPath(String value) {
+ try {
+ URI uri = URI.create(value);
+ if (uri.getScheme() == null) {
+ return Paths.get(value).toAbsolutePath().toString();
+ }
+ if ("file".equalsIgnoreCase(uri.getScheme())) {
+ return Paths.get(uri).toString();
+ }
+ } catch (IllegalArgumentException e) {
+ return Paths.get(value).toAbsolutePath().toString();
+ }
+ throw new TsFileSparkException(Messages.format("error.spark.local_output_paths_only", value));
+ }
+
+ public TsFileTableOptions options() {
+ return options;
+ }
+
+ public Path outputPath() {
+ return Paths.get(outputPath);
+ }
+
+ public int timeColumnIndex() {
+ return timeColumnIndex;
+ }
+
+ public DataType timeColumnType() {
+ return timeColumnType;
+ }
+
+ public List columns() {
+ return columns;
+ }
+
+ public TableSchema tableSchema() {
+ if (tableSchema == null) {
+ tableSchema = buildTableSchema(options, columns);
+ }
+ return tableSchema;
+ }
+
+ public List columnNames() {
+ List names = new ArrayList<>(columns.size());
+ for (WriteColumn column : columns) {
+ names.add(column.name());
+ }
+ return names;
+ }
+
+ public List dataTypes() {
+ List dataTypes = new ArrayList<>(columns.size());
+ for (WriteColumn column : columns) {
+ dataTypes.add(column.tsType());
+ }
+ return dataTypes;
+ }
+
+ public List categories() {
+ List categories = new ArrayList<>(columns.size());
+ for (WriteColumn column : columns) {
+ categories.add(column.category());
+ }
+ return categories;
+ }
+
+ public static class WriteColumn implements Serializable {
+ private final String name;
+ private final int inputIndex;
+ private final DataType sparkType;
+ private final TSDataType tsType;
+ private final ColumnCategory category;
+
+ public WriteColumn(
+ String name,
+ int inputIndex,
+ DataType sparkType,
+ TSDataType tsType,
+ ColumnCategory category) {
+ this.name = name;
+ this.inputIndex = inputIndex;
+ this.sparkType = sparkType;
+ this.tsType = tsType;
+ this.category = category;
+ }
+
+ public String name() {
+ return name;
+ }
+
+ public int inputIndex() {
+ return inputIndex;
+ }
+
+ public DataType sparkType() {
+ return sparkType;
+ }
+
+ public TSDataType tsType() {
+ return tsType;
+ }
+
+ public ColumnCategory category() {
+ return category;
+ }
+ }
+}
diff --git a/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriterCommitMessage.java b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriterCommitMessage.java
new file mode 100644
index 000000000..5b7cfc085
--- /dev/null
+++ b/java/spark-tsfile/src/main/java/org/apache/tsfile/spark/TsFileTableWriterCommitMessage.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+
+public class TsFileTableWriterCommitMessage implements WriterCommitMessage {
+
+ private final String tempFile;
+ private final String finalFile;
+
+ public TsFileTableWriterCommitMessage(String tempFile, String finalFile) {
+ this.tempFile = tempFile;
+ this.finalFile = finalFile;
+ }
+
+ public String tempFile() {
+ return tempFile;
+ }
+
+ public String finalFile() {
+ return finalFile;
+ }
+}
diff --git a/java/spark-tsfile/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/java/spark-tsfile/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 000000000..340ae918b
--- /dev/null
+++ b/java/spark-tsfile/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+org.apache.tsfile.spark.TsFileTableProvider
diff --git a/java/spark-tsfile/src/test/java/org/apache/tsfile/spark/TsFileTableConnectorTest.java b/java/spark-tsfile/src/test/java/org/apache/tsfile/spark/TsFileTableConnectorTest.java
new file mode 100644
index 000000000..2727ac759
--- /dev/null
+++ b/java/spark-tsfile/src/test/java/org/apache/tsfile/spark/TsFileTableConnectorTest.java
@@ -0,0 +1,1139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tsfile.spark;
+
+import org.apache.tsfile.enums.ColumnCategory;
+import org.apache.tsfile.enums.TSDataType;
+import org.apache.tsfile.file.metadata.TableSchema;
+import org.apache.tsfile.write.TsFileWriter;
+import org.apache.tsfile.write.record.Tablet;
+import org.apache.tsfile.write.schema.IMeasurementSchema;
+import org.apache.tsfile.write.schema.MeasurementSchema;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.sources.And;
+import org.apache.spark.sql.sources.EqualTo;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.GreaterThan;
+import org.apache.spark.sql.sources.GreaterThanOrEqual;
+import org.apache.spark.sql.sources.IsNotNull;
+import org.apache.spark.sql.sources.LessThan;
+import org.apache.spark.sql.sources.LessThanOrEqual;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TsFileTableConnectorTest {
+
+ static {
+ System.setProperty("tsfile.locale", "en");
+ }
+
+ private static SparkSession spark;
+
+ @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+ @BeforeClass
+ public static void startSpark() {
+ Assume.assumeTrue(
+ "Spark 3.x with Hadoop 3.3 cannot start on JDK "
+ + Runtime.version().feature()
+ + " because javax.security.auth.Subject.getSubject is unsupported",
+ Runtime.version().feature() < 24);
+ spark =
+ SparkSession.builder()
+ .master("local[2]")
+ .appName("tsfile-spark-connector-test")
+ .config("spark.ui.enabled", "false")
+ .config("spark.driver.host", "127.0.0.1")
+ .config("spark.driver.bindAddress", "127.0.0.1")
+ .config("spark.sql.shuffle.partitions", "2")
+ .config("spark.sql.session.timeZone", "UTC")
+ .getOrCreate();
+ spark.sparkContext().setLogLevel("ERROR");
+ }
+
+ @AfterClass
+ public static void stopSpark() {
+ if (spark != null) {
+ spark.stop();
+ spark = null;
+ }
+ }
+
+ @Test
+ public void parseReadAndWriteOptions() {
+ Map readOptions = new HashMap<>();
+ readOptions.put("path", "/tmp/table.tsfile");
+ readOptions.put("model", "table");
+ readOptions.put("table", "Weather");
+ readOptions.put("timestampAs", "timestamp");
+ readOptions.put("timestampPrecision", "us");
+ readOptions.put("mergeSchema", "true");
+
+ TsFileTableOptions parsedRead =
+ TsFileTableOptions.forRead(new CaseInsensitiveStringMap(readOptions));
+ assertEquals("/tmp/table.tsfile", parsedRead.path());
+ assertEquals("weather", parsedRead.table());
+ assertEquals(TsFileTableOptions.TimestampAs.TIMESTAMP, parsedRead.timestampAs());
+ assertEquals(TsFileTableOptions.TimestampPrecision.US, parsedRead.timestampPrecision());
+ assertTrue(parsedRead.mergeSchema());
+
+ Map writeOptions = new HashMap<>();
+ writeOptions.put("path", "/tmp/out");
+ writeOptions.put("table", "Weather");
+ writeOptions.put("tagColumns", "City, Station");
+ writeOptions.put("fieldColumns", "Temperature");
+ writeOptions.put("nullTagPolicy", "error");
+ TsFileTableOptions parsedWrite =
+ TsFileTableOptions.forWrite(new CaseInsensitiveStringMap(writeOptions));
+ assertEquals(Arrays.asList("City", "Station"), parsedWrite.tagColumns());
+ assertEquals(Collections.singletonList("Temperature"), parsedWrite.fieldColumns());
+
+ assertFailsContaining(
+ "Unsupported TsFile connector option",
+ () -> {
+ Map invalid = new HashMap<>(readOptions);
+ invalid.put("unknown", "x");
+ TsFileTableOptions.forRead(new CaseInsensitiveStringMap(invalid));
+ });
+ assertFailsContaining(
+ "Only TsFile table model is supported",
+ () -> {
+ Map invalid = new HashMap<>(readOptions);
+ invalid.put("model", "tree");
+ TsFileTableOptions.forRead(new CaseInsensitiveStringMap(invalid));
+ });
+ }
+
+ @Test
+ public void missingTableOnWriteFails() {
+ Dataset rows = spark.createDataFrame(sampleRows(), sampleSchema());
+ assertFailsContaining(
+ "requires option \"table\"",
+ () ->
+ rows.write()
+ .format("tsfile")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(temporaryFolder.newFolder("missing-table").getAbsolutePath()));
+ }
+
+ @Test
+ public void missingTagColumnsOnWriteFails() {
+ Dataset rows = spark.createDataFrame(sampleRows(), sampleSchema());
+ assertFailsContaining(
+ "requires option \"tagColumns\"",
+ () ->
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .mode(SaveMode.Append)
+ .save(temporaryFolder.newFolder("missing-tags").getAbsolutePath()));
+ }
+
+ @Test
+ public void invalidTimestampPrecisionFails() {
+ Map options = new HashMap<>();
+ options.put("path", "/tmp/weather.tsfile");
+ options.put("timestampPrecision", "seconds");
+
+ assertFailsContaining(
+ "timestampPrecision must be one of ms, us, or ns",
+ () -> TsFileTableOptions.forRead(new CaseInsensitiveStringMap(options)));
+ }
+
+ @Test
+ public void singleTableFileCanInferTable() throws Exception {
+ File file = temporaryFolder.newFile("single.tsfile");
+ writeWeatherFile(file, "Weather", 0);
+
+ Dataset df = spark.read().format("tsfile").load(file.getAbsolutePath());
+
+ assertEquals(3, df.count());
+ assertEquals(
+ Arrays.asList("time", "city", "temperature", "humidity"),
+ Arrays.asList(df.schema().fieldNames()));
+ }
+
+ @Test
+ public void multiTableFileRequiresTableOption() throws Exception {
+ File file = temporaryFolder.newFile("multi.tsfile");
+ try (TsFileWriter writer = new TsFileWriter(file)) {
+ writeWeatherTable(writer, "weather", 0);
+ writeWeatherTable(writer, "traffic", 10);
+ }
+
+ assertFailsContaining(
+ "multiple tables",
+ () -> spark.read().format("tsfile").load(file.getAbsolutePath()).count());
+ }
+
+ @Test
+ public void specifiedTableReadsMultiTableFile() throws Exception {
+ File file = temporaryFolder.newFile("specified-table.tsfile");
+ try (TsFileWriter writer = new TsFileWriter(file)) {
+ writeWeatherTable(writer, "weather", 0);
+ writeWeatherTable(writer, "traffic", 10);
+ }
+
+ List rows =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "traffic")
+ .load(file.getAbsolutePath())
+ .orderBy("time")
+ .collectAsList();
+
+ assertEquals(3, rows.size());
+ assertEquals(10L, rows.get(0).getLong(rows.get(0).fieldIndex("time")));
+ assertEquals(12L, rows.get(2).getLong(rows.get(2).fieldIndex("time")));
+ }
+
+ @Test
+ public void directoryInferenceFailsWhenLaterFileHasMultipleTables() throws Exception {
+ File directory = temporaryFolder.newFolder("multi-table-later-file");
+ writeWeatherFile(new File(directory, "part-a.tsfile"), "weather", 0);
+ try (TsFileWriter writer = new TsFileWriter(new File(directory, "part-b.tsfile"))) {
+ writeWeatherTable(writer, "weather", 10);
+ writeWeatherTable(writer, "traffic", 20);
+ }
+
+ assertFailsContaining(
+ "multiple tables",
+ () -> spark.read().format("tsfile").load(directory.getAbsolutePath()).count());
+ }
+
+ @Test
+ public void lowerCaseDuplicateColumnsFail() {
+ StructType schema =
+ new StructType()
+ .add("time", DataTypes.LongType, false)
+ .add("City", DataTypes.StringType, false)
+ .add("city", DataTypes.IntegerType, true);
+ assertFailsContaining(
+ "Duplicate DataFrame column",
+ () -> TsFileTableWriteContext.build(writeOptions("/tmp/out", "City"), schema));
+ }
+
+ @Test
+ public void nonStringTagColumnFails() {
+ StructType schema =
+ new StructType()
+ .add("time", DataTypes.LongType, false)
+ .add("city", DataTypes.IntegerType, false)
+ .add("temperature", DataTypes.IntegerType, true);
+ assertFailsContaining(
+ "TAG column must be StringType",
+ () -> TsFileTableWriteContext.build(writeOptions("/tmp/out", "city"), schema));
+ }
+
+ @Test
+ public void nullTagWriteFails() {
+ StructType schema =
+ new StructType()
+ .add("time", DataTypes.LongType, false)
+ .add("city", DataTypes.StringType, true)
+ .add("temperature", DataTypes.IntegerType, true);
+ Dataset rows =
+ spark.createDataFrame(Collections.singletonList(RowFactory.create(1L, null, 10)), schema);
+
+ assertFailsContaining(
+ "TAG column must not be null",
+ () ->
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(temporaryFolder.newFolder("null-tag").getAbsolutePath()));
+ }
+
+ @Test
+ public void unsupportedCategoryFails() {
+ TableSchema schema =
+ tableSchema(
+ "weather",
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("attr", TSDataType.STRING, ColumnCategory.ATTRIBUTE));
+ assertFailsContaining(
+ "Column category ATTRIBUTE is not supported",
+ () ->
+ TsFileTableSchema.fromTableSchema(schema, "time", TsFileTableOptions.TimestampAs.LONG));
+ }
+
+ @Test
+ public void unsupportedTimeCategoryAndObjectTypeFail() {
+ TableSchema timeCategorySchema =
+ tableSchema(
+ "weather",
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("event_time", TSDataType.TIMESTAMP, ColumnCategory.TIME));
+ assertFailsContaining(
+ "Column category TIME is not supported",
+ () ->
+ TsFileTableSchema.fromTableSchema(
+ timeCategorySchema, "time", TsFileTableOptions.TimestampAs.LONG));
+
+ TableSchema objectTypeSchema =
+ tableSchema(
+ "weather",
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("payload", TSDataType.OBJECT, ColumnCategory.FIELD));
+ assertFailsContaining(
+ "Unsupported TsFile data type for Spark connector: OBJECT",
+ () ->
+ TsFileTableSchema.fromTableSchema(
+ objectTypeSchema, "time", TsFileTableOptions.TimestampAs.LONG));
+ }
+
+ @Test
+ public void readsSingleFileAndSparseFieldNulls() throws Exception {
+ File file = temporaryFolder.newFile("sparse.tsfile");
+ writeWeatherFile(file, "weather", 0);
+
+ List rows =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(file.getAbsolutePath())
+ .orderBy("time")
+ .collectAsList();
+
+ assertEquals(3, rows.size());
+ assertEquals(20, rows.get(0).getInt(rows.get(0).fieldIndex("temperature")));
+ assertTrue(rows.get(1).isNullAt(rows.get(1).fieldIndex("temperature")));
+ assertEquals(32L, rows.get(2).getLong(rows.get(2).fieldIndex("humidity")));
+ }
+
+ @Test
+ public void readsDirectoryOfMultipleFiles() throws Exception {
+ File directory = temporaryFolder.newFolder("multi-file");
+ writeWeatherFile(new File(directory, "part-a.tsfile"), "weather", 0);
+ writeWeatherFile(new File(directory, "part-b.tsfile"), "weather", 100);
+
+ Dataset df =
+ spark.read().format("tsfile").option("table", "weather").load(directory.getAbsolutePath());
+
+ assertEquals(6, df.count());
+ assertEquals(1, df.where("time = 100").count());
+ }
+
+ @Test
+ public void readsGlobPath() throws Exception {
+ File directory = temporaryFolder.newFolder("glob-read");
+ writeWeatherFile(new File(directory, "part-a.tsfile"), "weather", 0);
+ writeWeatherFile(new File(directory, "part-b.tsfile"), "weather", 100);
+ writeWeatherFile(new File(directory, "other.tsfile"), "weather", 1000);
+
+ Dataset df =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(new File(directory, "part-*.tsfile").getAbsolutePath());
+
+ assertEquals(6, df.count());
+ assertEquals(1, df.where("time = 100").count());
+ assertEquals(0, df.where("time = 1000").count());
+ }
+
+ @Test
+ public void incompatibleSchemaFails() throws Exception {
+ File directory = temporaryFolder.newFolder("incompatible");
+ writeWeatherFile(new File(directory, "part-a.tsfile"), "weather", 0);
+ writeTsFile(
+ new File(directory, "part-b.tsfile"),
+ "weather",
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.DOUBLE, ColumnCategory.FIELD));
+
+ assertFailsContaining(
+ "Incompatible TsFile table schema",
+ () ->
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(directory.getAbsolutePath())
+ .count());
+ }
+
+ @Test
+ public void mergeSchemaUnionsFieldColumns() throws Exception {
+ File directory = temporaryFolder.newFolder("merge-schema");
+ try (TsFileWriter writer = new TsFileWriter(new File(directory, "part-a.tsfile"))) {
+ writeTable(
+ writer,
+ "weather",
+ new Object[][] {
+ {0L, "beijing", 20},
+ {1L, "shanghai", 21}
+ },
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.INT32, ColumnCategory.FIELD));
+ }
+ try (TsFileWriter writer = new TsFileWriter(new File(directory, "part-b.tsfile"))) {
+ writeTable(
+ writer,
+ "weather",
+ new Object[][] {
+ {10L, "beijing", 30L},
+ {11L, "shanghai", 31L}
+ },
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("humidity", TSDataType.INT64, ColumnCategory.FIELD));
+ }
+
+ Dataset df =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("mergeSchema", "true")
+ .load(directory.getAbsolutePath());
+ List rows = df.orderBy("time").collectAsList();
+
+ assertEquals(
+ Arrays.asList("time", "city", "temperature", "humidity"),
+ Arrays.asList(df.schema().fieldNames()));
+ assertEquals(4, rows.size());
+ assertEquals(20, rows.get(0).getInt(rows.get(0).fieldIndex("temperature")));
+ assertTrue(rows.get(0).isNullAt(rows.get(0).fieldIndex("humidity")));
+ assertTrue(rows.get(2).isNullAt(rows.get(2).fieldIndex("temperature")));
+ assertEquals(30L, rows.get(2).getLong(rows.get(2).fieldIndex("humidity")));
+ }
+
+ @Test
+ public void mergeSchemaRejectsFieldTypeConflict() throws Exception {
+ File directory = temporaryFolder.newFolder("merge-schema-conflict");
+ writeWeatherFile(new File(directory, "part-a.tsfile"), "weather", 0);
+ writeTsFile(
+ new File(directory, "part-b.tsfile"),
+ "weather",
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.DOUBLE, ColumnCategory.FIELD));
+
+ assertFailsContaining(
+ "FIELD column temperature has type DOUBLE but expected INT32",
+ () ->
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("mergeSchema", "true")
+ .load(directory.getAbsolutePath())
+ .count());
+ }
+
+ @Test
+ public void mergeSchemaRejectsTagMismatch() throws Exception {
+ File directory = temporaryFolder.newFolder("merge-schema-tag-mismatch");
+ try (TsFileWriter writer = new TsFileWriter(new File(directory, "part-a.tsfile"))) {
+ writeTable(
+ writer,
+ "weather",
+ new Object[][] {{0L, "beijing", 20}},
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.INT32, ColumnCategory.FIELD));
+ }
+ try (TsFileWriter writer = new TsFileWriter(new File(directory, "part-b.tsfile"))) {
+ writeTable(
+ writer,
+ "weather",
+ new Object[][] {{1L, "station-a", 21}},
+ new ColumnSpec("station", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.INT32, ColumnCategory.FIELD));
+ }
+
+ assertFailsContaining(
+ "TAG column",
+ () ->
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("mergeSchema", "true")
+ .load(directory.getAbsolutePath())
+ .count());
+ }
+
+ @Test
+ public void supportsColumnPruningAndTimeTagOnlyProjection() throws Exception {
+ File file = temporaryFolder.newFile("projection.tsfile");
+ writeWeatherFile(file, "weather", 0);
+
+ List projected =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(file.getAbsolutePath())
+ .select("time", "city")
+ .orderBy("time")
+ .collectAsList();
+
+ assertEquals(3, projected.size());
+ assertEquals(2, projected.get(0).size());
+ assertEquals(0L, projected.get(0).getLong(0));
+ assertEquals("beijing", projected.get(0).getString(1));
+ assertEquals(1L, projected.get(1).getLong(0));
+ assertEquals("shanghai", projected.get(1).getString(1));
+ assertEquals(2L, projected.get(2).getLong(0));
+ assertEquals("beijing", projected.get(2).getString(1));
+
+ TsFileTableSchema tableSchema =
+ TsFileTableSchemaInferer.infer(readOptions(file.getAbsolutePath(), "weather"))
+ .tableSchema();
+ StructType requiredSchema =
+ new StructType().add("time", DataTypes.LongType, false).add("city", DataTypes.StringType);
+ TsFileTableReadContext context =
+ new TsFileTableReadContext(
+ readOptions(file.getAbsolutePath(), "weather"),
+ Collections.singletonList(file.getAbsolutePath()),
+ tableSchema,
+ requiredSchema,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE,
+ Collections.emptyMap());
+ assertEquals(Arrays.asList("city", "temperature"), context.queryColumns());
+ assertEquals("temperature", context.hiddenFieldColumn());
+ }
+
+ @Test
+ public void timestampMsPushdownKeepsFractionalBoundaryRows() throws Exception {
+ File file = temporaryFolder.newFile("timestamp-ms.tsfile");
+ try (TsFileWriter writer = new TsFileWriter(file)) {
+ writeTable(
+ writer,
+ "weather",
+ new Object[][] {
+ {9L, "beijing", 19},
+ {10L, "beijing", 20},
+ {11L, "beijing", 21}
+ },
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.INT32, ColumnCategory.FIELD));
+ }
+
+ Dataset df =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("timestampAs", "timestamp")
+ .option("timestampPrecision", "ms")
+ .load(file.getAbsolutePath());
+
+ assertEquals(2, df.where("time < timestamp '1970-01-01 00:00:00.0105'").count());
+ assertEquals(1, df.where("time >= timestamp '1970-01-01 00:00:00.0105'").count());
+ assertEquals(0, df.where("time = timestamp '1970-01-01 00:00:00.0105'").count());
+ assertEquals(1, df.where("time = timestamp '1970-01-01 00:00:00.010'").count());
+ }
+
+ @Test
+ public void roundTripWriteAndRead() throws Exception {
+ File output = temporaryFolder.newFolder("round-trip");
+ Dataset rows = spark.createDataFrame(sampleRows(), sampleSchema());
+
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(output.getAbsolutePath());
+
+ List readRows =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(output.getAbsolutePath())
+ .orderBy("time")
+ .collectAsList();
+
+ assertEquals(3, readRows.size());
+ assertEquals("beijing", readRows.get(0).getString(readRows.get(0).fieldIndex("city")));
+ assertEquals(20, readRows.get(0).getInt(readRows.get(0).fieldIndex("temperature")));
+ assertTrue(readRows.get(1).isNullAt(readRows.get(1).fieldIndex("temperature")));
+ }
+
+ @Test
+ public void sqlTemporaryViewReadsTsFile() throws Exception {
+ File file = temporaryFolder.newFile("sql-view.tsfile");
+ writeWeatherFile(file, "weather", 0);
+
+ spark.sql(
+ "CREATE OR REPLACE TEMPORARY VIEW weather_sql USING tsfile OPTIONS (path '"
+ + file.getAbsolutePath().replace("'", "\\'")
+ + "', table 'weather')");
+
+ List rows =
+ spark
+ .sql(
+ "SELECT time, city, temperature FROM weather_sql "
+ + "WHERE city = 'beijing' ORDER BY time")
+ .collectAsList();
+
+ assertEquals(2, rows.size());
+ assertEquals(0L, rows.get(0).getLong(0));
+ assertEquals("beijing", rows.get(0).getString(1));
+ assertEquals(22, rows.get(1).getInt(2));
+ }
+
+ @Test
+ public void sqlCtasWritesTsFile() throws Exception {
+ File output = temporaryFolder.newFolder("sql-ctas");
+ Dataset rows = spark.createDataFrame(sampleRows(), sampleSchema());
+ String sourceView = "weather_ctas_source";
+ String catalog = "tsfile_ctas";
+ String ctasTable = "weather_ctas_" + System.nanoTime();
+ rows.createOrReplaceTempView(sourceView);
+ spark.conf().set("spark.sql.catalog." + catalog, TsFileTableCatalog.class.getName());
+
+ try {
+ spark.sql(
+ "CREATE TABLE "
+ + catalog
+ + "."
+ + ctasTable
+ + " USING tsfile TBLPROPERTIES ('path'='"
+ + escapeSql(output.getAbsolutePath())
+ + "', 'table'='weather', 'tagColumns'='city') AS SELECT * FROM "
+ + sourceView);
+
+ assertEquals(
+ 3,
+ spark
+ .sql("SELECT count(*) FROM " + catalog + "." + ctasTable)
+ .collectAsList()
+ .get(0)
+ .getLong(0));
+
+ List readRows =
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(output.getAbsolutePath())
+ .orderBy("time")
+ .collectAsList();
+
+ assertEquals(3, readRows.size());
+ assertEquals("beijing", readRows.get(0).getString(readRows.get(0).fieldIndex("city")));
+ assertEquals(20, readRows.get(0).getInt(readRows.get(0).fieldIndex("temperature")));
+ } finally {
+ spark.sql("DROP TABLE IF EXISTS " + catalog + "." + ctasTable);
+ spark.catalog().dropTempView(sourceView);
+ }
+ }
+
+ @Test
+ public void writesOneTsFilePerNonEmptySparkPartition() throws Exception {
+ File output = temporaryFolder.newFolder("partitioned");
+ Dataset rows =
+ spark
+ .range(0, 4, 1, 2)
+ .selectExpr(
+ "id as time",
+ "concat('city', cast(id % 2 as string)) as city",
+ "cast(id as int) as temperature",
+ "cast(id as bigint) as humidity");
+
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(output.getAbsolutePath());
+
+ File[] tsFiles = output.listFiles((dir, name) -> name.endsWith(".tsfile"));
+ assertNotNull(tsFiles);
+ assertEquals(2, tsFiles.length);
+ assertFalse(new File(output, "_temporary").exists());
+ }
+
+ @Test
+ public void appendTwiceCreatesDistinctTsFiles() throws Exception {
+ File output = temporaryFolder.newFolder("append-twice");
+ Dataset rows =
+ spark
+ .range(0, 4, 1, 2)
+ .selectExpr(
+ "id as time",
+ "concat('city', cast(id % 2 as string)) as city",
+ "cast(id as int) as temperature",
+ "cast(id as bigint) as humidity");
+
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(output.getAbsolutePath());
+ File[] firstWriteFiles = output.listFiles((dir, name) -> name.endsWith(".tsfile"));
+ assertNotNull(firstWriteFiles);
+ assertEquals(2, firstWriteFiles.length);
+
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(output.getAbsolutePath());
+
+ File[] secondWriteFiles = output.listFiles((dir, name) -> name.endsWith(".tsfile"));
+ assertNotNull(secondWriteFiles);
+ assertEquals(4, secondWriteFiles.length);
+ assertEquals(
+ 8,
+ spark
+ .read()
+ .format("tsfile")
+ .option("table", "weather")
+ .load(output.getAbsolutePath())
+ .count());
+ }
+
+ @Test
+ public void overwriteModeIsNotSupported() throws Exception {
+ File output = temporaryFolder.newFolder("overwrite-unsupported");
+ Dataset rows = spark.createDataFrame(sampleRows(), sampleSchema());
+
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Append)
+ .save(output.getAbsolutePath());
+
+ assertFailsContaining(
+ "does not support",
+ () ->
+ rows.write()
+ .format("tsfile")
+ .option("table", "weather")
+ .option("tagColumns", "city")
+ .mode(SaveMode.Overwrite)
+ .save(output.getAbsolutePath()));
+
+ File[] tsFiles = output.listFiles((dir, name) -> name.endsWith(".tsfile"));
+ assertNotNull(tsFiles);
+ assertEquals(2, tsFiles.length);
+ }
+
+ @Test
+ public void commitDoesNotOverwriteExistingOutputFile() throws Exception {
+ File output = temporaryFolder.newFolder("commit-collision");
+ Path outputPath = output.toPath();
+ Path tempFile =
+ outputPath.resolve("_temporary").resolve("query").resolve("part-query-0.tsfile");
+ Path finalFile = outputPath.resolve("part-query-0.tsfile");
+ Files.createDirectories(tempFile.getParent());
+ Files.write(tempFile, "new".getBytes(StandardCharsets.UTF_8));
+ Files.write(finalFile, "old".getBytes(StandardCharsets.UTF_8));
+ TsFileTableBatchWrite batchWrite =
+ new TsFileTableBatchWrite(
+ TsFileTableWriteContext.build(
+ writeOptions(output.getAbsolutePath(), "city"), sampleSchema()),
+ "query");
+
+ assertFailsContaining(
+ "already exists",
+ () ->
+ batchWrite.commit(
+ new TsFileTableWriterCommitMessage[] {
+ new TsFileTableWriterCommitMessage(tempFile.toString(), finalFile.toString())
+ }));
+
+ assertEquals("old", Files.readString(finalFile, StandardCharsets.UTF_8));
+ assertEquals("new", Files.readString(tempFile, StandardCharsets.UTF_8));
+ }
+
+ @Test
+ public void abortDoesNotDeleteExistingOutputFile() throws Exception {
+ File output = temporaryFolder.newFolder("abort-collision");
+ Path outputPath = output.toPath();
+ Path tempFile =
+ outputPath.resolve("_temporary").resolve("query").resolve("part-query-0.tsfile");
+ Path finalFile = outputPath.resolve("part-query-0.tsfile");
+ Files.createDirectories(tempFile.getParent());
+ Files.write(tempFile, "new".getBytes(StandardCharsets.UTF_8));
+ Files.write(finalFile, "old".getBytes(StandardCharsets.UTF_8));
+ TsFileTableBatchWrite batchWrite =
+ new TsFileTableBatchWrite(
+ TsFileTableWriteContext.build(
+ writeOptions(output.getAbsolutePath(), "city"), sampleSchema()),
+ "query");
+
+ batchWrite.abort(
+ new TsFileTableWriterCommitMessage[] {
+ new TsFileTableWriterCommitMessage(tempFile.toString(), finalFile.toString())
+ });
+
+ assertFalse(Files.exists(tempFile));
+ assertTrue(Files.exists(finalFile));
+ assertEquals("old", Files.readString(finalFile, StandardCharsets.UTF_8));
+ }
+
+ @Test
+ public void externalReadSchemaIsValidatedAndUsed() throws Exception {
+ assertTrue(new TsFileTableProvider().supportsExternalMetadata());
+ File file = temporaryFolder.newFile("external-schema.tsfile");
+ writeWeatherFile(file, "weather", 0);
+ StructType projectionSchema =
+ new StructType()
+ .add("time", DataTypes.LongType, false)
+ .add("city", DataTypes.StringType, false);
+
+ Dataset projected =
+ spark
+ .read()
+ .format("tsfile")
+ .schema(projectionSchema)
+ .option("table", "weather")
+ .load(file.getAbsolutePath());
+ assertEquals(Arrays.asList("time", "city"), Arrays.asList(projected.schema().fieldNames()));
+ assertEquals(3, projected.count());
+
+ StructType incompatibleSchema =
+ new StructType()
+ .add("time", DataTypes.LongType, false)
+ .add("temperature", DataTypes.DoubleType, true);
+ assertFailsContaining(
+ "External Spark schema column temperature has type DoubleType",
+ () ->
+ spark
+ .read()
+ .format("tsfile")
+ .schema(incompatibleSchema)
+ .option("table", "weather")
+ .load(file.getAbsolutePath())
+ .count());
+ }
+
+ @Test
+ public void pushesSupportedTimeRangeAndTagFilters() {
+ TsFileTableFilterTranslator translator =
+ new TsFileTableFilterTranslator(
+ weatherSchema(), readOptions("/tmp/weather.tsfile", "weather"));
+ Filter[] filters =
+ new Filter[] {
+ new And(
+ new And(new GreaterThanOrEqual("time", 10L), new LessThanOrEqual("time", 20L)),
+ new EqualTo("city", "beijing"))
+ };
+
+ Filter[] residuals = translator.pushFilters(filters);
+
+ assertEquals(0, residuals.length);
+ assertEquals(3, translator.pushedFilters().length);
+ assertEquals(10L, translator.startTime());
+ assertEquals(20L, translator.endTime());
+ assertEquals(Collections.singletonMap("city", "beijing"), translator.tagEqualities());
+ }
+
+ @Test
+ public void pushesTimeEqualityAndExclusiveBounds() {
+ TsFileTableFilterTranslator translator =
+ new TsFileTableFilterTranslator(
+ weatherSchema(), readOptions("/tmp/weather.tsfile", "weather"));
+
+ Filter[] residuals =
+ translator.pushFilters(
+ new Filter[] {
+ new EqualTo("time", 100L), new GreaterThan("time", 99L), new LessThan("time", 101L)
+ });
+
+ assertEquals(0, residuals.length);
+ assertEquals(3, translator.pushedFilters().length);
+ assertEquals(100L, translator.startTime());
+ assertEquals(100L, translator.endTime());
+ }
+
+ @Test
+ public void pushesTimestampMsFiltersWithPrecisionAwareBounds() {
+ TsFileTableOptions options = readTimestampOptions("/tmp/weather.tsfile", "weather", "ms");
+
+ assertPushedTimeRange(
+ new LessThan("time", timestampMicros(10_500L)), options, Long.MIN_VALUE, 10L);
+ assertPushedTimeRange(
+ new GreaterThanOrEqual("time", timestampMicros(10_500L)), options, 11L, Long.MAX_VALUE);
+ assertPushedTimeRange(new EqualTo("time", timestampMicros(10_000L)), options, 10L, 10L);
+ assertPushedTimeRange(new EqualTo("time", timestampMicros(10_500L)), options, 1L, 0L);
+ }
+
+ @Test
+ public void pushesTimestampNsFiltersWithMicrosRanges() {
+ TsFileTableOptions options = readTimestampOptions("/tmp/weather.tsfile", "weather", "ns");
+
+ assertPushedTimeRange(new EqualTo("time", timestampMicros(10L)), options, 10_000L, 10_999L);
+ assertPushedTimeRange(
+ new GreaterThan("time", timestampMicros(10L)), options, 11_000L, Long.MAX_VALUE);
+ assertPushedTimeRange(
+ new LessThanOrEqual("time", timestampMicros(10L)), options, Long.MIN_VALUE, 10_999L);
+ }
+
+ @Test
+ public void leavesUnsupportedFiltersAsResiduals() {
+ TsFileTableFilterTranslator translator =
+ new TsFileTableFilterTranslator(
+ weatherSchema(), readOptions("/tmp/weather.tsfile", "weather"));
+
+ Filter[] residuals =
+ translator.pushFilters(
+ new Filter[] {
+ new EqualTo("city", "beijing"), new EqualTo("temperature", 20), new IsNotNull("city")
+ });
+
+ assertEquals(2, residuals.length);
+ assertEquals(1, translator.pushedFilters().length);
+ assertEquals(Collections.singletonMap("city", "beijing"), translator.tagEqualities());
+ }
+
+ @Test
+ public void pushdownCanBeDisabled() {
+ Map options = new HashMap<>();
+ options.put("path", "/tmp/weather.tsfile");
+ options.put("table", "weather");
+ options.put("pushdown", "false");
+ TsFileTableFilterTranslator translator =
+ new TsFileTableFilterTranslator(
+ weatherSchema(), TsFileTableOptions.forRead(new CaseInsensitiveStringMap(options)));
+ Filter[] filters = new Filter[] {new EqualTo("city", "beijing"), new EqualTo("time", 1L)};
+
+ Filter[] residuals = translator.pushFilters(filters);
+
+ assertEquals(2, residuals.length);
+ assertEquals(0, translator.pushedFilters().length);
+ assertEquals(Long.MIN_VALUE, translator.startTime());
+ assertEquals(Long.MAX_VALUE, translator.endTime());
+ assertTrue(translator.tagEqualities().isEmpty());
+ }
+
+ private static TsFileTableOptions readOptions(String path, String table) {
+ Map options = new HashMap<>();
+ options.put("path", path);
+ options.put("table", table);
+ return TsFileTableOptions.forRead(new CaseInsensitiveStringMap(options));
+ }
+
+ private static TsFileTableOptions readTimestampOptions(
+ String path, String table, String timestampPrecision) {
+ Map options = new HashMap<>();
+ options.put("path", path);
+ options.put("table", table);
+ options.put("timestampAs", "timestamp");
+ options.put("timestampPrecision", timestampPrecision);
+ return TsFileTableOptions.forRead(new CaseInsensitiveStringMap(options));
+ }
+
+ private static Timestamp timestampMicros(long micros) {
+ long seconds = Math.floorDiv(micros, 1_000_000L);
+ long microsOfSecond = Math.floorMod(micros, 1_000_000L);
+ return Timestamp.from(Instant.ofEpochSecond(seconds, microsOfSecond * 1_000L));
+ }
+
+ private static String escapeSql(String value) {
+ return value.replace("'", "\\'");
+ }
+
+ private static void assertPushedTimeRange(
+ Filter filter, TsFileTableOptions options, long startTime, long endTime) {
+ TsFileTableFilterTranslator translator =
+ new TsFileTableFilterTranslator(weatherSchema(), options);
+
+ Filter[] residuals = translator.pushFilters(new Filter[] {filter});
+
+ assertEquals(0, residuals.length);
+ assertEquals(1, translator.pushedFilters().length);
+ assertEquals(startTime, translator.startTime());
+ assertEquals(endTime, translator.endTime());
+ }
+
+ private static TsFileTableOptions writeOptions(String path, String tagColumns) {
+ Map options = new HashMap<>();
+ options.put("path", path);
+ options.put("table", "weather");
+ options.put("tagColumns", tagColumns);
+ return TsFileTableOptions.forWrite(new CaseInsensitiveStringMap(options));
+ }
+
+ private static StructType sampleSchema() {
+ return new StructType()
+ .add("time", DataTypes.LongType, false)
+ .add("city", DataTypes.StringType, false)
+ .add("temperature", DataTypes.IntegerType, true)
+ .add("humidity", DataTypes.LongType, true);
+ }
+
+ private static List sampleRows() {
+ return Arrays.asList(
+ RowFactory.create(0L, "beijing", 20, 30L),
+ RowFactory.create(1L, "shanghai", null, 31L),
+ RowFactory.create(2L, "beijing", 22, 32L));
+ }
+
+ private static void writeWeatherFile(File file, String tableName, long offset) throws Exception {
+ try (TsFileWriter writer = new TsFileWriter(file)) {
+ writeWeatherTable(writer, tableName, offset);
+ }
+ }
+
+ private static void writeWeatherTable(TsFileWriter writer, String tableName, long offset)
+ throws Exception {
+ writeTable(
+ writer,
+ tableName,
+ new Object[][] {
+ {offset, "beijing", 20, 30L},
+ {offset + 1, "shanghai", null, 31L},
+ {offset + 2, "beijing", 22, 32L}
+ },
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.INT32, ColumnCategory.FIELD),
+ new ColumnSpec("humidity", TSDataType.INT64, ColumnCategory.FIELD));
+ }
+
+ private static void writeTsFile(File file, String tableName, ColumnSpec... columns)
+ throws Exception {
+ try (TsFileWriter writer = new TsFileWriter(file)) {
+ writeTable(writer, tableName, new Object[][] {{0L, "beijing", 20.0D}}, columns);
+ }
+ }
+
+ private static void writeTable(
+ TsFileWriter writer, String tableName, Object[][] rows, ColumnSpec... columns)
+ throws Exception {
+ writer.registerTableSchema(tableSchema(tableName, columns));
+ List columnNames = new ArrayList<>();
+ List dataTypes = new ArrayList<>();
+ List categories = new ArrayList<>();
+ for (ColumnSpec column : columns) {
+ columnNames.add(column.name.toLowerCase(Locale.ROOT));
+ dataTypes.add(column.type);
+ categories.add(column.category);
+ }
+ Tablet tablet =
+ new Tablet(tableName, columnNames, dataTypes, categories, Math.max(1, rows.length));
+ for (int rowIndex = 0; rowIndex < rows.length; rowIndex++) {
+ tablet.addTimestamp(rowIndex, ((Number) rows[rowIndex][0]).longValue());
+ for (int columnIndex = 0; columnIndex < columns.length; columnIndex++) {
+ tablet.addValue(columnNames.get(columnIndex), rowIndex, rows[rowIndex][columnIndex + 1]);
+ }
+ }
+ writer.writeTable(tablet);
+ }
+
+ private static TableSchema tableSchema(String tableName, ColumnSpec... columns) {
+ List measurementSchemas = new ArrayList<>();
+ List categories = new ArrayList<>();
+ for (ColumnSpec column : columns) {
+ measurementSchemas.add(
+ new MeasurementSchema(column.name.toLowerCase(Locale.ROOT), column.type));
+ categories.add(column.category);
+ }
+ return new TableSchema(tableName, measurementSchemas, categories);
+ }
+
+ private static TsFileTableSchema weatherSchema() {
+ return TsFileTableSchema.fromTableSchema(
+ tableSchema(
+ "weather",
+ new ColumnSpec("city", TSDataType.STRING, ColumnCategory.TAG),
+ new ColumnSpec("temperature", TSDataType.INT32, ColumnCategory.FIELD),
+ new ColumnSpec("humidity", TSDataType.INT64, ColumnCategory.FIELD)),
+ "time",
+ TsFileTableOptions.TimestampAs.LONG);
+ }
+
+ private static void assertFailsContaining(String expectedMessage, ThrowingRunnable runnable) {
+ try {
+ runnable.run();
+ } catch (Throwable error) {
+ if (!containsMessage(error, expectedMessage)) {
+ AssertionError assertionError =
+ new AssertionError("Expected failure containing: " + expectedMessage, error);
+ throw assertionError;
+ }
+ return;
+ }
+ fail("Expected failure containing: " + expectedMessage);
+ }
+
+ private static boolean containsMessage(Throwable error, String expectedMessage) {
+ Throwable current = error;
+ while (current != null) {
+ if (current.getMessage() != null && current.getMessage().contains(expectedMessage)) {
+ return true;
+ }
+ current = current.getCause();
+ }
+ return false;
+ }
+
+ private interface ThrowingRunnable {
+ void run() throws Throwable;
+ }
+
+ private static class ColumnSpec {
+ private final String name;
+ private final TSDataType type;
+ private final ColumnCategory category;
+
+ private ColumnSpec(String name, TSDataType type, ColumnCategory category) {
+ this.name = name;
+ this.type = type;
+ this.category = category;
+ }
+ }
+}