Skip to main content
Version: 2.3.10

HdfsFile

HDFS File Sink Connector

Support Those Engines

Spark
Flink
SeaTunnel Zeta

Key Features

By default, we use 2PC commit to ensure exactly-once

  • file format type
    • text
    • csv
    • parquet
    • orc
    • json
    • excel
    • xml
    • binary
  • compress codec
    • lzo

Description

Output data to hdfs file

Supported DataSource Info

DatasourceSupported Versions
HdfsFilehadoop 2.x and 3.x

Sink Options

NameTypeRequiredDefaultDescription
fs.defaultFSstringyes-The hadoop cluster address that start with hdfs://, for example: hdfs://hadoopcluster
pathstringyes-The target dir path is required.
tmp_pathstringyes/tmp/seatunnelThe result file will write to a tmp path first and then use mv to submit tmp dir to target dir. Need a hdfs path.
hdfs_site_pathstringno-The path of hdfs-site.xml, used to load ha configuration of namenodes
custom_filenamebooleannofalseWhether you need custom the filename
file_name_expressionstringno"${transactionId}"Only used when custom_filename is true.file_name_expression describes the file expression which will be created into the path. We can add the variable ${now} or ${uuid} in the file_name_expression, like test_${uuid}_${now},${now} represents the current time, and its format can be defined by specifying the option filename_time_format.Please note that, If is_enable_transaction is true, we will auto add ${transactionId}_ in the head of the file.
filename_time_formatstringno"yyyy.MM.dd"Only used when custom_filename is true.When the format in the file_name_expression parameter is xxxx-${now} , filename_time_format can specify the time format of the path, and the default value is yyyy.MM.dd . The commonly used time formats are listed as follows:[y:Year,M:Month,d:Day of month,H:Hour in day (0-23),m:Minute in hour,s:Second in minute]
file_format_typestringno"csv"We supported as the following file types:text csv parquet orc json excel xml binary.Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is txt.
filename_extensionstringno-Override the default file name extensions with custom file name extensions. E.g. .xml, .json, dat, .customtype
field_delimiterstringno'\001'Only used when file_format is text,The separator between columns in a row of data. Only needed by text file format.
row_delimiterstringno"\n"Only used when file_format is text,The separator between rows in a file. Only needed by text file format.
have_partitionbooleannofalseWhether you need processing partitions.
partition_byarrayno-Only used then have_partition is true,Partition data based on selected fields.
partition_dir_expressionstringno"${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/"Only used then have_partition is true,If the partition_by is specified, we will generate the corresponding partition directory based on the partition information, and the final file will be placed in the partition directory. Default partition_dir_expression is ${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/. k0 is the first partition field and v0 is the value of the first partition field.
is_partition_field_write_in_filebooleannofalseOnly used when have_partition is true. If is_partition_field_write_in_file is true, the partition field and the value of it will be write into data file.For example, if you want to write a Hive Data File, Its value should be false.
sink_columnsarraynoWhen this parameter is empty, all fields are sink columns.Which columns need be write to file, default value is all of the columns get from Transform or Source. The order of the fields determines the order in which the file is actually written.
is_enable_transactionbooleannotrueIf is_enable_transaction is true, we will ensure that data will not be lost or duplicated when it is written to the target directory.Please note that, If is_enable_transaction is true, we will auto add ${transactionId}_ in the head of the file.Only support true now.
batch_sizeintno1000000The maximum number of rows in a file. For SeaTunnel Engine, the number of lines in the file is determined by batch_size and checkpoint.interval jointly decide. If the value of checkpoint.interval is large enough, sink writer will write rows in a file until the rows in the file larger than batch_size. If checkpoint.interval is small, the sink writer will create a new file when a new checkpoint trigger.
compress_codecstringnononeThe compress codec of files and the details that supported as the following shown:[txt: lzo none,json: lzo none,csv: lzo none,orc: lzo snappy lz4 zlib none,parquet: lzo snappy lz4 gzip brotli zstd none].Tips: excel type does not support any compression format.
krb5_pathstringno/etc/krb5.confThe krb5 path of kerberos
kerberos_principalstringno-The principal of kerberos
kerberos_keytab_pathstringno-The keytab path of kerberos
compress_codecstringnononecompress codec
common-optionsobjectno-Sink plugin common parameters, please refer to Sink Common Options for details
max_rows_in_memoryintno-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_namestringnoSheet${Random number}Only used when file_format is excel.Writer the sheet of the workbook
csv_string_quote_modeenumnoMINIMALOnly used when file_format is csv.
xml_root_tagstringnoRECORDSOnly used when file_format is xml, specifies the tag name of the root element within the XML file.
xml_row_tagstringnoRECORDOnly used when file_format is xml, specifies the tag name of the data rows within the XML file
xml_use_attr_formatbooleanno-Only used when file_format is xml, specifies Whether to process data using the tag attribute format.
single_file_modebooleannofalseEach 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_databooleannofalseWhen there is no data synchronization upstream, the corresponding data files are still generated.
parquet_avro_write_timestamp_as_int96booleannofalseOnly used when file_format is parquet.
parquet_avro_write_fixed_as_int96arrayno-Only used when file_format is parquet.
enable_header_writebooleannofalseOnly used when file_format_type is text,csv.
false:don't write header,true:write header.
encodingstringno"UTF-8"Only used when file_format_type is json,text,csv,xml.
remote_userstringno-The remote user name of hdfs.

Tips

If you use spark/flink, In order to use this connector, You must ensure your spark/flink cluster already integrated hadoop. The tested hadoop version is 2.x. If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you download and install SeaTunnel Engine. You can check the jar package under ${SEATUNNEL_HOME}/lib to confirm this.

Task Example

Simple:

This example defines a SeaTunnel synchronization task that automatically generates data through FakeSource and sends it to Hdfs.

# Defining the runtime environment
env {
parallelism = 1
job.mode = "BATCH"
}

source {
# This is a example source plugin **only for test and demonstrate the feature source plugin**
FakeSource {
parallelism = 1
plugin_output = "fake"
row.num = 16
schema = {
fields {
c_map = "map<string, smallint>"
c_array = "array<int>"
c_string = string
c_boolean = boolean
c_tinyint = tinyint
c_smallint = smallint
c_int = int
c_bigint = bigint
c_float = float
c_double = double
c_decimal = "decimal(30, 8)"
c_bytes = bytes
c_date = date
c_timestamp = timestamp
}
}
}
# If you would like to get more information about how to configure seatunnel and see full list of source plugins,
# please go to https://seatunnel.apache.org/docs/connector-v2/source
}

transform {
# If you would like to get more information about how to configure seatunnel and see full list of transform plugins,
# please go to https://seatunnel.apache.org/docs/transform-v2
}

sink {
HdfsFile {
fs.defaultFS = "hdfs://hadoopcluster"
path = "/tmp/hive/warehouse/test2"
file_format_type = "orc"
}
# If you would like to get more information about how to configure seatunnel and see full list of sink plugins,
# please go to https://seatunnel.apache.org/docs/connector-v2/sink
}

For orc file format simple config

HdfsFile {
fs.defaultFS = "hdfs://hadoopcluster"
path = "/tmp/hive/warehouse/test2"
file_format_type = "orc"
}

For text file format with have_partition and custom_filename and sink_columns

HdfsFile {
fs.defaultFS = "hdfs://hadoopcluster"
path = "/tmp/hive/warehouse/test2"
file_format_type = "text"
field_delimiter = "\t"
row_delimiter = "\n"
have_partition = true
partition_by = ["age"]
partition_dir_expression = "${k0}=${v0}"
is_partition_field_write_in_file = true
custom_filename = true
file_name_expression = "${transactionId}_${now}"
filename_time_format = "yyyy.MM.dd"
sink_columns = ["name","age"]
is_enable_transaction = true
}

For parquet file format with have_partition and custom_filename and sink_columns

HdfsFile {
fs.defaultFS = "hdfs://hadoopcluster"
path = "/tmp/hive/warehouse/test2"
have_partition = true
partition_by = ["age"]
partition_dir_expression = "${k0}=${v0}"
is_partition_field_write_in_file = true
custom_filename = true
file_name_expression = "${transactionId}_${now}"
filename_time_format = "yyyy.MM.dd"
file_format_type = "parquet"
sink_columns = ["name","age"]
is_enable_transaction = true
}

For kerberos simple config

HdfsFile {
fs.defaultFS = "hdfs://hadoopcluster"
path = "/tmp/hive/warehouse/test2"
hdfs_site_path = "/path/to/your/hdfs_site_path"
kerberos_principal = "your_principal@EXAMPLE.COM"
kerberos_keytab_path = "/path/to/your/keytab/file.keytab"
}

enable_header_write [boolean]

Only used when file_format_type is text,csv.false:don't write header,true:write header.

csv_string_quote_mode [string]

When File Format is CSV,The string quote mode of CSV.

  • ALL: All String fields will be quoted.
  • MINIMAL: Quotes fields which contain special characters such as a the field delimiter, quote character or any of the characters in the line separator string.
  • NONE: Never quotes fields. When the delimiter occurs in data, the printer prefixes it with the escape character. If the escape character is not set, format validation throws an exception.

For compress simple config

HdfsFile {
fs.defaultFS = "hdfs://hadoopcluster"
path = "/tmp/hive/warehouse/test2"
compress_codec = "lzo"
}

Changelog

Change Log
ChangeCommitVersion
Revert " [improve] update localfile connector config" (#9018)https://github.com/apache/seatunnel/commit/cdc79e13a2.3.10
[improve] update localfile connector config (#8765)https://github.com/apache/seatunnel/commit/def369a852.3.10
[Feature][Connector-V2] Add filename_extension parameter for read/write file (#8769)https://github.com/apache/seatunnel/commit/78b23c0ef2.3.10
[Improve] restruct connector common options (#8634)https://github.com/apache/seatunnel/commit/f3499a6ee2.3.10
[Feature][Connector-V2] Support create emtpy file when no data (#8543)https://github.com/apache/seatunnel/commit/275db78912.3.10
[Feature][Connector-V2] Support single file mode in file sink (#8518)https://github.com/apache/seatunnel/commit/e893deed52.3.10
[Feature][File] Support config null format for text file read (#8109)https://github.com/apache/seatunnel/commit/2dbf02df42.3.9
[Feature][Restapi] Allow metrics information to be associated to logical plan nodes (#7786)https://github.com/apache/seatunnel/commit/6b7c53d032.3.9
[Improve][Connector-V2] Support read archive compress file (#7633)https://github.com/apache/seatunnel/commit/3f98cd8a12.3.8
[Improve][Files] Support write fixed/timestamp as int96 of parquet (#6971)https://github.com/apache/seatunnel/commit/1a48a9c492.3.6
Add support for XML file type to various file connectors such as SFTP, FTP, LocalFile, HdfsFile, and more. (#6327)https://github.com/apache/seatunnel/commit/ec533ecd92.3.5
[Refactor][File Connector] Put Multiple Table File API to File Base Module (#6033)https://github.com/apache/seatunnel/commit/c324d663b2.3.4
[Improve][connector-file] unifiy option between file source/sink and update document (#5680)https://github.com/apache/seatunnel/commit/8d87cf8fc2.3.4
[Feature] Support LZO compress on File Read (#5083)https://github.com/apache/seatunnel/commit/a4a1901092.3.4
Support config column/primaryKey/constraintKey in schema (#5564)https://github.com/apache/seatunnel/commit/eac76b4e52.3.4
[Feature][File Connector]optionrule FILE_FORMAT_TYPE is text/csv ,add parameter BaseSinkConfig.ENABLE_HEADER_WRITE: #5566 (#5567)https://github.com/apache/seatunnel/commit/0e02db7682.3.4
[Feature][Connector V2][File] Add config of 'file_filter_pattern', which used for filtering files. (#5153)https://github.com/apache/seatunnel/commit/a3c13e59e2.3.3
[Feature][ConnectorV2]add file excel sink and source (#4164)https://github.com/apache/seatunnel/commit/e3b97ae5d2.3.2
Change file type to file_format_type in file source/sink (#4249)https://github.com/apache/seatunnel/commit/973a2fae32.3.1
Merge branch 'dev' into merge/cdchttps://github.com/apache/seatunnel/commit/4324ee1912.3.1
[Improve][Project] Code format with spotless plugin.https://github.com/apache/seatunnel/commit/423b583032.3.1
[improve][api] Refactoring schema parse (#4157)https://github.com/apache/seatunnel/commit/b2f573a132.3.1
[Improve][build] Give the maven module a human readable name (#4114)https://github.com/apache/seatunnel/commit/d7cd601052.3.1
[Improve][Project] Code format with spotless plugin. (#4101)https://github.com/apache/seatunnel/commit/a2ab166562.3.1
[Feature][Connector-V2][File] Support compress (#3899)https://github.com/apache/seatunnel/commit/55602f6b12.3.1
[Feature][Connector] add get source method to all source connector (#3846)https://github.com/apache/seatunnel/commit/417178fb82.3.1
[Improve][Connector-V2][File] Improve file connector option rule and document (#3812)https://github.com/apache/seatunnel/commit/bd76077662.3.1
[Feature][Shade] Add seatunnel hadoop3 uber (#3755)https://github.com/apache/seatunnel/commit/5a024bdf82.3.0
[Hotfix][OptionRule] Fix option rule about all connectors (#3592)https://github.com/apache/seatunnel/commit/226dc6a112.3.0
[Feature][Connector-V2][File] Add option and factory for file connectors (#3375)https://github.com/apache/seatunnel/commit/db286e8632.3.0
[Improve][Connector] Improve write parquet (#2943)https://github.com/apache/seatunnel/commit/8fd9663942.3.0-beta
[Fix][Connector-V2] Fix HiveSource Connector read orc table error (#2845)https://github.com/apache/seatunnel/commit/61720306e2.2.0-beta
[Improve][Connector-V2] Improve read parquet (#2841)https://github.com/apache/seatunnel/commit/e19bc82f92.2.0-beta
[Improve][Connector-V2] Refactor hdfs file sink connector code structure (#2701)https://github.com/apache/seatunnel/commit/6129c02562.2.0-beta
[#2606]Dependency management split (#2630)https://github.com/apache/seatunnel/commit/fc047be692.2.0-beta
[chore][connector-common] Rename SeatunnelSchema to SeaTunnelSchema (#2538)https://github.com/apache/seatunnel/commit/7dc2a27382.2.0-beta
[Feature][Connector-V2] Add hdfs file json support (#2451)https://github.com/apache/seatunnel/commit/84f6b17c12.2.0-beta
[Improve][Connector-V2] Refactor the package of hdfs file connector (#2402)https://github.com/apache/seatunnel/commit/87d0624c52.2.0-beta
[Feature][Connector-V2] Add hdfs file source connector (#2420)https://github.com/apache/seatunnel/commit/4fb6f2a212.2.0-beta
[Feature][Connector-V2] Add json file sink & json format (#2385)https://github.com/apache/seatunnel/commit/dd68c06b02.2.0-beta
[Imporve][Connector-V2] Remove redundant type judge logic because of pr #2315 (#2370)https://github.com/apache/seatunnel/commit/42e8c25e52.2.0-beta
[Feature][Connector-V2] Support orc file format in file connector (#2369)https://github.com/apache/seatunnel/commit/f44fe1e032.2.0-beta
[improve][UT] Upgrade junit to 5.+ (#2305)https://github.com/apache/seatunnel/commit/362319ff32.2.0-beta
[Connector-V2] Add parquet writer in file connector (#2273)https://github.com/apache/seatunnel/commit/c95cc72cf2.2.0-beta
[checkstyle] Improved validation scope of MagicNumber (#2194)https://github.com/apache/seatunnel/commit/6d08b5f362.2.0-beta
[Connector-V2] Add Hive sink connector v2 (#2158)https://github.com/apache/seatunnel/commit/23ad4ee732.2.0-beta
[Connector-V2] Add File Sink Connector (#2117)https://github.com/apache/seatunnel/commit/e2283da642.2.0-beta