Flink Doris Connector

Flink Doris Connector can support read and write data stored in Doris through Flink.

  • You can map the Doris table toDataStream or Table.

Version Compatibility

ConnectorFlinkDorisJavaScala
1.0.01.11.20.13+82.12
1.0.01.13.x0.13.+82.12

For Flink 1.13.x version adaptation issues

  1. <properties>
  2. <scala.version>2.12</scala.version>
  3. <flink.version>1.11.2</flink.version>
  4. <libthrift.version>0.9.3</libthrift.version>
  5. <arrow.version>0.15.1</arrow.version>
  6. <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
  7. <doris.home>${basedir}/../../</doris.home>
  8. <doris.thirdparty>${basedir}/../../thirdparty</doris.thirdparty>
  9. </properties>

Just change the flink.version here to be the same as your Flink cluster version, and edit again

Build and Install

Execute following command in dir extension/flink-doris-connector/:

Notice:

  1. If you have not compiled the doris source code as a whole, you need to compile the Doris source code first, otherwise the thrift command will not be found, and you need to execute sh build.sh in the incubator-doris directory.
  2. It is recommended to compile under the docker compile environment apache/incubator-doris:build-env-1.2 of doris, because the JDK version below 1.3 is 11, there will be compilation problems.
  1. sh build.sh

After successful compilation, the file doris-flink-1.0.0-SNAPSHOT.jar will be generated in the output/ directory. Copy this file to ClassPath in Flink to use Flink-Doris-Connector. For example, Flink running in Local mode, put this file in the jars/ folder. Flink running in Yarn cluster mode, put this file in the pre-deployment package.

Remarks:

  1. Doris FE should be configured to enable http v2 in the configuration
  2. Scala version currently only supports 2.12.x version

conf/fe.conf

  1. enable_http_server_v2 = true

How to use

The purpose of this step is to register the Doris data source on Flink. This step is operated on Flink. There are two ways to use sql and java. The following are examples to illustrate

SQL

The purpose of this step is to register the Doris data source on Flink. This step is operated on Flink

  1. CREATE TABLE flink_doris_source (
  2. name STRING,
  3. age INT,
  4. price DECIMAL(5,2),
  5. sale DOUBLE
  6. )
  7. WITH (
  8. 'connector' = 'doris',
  9. 'fenodes' = '$YOUR_DORIS_FE_HOSTNAME:$YOUR_DORIS_FE_RESFUL_PORT',
  10. 'table.identifier' = '$YOUR_DORIS_DATABASE_NAME.$YOUR_DORIS_TABLE_NAME',
  11. 'username' = '$YOUR_DORIS_USERNAME',
  12. 'password' = '$YOUR_DORIS_PASSWORD'
  13. );
  14. CREATE TABLE flink_doris_sink (
  15. name STRING,
  16. age INT,
  17. price DECIMAL(5,2),
  18. sale DOUBLE
  19. )
  20. WITH (
  21. 'connector' = 'doris',
  22. 'fenodes' = '$YOUR_DORIS_FE_HOSTNAME:$YOUR_DORIS_FE_RESFUL_PORT',
  23. 'table.identifier' = '$YOUR_DORIS_DATABASE_NAME.$YOUR_DORIS_TABLE_NAME',
  24. 'username' = '$YOUR_DORIS_USERNAME',
  25. 'password' = '$YOUR_DORIS_PASSWORD'
  26. );
  27. INSERT INTO flink_doris_sink select name,age,price,sale from flink_doris_source

DataStreamSource

  1. Properties properties = new Properties();
  2. properties.put("fenodes","FE_IP:8030");
  3. properties.put("username","root");
  4. properties.put("password","");
  5. properties.put("table.identifier","db.table");
  6. env.addSource(new DorisSourceFunction(new DorisStreamOptions(properties),new SimpleListDeserializationSchema())).print();

DataStreamSink

  1. // -------- sink with raw json string stream --------
  2. Properties pro = new Properties();
  3. pro.setProperty("format", "json");
  4. pro.setProperty("strip_outer_array", "true");
  5. env.fromElements( "{\"longitude\": \"116.405419\", \"city\": \"北京\", \"latitude\": \"39.916927\"}")
  6. .addSink(
  7. DorisSink.sink(
  8. DorisReadOptions.builder().build(),
  9. DorisExecutionOptions.builder()
  10. .setBatchSize(3)
  11. .setBatchIntervalMs(0l)
  12. .setMaxRetries(3)
  13. .setStreamLoadProp(pro).build(),
  14. DorisOptions.builder()
  15. .setFenodes("FE_IP:8030")
  16. .setTableIdentifier("db.table")
  17. .setUsername("root")
  18. .setPassword("").build()
  19. ));
  20. OR
  21. env.fromElements("{\"longitude\": \"116.405419\", \"city\": \"北京\", \"latitude\": \"39.916927\"}")
  22. .addSink(
  23. DorisSink.sink(
  24. DorisOptions.builder()
  25. .setFenodes("FE_IP:8030")
  26. .setTableIdentifier("db.table")
  27. .setUsername("root")
  28. .setPassword("").build()
  29. ));
  30. // -------- sink with RowData stream --------
  31. DataStream<RowData> source = env.fromElements("")
  32. .map(new MapFunction<String, RowData>() {
  33. @Override
  34. public RowData map(String value) throws Exception {
  35. GenericRowData genericRowData = new GenericRowData(3);
  36. genericRowData.setField(0, StringData.fromString("北京"));
  37. genericRowData.setField(1, 116.405419);
  38. genericRowData.setField(2, 39.916927);
  39. return genericRowData;
  40. }
  41. });
  42. String[] fields = {"city", "longitude", "latitude"};
  43. LogicalType[] types = {new VarCharType(), new DoubleType(), new DoubleType()};
  44. source.addSink(
  45. DorisSink.sink(
  46. fields,
  47. types,
  48. DorisReadOptions.builder().build(),
  49. DorisExecutionOptions.builder()
  50. .setBatchSize(3)
  51. .setBatchIntervalMs(0L)
  52. .setMaxRetries(3)
  53. .build(),
  54. DorisOptions.builder()
  55. .setFenodes("FE_IP:8030")
  56. .setTableIdentifier("db.table")
  57. .setUsername("root")
  58. .setPassword("").build()
  59. ));

DataSetSink

  1. MapOperator<String, RowData> data = env.fromElements("")
  2. .map(new MapFunction<String, RowData>() {
  3. @Override
  4. public RowData map(String value) throws Exception {
  5. GenericRowData genericRowData = new GenericRowData(3);
  6. genericRowData.setField(0, StringData.fromString("北京"));
  7. genericRowData.setField(1, 116.405419);
  8. genericRowData.setField(2, 39.916927);
  9. return genericRowData;
  10. }
  11. });
  12. DorisOptions dorisOptions = DorisOptions.builder()
  13. .setFenodes("FE_IP:8030")
  14. .setTableIdentifier("db.table")
  15. .setUsername("root")
  16. .setPassword("").build();
  17. DorisReadOptions readOptions = DorisReadOptions.defaults();
  18. DorisExecutionOptions executionOptions = DorisExecutionOptions.defaults();
  19. LogicalType[] types = {new VarCharType(), new DoubleType(), new DoubleType()};
  20. String[] fiels = {"city", "longitude", "latitude"};
  21. DorisDynamicOutputFormat outputFormat =
  22. new DorisDynamicOutputFormat(dorisOptions, readOptions, executionOptions, types, fiels);
  23. outputFormat.open(0, 1);
  24. data.output(outputFormat);
  25. outputFormat.close();

General

KeyDefault ValueComment
fenodesDoris FE http address, support multiple addresses, separated by commas
table.identifierDoris table identifier, eg, db1.tbl1
usernameDoris username
passwordDoris password
doris.request.retries3Number of retries to send requests to Doris
doris.request.connect.timeout.ms30000Connection timeout for sending requests to Doris
doris.request.read.timeout.ms30000Read timeout for sending request to Doris
doris.request.query.timeout.s3600Query the timeout time of doris, the default is 1 hour, -1 means no timeout limit
doris.request.tablet.sizeInteger.MAX_VALUEThe number of Doris Tablets corresponding to an Partition. The smaller this value is set, the more partitions will be generated. This will increase the parallelism on the flink side, but at the same time will cause greater pressure on Doris.
doris.batch.size1024The maximum number of rows to read data from BE at one time. Increasing this value can reduce the number of connections between Flink and Doris. Thereby reducing the extra time overhead caused by network delay.
doris.exec.mem.limit2147483648Memory limit for a single query. The default is 2GB, in bytes.
doris.deserialize.arrow.asyncfalseWhether to support asynchronous conversion of Arrow format to RowBatch required for flink-doris-connector iteration
doris.deserialize.queue.size64Asynchronous conversion of the internal processing queue in Arrow format takes effect when doris.deserialize.arrow.async is true
doris.read.fieldList of column names in the Doris table, separated by commas
doris.filter.queryFilter expression of the query, which is transparently transmitted to Doris. Doris uses this expression to complete source-side data filtering.
sink.batch.size100Maximum number of lines in a single write BE
sink.max-retries1Number of retries after writing BE failed
sink.batch.interval1sThe flush interval, after which the asynchronous thread will write the data in the cache to BE. The default value is 1 second, and the time units are ms, s, min, h, and d. Set to 0 to turn off periodic writing.
sink.properties.*The stream load parameters.eg:sink.properties.column_separator’ = ‘,’. Setting ‘sink.properties.escape_delimiters’ = ‘true’ if you want to use a control char as a separator, so that such as ‘\x01’ will translate to binary 0x01
Support JSON format import, you need to enable both ‘sink.properties.format’ =’json’ and ‘sink.properties.strip_outer_array’ =’true’
Doris TypeFlink Type
NULL_TYPENULL
BOOLEANBOOLEAN
TINYINTTINYINT
SMALLINTSMALLINT
INTINT
BIGINTBIGINT
FLOATFLOAT
DOUBLEDOUBLE
DATESTRING
DATETIMESTRING
DECIMALDECIMAL
CHARSTRING
LARGEINTSTRING
VARCHARSTRING
DECIMALV2DECIMAL
TIMEDOUBLE
HLLUnsupported datatype