aboutsummaryrefslogtreecommitdiffhomepage
path: root/test/cpp
diff options
context:
space:
mode:
authorGravatar David Garcia Quintas <dgq@google.com>2017-12-06 16:16:49 -0800
committerGravatar David Garcia Quintas <dgq@google.com>2017-12-06 16:16:49 -0800
commitd1966582797e0547de0f81e032b4bc5617030c79 (patch)
treed60ed7e87bbfb7d0aaa2cee7687e85155e17bf31 /test/cpp
parent8c5c654fa32c3f139babb6a5180c01028e19b631 (diff)
parent95e23594b026eca21151f31b039bd2a51ea67109 (diff)
Merge branch 'master' of github.com:grpc/grpc into removed_expiration_interval
Diffstat (limited to 'test/cpp')
-rw-r--r--test/cpp/client/BUILD51
-rw-r--r--test/cpp/client/client_channel_stress_test.cc327
-rw-r--r--test/cpp/codegen/proto_utils_test.cc112
-rw-r--r--test/cpp/common/auth_property_iterator_test.cc2
-rw-r--r--test/cpp/common/secure_auth_context_test.cc4
-rw-r--r--test/cpp/end2end/async_end2end_test.cc2
-rw-r--r--test/cpp/end2end/client_lb_end2end_test.cc9
-rw-r--r--test/cpp/end2end/end2end_test.cc4
-rw-r--r--test/cpp/end2end/grpclb_end2end_test.cc40
-rw-r--r--test/cpp/grpclb/grpclb_api_test.cc2
-rw-r--r--test/cpp/grpclb/grpclb_test.cc119
-rw-r--r--test/cpp/interop/http2_client.cc2
-rw-r--r--test/cpp/interop/interop_server.cc18
-rw-r--r--test/cpp/interop/server_helper.h20
-rw-r--r--test/cpp/interop/stress_test.cc4
-rw-r--r--test/cpp/microbenchmarks/bm_call_create.cc91
-rw-r--r--test/cpp/microbenchmarks/bm_chttp2_hpack.cc6
-rw-r--r--test/cpp/microbenchmarks/bm_chttp2_transport.cc8
-rw-r--r--test/cpp/microbenchmarks/bm_closure.cc60
-rw-r--r--test/cpp/microbenchmarks/bm_cq.cc31
-rw-r--r--test/cpp/microbenchmarks/bm_cq_multiple_threads.cc8
-rw-r--r--test/cpp/microbenchmarks/bm_error.cc8
-rw-r--r--test/cpp/microbenchmarks/bm_fullstack_trickle.cc6
-rw-r--r--test/cpp/microbenchmarks/bm_metadata.cc18
-rw-r--r--test/cpp/microbenchmarks/bm_pollset.cc11
-rw-r--r--test/cpp/microbenchmarks/fullstack_fixtures.h16
-rw-r--r--test/cpp/microbenchmarks/helpers.h8
-rw-r--r--test/cpp/naming/README.md2
-rwxr-xr-xtest/cpp/naming/create_private_dns_zone.sh4
-rwxr-xr-xtest/cpp/naming/private_dns_zone_init.sh152
-rw-r--r--test/cpp/naming/resolver_component_test.cc23
-rwxr-xr-xtest/cpp/naming/resolver_component_tests_runner.sh26
-rw-r--r--test/cpp/naming/resolver_component_tests_runner_invoker.cc4
-rwxr-xr-xtest/cpp/naming/resolver_gce_integration_tests_runner.sh156
-rw-r--r--test/cpp/naming/resolver_test_record_groups.yaml10
-rw-r--r--test/cpp/performance/writes_per_rpc_test.cc14
-rw-r--r--test/cpp/qps/BUILD2
-rw-r--r--test/cpp/qps/client_async.cc36
-rw-r--r--test/cpp/qps/client_sync.cc212
-rw-r--r--test/cpp/qps/histogram.h36
-rw-r--r--test/cpp/qps/json_run_localhost.cc6
-rw-r--r--test/cpp/qps/qps_interarrival_test.cc10
-rw-r--r--test/cpp/qps/qps_json_driver.cc2
-rw-r--r--test/cpp/qps/qps_worker.cc2
-rw-r--r--test/cpp/qps/server_async.cc2
-rw-r--r--test/cpp/qps/server_sync.cc2
-rw-r--r--test/cpp/thread_manager/thread_manager_test.cc2
-rw-r--r--test/cpp/util/BUILD100
-rw-r--r--test/cpp/util/grpc_tool.cc10
-rw-r--r--test/cpp/util/grpc_tool_test.cc32
50 files changed, 1266 insertions, 566 deletions
diff --git a/test/cpp/client/BUILD b/test/cpp/client/BUILD
new file mode 100644
index 0000000000..12825e88c2
--- /dev/null
+++ b/test/cpp/client/BUILD
@@ -0,0 +1,51 @@
+# Copyright 2017 gRPC authors.
+#
+# Licensed 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.
+
+licenses(["notice"]) # Apache v2
+
+load("//bazel:grpc_build_system.bzl", "grpc_cc_library", "grpc_cc_test", "grpc_package")
+
+grpc_package(name = "test/cpp/client")
+
+grpc_cc_test(
+ name = "credentials_test",
+ srcs = ["credentials_test.cc"],
+ external_deps = [
+ "gtest",
+ ],
+ deps = [
+ "//:gpr",
+ "//:grpc",
+ "//:grpc++",
+ ],
+)
+
+grpc_cc_test(
+ name = "client_channel_stress_test",
+ srcs = ["client_channel_stress_test.cc"],
+ deps = [
+ "//:gpr",
+ "//:grpc",
+ "//:grpc++",
+ "//:grpc_resolver_fake",
+ "//src/proto/grpc/lb/v1:load_balancer_proto",
+ "//src/proto/grpc/testing:echo_messages_proto",
+ "//src/proto/grpc/testing:echo_proto",
+ "//src/proto/grpc/testing/duplicate:echo_duplicate_proto",
+ "//test/core/util:gpr_test_util",
+ "//test/core/util:grpc_test_util",
+ "//test/cpp/end2end:test_service_impl",
+ "//test/cpp/util:test_util",
+ ],
+)
diff --git a/test/cpp/client/client_channel_stress_test.cc b/test/cpp/client/client_channel_stress_test.cc
new file mode 100644
index 0000000000..0954b28df0
--- /dev/null
+++ b/test/cpp/client/client_channel_stress_test.cc
@@ -0,0 +1,327 @@
+/*
+ *
+ * Copyright 2017 gRPC authors.
+ *
+ * Licensed 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 <atomic>
+#include <memory>
+#include <mutex>
+#include <sstream>
+#include <thread>
+
+#include <grpc++/channel.h>
+#include <grpc++/client_context.h>
+#include <grpc++/create_channel.h>
+#include <grpc++/server.h>
+#include <grpc++/server_builder.h>
+#include <grpc/grpc.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/thd.h>
+#include <grpc/support/time.h>
+
+#include "src/core/ext/filters/client_channel/resolver/fake/fake_resolver.h"
+#include "src/core/lib/iomgr/sockaddr.h"
+
+#include "test/core/util/port.h"
+#include "test/core/util/test_config.h"
+#include "test/cpp/end2end/test_service_impl.h"
+
+#include "src/proto/grpc/lb/v1/load_balancer.grpc.pb.h"
+#include "src/proto/grpc/testing/echo.grpc.pb.h"
+
+using grpc::lb::v1::LoadBalanceRequest;
+using grpc::lb::v1::LoadBalanceResponse;
+using grpc::lb::v1::LoadBalancer;
+
+namespace grpc {
+namespace testing {
+namespace {
+
+const size_t kNumBackends = 10;
+const size_t kNumBalancers = 5;
+const size_t kNumClientThreads = 100;
+const int kResolutionUpdateIntervalMs = 50;
+const int kServerlistUpdateIntervalMs = 10;
+const int kTestDurationSec = 30;
+
+using BackendServiceImpl = TestServiceImpl;
+
+class BalancerServiceImpl : public LoadBalancer::Service {
+ public:
+ using Stream = ServerReaderWriter<LoadBalanceResponse, LoadBalanceRequest>;
+
+ explicit BalancerServiceImpl(const std::vector<int>& all_backend_ports)
+ : all_backend_ports_(all_backend_ports) {}
+
+ Status BalanceLoad(ServerContext* context, Stream* stream) override {
+ gpr_log(GPR_INFO, "LB[%p]: Start BalanceLoad.", this);
+ LoadBalanceRequest request;
+ stream->Read(&request);
+ while (!shutdown_) {
+ stream->Write(BuildRandomResponseForBackends());
+ std::this_thread::sleep_for(
+ std::chrono::milliseconds(kServerlistUpdateIntervalMs));
+ }
+ gpr_log(GPR_INFO, "LB[%p]: Finish BalanceLoad.", this);
+ return Status::OK;
+ }
+
+ void Shutdown() { shutdown_ = true; }
+
+ private:
+ grpc::string Ip4ToPackedString(const char* ip_str) {
+ struct in_addr ip4;
+ GPR_ASSERT(inet_pton(AF_INET, ip_str, &ip4) == 1);
+ return grpc::string(reinterpret_cast<const char*>(&ip4), sizeof(ip4));
+ }
+
+ LoadBalanceResponse BuildRandomResponseForBackends() {
+ // Generate a random serverlist with varying size (if N =
+ // all_backend_ports_.size(), num_non_drop_entry is in [0, 2N],
+ // num_drop_entry is in [0, N]), order, duplicate, and drop rate.
+ size_t num_non_drop_entry =
+ std::rand() % (all_backend_ports_.size() * 2 + 1);
+ size_t num_drop_entry = std::rand() % (all_backend_ports_.size() + 1);
+ std::vector<int> random_backend_indices;
+ for (size_t i = 0; i < num_non_drop_entry; ++i) {
+ random_backend_indices.push_back(std::rand() % all_backend_ports_.size());
+ }
+ for (size_t i = 0; i < num_drop_entry; ++i) {
+ random_backend_indices.push_back(-1);
+ }
+ std::random_shuffle(random_backend_indices.begin(),
+ random_backend_indices.end());
+ // Build the response according to the random list generated above.
+ LoadBalanceResponse response;
+ for (int index : random_backend_indices) {
+ auto* server = response.mutable_server_list()->add_servers();
+ if (index < 0) {
+ server->set_drop(true);
+ server->set_load_balance_token("load_balancing");
+ } else {
+ server->set_ip_address(Ip4ToPackedString("127.0.0.1"));
+ server->set_port(all_backend_ports_[index]);
+ }
+ }
+ return response;
+ }
+
+ std::atomic_bool shutdown_{false};
+ const std::vector<int> all_backend_ports_;
+};
+
+class ClientChannelStressTest {
+ public:
+ void Run() {
+ Start();
+ // Keep updating resolution for the test duration.
+ gpr_log(GPR_INFO, "Start updating resolution.");
+ const auto wait_duration =
+ std::chrono::milliseconds(kResolutionUpdateIntervalMs);
+ std::vector<AddressData> addresses;
+ auto start_time = std::chrono::steady_clock::now();
+ while (true) {
+ if (std::chrono::duration_cast<std::chrono::seconds>(
+ std::chrono::steady_clock::now() - start_time)
+ .count() > kTestDurationSec) {
+ break;
+ }
+ // Generate a random subset of balancers.
+ addresses.clear();
+ for (const auto& balancer_server : balancer_servers_) {
+ // Select each address with probability of 0.8.
+ if (std::rand() % 10 < 8) {
+ addresses.emplace_back(AddressData{balancer_server.port_, true, ""});
+ }
+ }
+ std::random_shuffle(addresses.begin(), addresses.end());
+ SetNextResolution(addresses);
+ std::this_thread::sleep_for(wait_duration);
+ }
+ gpr_log(GPR_INFO, "Finish updating resolution.");
+ Shutdown();
+ }
+
+ private:
+ template <typename T>
+ struct ServerThread {
+ explicit ServerThread(const grpc::string& type,
+ const grpc::string& server_host, T* service)
+ : type_(type), service_(service) {
+ std::mutex mu;
+ // We need to acquire the lock here in order to prevent the notify_one
+ // by ServerThread::Start from firing before the wait below is hit.
+ std::unique_lock<std::mutex> lock(mu);
+ port_ = grpc_pick_unused_port_or_die();
+ gpr_log(GPR_INFO, "starting %s server on port %d", type_.c_str(), port_);
+ std::condition_variable cond;
+ thread_.reset(new std::thread(
+ std::bind(&ServerThread::Start, this, server_host, &mu, &cond)));
+ cond.wait(lock);
+ gpr_log(GPR_INFO, "%s server startup complete", type_.c_str());
+ }
+
+ void Start(const grpc::string& server_host, std::mutex* mu,
+ std::condition_variable* cond) {
+ // We need to acquire the lock here in order to prevent the notify_one
+ // below from firing before its corresponding wait is executed.
+ std::lock_guard<std::mutex> lock(*mu);
+ std::ostringstream server_address;
+ server_address << server_host << ":" << port_;
+ ServerBuilder builder;
+ builder.AddListeningPort(server_address.str(),
+ InsecureServerCredentials());
+ builder.RegisterService(service_);
+ server_ = builder.BuildAndStart();
+ cond->notify_one();
+ }
+
+ void Shutdown() {
+ gpr_log(GPR_INFO, "%s about to shutdown", type_.c_str());
+ server_->Shutdown(grpc_timeout_milliseconds_to_deadline(0));
+ thread_->join();
+ gpr_log(GPR_INFO, "%s shutdown completed", type_.c_str());
+ }
+
+ int port_;
+ grpc::string type_;
+ std::unique_ptr<Server> server_;
+ T* service_;
+ std::unique_ptr<std::thread> thread_;
+ };
+
+ struct AddressData {
+ int port;
+ bool is_balancer;
+ grpc::string balancer_name;
+ };
+
+ void SetNextResolution(const std::vector<AddressData>& address_data) {
+ grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
+ grpc_lb_addresses* addresses =
+ grpc_lb_addresses_create(address_data.size(), nullptr);
+ for (size_t i = 0; i < address_data.size(); ++i) {
+ char* lb_uri_str;
+ gpr_asprintf(&lb_uri_str, "ipv4:127.0.0.1:%d", address_data[i].port);
+ grpc_uri* lb_uri = grpc_uri_parse(&exec_ctx, lb_uri_str, true);
+ GPR_ASSERT(lb_uri != nullptr);
+ grpc_lb_addresses_set_address_from_uri(
+ addresses, i, lb_uri, address_data[i].is_balancer,
+ address_data[i].balancer_name.c_str(), nullptr);
+ grpc_uri_destroy(lb_uri);
+ gpr_free(lb_uri_str);
+ }
+ grpc_arg fake_addresses = grpc_lb_addresses_create_channel_arg(addresses);
+ grpc_channel_args fake_result = {1, &fake_addresses};
+ grpc_fake_resolver_response_generator_set_response(
+ &exec_ctx, response_generator_, &fake_result);
+ grpc_lb_addresses_destroy(&exec_ctx, addresses);
+ grpc_exec_ctx_finish(&exec_ctx);
+ }
+
+ void KeepSendingRequests() {
+ gpr_log(GPR_INFO, "Start sending requests.");
+ while (!shutdown_) {
+ ClientContext context;
+ context.set_deadline(grpc_timeout_milliseconds_to_deadline(1000));
+ EchoRequest request;
+ request.set_message("test");
+ EchoResponse response;
+ {
+ std::lock_guard<std::mutex> lock(stub_mutex_);
+ stub_->Echo(&context, request, &response);
+ }
+ }
+ gpr_log(GPR_INFO, "Finish sending requests.");
+ }
+
+ void CreateStub() {
+ ChannelArguments args;
+ response_generator_ = grpc_fake_resolver_response_generator_create();
+ args.SetPointer(GRPC_ARG_FAKE_RESOLVER_RESPONSE_GENERATOR,
+ response_generator_);
+ std::ostringstream uri;
+ uri << "fake:///servername_not_used";
+ channel_ =
+ CreateCustomChannel(uri.str(), InsecureChannelCredentials(), args);
+ stub_ = grpc::testing::EchoTestService::NewStub(channel_);
+ }
+
+ void Start() {
+ // Start the backends.
+ std::vector<int> backend_ports;
+ for (size_t i = 0; i < kNumBackends; ++i) {
+ backends_.emplace_back(new BackendServiceImpl());
+ backend_servers_.emplace_back(ServerThread<BackendServiceImpl>(
+ "backend", server_host_, backends_.back().get()));
+ backend_ports.push_back(backend_servers_.back().port_);
+ }
+ // Start the load balancers.
+ for (size_t i = 0; i < kNumBalancers; ++i) {
+ balancers_.emplace_back(new BalancerServiceImpl(backend_ports));
+ balancer_servers_.emplace_back(ServerThread<BalancerServiceImpl>(
+ "balancer", server_host_, balancers_.back().get()));
+ }
+ // Start sending RPCs in multiple threads.
+ CreateStub();
+ for (size_t i = 0; i < kNumClientThreads; ++i) {
+ client_threads_.emplace_back(
+ std::thread(&ClientChannelStressTest::KeepSendingRequests, this));
+ }
+ }
+
+ void Shutdown() {
+ shutdown_ = true;
+ for (size_t i = 0; i < client_threads_.size(); ++i) {
+ client_threads_[i].join();
+ }
+ for (size_t i = 0; i < balancers_.size(); ++i) {
+ balancers_[i]->Shutdown();
+ balancer_servers_[i].Shutdown();
+ }
+ for (size_t i = 0; i < backends_.size(); ++i) {
+ backend_servers_[i].Shutdown();
+ }
+ grpc_fake_resolver_response_generator_unref(response_generator_);
+ }
+
+ std::atomic_bool shutdown_{false};
+ const grpc::string server_host_ = "localhost";
+ std::shared_ptr<Channel> channel_;
+ std::unique_ptr<grpc::testing::EchoTestService::Stub> stub_;
+ std::mutex stub_mutex_;
+ std::vector<std::unique_ptr<BackendServiceImpl>> backends_;
+ std::vector<std::unique_ptr<BalancerServiceImpl>> balancers_;
+ std::vector<ServerThread<BackendServiceImpl>> backend_servers_;
+ std::vector<ServerThread<BalancerServiceImpl>> balancer_servers_;
+ grpc_fake_resolver_response_generator* response_generator_;
+ std::vector<std::thread> client_threads_;
+};
+
+} // namespace
+} // namespace testing
+} // namespace grpc
+
+int main(int argc, char** argv) {
+ grpc_init();
+ grpc_test_init(argc, argv);
+ grpc::testing::ClientChannelStressTest test;
+ test.Run();
+ grpc_shutdown();
+ return 0;
+}
diff --git a/test/cpp/codegen/proto_utils_test.cc b/test/cpp/codegen/proto_utils_test.cc
index fd05c90e9d..cc355bb24a 100644
--- a/test/cpp/codegen/proto_utils_test.cc
+++ b/test/cpp/codegen/proto_utils_test.cc
@@ -16,15 +16,16 @@
*
*/
+#include <grpc++/impl/codegen/grpc_library.h>
#include <grpc++/impl/codegen/proto_utils.h>
#include <grpc++/impl/grpc_library.h>
+#include <grpc/impl/codegen/byte_buffer.h>
+#include <grpc/slice.h>
#include <gtest/gtest.h>
namespace grpc {
namespace internal {
-static GrpcLibraryInitializer g_gli_initializer;
-
// Provide access to GrpcBufferWriter internals.
class GrpcBufferWriterPeer {
public:
@@ -44,35 +45,120 @@ class ProtoUtilsTest : public ::testing::Test {};
// GrpcBufferWriter Next()/Backup() invocations could result in a dangling
// pointer returned by Next() due to the interaction between grpc_slice inlining
// and GRPC_SLICE_START_PTR.
-TEST_F(ProtoUtilsTest, BackupNext) {
- // Ensure the GrpcBufferWriter internals are initialized.
- g_gli_initializer.summon();
-
+TEST_F(ProtoUtilsTest, TinyBackupThenNext) {
grpc_byte_buffer* bp;
- GrpcBufferWriter writer(&bp, 8192);
+ const int block_size = 1024;
+ GrpcBufferWriter writer(&bp, block_size, 8192);
GrpcBufferWriterPeer peer(&writer);
void* data;
int size;
// Allocate a slice.
ASSERT_TRUE(writer.Next(&data, &size));
- EXPECT_EQ(8192, size);
- // Return a single byte. Before the fix that this test acts as a regression
- // for, this would have resulted in an inlined backup slice.
+ EXPECT_EQ(block_size, size);
+ // Return a single byte.
writer.BackUp(1);
- EXPECT_TRUE(!peer.have_backup());
- // On the next allocation, the slice is non-inlined.
+ EXPECT_FALSE(peer.have_backup());
+ // On the next allocation, the returned slice is non-inlined.
ASSERT_TRUE(writer.Next(&data, &size));
- EXPECT_TRUE(peer.slice().refcount != NULL);
+ EXPECT_TRUE(peer.slice().refcount != nullptr);
+ EXPECT_EQ(block_size, size);
// Cleanup.
g_core_codegen_interface->grpc_byte_buffer_destroy(bp);
}
+namespace {
+
+// Set backup_size to 0 to indicate no backup is needed.
+void BufferWriterTest(int block_size, int total_size, int backup_size) {
+ grpc_byte_buffer* bp;
+ GrpcBufferWriter writer(&bp, block_size, total_size);
+
+ int written_size = 0;
+ void* data;
+ int size = 0;
+ bool backed_up_entire_slice = false;
+
+ while (written_size < total_size) {
+ EXPECT_TRUE(writer.Next(&data, &size));
+ EXPECT_GT(size, 0);
+ EXPECT_TRUE(data);
+ int write_size = size;
+ bool should_backup = false;
+ if (backup_size > 0 && size > backup_size) {
+ write_size = size - backup_size;
+ should_backup = true;
+ } else if (size == backup_size && !backed_up_entire_slice) {
+ // only backup entire slice once.
+ backed_up_entire_slice = true;
+ should_backup = true;
+ write_size = 0;
+ }
+ // May need a last backup.
+ if (write_size + written_size > total_size) {
+ write_size = total_size - written_size;
+ should_backup = true;
+ backup_size = size - write_size;
+ ASSERT_GT(backup_size, 0);
+ }
+ for (int i = 0; i < write_size; i++) {
+ ((uint8_t*)data)[i] = written_size % 128;
+ written_size++;
+ }
+ if (should_backup) {
+ writer.BackUp(backup_size);
+ }
+ }
+ EXPECT_EQ(grpc_byte_buffer_length(bp), (size_t)total_size);
+
+ grpc_byte_buffer_reader reader;
+ grpc_byte_buffer_reader_init(&reader, bp);
+ int read_bytes = 0;
+ while (read_bytes < total_size) {
+ grpc_slice s;
+ EXPECT_TRUE(grpc_byte_buffer_reader_next(&reader, &s));
+ for (size_t i = 0; i < GRPC_SLICE_LENGTH(s); i++) {
+ EXPECT_EQ(GRPC_SLICE_START_PTR(s)[i], read_bytes % 128);
+ read_bytes++;
+ }
+ grpc_slice_unref(s);
+ }
+ EXPECT_EQ(read_bytes, total_size);
+ grpc_byte_buffer_reader_destroy(&reader);
+ grpc_byte_buffer_destroy(bp);
+}
+
+TEST(WriterTest, TinyBlockTinyBackup) {
+ for (int i = 2; i < (int)GRPC_SLICE_INLINED_SIZE; i++) {
+ BufferWriterTest(i, 256, 1);
+ }
+}
+
+TEST(WriterTest, SmallBlockTinyBackup) { BufferWriterTest(64, 256, 1); }
+
+TEST(WriterTest, SmallBlockNoBackup) { BufferWriterTest(64, 256, 0); }
+
+TEST(WriterTest, SmallBlockFullBackup) { BufferWriterTest(64, 256, 64); }
+
+TEST(WriterTest, LargeBlockTinyBackup) { BufferWriterTest(4096, 8192, 1); }
+
+TEST(WriterTest, LargeBlockNoBackup) { BufferWriterTest(4096, 8192, 0); }
+
+TEST(WriterTest, LargeBlockFullBackup) { BufferWriterTest(4096, 8192, 4096); }
+
+TEST(WriterTest, LargeBlockLargeBackup) { BufferWriterTest(4096, 8192, 4095); }
+
+} // namespace
} // namespace internal
} // namespace grpc
int main(int argc, char** argv) {
+ // Ensure the GrpcBufferWriter internals are initialized.
+ grpc::internal::GrpcLibraryInitializer init;
+ init.summon();
+ grpc::GrpcLibraryCodegen lib;
+
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}
diff --git a/test/cpp/common/auth_property_iterator_test.cc b/test/cpp/common/auth_property_iterator_test.cc
index 4a097e8739..fce409aa2f 100644
--- a/test/cpp/common/auth_property_iterator_test.cc
+++ b/test/cpp/common/auth_property_iterator_test.cc
@@ -40,7 +40,7 @@ class TestAuthPropertyIterator : public AuthPropertyIterator {
class AuthPropertyIteratorTest : public ::testing::Test {
protected:
void SetUp() override {
- ctx_ = grpc_auth_context_create(NULL);
+ ctx_ = grpc_auth_context_create(nullptr);
grpc_auth_context_add_cstring_property(ctx_, "name", "chapi");
grpc_auth_context_add_cstring_property(ctx_, "name", "chapo");
grpc_auth_context_add_cstring_property(ctx_, "foo", "bar");
diff --git a/test/cpp/common/secure_auth_context_test.cc b/test/cpp/common/secure_auth_context_test.cc
index 0cc32c1853..7a0530c20a 100644
--- a/test/cpp/common/secure_auth_context_test.cc
+++ b/test/cpp/common/secure_auth_context_test.cc
@@ -42,7 +42,7 @@ TEST_F(SecureAuthContextTest, EmptyContext) {
}
TEST_F(SecureAuthContextTest, Properties) {
- grpc_auth_context* ctx = grpc_auth_context_create(NULL);
+ grpc_auth_context* ctx = grpc_auth_context_create(nullptr);
SecureAuthContext context(ctx, true);
context.AddProperty("name", "chapi");
context.AddProperty("name", "chapo");
@@ -60,7 +60,7 @@ TEST_F(SecureAuthContextTest, Properties) {
}
TEST_F(SecureAuthContextTest, Iterators) {
- grpc_auth_context* ctx = grpc_auth_context_create(NULL);
+ grpc_auth_context* ctx = grpc_auth_context_create(nullptr);
SecureAuthContext context(ctx, true);
context.AddProperty("name", "chapi");
context.AddProperty("name", "chapo");
diff --git a/test/cpp/end2end/async_end2end_test.cc b/test/cpp/end2end/async_end2end_test.cc
index 5b6e9aa312..1ea087e706 100644
--- a/test/cpp/end2end/async_end2end_test.cc
+++ b/test/cpp/end2end/async_end2end_test.cc
@@ -466,7 +466,7 @@ TEST_P(AsyncEnd2endTest, ReconnectChannel) {
// It needs 2 pollset_works to reconnect the channel with polling engine
// "poll"
char* s = gpr_getenv("GRPC_POLL_STRATEGY");
- if (s != NULL && 0 == strcmp(s, "poll")) {
+ if (s != nullptr && 0 == strcmp(s, "poll")) {
poller_slowdown_factor = 2;
}
gpr_free(s);
diff --git a/test/cpp/end2end/client_lb_end2end_test.cc b/test/cpp/end2end/client_lb_end2end_test.cc
index 95630a7714..f8bb12fde1 100644
--- a/test/cpp/end2end/client_lb_end2end_test.cc
+++ b/test/cpp/end2end/client_lb_end2end_test.cc
@@ -113,22 +113,23 @@ class ClientLbEnd2endTest : public ::testing::Test {
void SetNextResolution(const std::vector<int>& ports) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
- grpc_lb_addresses* addresses = grpc_lb_addresses_create(ports.size(), NULL);
+ grpc_lb_addresses* addresses =
+ grpc_lb_addresses_create(ports.size(), nullptr);
for (size_t i = 0; i < ports.size(); ++i) {
char* lb_uri_str;
gpr_asprintf(&lb_uri_str, "ipv4:127.0.0.1:%d", ports[i]);
grpc_uri* lb_uri = grpc_uri_parse(&exec_ctx, lb_uri_str, true);
- GPR_ASSERT(lb_uri != NULL);
+ GPR_ASSERT(lb_uri != nullptr);
grpc_lb_addresses_set_address_from_uri(addresses, i, lb_uri,
false /* is balancer */,
- "" /* balancer name */, NULL);
+ "" /* balancer name */, nullptr);
grpc_uri_destroy(lb_uri);
gpr_free(lb_uri_str);
}
const grpc_arg fake_addresses =
grpc_lb_addresses_create_channel_arg(addresses);
grpc_channel_args* fake_result =
- grpc_channel_args_copy_and_add(NULL, &fake_addresses, 1);
+ grpc_channel_args_copy_and_add(nullptr, &fake_addresses, 1);
grpc_fake_resolver_response_generator_set_response(
&exec_ctx, response_generator_, fake_result);
grpc_channel_args_destroy(&exec_ctx, fake_result);
diff --git a/test/cpp/end2end/end2end_test.cc b/test/cpp/end2end/end2end_test.cc
index 82ca39466e..c71034bbe8 100644
--- a/test/cpp/end2end/end2end_test.cc
+++ b/test/cpp/end2end/end2end_test.cc
@@ -711,7 +711,7 @@ TEST_P(End2endTest, ReconnectChannel) {
// It needs 2 pollset_works to reconnect the channel with polling engine
// "poll"
char* s = gpr_getenv("GRPC_POLL_STRATEGY");
- if (s != NULL && 0 == strcmp(s, "poll")) {
+ if (s != nullptr && 0 == strcmp(s, "poll")) {
poller_slowdown_factor = 2;
}
gpr_free(s);
@@ -1148,7 +1148,7 @@ TEST_P(End2endTest, ChannelState) {
CompletionQueue cq;
std::chrono::system_clock::time_point deadline =
std::chrono::system_clock::now() + std::chrono::milliseconds(10);
- channel_->NotifyOnStateChange(GRPC_CHANNEL_IDLE, deadline, &cq, NULL);
+ channel_->NotifyOnStateChange(GRPC_CHANNEL_IDLE, deadline, &cq, nullptr);
void* tag;
bool ok = true;
cq.Next(&tag, &ok);
diff --git a/test/cpp/end2end/grpclb_end2end_test.cc b/test/cpp/end2end/grpclb_end2end_test.cc
index c15ab88da1..bbf3da4663 100644
--- a/test/cpp/end2end/grpclb_end2end_test.cc
+++ b/test/cpp/end2end/grpclb_end2end_test.cc
@@ -353,11 +353,6 @@ class GrpclbEnd2endTest : public ::testing::Test {
"balancer", server_host_, balancers_.back().get()));
}
ResetStub();
- std::vector<AddressData> addresses;
- for (size_t i = 0; i < balancer_servers_.size(); ++i) {
- addresses.emplace_back(AddressData{balancer_servers_[i].port_, true, ""});
- }
- SetNextResolution(addresses);
}
void TearDown() override {
@@ -370,6 +365,14 @@ class GrpclbEnd2endTest : public ::testing::Test {
grpc_fake_resolver_response_generator_unref(response_generator_);
}
+ void SetNextResolutionAllBalancers() {
+ std::vector<AddressData> addresses;
+ for (size_t i = 0; i < balancer_servers_.size(); ++i) {
+ addresses.emplace_back(AddressData{balancer_servers_[i].port_, true, ""});
+ }
+ SetNextResolution(addresses);
+ }
+
void ResetStub(int fallback_timeout = 0) {
ChannelArguments args;
args.SetGrpclbFallbackTimeout(fallback_timeout);
@@ -581,6 +584,7 @@ class SingleBalancerTest : public GrpclbEnd2endTest {
};
TEST_F(SingleBalancerTest, Vanilla) {
+ SetNextResolutionAllBalancers();
const size_t kNumRpcsPerAddress = 100;
ScheduleResponseForBalancer(
0, BalancerServiceImpl::BuildResponseForBackends(GetBackendPorts(), {}),
@@ -608,6 +612,7 @@ TEST_F(SingleBalancerTest, Vanilla) {
}
TEST_F(SingleBalancerTest, InitiallyEmptyServerlist) {
+ SetNextResolutionAllBalancers();
const int kServerlistDelayMs = 500 * grpc_test_slowdown_factor();
const int kCallDeadlineMs = 1000 * grpc_test_slowdown_factor();
@@ -645,6 +650,7 @@ TEST_F(SingleBalancerTest, InitiallyEmptyServerlist) {
}
TEST_F(SingleBalancerTest, Fallback) {
+ SetNextResolutionAllBalancers();
const int kFallbackTimeoutMs = 200 * grpc_test_slowdown_factor();
const int kServerlistDelayMs = 500 * grpc_test_slowdown_factor();
const size_t kNumBackendInResolution = backends_.size() / 2;
@@ -711,6 +717,7 @@ TEST_F(SingleBalancerTest, Fallback) {
}
TEST_F(SingleBalancerTest, FallbackUpdate) {
+ SetNextResolutionAllBalancers();
const int kFallbackTimeoutMs = 200 * grpc_test_slowdown_factor();
const int kServerlistDelayMs = 500 * grpc_test_slowdown_factor();
const size_t kNumBackendInResolution = backends_.size() / 3;
@@ -818,6 +825,7 @@ TEST_F(SingleBalancerTest, FallbackUpdate) {
}
TEST_F(SingleBalancerTest, BackendsRestart) {
+ SetNextResolutionAllBalancers();
const size_t kNumRpcsPerAddress = 100;
ScheduleResponseForBalancer(
0, BalancerServiceImpl::BuildResponseForBackends(GetBackendPorts(), {}),
@@ -857,6 +865,7 @@ class UpdatesTest : public GrpclbEnd2endTest {
};
TEST_F(UpdatesTest, UpdateBalancers) {
+ SetNextResolutionAllBalancers();
const std::vector<int> first_backend{GetBackendPorts()[0]};
const std::vector<int> second_backend{GetBackendPorts()[1]};
ScheduleResponseForBalancer(
@@ -919,6 +928,7 @@ TEST_F(UpdatesTest, UpdateBalancers) {
// verify that the LB channel inside grpclb keeps the initial connection (which
// by definition is also present in the update).
TEST_F(UpdatesTest, UpdateBalancersRepeated) {
+ SetNextResolutionAllBalancers();
const std::vector<int> first_backend{GetBackendPorts()[0]};
const std::vector<int> second_backend{GetBackendPorts()[0]};
@@ -989,6 +999,9 @@ TEST_F(UpdatesTest, UpdateBalancersRepeated) {
}
TEST_F(UpdatesTest, UpdateBalancersDeadUpdate) {
+ std::vector<AddressData> addresses;
+ addresses.emplace_back(AddressData{balancer_servers_[0].port_, true, ""});
+ SetNextResolution(addresses);
const std::vector<int> first_backend{GetBackendPorts()[0]};
const std::vector<int> second_backend{GetBackendPorts()[1]};
@@ -1030,7 +1043,7 @@ TEST_F(UpdatesTest, UpdateBalancersDeadUpdate) {
EXPECT_EQ(0U, balancer_servers_[2].service_->request_count());
EXPECT_EQ(0U, balancer_servers_[2].service_->response_count());
- std::vector<AddressData> addresses;
+ addresses.clear();
addresses.emplace_back(AddressData{balancer_servers_[1].port_, true, ""});
gpr_log(GPR_INFO, "========= ABOUT TO UPDATE 1 ==========");
SetNextResolution(addresses);
@@ -1055,8 +1068,14 @@ TEST_F(UpdatesTest, UpdateBalancersDeadUpdate) {
balancers_[2]->NotifyDoneWithServerlists();
EXPECT_EQ(1U, balancer_servers_[0].service_->request_count());
EXPECT_EQ(1U, balancer_servers_[0].service_->response_count());
- EXPECT_EQ(1U, balancer_servers_[1].service_->request_count());
- EXPECT_EQ(1U, balancer_servers_[1].service_->response_count());
+ // The second balancer, published as part of the first update, may end up
+ // getting two requests (that is, 1 <= #req <= 2) if the LB call retry timer
+ // firing races with the arrival of the update containing the second
+ // balancer.
+ EXPECT_GE(balancer_servers_[1].service_->request_count(), 1U);
+ EXPECT_GE(balancer_servers_[1].service_->response_count(), 1U);
+ EXPECT_LE(balancer_servers_[1].service_->request_count(), 2U);
+ EXPECT_LE(balancer_servers_[1].service_->response_count(), 2U);
EXPECT_EQ(0U, balancer_servers_[2].service_->request_count());
EXPECT_EQ(0U, balancer_servers_[2].service_->response_count());
// Check LB policy name for the channel.
@@ -1064,6 +1083,7 @@ TEST_F(UpdatesTest, UpdateBalancersDeadUpdate) {
}
TEST_F(SingleBalancerTest, Drop) {
+ SetNextResolutionAllBalancers();
const size_t kNumRpcsPerAddress = 100;
const int num_of_drop_by_rate_limiting_addresses = 1;
const int num_of_drop_by_load_balancing_addresses = 2;
@@ -1107,6 +1127,7 @@ TEST_F(SingleBalancerTest, Drop) {
}
TEST_F(SingleBalancerTest, DropAllFirst) {
+ SetNextResolutionAllBalancers();
// All registered addresses are marked as "drop".
const int num_of_drop_by_rate_limiting_addresses = 1;
const int num_of_drop_by_load_balancing_addresses = 1;
@@ -1122,6 +1143,7 @@ TEST_F(SingleBalancerTest, DropAllFirst) {
}
TEST_F(SingleBalancerTest, DropAll) {
+ SetNextResolutionAllBalancers();
ScheduleResponseForBalancer(
0, BalancerServiceImpl::BuildResponseForBackends(GetBackendPorts(), {}),
0);
@@ -1152,6 +1174,7 @@ class SingleBalancerWithClientLoadReportingTest : public GrpclbEnd2endTest {
};
TEST_F(SingleBalancerWithClientLoadReportingTest, Vanilla) {
+ SetNextResolutionAllBalancers();
const size_t kNumRpcsPerAddress = 100;
ScheduleResponseForBalancer(
0, BalancerServiceImpl::BuildResponseForBackends(GetBackendPorts(), {}),
@@ -1186,6 +1209,7 @@ TEST_F(SingleBalancerWithClientLoadReportingTest, Vanilla) {
}
TEST_F(SingleBalancerWithClientLoadReportingTest, Drop) {
+ SetNextResolutionAllBalancers();
const size_t kNumRpcsPerAddress = 3;
const int num_of_drop_by_rate_limiting_addresses = 2;
const int num_of_drop_by_load_balancing_addresses = 1;
diff --git a/test/cpp/grpclb/grpclb_api_test.cc b/test/cpp/grpclb/grpclb_api_test.cc
index 54b5708e6a..03778e1fc5 100644
--- a/test/cpp/grpclb/grpclb_api_test.cc
+++ b/test/cpp/grpclb/grpclb_api_test.cc
@@ -48,7 +48,7 @@ grpc::string PackedStringToIp(const grpc_grpclb_ip_address& pb_ip) {
} else {
abort();
}
- GPR_ASSERT(inet_ntop(af, (void*)pb_ip.bytes, ip_str, 46) != NULL);
+ GPR_ASSERT(inet_ntop(af, (void*)pb_ip.bytes, ip_str, 46) != nullptr);
return ip_str;
}
diff --git a/test/cpp/grpclb/grpclb_test.cc b/test/cpp/grpclb/grpclb_test.cc
index d032c0dfcc..62ae495177 100644
--- a/test/cpp/grpclb/grpclb_test.cc
+++ b/test/cpp/grpclb/grpclb_test.cc
@@ -149,7 +149,7 @@ static void drain_cq(grpc_completion_queue* cq) {
grpc_event ev;
do {
ev = grpc_completion_queue_next(cq, grpc_timeout_seconds_to_deadline(5),
- NULL);
+ nullptr);
} while (ev.type != GRPC_QUEUE_SHUTDOWN);
}
@@ -188,16 +188,16 @@ static void start_lb_server(server_fixture* sf, int* ports, size_t nports,
// make sure we've received the initial metadata from the grpclb request.
GPR_ASSERT(request_metadata_recv.count > 0);
- GPR_ASSERT(request_metadata_recv.metadata != NULL);
+ GPR_ASSERT(request_metadata_recv.metadata != nullptr);
// receive request for backends
op = ops;
op->op = GRPC_OP_RECV_MESSAGE;
op->data.recv_message.recv_message = &request_payload_recv;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(202), NULL);
+ error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(202), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
CQ_EXPECT_COMPLETION(cqv, tag(202), 1);
cq_verify(cqv);
@@ -222,14 +222,14 @@ static void start_lb_server(server_fixture* sf, int* ports, size_t nports,
op->op = GRPC_OP_SEND_INITIAL_METADATA;
op->data.send_initial_metadata.count = 0;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
op->op = GRPC_OP_RECV_CLOSE_ON_SERVER;
op->data.recv_close_on_server.cancelled = &was_cancelled;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(201), NULL);
+ error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(201), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
gpr_log(GPR_INFO, "LB Server[%s](%s) after tag 201", sf->servers_hostport,
sf->balancer_name);
@@ -252,9 +252,10 @@ static void start_lb_server(server_fixture* sf, int* ports, size_t nports,
op->op = GRPC_OP_SEND_MESSAGE;
op->data.send_message.send_message = response_payload;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(203), NULL);
+ error =
+ grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(203), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
CQ_EXPECT_COMPLETION(cqv, tag(203), 1);
cq_verify(cqv);
@@ -274,9 +275,9 @@ static void start_lb_server(server_fixture* sf, int* ports, size_t nports,
grpc_slice status_details = grpc_slice_from_static_string("xyz");
op->data.send_status_from_server.status_details = &status_details;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(204), NULL);
+ error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(204), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
CQ_EXPECT_COMPLETION(cqv, tag(201), 1);
@@ -318,8 +319,8 @@ static void start_backend_server(server_fixture* sf) {
tag(100));
GPR_ASSERT(GRPC_CALL_OK == error);
gpr_log(GPR_INFO, "Server[%s] up", sf->servers_hostport);
- ev = grpc_completion_queue_next(sf->cq,
- grpc_timeout_seconds_to_deadline(60), NULL);
+ ev = grpc_completion_queue_next(
+ sf->cq, grpc_timeout_seconds_to_deadline(60), nullptr);
if (!ev.success) {
gpr_log(GPR_INFO, "Server[%s] being torn down", sf->servers_hostport);
cq_verifier_destroy(cqv);
@@ -341,14 +342,15 @@ static void start_backend_server(server_fixture* sf) {
op->op = GRPC_OP_SEND_INITIAL_METADATA;
op->data.send_initial_metadata.count = 0;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
op->op = GRPC_OP_RECV_CLOSE_ON_SERVER;
op->data.recv_close_on_server.cancelled = &was_cancelled;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(101), NULL);
+ error =
+ grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(101), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
gpr_log(GPR_INFO, "Server[%s] after tag 101", sf->servers_hostport);
@@ -359,15 +361,16 @@ static void start_backend_server(server_fixture* sf) {
op->op = GRPC_OP_RECV_MESSAGE;
op->data.recv_message.recv_message = &request_payload_recv;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(102), NULL);
+ error =
+ grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(102), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
ev = grpc_completion_queue_next(
- sf->cq, grpc_timeout_seconds_to_deadline(3), NULL);
+ sf->cq, grpc_timeout_seconds_to_deadline(3), nullptr);
if (ev.type == GRPC_OP_COMPLETE && ev.success) {
GPR_ASSERT(ev.tag = tag(102));
- if (request_payload_recv == NULL) {
+ if (request_payload_recv == nullptr) {
exit = true;
gpr_log(GPR_INFO,
"Server[%s] recv \"close\" from client, exiting. Call #%d",
@@ -388,13 +391,13 @@ static void start_backend_server(server_fixture* sf) {
op->op = GRPC_OP_SEND_MESSAGE;
op->data.send_message.send_message = response_payload;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error =
- grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(103), NULL);
+ error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(103),
+ nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
ev = grpc_completion_queue_next(
- sf->cq, grpc_timeout_seconds_to_deadline(3), NULL);
+ sf->cq, grpc_timeout_seconds_to_deadline(3), nullptr);
if (ev.type == GRPC_OP_COMPLETE && ev.success) {
GPR_ASSERT(ev.tag = tag(103));
} else {
@@ -422,9 +425,10 @@ static void start_backend_server(server_fixture* sf) {
grpc_slice_from_static_string("Backend server out a-ok");
op->data.send_status_from_server.status_details = &status_details;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(104), NULL);
+ error =
+ grpc_call_start_batch(s, ops, (size_t)(op - ops), tag(104), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
CQ_EXPECT_COMPLETION(cqv, tag(101), 1);
@@ -459,10 +463,10 @@ static void perform_request(client_fixture* cf) {
grpc_slice_from_copied_string("hello world");
grpc_slice host = grpc_slice_from_static_string("foo.test.google.fr:1234");
- c = grpc_channel_create_call(cf->client, NULL, GRPC_PROPAGATE_DEFAULTS,
+ c = grpc_channel_create_call(cf->client, nullptr, GRPC_PROPAGATE_DEFAULTS,
cf->cq, grpc_slice_from_static_string("/foo"),
&host, grpc_timeout_seconds_to_deadline(5),
- NULL);
+ nullptr);
gpr_log(GPR_INFO, "Call 0x%" PRIxPTR " created", (intptr_t)c);
GPR_ASSERT(c);
char* peer;
@@ -474,21 +478,21 @@ static void perform_request(client_fixture* cf) {
op->op = GRPC_OP_SEND_INITIAL_METADATA;
op->data.send_initial_metadata.count = 0;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
op->op = GRPC_OP_RECV_INITIAL_METADATA;
op->data.recv_initial_metadata.recv_initial_metadata = &initial_metadata_recv;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
op->op = GRPC_OP_RECV_STATUS_ON_CLIENT;
op->data.recv_status_on_client.trailing_metadata = &trailing_metadata_recv;
op->data.recv_status_on_client.status = &status;
op->data.recv_status_on_client.status_details = &details;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(c, ops, (size_t)(op - ops), tag(1), NULL);
+ error = grpc_call_start_batch(c, ops, (size_t)(op - ops), tag(1), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
for (i = 0; i < 4; i++) {
@@ -498,14 +502,14 @@ static void perform_request(client_fixture* cf) {
op->op = GRPC_OP_SEND_MESSAGE;
op->data.send_message.send_message = request_payload;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
op->op = GRPC_OP_RECV_MESSAGE;
op->data.recv_message.recv_message = &response_payload_recv;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(c, ops, (size_t)(op - ops), tag(2), NULL);
+ error = grpc_call_start_batch(c, ops, (size_t)(op - ops), tag(2), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
CQ_EXPECT_COMPLETION(cqv, tag(2), 1);
@@ -522,9 +526,9 @@ static void perform_request(client_fixture* cf) {
op = ops;
op->op = GRPC_OP_SEND_CLOSE_FROM_CLIENT;
op->flags = 0;
- op->reserved = NULL;
+ op->reserved = nullptr;
op++;
- error = grpc_call_start_batch(c, ops, (size_t)(op - ops), tag(3), NULL);
+ error = grpc_call_start_batch(c, ops, (size_t)(op - ops), tag(3), nullptr);
GPR_ASSERT(GRPC_CALL_OK == error);
CQ_EXPECT_COMPLETION(cqv, tag(1), 1);
@@ -550,20 +554,20 @@ static void setup_client(const server_fixture* lb_server,
const server_fixture* backends, client_fixture* cf) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
- char* expected_target_names = NULL;
+ char* expected_target_names = nullptr;
const char* backends_name = lb_server->servers_hostport;
gpr_asprintf(&expected_target_names, "%s;%s", backends_name, BALANCERS_NAME);
grpc_fake_resolver_response_generator* response_generator =
grpc_fake_resolver_response_generator_create();
- grpc_lb_addresses* addresses = grpc_lb_addresses_create(1, NULL);
+ grpc_lb_addresses* addresses = grpc_lb_addresses_create(1, nullptr);
char* lb_uri_str;
gpr_asprintf(&lb_uri_str, "ipv4:%s", lb_server->servers_hostport);
grpc_uri* lb_uri = grpc_uri_parse(&exec_ctx, lb_uri_str, true);
- GPR_ASSERT(lb_uri != NULL);
+ GPR_ASSERT(lb_uri != nullptr);
grpc_lb_addresses_set_address_from_uri(addresses, 0, lb_uri, true,
- lb_server->balancer_name, NULL);
+ lb_server->balancer_name, nullptr);
grpc_uri_destroy(lb_uri);
gpr_free(lb_uri_str);
@@ -571,22 +575,22 @@ static void setup_client(const server_fixture* lb_server,
const grpc_arg fake_addresses =
grpc_lb_addresses_create_channel_arg(addresses);
grpc_channel_args* fake_result =
- grpc_channel_args_copy_and_add(NULL, &fake_addresses, 1);
+ grpc_channel_args_copy_and_add(nullptr, &fake_addresses, 1);
grpc_lb_addresses_destroy(&exec_ctx, addresses);
const grpc_arg new_args[] = {
grpc_fake_transport_expected_targets_arg(expected_target_names),
grpc_fake_resolver_response_generator_arg(response_generator)};
- grpc_channel_args* args =
- grpc_channel_args_copy_and_add(NULL, new_args, GPR_ARRAY_SIZE(new_args));
+ grpc_channel_args* args = grpc_channel_args_copy_and_add(
+ nullptr, new_args, GPR_ARRAY_SIZE(new_args));
gpr_free(expected_target_names);
- cf->cq = grpc_completion_queue_create_for_next(NULL);
+ cf->cq = grpc_completion_queue_create_for_next(nullptr);
grpc_channel_credentials* fake_creds =
grpc_fake_transport_security_credentials_create();
cf->client =
- grpc_secure_channel_create(fake_creds, cf->server_uri, args, NULL);
+ grpc_secure_channel_create(fake_creds, cf->server_uri, args, nullptr);
grpc_fake_resolver_response_generator_set_response(
&exec_ctx, response_generator, fake_result);
grpc_channel_args_destroy(&exec_ctx, fake_result);
@@ -600,16 +604,16 @@ static void teardown_client(client_fixture* cf) {
grpc_completion_queue_shutdown(cf->cq);
drain_cq(cf->cq);
grpc_completion_queue_destroy(cf->cq);
- cf->cq = NULL;
+ cf->cq = nullptr;
grpc_channel_destroy(cf->client);
- cf->client = NULL;
+ cf->client = nullptr;
gpr_free(cf->server_uri);
}
static void setup_server(const char* host, server_fixture* sf) {
int assigned_port;
- sf->cq = grpc_completion_queue_create_for_next(NULL);
+ sf->cq = grpc_completion_queue_create_for_next(nullptr);
const char* colon_idx = strchr(host, ':');
if (colon_idx) {
const char* port_str = colon_idx + 1;
@@ -623,8 +627,8 @@ static void setup_server(const char* host, server_fixture* sf) {
grpc_server_credentials* server_creds =
grpc_fake_transport_security_server_credentials_create();
- sf->server = grpc_server_create(NULL, NULL);
- grpc_server_register_completion_queue(sf->server, sf->cq, NULL);
+ sf->server = grpc_server_create(nullptr, nullptr);
+ grpc_server_register_completion_queue(sf->server, sf->cq, nullptr);
GPR_ASSERT((assigned_port = grpc_server_add_secure_http2_port(
sf->server, sf->servers_hostport, server_creds)) > 0);
grpc_server_credentials_release(server_creds);
@@ -638,17 +642,17 @@ static void teardown_server(server_fixture* sf) {
gpr_log(GPR_INFO, "Server[%s] shutting down", sf->servers_hostport);
grpc_completion_queue* shutdown_cq =
- grpc_completion_queue_create_for_pluck(NULL);
+ grpc_completion_queue_create_for_pluck(nullptr);
grpc_server_shutdown_and_notify(sf->server, shutdown_cq, tag(1000));
GPR_ASSERT(grpc_completion_queue_pluck(shutdown_cq, tag(1000),
grpc_timeout_seconds_to_deadline(5),
- NULL)
+ nullptr)
.type == GRPC_OP_COMPLETE);
grpc_completion_queue_destroy(shutdown_cq);
grpc_server_destroy(sf->server);
gpr_thd_join(sf->tid);
- sf->server = NULL;
+ sf->server = nullptr;
grpc_completion_queue_shutdown(sf->cq);
drain_cq(sf->cq);
grpc_completion_queue_destroy(sf->cq);
@@ -689,14 +693,15 @@ static test_fixture setup_test_fixture(int lb_server_update_delay_ms) {
tf.lb_backends[i].lb_token_prefix = "";
}
setup_server("127.0.0.1", &tf.lb_backends[i]);
- gpr_thd_new(&tf.lb_backends[i].tid, fork_backend_server, &tf.lb_backends[i],
- &options);
+ gpr_thd_new(&tf.lb_backends[i].tid, "grpclb_backend", fork_backend_server,
+ &tf.lb_backends[i], &options);
}
tf.lb_server.lb_token_prefix = LB_TOKEN_PREFIX;
tf.lb_server.balancer_name = BALANCERS_NAME;
setup_server("127.0.0.1", &tf.lb_server);
- gpr_thd_new(&tf.lb_server.tid, fork_lb_server, &tf.lb_server, &options);
+ gpr_thd_new(&tf.lb_server.tid, "grpclb_server", fork_lb_server, &tf.lb_server,
+ &options);
setup_client(&tf.lb_server, tf.lb_backends, &tf.client);
return tf;
}
diff --git a/test/cpp/interop/http2_client.cc b/test/cpp/interop/http2_client.cc
index 1e04e57d09..2de7abcf17 100644
--- a/test/cpp/interop/http2_client.cc
+++ b/test/cpp/interop/http2_client.cc
@@ -217,7 +217,7 @@ int main(int argc, char** argv) {
"goaway", "max_streams", "ping",
"rst_after_data", "rst_after_header", "rst_during_data"};
char* joined_testcases =
- gpr_strjoin_sep(testcases, GPR_ARRAY_SIZE(testcases), "\n", NULL);
+ gpr_strjoin_sep(testcases, GPR_ARRAY_SIZE(testcases), "\n", nullptr);
gpr_log(GPR_ERROR, "Unsupported test case %s. Valid options are\n%s",
FLAGS_test_case.c_str(), joined_testcases);
diff --git a/test/cpp/interop/interop_server.cc b/test/cpp/interop/interop_server.cc
index a24cdc7d2d..30bd8bfef8 100644
--- a/test/cpp/interop/interop_server.cc
+++ b/test/cpp/interop/interop_server.cc
@@ -317,9 +317,15 @@ class TestServiceImpl : public TestService::Service {
void grpc::testing::interop::RunServer(
std::shared_ptr<ServerCredentials> creds) {
- GPR_ASSERT(FLAGS_port != 0);
+ RunServer(creds, FLAGS_port, nullptr);
+}
+
+void grpc::testing::interop::RunServer(
+ std::shared_ptr<ServerCredentials> creds, const int port,
+ ServerStartedCondition* server_started_condition) {
+ GPR_ASSERT(port != 0);
std::ostringstream server_address;
- server_address << "0.0.0.0:" << FLAGS_port;
+ server_address << "0.0.0.0:" << port;
TestServiceImpl service;
SimpleRequest request;
@@ -333,6 +339,14 @@ void grpc::testing::interop::RunServer(
}
std::unique_ptr<Server> server(builder.BuildAndStart());
gpr_log(GPR_INFO, "Server listening on %s", server_address.str().c_str());
+
+ // Signal that the server has started.
+ if (server_started_condition) {
+ std::unique_lock<std::mutex> lock(server_started_condition->mutex);
+ server_started_condition->server_started = true;
+ server_started_condition->condition.notify_all();
+ }
+
while (!gpr_atm_no_barrier_load(&g_got_sigint)) {
gpr_sleep_until(gpr_time_add(gpr_now(GPR_CLOCK_REALTIME),
gpr_time_from_seconds(5, GPR_TIMESPAN)));
diff --git a/test/cpp/interop/server_helper.h b/test/cpp/interop/server_helper.h
index 6af003fe93..1bf7db1e14 100644
--- a/test/cpp/interop/server_helper.h
+++ b/test/cpp/interop/server_helper.h
@@ -19,6 +19,7 @@
#ifndef GRPC_TEST_CPP_INTEROP_SERVER_HELPER_H
#define GRPC_TEST_CPP_INTEROP_SERVER_HELPER_H
+#include <condition_variable>
#include <memory>
#include <grpc/compression.h>
@@ -50,8 +51,27 @@ class InteropServerContextInspector {
namespace interop {
extern gpr_atm g_got_sigint;
+
+struct ServerStartedCondition {
+ std::mutex mutex;
+ std::condition_variable condition;
+ bool server_started = false;
+};
+
+/// Run gRPC interop server using port FLAGS_port.
+///
+/// \param creds The credentials associated with the server.
void RunServer(std::shared_ptr<ServerCredentials> creds);
+/// Run gRPC interop server.
+///
+/// \param creds The credentials associated with the server.
+/// \param port Port to use for the server.
+/// \param server_started_condition (optional) Struct holding mutex, condition
+/// variable, and condition used to notify when the server has started.
+void RunServer(std::shared_ptr<ServerCredentials> creds, int port,
+ ServerStartedCondition* server_started_condition);
+
} // namespace interop
} // namespace testing
} // namespace grpc
diff --git a/test/cpp/interop/stress_test.cc b/test/cpp/interop/stress_test.cc
index 991fc88ad2..4b39dc3211 100644
--- a/test/cpp/interop/stress_test.cc
+++ b/test/cpp/interop/stress_test.cc
@@ -36,9 +36,7 @@
#include "test/cpp/util/metrics_server.h"
#include "test/cpp/util/test_config.h"
-extern "C" {
extern void gpr_default_log(gpr_log_func_args* args);
-}
DEFINE_int32(metrics_port, 8081, "The metrics server port.");
@@ -230,7 +228,7 @@ int main(int argc, char** argv) {
log_level = FLAGS_log_level;
gpr_set_log_function(TestLogFunction);
- srand(time(NULL));
+ srand(time(nullptr));
// Parse the server addresses
std::vector<grpc::string> server_addresses;
diff --git a/test/cpp/microbenchmarks/bm_call_create.cc b/test/cpp/microbenchmarks/bm_call_create.cc
index ec5c1275d1..a45c577320 100644
--- a/test/cpp/microbenchmarks/bm_call_create.cc
+++ b/test/cpp/microbenchmarks/bm_call_create.cc
@@ -91,7 +91,7 @@ class InsecureChannel : public BaseChannelFixture {
public:
InsecureChannel()
: BaseChannelFixture(
- grpc_insecure_channel_create("localhost:1234", NULL, NULL)) {}
+ grpc_insecure_channel_create("localhost:1234", nullptr, nullptr)) {}
};
class LameChannel : public BaseChannelFixture {
@@ -105,14 +105,14 @@ template <class Fixture>
static void BM_CallCreateDestroy(benchmark::State& state) {
TrackCounters track_counters;
Fixture fixture;
- grpc_completion_queue* cq = grpc_completion_queue_create_for_next(NULL);
+ grpc_completion_queue* cq = grpc_completion_queue_create_for_next(nullptr);
gpr_timespec deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
- void* method_hdl =
- grpc_channel_register_call(fixture.channel(), "/foo/bar", NULL, NULL);
+ void* method_hdl = grpc_channel_register_call(fixture.channel(), "/foo/bar",
+ nullptr, nullptr);
while (state.KeepRunning()) {
grpc_call_unref(grpc_channel_create_registered_call(
- fixture.channel(), NULL, GRPC_PROPAGATE_DEFAULTS, cq, method_hdl,
- deadline, NULL));
+ fixture.channel(), nullptr, GRPC_PROPAGATE_DEFAULTS, cq, method_hdl,
+ deadline, nullptr));
}
grpc_completion_queue_destroy(cq);
track_counters.Finish(state);
@@ -161,7 +161,7 @@ static void BM_LameChannelCallCreateCore(benchmark::State& state) {
grpc_completion_queue* cq;
grpc_metadata_array initial_metadata_recv;
grpc_metadata_array trailing_metadata_recv;
- grpc_byte_buffer* response_payload_recv = NULL;
+ grpc_byte_buffer* response_payload_recv = nullptr;
grpc_status_code status;
grpc_slice details;
grpc::testing::EchoRequest send_request;
@@ -170,14 +170,14 @@ static void BM_LameChannelCallCreateCore(benchmark::State& state) {
channel = grpc_lame_client_channel_create(
"localhost:1234", GRPC_STATUS_UNAUTHENTICATED, "blah");
- cq = grpc_completion_queue_create_for_next(NULL);
+ cq = grpc_completion_queue_create_for_next(nullptr);
void* rc = grpc_channel_register_call(
- channel, "/grpc.testing.EchoTestService/Echo", NULL, NULL);
+ channel, "/grpc.testing.EchoTestService/Echo", nullptr, nullptr);
while (state.KeepRunning()) {
GPR_TIMER_SCOPE("BenchmarkCycle", 0);
grpc_call* call = grpc_channel_create_registered_call(
- channel, NULL, GRPC_PROPAGATE_DEFAULTS, cq, rc,
- gpr_inf_future(GPR_CLOCK_REALTIME), NULL);
+ channel, nullptr, GRPC_PROPAGATE_DEFAULTS, cq, rc,
+ gpr_inf_future(GPR_CLOCK_REALTIME), nullptr);
grpc_metadata_array_init(&initial_metadata_recv);
grpc_metadata_array_init(&trailing_metadata_recv);
grpc_byte_buffer* request_payload_send =
@@ -210,9 +210,9 @@ static void BM_LameChannelCallCreateCore(benchmark::State& state) {
GPR_ASSERT(GRPC_CALL_OK == grpc_call_start_batch(call, ops,
(size_t)(op - ops),
- (void*)1, NULL));
+ (void*)1, nullptr));
grpc_event ev = grpc_completion_queue_next(
- cq, gpr_inf_future(GPR_CLOCK_REALTIME), NULL);
+ cq, gpr_inf_future(GPR_CLOCK_REALTIME), nullptr);
GPR_ASSERT(ev.type != GRPC_QUEUE_SHUTDOWN);
GPR_ASSERT(ev.success != 0);
grpc_call_unref(call);
@@ -235,7 +235,7 @@ static void BM_LameChannelCallCreateCoreSeparateBatch(benchmark::State& state) {
grpc_completion_queue* cq;
grpc_metadata_array initial_metadata_recv;
grpc_metadata_array trailing_metadata_recv;
- grpc_byte_buffer* response_payload_recv = NULL;
+ grpc_byte_buffer* response_payload_recv = nullptr;
grpc_status_code status;
grpc_slice details;
grpc::testing::EchoRequest send_request;
@@ -244,14 +244,14 @@ static void BM_LameChannelCallCreateCoreSeparateBatch(benchmark::State& state) {
channel = grpc_lame_client_channel_create(
"localhost:1234", GRPC_STATUS_UNAUTHENTICATED, "blah");
- cq = grpc_completion_queue_create_for_next(NULL);
+ cq = grpc_completion_queue_create_for_next(nullptr);
void* rc = grpc_channel_register_call(
- channel, "/grpc.testing.EchoTestService/Echo", NULL, NULL);
+ channel, "/grpc.testing.EchoTestService/Echo", nullptr, nullptr);
while (state.KeepRunning()) {
GPR_TIMER_SCOPE("BenchmarkCycle", 0);
grpc_call* call = grpc_channel_create_registered_call(
- channel, NULL, GRPC_PROPAGATE_DEFAULTS, cq, rc,
- gpr_inf_future(GPR_CLOCK_REALTIME), NULL);
+ channel, nullptr, GRPC_PROPAGATE_DEFAULTS, cq, rc,
+ gpr_inf_future(GPR_CLOCK_REALTIME), nullptr);
grpc_metadata_array_init(&initial_metadata_recv);
grpc_metadata_array_init(&trailing_metadata_recv);
grpc_byte_buffer* request_payload_send =
@@ -271,7 +271,7 @@ static void BM_LameChannelCallCreateCoreSeparateBatch(benchmark::State& state) {
op++;
GPR_ASSERT(GRPC_CALL_OK == grpc_call_start_batch(call, ops,
(size_t)(op - ops),
- (void*)0, NULL));
+ (void*)nullptr, nullptr));
memset(ops, 0, sizeof(ops));
op = ops;
op->op = GRPC_OP_RECV_INITIAL_METADATA;
@@ -289,13 +289,13 @@ static void BM_LameChannelCallCreateCoreSeparateBatch(benchmark::State& state) {
GPR_ASSERT(GRPC_CALL_OK == grpc_call_start_batch(call, ops,
(size_t)(op - ops),
- (void*)1, NULL));
+ (void*)1, nullptr));
grpc_event ev = grpc_completion_queue_next(
- cq, gpr_inf_future(GPR_CLOCK_REALTIME), NULL);
+ cq, gpr_inf_future(GPR_CLOCK_REALTIME), nullptr);
GPR_ASSERT(ev.type != GRPC_QUEUE_SHUTDOWN);
GPR_ASSERT(ev.success == 0);
ev = grpc_completion_queue_next(cq, gpr_inf_future(GPR_CLOCK_REALTIME),
- NULL);
+ nullptr);
GPR_ASSERT(ev.type != GRPC_QUEUE_SHUTDOWN);
GPR_ASSERT(ev.success != 0);
grpc_call_unref(call);
@@ -538,7 +538,7 @@ static void BM_IsolatedFilter(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
size_t channel_size = grpc_channel_stack_size(
- filters.size() == 0 ? NULL : &filters[0], filters.size());
+ filters.size() == 0 ? nullptr : &filters[0], filters.size());
grpc_channel_stack* channel_stack =
static_cast<grpc_channel_stack*>(gpr_zalloc(channel_size));
GPR_ASSERT(GRPC_LOG_IF_ERROR(
@@ -559,8 +559,8 @@ static void BM_IsolatedFilter(benchmark::State& state) {
TestOp test_op_data;
grpc_call_element_args call_args;
call_args.call_stack = call_stack;
- call_args.server_transport_data = NULL;
- call_args.context = NULL;
+ call_args.server_transport_data = nullptr;
+ call_args.context = nullptr;
call_args.path = method;
call_args.start_time = start_time;
call_args.deadline = deadline;
@@ -569,9 +569,9 @@ static void BM_IsolatedFilter(benchmark::State& state) {
while (state.KeepRunning()) {
GPR_TIMER_SCOPE("BenchmarkCycle", 0);
GRPC_ERROR_UNREF(grpc_call_stack_init(&exec_ctx, channel_stack, 1,
- DoNothing, NULL, &call_args));
+ DoNothing, nullptr, &call_args));
typename TestOp::Op op(&exec_ctx, &test_op_data, call_stack);
- grpc_call_stack_destroy(&exec_ctx, call_stack, &final_info, NULL);
+ grpc_call_stack_destroy(&exec_ctx, call_stack, &final_info, nullptr);
op.Finish(&exec_ctx);
grpc_exec_ctx_flush(&exec_ctx);
// recreate arena every 64k iterations to avoid oom
@@ -708,14 +708,15 @@ class IsolatedCallFixture : public TrackCounters {
grpc_channel_stack_builder_set_name(builder, "dummy");
grpc_channel_stack_builder_set_target(builder, "dummy_target");
GPR_ASSERT(grpc_channel_stack_builder_append_filter(
- builder, &isolated_call_filter::isolated_call_filter, NULL, NULL));
+ builder, &isolated_call_filter::isolated_call_filter, nullptr,
+ nullptr));
{
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
channel_ = grpc_channel_create_with_builder(&exec_ctx, builder,
GRPC_CLIENT_CHANNEL);
grpc_exec_ctx_finish(&exec_ctx);
}
- cq_ = grpc_completion_queue_create_for_next(NULL);
+ cq_ = grpc_completion_queue_create_for_next(nullptr);
}
void Finish(benchmark::State& state) {
@@ -735,13 +736,13 @@ class IsolatedCallFixture : public TrackCounters {
static void BM_IsolatedCall_NoOp(benchmark::State& state) {
IsolatedCallFixture fixture;
gpr_timespec deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
- void* method_hdl =
- grpc_channel_register_call(fixture.channel(), "/foo/bar", NULL, NULL);
+ void* method_hdl = grpc_channel_register_call(fixture.channel(), "/foo/bar",
+ nullptr, nullptr);
while (state.KeepRunning()) {
GPR_TIMER_SCOPE("BenchmarkCycle", 0);
grpc_call_unref(grpc_channel_create_registered_call(
fixture.channel(), nullptr, GRPC_PROPAGATE_DEFAULTS, fixture.cq(),
- method_hdl, deadline, NULL));
+ method_hdl, deadline, nullptr));
}
fixture.Finish(state);
}
@@ -750,11 +751,11 @@ BENCHMARK(BM_IsolatedCall_NoOp);
static void BM_IsolatedCall_Unary(benchmark::State& state) {
IsolatedCallFixture fixture;
gpr_timespec deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
- void* method_hdl =
- grpc_channel_register_call(fixture.channel(), "/foo/bar", NULL, NULL);
+ void* method_hdl = grpc_channel_register_call(fixture.channel(), "/foo/bar",
+ nullptr, nullptr);
grpc_slice slice = grpc_slice_from_static_string("hello world");
grpc_byte_buffer* send_message = grpc_raw_byte_buffer_create(&slice, 1);
- grpc_byte_buffer* recv_message = NULL;
+ grpc_byte_buffer* recv_message = nullptr;
grpc_status_code status_code;
grpc_slice status_details = grpc_empty_slice();
grpc_metadata_array recv_initial_metadata;
@@ -780,10 +781,10 @@ static void BM_IsolatedCall_Unary(benchmark::State& state) {
GPR_TIMER_SCOPE("BenchmarkCycle", 0);
grpc_call* call = grpc_channel_create_registered_call(
fixture.channel(), nullptr, GRPC_PROPAGATE_DEFAULTS, fixture.cq(),
- method_hdl, deadline, NULL);
- grpc_call_start_batch(call, ops, 6, tag(1), NULL);
+ method_hdl, deadline, nullptr);
+ grpc_call_start_batch(call, ops, 6, tag(1), nullptr);
grpc_completion_queue_next(fixture.cq(),
- gpr_inf_future(GPR_CLOCK_MONOTONIC), NULL);
+ gpr_inf_future(GPR_CLOCK_MONOTONIC), nullptr);
grpc_call_unref(call);
}
fixture.Finish(state);
@@ -796,8 +797,8 @@ BENCHMARK(BM_IsolatedCall_Unary);
static void BM_IsolatedCall_StreamingSend(benchmark::State& state) {
IsolatedCallFixture fixture;
gpr_timespec deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
- void* method_hdl =
- grpc_channel_register_call(fixture.channel(), "/foo/bar", NULL, NULL);
+ void* method_hdl = grpc_channel_register_call(fixture.channel(), "/foo/bar",
+ nullptr, nullptr);
grpc_slice slice = grpc_slice_from_static_string("hello world");
grpc_byte_buffer* send_message = grpc_raw_byte_buffer_create(&slice, 1);
grpc_metadata_array recv_initial_metadata;
@@ -812,18 +813,18 @@ static void BM_IsolatedCall_StreamingSend(benchmark::State& state) {
&recv_initial_metadata;
grpc_call* call = grpc_channel_create_registered_call(
fixture.channel(), nullptr, GRPC_PROPAGATE_DEFAULTS, fixture.cq(),
- method_hdl, deadline, NULL);
- grpc_call_start_batch(call, ops, 2, tag(1), NULL);
+ method_hdl, deadline, nullptr);
+ grpc_call_start_batch(call, ops, 2, tag(1), nullptr);
grpc_completion_queue_next(fixture.cq(), gpr_inf_future(GPR_CLOCK_MONOTONIC),
- NULL);
+ nullptr);
memset(ops, 0, sizeof(ops));
ops[0].op = GRPC_OP_SEND_MESSAGE;
ops[0].data.send_message.send_message = send_message;
while (state.KeepRunning()) {
GPR_TIMER_SCOPE("BenchmarkCycle", 0);
- grpc_call_start_batch(call, ops, 1, tag(2), NULL);
+ grpc_call_start_batch(call, ops, 1, tag(2), nullptr);
grpc_completion_queue_next(fixture.cq(),
- gpr_inf_future(GPR_CLOCK_MONOTONIC), NULL);
+ gpr_inf_future(GPR_CLOCK_MONOTONIC), nullptr);
}
grpc_call_unref(call);
fixture.Finish(state);
diff --git a/test/cpp/microbenchmarks/bm_chttp2_hpack.cc b/test/cpp/microbenchmarks/bm_chttp2_hpack.cc
index 45f5382fde..3fff8b02d6 100644
--- a/test/cpp/microbenchmarks/bm_chttp2_hpack.cc
+++ b/test/cpp/microbenchmarks/bm_chttp2_hpack.cc
@@ -85,7 +85,7 @@ static void BM_HpackEncoderEncodeDeadline(benchmark::State& state) {
(size_t)1024,
&stats,
};
- grpc_chttp2_encode_header(&exec_ctx, &c, NULL, 0, &b, &hopt, &outbuf);
+ grpc_chttp2_encode_header(&exec_ctx, &c, nullptr, 0, &b, &hopt, &outbuf);
grpc_slice_buffer_reset_and_unref_internal(&exec_ctx, &outbuf);
grpc_exec_ctx_flush(&exec_ctx);
}
@@ -136,7 +136,7 @@ static void BM_HpackEncoderEncodeHeader(benchmark::State& state) {
(size_t)state.range(1),
&stats,
};
- grpc_chttp2_encode_header(&exec_ctx, &c, NULL, 0, &b, &hopt, &outbuf);
+ grpc_chttp2_encode_header(&exec_ctx, &c, nullptr, 0, &b, &hopt, &outbuf);
if (!logged_representative_output && state.iterations() > 3) {
logged_representative_output = true;
for (size_t i = 0; i < outbuf.count; i++) {
@@ -775,7 +775,7 @@ static void OnHeaderNew(grpc_exec_ctx* exec_ctx, void* user_data,
grpc_millis* cached_timeout =
static_cast<grpc_millis*>(grpc_mdelem_get_user_data(md, free_timeout));
grpc_millis timeout;
- if (cached_timeout != NULL) {
+ if (cached_timeout != nullptr) {
timeout = *cached_timeout;
} else {
if (!grpc_http2_decode_timeout(GRPC_MDVALUE(md), &timeout)) {
diff --git a/test/cpp/microbenchmarks/bm_chttp2_transport.cc b/test/cpp/microbenchmarks/bm_chttp2_transport.cc
index 154cc91778..be4da4d0bd 100644
--- a/test/cpp/microbenchmarks/bm_chttp2_transport.cc
+++ b/test/cpp/microbenchmarks/bm_chttp2_transport.cc
@@ -100,7 +100,7 @@ class DummyEndpoint : public grpc_endpoint {
GRPC_CLOSURE_SCHED(exec_ctx, cb, GRPC_ERROR_NONE);
}
- static grpc_workqueue* get_workqueue(grpc_endpoint* ep) { return NULL; }
+ static grpc_workqueue* get_workqueue(grpc_endpoint* ep) { return nullptr; }
static void add_to_pollset(grpc_exec_ctx* exec_ctx, grpc_endpoint* ep,
grpc_pollset* pollset) {}
@@ -137,7 +137,7 @@ class Fixture {
grpc_channel_args c_args = args.c_channel_args();
ep_ = new DummyEndpoint;
t_ = grpc_create_chttp2_transport(exec_ctx(), &c_args, ep_, client);
- grpc_chttp2_transport_start_reading(exec_ctx(), t_, NULL);
+ grpc_chttp2_transport_start_reading(exec_ctx(), t_, nullptr, nullptr);
FlushExecCtx();
}
@@ -222,7 +222,7 @@ class Stream {
}
grpc_transport_init_stream(f_->exec_ctx(), f_->transport(),
static_cast<grpc_stream*>(stream_), &refcount_,
- NULL, arena_);
+ nullptr, arena_);
}
void DestroyThen(grpc_exec_ctx* exec_ctx, grpc_closure* closure) {
@@ -586,7 +586,7 @@ static void BM_TransportStreamRecv(benchmark::State& state) {
});
drain_start = MakeClosure([&](grpc_exec_ctx* exec_ctx, grpc_error* error) {
- if (recv_stream == NULL) {
+ if (recv_stream == nullptr) {
GPR_ASSERT(!state.KeepRunning());
return;
}
diff --git a/test/cpp/microbenchmarks/bm_closure.cc b/test/cpp/microbenchmarks/bm_closure.cc
index 458a2ceb13..2434d4e84e 100644
--- a/test/cpp/microbenchmarks/bm_closure.cc
+++ b/test/cpp/microbenchmarks/bm_closure.cc
@@ -59,7 +59,7 @@ static void BM_ClosureInitAgainstExecCtx(benchmark::State& state) {
grpc_closure c;
while (state.KeepRunning()) {
benchmark::DoNotOptimize(
- GRPC_CLOSURE_INIT(&c, DoNothing, NULL, grpc_schedule_on_exec_ctx));
+ GRPC_CLOSURE_INIT(&c, DoNothing, nullptr, grpc_schedule_on_exec_ctx));
}
track_counters.Finish(state);
}
@@ -72,7 +72,7 @@ static void BM_ClosureInitAgainstCombiner(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
benchmark::DoNotOptimize(GRPC_CLOSURE_INIT(
- &c, DoNothing, NULL, grpc_combiner_scheduler(combiner)));
+ &c, DoNothing, nullptr, grpc_combiner_scheduler(combiner)));
}
GRPC_COMBINER_UNREF(&exec_ctx, combiner, "finished");
grpc_exec_ctx_finish(&exec_ctx);
@@ -83,7 +83,7 @@ BENCHMARK(BM_ClosureInitAgainstCombiner);
static void BM_ClosureRunOnExecCtx(benchmark::State& state) {
TrackCounters track_counters;
grpc_closure c;
- GRPC_CLOSURE_INIT(&c, DoNothing, NULL, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_RUN(&exec_ctx, &c, GRPC_ERROR_NONE);
@@ -100,7 +100,7 @@ static void BM_ClosureCreateAndRun(benchmark::State& state) {
while (state.KeepRunning()) {
GRPC_CLOSURE_RUN(
&exec_ctx,
- GRPC_CLOSURE_CREATE(DoNothing, NULL, grpc_schedule_on_exec_ctx),
+ GRPC_CLOSURE_CREATE(DoNothing, nullptr, grpc_schedule_on_exec_ctx),
GRPC_ERROR_NONE);
}
grpc_exec_ctx_finish(&exec_ctx);
@@ -115,7 +115,7 @@ static void BM_ClosureInitAndRun(benchmark::State& state) {
while (state.KeepRunning()) {
GRPC_CLOSURE_RUN(
&exec_ctx,
- GRPC_CLOSURE_INIT(&c, DoNothing, NULL, grpc_schedule_on_exec_ctx),
+ GRPC_CLOSURE_INIT(&c, DoNothing, nullptr, grpc_schedule_on_exec_ctx),
GRPC_ERROR_NONE);
}
grpc_exec_ctx_finish(&exec_ctx);
@@ -126,7 +126,7 @@ BENCHMARK(BM_ClosureInitAndRun);
static void BM_ClosureSchedOnExecCtx(benchmark::State& state) {
TrackCounters track_counters;
grpc_closure c;
- GRPC_CLOSURE_INIT(&c, DoNothing, NULL, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c, GRPC_ERROR_NONE);
@@ -141,8 +141,8 @@ static void BM_ClosureSched2OnExecCtx(benchmark::State& state) {
TrackCounters track_counters;
grpc_closure c1;
grpc_closure c2;
- GRPC_CLOSURE_INIT(&c1, DoNothing, NULL, grpc_schedule_on_exec_ctx);
- GRPC_CLOSURE_INIT(&c2, DoNothing, NULL, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c1, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c2, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c1, GRPC_ERROR_NONE);
@@ -159,9 +159,9 @@ static void BM_ClosureSched3OnExecCtx(benchmark::State& state) {
grpc_closure c1;
grpc_closure c2;
grpc_closure c3;
- GRPC_CLOSURE_INIT(&c1, DoNothing, NULL, grpc_schedule_on_exec_ctx);
- GRPC_CLOSURE_INIT(&c2, DoNothing, NULL, grpc_schedule_on_exec_ctx);
- GRPC_CLOSURE_INIT(&c3, DoNothing, NULL, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c1, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c2, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&c3, DoNothing, nullptr, grpc_schedule_on_exec_ctx);
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c1, GRPC_ERROR_NONE);
@@ -182,7 +182,7 @@ static void BM_AcquireMutex(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
gpr_mu_lock(&mu);
- DoNothing(&exec_ctx, NULL, GRPC_ERROR_NONE);
+ DoNothing(&exec_ctx, nullptr, GRPC_ERROR_NONE);
gpr_mu_unlock(&mu);
}
grpc_exec_ctx_finish(&exec_ctx);
@@ -198,7 +198,7 @@ static void BM_TryAcquireMutex(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
if (gpr_mu_trylock(&mu)) {
- DoNothing(&exec_ctx, NULL, GRPC_ERROR_NONE);
+ DoNothing(&exec_ctx, nullptr, GRPC_ERROR_NONE);
gpr_mu_unlock(&mu);
} else {
abort();
@@ -216,7 +216,7 @@ static void BM_AcquireSpinlock(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
gpr_spinlock_lock(&mu);
- DoNothing(&exec_ctx, NULL, GRPC_ERROR_NONE);
+ DoNothing(&exec_ctx, nullptr, GRPC_ERROR_NONE);
gpr_spinlock_unlock(&mu);
}
grpc_exec_ctx_finish(&exec_ctx);
@@ -231,7 +231,7 @@ static void BM_TryAcquireSpinlock(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
if (gpr_spinlock_trylock(&mu)) {
- DoNothing(&exec_ctx, NULL, GRPC_ERROR_NONE);
+ DoNothing(&exec_ctx, nullptr, GRPC_ERROR_NONE);
gpr_spinlock_unlock(&mu);
} else {
abort();
@@ -246,7 +246,7 @@ static void BM_ClosureSchedOnCombiner(benchmark::State& state) {
TrackCounters track_counters;
grpc_combiner* combiner = grpc_combiner_create();
grpc_closure c;
- GRPC_CLOSURE_INIT(&c, DoNothing, NULL, grpc_combiner_scheduler(combiner));
+ GRPC_CLOSURE_INIT(&c, DoNothing, nullptr, grpc_combiner_scheduler(combiner));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c, GRPC_ERROR_NONE);
@@ -263,8 +263,8 @@ static void BM_ClosureSched2OnCombiner(benchmark::State& state) {
grpc_combiner* combiner = grpc_combiner_create();
grpc_closure c1;
grpc_closure c2;
- GRPC_CLOSURE_INIT(&c1, DoNothing, NULL, grpc_combiner_scheduler(combiner));
- GRPC_CLOSURE_INIT(&c2, DoNothing, NULL, grpc_combiner_scheduler(combiner));
+ GRPC_CLOSURE_INIT(&c1, DoNothing, nullptr, grpc_combiner_scheduler(combiner));
+ GRPC_CLOSURE_INIT(&c2, DoNothing, nullptr, grpc_combiner_scheduler(combiner));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c1, GRPC_ERROR_NONE);
@@ -283,9 +283,9 @@ static void BM_ClosureSched3OnCombiner(benchmark::State& state) {
grpc_closure c1;
grpc_closure c2;
grpc_closure c3;
- GRPC_CLOSURE_INIT(&c1, DoNothing, NULL, grpc_combiner_scheduler(combiner));
- GRPC_CLOSURE_INIT(&c2, DoNothing, NULL, grpc_combiner_scheduler(combiner));
- GRPC_CLOSURE_INIT(&c3, DoNothing, NULL, grpc_combiner_scheduler(combiner));
+ GRPC_CLOSURE_INIT(&c1, DoNothing, nullptr, grpc_combiner_scheduler(combiner));
+ GRPC_CLOSURE_INIT(&c2, DoNothing, nullptr, grpc_combiner_scheduler(combiner));
+ GRPC_CLOSURE_INIT(&c3, DoNothing, nullptr, grpc_combiner_scheduler(combiner));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c1, GRPC_ERROR_NONE);
@@ -305,8 +305,10 @@ static void BM_ClosureSched2OnTwoCombiners(benchmark::State& state) {
grpc_combiner* combiner2 = grpc_combiner_create();
grpc_closure c1;
grpc_closure c2;
- GRPC_CLOSURE_INIT(&c1, DoNothing, NULL, grpc_combiner_scheduler(combiner1));
- GRPC_CLOSURE_INIT(&c2, DoNothing, NULL, grpc_combiner_scheduler(combiner2));
+ GRPC_CLOSURE_INIT(&c1, DoNothing, nullptr,
+ grpc_combiner_scheduler(combiner1));
+ GRPC_CLOSURE_INIT(&c2, DoNothing, nullptr,
+ grpc_combiner_scheduler(combiner2));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c1, GRPC_ERROR_NONE);
@@ -328,10 +330,14 @@ static void BM_ClosureSched4OnTwoCombiners(benchmark::State& state) {
grpc_closure c2;
grpc_closure c3;
grpc_closure c4;
- GRPC_CLOSURE_INIT(&c1, DoNothing, NULL, grpc_combiner_scheduler(combiner1));
- GRPC_CLOSURE_INIT(&c2, DoNothing, NULL, grpc_combiner_scheduler(combiner2));
- GRPC_CLOSURE_INIT(&c3, DoNothing, NULL, grpc_combiner_scheduler(combiner1));
- GRPC_CLOSURE_INIT(&c4, DoNothing, NULL, grpc_combiner_scheduler(combiner2));
+ GRPC_CLOSURE_INIT(&c1, DoNothing, nullptr,
+ grpc_combiner_scheduler(combiner1));
+ GRPC_CLOSURE_INIT(&c2, DoNothing, nullptr,
+ grpc_combiner_scheduler(combiner2));
+ GRPC_CLOSURE_INIT(&c3, DoNothing, nullptr,
+ grpc_combiner_scheduler(combiner1));
+ GRPC_CLOSURE_INIT(&c4, DoNothing, nullptr,
+ grpc_combiner_scheduler(combiner2));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
GRPC_CLOSURE_SCHED(&exec_ctx, &c1, GRPC_ERROR_NONE);
diff --git a/test/cpp/microbenchmarks/bm_cq.cc b/test/cpp/microbenchmarks/bm_cq.cc
index dac702b08b..f0dede7333 100644
--- a/test/cpp/microbenchmarks/bm_cq.cc
+++ b/test/cpp/microbenchmarks/bm_cq.cc
@@ -47,7 +47,7 @@ static void BM_CreateDestroyCpp2(benchmark::State& state) {
TrackCounters track_counters;
while (state.KeepRunning()) {
grpc_completion_queue* core_cq =
- grpc_completion_queue_create_for_next(NULL);
+ grpc_completion_queue_create_for_next(nullptr);
CompletionQueue cq(core_cq);
}
track_counters.Finish(state);
@@ -59,7 +59,8 @@ static void BM_CreateDestroyCore(benchmark::State& state) {
while (state.KeepRunning()) {
// TODO: sreek Templatize this benchmark and pass completion type and
// polling type as parameters
- grpc_completion_queue_destroy(grpc_completion_queue_create_for_next(NULL));
+ grpc_completion_queue_destroy(
+ grpc_completion_queue_create_for_next(nullptr));
}
track_counters.Finish(state);
}
@@ -83,7 +84,7 @@ static void BM_Pass1Cpp(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
GPR_ASSERT(grpc_cq_begin_op(c_cq, &dummy_tag));
grpc_cq_end_op(&exec_ctx, c_cq, &dummy_tag, GRPC_ERROR_NONE,
- DoneWithCompletionOnStack, NULL, &completion);
+ DoneWithCompletionOnStack, nullptr, &completion);
grpc_exec_ctx_finish(&exec_ctx);
void* tag;
bool ok;
@@ -96,16 +97,16 @@ BENCHMARK(BM_Pass1Cpp);
static void BM_Pass1Core(benchmark::State& state) {
TrackCounters track_counters;
// TODO: sreek Templatize this benchmark and pass polling_type as a param
- grpc_completion_queue* cq = grpc_completion_queue_create_for_next(NULL);
+ grpc_completion_queue* cq = grpc_completion_queue_create_for_next(nullptr);
gpr_timespec deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
while (state.KeepRunning()) {
grpc_cq_completion completion;
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
- GPR_ASSERT(grpc_cq_begin_op(cq, NULL));
- grpc_cq_end_op(&exec_ctx, cq, NULL, GRPC_ERROR_NONE,
- DoneWithCompletionOnStack, NULL, &completion);
+ GPR_ASSERT(grpc_cq_begin_op(cq, nullptr));
+ grpc_cq_end_op(&exec_ctx, cq, nullptr, GRPC_ERROR_NONE,
+ DoneWithCompletionOnStack, nullptr, &completion);
grpc_exec_ctx_finish(&exec_ctx);
- grpc_completion_queue_next(cq, deadline, NULL);
+ grpc_completion_queue_next(cq, deadline, nullptr);
}
grpc_completion_queue_destroy(cq);
track_counters.Finish(state);
@@ -115,16 +116,16 @@ BENCHMARK(BM_Pass1Core);
static void BM_Pluck1Core(benchmark::State& state) {
TrackCounters track_counters;
// TODO: sreek Templatize this benchmark and pass polling_type as a param
- grpc_completion_queue* cq = grpc_completion_queue_create_for_pluck(NULL);
+ grpc_completion_queue* cq = grpc_completion_queue_create_for_pluck(nullptr);
gpr_timespec deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
while (state.KeepRunning()) {
grpc_cq_completion completion;
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
- GPR_ASSERT(grpc_cq_begin_op(cq, NULL));
- grpc_cq_end_op(&exec_ctx, cq, NULL, GRPC_ERROR_NONE,
- DoneWithCompletionOnStack, NULL, &completion);
+ GPR_ASSERT(grpc_cq_begin_op(cq, nullptr));
+ grpc_cq_end_op(&exec_ctx, cq, nullptr, GRPC_ERROR_NONE,
+ DoneWithCompletionOnStack, nullptr, &completion);
grpc_exec_ctx_finish(&exec_ctx);
- grpc_completion_queue_pluck(cq, NULL, deadline, NULL);
+ grpc_completion_queue_pluck(cq, nullptr, deadline, nullptr);
}
grpc_completion_queue_destroy(cq);
track_counters.Finish(state);
@@ -134,10 +135,10 @@ BENCHMARK(BM_Pluck1Core);
static void BM_EmptyCore(benchmark::State& state) {
TrackCounters track_counters;
// TODO: sreek Templatize this benchmark and pass polling_type as a param
- grpc_completion_queue* cq = grpc_completion_queue_create_for_next(NULL);
+ grpc_completion_queue* cq = grpc_completion_queue_create_for_next(nullptr);
gpr_timespec deadline = gpr_inf_past(GPR_CLOCK_MONOTONIC);
while (state.KeepRunning()) {
- grpc_completion_queue_next(cq, deadline, NULL);
+ grpc_completion_queue_next(cq, deadline, nullptr);
}
grpc_completion_queue_destroy(cq);
track_counters.Finish(state);
diff --git a/test/cpp/microbenchmarks/bm_cq_multiple_threads.cc b/test/cpp/microbenchmarks/bm_cq_multiple_threads.cc
index 8d4349a297..7ccebb55ee 100644
--- a/test/cpp/microbenchmarks/bm_cq_multiple_threads.cc
+++ b/test/cpp/microbenchmarks/bm_cq_multiple_threads.cc
@@ -80,7 +80,7 @@ static grpc_error* pollset_work(grpc_exec_ctx* exec_ctx, grpc_pollset* ps,
gpr_mu_unlock(&ps->mu);
GPR_ASSERT(grpc_cq_begin_op(g_cq, g_tag));
- grpc_cq_end_op(exec_ctx, g_cq, g_tag, GRPC_ERROR_NONE, cq_done_cb, NULL,
+ grpc_cq_end_op(exec_ctx, g_cq, g_tag, GRPC_ERROR_NONE, cq_done_cb, nullptr,
(grpc_cq_completion*)gpr_malloc(sizeof(grpc_cq_completion)));
grpc_exec_ctx_flush(exec_ctx);
gpr_mu_lock(&ps->mu);
@@ -108,7 +108,7 @@ static void setup() {
g_old_vtable = grpc_get_event_engine_test_only();
grpc_set_event_engine_test_only(&g_vtable);
- g_cq = grpc_completion_queue_create_for_next(NULL);
+ g_cq = grpc_completion_queue_create_for_next(nullptr);
}
static void teardown() {
@@ -116,7 +116,7 @@ static void teardown() {
/* Drain any events */
gpr_timespec deadline = gpr_time_0(GPR_CLOCK_MONOTONIC);
- while (grpc_completion_queue_next(g_cq, deadline, NULL).type !=
+ while (grpc_completion_queue_next(g_cq, deadline, nullptr).type !=
GRPC_QUEUE_SHUTDOWN) {
/* Do nothing */
}
@@ -151,7 +151,7 @@ static void BM_Cq_Throughput(benchmark::State& state) {
}
while (state.KeepRunning()) {
- GPR_ASSERT(grpc_completion_queue_next(g_cq, deadline, NULL).type ==
+ GPR_ASSERT(grpc_completion_queue_next(g_cq, deadline, nullptr).type ==
GRPC_OP_COMPLETE);
}
diff --git a/test/cpp/microbenchmarks/bm_error.cc b/test/cpp/microbenchmarks/bm_error.cc
index aa7822653f..bbd8b3c339 100644
--- a/test/cpp/microbenchmarks/bm_error.cc
+++ b/test/cpp/microbenchmarks/bm_error.cc
@@ -251,7 +251,7 @@ static void BM_ErrorGetStatus(benchmark::State& state) {
grpc_status_code status;
grpc_slice slice;
grpc_error_get_status(&exec_ctx, fixture.error(), fixture.deadline(),
- &status, &slice, NULL);
+ &status, &slice, nullptr, nullptr);
}
grpc_exec_ctx_finish(&exec_ctx);
track_counters.Finish(state);
@@ -265,7 +265,7 @@ static void BM_ErrorGetStatusCode(benchmark::State& state) {
while (state.KeepRunning()) {
grpc_status_code status;
grpc_error_get_status(&exec_ctx, fixture.error(), fixture.deadline(),
- &status, NULL, NULL);
+ &status, nullptr, nullptr, nullptr);
}
grpc_exec_ctx_finish(&exec_ctx);
track_counters.Finish(state);
@@ -278,8 +278,8 @@ static void BM_ErrorHttpError(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
grpc_http2_error_code error;
- grpc_error_get_status(&exec_ctx, fixture.error(), fixture.deadline(), NULL,
- NULL, &error);
+ grpc_error_get_status(&exec_ctx, fixture.error(), fixture.deadline(),
+ nullptr, nullptr, &error, nullptr);
}
grpc_exec_ctx_finish(&exec_ctx);
track_counters.Finish(state);
diff --git a/test/cpp/microbenchmarks/bm_fullstack_trickle.cc b/test/cpp/microbenchmarks/bm_fullstack_trickle.cc
index f75c3e4436..5e72213823 100644
--- a/test/cpp/microbenchmarks/bm_fullstack_trickle.cc
+++ b/test/cpp/microbenchmarks/bm_fullstack_trickle.cc
@@ -216,10 +216,10 @@ class TrickledCHTTP2 : public EndpointPairFixture {
void UpdateStats(grpc_chttp2_transport* t, Stats* s,
size_t backlog) GPR_ATTRIBUTE_NO_TSAN {
if (backlog == 0) {
- if (t->lists[GRPC_CHTTP2_LIST_STALLED_BY_STREAM].head != NULL) {
+ if (t->lists[GRPC_CHTTP2_LIST_STALLED_BY_STREAM].head != nullptr) {
s->streams_stalled_due_to_stream_flow_control++;
}
- if (t->lists[GRPC_CHTTP2_LIST_STALLED_BY_TRANSPORT].head != NULL) {
+ if (t->lists[GRPC_CHTTP2_LIST_STALLED_BY_TRANSPORT].head != nullptr) {
s->streams_stalled_due_to_transport_flow_control++;
}
}
@@ -445,7 +445,7 @@ BENCHMARK(BM_PumpUnbalancedUnary_Trickle)->Apply(UnaryTrickleArgs);
} // namespace testing
} // namespace grpc
-extern "C" gpr_timespec (*gpr_now_impl)(gpr_clock_type clock_type);
+extern gpr_timespec (*gpr_now_impl)(gpr_clock_type clock_type);
int main(int argc, char** argv) {
::benchmark::Initialize(&argc, argv);
diff --git a/test/cpp/microbenchmarks/bm_metadata.cc b/test/cpp/microbenchmarks/bm_metadata.cc
index 1ed05f7466..73bce08466 100644
--- a/test/cpp/microbenchmarks/bm_metadata.cc
+++ b/test/cpp/microbenchmarks/bm_metadata.cc
@@ -92,7 +92,7 @@ static void BM_MetadataFromNonInternedSlices(benchmark::State& state) {
gpr_slice v = grpc_slice_from_static_string("value");
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
- GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, NULL));
+ GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, nullptr));
}
grpc_exec_ctx_finish(&exec_ctx);
track_counters.Finish(state);
@@ -105,7 +105,7 @@ static void BM_MetadataFromInternedSlices(benchmark::State& state) {
gpr_slice v = grpc_slice_intern(grpc_slice_from_static_string("value"));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
- GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, NULL));
+ GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, nullptr));
}
grpc_exec_ctx_finish(&exec_ctx);
grpc_slice_unref(k);
@@ -120,9 +120,9 @@ static void BM_MetadataFromInternedSlicesAlreadyInIndex(
gpr_slice k = grpc_slice_intern(grpc_slice_from_static_string("key"));
gpr_slice v = grpc_slice_intern(grpc_slice_from_static_string("value"));
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
- grpc_mdelem seed = grpc_mdelem_create(&exec_ctx, k, v, NULL);
+ grpc_mdelem seed = grpc_mdelem_create(&exec_ctx, k, v, nullptr);
while (state.KeepRunning()) {
- GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, NULL));
+ GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, nullptr));
}
GRPC_MDELEM_UNREF(&exec_ctx, seed);
grpc_exec_ctx_finish(&exec_ctx);
@@ -138,7 +138,7 @@ static void BM_MetadataFromInternedKey(benchmark::State& state) {
gpr_slice v = grpc_slice_from_static_string("value");
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
- GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, NULL));
+ GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, nullptr));
}
grpc_exec_ctx_finish(&exec_ctx);
grpc_slice_unref(k);
@@ -209,7 +209,7 @@ static void BM_MetadataFromStaticMetadataStrings(benchmark::State& state) {
gpr_slice v = GRPC_MDSTR_200;
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
- GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, NULL));
+ GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, nullptr));
}
grpc_exec_ctx_finish(&exec_ctx);
grpc_slice_unref(k);
@@ -224,7 +224,7 @@ static void BM_MetadataFromStaticMetadataStringsNotIndexed(
gpr_slice v = GRPC_MDSTR_GZIP;
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (state.KeepRunning()) {
- GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, NULL));
+ GRPC_MDELEM_UNREF(&exec_ctx, grpc_mdelem_create(&exec_ctx, k, v, nullptr));
}
grpc_exec_ctx_finish(&exec_ctx);
grpc_slice_unref(k);
@@ -273,7 +273,7 @@ static void BM_MetadataRefUnrefAllocated(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
grpc_mdelem el =
grpc_mdelem_create(&exec_ctx, grpc_slice_from_static_string("a"),
- grpc_slice_from_static_string("b"), NULL);
+ grpc_slice_from_static_string("b"), nullptr);
while (state.KeepRunning()) {
GRPC_MDELEM_UNREF(&exec_ctx, GRPC_MDELEM_REF(el));
}
@@ -287,7 +287,7 @@ static void BM_MetadataRefUnrefStatic(benchmark::State& state) {
TrackCounters track_counters;
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
grpc_mdelem el =
- grpc_mdelem_create(&exec_ctx, GRPC_MDSTR_STATUS, GRPC_MDSTR_200, NULL);
+ grpc_mdelem_create(&exec_ctx, GRPC_MDSTR_STATUS, GRPC_MDSTR_200, nullptr);
while (state.KeepRunning()) {
GRPC_MDELEM_UNREF(&exec_ctx, GRPC_MDELEM_REF(el));
}
diff --git a/test/cpp/microbenchmarks/bm_pollset.cc b/test/cpp/microbenchmarks/bm_pollset.cc
index 92d76f307e..4da79693f1 100644
--- a/test/cpp/microbenchmarks/bm_pollset.cc
+++ b/test/cpp/microbenchmarks/bm_pollset.cc
@@ -117,7 +117,7 @@ static void BM_PollEmptyPollset(benchmark::State& state) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
gpr_mu_lock(mu);
while (state.KeepRunning()) {
- GRPC_ERROR_UNREF(grpc_pollset_work(&exec_ctx, ps, NULL, 0));
+ GRPC_ERROR_UNREF(grpc_pollset_work(&exec_ctx, ps, nullptr, 0));
}
grpc_closure shutdown_ps_closure;
GRPC_CLOSURE_INIT(&shutdown_ps_closure, shutdown_ps, ps,
@@ -145,7 +145,8 @@ static void BM_PollAddFd(benchmark::State& state) {
grpc_pollset_add_fd(&exec_ctx, ps, fd);
grpc_exec_ctx_flush(&exec_ctx);
}
- grpc_fd_orphan(&exec_ctx, fd, NULL, NULL, false /* already_closed */, "xxx");
+ grpc_fd_orphan(&exec_ctx, fd, nullptr, nullptr, false /* already_closed */,
+ "xxx");
grpc_closure shutdown_ps_closure;
GRPC_CLOSURE_INIT(&shutdown_ps_closure, shutdown_ps, ps,
grpc_schedule_on_exec_ctx);
@@ -240,10 +241,10 @@ static void BM_SingleThreadPollOneFd(benchmark::State& state) {
gpr_mu_lock(mu);
while (!done) {
GRPC_ERROR_UNREF(
- grpc_pollset_work(&exec_ctx, ps, NULL, GRPC_MILLIS_INF_FUTURE));
+ grpc_pollset_work(&exec_ctx, ps, nullptr, GRPC_MILLIS_INF_FUTURE));
}
- grpc_fd_orphan(&exec_ctx, wakeup, NULL, NULL, false /* already_closed */,
- "done");
+ grpc_fd_orphan(&exec_ctx, wakeup, nullptr, nullptr,
+ false /* already_closed */, "done");
wakeup_fd.read_fd = 0;
grpc_closure shutdown_ps_closure;
GRPC_CLOSURE_INIT(&shutdown_ps_closure, shutdown_ps, ps,
diff --git a/test/cpp/microbenchmarks/fullstack_fixtures.h b/test/cpp/microbenchmarks/fullstack_fixtures.h
index 71bbb393db..7e20843875 100644
--- a/test/cpp/microbenchmarks/fullstack_fixtures.h
+++ b/test/cpp/microbenchmarks/fullstack_fixtures.h
@@ -174,7 +174,7 @@ class EndpointPairFixture : public BaseFixture {
const grpc_channel_args* server_args =
grpc_server_get_channel_args(server_->c_server());
server_transport_ = grpc_create_chttp2_transport(
- &exec_ctx, server_args, endpoints.server, 0 /* is_client */);
+ &exec_ctx, server_args, endpoints.server, false /* is_client */);
grpc_pollset** pollsets;
size_t num_pollsets = 0;
@@ -185,8 +185,9 @@ class EndpointPairFixture : public BaseFixture {
}
grpc_server_setup_transport(&exec_ctx, server_->c_server(),
- server_transport_, NULL, server_args);
- grpc_chttp2_transport_start_reading(&exec_ctx, server_transport_, NULL);
+ server_transport_, nullptr, server_args);
+ grpc_chttp2_transport_start_reading(&exec_ctx, server_transport_, nullptr,
+ nullptr);
}
/* create channel */
@@ -196,13 +197,14 @@ class EndpointPairFixture : public BaseFixture {
fixture_configuration.ApplyCommonChannelArguments(&args);
grpc_channel_args c_args = args.c_channel_args();
- client_transport_ =
- grpc_create_chttp2_transport(&exec_ctx, &c_args, endpoints.client, 1);
+ client_transport_ = grpc_create_chttp2_transport(&exec_ctx, &c_args,
+ endpoints.client, true);
GPR_ASSERT(client_transport_);
grpc_channel* channel =
grpc_channel_create(&exec_ctx, "target", &c_args,
GRPC_CLIENT_DIRECT_CHANNEL, client_transport_);
- grpc_chttp2_transport_start_reading(&exec_ctx, client_transport_, NULL);
+ grpc_chttp2_transport_start_reading(&exec_ctx, client_transport_, nullptr,
+ nullptr);
channel_ = CreateChannelInternal("", channel);
}
@@ -244,7 +246,7 @@ class SockPair : public EndpointPairFixture {
SockPair(Service* service, const FixtureConfiguration& fixture_configuration =
FixtureConfiguration())
: EndpointPairFixture(service,
- grpc_iomgr_create_endpoint_pair("test", NULL),
+ grpc_iomgr_create_endpoint_pair("test", nullptr),
fixture_configuration) {}
};
diff --git a/test/cpp/microbenchmarks/helpers.h b/test/cpp/microbenchmarks/helpers.h
index 07be589df6..afa3e0f3ca 100644
--- a/test/cpp/microbenchmarks/helpers.h
+++ b/test/cpp/microbenchmarks/helpers.h
@@ -54,10 +54,10 @@ class Library {
};
#ifdef GPR_LOW_LEVEL_COUNTERS
-extern "C" gpr_atm gpr_mu_locks;
-extern "C" gpr_atm gpr_counter_atm_cas;
-extern "C" gpr_atm gpr_counter_atm_add;
-extern "C" gpr_atm gpr_now_call_count;
+extern gpr_atm gpr_mu_locks;
+extern gpr_atm gpr_counter_atm_cas;
+extern gpr_atm gpr_counter_atm_add;
+extern gpr_atm gpr_now_call_count;
#endif
class TrackCounters {
diff --git a/test/cpp/naming/README.md b/test/cpp/naming/README.md
index e33184620c..e0dd208465 100644
--- a/test/cpp/naming/README.md
+++ b/test/cpp/naming/README.md
@@ -31,7 +31,7 @@ After making a change to `resolver_test_record_groups.yaml`:
3. From the repo root, run:
```
-$ test/cpp/naming/create_dns_private_zone.sh
+$ test/cpp/naming/create_private_dns_zone.sh
$ test/cpp/naming/private_dns_zone_init.sh
```
diff --git a/test/cpp/naming/create_private_dns_zone.sh b/test/cpp/naming/create_private_dns_zone.sh
index 3d7520b90a..55a4cfe36e 100755
--- a/test/cpp/naming/create_private_dns_zone.sh
+++ b/test/cpp/naming/create_private_dns_zone.sh
@@ -20,8 +20,8 @@ set -ex
cd $(dirname $0)/../../..
gcloud alpha dns managed-zones create \
- resolver-tests-version-1-grpctestingexp-zone-id \
- --dns-name=resolver-tests-version-1.grpctestingexp. \
+ resolver-tests-version-4-grpctestingexp-zone-id \
+ --dns-name=resolver-tests-version-4.grpctestingexp. \
--description="GCE-DNS-private-zone-for-GRPC-testing" \
--visibility=private \
--networks=default
diff --git a/test/cpp/naming/private_dns_zone_init.sh b/test/cpp/naming/private_dns_zone_init.sh
index 4eaf750ab7..8fa5a8a475 100755
--- a/test/cpp/naming/private_dns_zone_init.sh
+++ b/test/cpp/naming/private_dns_zone_init.sh
@@ -19,197 +19,197 @@ set -ex
cd $(dirname $0)/../../..
-gcloud dns record-sets transaction start -z=resolver-tests-version-1-grpctestingexp-zone-id
+gcloud dns record-sets transaction start -z=resolver-tests-version-4-grpctestingexp-zone-id
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv4-single-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv4-single-target.resolver-tests-version-4.grpctestingexp. \
--type=SRV \
--ttl=2100 \
- "0 0 1234 ipv4-single-target.resolver-tests-version-1.grpctestingexp."
+ "0 0 1234 ipv4-single-target.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-single-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-single-target.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv4-multi-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv4-multi-target.resolver-tests-version-4.grpctestingexp. \
--type=SRV \
--ttl=2100 \
- "0 0 1234 ipv4-multi-target.resolver-tests-version-1.grpctestingexp."
+ "0 0 1234 ipv4-multi-target.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-multi-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-multi-target.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.5" "1.2.3.6" "1.2.3.7"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv6-single-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv6-single-target.resolver-tests-version-4.grpctestingexp. \
--type=SRV \
--ttl=2100 \
- "0 0 1234 ipv6-single-target.resolver-tests-version-1.grpctestingexp."
+ "0 0 1234 ipv6-single-target.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv6-single-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv6-single-target.resolver-tests-version-4.grpctestingexp. \
--type=AAAA \
--ttl=2100 \
"2607:f8b0:400a:801::1001"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv6-multi-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv6-multi-target.resolver-tests-version-4.grpctestingexp. \
--type=SRV \
--ttl=2100 \
- "0 0 1234 ipv6-multi-target.resolver-tests-version-1.grpctestingexp."
+ "0 0 1234 ipv6-multi-target.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv6-multi-target.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv6-multi-target.resolver-tests-version-4.grpctestingexp. \
--type=AAAA \
--ttl=2100 \
"2607:f8b0:400a:801::1002" "2607:f8b0:400a:801::1003" "2607:f8b0:400a:801::1004"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpc_config.srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. \
--type=TXT \
--ttl=2100 \
'"grpc_config=[{\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"SimpleService\",\"waitForReady\":true}]}]}}]"'
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. \
- --type=A \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. \
+ --type=SRV \
--ttl=2100 \
- "1.2.3.4"
+ "0 0 1234 ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. \
- --type=SRV \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. \
+ --type=A \
--ttl=2100 \
- "0 0 1234 ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp."
+ "1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpc_config.ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. \
--type=TXT \
--ttl=2100 \
'"grpc_config=[{\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"NoSrvSimpleService\",\"waitForReady\":true}]}]}}]"'
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. \
- --type=A \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpc_config.ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. \
+ --type=TXT \
--ttl=2100 \
- "1.2.3.4"
+ '"grpc_config=[{\"clientLanguage\":[\"python\"],\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"PythonService\",\"waitForReady\":true}]}]}}]"'
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. \
- --type=TXT \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. \
+ --type=A \
--ttl=2100 \
- '"grpc_config=[{\"clientLanguage\":[\"python\"],\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"PythonService\",\"waitForReady\":true}]}]}}]"'
+ "1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpc_config.ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. \
--type=TXT \
--ttl=2100 \
'"grpc_config=[{\"percentage\":0,\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"CppService\",\"waitForReady\":true}]}]}}]"'
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. \
- --type=A \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpc_config.ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. \
+ --type=TXT \
--ttl=2100 \
- "1.2.3.4"
+ '"grpc_config=[{\"clientLanguage\":[\"go\"],\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"GoService\",\"waitForReady\":true}]}]}},{\"clientLanguage\":[\"c++\"],\"serviceConfig\":{" "\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"CppService\",\"waitForReady\":true}]}]}}]"'
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. \
- --type=TXT \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. \
+ --type=A \
--ttl=2100 \
- '"grpc_config=[{\"clientLanguage\":[\"go\"],\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"GoService\",\"waitForReady\":true}]}]}},{\"clientLanguage\":[\"c++\"],\"serviceConfig\":{" "\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"CppService\",\"waitForReady\":true}]}]}}]"'
+ "1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpc_config.ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. \
--type=TXT \
--ttl=2100 \
'"grpc_config=[{\"percentage\":0,\"serviceConfig\":{\"loadBalancingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"NeverPickedService\",\"waitForReady\":true}]}]}},{\"percentage\":100,\"serviceConfig\":{\"loadBalanc" "ingPolicy\":\"round_robin\",\"methodConfig\":[{\"name\":[{\"method\":\"Foo\",\"service\":\"AlwaysPickedService\",\"waitForReady\":true}]}]}}]"'
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. \
--type=SRV \
--ttl=2100 \
- "0 0 1234 balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp."
+ "0 0 1234 balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. \
--type=A \
--ttl=2100 \
"1.2.3.4"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=_grpclb._tcp.srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=_grpclb._tcp.srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. \
--type=SRV \
--ttl=2100 \
- "0 0 1234 balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp."
+ "0 0 1234 balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp."
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. \
--type=AAAA \
--ttl=2100 \
"2607:f8b0:400a:801::1002"
gcloud dns record-sets transaction add \
- -z=resolver-tests-version-1-grpctestingexp-zone-id \
- --name=srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. \
+ -z=resolver-tests-version-4-grpctestingexp-zone-id \
+ --name=srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. \
--type=AAAA \
--ttl=2100 \
"2607:f8b0:400a:801::1002"
-gcloud dns record-sets transaction describe -z=resolver-tests-version-1-grpctestingexp-zone-id
-gcloud dns record-sets transaction execute -z=resolver-tests-version-1-grpctestingexp-zone-id
-gcloud dns record-sets list -z=resolver-tests-version-1-grpctestingexp-zone-id
+gcloud dns record-sets transaction describe -z=resolver-tests-version-4-grpctestingexp-zone-id
+gcloud dns record-sets transaction execute -z=resolver-tests-version-4-grpctestingexp-zone-id
+gcloud dns record-sets list -z=resolver-tests-version-4-grpctestingexp-zone-id
diff --git a/test/cpp/naming/resolver_component_test.cc b/test/cpp/naming/resolver_component_test.cc
index 94adbcbd35..6f1f0c44b9 100644
--- a/test/cpp/naming/resolver_component_test.cc
+++ b/test/cpp/naming/resolver_component_test.cc
@@ -157,7 +157,7 @@ void ArgsInit(grpc_exec_ctx* exec_ctx, ArgsStruct* args) {
grpc_pollset_set_add_pollset(exec_ctx, args->pollset_set, args->pollset);
args->lock = grpc_combiner_create();
gpr_atm_rel_store(&args->done_atm, 0);
- args->channel_args = NULL;
+ args->channel_args = nullptr;
}
void DoNothing(grpc_exec_ctx* exec_ctx, void* arg, grpc_error* error) {}
@@ -167,7 +167,8 @@ void ArgsFinish(grpc_exec_ctx* exec_ctx, ArgsStruct* args) {
grpc_pollset_set_del_pollset(exec_ctx, args->pollset_set, args->pollset);
grpc_pollset_set_destroy(exec_ctx, args->pollset_set);
grpc_closure DoNothing_cb;
- GRPC_CLOSURE_INIT(&DoNothing_cb, DoNothing, NULL, grpc_schedule_on_exec_ctx);
+ GRPC_CLOSURE_INIT(&DoNothing_cb, DoNothing, nullptr,
+ grpc_schedule_on_exec_ctx);
grpc_pollset_shutdown(exec_ctx, args->pollset, &DoNothing_cb);
// exec_ctx needs to be flushed before calling grpc_pollset_destroy()
grpc_channel_args_destroy(exec_ctx, args->channel_args);
@@ -194,7 +195,7 @@ void PollPollsetUntilRequestDone(ArgsStruct* args) {
gpr_log(GPR_DEBUG, "done=%d, time_left=%" PRId64 ".%09d", done,
time_left.tv_sec, time_left.tv_nsec);
GPR_ASSERT(gpr_time_cmp(time_left, gpr_time_0(GPR_TIMESPAN)) >= 0);
- grpc_pollset_worker* worker = NULL;
+ grpc_pollset_worker* worker = nullptr;
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
gpr_mu_lock(args->mu);
GRPC_LOG_IF_ERROR("pollset_work",
@@ -212,12 +213,12 @@ void CheckServiceConfigResultLocked(grpc_channel_args* channel_args,
const grpc_arg* service_config_arg =
grpc_channel_args_find(channel_args, GRPC_ARG_SERVICE_CONFIG);
if (args->expected_service_config_string != "") {
- GPR_ASSERT(service_config_arg != NULL);
+ GPR_ASSERT(service_config_arg != nullptr);
GPR_ASSERT(service_config_arg->type == GRPC_ARG_STRING);
EXPECT_EQ(service_config_arg->value.string,
args->expected_service_config_string);
} else {
- GPR_ASSERT(service_config_arg == NULL);
+ GPR_ASSERT(service_config_arg == nullptr);
}
}
@@ -226,11 +227,11 @@ void CheckLBPolicyResultLocked(grpc_channel_args* channel_args,
const grpc_arg* lb_policy_arg =
grpc_channel_args_find(channel_args, GRPC_ARG_LB_POLICY_NAME);
if (args->expected_lb_policy != "") {
- GPR_ASSERT(lb_policy_arg != NULL);
+ GPR_ASSERT(lb_policy_arg != nullptr);
GPR_ASSERT(lb_policy_arg->type == GRPC_ARG_STRING);
EXPECT_EQ(lb_policy_arg->value.string, args->expected_lb_policy);
} else {
- GPR_ASSERT(lb_policy_arg == NULL);
+ GPR_ASSERT(lb_policy_arg == nullptr);
}
}
@@ -240,7 +241,7 @@ void CheckResolverResultLocked(grpc_exec_ctx* exec_ctx, void* argsp,
grpc_channel_args* channel_args = args->channel_args;
const grpc_arg* channel_arg =
grpc_channel_args_find(channel_args, GRPC_ARG_LB_ADDRESSES);
- GPR_ASSERT(channel_arg != NULL);
+ GPR_ASSERT(channel_arg != nullptr);
GPR_ASSERT(channel_arg->type == GRPC_ARG_POINTER);
grpc_lb_addresses* addresses =
(grpc_lb_addresses*)channel_arg->value.pointer.p;
@@ -272,7 +273,7 @@ void CheckResolverResultLocked(grpc_exec_ctx* exec_ctx, void* argsp,
gpr_atm_rel_store(&args->done_atm, 1);
gpr_mu_lock(args->mu);
GRPC_LOG_IF_ERROR("pollset_kick",
- grpc_pollset_kick(exec_ctx, args->pollset, NULL));
+ grpc_pollset_kick(exec_ctx, args->pollset, nullptr));
gpr_mu_unlock(args->mu);
}
@@ -284,12 +285,12 @@ TEST(ResolverComponentTest, TestResolvesRelevantRecords) {
args.expected_service_config_string = FLAGS_expected_chosen_service_config;
args.expected_lb_policy = FLAGS_expected_lb_policy;
// maybe build the address with an authority
- char* whole_uri = NULL;
+ char* whole_uri = nullptr;
GPR_ASSERT(asprintf(&whole_uri, "dns://%s/%s",
FLAGS_local_dns_server_address.c_str(),
FLAGS_target_name.c_str()));
// create resolver and resolve
- grpc_resolver* resolver = grpc_resolver_create(&exec_ctx, whole_uri, NULL,
+ grpc_resolver* resolver = grpc_resolver_create(&exec_ctx, whole_uri, nullptr,
args.pollset_set, args.lock);
gpr_free(whole_uri);
grpc_closure on_resolver_result_changed;
diff --git a/test/cpp/naming/resolver_component_tests_runner.sh b/test/cpp/naming/resolver_component_tests_runner.sh
index 407db5ed66..11a45d72ce 100755
--- a/test/cpp/naming/resolver_component_tests_runner.sh
+++ b/test/cpp/naming/resolver_component_tests_runner.sh
@@ -73,7 +73,7 @@ EXIT_CODE=0
# in the resolver.
$FLAGS_test_bin_path \
- --target_name='srv-ipv4-single-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-single-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -81,7 +81,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='srv-ipv4-multi-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-multi-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.5:1234,True;1.2.3.6:1234,True;1.2.3.7:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -89,7 +89,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='srv-ipv6-single-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv6-single-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='[2607:f8b0:400a:801::1001]:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -97,7 +97,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='srv-ipv6-multi-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv6-multi-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='[2607:f8b0:400a:801::1002]:1234,True;[2607:f8b0:400a:801::1003]:1234,True;[2607:f8b0:400a:801::1004]:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -105,7 +105,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:1234,True' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"SimpleService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' \
@@ -113,7 +113,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"NoSrvSimpleService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' \
@@ -121,7 +121,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -129,7 +129,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -137,7 +137,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"CppService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' \
@@ -145,7 +145,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"AlwaysPickedService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' \
@@ -153,7 +153,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:1234,True;1.2.3.4:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -161,7 +161,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='[2607:f8b0:400a:801::1002]:1234,True;[2607:f8b0:400a:801::1002]:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' \
@@ -169,7 +169,7 @@ $FLAGS_test_bin_path \
wait $! || EXIT_CODE=1
$FLAGS_test_bin_path \
- --target_name='ipv4-config-causing-fallback-to-tcp.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-config-causing-fallback-to-tcp.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwo","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooThree","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooFour","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooFive","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooSix","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooSeven","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooEight","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooNine","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTen","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooEleven","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]}]}' \
--expected_lb_policy='' \
diff --git a/test/cpp/naming/resolver_component_tests_runner_invoker.cc b/test/cpp/naming/resolver_component_tests_runner_invoker.cc
index 7ea005cc1d..0beb27de1b 100644
--- a/test/cpp/naming/resolver_component_tests_runner_invoker.cc
+++ b/test/cpp/naming/resolver_component_tests_runner_invoker.cc
@@ -60,8 +60,8 @@ static void register_sighandler() {
struct sigaction act;
memset(&act, 0, sizeof(act));
act.sa_handler = sighandler;
- sigaction(SIGINT, &act, NULL);
- sigaction(SIGTERM, &act, NULL);
+ sigaction(SIGINT, &act, nullptr);
+ sigaction(SIGTERM, &act, nullptr);
}
namespace {
diff --git a/test/cpp/naming/resolver_gce_integration_tests_runner.sh b/test/cpp/naming/resolver_gce_integration_tests_runner.sh
index b20d18d9d1..091f9efbbd 100755
--- a/test/cpp/naming/resolver_gce_integration_tests_runner.sh
+++ b/test/cpp/naming/resolver_gce_integration_tests_runner.sh
@@ -34,191 +34,191 @@ echo "Sanity check DNS records are resolveable with dig:"
EXIT_CODE=0
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv4-single-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv4-single-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-single-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-single-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-single-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-single-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-single-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-single-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv4-multi-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv4-multi-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-multi-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-multi-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-multi-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-multi-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-multi-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-multi-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv6-single-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv6-single-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv6-single-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv6-single-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig AAAA ipv6-single-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig AAAA ipv6-single-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig AAAA ipv6-single-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig AAAA ipv6-single-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv6-multi-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv6-multi-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv6-multi-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv6-multi-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig AAAA ipv6-multi-target.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig AAAA ipv6-multi-target.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig AAAA ipv6-multi-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig AAAA ipv6-multi-target.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig TXT srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig TXT _grpc_config.srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig TXT srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig TXT _grpc_config.srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig TXT ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig TXT _grpc_config.ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig TXT ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig TXT _grpc_config.ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig TXT _grpc_config.ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig TXT _grpc_config.ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig TXT ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig TXT ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig TXT ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig TXT _grpc_config.ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig TXT ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig TXT _grpc_config.ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig TXT _grpc_config.ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig TXT _grpc_config.ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig TXT ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig TXT ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig TXT ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig TXT _grpc_config.ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig TXT ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig TXT _grpc_config.ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A balancer-for-ipv4-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig A srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig A srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig A srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig A srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig SRV _grpclb._tcp.srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig SRV _grpclb._tcp.srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig SRV _grpclb._tcp.srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig AAAA balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig AAAA balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig AAAA balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig AAAA balancer-for-ipv6-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
ONE_FAILED=0
-dig AAAA srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
+dig AAAA srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. | grep 'ANSWER SECTION' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Sanity check: dig AAAA srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Sanity check: dig AAAA srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
exit 1
fi
@@ -226,133 +226,133 @@ echo "Sanity check PASSED. Run resolver tests:"
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv4-single-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-single-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv4-single-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv4-single-target.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv4-multi-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-multi-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.5:1234,True;1.2.3.6:1234,True;1.2.3.7:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv4-multi-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv4-multi-target.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv6-single-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv6-single-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='[2607:f8b0:400a:801::1001]:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv6-single-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv6-single-target.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv6-multi-target.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv6-multi-target.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='[2607:f8b0:400a:801::1002]:1234,True;[2607:f8b0:400a:801::1003]:1234,True;[2607:f8b0:400a:801::1004]:1234,True' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv6-multi-target.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv6-multi-target.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:1234,True' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"SimpleService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv4-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv4-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"NoSrvSimpleService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: ipv4-no-srv-simple-service-config.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: ipv4-no-srv-simple-service-config.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: ipv4-no-config-for-cpp.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: ipv4-no-config-for-cpp.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: ipv4-cpp-config-has-zero-percentage.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: ipv4-cpp-config-has-zero-percentage.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"CppService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: ipv4-second-language-is-cpp.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: ipv4-second-language-is-cpp.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:443,False' \
--expected_chosen_service_config='{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"AlwaysPickedService","waitForReady":true}]}]}' \
--expected_lb_policy='round_robin' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: ipv4-config-with-percentages.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: ipv4-config-with-percentages.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='1.2.3.4:1234,True;1.2.3.4:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv4-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
ONE_FAILED=0
bins/$CONFIG/resolver_component_test \
- --target_name='srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp.' \
+ --target_name='srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp.' \
--expected_addrs='[2607:f8b0:400a:801::1002]:1234,True;[2607:f8b0:400a:801::1002]:443,False' \
--expected_chosen_service_config='' \
--expected_lb_policy='' || ONE_FAILED=1
if [[ "$ONE_FAILED" != 0 ]]; then
- echo "Test based on target record: srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-1.grpctestingexp. FAILED"
+ echo "Test based on target record: srv-ipv6-target-has-backend-and-balancer.resolver-tests-version-4.grpctestingexp. FAILED"
EXIT_CODE=1
fi
diff --git a/test/cpp/naming/resolver_test_record_groups.yaml b/test/cpp/naming/resolver_test_record_groups.yaml
index 2b3204335c..6c4f89d09b 100644
--- a/test/cpp/naming/resolver_test_record_groups.yaml
+++ b/test/cpp/naming/resolver_test_record_groups.yaml
@@ -1,4 +1,4 @@
-resolver_tests_common_zone_name: resolver-tests-version-1.grpctestingexp.
+resolver_tests_common_zone_name: resolver-tests-version-4.grpctestingexp.
resolver_component_tests:
- expected_addrs:
- {address: '1.2.3.4:1234', is_balancer: true}
@@ -58,7 +58,7 @@ resolver_component_tests:
- {TTL: '2100', data: 0 0 1234 ipv4-simple-service-config, type: SRV}
ipv4-simple-service-config:
- {TTL: '2100', data: 1.2.3.4, type: A}
- srv-ipv4-simple-service-config:
+ _grpc_config.srv-ipv4-simple-service-config:
- {TTL: '2100', data: 'grpc_config=[{"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"SimpleService","waitForReady":true}]}]}}]',
type: TXT}
- expected_addrs:
@@ -69,6 +69,7 @@ resolver_component_tests:
records:
ipv4-no-srv-simple-service-config:
- {TTL: '2100', data: 1.2.3.4, type: A}
+ _grpc_config.ipv4-no-srv-simple-service-config:
- {TTL: '2100', data: 'grpc_config=[{"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"NoSrvSimpleService","waitForReady":true}]}]}}]',
type: TXT}
- expected_addrs:
@@ -79,6 +80,7 @@ resolver_component_tests:
records:
ipv4-no-config-for-cpp:
- {TTL: '2100', data: 1.2.3.4, type: A}
+ _grpc_config.ipv4-no-config-for-cpp:
- {TTL: '2100', data: 'grpc_config=[{"clientLanguage":["python"],"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"PythonService","waitForReady":true}]}]}}]',
type: TXT}
- expected_addrs:
@@ -89,6 +91,7 @@ resolver_component_tests:
records:
ipv4-cpp-config-has-zero-percentage:
- {TTL: '2100', data: 1.2.3.4, type: A}
+ _grpc_config.ipv4-cpp-config-has-zero-percentage:
- {TTL: '2100', data: 'grpc_config=[{"percentage":0,"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"CppService","waitForReady":true}]}]}}]',
type: TXT}
- expected_addrs:
@@ -99,6 +102,7 @@ resolver_component_tests:
records:
ipv4-second-language-is-cpp:
- {TTL: '2100', data: 1.2.3.4, type: A}
+ _grpc_config.ipv4-second-language-is-cpp:
- {TTL: '2100', data: 'grpc_config=[{"clientLanguage":["go"],"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"GoService","waitForReady":true}]}]}},{"clientLanguage":["c++"],"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"CppService","waitForReady":true}]}]}}]',
type: TXT}
- expected_addrs:
@@ -109,6 +113,7 @@ resolver_component_tests:
records:
ipv4-config-with-percentages:
- {TTL: '2100', data: 1.2.3.4, type: A}
+ _grpc_config.ipv4-config-with-percentages:
- {TTL: '2100', data: 'grpc_config=[{"percentage":0,"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"NeverPickedService","waitForReady":true}]}]}},{"percentage":100,"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"AlwaysPickedService","waitForReady":true}]}]}}]',
type: TXT}
- expected_addrs:
@@ -145,5 +150,6 @@ resolver_component_tests:
records:
ipv4-config-causing-fallback-to-tcp:
- {TTL: '2100', data: 1.2.3.4, type: A}
+ _grpc_config.ipv4-config-causing-fallback-to-tcp:
- {TTL: '2100', data: 'grpc_config=[{"serviceConfig":{"loadBalancingPolicy":"round_robin","methodConfig":[{"name":[{"method":"Foo","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwo","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooThree","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooFour","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooFive","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooSix","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooSeven","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooEight","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooNine","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTen","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooEleven","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]},{"name":[{"method":"FooTwelve","service":"SimpleService","waitForReady":true}]}]}}]',
type: TXT}
diff --git a/test/cpp/performance/writes_per_rpc_test.cc b/test/cpp/performance/writes_per_rpc_test.cc
index 6c23245021..1c6f44dd7d 100644
--- a/test/cpp/performance/writes_per_rpc_test.cc
+++ b/test/cpp/performance/writes_per_rpc_test.cc
@@ -89,7 +89,7 @@ class EndpointPairFixture {
const grpc_channel_args* server_args =
grpc_server_get_channel_args(server_->c_server());
grpc_transport* transport = grpc_create_chttp2_transport(
- &exec_ctx, server_args, endpoints.server, 0 /* is_client */);
+ &exec_ctx, server_args, endpoints.server, false /* is_client */);
grpc_pollset** pollsets;
size_t num_pollsets = 0;
@@ -100,8 +100,9 @@ class EndpointPairFixture {
}
grpc_server_setup_transport(&exec_ctx, server_->c_server(), transport,
- NULL, server_args);
- grpc_chttp2_transport_start_reading(&exec_ctx, transport, NULL);
+ nullptr, server_args);
+ grpc_chttp2_transport_start_reading(&exec_ctx, transport, nullptr,
+ nullptr);
}
/* create channel */
@@ -111,12 +112,13 @@ class EndpointPairFixture {
ApplyCommonChannelArguments(&args);
grpc_channel_args c_args = args.c_channel_args();
- grpc_transport* transport =
- grpc_create_chttp2_transport(&exec_ctx, &c_args, endpoints.client, 1);
+ grpc_transport* transport = grpc_create_chttp2_transport(
+ &exec_ctx, &c_args, endpoints.client, true);
GPR_ASSERT(transport);
grpc_channel* channel = grpc_channel_create(
&exec_ctx, "target", &c_args, GRPC_CLIENT_DIRECT_CHANNEL, transport);
- grpc_chttp2_transport_start_reading(&exec_ctx, transport, NULL);
+ grpc_chttp2_transport_start_reading(&exec_ctx, transport, nullptr,
+ nullptr);
channel_ = CreateChannelInternal("", channel);
}
diff --git a/test/cpp/qps/BUILD b/test/cpp/qps/BUILD
index 0d91d52f22..f1abb19e64 100644
--- a/test/cpp/qps/BUILD
+++ b/test/cpp/qps/BUILD
@@ -106,7 +106,7 @@ grpc_cc_library(
"histogram.h",
"stats.h",
],
- deps = ["//:gpr"],
+ deps = ["//test/core/util:grpc_test_util"],
)
grpc_cc_test(
diff --git a/test/cpp/qps/client_async.cc b/test/cpp/qps/client_async.cc
index f7cdfc2bd7..07888214e7 100644
--- a/test/cpp/qps/client_async.cc
+++ b/test/cpp/qps/client_async.cc
@@ -236,6 +236,22 @@ class AsyncClient : public ClientImpl<StubType, RequestType> {
this->EndThreads(); // this needed for resolution
}
+ ClientRpcContext* ProcessTag(size_t thread_idx, void* tag) {
+ ClientRpcContext* ctx = ClientRpcContext::detag(tag);
+ if (shutdown_state_[thread_idx]->shutdown) {
+ ctx->TryCancel();
+ delete ctx;
+ bool ok;
+ while (cli_cqs_[cq_[thread_idx]]->Next(&tag, &ok)) {
+ ctx = ClientRpcContext::detag(tag);
+ ctx->TryCancel();
+ delete ctx;
+ }
+ return nullptr;
+ }
+ return ctx;
+ }
+
void ThreadFunc(size_t thread_idx, Client::Thread* t) override final {
void* got_tag;
bool ok;
@@ -245,9 +261,13 @@ class AsyncClient : public ClientImpl<StubType, RequestType> {
if (!cli_cqs_[cq_[thread_idx]]->Next(&got_tag, &ok)) {
return;
}
- ClientRpcContext* ctx = ClientRpcContext::detag(got_tag);
std::mutex* shutdown_mu = &shutdown_state_[thread_idx]->mutex;
shutdown_mu->lock();
+ ClientRpcContext* ctx = ProcessTag(thread_idx, got_tag);
+ if (ctx == nullptr) {
+ shutdown_mu->unlock();
+ return;
+ }
while (cli_cqs_[cq_[thread_idx]]->DoThenAsyncNext(
[&, ctx, ok, entry_ptr, shutdown_mu]() {
if (!ctx->RunNextState(ok, entry_ptr)) {
@@ -260,19 +280,9 @@ class AsyncClient : public ClientImpl<StubType, RequestType> {
},
&got_tag, &ok, gpr_inf_future(GPR_CLOCK_REALTIME))) {
t->UpdateHistogram(entry_ptr);
- // Got a regular event, so process it
- ctx = ClientRpcContext::detag(got_tag);
- // Proceed while holding a lock to make sure that
- // this thread isn't supposed to shut down
shutdown_mu->lock();
- if (shutdown_state_[thread_idx]->shutdown) {
- ctx->TryCancel();
- delete ctx;
- while (cli_cqs_[cq_[thread_idx]]->Next(&got_tag, &ok)) {
- ctx = ClientRpcContext::detag(got_tag);
- ctx->TryCancel();
- delete ctx;
- }
+ ctx = ProcessTag(thread_idx, got_tag);
+ if (ctx == nullptr) {
shutdown_mu->unlock();
return;
}
diff --git a/test/cpp/qps/client_sync.cc b/test/cpp/qps/client_sync.cc
index 9f20b148eb..82a3f0042d 100644
--- a/test/cpp/qps/client_sync.cc
+++ b/test/cpp/qps/client_sync.cc
@@ -60,21 +60,20 @@ class SynchronousClient
SetupLoadTest(config, num_threads_);
}
- virtual ~SynchronousClient(){};
+ virtual ~SynchronousClient() {}
- virtual void InitThreadFuncImpl(size_t thread_idx) = 0;
+ virtual bool InitThreadFuncImpl(size_t thread_idx) = 0;
virtual bool ThreadFuncImpl(HistogramEntry* entry, size_t thread_idx) = 0;
void ThreadFunc(size_t thread_idx, Thread* t) override {
- InitThreadFuncImpl(thread_idx);
+ if (!InitThreadFuncImpl(thread_idx)) {
+ return;
+ }
for (;;) {
// run the loop body
HistogramEntry entry;
const bool thread_still_ok = ThreadFuncImpl(&entry, thread_idx);
t->UpdateHistogram(&entry);
- if (!thread_still_ok) {
- gpr_log(GPR_ERROR, "Finishing client thread due to RPC error");
- }
if (!thread_still_ok || ThreadCompleted()) {
return;
}
@@ -109,9 +108,6 @@ class SynchronousClient
size_t num_threads_;
std::vector<SimpleResponse> responses_;
-
- private:
- void DestroyMultithreading() override final { EndThreads(); }
};
class SynchronousUnaryClient final : public SynchronousClient {
@@ -122,7 +118,7 @@ class SynchronousUnaryClient final : public SynchronousClient {
}
~SynchronousUnaryClient() {}
- void InitThreadFuncImpl(size_t thread_idx) override {}
+ bool InitThreadFuncImpl(size_t thread_idx) override { return true; }
bool ThreadFuncImpl(HistogramEntry* entry, size_t thread_idx) override {
if (!WaitToIssue(thread_idx)) {
@@ -140,6 +136,9 @@ class SynchronousUnaryClient final : public SynchronousClient {
entry->set_status(s.error_code());
return true;
}
+
+ private:
+ void DestroyMultithreading() override final { EndThreads(); }
};
template <class StreamType>
@@ -149,31 +148,30 @@ class SynchronousStreamingClient : public SynchronousClient {
: SynchronousClient(config),
context_(num_threads_),
stream_(num_threads_),
+ stream_mu_(num_threads_),
+ shutdown_(num_threads_),
messages_per_stream_(config.messages_per_stream()),
messages_issued_(num_threads_) {
StartThreads(num_threads_);
}
virtual ~SynchronousStreamingClient() {
- std::vector<std::thread> cleanup_threads;
- for (size_t i = 0; i < num_threads_; i++) {
- cleanup_threads.emplace_back([this, i]() {
- auto stream = &stream_[i];
- if (*stream) {
- // forcibly cancel the streams, then finish
- context_[i].TryCancel();
- (*stream)->Finish().IgnoreError();
- // don't log any error message on !ok since this was canceled
- }
- });
- }
- for (auto& th : cleanup_threads) {
- th.join();
- }
+ CleanupAllStreams([this](size_t thread_idx) {
+ // Don't log any kind of error since we may have canceled this
+ stream_[thread_idx]->Finish().IgnoreError();
+ });
}
protected:
std::vector<grpc::ClientContext> context_;
std::vector<std::unique_ptr<StreamType>> stream_;
+ // stream_mu_ is only needed when changing an element of stream_ or context_
+ std::vector<std::mutex> stream_mu_;
+ // use struct Bool rather than bool because vector<bool> is not concurrent
+ struct Bool {
+ bool val;
+ Bool() : val(false) {}
+ };
+ std::vector<Bool> shutdown_;
const int messages_per_stream_;
std::vector<int> messages_issued_;
@@ -182,27 +180,26 @@ class SynchronousStreamingClient : public SynchronousClient {
// don't set the value since the stream is failed and shouldn't be timed
entry->set_status(s.error_code());
if (!s.ok()) {
- gpr_log(GPR_ERROR, "Stream %" PRIuPTR " received an error %s", thread_idx,
- s.error_message().c_str());
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ gpr_log(GPR_ERROR, "Stream %" PRIuPTR " received an error %s",
+ thread_idx, s.error_message().c_str());
+ }
}
+ // Lock the stream_mu_ now because the client context could change
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
context_[thread_idx].~ClientContext();
new (&context_[thread_idx]) ClientContext();
}
-};
-class SynchronousStreamingPingPongClient final
- : public SynchronousStreamingClient<
- grpc::ClientReaderWriter<SimpleRequest, SimpleResponse>> {
- public:
- SynchronousStreamingPingPongClient(const ClientConfig& config)
- : SynchronousStreamingClient(config) {}
- ~SynchronousStreamingPingPongClient() {
+ void CleanupAllStreams(std::function<void(size_t)> cleaner) {
std::vector<std::thread> cleanup_threads;
for (size_t i = 0; i < num_threads_; i++) {
- cleanup_threads.emplace_back([this, i]() {
- auto stream = &stream_[i];
- if (*stream) {
- (*stream)->WritesDone();
+ cleanup_threads.emplace_back([this, i, cleaner] {
+ std::lock_guard<std::mutex> l(stream_mu_[i]);
+ shutdown_[i].val = true;
+ if (stream_[i]) {
+ cleaner(i);
}
});
}
@@ -211,10 +208,36 @@ class SynchronousStreamingPingPongClient final
}
}
- void InitThreadFuncImpl(size_t thread_idx) override {
+ private:
+ void DestroyMultithreading() override final {
+ CleanupAllStreams(
+ [this](size_t thread_idx) { context_[thread_idx].TryCancel(); });
+ EndThreads();
+ }
+};
+
+class SynchronousStreamingPingPongClient final
+ : public SynchronousStreamingClient<
+ grpc::ClientReaderWriter<SimpleRequest, SimpleResponse>> {
+ public:
+ SynchronousStreamingPingPongClient(const ClientConfig& config)
+ : SynchronousStreamingClient(config) {}
+ ~SynchronousStreamingPingPongClient() {
+ CleanupAllStreams(
+ [this](size_t thread_idx) { stream_[thread_idx]->WritesDone(); });
+ }
+
+ private:
+ bool InitThreadFuncImpl(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] = stub->StreamingCall(&context_[thread_idx]);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] = stub->StreamingCall(&context_[thread_idx]);
+ } else {
+ return false;
+ }
messages_issued_[thread_idx] = 0;
+ return true;
}
bool ThreadFuncImpl(HistogramEntry* entry, size_t thread_idx) override {
@@ -239,7 +262,13 @@ class SynchronousStreamingPingPongClient final
stream_[thread_idx]->WritesDone();
FinishStream(entry, thread_idx);
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] = stub->StreamingCall(&context_[thread_idx]);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] = stub->StreamingCall(&context_[thread_idx]);
+ } else {
+ stream_[thread_idx].reset();
+ return false;
+ }
messages_issued_[thread_idx] = 0;
return true;
}
@@ -251,25 +280,24 @@ class SynchronousStreamingFromClientClient final
SynchronousStreamingFromClientClient(const ClientConfig& config)
: SynchronousStreamingClient(config), last_issue_(num_threads_) {}
~SynchronousStreamingFromClientClient() {
- std::vector<std::thread> cleanup_threads;
- for (size_t i = 0; i < num_threads_; i++) {
- cleanup_threads.emplace_back([this, i]() {
- auto stream = &stream_[i];
- if (*stream) {
- (*stream)->WritesDone();
- }
- });
- }
- for (auto& th : cleanup_threads) {
- th.join();
- }
+ CleanupAllStreams(
+ [this](size_t thread_idx) { stream_[thread_idx]->WritesDone(); });
}
- void InitThreadFuncImpl(size_t thread_idx) override {
+ private:
+ std::vector<double> last_issue_;
+
+ bool InitThreadFuncImpl(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] = stub->StreamingFromClient(&context_[thread_idx],
- &responses_[thread_idx]);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] = stub->StreamingFromClient(&context_[thread_idx],
+ &responses_[thread_idx]);
+ } else {
+ return false;
+ }
last_issue_[thread_idx] = UsageTimer::Now();
+ return true;
}
bool ThreadFuncImpl(HistogramEntry* entry, size_t thread_idx) override {
@@ -287,13 +315,16 @@ class SynchronousStreamingFromClientClient final
stream_[thread_idx]->WritesDone();
FinishStream(entry, thread_idx);
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] = stub->StreamingFromClient(&context_[thread_idx],
- &responses_[thread_idx]);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] = stub->StreamingFromClient(&context_[thread_idx],
+ &responses_[thread_idx]);
+ } else {
+ stream_[thread_idx].reset();
+ return false;
+ }
return true;
}
-
- private:
- std::vector<double> last_issue_;
};
class SynchronousStreamingFromServerClient final
@@ -301,12 +332,24 @@ class SynchronousStreamingFromServerClient final
public:
SynchronousStreamingFromServerClient(const ClientConfig& config)
: SynchronousStreamingClient(config), last_recv_(num_threads_) {}
- void InitThreadFuncImpl(size_t thread_idx) override {
+ ~SynchronousStreamingFromServerClient() {}
+
+ private:
+ std::vector<double> last_recv_;
+
+ bool InitThreadFuncImpl(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] =
- stub->StreamingFromServer(&context_[thread_idx], request_);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] =
+ stub->StreamingFromServer(&context_[thread_idx], request_);
+ } else {
+ return false;
+ }
last_recv_[thread_idx] = UsageTimer::Now();
+ return true;
}
+
bool ThreadFuncImpl(HistogramEntry* entry, size_t thread_idx) override {
GPR_TIMER_SCOPE("SynchronousStreamingFromServerClient::ThreadFunc", 0);
if (stream_[thread_idx]->Read(&responses_[thread_idx])) {
@@ -317,13 +360,16 @@ class SynchronousStreamingFromServerClient final
}
FinishStream(entry, thread_idx);
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] =
- stub->StreamingFromServer(&context_[thread_idx], request_);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] =
+ stub->StreamingFromServer(&context_[thread_idx], request_);
+ } else {
+ stream_[thread_idx].reset();
+ return false;
+ }
return true;
}
-
- private:
- std::vector<double> last_recv_;
};
class SynchronousStreamingBothWaysClient final
@@ -333,24 +379,22 @@ class SynchronousStreamingBothWaysClient final
SynchronousStreamingBothWaysClient(const ClientConfig& config)
: SynchronousStreamingClient(config) {}
~SynchronousStreamingBothWaysClient() {
- std::vector<std::thread> cleanup_threads;
- for (size_t i = 0; i < num_threads_; i++) {
- cleanup_threads.emplace_back([this, i]() {
- auto stream = &stream_[i];
- if (*stream) {
- (*stream)->WritesDone();
- }
- });
- }
- for (auto& th : cleanup_threads) {
- th.join();
- }
+ CleanupAllStreams(
+ [this](size_t thread_idx) { stream_[thread_idx]->WritesDone(); });
}
- void InitThreadFuncImpl(size_t thread_idx) override {
+ private:
+ bool InitThreadFuncImpl(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
- stream_[thread_idx] = stub->StreamingBothWays(&context_[thread_idx]);
+ std::lock_guard<std::mutex> l(stream_mu_[thread_idx]);
+ if (!shutdown_[thread_idx].val) {
+ stream_[thread_idx] = stub->StreamingBothWays(&context_[thread_idx]);
+ } else {
+ return false;
+ }
+ return true;
}
+
bool ThreadFuncImpl(HistogramEntry* entry, size_t thread_idx) override {
// TODO (vjpai): Do this
return true;
diff --git a/test/cpp/qps/histogram.h b/test/cpp/qps/histogram.h
index e31d5d78a8..ba72b5b332 100644
--- a/test/cpp/qps/histogram.h
+++ b/test/cpp/qps/histogram.h
@@ -19,8 +19,8 @@
#ifndef TEST_QPS_HISTOGRAM_H
#define TEST_QPS_HISTOGRAM_H
-#include <grpc/support/histogram.h>
#include "src/proto/grpc/testing/stats.pb.h"
+#include "test/core/util/histogram.h"
namespace grpc {
namespace testing {
@@ -29,36 +29,36 @@ class Histogram {
public:
// TODO: look into making histogram params not hardcoded for C++
Histogram()
- : impl_(gpr_histogram_create(default_resolution(),
- default_max_possible())) {}
+ : impl_(grpc_histogram_create(default_resolution(),
+ default_max_possible())) {}
~Histogram() {
- if (impl_) gpr_histogram_destroy(impl_);
+ if (impl_) grpc_histogram_destroy(impl_);
}
Histogram(Histogram&& other) : impl_(other.impl_) { other.impl_ = nullptr; }
- void Merge(const Histogram& h) { gpr_histogram_merge(impl_, h.impl_); }
- void Add(double value) { gpr_histogram_add(impl_, value); }
+ void Merge(const Histogram& h) { grpc_histogram_merge(impl_, h.impl_); }
+ void Add(double value) { grpc_histogram_add(impl_, value); }
double Percentile(double pctile) const {
- return gpr_histogram_percentile(impl_, pctile);
+ return grpc_histogram_percentile(impl_, pctile);
}
- double Count() const { return gpr_histogram_count(impl_); }
+ double Count() const { return grpc_histogram_count(impl_); }
void Swap(Histogram* other) { std::swap(impl_, other->impl_); }
void FillProto(HistogramData* p) {
size_t n;
- const auto* data = gpr_histogram_get_contents(impl_, &n);
+ const auto* data = grpc_histogram_get_contents(impl_, &n);
for (size_t i = 0; i < n; i++) {
p->add_bucket(data[i]);
}
- p->set_min_seen(gpr_histogram_minimum(impl_));
- p->set_max_seen(gpr_histogram_maximum(impl_));
- p->set_sum(gpr_histogram_sum(impl_));
- p->set_sum_of_squares(gpr_histogram_sum_of_squares(impl_));
- p->set_count(gpr_histogram_count(impl_));
+ p->set_min_seen(grpc_histogram_minimum(impl_));
+ p->set_max_seen(grpc_histogram_maximum(impl_));
+ p->set_sum(grpc_histogram_sum(impl_));
+ p->set_sum_of_squares(grpc_histogram_sum_of_squares(impl_));
+ p->set_count(grpc_histogram_count(impl_));
}
void MergeProto(const HistogramData& p) {
- gpr_histogram_merge_contents(impl_, &*p.bucket().begin(), p.bucket_size(),
- p.min_seen(), p.max_seen(), p.sum(),
- p.sum_of_squares(), p.count());
+ grpc_histogram_merge_contents(impl_, &*p.bucket().begin(), p.bucket_size(),
+ p.min_seen(), p.max_seen(), p.sum(),
+ p.sum_of_squares(), p.count());
}
static double default_resolution() { return 0.01; }
@@ -68,7 +68,7 @@ class Histogram {
Histogram(const Histogram&);
Histogram& operator=(const Histogram&);
- gpr_histogram* impl_;
+ grpc_histogram* impl_;
};
} // namespace testing
} // namespace grpc
diff --git a/test/cpp/qps/json_run_localhost.cc b/test/cpp/qps/json_run_localhost.cc
index 4b788eae73..db8b2a3943 100644
--- a/test/cpp/qps/json_run_localhost.cc
+++ b/test/cpp/qps/json_run_localhost.cc
@@ -46,7 +46,7 @@ std::string as_string(const T& val) {
static void sighandler(int sig) {
const int errno_saved = errno;
- if (g_driver != NULL) g_driver->Interrupt();
+ if (g_driver != nullptr) g_driver->Interrupt();
for (int i = 0; i < kNumWorkers; ++i) {
if (g_workers[i]) g_workers[i]->Interrupt();
}
@@ -58,8 +58,8 @@ static void register_sighandler() {
memset(&act, 0, sizeof(act));
act.sa_handler = sighandler;
- sigaction(SIGINT, &act, NULL);
- sigaction(SIGTERM, &act, NULL);
+ sigaction(SIGINT, &act, nullptr);
+ sigaction(SIGTERM, &act, nullptr);
}
static void LogStatus(int status, const char* label) {
diff --git a/test/cpp/qps/qps_interarrival_test.cc b/test/cpp/qps/qps_interarrival_test.cc
index 461bf624ce..625b7db426 100644
--- a/test/cpp/qps/qps_interarrival_test.cc
+++ b/test/cpp/qps/qps_interarrival_test.cc
@@ -20,7 +20,7 @@
#include <iostream>
// Use the C histogram rather than C++ to avoid depending on proto
-#include <grpc/support/histogram.h>
+#include "test/core/util/histogram.h"
#include "test/cpp/qps/interarrival.h"
#include "test/cpp/util/test_config.h"
@@ -31,21 +31,21 @@ using grpc::testing::RandomDistInterface;
static void RunTest(RandomDistInterface&& r, int threads, std::string title) {
InterarrivalTimer timer;
timer.init(r, threads);
- gpr_histogram* h(gpr_histogram_create(0.01, 60e9));
+ grpc_histogram* h(grpc_histogram_create(0.01, 60e9));
for (int i = 0; i < 10000000; i++) {
for (int j = 0; j < threads; j++) {
- gpr_histogram_add(h, timer.next(j));
+ grpc_histogram_add(h, timer.next(j));
}
}
std::cout << title << " Distribution" << std::endl;
std::cout << "Value, Percentile" << std::endl;
for (double pct = 0.0; pct < 100.0; pct += 1.0) {
- std::cout << gpr_histogram_percentile(h, pct) << "," << pct << std::endl;
+ std::cout << grpc_histogram_percentile(h, pct) << "," << pct << std::endl;
}
- gpr_histogram_destroy(h);
+ grpc_histogram_destroy(h);
}
using grpc::testing::ExpDist;
diff --git a/test/cpp/qps/qps_json_driver.cc b/test/cpp/qps/qps_json_driver.cc
index 1672527426..b2449da69c 100644
--- a/test/cpp/qps/qps_json_driver.cc
+++ b/test/cpp/qps/qps_json_driver.cc
@@ -181,7 +181,7 @@ static bool QpsDriver() {
if (scfile) {
// Read the json data from disk
FILE* json_file = fopen(FLAGS_scenarios_file.c_str(), "r");
- GPR_ASSERT(json_file != NULL);
+ GPR_ASSERT(json_file != nullptr);
fseek(json_file, 0, SEEK_END);
long len = ftell(json_file);
char* data = new char[len];
diff --git a/test/cpp/qps/qps_worker.cc b/test/cpp/qps/qps_worker.cc
index c288b03ec5..4c9ab0ea43 100644
--- a/test/cpp/qps/qps_worker.cc
+++ b/test/cpp/qps/qps_worker.cc
@@ -32,12 +32,12 @@
#include <grpc/grpc.h>
#include <grpc/support/alloc.h>
#include <grpc/support/cpu.h>
-#include <grpc/support/histogram.h>
#include <grpc/support/host_port.h>
#include <grpc/support/log.h>
#include "src/proto/grpc/testing/services.pb.h"
#include "test/core/util/grpc_profiler.h"
+#include "test/core/util/histogram.h"
#include "test/cpp/qps/client.h"
#include "test/cpp/qps/server.h"
#include "test/cpp/util/create_test_channel.h"
diff --git a/test/cpp/qps/server_async.cc b/test/cpp/qps/server_async.cc
index 23c7e85283..72ae772147 100644
--- a/test/cpp/qps/server_async.cc
+++ b/test/cpp/qps/server_async.cc
@@ -79,7 +79,7 @@ class AsyncQpsServerTest final : public grpc::testing::Server {
auto port_num = port();
// Negative port number means inproc server, so no listen port needed
if (port_num >= 0) {
- char* server_address = NULL;
+ char* server_address = nullptr;
gpr_join_host_port(&server_address, "::", port_num);
builder.AddListeningPort(server_address,
Server::CreateServerCredentials(config));
diff --git a/test/cpp/qps/server_sync.cc b/test/cpp/qps/server_sync.cc
index 4ef57bd08b..ea89a30e2e 100644
--- a/test/cpp/qps/server_sync.cc
+++ b/test/cpp/qps/server_sync.cc
@@ -159,7 +159,7 @@ class SynchronousServer final : public grpc::testing::Server {
auto port_num = port();
// Negative port number means inproc server, so no listen port needed
if (port_num >= 0) {
- char* server_address = NULL;
+ char* server_address = nullptr;
gpr_join_host_port(&server_address, "::", port_num);
builder.AddListeningPort(server_address,
Server::CreateServerCredentials(config));
diff --git a/test/cpp/thread_manager/thread_manager_test.cc b/test/cpp/thread_manager/thread_manager_test.cc
index af90d44495..8282d46694 100644
--- a/test/cpp/thread_manager/thread_manager_test.cc
+++ b/test/cpp/thread_manager/thread_manager_test.cc
@@ -111,7 +111,7 @@ void ThreadManagerTest::PerformTest() {
} // namespace grpc
int main(int argc, char** argv) {
- std::srand(std::time(NULL));
+ std::srand(std::time(nullptr));
grpc::testing::InitTest(&argc, &argv, true);
grpc::ThreadManagerTest test_rpc_manager;
diff --git a/test/cpp/util/BUILD b/test/cpp/util/BUILD
index 2559c18c32..19e15b1adf 100644
--- a/test/cpp/util/BUILD
+++ b/test/cpp/util/BUILD
@@ -43,6 +43,7 @@ grpc_cc_library(
"proto_reflection_descriptor_database.h",
],
deps = [
+ "//:grpc++",
"//:grpc++_config_proto",
"//src/proto/grpc/reflection/v1alpha:reflection_proto",
],
@@ -135,6 +136,105 @@ grpc_cc_library(
)
grpc_cc_test(
+ name = "grpc_tool_test",
+ srcs = [
+ "grpc_tool_test.cc",
+ ],
+ deps = [
+ ":grpc_cli_libs",
+ ":test_util",
+ "//:grpc++_reflection",
+ "//src/proto/grpc/testing:echo_proto",
+ "//src/proto/grpc/testing:echo_messages_proto",
+ "//test/core/util:grpc_test_util",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
+ name = "byte_buffer_test",
+ srcs = [
+ "byte_buffer_test.cc",
+ ],
+ deps = [
+ ":test_util",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
+ name = "slice_test",
+ srcs = [
+ "slice_test.cc",
+ ],
+ deps = [
+ ":test_util",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
+ name = "string_ref_test",
+ srcs = [
+ "string_ref_test.cc",
+ ],
+ deps = [
+ "//:grpc++",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
+ name = "time_test",
+ srcs = [
+ "time_test.cc",
+ ],
+ deps = [
+ ":test_util",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
+ name = "status_test",
+ srcs = [
+ "status_test.cc",
+ ],
+ deps = [
+ ":test_util",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
+ name = "cli_call_test",
+ srcs = [
+ "cli_call_test.cc",
+ ],
+ deps = [
+ ":grpc_cli_libs",
+ ":test_util",
+ "//src/proto/grpc/testing:echo_proto",
+ "//test/core/util:grpc_test_util",
+ ],
+ external_deps = [
+ "gtest",
+ ],
+)
+
+grpc_cc_test(
name = "error_details_test",
srcs = [
"error_details_test.cc",
diff --git a/test/cpp/util/grpc_tool.cc b/test/cpp/util/grpc_tool.cc
index cd6084ac6d..a6d08cd83c 100644
--- a/test/cpp/util/grpc_tool.cc
+++ b/test/cpp/util/grpc_tool.cc
@@ -230,7 +230,7 @@ const Command* FindCommand(const grpc::string& name) {
return &ops[i];
}
}
- return NULL;
+ return nullptr;
}
} // namespace
@@ -245,13 +245,13 @@ int GrpcToolMainLib(int argc, const char** argv, const CliCredentials& cred,
argv += 2;
const Command* cmd = FindCommand(command);
- if (cmd != NULL) {
+ if (cmd != nullptr) {
GrpcTool grpc_tool;
if (argc < cmd->min_args || argc > cmd->max_args) {
// Force the command to print its usage message
fprintf(stderr, "\nWrong number of arguments for %s\n", command.c_str());
grpc_tool.SetPrintCommandMode(1);
- return cmd->function(&grpc_tool, -1, NULL, cred, callback);
+ return cmd->function(&grpc_tool, -1, nullptr, cred, callback);
}
const bool ok = cmd->function(&grpc_tool, argc, argv, cred, callback);
return ok ? 0 : 1;
@@ -281,11 +281,11 @@ bool GrpcTool::Help(int argc, const char** argv, const CliCredentials& cred,
Usage("");
} else {
const Command* cmd = FindCommand(argv[0]);
- if (cmd == NULL) {
+ if (cmd == nullptr) {
Usage("Unknown command '" + grpc::string(argv[0]) + "'");
}
SetPrintCommandMode(0);
- cmd->function(this, -1, NULL, cred, callback);
+ cmd->function(this, -1, nullptr, cred, callback);
}
return true;
}
diff --git a/test/cpp/util/grpc_tool_test.cc b/test/cpp/util/grpc_tool_test.cc
index d0b3d7b81b..1c07b2a8ef 100644
--- a/test/cpp/util/grpc_tool_test.cc
+++ b/test/cpp/util/grpc_tool_test.cc
@@ -395,7 +395,7 @@ TEST_F(GrpcToolTest, CallCommand) {
std::bind(PrintStream, &output_stream,
std::placeholders::_1)));
// Expected output: "message: \"Hello\""
- EXPECT_TRUE(NULL !=
+ EXPECT_TRUE(nullptr !=
strstr(output_stream.str().c_str(), "message: \"Hello\""));
ShutdownServer();
}
@@ -421,9 +421,9 @@ TEST_F(GrpcToolTest, CallCommandBatch) {
// Expected output: "message: "Hello0"\nmessage: "Hello1"\nmessage:
// "Hello2"\n"
- EXPECT_TRUE(NULL != strstr(output_stream.str().c_str(),
- "message: \"Hello0\"\nmessage: "
- "\"Hello1\"\nmessage: \"Hello2\"\n"));
+ EXPECT_TRUE(nullptr != strstr(output_stream.str().c_str(),
+ "message: \"Hello0\"\nmessage: "
+ "\"Hello1\"\nmessage: \"Hello2\"\n"));
std::cin.rdbuf(orig);
ShutdownServer();
}
@@ -448,8 +448,8 @@ TEST_F(GrpcToolTest, CallCommandBatchWithBadRequest) {
FLAGS_batch = false;
// Expected output: "message: "Hello0"\nmessage: "Hello2"\n"
- EXPECT_TRUE(NULL != strstr(output_stream.str().c_str(),
- "message: \"Hello0\"\nmessage: \"Hello2\"\n"));
+ EXPECT_TRUE(nullptr != strstr(output_stream.str().c_str(),
+ "message: \"Hello0\"\nmessage: \"Hello2\"\n"));
std::cin.rdbuf(orig);
ShutdownServer();
}
@@ -473,8 +473,8 @@ TEST_F(GrpcToolTest, CallCommandRequestStream) {
std::placeholders::_1)));
// Expected output: "message: \"Hello0Hello1Hello2\""
- EXPECT_TRUE(NULL != strstr(output_stream.str().c_str(),
- "message: \"Hello0Hello1Hello2\""));
+ EXPECT_TRUE(nullptr != strstr(output_stream.str().c_str(),
+ "message: \"Hello0Hello1Hello2\""));
std::cin.rdbuf(orig);
ShutdownServer();
}
@@ -498,7 +498,7 @@ TEST_F(GrpcToolTest, CallCommandRequestStreamWithBadRequest) {
std::placeholders::_1)));
// Expected output: "message: \"Hello0Hello2\""
- EXPECT_TRUE(NULL !=
+ EXPECT_TRUE(nullptr !=
strstr(output_stream.str().c_str(), "message: \"Hello0Hello2\""));
std::cin.rdbuf(orig);
ShutdownServer();
@@ -521,8 +521,8 @@ TEST_F(GrpcToolTest, CallCommandResponseStream) {
for (int i = 0; i < kServerDefaultResponseStreamsToSend; i++) {
grpc::string expected_response_text =
"message: \"Hello" + grpc::to_string(i) + "\"\n";
- EXPECT_TRUE(NULL != strstr(output_stream.str().c_str(),
- expected_response_text.c_str()));
+ EXPECT_TRUE(nullptr != strstr(output_stream.str().c_str(),
+ expected_response_text.c_str()));
}
ShutdownServer();
@@ -547,9 +547,9 @@ TEST_F(GrpcToolTest, CallCommandBidiStream) {
// Expected output: "message: \"Hello0\"\nmessage: \"Hello1\"\nmessage:
// \"Hello2\"\n\n"
- EXPECT_TRUE(NULL != strstr(output_stream.str().c_str(),
- "message: \"Hello0\"\nmessage: "
- "\"Hello1\"\nmessage: \"Hello2\"\n"));
+ EXPECT_TRUE(nullptr != strstr(output_stream.str().c_str(),
+ "message: \"Hello0\"\nmessage: "
+ "\"Hello1\"\nmessage: \"Hello2\"\n"));
std::cin.rdbuf(orig);
ShutdownServer();
}
@@ -573,8 +573,8 @@ TEST_F(GrpcToolTest, CallCommandBidiStreamWithBadRequest) {
// Expected output: "message: \"Hello0\"\nmessage: \"Hello1\"\nmessage:
// \"Hello2\"\n\n"
- EXPECT_TRUE(NULL != strstr(output_stream.str().c_str(),
- "message: \"Hello0\"\nmessage: \"Hello2\"\n"));
+ EXPECT_TRUE(nullptr != strstr(output_stream.str().c_str(),
+ "message: \"Hello0\"\nmessage: \"Hello2\"\n"));
std::cin.rdbuf(orig);
ShutdownServer();