You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/10/20 11:06:41 UTC

[inlong-website] branch master updated: [INLONG-6238][Doc] Add configuration doc for DataProxy (#571)

This is an automated email from the ASF dual-hosted git repository.

dockerzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/inlong-website.git


The following commit(s) were added to refs/heads/master by this push:
     new 3953f07a4c [INLONG-6238][Doc] Add configuration doc for DataProxy (#571)
3953f07a4c is described below

commit 3953f07a4c48514bebfb1a740fe0c47dbedafd10
Author: Goson Zhang <46...@qq.com>
AuthorDate: Thu Oct 20 19:06:35 2022 +0800

    [INLONG-6238][Doc] Add configuration doc for DataProxy (#571)
---
 docs/modules/dataproxy/configuration.md            | 186 ++++++++++++++++++++
 docs/modules/dataproxy/overview.md                 | 139 +--------------
 docs/modules/dataproxy/quick_start.md              |   1 +
 .../current/modules/dataproxy/configuration.md     | 187 +++++++++++++++++++++
 .../current/modules/dataproxy/overview.md          | 138 +--------------
 .../current/modules/dataproxy/quick_start.md       |   1 +
 6 files changed, 377 insertions(+), 275 deletions(-)

diff --git a/docs/modules/dataproxy/configuration.md b/docs/modules/dataproxy/configuration.md
new file mode 100644
index 0000000000..e345ce4806
--- /dev/null
+++ b/docs/modules/dataproxy/configuration.md
@@ -0,0 +1,186 @@
+---
+title: Configuration
+sidebar_position: 3
+---
+## Basic Configuration (common.properties)
+
+| Parameter                                                     | Description                                                                                         | Default                                  | Notes                                                                                                                                                          |
+|--------------------------------------------------------|-----------------------------------------------------------------------------------------------------|--------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| manager.hosts                                           | InLong-Manager http host and port                                                                   | 127.0.0.1:8083                        | Empty is not allowed, the format is configured according to the format of {ip1:port1}[,{ip2:port2}][,{ip3:port3}]                                              |
+| manager.auth.secretId                                   | InLong-Manager authentic secretId                                                                   | `blank`                                 | If InLong-Manager doesn't open authentic service, this parameter can be empty.                                                                                 |
+| manager.auth.secretKey                                  | InLong-Manager authentic secretKey                                                                  | `blank`                                    |                                                                                                                                                                |
+| proxy.cluster.tag                                       | The cluster Tag name where the dataproxy is located                                                 | default_cluster                      | A cluster Tag can contain multiple dataproxy and MQ clusters                                                                                                   |
+| proxy.cluster.name                                      | The cluster name where dataproxy is located                                                         | default_dataproxy                    | used to distinguish different environments                                                                                                                     |
+| proxy.cluster.inCharges                                 | The incharge of the cluster where dataproxy is located                                              | admin                                |                                                                                                                                                                |
+| configCheckInterval                                     | Configuration file checking and loading interval of the conf directory (unit: ms)                   | 10000                          |                                                                                                                                                                |
+| metricDomains                                           | JMX domain name                                                                                     | DataProxy                             | Obtain the following configuration items of "metricDomains.${metricDomains}.xxx" through this configuration value                                              |
+| metricDomains.DataProxy.domainListeners                 | The class for indicator monitoring, and the service is started through reflection of the class name | org.apache.inlong.dataproxy.metrics.prometheus.PrometheusMetricListener    | If there are multiple indicator monitoring class configurations, separate them by spaces, carriage returns, or line feeds                                      |
+| metricDomains.DataProxy.snapshotInterval                | Time interval for periodic reporting of indicators (unit: ms)                                       | 60000                             |                                                                                                                                                                |
+| prometheusHttpPort                                      | The port when reporting using Prometheus                                                            | 9081                               |                                                                                                                                                                |
+| audit.enable                                            | Whether to enable data reporting InLong-Audit service                                               | true                                |                                                                                                                                                                |
+| audit.proxys                                            | The address of the InLong-Audit service                                                             | 127.0.0.1:10081                      | The format is configured according to {ip1:port1}[ {ip2:port2}][ {ip3:port3}], and different ip:ports are separated by spaces, carriage returns, or line feeds |
+
+
+## Log Output Configuration(log4j2.xml)
+DataProxy uses Log4j2 to output logs, and the related configuration is set based on Log4j2. This configuration only mentions common setting items:
+
+| Parameter                                                     | Description                                                            | Default                                  | Notes                              |
+|-------------------------|------------------------------------------------------------------------|-------|----------------|
+| basePath      | Storage path of log files                                                              | ${sys:dataproxy.log.path}    |                |
+| every_file_size      | The size of each log file                                                         | 1G    |                |
+| output_log_level | log output level                                                                | DEBUG | It is recommended to set to INFO when using online |
+| rolling_max    | The number of logs of the same type that can be stored in the same directory                            | 50    |                |
+| debug_max    | The number of DEBUG type logs that can be stored in the same directory | 7     |                |
+| info_max    | The number of INFO type logs that can be stored in the same directory  | 7     |                |
+| warn_max    | The number of WARN type logs that can be stored in the same directory  | 7     |                |
+| error_max    | The number of ERROR type logs that can be stored in the same directory | 7     |                |
+
+
+## Other Auto-Update Configurations
+After DataProxy is started and successfully linked to the Manager, it will automatically synchronize the running configuration from the Manager and update it regularly. The following configuration cannot be modified.
+
+| File Name             | Description                    |  Notes             |
+|-----------------------|-----------------------|-----------------|
+| mq_cluster.properties | MQ cluster configuration                |     |
+| topics.properties     | Mapping configuration of groupId to MQ topic |     |
+| weight.properties     | System Load Weight Configuration             |  |
+
+
+
+## Source-Channel-Sink Configuration(dataproxy-{tube|pulsar}.conf)
+DataProxy supports configurable source-channel-sink, which is consistent to flume's configuration file structure,  so it should be modified according to the configuration file definition of Apache flume. The configuration file is placed in the dataproxy-{tube|pulsar}.conf file. Currently, dataproxy-pulsar.conf and dataproxy-tube.conf are supported to distinguish different middleware types. The specific type can be specified at startup. The default (when not specified) ) using dataproxy-p [...]
+
+- Source configuration example:
+```shell
+agent1.sources.tcp-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-more1 ch-more2 ch-more3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-transfer ch -Back
+Define the channel used in the source. Note that if the configuration below this source uses the channel, it needs to be annotated here
+
+agent1.sources.tcp-source.type = org.apache.flume.source.SimpleTcpSource
+tcp resolution type definition, here provide the class name for instantiation, SimpleTcpSource is mainly to initialize the configuration and start port monitoring
+
+agent1.sources.tcp-source.msg-factory-name = org.apache.flume.source.ServerMessageFactory
+Handler used for message structure analysis, and set read stream handler and write stream handler
+
+agent1.sources.tcp-source.host = 0.0.0.0
+tcp ip binding monitoring, binding all network cards by default
+
+agent1.sources.tcp-source.port = 46801
+tcp port binding, port 46801 is bound by default
+
+agent1.sources.tcp-source.highWaterMark=2621440
+The concept of netty, set the netty high water level value
+
+agent1.sources.tcp-source.enableExceptionReturn=true
+The new function of v1.7 version, optional, the default is false, used to open the exception channel, when an exception occurs, the data is written to the exception channel to prevent other normal data transmission (the open source version does not add this function), Details  |  Increase the local disk of abnormal data landing
+
+agent1.sources.tcp-source.max-msg-length = 524288
+Limit the size of a single package, here if the compressed package is transmitted, it is the compressed package size, the limit is 512KB
+
+agent1.sources.tcp-source.topic = test_token
+The default topic value, if the mapping relationship between groupId and topic cannot be found, it will be sent to this topic
+
+agent1.sources.tcp-source.attr = m=9
+The default value of m is set, where the value of m is the version of inlong's internal TdMsg protocol
+
+agent1.sources.tcp-source.connections = 5000
+Concurrent connections go online, new connections will be broken when the upper limit is exceeded
+
+agent1.sources.tcp-source.max-threads = 64
+Netty thread pool work thread upper limit, generally recommended to choose twice the cpu
+
+agent1.sources.tcp-source.receiveBufferSize = 524288
+Netty server tcp tuning parameters
+
+agent1.sources.tcp-source.sendBufferSize = 524288
+Netty server tcp tuning parameters
+
+agent1.sources.tcp-source.custom-cp = true
+Whether to use the self-developed channel process, the self-developed channel process can select the alternate channel to send when the main channel is blocked
+
+agent1.sources.tcp-source.selector.type = org.apache.flume.channel.FailoverChannelSelector
+This channel selector is a self-developed channel selector, which is not much different from the official website, mainly because of the channel master-slave selection logic
+
+agent1.sources.tcp-source.selector.master = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
+Specify the master channel, these channels will be preferentially selected for data push. Those channels that are not in the master, transfer, fileMetric, and slaMetric configuration items, but are in
+There are defined channels in channels, which are all classified as slave channels. When the master channel is full, the slave channel will be selected. Generally, the file channel type is recommended for the slave channel.
+
+agent1.sources.tcp-source.selector.transfer = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
+Specify the transfer channel to accept the transfer type data. The transfer here generally refers to the data pushed to the non-tube cluster, which is only for forwarding, and it is reserved for subsequent functions.
+
+agent1.sources.tcp-source.selector.fileMetric = ch-back
+Specify the fileMetric channel to receive the metric data reported by the agent
+```
+
+- Channel configuration examples, memory channel:
+
+```shell
+agent1.channels.ch-more1.type = memory
+memory channel type
+
+agent1.channels.ch-more1.capacity = 10000000
+Memory channel queue size, the maximum number of messages that can be cached
+
+agent1.channels.ch-more1.keep-alive = 0
+
+agent1.channels.ch-more1.transactionCapacity = 20
+The maximum number of batches are processed in atomic operations, and the memory channel needs to be locked when used, so there will be a batch process to increase efficiency
+```
+
+- Channel configuration examples, file channel:
+
+```shell
+agent1.channels.ch-msg5.type = file
+file channel type
+
+agent1.channels.ch-msg5.capacity = 100000000
+The maximum number of messages that can be cached in a file channel
+
+agent1.channels.ch-msg5.maxFileSize = 1073741824
+file channel file maximum limit, the number of bytes
+
+agent1.channels.ch-msg5.minimumRequiredSpace = 1073741824
+The minimum free space of the disk where the file channel is located. Setting this value can prevent the disk from being full
+
+agent1.channels.ch-msg5.checkpointDir = /data/work/file/ch-msg5/check
+file channel checkpoint path
+
+agent1.channels.ch-msg5.dataDirs = /data/work/file/ch-msg5/data
+file channel data path
+
+agent1.channels.ch-msg5.fsyncPerTransaction = false
+Whether to synchronize the disk for each atomic operation, it is recommended to change it to false, otherwise it will affect the performance
+
+agent1.channels.ch-msg5.fsyncInterval = 5
+The time interval between data flush from memory to disk, in seconds
+```
+
+- Sink configuration example:
+
+```shell
+agent1.sinks.meta-sink-more1.channel = ch-msg1
+The upstream channel name of the sink
+
+agent1.sinks.meta-sink-more1.type = org.apache.flume.sink.MetaSink
+The sink class is implemented, where the message is implemented to push data to the tube cluster
+
+agent1.sinks.meta-sink-more1.master-host-port-list =
+Tube cluster master node list
+
+agent1.sinks.meta-sink-more1.send_timeout = 30000
+Timeout limit when sending to tube
+
+agent1.sinks.meta-sink-more1.stat-interval-sec = 60
+Sink indicator statistics interval time, in seconds
+
+agent1.sinks.meta-sink-more1.thread-num = 8
+Sink class sends messages to the worker thread, 8 means to start 8 concurrent threads
+
+agent1.sinks.meta-sink-more1.client-id-cache = true
+agent id cache, used to check the data reported by the agent to remove duplicates
+
+agent1.sinks.meta-sink-more1.max-survived-time = 300000
+Maximum cache time
+
+agent1.sinks.meta-sink-more1.max-survived-size = 3000000
+Maximum number of caches
+```
\ No newline at end of file
diff --git a/docs/modules/dataproxy/overview.md b/docs/modules/dataproxy/overview.md
index c03d9cab35..59fd821797 100644
--- a/docs/modules/dataproxy/overview.md
+++ b/docs/modules/dataproxy/overview.md
@@ -1,5 +1,6 @@
 ---
 title: Overview
+sidebar_position: 1
 ---
 
 InLong DataProxy mainly consists of connection convergence, routing, data compression, and protocol conversion. DataProxy acts as a bridge from the InLong Agent to the message queue, When the DataProxy pulls the metadata of the data streams from the Manager module, the corresponding relationship between the data streams and the topic name of the message queue is determined. When DataProxy receives a message, it will first be sent to Memory Channel for compression.
@@ -13,144 +14,6 @@ The overall architecture of InLong DataProxy is based on Apache Flume, which ext
 - The channel layer has a selector, which is used to choose which type of channel to go. If the memory is eventually full, the data will be processed.
 - The data of the channel layer will be forwarded through the sink layer. The main purpose here is to convert the data to the TDMsg1 format and push it to the cache layer (tube is more commonly used here)
 
-## DataProxy Configuration
-
-DataProxy supports configurable source-channel-sink, which is similar to flume's configuration file structure. The configuration file name is such as dataproxy-*.conf. Currently, dataproxy-pulsar.conf and dataproxy-tube.conf are supported to distinguish different message middleware types. The specific type can be specified when startup. The default (when not specified) ) using dataproxy-pulsar.conf as configuration file.
-
-- Source configuration example:
-```shell
-agent1.sources.tcp-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-more1 ch-more2 ch-more3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-transfer ch -Back
-Define the channel used in the source. Note that if the configuration below this source uses the channel, it needs to be annotated here
-
-agent1.sources.tcp-source.type = org.apache.flume.source.SimpleTcpSource
-tcp resolution type definition, here provide the class name for instantiation, SimpleTcpSource is mainly to initialize the configuration and start port monitoring
-
-agent1.sources.tcp-source.msg-factory-name = org.apache.flume.source.ServerMessageFactory
-Handler used for message structure analysis, and set read stream handler and write stream handler
-
-agent1.sources.tcp-source.host = 0.0.0.0
-tcp ip binding monitoring, binding all network cards by default
-
-agent1.sources.tcp-source.port = 46801
-tcp port binding, port 46801 is bound by default
-
-agent1.sources.tcp-source.highWaterMark=2621440
-The concept of netty, set the netty high water level value
-
-agent1.sources.tcp-source.enableExceptionReturn=true
-The new function of v1.7 version, optional, the default is false, used to open the exception channel, when an exception occurs, the data is written to the exception channel to prevent other normal data transmission (the open source version does not add this function), Details  |  Increase the local disk of abnormal data landing
-
-agent1.sources.tcp-source.max-msg-length = 524288
-Limit the size of a single package, here if the compressed package is transmitted, it is the compressed package size, the limit is 512KB
-
-agent1.sources.tcp-source.topic = test_token
-The default topic value, if the mapping relationship between groupId and topic cannot be found, it will be sent to this topic
-
-agent1.sources.tcp-source.attr = m=9
-The default value of m is set, where the value of m is the version of inlong's internal TdMsg protocol
-
-agent1.sources.tcp-source.connections = 5000
-Concurrent connections go online, new connections will be broken when the upper limit is exceeded
-
-agent1.sources.tcp-source.max-threads = 64
-Netty thread pool work thread upper limit, generally recommended to choose twice the cpu
-
-agent1.sources.tcp-source.receiveBufferSize = 524288
-Netty server tcp tuning parameters
-
-agent1.sources.tcp-source.sendBufferSize = 524288
-Netty server tcp tuning parameters
-
-agent1.sources.tcp-source.custom-cp = true
-Whether to use the self-developed channel process, the self-developed channel process can select the alternate channel to send when the main channel is blocked
-
-agent1.sources.tcp-source.selector.type = org.apache.flume.channel.FailoverChannelSelector
-This channel selector is a self-developed channel selector, which is not much different from the official website, mainly because of the channel master-slave selection logic
-
-agent1.sources.tcp-source.selector.master = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
-Specify the master channel, these channels will be preferentially selected for data push. Those channels that are not in the master, transfer, fileMetric, and slaMetric configuration items, but are in
-There are defined channels in channels, which are all classified as slave channels. When the master channel is full, the slave channel will be selected. Generally, the file channel type is recommended for the slave channel.
-
-agent1.sources.tcp-source.selector.transfer = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
-Specify the transfer channel to accept the transfer type data. The transfer here generally refers to the data pushed to the non-tube cluster, which is only for forwarding, and it is reserved for subsequent functions.
-
-agent1.sources.tcp-source.selector.fileMetric = ch-back
-Specify the fileMetric channel to receive the metric data reported by the agent
-```
-
-- Channel configuration examples, memory channel:
-
-```shell
-agent1.channels.ch-more1.type = memory
-memory channel type
-
-agent1.channels.ch-more1.capacity = 10000000
-Memory channel queue size, the maximum number of messages that can be cached
-
-agent1.channels.ch-more1.keep-alive = 0
-
-agent1.channels.ch-more1.transactionCapacity = 20
-The maximum number of batches are processed in atomic operations, and the memory channel needs to be locked when used, so there will be a batch process to increase efficiency
-```
-
-- Channel configuration examples, file channel:
-
-```shell
-agent1.channels.ch-msg5.type = file
-file channel type
-
-agent1.channels.ch-msg5.capacity = 100000000
-The maximum number of messages that can be cached in a file channel
-
-agent1.channels.ch-msg5.maxFileSize = 1073741824
-file channel file maximum limit, the number of bytes
-
-agent1.channels.ch-msg5.minimumRequiredSpace = 1073741824
-The minimum free space of the disk where the file channel is located. Setting this value can prevent the disk from being full
-
-agent1.channels.ch-msg5.checkpointDir = /data/work/file/ch-msg5/check
-file channel checkpoint path
-
-agent1.channels.ch-msg5.dataDirs = /data/work/file/ch-msg5/data
-file channel data path
-
-agent1.channels.ch-msg5.fsyncPerTransaction = false
-Whether to synchronize the disk for each atomic operation, it is recommended to change it to false, otherwise it will affect the performance
-
-agent1.channels.ch-msg5.fsyncInterval = 5
-The time interval between data flush from memory to disk, in seconds
-```
-
-- Sink configuration example:
-
-```shell
-agent1.sinks.meta-sink-more1.channel = ch-msg1
-The upstream channel name of the sink
-
-agent1.sinks.meta-sink-more1.type = org.apache.flume.sink.MetaSink
-The sink class is implemented, where the message is implemented to push data to the tube cluster
-
-agent1.sinks.meta-sink-more1.master-host-port-list =
-Tube cluster master node list
-
-agent1.sinks.meta-sink-more1.send_timeout = 30000
-Timeout limit when sending to tube
-
-agent1.sinks.meta-sink-more1.stat-interval-sec = 60
-Sink indicator statistics interval time, in seconds
-
-agent1.sinks.meta-sink-more1.thread-num = 8
-Sink class sends messages to the worker thread, 8 means to start 8 concurrent threads
-
-agent1.sinks.meta-sink-more1.client-id-cache = true
-agent id cache, used to check the data reported by the agent to remove duplicates
-
-agent1.sinks.meta-sink-more1.max-survived-time = 300000
-Maximum cache time
-
-agent1.sinks.meta-sink-more1.max-survived-size = 3000000
-Maximum number of caches
-```
 
 ## Monitor Metrics configuration
 
diff --git a/docs/modules/dataproxy/quick_start.md b/docs/modules/dataproxy/quick_start.md
index e2804e740a..bd53d86cf9 100644
--- a/docs/modules/dataproxy/quick_start.md
+++ b/docs/modules/dataproxy/quick_start.md
@@ -1,5 +1,6 @@
 ---
 title: Deployment
+sidebar_position: 2
 ---
 
 All deploying files at `inlong-dataproxy` directory.
diff --git a/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/configuration.md b/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/configuration.md
new file mode 100644
index 0000000000..ac48207ed9
--- /dev/null
+++ b/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/configuration.md
@@ -0,0 +1,187 @@
+---
+title: 配置
+sidebar_position: 3
+---
+
+## 基本配置( common.properties )
+
+| 参数                                                      | 描述                          | 默认值                                  | 备注                                                                             |
+|---------------------------------------------------------|-----------------------------|--------------------------------------|--------------------------------------------------------------------------------|
+| manager.hosts                                           | Manager 服务器 IP 和端口列表        | 127.0.0.1:8083                      | 不允许为空,格式按照 {ip1:port1}[,{ip2:port2}][,{ip3:port3}] 格式进行配置                      |
+| manager.auth.secretId                                   | 登录所需帐号                      | 无                                    | 允许为空不填写                                                                        |
+| manager.auth.secretKey                                  | 登录所需密码                      | 无                                    | 允许为空不填写                                                                        |
+| proxy.cluster.tag                                       | dataproxy 所处的集群 Tag 名       | default_cluster                      | 一个集群 Tag 里可以包含多个 dataproxy、MQ 的集群                                              |
+| proxy.cluster.name                                      | dataproxy 所处的集群名            | default_dataproxy                    | 用于区分不同的环境                                                                      |
+| proxy.cluster.inCharges                                 | dataproxy 所处的集群负责人          | admin                                |                                                                                |
+| configCheckInterval                                     | conf 目录的配置文件检查及加载间隔(单位 ms ) | 10000                          |                                                                                |
+| metricDomains                                           | JMX 域名                      | DataProxy                             | 通过该配置值获取如下 “metricDomains.${metricDomains}.xxx” 的配置项                           |
+| metricDomains.DataProxy.domainListeners                 | 指标监听的类,通过该类名反射开启服务          | org.apache.inlong.dataproxy.metrics.prometheus.PrometheusMetricListener    | 如果存在多个指标监听类配置,通过空格、回车,或换行符进行分隔                                                 |
+| metricDomains.DataProxy.snapshotInterval                | 周期性上报指标的时隙(单位 ms )          | 60000                             |                                                                                |
+| prometheusHttpPort                                      | 使用 Prometheus 上报时设置的端口号     | 9081                               |                                                                                |
+| audit.enable                                            | 是否开启数据上报 InLong-Audit 服务    | true                                |                                                                                |
+| audit.proxys                                            | InLong-Audit 服务的地址          | 127.0.0.1:10081                      | 格式按照 {ip1:port1}[ {ip2:port2}][ {ip3:port3}] 进行配置,不同 ip:port 间通过空格、回车,或换行符进行分隔 |
+
+
+## 日志输出配置( log4j2.xml )
+DataProxy 采用的是 Log4j2 输出日志,相关配置基于 Log4j2 进行设置,本配置只提及常用设置项:
+
+| 参数                      | 描述                    | 默认值   | 备注              |
+|-------------------------|-----------------------|-------|-----------------|
+| basePath      | 日志文件的存储路径             | ${sys:dataproxy.log.path}    |                 |
+| every_file_size      | 每个日志文件的大小,单位字节        | 1G    |                 |
+| output_log_level | 日志输出级别                | DEBUG | 线上使用时建议设置为 INFO |
+| rolling_max    | 同类型日志同一目录下可存放的数量      | 50    |                 |
+| debug_max    | DEBUG 类型日志同一目录下可存放的数量 | 7     |                 |
+| info_max    | INFO 类型日志同一目录下可存放的数量  | 7     |                 |
+| warn_max    | WARN 类型日志同一目录下可存放的数量  | 7     |                 |
+| error_max    | ERROR 类型日志同一目录下可存放的数量 | 7     |                 |
+
+
+## 其他自动更新配置
+DataProxy 启动并成功链接到 Manager 后,会自动从 Manager 同步运行配置,并定期更新,如下部分配置不可修改
+
+| 配置文件名            | 描述                         |  备注             |
+|------------------|----------------------------|-----------------|
+| mq_cluster.properties         | MQ 集群配置                    |     |
+| topics.properties  | groupId 到 MQ 的 Topic 的映射配置 |     |
+| weight.properties | 系统负载权重配置                   |  |
+
+
+
+## source-channel-sink 管道配置(dataproxy-{tube|pulsar}.conf)
+
+DataProxy 支持配置化的 source-channel-sink,配置方式与 flume 的配置文件结构一致,修改该配置文件时要按照 Apache flume 的配置文件定义来进行修改。配置文件放在 dataproxy-{tube|pulsar}.conf 文件中,目前支持 dataproxy-pulsar.conf 和 dataproxy-tube.conf 两种,用于区分不同的中间件类型,具体类型可以在启动时指定,默认(未指定时)使用 dataproxy-pulsar.conf 作为配置文件。 如下是针对该配置文件的示例:
+
+- Source 配置示例:
+
+```shell
+agent1.sources.tcp-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-more1 ch-more2 ch-more3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-transfer ch-back
+定义source中使用到的channel,注意此source下面的配置如果有使用到channel,均需要在此注释
+
+agent1.sources.tcp-source.type = org.apache.flume.source.SimpleTcpSource
+tcp解析类型定义,这里提供类名用于实例化,SimpleTcpSource主要是初始化配置并启动端口监听
+
+agent1.sources.tcp-source.msg-factory-name = org.apache.flume.source.ServerMessageFactory
+用于构造消息解析的handler,并设置read stream handler和write stream handler
+
+agent1.sources.tcp-source.host = 0.0.0.0    
+tcp ip绑定监听,默认绑定所有网卡
+
+agent1.sources.tcp-source.port = 46801
+tcp 端口绑定,默认绑定46801端口
+
+agent1.sources.tcp-source.highWaterMark=2621440 
+netty概念,设置netty高水位值
+
+agent1.sources.tcp-source.max-msg-length = 524288
+限制单个包大小,这里如果传输的是压缩包,则是压缩包大小,限制512KB
+
+agent1.sources.tcp-source.topic = test_token
+默认topic值,如果groupId和topic的映射关系找不到,则发送到此topic中
+
+agent1.sources.tcp-source.attr = m=9
+默认m值设置,这里的m值是inlong内部TdMsg协议的版本
+
+agent1.sources.tcp-source.connections = 5000
+并发连接上线,超过上限值时会对新连接做断链处理
+
+agent1.sources.tcp-source.max-threads = 64
+netty线程池工作线程上限,一般推荐选择cpu的两倍
+
+agent1.sources.tcp-source.receiveBufferSize = 524288
+netty server tcp调优参数
+
+agent1.sources.tcp-source.sendBufferSize = 524288
+netty server tcp调优参数
+
+agent1.sources.tcp-source.custom-cp = true
+是否使用自研的channel process,自研channel process可在主channel阻塞时,选择备用channel发送
+
+agent1.sources.tcp-source.selector.type = org.apache.flume.channel.FailoverChannelSelector
+这个channel selector就是自研的channel selector,和官网的差别不大,主要是有channel主从选择逻辑
+
+agent1.sources.tcp-source.selector.master = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
+指定master channel,这些channel会被优先选择用于数据推送。那些不在master、transfer、fileMetric、slaMetric配置项里的channel,但在
+channels里面有定义的channel,统归为slave channel,当master channel都被占满时,就会选择使用slave channel,slave channel一般建议使用file channel类型
+
+agent1.sources.tcp-source.selector.transfer = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
+指定transfer channel,承接transfer类型的数据,这里的transfer一般是指推送到非tube集群的数据,仅做转发,这里预留出来供后续功能使用
+
+agent1.sources.tcp-source.selector.fileMetric = ch-back
+指定fileMetric channel,用于接收agent上报的指标数据
+```
+
+- Channel 配置示例,memory channel:
+
+```shell
+agent1.channels.ch-more1.type = memory
+memory channel类型
+
+agent1.channels.ch-more1.capacity = 10000000
+memory channel 队列大小,可缓存最大消息条数
+
+agent1.channels.ch-more1.keep-alive = 0
+
+agent1.channels.ch-more1.transactionCapacity = 20
+原子操作时批量处理最大条数,memory channel使用时需要用到加锁,因此会有批处理流程增加效率
+```
+
+- Channel 配置示例,file channel:
+
+```shell
+agent1.channels.ch-msg5.type = file
+file channel类型
+
+agent1.channels.ch-msg5.capacity = 100000000
+file channel最大可缓存消息条数
+
+agent1.channels.ch-msg5.maxFileSize = 1073741824
+file channel文件最大上限,字节数
+
+agent1.channels.ch-msg5.minimumRequiredSpace = 1073741824
+file channel所在磁盘最小可用空间,设置此值可以防止磁盘写满
+
+agent1.channels.ch-msg5.checkpointDir = /data/work/file/ch-msg5/check
+file channel checkpoint路径
+
+agent1.channels.ch-msg5.dataDirs = /data/work/file/ch-msg5/data
+file channel数据路径
+
+agent1.channels.ch-msg5.fsyncPerTransaction = false
+是否对每个原子操作做同步磁盘,建议改false,否则会对性能有影响
+
+agent1.channels.ch-msg5.fsyncInterval = 5
+数据从内存flush到磁盘的时间间隔,单位秒
+```
+
+- Sink 配置示例:
+
+```shell
+agent1.sinks.meta-sink-more1.channel = ch-msg1
+sink的上游channel名称
+
+agent1.sinks.meta-sink-more1.type = org.apache.flume.sink.MetaSink
+sink类实现,此处实现消息向tube集群推送数据
+
+agent1.sinks.meta-sink-more1.master-host-port-list = 
+tube集群master节点列表
+
+agent1.sinks.meta-sink-more1.send_timeout = 30000
+发送到tube时超时时间限制
+
+agent1.sinks.meta-sink-more1.stat-interval-sec = 60
+sink指标统计间隔时间,单位秒
+
+agent1.sinks.meta-sink-more1.thread-num = 8
+Sink类发送消息的工作线程,8表示启动8个并发线程
+
+agent1.sinks.meta-sink-more1.client-id-cache = true
+agent id缓存,用于检查agent上报数据去重
+
+agent1.sinks.meta-sink-more1.max-survived-time = 300000
+缓存最大时间
+
+agent1.sinks.meta-sink-more1.max-survived-size = 3000000
+缓存最大个数
+```
+    
\ No newline at end of file
diff --git a/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/overview.md b/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/overview.md
index 1917508d9e..539f96d2fe 100644
--- a/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/overview.md
+++ b/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/overview.md
@@ -1,5 +1,6 @@
 ---
 title: 总览
+sidebar_position: 1
 ---
 
 InLong DataProxy 主要有连接收敛、路由、数据压缩和协议转换等作用。DataProxy 充当了 InLong 采集端到消息队列的桥梁,
@@ -16,143 +17,6 @@ InLong DataProxy 整体架构基于 Apache Flume,扩展了 Source 层和 Sink
 - channel层的数据会通过sink层做转发,这里主要是将数据转为TDMsg1的格式,并推送到cache层(这里用的比较多的是tube)
 
 
-## DataProxy功能配置说明
-
-DataProxy 支持配置化的 source-channel-sink,配置方式与 flume 的配置文件结构类似。配置文件放在 dataproxy-*.conf 文件中,目前支持 dataproxy-pulsar.conf 和 dataproxy-tube.conf 两种,用于区分不同的中间件类型,具体类型可以在启动时指定,默认(未指定时)使用 dataproxy-pulsar.conf 作为配置文件
-
-- Source配置示例:
-
-```shell
-agent1.sources.tcp-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-more1 ch-more2 ch-more3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-transfer ch-back
-定义source中使用到的channel,注意此source下面的配置如果有使用到channel,均需要在此注释
-
-agent1.sources.tcp-source.type = org.apache.flume.source.SimpleTcpSource
-tcp解析类型定义,这里提供类名用于实例化,SimpleTcpSource主要是初始化配置并启动端口监听
-
-agent1.sources.tcp-source.msg-factory-name = org.apache.flume.source.ServerMessageFactory
-用于构造消息解析的handler,并设置read stream handler和write stream handler
-
-agent1.sources.tcp-source.host = 0.0.0.0    
-tcp ip绑定监听,默认绑定所有网卡
-
-agent1.sources.tcp-source.port = 46801
-tcp 端口绑定,默认绑定46801端口
-
-agent1.sources.tcp-source.highWaterMark=2621440 
-netty概念,设置netty高水位值
-
-agent1.sources.tcp-source.max-msg-length = 524288
-限制单个包大小,这里如果传输的是压缩包,则是压缩包大小,限制512KB
-
-agent1.sources.tcp-source.topic = test_token
-默认topic值,如果groupId和topic的映射关系找不到,则发送到此topic中
-
-agent1.sources.tcp-source.attr = m=9
-默认m值设置,这里的m值是inlong内部TdMsg协议的版本
-
-agent1.sources.tcp-source.connections = 5000
-并发连接上线,超过上限值时会对新连接做断链处理
-
-agent1.sources.tcp-source.max-threads = 64
-netty线程池工作线程上限,一般推荐选择cpu的两倍
-
-agent1.sources.tcp-source.receiveBufferSize = 524288
-netty server tcp调优参数
-
-agent1.sources.tcp-source.sendBufferSize = 524288
-netty server tcp调优参数
-
-agent1.sources.tcp-source.custom-cp = true
-是否使用自研的channel process,自研channel process可在主channel阻塞时,选择备用channel发送
-
-agent1.sources.tcp-source.selector.type = org.apache.flume.channel.FailoverChannelSelector
-这个channel selector就是自研的channel selector,和官网的差别不大,主要是有channel主从选择逻辑
-
-agent1.sources.tcp-source.selector.master = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
-指定master channel,这些channel会被优先选择用于数据推送。那些不在master、transfer、fileMetric、slaMetric配置项里的channel,但在
-channels里面有定义的channel,统归为slave channel,当master channel都被占满时,就会选择使用slave channel,slave channel一般建议使用file channel类型
-
-agent1.sources.tcp-source.selector.transfer = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9
-指定transfer channel,承接transfer类型的数据,这里的transfer一般是指推送到非tube集群的数据,仅做转发,这里预留出来供后续功能使用
-
-agent1.sources.tcp-source.selector.fileMetric = ch-back
-指定fileMetric channel,用于接收agent上报的指标数据
-```
-
-- Channel配置示例,memory channel:
-
-```shell
-agent1.channels.ch-more1.type = memory
-memory channel类型
-
-agent1.channels.ch-more1.capacity = 10000000
-memory channel 队列大小,可缓存最大消息条数
-
-agent1.channels.ch-more1.keep-alive = 0
-
-agent1.channels.ch-more1.transactionCapacity = 20
-原子操作时批量处理最大条数,memory channel使用时需要用到加锁,因此会有批处理流程增加效率
-```
-
-- Channel配置示例,file channel:
-
-```shell
-agent1.channels.ch-msg5.type = file
-file channel类型
-
-agent1.channels.ch-msg5.capacity = 100000000
-file channel最大可缓存消息条数
-
-agent1.channels.ch-msg5.maxFileSize = 1073741824
-file channel文件最大上限,字节数
-
-agent1.channels.ch-msg5.minimumRequiredSpace = 1073741824
-file channel所在磁盘最小可用空间,设置此值可以防止磁盘写满
-
-agent1.channels.ch-msg5.checkpointDir = /data/work/file/ch-msg5/check
-file channel checkpoint路径
-
-agent1.channels.ch-msg5.dataDirs = /data/work/file/ch-msg5/data
-file channel数据路径
-
-agent1.channels.ch-msg5.fsyncPerTransaction = false
-是否对每个原子操作做同步磁盘,建议改false,否则会对性能有影响
-
-agent1.channels.ch-msg5.fsyncInterval = 5
-数据从内存flush到磁盘的时间间隔,单位秒
-```
-
-- Sink配置示例:
-
-```shell
-agent1.sinks.meta-sink-more1.channel = ch-msg1
-sink的上游channel名称
-
-agent1.sinks.meta-sink-more1.type = org.apache.flume.sink.MetaSink
-sink类实现,此处实现消息向tube集群推送数据
-
-agent1.sinks.meta-sink-more1.master-host-port-list = 
-tube集群master节点列表
-
-agent1.sinks.meta-sink-more1.send_timeout = 30000
-发送到tube时超时时间限制
-
-agent1.sinks.meta-sink-more1.stat-interval-sec = 60
-sink指标统计间隔时间,单位秒
-
-agent1.sinks.meta-sink-more1.thread-num = 8
-Sink类发送消息的工作线程,8表示启动8个并发线程
-
-agent1.sinks.meta-sink-more1.client-id-cache = true
-agent id缓存,用于检查agent上报数据去重
-
-agent1.sinks.meta-sink-more1.max-survived-time = 300000
-缓存最大时间
-
-agent1.sinks.meta-sink-more1.max-survived-size = 3000000
-缓存最大个数
-```
-    
 ## 监控指标配置
 
   DataProxy提供了JMX方式的监控指标Listener能力,用户可以实现MetricListener接口,注册后可以定期接收监控指标,用户选择将指标上报自定义的监控系统。Source和Sink模块可以通过将指标数据统计到org.apache.inlong.commons.config.metrics.MetricItemSet的子类中,并注册到MBeanServer。用户自定义的MetricListener通过JMX方式收集指标数据并上报到外部监控系统
diff --git a/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/quick_start.md b/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/quick_start.md
index aeef58eff9..a0731a10f1 100644
--- a/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/quick_start.md
+++ b/i18n/zh-CN/docusaurus-plugin-content-docs/current/modules/dataproxy/quick_start.md
@@ -1,5 +1,6 @@
 ---
 title: 安装部署
+sidebar_position: 2
 ---
 
 所有的安装文件都在 `inlong-dataproxy` 目录下。