You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@brpc.apache.org by GitBox <gi...@apache.org> on 2022/11/29 03:54:16 UTC

[GitHub] [incubator-brpc] yanglimingcn opened a new pull request, #2027: add timeout concurrency limiter

yanglimingcn opened a new pull request, #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027

   #2013


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] yanglimingcn commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
yanglimingcn commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037875128


##########
src/brpc/policy/timeout_concurrency_limiter.cpp:
##########
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "brpc/policy/timeout_concurrency_limiter.h"
+#include "brpc/controller.h"
+#include "brpc/errno.pb.h"
+#include <cmath>
+#include <gflags/gflags.h>
+
+namespace brpc {
+namespace policy {
+
+DEFINE_int32(timeout_cl_sample_window_size_ms, 1000,
+             "Duration of the sampling window.");
+DEFINE_int32(timeout_cl_min_sample_count, 100,
+             "During the duration of the sampling window, if the number of "
+             "requests collected is less than this value, the sampling window "
+             "will be discarded.");
+DEFINE_int32(timeout_cl_max_sample_count, 200,
+             "During the duration of the sampling window, once the number of "
+             "requests collected is greater than this value, even if the "
+             "duration of the window has not ended, the max_concurrency will "
+             "be updated and a new sampling window will be started.");
+DEFINE_double(timeout_cl_sampling_interval_ms, 0.1,
+              "Interval for sampling request in auto concurrency limiter");
+DEFINE_int32(timeout_cl_initial_avg_latency_us, 500,
+             "Initial max concurrency for gradient concurrency limiter");
+DEFINE_bool(
+    timeout_cl_enable_error_punish, true,
+    "Whether to consider failed requests when calculating maximum concurrency");
+DEFINE_double(
+    timeout_cl_fail_punish_ratio, 1.0,
+    "Use the failed requests to punish normal requests. The larger "
+    "the configuration item, the more aggressive the penalty strategy.");
+DEFINE_int32(timeout_cl_default_timeout_ms, 500,
+             "Default timeout for rpc request");
+
+TimeoutConcurrencyLimiter::TimeoutConcurrencyLimiter()
+    : _avg_latency_us(FLAGS_timeout_cl_initial_avg_latency_us),
+      _last_sampling_time_us(0) {}
+
+TimeoutConcurrencyLimiter *TimeoutConcurrencyLimiter::New(
+    const AdaptiveMaxConcurrency &) const {
+    return new (std::nothrow) TimeoutConcurrencyLimiter;
+}
+
+bool TimeoutConcurrencyLimiter::OnRequested(int current_concurrency,
+                                            Controller *cntl) {
+    auto timeout_ms = FLAGS_timeout_cl_default_timeout_ms;
+    if (cntl != nullptr && cntl->timeout_ms() != UNSET_MAGIC_NUM) {
+        timeout_ms = cntl->timeout_ms();
+    }
+    return current_concurrency * _avg_latency_us < timeout_ms * 1000;

Review Comment:
   嗯,我把current_concurrency当做了队列,认为再下一个请求的处理时间是current_concurrency * _avg_latency_us,你这么一说我想了一下,其实这个_avg_latency_us包含了排队时间了,应该是这么理解吧。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] serverglen commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
serverglen commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1035447279


##########
src/brpc/concurrency_limiter.h:
##########
@@ -33,7 +33,7 @@ class ConcurrencyLimiter {
     // false when the concurrency reaches the upper limit, otherwise it 
     // returns true. Normally, when OnRequested returns false, you should 
     // return an ELIMIT error directly.
-    virtual bool OnRequested(int current_concurrency) = 0;
+    virtual bool OnRequested(int current_concurrency, int32_t timeout_ms) = 0;

Review Comment:
   传递Controller*会不会通用点?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm commented on pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm commented on PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#issuecomment-1338811947

   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037923288


##########
src/brpc/policy/timeout_concurrency_limiter.cpp:
##########
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "brpc/policy/timeout_concurrency_limiter.h"
+#include "brpc/controller.h"
+#include "brpc/errno.pb.h"
+#include <cmath>
+#include <gflags/gflags.h>
+
+namespace brpc {
+namespace policy {
+
+DEFINE_int32(timeout_cl_sample_window_size_ms, 1000,
+             "Duration of the sampling window.");
+DEFINE_int32(timeout_cl_min_sample_count, 100,
+             "During the duration of the sampling window, if the number of "
+             "requests collected is less than this value, the sampling window "
+             "will be discarded.");
+DEFINE_int32(timeout_cl_max_sample_count, 200,
+             "During the duration of the sampling window, once the number of "
+             "requests collected is greater than this value, even if the "
+             "duration of the window has not ended, the max_concurrency will "
+             "be updated and a new sampling window will be started.");
+DEFINE_double(timeout_cl_sampling_interval_ms, 0.1,
+              "Interval for sampling request in auto concurrency limiter");
+DEFINE_int32(timeout_cl_initial_avg_latency_us, 500,
+             "Initial max concurrency for gradient concurrency limiter");
+DEFINE_bool(
+    timeout_cl_enable_error_punish, true,
+    "Whether to consider failed requests when calculating maximum concurrency");
+DEFINE_double(
+    timeout_cl_fail_punish_ratio, 1.0,
+    "Use the failed requests to punish normal requests. The larger "
+    "the configuration item, the more aggressive the penalty strategy.");
+DEFINE_int32(timeout_cl_default_timeout_ms, 500,
+             "Default timeout for rpc request");
+
+TimeoutConcurrencyLimiter::TimeoutConcurrencyLimiter()
+    : _avg_latency_us(FLAGS_timeout_cl_initial_avg_latency_us),
+      _last_sampling_time_us(0) {}
+
+TimeoutConcurrencyLimiter *TimeoutConcurrencyLimiter::New(
+    const AdaptiveMaxConcurrency &) const {
+    return new (std::nothrow) TimeoutConcurrencyLimiter;
+}
+
+bool TimeoutConcurrencyLimiter::OnRequested(int current_concurrency,
+                                            Controller *cntl) {
+    auto timeout_ms = FLAGS_timeout_cl_default_timeout_ms;
+    if (cntl != nullptr && cntl->timeout_ms() != UNSET_MAGIC_NUM) {
+        timeout_ms = cntl->timeout_ms();

Review Comment:
   哦,如果是不同method分别统计avg_latency的话,那这里用请求级别timeout是有意义的。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037831763


##########
src/brpc/policy/timeout_concurrency_limiter.cpp:
##########
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "brpc/policy/timeout_concurrency_limiter.h"
+#include "brpc/controller.h"
+#include "brpc/errno.pb.h"
+#include <cmath>
+#include <gflags/gflags.h>
+
+namespace brpc {
+namespace policy {
+
+DEFINE_int32(timeout_cl_sample_window_size_ms, 1000,
+             "Duration of the sampling window.");
+DEFINE_int32(timeout_cl_min_sample_count, 100,
+             "During the duration of the sampling window, if the number of "
+             "requests collected is less than this value, the sampling window "
+             "will be discarded.");
+DEFINE_int32(timeout_cl_max_sample_count, 200,
+             "During the duration of the sampling window, once the number of "
+             "requests collected is greater than this value, even if the "
+             "duration of the window has not ended, the max_concurrency will "
+             "be updated and a new sampling window will be started.");
+DEFINE_double(timeout_cl_sampling_interval_ms, 0.1,
+              "Interval for sampling request in auto concurrency limiter");
+DEFINE_int32(timeout_cl_initial_avg_latency_us, 500,
+             "Initial max concurrency for gradient concurrency limiter");
+DEFINE_bool(
+    timeout_cl_enable_error_punish, true,
+    "Whether to consider failed requests when calculating maximum concurrency");
+DEFINE_double(
+    timeout_cl_fail_punish_ratio, 1.0,
+    "Use the failed requests to punish normal requests. The larger "
+    "the configuration item, the more aggressive the penalty strategy.");
+DEFINE_int32(timeout_cl_default_timeout_ms, 500,
+             "Default timeout for rpc request");
+
+TimeoutConcurrencyLimiter::TimeoutConcurrencyLimiter()
+    : _avg_latency_us(FLAGS_timeout_cl_initial_avg_latency_us),
+      _last_sampling_time_us(0) {}
+
+TimeoutConcurrencyLimiter *TimeoutConcurrencyLimiter::New(
+    const AdaptiveMaxConcurrency &) const {
+    return new (std::nothrow) TimeoutConcurrencyLimiter;
+}
+
+bool TimeoutConcurrencyLimiter::OnRequested(int current_concurrency,
+                                            Controller *cntl) {
+    auto timeout_ms = FLAGS_timeout_cl_default_timeout_ms;
+    if (cntl != nullptr && cntl->timeout_ms() != UNSET_MAGIC_NUM) {
+        timeout_ms = cntl->timeout_ms();
+    }
+    return current_concurrency * _avg_latency_us < timeout_ms * 1000;

Review Comment:
   这里好像不需要乘以current_concurrency,我理解_avg_latency_us < timeout_ms * 1000就可以了。就是说服务过载时平响肯定会增大,增大到阈值就开始限流就可以了。



##########
src/brpc/policy/timeout_concurrency_limiter.cpp:
##########
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "brpc/policy/timeout_concurrency_limiter.h"
+#include "brpc/controller.h"
+#include "brpc/errno.pb.h"
+#include <cmath>
+#include <gflags/gflags.h>
+
+namespace brpc {
+namespace policy {
+
+DEFINE_int32(timeout_cl_sample_window_size_ms, 1000,
+             "Duration of the sampling window.");
+DEFINE_int32(timeout_cl_min_sample_count, 100,
+             "During the duration of the sampling window, if the number of "
+             "requests collected is less than this value, the sampling window "
+             "will be discarded.");
+DEFINE_int32(timeout_cl_max_sample_count, 200,
+             "During the duration of the sampling window, once the number of "
+             "requests collected is greater than this value, even if the "
+             "duration of the window has not ended, the max_concurrency will "
+             "be updated and a new sampling window will be started.");
+DEFINE_double(timeout_cl_sampling_interval_ms, 0.1,
+              "Interval for sampling request in auto concurrency limiter");
+DEFINE_int32(timeout_cl_initial_avg_latency_us, 500,
+             "Initial max concurrency for gradient concurrency limiter");
+DEFINE_bool(
+    timeout_cl_enable_error_punish, true,
+    "Whether to consider failed requests when calculating maximum concurrency");
+DEFINE_double(
+    timeout_cl_fail_punish_ratio, 1.0,
+    "Use the failed requests to punish normal requests. The larger "
+    "the configuration item, the more aggressive the penalty strategy.");
+DEFINE_int32(timeout_cl_default_timeout_ms, 500,
+             "Default timeout for rpc request");
+
+TimeoutConcurrencyLimiter::TimeoutConcurrencyLimiter()
+    : _avg_latency_us(FLAGS_timeout_cl_initial_avg_latency_us),
+      _last_sampling_time_us(0) {}
+
+TimeoutConcurrencyLimiter *TimeoutConcurrencyLimiter::New(
+    const AdaptiveMaxConcurrency &) const {
+    return new (std::nothrow) TimeoutConcurrencyLimiter;
+}
+
+bool TimeoutConcurrencyLimiter::OnRequested(int current_concurrency,
+                                            Controller *cntl) {
+    auto timeout_ms = FLAGS_timeout_cl_default_timeout_ms;
+    if (cntl != nullptr && cntl->timeout_ms() != UNSET_MAGIC_NUM) {
+        timeout_ms = cntl->timeout_ms();

Review Comment:
   感觉这里使用Controller的timeout_ms不太合适,那个是请求级别的,每个请求设置的timeout可能不一样,比如有的请求timeout只有10ms,然后这里avg_latency是100ms,就返回了拒绝。另外就是不是所有协议都能从server的Controller拿到client端的timeout_ms。
   可以只用FLAGS_timeout_cl_default_timeout_ms这个。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037924057


##########
src/brpc/policy/timeout_concurrency_limiter.cpp:
##########
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "brpc/policy/timeout_concurrency_limiter.h"
+#include "brpc/controller.h"
+#include "brpc/errno.pb.h"
+#include <cmath>
+#include <gflags/gflags.h>
+
+namespace brpc {
+namespace policy {
+
+DEFINE_int32(timeout_cl_sample_window_size_ms, 1000,
+             "Duration of the sampling window.");
+DEFINE_int32(timeout_cl_min_sample_count, 100,
+             "During the duration of the sampling window, if the number of "
+             "requests collected is less than this value, the sampling window "
+             "will be discarded.");
+DEFINE_int32(timeout_cl_max_sample_count, 200,
+             "During the duration of the sampling window, once the number of "
+             "requests collected is greater than this value, even if the "
+             "duration of the window has not ended, the max_concurrency will "
+             "be updated and a new sampling window will be started.");
+DEFINE_double(timeout_cl_sampling_interval_ms, 0.1,
+              "Interval for sampling request in auto concurrency limiter");
+DEFINE_int32(timeout_cl_initial_avg_latency_us, 500,
+             "Initial max concurrency for gradient concurrency limiter");
+DEFINE_bool(
+    timeout_cl_enable_error_punish, true,
+    "Whether to consider failed requests when calculating maximum concurrency");
+DEFINE_double(
+    timeout_cl_fail_punish_ratio, 1.0,
+    "Use the failed requests to punish normal requests. The larger "
+    "the configuration item, the more aggressive the penalty strategy.");
+DEFINE_int32(timeout_cl_default_timeout_ms, 500,
+             "Default timeout for rpc request");
+
+TimeoutConcurrencyLimiter::TimeoutConcurrencyLimiter()
+    : _avg_latency_us(FLAGS_timeout_cl_initial_avg_latency_us),
+      _last_sampling_time_us(0) {}
+
+TimeoutConcurrencyLimiter *TimeoutConcurrencyLimiter::New(
+    const AdaptiveMaxConcurrency &) const {
+    return new (std::nothrow) TimeoutConcurrencyLimiter;
+}
+
+bool TimeoutConcurrencyLimiter::OnRequested(int current_concurrency,
+                                            Controller *cntl) {
+    auto timeout_ms = FLAGS_timeout_cl_default_timeout_ms;
+    if (cntl != nullptr && cntl->timeout_ms() != UNSET_MAGIC_NUM) {
+        timeout_ms = cntl->timeout_ms();
+    }
+    return current_concurrency * _avg_latency_us < timeout_ms * 1000;

Review Comment:
   对,_avg_latency_us包含了排队时间



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] serverglen commented on pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
serverglen commented on PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#issuecomment-1371721339

   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037832851


##########
test/brpc_timeout_concurrency_limiter_unittest.cpp:
##########
@@ -0,0 +1,92 @@
+// Copyright (c) 2012 The Chromium Authors. All rights reserved.

Review Comment:
   license不对



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] yanglimingcn commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
yanglimingcn commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037862874


##########
src/brpc/policy/timeout_concurrency_limiter.cpp:
##########
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "brpc/policy/timeout_concurrency_limiter.h"
+#include "brpc/controller.h"
+#include "brpc/errno.pb.h"
+#include <cmath>
+#include <gflags/gflags.h>
+
+namespace brpc {
+namespace policy {
+
+DEFINE_int32(timeout_cl_sample_window_size_ms, 1000,
+             "Duration of the sampling window.");
+DEFINE_int32(timeout_cl_min_sample_count, 100,
+             "During the duration of the sampling window, if the number of "
+             "requests collected is less than this value, the sampling window "
+             "will be discarded.");
+DEFINE_int32(timeout_cl_max_sample_count, 200,
+             "During the duration of the sampling window, once the number of "
+             "requests collected is greater than this value, even if the "
+             "duration of the window has not ended, the max_concurrency will "
+             "be updated and a new sampling window will be started.");
+DEFINE_double(timeout_cl_sampling_interval_ms, 0.1,
+              "Interval for sampling request in auto concurrency limiter");
+DEFINE_int32(timeout_cl_initial_avg_latency_us, 500,
+             "Initial max concurrency for gradient concurrency limiter");
+DEFINE_bool(
+    timeout_cl_enable_error_punish, true,
+    "Whether to consider failed requests when calculating maximum concurrency");
+DEFINE_double(
+    timeout_cl_fail_punish_ratio, 1.0,
+    "Use the failed requests to punish normal requests. The larger "
+    "the configuration item, the more aggressive the penalty strategy.");
+DEFINE_int32(timeout_cl_default_timeout_ms, 500,
+             "Default timeout for rpc request");
+
+TimeoutConcurrencyLimiter::TimeoutConcurrencyLimiter()
+    : _avg_latency_us(FLAGS_timeout_cl_initial_avg_latency_us),
+      _last_sampling_time_us(0) {}
+
+TimeoutConcurrencyLimiter *TimeoutConcurrencyLimiter::New(
+    const AdaptiveMaxConcurrency &) const {
+    return new (std::nothrow) TimeoutConcurrencyLimiter;
+}
+
+bool TimeoutConcurrencyLimiter::OnRequested(int current_concurrency,
+                                            Controller *cntl) {
+    auto timeout_ms = FLAGS_timeout_cl_default_timeout_ms;
+    if (cntl != nullptr && cntl->timeout_ms() != UNSET_MAGIC_NUM) {
+        timeout_ms = cntl->timeout_ms();

Review Comment:
   请求级别的timeout是不是更灵活呢?如果avg_latency是100ms,用户设置的是10ms,确实应该拒绝吧。基于timeout的limiter更适合接口级别的设置,因为一个服务不同接口的延迟差异还是挺大的。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] yanglimingcn commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
yanglimingcn commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1036974970


##########
src/brpc/concurrency_limiter.h:
##########
@@ -33,7 +33,7 @@ class ConcurrencyLimiter {
     // false when the concurrency reaches the upper limit, otherwise it 
     // returns true. Normally, when OnRequested returns false, you should 
     // return an ELIMIT error directly.
-    virtual bool OnRequested(int current_concurrency) = 0;
+    virtual bool OnRequested(int current_concurrency, int32_t timeout_ms) = 0;

Review Comment:
   我觉得Controller*是OK的,@wwbmmm 你的意见是怎么样的,如果没有异议,我改成Controller*。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] yanglimingcn commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
yanglimingcn commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1036974970


##########
src/brpc/concurrency_limiter.h:
##########
@@ -33,7 +33,7 @@ class ConcurrencyLimiter {
     // false when the concurrency reaches the upper limit, otherwise it 
     // returns true. Normally, when OnRequested returns false, you should 
     // return an ELIMIT error directly.
-    virtual bool OnRequested(int current_concurrency) = 0;
+    virtual bool OnRequested(int current_concurrency, int32_t timeout_ms) = 0;

Review Comment:
   @wwbmmm 你的意见是怎么样的,如果没有异议,我改成Controller*。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] yanglimingcn commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
yanglimingcn commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1035456329


##########
src/brpc/concurrency_limiter.h:
##########
@@ -33,7 +33,7 @@ class ConcurrencyLimiter {
     // false when the concurrency reaches the upper limit, otherwise it 
     // returns true. Normally, when OnRequested returns false, you should 
     // return an ELIMIT error directly.
-    virtual bool OnRequested(int current_concurrency) = 0;
+    virtual bool OnRequested(int current_concurrency, int32_t timeout_ms) = 0;

Review Comment:
   是通用一些,但是意思上就没那么明确了,看看大家的意见



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] serverglen commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
serverglen commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1036648918


##########
src/brpc/concurrency_limiter.h:
##########
@@ -33,7 +33,7 @@ class ConcurrencyLimiter {
     // false when the concurrency reaches the upper limit, otherwise it 
     // returns true. Normally, when OnRequested returns false, you should 
     // return an ELIMIT error directly.
-    virtual bool OnRequested(int current_concurrency) = 0;
+    virtual bool OnRequested(int current_concurrency, int32_t timeout_ms) = 0;

Review Comment:
   个人觉得Controller*会好点,后面如果还需要实现其他限流策略,也可以复用Controller中的数据,也方便扩展。



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm merged pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm merged PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org


[GitHub] [incubator-brpc] wwbmmm commented on a diff in pull request #2027: add timeout concurrency limiter

Posted by GitBox <gi...@apache.org>.
wwbmmm commented on code in PR #2027:
URL: https://github.com/apache/incubator-brpc/pull/2027#discussion_r1037833869


##########
src/brpc/concurrency_limiter.h:
##########
@@ -33,7 +33,7 @@ class ConcurrencyLimiter {
     // false when the concurrency reaches the upper limit, otherwise it 
     // returns true. Normally, when OnRequested returns false, you should 
     // return an ELIMIT error directly.
-    virtual bool OnRequested(int current_concurrency) = 0;
+    virtual bool OnRequested(int current_concurrency, int32_t timeout_ms) = 0;

Review Comment:
   > 我觉得Controller_是OK的,@wwbmmm 你的意见是怎么样的,如果没有异议,我改成Controller_。
   
   可以看下我下面的评论,我认为这个地方暂时没必要传入timeout_ms,所以也没必要传入Controller



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@brpc.apache.org
For additional commands, e-mail: dev-help@brpc.apache.org