This is an automated email from the ASF dual-hosted git repository. corgy pushed a commit to branch dev in repository https://gitbox.apache.org/repos/asf/seatunnel.git
The following commit(s) were added to refs/heads/dev by this push:
new ea5bc51067 [Feature] [connector-file] Add configurable sheet_max_rows
support for Excel sink connector (#9668)
ea5bc51067 is described below
commit ea5bc5106779fd6751dc3bd3d8462f400358b64c
Author: zhenyue-xu <[email protected]>
AuthorDate: Mon Aug 11 13:43:00 2025 +0800
[Feature] [connector-file] Add configurable sheet_max_rows support for
Excel sink connector (#9668)
---
docs/en/connector-v2/sink/CosFile.md | 5 +
docs/en/connector-v2/sink/FtpFile.md | 5 +
docs/en/connector-v2/sink/HdfsFile.md | 1 +
docs/en/connector-v2/sink/LocalFile.md | 7 +-
docs/en/connector-v2/sink/ObsFile.md | 1 +
docs/en/connector-v2/sink/OssFile.md | 45 ++---
docs/en/connector-v2/sink/OssJindoFile.md | 73 ++++----
docs/en/connector-v2/sink/S3File.md | 5 +
docs/en/connector-v2/sink/SftpFile.md | 5 +
.../seatunnel/file/config/BaseFileSinkConfig.java | 7 +
.../seatunnel/file/config/FileBaseSinkOptions.java | 6 +
.../seatunnel/file/sink/util/ExcelGenerator.java | 39 +++--
.../seatunnel/file/writer/ExcelGeneratorTest.java | 183 +++++++++++++++++++++
13 files changed, 315 insertions(+), 67 deletions(-)
diff --git a/docs/en/connector-v2/sink/CosFile.md
b/docs/en/connector-v2/sink/CosFile.md
index 0a544fae06..457bee3770 100644
--- a/docs/en/connector-v2/sink/CosFile.md
+++ b/docs/en/connector-v2/sink/CosFile.md
@@ -65,6 +65,7 @@ To use this connector you need put
hadoop-cos-{hadoop.version}-{version}.jar and
| compress_codec | string | no | none
|
|
| common-options | object | no | -
|
|
| max_rows_in_memory | int | no | -
| Only used when file_format is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file_format is excel.
|
| sheet_name | string | no | Sheet${Random
number} | Only used when file_format is excel.
|
| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
@@ -203,6 +204,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows [int]
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name [string]
Writer the sheet of the workbook
diff --git a/docs/en/connector-v2/sink/FtpFile.md
b/docs/en/connector-v2/sink/FtpFile.md
index 8bba44368d..d72e504005 100644
--- a/docs/en/connector-v2/sink/FtpFile.md
+++ b/docs/en/connector-v2/sink/FtpFile.md
@@ -65,6 +65,7 @@ By default, we use 2PC commit to ensure `exactly-once`
| compress_codec | string | no | none
|
|
| common-options | object | no | -
|
|
| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file_format_type is excel.
|
| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
@@ -216,6 +217,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows [int]
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name [string]
Writer the sheet of the workbook
diff --git a/docs/en/connector-v2/sink/HdfsFile.md
b/docs/en/connector-v2/sink/HdfsFile.md
index e4cd56bf4e..6514599cb3 100644
--- a/docs/en/connector-v2/sink/HdfsFile.md
+++ b/docs/en/connector-v2/sink/HdfsFile.md
@@ -72,6 +72,7 @@ Output data to hdfs file
| compress_codec | string | no | none
| compress codec
[...]
| common-options | object | no | -
| Sink plugin common parameters, please refer to
[Sink Common Options](../sink-common-options.md) for details
[...]
| max_rows_in_memory | int | no | -
| Only used when file_format is excel.When File
Format is Excel,The maximum number of data items that can be cached in the
memory.
[...]
+| sheet_max_rows | int | no | 1048576
| Only used when file_format is excel.
[...]
| sheet_name | string | no | Sheet${Random
number} | Only used when file_format is excel.Writer the
sheet of the workbook
[...]
| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
[...]
| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml, specifies the
tag name of the root element within the XML file.
[...]
diff --git a/docs/en/connector-v2/sink/LocalFile.md
b/docs/en/connector-v2/sink/LocalFile.md
index df83f2acdb..5ea663514e 100644
--- a/docs/en/connector-v2/sink/LocalFile.md
+++ b/docs/en/connector-v2/sink/LocalFile.md
@@ -48,7 +48,7 @@ By default, we use 2PC commit to ensure `exactly-once`
| filename_time_format | string | no | "yyyy.MM.dd"
| Only used when custom_filename is true
|
| file_format_type | string | no | "csv"
|
|
| filename_extension | string | no | -
| Override the default file name extensions with
custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype`
|
-| field_delimiter | string | no | '\001' for text
and ',' for csv | Only used when file_format_type is text and csv
|
+| field_delimiter | string | no | '\001' for text
and ',' for csv | Only used when file_format_type is text and csv
|
| row_delimiter | string | no | "\n"
| Only used when file_format_type is `text`, `csv`
and `json`
|
| have_partition | boolean | no | false
| Whether you need processing partitions.
|
| partition_by | array | no | -
| Only used then have_partition is true
|
@@ -60,6 +60,7 @@ By default, we use 2PC commit to ensure `exactly-once`
| compress_codec | string | no | none
|
|
| common-options | object | no | -
|
|
| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file_format_type is excel.
|
| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
@@ -185,6 +186,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows [int]
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name [string]
Writer the sheet of the workbook
diff --git a/docs/en/connector-v2/sink/ObsFile.md
b/docs/en/connector-v2/sink/ObsFile.md
index 7e7e816e6d..c3a171f4fb 100644
--- a/docs/en/connector-v2/sink/ObsFile.md
+++ b/docs/en/connector-v2/sink/ObsFile.md
@@ -82,6 +82,7 @@ It only supports hadoop version **2.9.X+**.
| common-options | object | no | -
| [Tips](#common_options)
|
| max_rows_in_memory | int | no | -
| When File Format is Excel,The maximum number of data
items that can be cached in the memory.Only used when file_format is excel.
|
| sheet_name | string | no | Sheet${Random
number} | Writer the sheet of the workbook. Only used when
file_format is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file format_type is excel.
|
### Tips
diff --git a/docs/en/connector-v2/sink/OssFile.md
b/docs/en/connector-v2/sink/OssFile.md
index e5c7bcbfb2..c691e4d502 100644
--- a/docs/en/connector-v2/sink/OssFile.md
+++ b/docs/en/connector-v2/sink/OssFile.md
@@ -110,26 +110,27 @@ If write to `csv`, `text`, `json` file type, All column
will be string.
| have_partition | boolean | no | false
| Whether you need processing partitions.
|
| partition_by | array | no | -
| Only used then have_partition is true
|
| partition_dir_expression | string | no |
"${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is
true
|
-| is_partition_field_write_in_file | boolean | no | false
| Only used then have_partition is true
|
-| sink_columns | array | no |
| When this parameter is empty, all fields are sink
columns
|
-| is_enable_transaction | boolean | no | true
|
|
-| batch_size | int | no | 1000000
|
|
-| compress_codec | string | no | none
|
|
-| common-options | object | no | -
|
|
-| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
-| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
-| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
-| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
-| xml_row_tag | string | no | RECORD
| Only used when file_format is xml.
|
-| xml_use_attr_format | boolean | no | -
| Only used when file_format is xml.
|
-| single_file_mode | boolean | no | false
| Each parallelism will only output one file. When
this parameter is turned on, batch_size will not take effect. The output file
name does not have a file block suffix. |
-| create_empty_file_when_no_data | boolean | no | false
| When there is no data synchronization upstream,
the corresponding data files are still generated.
|
-| parquet_avro_write_timestamp_as_int96 | boolean | no | false
| Only used when file_format is parquet.
|
-| parquet_avro_write_fixed_as_int96 | array | no | -
| Only used when file_format is parquet.
|
-| enable_header_write | boolean | no | false
| Only used when file_format_type is text,csv.<br/>
false:don't write header,true:write header.
|
-| encoding | string | no | "UTF-8"
| Only used when file_format_type is
json,text,csv,xml.
|
-| schema_save_mode | Enum | no |
CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous
task, do different treatment of the target path
|
-| data_save_mode | Enum | no | APPEND_DATA
| Before opening the synchronous task, the data file
in the target path is differently processed
|
+| is_partition_field_write_in_file | boolean | no | false
| Only used then have_partition is true
|
+| sink_columns | array | no |
| When this parameter is empty, all fields are sink
columns
|
+| is_enable_transaction | boolean | no | true
|
|
+| batch_size | int | no | 1000000
|
|
+| compress_codec | string | no | none
|
|
+| common-options | object | no | -
|
|
+| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file format_type is excel.
|
+| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
+| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
+| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
+| xml_row_tag | string | no | RECORD
| Only used when file_format is xml.
|
+| xml_use_attr_format | boolean | no | -
| Only used when file_format is xml.
|
+| single_file_mode | boolean | no | false
| Each parallelism will only output one file. When
this parameter is turned on, batch_size will not take effect. The output file
name does not have a file block suffix. |
+| create_empty_file_when_no_data | boolean | no | false
| When there is no data synchronization upstream, the
corresponding data files are still generated.
|
+| parquet_avro_write_timestamp_as_int96 | boolean | no | false
| Only used when file_format is parquet.
|
+| parquet_avro_write_fixed_as_int96 | array | no | -
| Only used when file_format is parquet.
|
+| enable_header_write | boolean | no | false
| Only used when file_format_type is text,csv.<br/>
false:don't write header,true:write header.
|
+| encoding | string | no | "UTF-8"
| Only used when file_format_type is
json,text,csv,xml.
|
+| schema_save_mode | Enum | no |
CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous
task, do different treatment of the target path
|
+| data_save_mode | Enum | no | APPEND_DATA
| Before opening the synchronous task, the data file
in the target path is differently processed
|
### path [string]
@@ -258,6 +259,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows [int]
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name [string]
Writer the sheet of the workbook
diff --git a/docs/en/connector-v2/sink/OssJindoFile.md
b/docs/en/connector-v2/sink/OssJindoFile.md
index 941dd02421..07de7f0f87 100644
--- a/docs/en/connector-v2/sink/OssJindoFile.md
+++ b/docs/en/connector-v2/sink/OssJindoFile.md
@@ -44,41 +44,42 @@ It only supports hadoop version **2.9.X+**.
## Options
-| Name | Type | Required | Default
| Description
|
-|---------------------------------------|---------|----------|--------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| path | string | yes | -
|
|
-| tmp_path | string | no | /tmp/seatunnel
| The result file will write to a tmp path first and
then use `mv` to submit tmp dir to target dir. Need a OSS dir.
|
-| bucket | string | yes | -
|
|
-| access_key | string | yes | -
|
|
-| access_secret | string | yes | -
|
|
-| endpoint | string | yes | -
|
|
-| custom_filename | boolean | no | false
| Whether you need custom the filename
|
-| file_name_expression | string | no |
"${transactionId}" | Only used when custom_filename is
true
|
-| filename_time_format | string | no | "yyyy.MM.dd"
| Only used when custom_filename is true
|
-| file_format_type | string | no | "csv"
|
|
-| filename_extension | string | no | -
| Override the default file name extensions with
custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype`
|
-| field_delimiter | string | no | '\001' for text
and ',' for csv | Only used when file_format_type is text and csv
|
-| row_delimiter | string | no | "\n"
| Only used when file_format_type is `text`, `csv`
and `json`
|
-| have_partition | boolean | no | false
| Whether you need processing partitions.
|
-| partition_by | array | no | -
| Only used then have_partition is true
|
-| partition_dir_expression | string | no |
"${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is
true
|
-| is_partition_field_write_in_file | boolean | no | false
| Only used then have_partition is true
|
-| sink_columns | array | no |
| When this parameter is empty, all fields are sink
columns
|
-| is_enable_transaction | boolean | no | true
|
|
-| batch_size | int | no | 1000000
|
|
-| compress_codec | string | no | none
|
|
-| common-options | object | no | -
|
|
-| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
-| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
-| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
-| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
-| xml_row_tag | string | no | RECORD
| Only used when file_format is xml.
|
-| xml_use_attr_format | boolean | no | -
| Only used when file_format is xml.
|
+| Name | Type | Required | Default
| Description
|
+|---------------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| path | string | yes | -
|
|
+| tmp_path | string | no | /tmp/seatunnel
| The result file will write to a tmp path first and
then use `mv` to submit tmp dir to target dir. Need a OSS dir.
|
+| bucket | string | yes | -
|
|
+| access_key | string | yes | -
|
|
+| access_secret | string | yes | -
|
|
+| endpoint | string | yes | -
|
|
+| custom_filename | boolean | no | false
| Whether you need custom the filename
|
+| file_name_expression | string | no |
"${transactionId}" | Only used when custom_filename is
true
|
+| filename_time_format | string | no | "yyyy.MM.dd"
| Only used when custom_filename is true
|
+| file_format_type | string | no | "csv"
|
|
+| filename_extension | string | no | -
| Override the default file name extensions with
custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype`
|
+| field_delimiter | string | no | '\001' for text
and ',' for csv | Only used when file_format_type is text and csv
|
+| row_delimiter | string | no | "\n"
| Only used when file_format_type is `text`, `csv`
and `json`
|
+| have_partition | boolean | no | false
| Whether you need processing partitions.
|
+| partition_by | array | no | -
| Only used then have_partition is true
|
+| partition_dir_expression | string | no |
"${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is
true
|
+| is_partition_field_write_in_file | boolean | no | false
| Only used then have_partition is true
|
+| sink_columns | array | no |
| When this parameter is empty, all fields are sink
columns
|
+| is_enable_transaction | boolean | no | true
|
|
+| batch_size | int | no | 1000000
|
|
+| compress_codec | string | no | none
|
|
+| common-options | object | no | -
|
|
+| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file_format_type is excel.
|
+| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
+| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
+| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
+| xml_row_tag | string | no | RECORD
| Only used when file_format is xml.
|
+| xml_use_attr_format | boolean | no | -
| Only used when file_format is xml.
|
| single_file_mode | boolean | no | false
| Each parallelism will only output one file. When
this parameter is turned on, batch_size will not take effect. The output file
name does not have a file block suffix. |
-| create_empty_file_when_no_data | boolean | no | false
| When there is no data synchronization upstream,
the corresponding data files are still generated.
|
-| parquet_avro_write_timestamp_as_int96 | boolean | no | false
| Only used when file_format is parquet.
|
-| parquet_avro_write_fixed_as_int96 | array | no | -
| Only used when file_format is parquet.
|
-| encoding | string | no | "UTF-8"
| Only used when file_format_type is
json,text,csv,xml.
|
+| create_empty_file_when_no_data | boolean | no | false
| When there is no data synchronization upstream,
the corresponding data files are still generated.
|
+| parquet_avro_write_timestamp_as_int96 | boolean | no | false
| Only used when file_format is parquet.
|
+| parquet_avro_write_fixed_as_int96 | array | no | -
| Only used when file_format is parquet.
|
+| encoding | string | no | "UTF-8"
| Only used when file_format_type is
json,text,csv,xml.
|
### path [string]
@@ -207,6 +208,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows [int]
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name [string]
Writer the sheet of the workbook
diff --git a/docs/en/connector-v2/sink/S3File.md
b/docs/en/connector-v2/sink/S3File.md
index f6dc178f84..7b53ef5856 100644
--- a/docs/en/connector-v2/sink/S3File.md
+++ b/docs/en/connector-v2/sink/S3File.md
@@ -126,6 +126,7 @@ If write to `csv`, `text` file type, All column will be
string.
| compress_codec | string | no | none
|
|
| common-options | object | no | -
|
|
| max_rows_in_memory | int | no | -
| Only used when file_format is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file_format is excel.
|
| sheet_name | string | no | Sheet${Random
number} | Only used when file_format is excel.
|
| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml,
specifies the tag name of the root element within the XML file.
|
@@ -263,6 +264,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows [int]
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name [string]
Writer the sheet of the workbook
diff --git a/docs/en/connector-v2/sink/SftpFile.md
b/docs/en/connector-v2/sink/SftpFile.md
index 0ac163f144..c939855a54 100644
--- a/docs/en/connector-v2/sink/SftpFile.md
+++ b/docs/en/connector-v2/sink/SftpFile.md
@@ -63,6 +63,7 @@ If you use SeaTunnel Engine, It automatically integrated the
hadoop jar when you
| compress_codec | string | no | none
|
|
| common-options | object | no | -
|
|
| max_rows_in_memory | int | no | -
| Only used when file_format_type is excel.
|
+| sheet_max_rows | int | no | 1048576
| Only used when file_format_type is excel.
|
| sheet_name | string | no | Sheet${Random
number} | Only used when file_format_type is excel.
|
| csv_string_quote_mode | enum | no | MINIMAL
| Only used when file_format is csv.
|
| xml_root_tag | string | no | RECORDS
| Only used when file_format is xml.
|
@@ -204,6 +205,10 @@ Sink plugin common parameters, please refer to [Sink
Common Options](../sink-com
When File Format is Excel,The maximum number of data items that can be cached
in the memory.
+### sheet_max_rows
+
+When file format is Excel, the maximum number of rows per sheet.
+
### sheet_name
Writer the sheet of the workbook
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSinkConfig.java
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSinkConfig.java
index 12c3ee01c9..9be6b93f81 100644
---
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSinkConfig.java
+++
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSinkConfig.java
@@ -39,6 +39,7 @@ public class BaseFileSinkConfig implements DelimiterConfig,
Serializable {
private static final long serialVersionUID = 1L;
protected CompressFormat compressFormat =
FileBaseSinkOptions.COMPRESS_CODEC.defaultValue();
protected String fieldDelimiter;
+ protected int sheetMaxRows =
FileBaseSinkOptions.SHEET_MAX_ROWS.defaultValue();
protected String rowDelimiter =
FileBaseSinkOptions.ROW_DELIMITER.defaultValue();
protected int batchSize = FileBaseSinkOptions.BATCH_SIZE.defaultValue();
protected String path;
@@ -62,6 +63,12 @@ public class BaseFileSinkConfig implements DelimiterConfig,
Serializable {
this.batchSize =
config.getInt(FileBaseSinkOptions.BATCH_SIZE.key());
}
+ if (config.hasPath(FileBaseSinkOptions.SHEET_MAX_ROWS.key())
+ && StringUtils.isNotEmpty(
+
config.getString(FileBaseSinkOptions.SHEET_MAX_ROWS.key()))) {
+ this.sheetMaxRows =
config.getInt(FileBaseSinkOptions.SHEET_MAX_ROWS.key());
+ }
+
if (config.hasPath(FileBaseSinkOptions.ROW_DELIMITER.key())) {
this.rowDelimiter =
config.getString(FileBaseSinkOptions.ROW_DELIMITER.key());
}
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java
index 1f385e54cf..ce8d7d531c 100644
---
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java
+++
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java
@@ -108,6 +108,12 @@ public class FileBaseSinkOptions extends FileBaseOptions {
.withDescription(
"The separator between columns in a row of data.
Only needed by `text` and `csv` file format");
+ public static final Option<Integer> SHEET_MAX_ROWS =
+ Options.key("sheet_max_rows")
+ .intType()
+ .defaultValue(1048576)
+ .withDescription("Only needed by `excel` file format");
+
public static final Option<String> ROW_DELIMITER =
Options.key("row_delimiter")
.stringType()
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/util/ExcelGenerator.java
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/util/ExcelGenerator.java
index 3ecb612f44..d00b3a0254 100644
---
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/util/ExcelGenerator.java
+++
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/util/ExcelGenerator.java
@@ -47,7 +47,6 @@ import java.time.LocalTime;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
-import java.util.Random;
public class ExcelGenerator {
private final List<Integer> sinkColumnsIndexInRow;
@@ -62,8 +61,25 @@ public class ExcelGenerator {
private final CellStyle dateCellStyle;
private final CellStyle dateTimeCellStyle;
private final CellStyle timeCellStyle;
- private final Sheet st;
- private int row = 0;
+ private Sheet st;
+
+ private final int sheetMaxRows;
+ private static final int HEADER_ROWS = 1;
+
+ private int currentSheetIndex = 0;
+ private int currentRowInSheet = 0;
+
+ private void createNewSheet() {
+ currentSheetIndex++;
+ String newSheetName = String.format("Sheet%d", currentSheetIndex);
+ this.st = wb.createSheet(newSheetName);
+ Row headerRow = st.createRow(0);
+ for (Integer i : sinkColumnsIndexInRow) {
+ String fieldName = seaTunnelRowType.getFieldName(i);
+ headerRow.createCell(i).setCellValue(fieldName);
+ }
+ currentRowInSheet = 0;
+ }
public ExcelGenerator(
List<Integer> sinkColumnsIndexInRow,
@@ -77,11 +93,8 @@ public class ExcelGenerator {
wb = new SXSSFWorkbook();
}
Optional<String> sheetName =
Optional.ofNullable(fileSinkConfig.getSheetName());
- Random random = new Random();
- this.st =
- wb.createSheet(
- sheetName.orElseGet(() -> String.format("Sheet%d",
random.nextInt())));
- Row row = st.createRow(this.row);
+ this.st = wb.createSheet(sheetName.orElseGet(() ->
String.format("Sheet%d", 0)));
+ Row row = st.createRow(0);
for (Integer i : sinkColumnsIndexInRow) {
String fieldName = seaTunnelRowType.getFieldName(i);
row.createCell(i).setCellValue(fieldName);
@@ -90,24 +103,26 @@ public class ExcelGenerator {
this.dateTimeFormat = fileSinkConfig.getDatetimeFormat();
this.timeFormat = fileSinkConfig.getTimeFormat();
this.fieldDelimiter = fileSinkConfig.getFieldDelimiter();
+ this.sheetMaxRows = fileSinkConfig.getSheetMaxRows();
wholeNumberCellStyle = createStyle(wb, "General");
stringCellStyle = createStyle(wb, "@");
dateCellStyle = createStyle(wb, dateFormat.getValue());
dateTimeCellStyle = createStyle(wb, dateTimeFormat.getValue());
timeCellStyle = createStyle(wb, timeFormat.getValue());
-
- this.row += 1;
}
public void writeData(SeaTunnelRow seaTunnelRow) {
- Row excelRow = this.st.createRow(this.row);
+ if (currentRowInSheet >= sheetMaxRows - HEADER_ROWS) {
+ createNewSheet();
+ }
+ Row excelRow = this.st.createRow(currentRowInSheet + HEADER_ROWS);
SeaTunnelDataType<?>[] fieldTypes = seaTunnelRowType.getFieldTypes();
for (Integer i : sinkColumnsIndexInRow) {
Cell cell = excelRow.createCell(i);
Object value = seaTunnelRow.getField(i);
setCellValue(fieldTypes[i], seaTunnelRowType.getFieldName(i),
value, cell);
}
- this.row += 1;
+ currentRowInSheet++;
}
public void flushAndCloseExcel(OutputStream output) throws IOException {
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/ExcelGeneratorTest.java
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/ExcelGeneratorTest.java
new file mode 100644
index 0000000000..4c69376087
--- /dev/null
+++
b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/ExcelGeneratorTest.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.seatunnel.connectors.seatunnel.file.writer;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.utils.DateTimeUtils;
+import org.apache.seatunnel.common.utils.DateUtils;
+import org.apache.seatunnel.common.utils.TimeUtils;
+import
org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.util.ExcelGenerator;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import com.alibaba.excel.EasyExcel;
+import com.alibaba.excel.context.AnalysisContext;
+import com.alibaba.excel.event.AnalysisEventListener;
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@Slf4j
+public class ExcelGeneratorTest {
+
+ private FileSinkConfig fileSinkConfig;
+ private SeaTunnelRowType rowType;
+ private List<Integer> sinkColumnsIndexInRow;
+
+ @BeforeEach
+ public void setUp() {
+ fileSinkConfig = mock(FileSinkConfig.class);
+ when(fileSinkConfig.getMaxRowsInMemory()).thenReturn(100);
+ when(fileSinkConfig.getSheetName()).thenReturn("TestSheet");
+
when(fileSinkConfig.getDateFormat()).thenReturn(DateUtils.Formatter.YYYY_MM_DD);
+ when(fileSinkConfig.getDatetimeFormat())
+ .thenReturn(DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS);
+
when(fileSinkConfig.getTimeFormat()).thenReturn(TimeUtils.Formatter.HH_MM_SS);
+ when(fileSinkConfig.getSheetMaxRows()).thenReturn(1048576);
+ rowType =
+ new SeaTunnelRowType(
+ new String[] {"id", "name", "age", "email"},
+ new
org.apache.seatunnel.api.table.type.SeaTunnelDataType[] {
+ BasicType.INT_TYPE,
+ BasicType.STRING_TYPE,
+ BasicType.INT_TYPE,
+ BasicType.STRING_TYPE
+ });
+
+ sinkColumnsIndexInRow = Arrays.asList(0, 1, 2, 3);
+ }
+
+ @Test
+ public void testGenerateBasicExcelFile() throws IOException {
+ File outputDir = new File("target/test-output");
+ if (!outputDir.exists()) {
+ outputDir.mkdirs();
+ }
+
+ File outputFile = new File(outputDir, "basic-test.xlsx");
+
+ ExcelGenerator excelGenerator =
+ new ExcelGenerator(sinkColumnsIndexInRow, rowType,
fileSinkConfig);
+
+ SeaTunnelRow[] testData = {
+ new SeaTunnelRow(new Object[] {1, "Alice", 25, "[email protected]"}),
+ new SeaTunnelRow(new Object[] {2, "Bob", 30, "[email protected]"}),
+ new SeaTunnelRow(new Object[] {3, "Charlie", 35,
"[email protected]"}),
+ new SeaTunnelRow(new Object[] {4, "Diana", 28, "[email protected]"}),
+ new SeaTunnelRow(new Object[] {5, null, 22, null})
+ };
+
+ for (SeaTunnelRow row : testData) {
+ excelGenerator.writeData(row);
+ }
+
+ try (FileOutputStream fos = new FileOutputStream(outputFile)) {
+ excelGenerator.flushAndCloseExcel(fos);
+ }
+
+ assertTrue("File should exist", outputFile.exists());
+ assertTrue("File should not be empty", outputFile.length() > 0);
+
+ validateGeneratedFile(outputFile, 5, 0);
+ }
+
+ @Test
+ public void testGenerateLargeDataFile() throws IOException {
+ File outputDir = new File("target/test-output");
+ if (!outputDir.exists()) {
+ outputDir.mkdirs();
+ }
+
+ File outputFile = new File(outputDir, "large-test.xlsx");
+
+ ExcelGenerator excelGenerator =
+ new ExcelGenerator(sinkColumnsIndexInRow, rowType,
fileSinkConfig);
+
+ int totalRows = 1200000;
+
+ for (int i = 1; i <= totalRows; i++) {
+ SeaTunnelRow row =
+ new SeaTunnelRow(
+ new Object[] {
+ i, "User" + i, 20 + (i % 50), "user" + i +
"@example.com"
+ });
+ excelGenerator.writeData(row);
+ }
+
+ try (FileOutputStream fos = new FileOutputStream(outputFile)) {
+ excelGenerator.flushAndCloseExcel(fos);
+ }
+
+ assertTrue("Large file should exist", outputFile.exists());
+ validateGeneratedFile(outputFile, 1048575, 0);
+ validateGeneratedFile(outputFile, totalRows - 1048575, 1);
+ }
+
+ private void validateGeneratedFile(File file, int expectedDataRows, int
sheetNo)
+ throws IOException {
+ AtomicInteger rowCount = new AtomicInteger(0);
+ AtomicBoolean headerValid = new AtomicBoolean(false);
+ EasyExcel.read(file)
+ .registerReadListener(
+ new AnalysisEventListener<Map<Integer, String>>() {
+ @Override
+ public void invoke(Map<Integer, String> data,
AnalysisContext context) {
+ rowCount.incrementAndGet();
+ if (rowCount.get() % 50000 == 0) {
+ log.info("Processed " + rowCount.get() + "
rows");
+ }
+ }
+
+ @Override
+ public void invokeHeadMap(
+ Map<Integer, String> headMap,
AnalysisContext context) {
+ headerValid.set(
+ "id".equals(headMap.get(0))
+ &&
"name".equals(headMap.get(1))
+ && "age".equals(headMap.get(2))
+ &&
"email".equals(headMap.get(3)));
+ }
+
+ @Override
+ public void doAfterAllAnalysed(AnalysisContext
context) {
+ log.info("Validation completed. Total rows: "
+ rowCount.get());
+ }
+ })
+ .sheet(sheetNo)
+ .doRead();
+
+ assertTrue("Headers should be valid", headerValid.get());
+ assertEquals("Should have correct number of rows", expectedDataRows,
rowCount.get());
+ }
+}
