Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Feature] Flink Doris Connector (#5372) #5375

Merged
merged 15 commits into from
Apr 23, 2021

Conversation

JNSimba
Copy link
Member

@JNSimba JNSimba commented Feb 9, 2021

Proposed changes

Refer to Spark connector Doris and make an example of Flink Connector Doris

1.Customize DorisSource to implement interface org.apache.flink.streaming.api.functions.source.RichSourceFunction

2.Get dorisPartitions by calling the FE execution plan Api during initialization

3.Traverse dorisPartitions, read Doris data through scalaValueReader.class

Types of changes

What types of changes does your code introduce to Doris?
Put an x in the boxes that apply

  • Bugfix (non-breaking change which fixes an issue)
  • New feature (non-breaking change which adds functionality)
  • Breaking change (fix or feature that would cause existing functionality to not work as expected)
  • Documentation Update (if none of the other choices apply)
  • Code refactor (Modify the code structure, format the code, etc...)

Checklist

Put an x in the boxes that apply. You can also fill these out after creating the PR. If you're unsure about any of them, don't hesitate to ask. We're here to help! This is simply a reminder of what we are going to look for before merging your code.

  • I have created an issue on (Fix #5371) and described the bug/feature there in detail

  • Compiling and unit tests pass locally with my changes

  • I have added tests that prove my fix is effective or that my feature works

  • If these changes need document changes, I have updated the document

  • Any dependent changes have been merged

Further comments

If this is a relatively large or complex change, kick off the discussion at dev@doris.apache.org by explaining why you chose the solution you did and what alternatives you considered, etc...

@EmmyMiao87 EmmyMiao87 added area/spark-connector Issues or PRs related to Spark connector good first issue labels Feb 9, 2021
this.connectTimeout, this.socketTimeout, this.retries);
open();
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You need to write some annotation

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modification, submitted soon, and added flink SQL to create doris table

logger.warn(ErrorMessages.CONNECT_FAILED_MESSAGE, routing, e);
ex = e;
}
if (isConnected) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is better to be in try {}(just below isConnected = true;)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modification, submitted soon, and added flink SQL to create doris table

logger.info("Success connect to {}.", routing);
break;
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is better to return just after isConnected = true

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modification, submitted soon, and added flink SQL to create doris table

private void close() {
logger.trace("Connect status before close with '{}' is '{}'.", routing, isConnected);
isConnected = false;
if (null != client) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should close it before set client null

logger.debug("CloseScanner to '{}', parameter is '{}'.", routing, closeParams);
if (!isConnected) {
try {
open();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why open here?this will create a new client, do nothing, then close it?

}
return false;
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add some annotation

@hf200012
Copy link
Contributor

Modification, submitted soon, and added flink SQL to create doris table

…nnector-jdbc, add FlinkSql to create Doris table, and write to Doris table through streamload[Fix]
@JNSimba
Copy link
Member Author

JNSimba commented Mar 19, 2021

Refer to User-defined Sources & Sinks And flink-connector-jdbc, add flink sql to create doris table, and write doris table through streamload。

eg:

         EnvironmentSettings settings = EnvironmentSettings.newInstance()
                .useBlinkPlanner()
                .inStreamingMode()
                .build();
        TableEnvironment tEnv = TableEnvironment.create(settings);
        tEnv.getConfig().getConfiguration().setString("job.name","test");

        tEnv.executeSql(
                "CREATE TABLE doris_test (" +
                        "name STRING," +
                        "age INT," +
                        "price DECIMAL(5,2)," +
                        "sale DOUBLE" +
                        ") " +
                        "WITH (\n" +
                        "  'connector' = 'doris',\n" +
                        "  'fenodes' = 'FE_IP:8030',\n" +
                        "  'table.identifier' = 'demo.doris_test_source_2',\n" +
                        "  'username' = 'root',\n" +
                        "  'password' = ''" +
                        ")");
        tEnv.executeSql(
                "CREATE TABLE doris_test_sink (" +
                        "name STRING," +
                        "age INT," +
                        "price DECIMAL(5,2)," +
                        "sale DOUBLE" +
                        ") " +
                        "WITH (\n" +
                        "  'connector' = 'doris',\n" +
                        "  'fenodes' = 'FE_IP:8030',\n" +
                        "  'table.identifier' = 'demo.doris_test_sink_2',\n" +
                        "  'username' = 'root',\n" +
                        "  'password' = ''\n" +
                        ")");

        tEnv.executeSql("INSERT INTO doris_test_sink select name,age,price,sale from doris_test");

@JNSimba JNSimba changed the title [Feature] Add flink datastream connector example (#5371) [Feature] Flink Doris Connector (#5372) Mar 24, 2021
}

@Override
public void writeRecord(RowData row) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to buffer a batch of rows before triggering stream load

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes,already submitted

@JNSimba
Copy link
Member Author

JNSimba commented Mar 29, 2021

Added batch.size and max-retries configuration items, can set the batch size and the number of retries

@JNSimba
Copy link
Member Author

JNSimba commented Apr 6, 2021

add build script

morningman
morningman previously approved these changes Apr 7, 2021
Copy link
Contributor

@morningman morningman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@morningman morningman added the approved Indicates a PR has been approved by one committer. label Apr 7, 2021
@EmmyMiao87
Copy link
Contributor

It is recommended that you add usage methods and usage examples to the document later.

@JNSimba
Copy link
Member Author

JNSimba commented Apr 8, 2021

It is recommended that you add usage methods and usage examples to the document later.

already add.

docs/en/extending-doris/flink-doris-connector.md Outdated Show resolved Hide resolved
docs/en/extending-doris/flink-doris-connector.md Outdated Show resolved Hide resolved
docs/en/extending-doris/flink-doris-connector.md Outdated Show resolved Hide resolved
-->

# Flink Doris Connector

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

中文文档修改意见同英文

sh build.sh
```

编译成功后,会在 `output/` 目录下生成文件 `doris-flink-1.0.0-SNAPSHOT.jar`。将此文件复制到 `Flink` 的 `ClassPath` 中即可使用 `Flink-Doris-Connector`。例如,`Local` 模式运行的 `Flink`,将此文件放入 `jars/` 文件夹下。`Yarn`集群模式运行的`Flink`,则将此文件放入预部署包中。
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

我尝试编译了一下,编译完成后发现有两个jar包。一个叫 doris-flink-1.0-SNAPSHOT.jar, 一个叫original-doris-flink-1.0-SNAPSHOT.jar。这个 original-doris-flink-1.0-SNAPSHOT.jar 是干啥用的吗?

Copy link
Member Author

@JNSimba JNSimba Apr 11, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

已删除 original-doris-flink-1.0-SNAPSHOT.jar

| username | -- | 访问Doris的用户名 |
| password | -- | 访问Doris的密码 |
| sink.batch.size | 100 | 单次写BE的最大行数 |
| sink.max-retries | 1 | 写BE失败之后的重试次数 |
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

看起来可以调整的参数好像不止这些,可以后续把参数的含义都补充一下。比如mem limit这种。

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

参考spark-connector的参数,修改中,最近就会提交


@Override
public String toString() {
return "RespContent{" +
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Json tostring is better?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modification, submitted soon

DataStreamSource<String> source = env.fromElements("[{\"name\":\"doris\"}]\t1");
tEnv.createTemporaryView("doris_test",source,$("name"));

// Table wordWithCount = tEnv.sqlQuery("select name FROM doris_test ");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove it

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modification, submitted soon

}

public void convertArrowToRowBatch() throws DorisException {
try {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method is worth having a unit test ~

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modification, submitted soon

new ByteArrayInputStream(nextResult.getRows()),
rootAllocator
);
this.offsetInRowBatch = 0;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Although I don’t know whether it’s appropriate to put this while and exception logic in the constructor,
but could you please changed to a more elegant way of writing.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, it will be modified soon

JNSimba and others added 4 commits April 11, 2021 22:14
Co-authored-by: EmmyMiao87 <522274284@qq.com>
Co-authored-by: EmmyMiao87 <522274284@qq.com>
Co-authored-by: EmmyMiao87 <522274284@qq.com>
…is modified to BE, and re-acquire other surviving BE nodes after failure
@JNSimba
Copy link
Member Author

JNSimba commented Apr 14, 2021

1.Merge DorisOptions and Settings configuration ;
2.Streamload is modified to BE, and retry other alive BE nodes after failure
3.add more options reference spark-connector

morningman
morningman previously approved these changes Apr 20, 2021
Copy link
Contributor

@morningman morningman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@JNSimba
Copy link
Member Author

JNSimba commented Apr 20, 2021

add pom profile patch

Copy link
Contributor

@morningman morningman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@morningman morningman merged commit 7eea811 into apache:master Apr 23, 2021
@xhmz
Copy link
Contributor

xhmz commented Jul 5, 2021

Failed to get response from Doris http://10.0.10.28:8030/rest/v1/system?path=//backends, http code is 404

@JNSimba
Copy link
Member Author

JNSimba commented Jul 5, 2021

Failed to get response from Doris http://10.0.10.28:8030/rest/v1/system?path=//backends, http code is 404

enable http v2

@xhmz
Copy link
Contributor

xhmz commented Jul 5, 2021

enable_http_server_v2 ?

@JNSimba
Copy link
Member Author

JNSimba commented Jul 5, 2021

enable_http_server_v2 ?

yes,enable_http_server_v2= true

@morningman morningman mentioned this pull request Oct 10, 2021
@meitianjinbu
Copy link

my doris version 0.15.0 rc02,enable_http_server_v2 is true default?but failed to get response from doris http://10.0.10.28:8030/api/backends?is_alive=true

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
approved Indicates a PR has been approved by one committer. area/spark-connector Issues or PRs related to Spark connector good first issue
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants