This is an automated email from the ASF dual-hosted git repository.
szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new bf1303354 RATIS-2106. Add configuration reference for RaftClient
(#1107)
bf1303354 is described below
commit bf13033547dcb4a27938f9e5f0a923de47a59f45
Author: Siyao Meng <[email protected]>
AuthorDate: Tue Jun 4 19:57:39 2024 -0700
RATIS-2106. Add configuration reference for RaftClient (#1107)
---
.../apache/ratis/client/RaftClientConfigKeys.java | 7 ++-
ratis-docs/src/site/markdown/configurations.md | 63 +++++++++++++++++++++-
2 files changed, 65 insertions(+), 5 deletions(-)
diff --git
a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
index 7360a9cad..925324c21 100644
---
a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
+++
b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
@@ -42,7 +42,7 @@ public interface RaftClientConfigKeys {
String PREFIX = RaftClientConfigKeys.PREFIX + ".rpc";
String REQUEST_TIMEOUT_KEY = PREFIX + ".request.timeout";
- TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(3000,
TimeUnit.MILLISECONDS);
+ TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(3,
TimeUnit.SECONDS);
static TimeDuration requestTimeout(RaftProperties properties) {
return
getTimeDuration(properties.getTimeDuration(REQUEST_TIMEOUT_DEFAULT.getUnit()),
REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT, getDefaultLog());
@@ -52,8 +52,7 @@ public interface RaftClientConfigKeys {
}
String WATCH_REQUEST_TIMEOUT_KEY = PREFIX + ".watch.request.timeout";
- TimeDuration WATCH_REQUEST_TIMEOUT_DEFAULT =
- TimeDuration.valueOf(10000, TimeUnit.MILLISECONDS);
+ TimeDuration WATCH_REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(10,
TimeUnit.SECONDS);
static TimeDuration watchRequestTimeout(RaftProperties properties) {
return
getTimeDuration(properties.getTimeDuration(WATCH_REQUEST_TIMEOUT_DEFAULT.getUnit()),
WATCH_REQUEST_TIMEOUT_KEY, WATCH_REQUEST_TIMEOUT_DEFAULT,
getDefaultLog());
@@ -125,7 +124,7 @@ public interface RaftClientConfigKeys {
}
String REQUEST_TIMEOUT_KEY = PREFIX + ".request.timeout";
- TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(10000,
TimeUnit.MILLISECONDS);
+ TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(10,
TimeUnit.SECONDS);
static TimeDuration requestTimeout(RaftProperties properties) {
return
getTimeDuration(properties.getTimeDuration(REQUEST_TIMEOUT_DEFAULT.getUnit()),
REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT, getDefaultLog());
diff --git a/ratis-docs/src/site/markdown/configurations.md
b/ratis-docs/src/site/markdown/configurations.md
index 95505fd54..71eae7d3d 100644
--- a/ratis-docs/src/site/markdown/configurations.md
+++ b/ratis-docs/src/site/markdown/configurations.md
@@ -675,4 +675,65 @@ For examples,
2. However, setConf to a 3-member group by removing 2 of members and adding
2 new members is a majority-add.
- Note also that adding 1 new member to an 1-member group is always allowed,
- although it is a majority-add.
\ No newline at end of file
+ although it is a majority-add.
+
+
+## Client Configurations
+
+Client configurations are located at `RaftClientConfigKeys`.
+
+### RPC - Configurations related to Client RPC timeout.
+
+| **Property** | `raft.client.rpc.request.timeout` |
+|:----------------|:------------------------------------------|
+| **Description** | client side timeout for sending a request |
+| **Type** | TimeDuration |
+| **Default** | 3s |
+
+| **Property** | `raft.client.rpc.watch.request.timeout` |
+|:----------------|:------------------------------------------------|
+| **Description** | client side timeout for sending a watch request |
+| **Type** | TimeDuration |
+| **Default** | 10s |
+
+### Async - Configurations related to async requests.
+
+| **Property** | `raft.client.async.outstanding-requests.max` |
+|:----------------|:---------------------------------------------|
+| **Description** | maximum number of outstanding async requests |
+| **Type** | int |
+| **Default** | 100 |
+
+### DataStream - Configurations related to DataStream Api.
+
+| **Property** | `raft.client.data-stream.outstanding-requests.max` |
+|:----------------|:---------------------------------------------------|
+| **Description** | maximum number of outstanding data stream requests |
+| **Type** | int |
+| **Default** | 100 |
+
+| **Property** | `raft.client.data-stream.flush.request.count.min`
|
+|:----------------|:-----------------------------------------------------------------|
+| **Description** | minimum number of requests before data stream flush would
happen |
+| **Type** | int
|
+| **Default** | 0
|
+
+| **Property** | `raft.client.data-stream.flush.request.bytes.min`
|
+|:----------------|:--------------------------------------------------------------|
+| **Description** | minimum number of bytes before data stream flush would
happen |
+| **Type** | SizeInBytes
|
+| **Default** | 1MB
|
+
+| **Property** | `raft.client.data-stream.request.timeout` |
+|:----------------|:------------------------------------------|
+| **Description** | timeout for data stream request |
+| **Type** | TimeDuration |
+| **Default** | 10s |
+
+### MessageStream - Configurations related to MessageStream Api.
+
+| **Property** | `raft.client.message-stream.submessage-size` |
+|:----------------|:---------------------------------------------|
+| **Description** | maximum size of a sub message |
+| **Type** | SizeInBytes |
+| **Default** | 1MB |