Skip to content

Commit

Permalink
[fix](arrow-flight-sql) Arrow flight server supports data forwarding …
Browse files Browse the repository at this point in the history
…when BE uses public vip (apache#43281)

If there is a Doris cluster, its FE node can be accessed by the external
network, and all its BE nodes can only be accessed by the internal
network.

This is fine when using Mysql client and JDBC to connect to Doris to
execute queries, and the query results will be returned to the client by
the Doris FE node.

However, using Arrow Flight SQL to connect to Doris cannot execute
queries, because the ADBC ​​client needs to connect to the Doris BE node
to pull query results, but the Doris BE node is not allowed to be
accessed by the external network.

In a production environment, it is often inconvenient to expose Doris BE
nodes to the external network. However, a reverse proxy (such as nginx)
can be added to all Doris BE nodes, and the external client will be
randomly routed to a Doris BE node when connecting to nginx.

The query results of Arrow Flight SQL will be randomly saved on a Doris
BE node. If it is different from the Doris BE node randomly routed by
nginx, data forwarding needs to be done inside the Doris BE node.

1. The Ticket returned by Doris FE Arrow Flight Server to ADBC ​​client
contains the IP and Brpc Port of the Doris BE node where the query
result is located.

2. Doris BE Arrow Flight Server receives a request to pull data. If the
IP:BrpcPort in the Ticket is not itself, it pulls the query result Block
from the Doris BE node specified by IP:BrpcPort, converts it to Arrow
Batch and returns it to ADBC ​​Client; if the IP:BrpcPort in the Ticket
is itself, it is the same as before.

1. If the data is not in the current BE node, you can pull the data from
other BE nodes asynchronously and cache at least one Block locally in
the current BE node, which will reduce the time consumption of
serialization, deserialization, and RPC.

1. Create a Doris cluster with 1 FE and 2 BE, and modify
`arrow_flight_sql_port` in `fe.conf` and `be.conf`.

2. Root executes `systemctl status nginx` to check whether Nginx is
installed. If not, yum install is recommended.

3. `vim /etc/nginx/nginx.conf` adds `underscores_in_headers on;`

4. `touch /etc/nginx/conf.d/arrowflight.conf` creates a file, and `vim
/etc/nginx/conf.d/arrowflight.conf` adds:

```
upstream arrowflight {
    server {BE1_ip}:{BE1_arrow_flight_sql_port};
    server {BE2_IP}:{BE2_arrow_flight_sql_port};
}

server {
    listen {nginx port} http2;
    listen [::]:{nginx port} http2;
    server_name doris.arrowflight.com;

    #ssl_certificate   /etc/nginx/cert/myCA.pem;
    #ssl_certificate_key /etc/nginx/cert/myCA.key;

    location / {
        grpc_pass grpc://arrowflight;
        grpc_set_header X-Real-IP $remote_addr;
        grpc_set_header X-Forwarded-For $proxy_add_x_forwarded_for;
        grpc_set_header X-Forwarded-Proto $scheme;

        proxy_read_timeout 60s;
        proxy_send_timeout 60s;

        #proxy_http_version 1.1;
        #proxy_set_header Connection "";
    }
}
```

Where {BE1_ip}:{BE1_arrow_flight_sql_port} is the IP of BE 1 and
arrow_flight_sql_port in be.conf, and similarly
{BE2_IP}:{BE2_arrow_flight_sql_port}. `{nginx port}` is any available
port.

6. Add in be.conf of all BEs:

```
public_access_ip={nginx ip}
public_access_port={nginx port}
```

---

如果存在一个 Doris 集群,它的 FE 节点可以被外部网络访问,它的所有 BE 节点只可以被内网访问。

这在使用 Mysql client 和 JDBC 连接 Doris 执行查询是没问题的,查询结果将由 Doris FE 节点返回给
client。

但使用 Arrow Flight SQL 连接 Doris 无法执行查询,因为 ADBC client 需要连接 Doris BE
节点拉取查询结果,但 Doris BE 节点不允许被外网访问。

生产环境中,很多时候不方便在外网暴露 Doris BE 节点。但可以为所有 Doris BE 节点增加了一层反向代理(比如 nginx),外网的
client 连接 nginx 时会随机路由到一台 Doris BE 节点上。

Arrow Flight SQL 查询结果会随机保存在一台 Doris BE 节点上,如果和 nginx 随机路由的 Doris BE
节点不同,需要在 Doris BE 节点内部做一次数据转发。

1. Doris FE Arrow Flight Server 向 ADBC client 返回的 Ticket 中包含查询结果所在 Doris
BE节点的 IP 和 Brpc Port。
2. Doris BE Arrow Flight Server 收到拉取数据请求。如果 Ticket 中的 IP:BrpcPort
不是自己,则从 IP:BrpcPort 指定的 Doris BE 节点拉取查询结果Block,转为 Arrow Batch 后返回 ADBC
Client;如果 Ticket 中的 IP:BrpcPort 是自己,则和过去一样。

1. 若数据不在当前 BE 节点,可以异步的从其他 BE 节点拉取数据,并在当前 BE 节点本地缓存至少一个
Block,这将减少序列化、反序列化、RPC 的耗时。

1. 创建一个 1 FE 和 2 BE 的 Doris 集群,修改 `fe.conf` 和 `be.conf` 中的
`arrow_flight_sql_port`。
2. Root 执行 `systemctl status nginx` 查看是否安装 Nginx,若没有则推荐 yum install。
3. `vim /etc/nginx/nginx.conf` 增加 `underscores_in_headers on;`
4. `touch /etc/nginx/conf.d/arrowflight.conf` 创建文件,`vim
/etc/nginx/conf.d/arrowflight.conf` 增加:

```
upstream arrowflight {
    server {BE1_ip}:{BE1_arrow_flight_sql_port};
    server {BE2_IP}:{BE2_arrow_flight_sql_port};
}

server {
    listen {nginx port} http2;
    listen [::]:{nginx port} http2;
    server_name doris.arrowflight.com;

    #ssl_certificate   /etc/nginx/cert/myCA.pem;
    #ssl_certificate_key /etc/nginx/cert/myCA.key;

    location / {
        grpc_pass grpc://arrowflight;
        grpc_set_header X-Real-IP $remote_addr;
        grpc_set_header X-Forwarded-For $proxy_add_x_forwarded_for;
        grpc_set_header X-Forwarded-Proto $scheme;

        proxy_read_timeout 60s;
        proxy_send_timeout 60s;

        #proxy_http_version 1.1;
        #proxy_set_header Connection "";
    }
}
```

其中 {BE1_ip}:{BE1_arrow_flight_sql_port} 是 BE 1 的 IP 和 be.conf 中的
arrow_flight_sql_port,同理 {BE2_IP}:{BE2_arrow_flight_sql_port}。`{nginx
port}` 是一个任意可用端口。

6. 在所有 BE 的 be.conf 中增加

```
public_access_ip={nginx ip}
public_access_port={nginx port}
```
  • Loading branch information
xinyiZzz committed Nov 20, 2024
1 parent 04516fb commit 97e009f
Show file tree
Hide file tree
Showing 24 changed files with 858 additions and 250 deletions.
7 changes: 7 additions & 0 deletions be/src/common/config.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ DEFINE_Int32(brpc_port, "8060");
DEFINE_Int32(arrow_flight_sql_port, "-1");

DEFINE_mString(public_access_ip, "");
DEFINE_Int32(public_access_port, "-1");

// the number of bthreads for brpc, the default value is set to -1,
// which means the number of bthreads is #cpu-cores
Expand Down Expand Up @@ -535,6 +536,8 @@ DEFINE_Int32(brpc_light_work_pool_threads, "-1");
DEFINE_Int32(brpc_heavy_work_pool_max_queue_size, "-1");
DEFINE_Int32(brpc_light_work_pool_max_queue_size, "-1");
DEFINE_mBool(enable_bthread_transmit_block, "true");
DEFINE_Int32(brpc_arrow_flight_work_pool_threads, "-1");
DEFINE_Int32(brpc_arrow_flight_work_pool_max_queue_size, "-1");

//Enable brpc builtin services, see:
//https://brpc.apache.org/docs/server/basics/#disable-built-in-services-completely
Expand Down Expand Up @@ -643,7 +646,11 @@ DEFINE_Int32(load_process_safe_mem_permit_percent, "5");
// result buffer cancelled time (unit: second)
DEFINE_mInt32(result_buffer_cancelled_interval_time, "300");

// arrow flight result sink buffer rows size, default 4096 * 8
DEFINE_mInt32(arrow_flight_result_sink_buffer_size_rows, "32768");
// The timeout for ADBC Client to wait for data using arrow flight reader.
// If the query is very complex and no result is generated after this time, consider increasing this timeout.
DEFINE_mInt32(arrow_flight_reader_brpc_controller_timeout_ms, "300000");

// the increased frequency of priority for remaining tasks in BlockingPriorityQueue
DEFINE_mInt32(priority_queue_remaining_tasks_increased_frequency, "512");
Expand Down
6 changes: 6 additions & 0 deletions be/src/common/config.h
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ DECLARE_Int32(arrow_flight_sql_port);
// For ADBC client fetch result, default is empty, the ADBC client uses the backend ip to fetch the result.
// If ADBC client cannot access the backend ip, can set public_access_ip to modify the fetch result ip.
DECLARE_mString(public_access_ip);
DECLARE_Int32(public_access_port);

// the number of bthreads for brpc, the default value is set to -1,
// which means the number of bthreads is #cpu-cores
Expand Down Expand Up @@ -584,6 +585,8 @@ DECLARE_Int32(brpc_light_work_pool_threads);
DECLARE_Int32(brpc_heavy_work_pool_max_queue_size);
DECLARE_Int32(brpc_light_work_pool_max_queue_size);
DECLARE_mBool(enable_bthread_transmit_block);
DECLARE_Int32(brpc_arrow_flight_work_pool_threads);
DECLARE_Int32(brpc_arrow_flight_work_pool_max_queue_size);

// The maximum amount of data that can be processed by a stream load
DECLARE_mInt64(streaming_load_max_mb);
Expand Down Expand Up @@ -693,6 +696,9 @@ DECLARE_mInt32(result_buffer_cancelled_interval_time);

// arrow flight result sink buffer rows size, default 4096 * 8
DECLARE_mInt32(arrow_flight_result_sink_buffer_size_rows);
// The timeout for ADBC Client to wait for data using arrow flight reader.
// If the query is very complex and no result is generated after this time, consider increasing this timeout.
DECLARE_mInt32(arrow_flight_reader_brpc_controller_timeout_ms);

// the increased frequency of priority for remaining tasks in BlockingPriorityQueue
DECLARE_mInt32(priority_queue_remaining_tasks_increased_frequency);
Expand Down
2 changes: 1 addition & 1 deletion be/src/pipeline/exec/memory_scratch_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ Status MemoryScratchSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
{
SCOPED_TIMER(local_state._get_arrow_schema_timer);
// After expr executed, use recaculated schema as final schema
RETURN_IF_ERROR(get_arrow_schema(block, &block_arrow_schema, state->timezone()));
RETURN_IF_ERROR(get_arrow_schema_from_block(block, &block_arrow_schema, state->timezone()));
}
{
SCOPED_TIMER(local_state._convert_block_to_arrow_batch_timer);
Expand Down
3 changes: 1 addition & 2 deletions be/src/pipeline/exec/result_file_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -92,8 +92,7 @@ Status ResultFileSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& i
_sender = _parent->cast<ResultFileSinkOperatorX>()._sender;
} else {
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
state->fragment_instance_id(), p._buf_size, &_sender, state->execution_timeout(),
state->batch_size()));
state->fragment_instance_id(), p._buf_size, &_sender, state));
}
_sender->set_dependency(state->fragment_instance_id(), _dependency->shared_from_this());

Expand Down
19 changes: 6 additions & 13 deletions be/src/pipeline/exec/result_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,7 @@ Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info)
} else {
auto& p = _parent->cast<ResultSinkOperatorX>();
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
fragment_instance_id, p._result_sink_buffer_size_rows, &_sender,
state->execution_timeout(), state->batch_size()));
fragment_instance_id, p._result_sink_buffer_size_rows, &_sender, state));
}
_sender->set_dependency(fragment_instance_id, _dependency->shared_from_this());
return Status::OK();
Expand Down Expand Up @@ -81,16 +80,11 @@ Status ResultSinkLocalState::open(RuntimeState* state) {
}
case TResultSinkType::ARROW_FLIGHT_PROTOCAL: {
std::shared_ptr<arrow::Schema> arrow_schema;
RETURN_IF_ERROR(convert_expr_ctxs_arrow_schema(_output_vexpr_ctxs, &arrow_schema,
state->timezone()));
if (state->query_options().enable_parallel_result_sink) {
state->exec_env()->result_mgr()->register_arrow_schema(state->query_id(), arrow_schema);
} else {
state->exec_env()->result_mgr()->register_arrow_schema(state->fragment_instance_id(),
arrow_schema);
}
RETURN_IF_ERROR(get_arrow_schema_from_expr_ctxs(_output_vexpr_ctxs, &arrow_schema,
state->timezone()));
_sender->register_arrow_schema(arrow_schema);
_writer.reset(new (std::nothrow) vectorized::VArrowFlightResultWriter(
_sender.get(), _output_vexpr_ctxs, _profile, arrow_schema));
_sender.get(), _output_vexpr_ctxs, _profile));
break;
}
default:
Expand Down Expand Up @@ -135,8 +129,7 @@ Status ResultSinkOperatorX::open(RuntimeState* state) {

if (state->query_options().enable_parallel_result_sink) {
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
state->query_id(), _result_sink_buffer_size_rows, &_sender,
state->execution_timeout(), state->batch_size()));
state->query_id(), _result_sink_buffer_size_rows, &_sender, state));
}
return vectorized::VExpr::open(_output_vexpr_ctxs, state);
}
Expand Down
Loading

0 comments on commit 97e009f

Please sign in to comment.