From 70d5e5af1752434467ed17bca3294ccf386cbc57 Mon Sep 17 00:00:00 2001 From: Soheil Hassas Yeganeh Date: Tue, 7 May 2019 22:08:44 -0400 Subject: [PATCH 1/2] Introduce GRPC_TRACE_FLAG_ENABLED macro to mark trace branches unlikely. This is a trivial change and results in better code generation, because it marks the trace path unlikely. The changes are all mechanical, except I did some minor edits in two macros in call_stack.h and http to apply best practices. --- .../client_channel/channel_connectivity.cc | 2 +- .../filters/client_channel/client_channel.cc | 106 +++++++++--------- .../health/health_check_client.cc | 18 +-- .../client_channel/lb_policy/grpclb/grpclb.cc | 34 +++--- .../lb_policy/pick_first/pick_first.cc | 18 +-- .../lb_policy/round_robin/round_robin.cc | 20 ++-- .../lb_policy/subchannel_list.h | 24 ++-- .../client_channel/lb_policy/xds/xds.cc | 44 ++++---- .../resolver/dns/c_ares/grpc_ares_wrapper.cc | 4 +- .../resolver/dns/c_ares/grpc_ares_wrapper.h | 2 +- .../client_channel/resolving_lb_policy.cc | 22 ++-- .../message_compress_filter.cc | 4 +- .../chttp2/transport/chttp2_transport.cc | 23 ++-- .../transport/chttp2/transport/flow_control.h | 2 +- .../chttp2/transport/frame_settings.cc | 7 +- .../chttp2/transport/hpack_encoder.cc | 4 +- .../chttp2/transport/hpack_parser.cc | 4 +- .../transport/chttp2/transport/hpack_table.cc | 4 +- .../ext/transport/chttp2/transport/internal.h | 9 +- .../ext/transport/chttp2/transport/parsing.cc | 8 +- .../chttp2/transport/stream_lists.cc | 6 +- .../ext/transport/chttp2/transport/writing.cc | 14 ++- .../ext/transport/inproc/inproc_transport.cc | 12 +- src/core/lib/channel/channel_stack.h | 8 +- src/core/lib/channel/handshaker.cc | 8 +- src/core/lib/debug/trace.h | 4 + src/core/lib/http/parser.cc | 2 +- src/core/lib/iomgr/call_combiner.cc | 28 ++--- src/core/lib/iomgr/call_combiner.h | 2 +- src/core/lib/iomgr/ev_epoll1_linux.cc | 52 ++++----- src/core/lib/iomgr/ev_epollex_linux.cc | 64 +++++------ src/core/lib/iomgr/ev_poll_posix.cc | 14 +-- src/core/lib/iomgr/ev_posix.cc | 2 +- src/core/lib/iomgr/ev_posix.h | 2 +- src/core/lib/iomgr/executor.cc | 8 +- src/core/lib/iomgr/lockfree_event.cc | 6 +- src/core/lib/iomgr/resource_quota.cc | 20 ++-- .../lib/iomgr/socket_utils_common_posix.cc | 4 +- src/core/lib/iomgr/tcp_client_custom.cc | 4 +- src/core/lib/iomgr/tcp_client_posix.cc | 6 +- src/core/lib/iomgr/tcp_custom.cc | 18 +-- src/core/lib/iomgr/tcp_posix.cc | 60 +++++----- src/core/lib/iomgr/tcp_server_custom.cc | 6 +- src/core/lib/iomgr/tcp_server_posix.cc | 2 +- src/core/lib/iomgr/timer_generic.cc | 30 ++--- src/core/lib/iomgr/timer_manager.cc | 23 ++-- .../credentials/jwt/jwt_credentials.cc | 2 +- .../credentials/oauth2/oauth2_credentials.cc | 2 +- .../credentials/plugin/plugin_credentials.cc | 14 +-- .../lib/security/transport/secure_endpoint.cc | 8 +- src/core/lib/surface/api_trace.h | 2 +- src/core/lib/surface/call.cc | 6 +- src/core/lib/surface/call.h | 3 +- src/core/lib/surface/completion_queue.cc | 45 ++++---- src/core/lib/surface/server.cc | 3 +- src/core/lib/transport/bdp_estimator.cc | 6 +- src/core/lib/transport/bdp_estimator.h | 4 +- src/core/lib/transport/connectivity_state.cc | 8 +- src/core/tsi/fake_transport_security.cc | 8 +- src/core/tsi/ssl_transport_security.cc | 2 +- 60 files changed, 451 insertions(+), 426 deletions(-) diff --git a/src/core/ext/filters/client_channel/channel_connectivity.cc b/src/core/ext/filters/client_channel/channel_connectivity.cc index 9f970f6affa..232183d61ff 100644 --- a/src/core/ext/filters/client_channel/channel_connectivity.cc +++ b/src/core/ext/filters/client_channel/channel_connectivity.cc @@ -125,7 +125,7 @@ static void partly_done(state_watcher* w, bool due_to_completion, gpr_mu_lock(&w->mu); if (due_to_completion) { - if (grpc_trace_operation_failures.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures)) { GRPC_LOG_IF_ERROR("watch_completion_error", GRPC_ERROR_REF(error)); } GRPC_ERROR_UNREF(error); diff --git a/src/core/ext/filters/client_channel/client_channel.cc b/src/core/ext/filters/client_channel/client_channel.cc index 248e7811bba..6075affecb0 100644 --- a/src/core/ext/filters/client_channel/client_channel.cc +++ b/src/core/ext/filters/client_channel/client_channel.cc @@ -970,7 +970,7 @@ class ChannelData::ClientChannelControlHelper UniquePtr picker) override { grpc_error* disconnect_error = chand_->disconnect_error_.Load(MemoryOrder::ACQUIRE); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { const char* extra = disconnect_error == GRPC_ERROR_NONE ? "" : " (ignoring -- channel shutting down)"; @@ -1105,7 +1105,7 @@ ChannelData::ChannelData(grpc_channel_element_args* args, grpc_error** error) } else { grpc_pollset_set_add_pollset_set(resolving_lb_policy_->interested_parties(), interested_parties_); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p: created resolving_lb_policy=%p", this, resolving_lb_policy_.get()); } @@ -1136,7 +1136,7 @@ bool ChannelData::ProcessResolverResultLocked( ChannelData* chand = static_cast(arg); ProcessedResolverResult resolver_result(result); char* service_config_json = gpr_strdup(resolver_result.service_config_json()); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p: resolver returned service config: \"%s\"", chand, service_config_json); } @@ -1407,7 +1407,7 @@ void CallData::StartTransportStreamOpBatch( } // If we've previously been cancelled, immediately fail any new batches. if (GPR_UNLIKELY(calld->cancel_error_ != GRPC_ERROR_NONE)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: failing batch with error: %s", chand, calld, grpc_error_string(calld->cancel_error_)); } @@ -1426,7 +1426,7 @@ void CallData::StartTransportStreamOpBatch( GRPC_ERROR_UNREF(calld->cancel_error_); calld->cancel_error_ = GRPC_ERROR_REF(batch->payload->cancel_stream.cancel_error); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: recording cancel_error=%s", chand, calld, grpc_error_string(calld->cancel_error_)); } @@ -1454,7 +1454,7 @@ void CallData::StartTransportStreamOpBatch( // the channel combiner, which is more efficient (especially for // streaming calls). if (calld->subchannel_call_ != nullptr) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: starting batch on subchannel_call=%p", chand, calld, calld->subchannel_call_.get()); @@ -1466,7 +1466,7 @@ void CallData::StartTransportStreamOpBatch( // For batches containing a send_initial_metadata op, enter the channel // combiner to start a pick. if (GPR_LIKELY(batch->send_initial_metadata)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: entering client_channel combiner", chand, calld); } @@ -1477,7 +1477,7 @@ void CallData::StartTransportStreamOpBatch( GRPC_ERROR_NONE); } else { // For all other batches, release the call combiner. - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: saved batch, yielding call combiner", chand, calld); @@ -1535,7 +1535,7 @@ void CallData::MaybeCacheSendOpsForBatch(PendingBatch* pending) { } void CallData::FreeCachedSendInitialMetadata(ChannelData* chand) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: destroying calld->send_initial_metadata", chand, this); @@ -1544,7 +1544,7 @@ void CallData::FreeCachedSendInitialMetadata(ChannelData* chand) { } void CallData::FreeCachedSendMessage(ChannelData* chand, size_t idx) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: destroying calld->send_messages[%" PRIuPTR "]", chand, this, idx); @@ -1553,7 +1553,7 @@ void CallData::FreeCachedSendMessage(ChannelData* chand, size_t idx) { } void CallData::FreeCachedSendTrailingMetadata(ChannelData* chand) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: destroying calld->send_trailing_metadata", chand, this); @@ -1630,7 +1630,7 @@ void CallData::PendingBatchesAdd(grpc_call_element* elem, grpc_transport_stream_op_batch* batch) { ChannelData* chand = static_cast(elem->channel_data); const size_t idx = GetBatchIndex(batch); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: adding pending batch at index %" PRIuPTR, chand, this, idx); @@ -1659,7 +1659,7 @@ void CallData::PendingBatchesAdd(grpc_call_element* elem, } if (GPR_UNLIKELY(bytes_buffered_for_retry_ > chand->per_rpc_retry_buffer_size())) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: exceeded retry buffer size, committing", chand, this); @@ -1672,7 +1672,7 @@ void CallData::PendingBatchesAdd(grpc_call_element* elem, // If we are not going to retry and have not yet started, pretend // retries are disabled so that we don't bother with retry overhead. if (num_attempts_completed_ == 0) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: disabling retries before first attempt", chand, this); @@ -1713,7 +1713,7 @@ void CallData::MaybeClearPendingBatch(grpc_call_element* elem, (!batch->recv_trailing_metadata || batch->payload->recv_trailing_metadata.recv_trailing_metadata_ready == nullptr)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: clearing pending batch", chand, this); } @@ -1736,7 +1736,7 @@ void CallData::PendingBatchesFail( grpc_call_element* elem, grpc_error* error, YieldCallCombinerPredicate yield_call_combiner_predicate) { GPR_ASSERT(error != GRPC_ERROR_NONE); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { size_t num_batches = 0; for (size_t i = 0; i < GPR_ARRAY_SIZE(pending_batches_); ++i) { if (pending_batches_[i].batch != nullptr) ++num_batches; @@ -1790,7 +1790,7 @@ void CallData::PendingBatchesResume(grpc_call_element* elem) { return; } // Retries not enabled; send down batches as-is. - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { size_t num_batches = 0; for (size_t i = 0; i < GPR_ARRAY_SIZE(pending_batches_); ++i) { if (pending_batches_[i].batch != nullptr) ++num_batches; @@ -1831,7 +1831,7 @@ CallData::PendingBatch* CallData::PendingBatchFind(grpc_call_element* elem, PendingBatch* pending = &pending_batches_[i]; grpc_transport_stream_op_batch* batch = pending->batch; if (batch != nullptr && predicate(batch)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: %s pending batch at index %" PRIuPTR, chand, this, log_message, i); @@ -1851,7 +1851,7 @@ void CallData::RetryCommit(grpc_call_element* elem, ChannelData* chand = static_cast(elem->channel_data); if (retry_committed_) return; retry_committed_ = true; - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: committing retries", chand, this); } if (retry_state != nullptr) { @@ -1886,7 +1886,7 @@ void CallData::DoRetry(grpc_call_element* elem, } next_attempt_time = retry_backoff_->NextAttemptTime(); } - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: retrying failed call in %" PRId64 " ms", chand, this, next_attempt_time - ExecCtx::Get()->Now()); @@ -1916,7 +1916,7 @@ bool CallData::MaybeRetry(grpc_call_element* elem, retry_state = static_cast( batch_data->subchannel_call->GetParentData()); if (retry_state->retry_dispatched) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: retry already dispatched", chand, this); } @@ -1928,14 +1928,14 @@ bool CallData::MaybeRetry(grpc_call_element* elem, if (retry_throttle_data_ != nullptr) { retry_throttle_data_->RecordSuccess(); } - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: call succeeded", chand, this); } return false; } // Status is not OK. Check whether the status is retryable. if (!retry_policy->retryable_status_codes.Contains(status)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: status %s not configured as retryable", chand, this, grpc_status_code_to_string(status)); @@ -1951,14 +1951,14 @@ bool CallData::MaybeRetry(grpc_call_element* elem, // checks, so that we don't fail to record failures due to other factors. if (retry_throttle_data_ != nullptr && !retry_throttle_data_->RecordFailure()) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: retries throttled", chand, this); } return false; } // Check whether the call is committed. if (retry_committed_) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: retries already committed", chand, this); } @@ -1967,7 +1967,7 @@ bool CallData::MaybeRetry(grpc_call_element* elem, // Check whether we have retries remaining. ++num_attempts_completed_; if (num_attempts_completed_ >= retry_policy->max_attempts) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: exceeded %d retry attempts", chand, this, retry_policy->max_attempts); } @@ -1975,7 +1975,7 @@ bool CallData::MaybeRetry(grpc_call_element* elem, } // If the call was cancelled from the surface, don't retry. if (cancel_error_ != GRPC_ERROR_NONE) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: call cancelled from surface, not retrying", chand, this); @@ -1988,14 +1988,14 @@ bool CallData::MaybeRetry(grpc_call_element* elem, // If the value is "-1" or any other unparseable string, we do not retry. uint32_t ms; if (!grpc_parse_slice_to_uint32(GRPC_MDVALUE(*server_pushback_md), &ms)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: not retrying due to server push-back", chand, this); } return false; } else { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: server push-back: retry in %u ms", chand, this, ms); } @@ -2098,7 +2098,7 @@ void CallData::RecvInitialMetadataReady(void* arg, grpc_error* error) { grpc_call_element* elem = batch_data->elem; ChannelData* chand = static_cast(elem->channel_data); CallData* calld = static_cast(elem->call_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: got recv_initial_metadata_ready, error=%s", chand, calld, grpc_error_string(error)); @@ -2122,7 +2122,7 @@ void CallData::RecvInitialMetadataReady(void* arg, grpc_error* error) { if (GPR_UNLIKELY((retry_state->trailing_metadata_available || error != GRPC_ERROR_NONE) && !retry_state->completed_recv_trailing_metadata)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: deferring recv_initial_metadata_ready " "(Trailers-Only)", @@ -2188,7 +2188,7 @@ void CallData::RecvMessageReady(void* arg, grpc_error* error) { grpc_call_element* elem = batch_data->elem; ChannelData* chand = static_cast(elem->channel_data); CallData* calld = static_cast(elem->call_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: got recv_message_ready, error=%s", chand, calld, grpc_error_string(error)); } @@ -2210,7 +2210,7 @@ void CallData::RecvMessageReady(void* arg, grpc_error* error) { if (GPR_UNLIKELY( (retry_state->recv_message == nullptr || error != GRPC_ERROR_NONE) && !retry_state->completed_recv_trailing_metadata)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: deferring recv_message_ready (nullptr " "message and recv_trailing_metadata pending)", @@ -2348,7 +2348,7 @@ void CallData::AddClosuresToFailUnstartedPendingBatches( for (size_t i = 0; i < GPR_ARRAY_SIZE(pending_batches_); ++i) { PendingBatch* pending = &pending_batches_[i]; if (PendingBatchIsUnstarted(pending, retry_state)) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: failing unstarted pending batch at index " "%" PRIuPTR, @@ -2394,7 +2394,7 @@ void CallData::RecvTrailingMetadataReady(void* arg, grpc_error* error) { grpc_call_element* elem = batch_data->elem; ChannelData* chand = static_cast(elem->channel_data); CallData* calld = static_cast(elem->call_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: got recv_trailing_metadata_ready, error=%s", chand, calld, grpc_error_string(error)); @@ -2410,7 +2410,7 @@ void CallData::RecvTrailingMetadataReady(void* arg, grpc_error* error) { batch_data->batch.payload->recv_trailing_metadata.recv_trailing_metadata; calld->GetCallStatus(elem, md_batch, GRPC_ERROR_REF(error), &status, &server_pushback_md); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: call finished, status=%s", chand, calld, grpc_status_code_to_string(status)); } @@ -2489,7 +2489,7 @@ void CallData::AddClosuresForReplayOrPendingSendOps( } } if (have_pending_send_message_ops || have_pending_send_trailing_metadata_op) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: starting next batch for pending send op(s)", chand, this); @@ -2508,7 +2508,7 @@ void CallData::OnComplete(void* arg, grpc_error* error) { grpc_call_element* elem = batch_data->elem; ChannelData* chand = static_cast(elem->channel_data); CallData* calld = static_cast(elem->call_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { char* batch_str = grpc_transport_stream_op_batch_string(&batch_data->batch); gpr_log(GPR_INFO, "chand=%p calld=%p: got on_complete, error=%s, batch=%s", chand, calld, grpc_error_string(error), batch_str); @@ -2584,7 +2584,7 @@ void CallData::AddClosureForSubchannelBatch( batch->handler_private.extra_arg = subchannel_call_.get(); GRPC_CLOSURE_INIT(&batch->handler_private.closure, StartBatchInCallCombiner, batch, grpc_schedule_on_exec_ctx); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { char* batch_str = grpc_transport_stream_op_batch_string(batch); gpr_log(GPR_INFO, "chand=%p calld=%p: starting subchannel batch: %s", chand, this, batch_str); @@ -2647,7 +2647,7 @@ void CallData::AddRetriableSendMessageOp(grpc_call_element* elem, SubchannelCallRetryState* retry_state, SubchannelCallBatchData* batch_data) { ChannelData* chand = static_cast(elem->channel_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: starting calld->send_messages[%" PRIuPTR "]", chand, this, retry_state->started_send_message_count); @@ -2730,7 +2730,7 @@ void CallData::AddRetriableRecvTrailingMetadataOp( void CallData::StartInternalRecvTrailingMetadata(grpc_call_element* elem) { ChannelData* chand = static_cast(elem->channel_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: call failed but recv_trailing_metadata not " "started; starting it internally", @@ -2762,7 +2762,7 @@ CallData::MaybeCreateSubchannelBatchForReplay( if (seen_send_initial_metadata_ && !retry_state->started_send_initial_metadata && !pending_send_initial_metadata_) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: replaying previously completed " "send_initial_metadata op", @@ -2778,7 +2778,7 @@ CallData::MaybeCreateSubchannelBatchForReplay( retry_state->started_send_message_count == retry_state->completed_send_message_count && !pending_send_message_) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: replaying previously completed " "send_message op", @@ -2798,7 +2798,7 @@ CallData::MaybeCreateSubchannelBatchForReplay( retry_state->started_send_message_count == send_messages_.size() && !retry_state->started_send_trailing_metadata && !pending_send_trailing_metadata_) { - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: replaying previously completed " "send_trailing_metadata op", @@ -2941,7 +2941,7 @@ void CallData::StartRetriableSubchannelBatches(void* arg, grpc_error* ignored) { grpc_call_element* elem = static_cast(arg); ChannelData* chand = static_cast(elem->channel_data); CallData* calld = static_cast(elem->call_data); - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: constructing retriable batches", chand, calld); } @@ -2966,7 +2966,7 @@ void CallData::StartRetriableSubchannelBatches(void* arg, grpc_error* ignored) { // Now add pending batches. calld->AddSubchannelBatchesForPendingBatches(elem, retry_state, &closures); // Start batches on subchannel call. - if (grpc_client_channel_call_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_call_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: starting %" PRIuPTR " retriable batches on subchannel_call=%p", @@ -2992,7 +2992,7 @@ void CallData::CreateSubchannelCall(grpc_call_element* elem) { grpc_error* error = GRPC_ERROR_NONE; subchannel_call_ = pick_.pick.connected_subchannel->CreateCall(call_args, &error); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: create subchannel_call=%p: error=%s", chand, this, subchannel_call_.get(), grpc_error_string(error)); } @@ -3012,7 +3012,7 @@ void CallData::PickDone(void* arg, grpc_error* error) { ChannelData* chand = static_cast(elem->channel_data); CallData* calld = static_cast(elem->call_data); if (error != GRPC_ERROR_NONE) { - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: failed to pick subchannel: error=%s", chand, calld, grpc_error_string(error)); @@ -3041,7 +3041,7 @@ class CallData::QueuedPickCanceller { auto* self = static_cast(arg); auto* chand = static_cast(self->elem_->channel_data); auto* calld = static_cast(self->elem_->call_data); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: cancelling queued pick: " "error=%s self=%p calld->pick_canceller=%p", @@ -3065,7 +3065,7 @@ class CallData::QueuedPickCanceller { void CallData::RemoveCallFromQueuedPicksLocked(grpc_call_element* elem) { auto* chand = static_cast(elem->channel_data); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: removing from queued picks list", chand, this); } @@ -3077,7 +3077,7 @@ void CallData::RemoveCallFromQueuedPicksLocked(grpc_call_element* elem) { void CallData::AddCallToQueuedPicksLocked(grpc_call_element* elem) { auto* chand = static_cast(elem->channel_data); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: adding to queued picks list", chand, this); } @@ -3090,7 +3090,7 @@ void CallData::AddCallToQueuedPicksLocked(grpc_call_element* elem) { void CallData::ApplyServiceConfigToCallLocked(grpc_call_element* elem) { ChannelData* chand = static_cast(elem->channel_data); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: applying service config to call", chand, this); } @@ -3199,7 +3199,7 @@ void CallData::StartPickLocked(void* arg, grpc_error* error) { // Attempt pick. error = GRPC_ERROR_NONE; auto pick_result = chand->picker()->Pick(&calld->pick_.pick, &error); - if (grpc_client_channel_routing_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_client_channel_routing_trace)) { gpr_log(GPR_INFO, "chand=%p calld=%p: LB pick returned %s (connected_subchannel=%p, " "error=%s)", diff --git a/src/core/ext/filters/client_channel/health/health_check_client.cc b/src/core/ext/filters/client_channel/health/health_check_client.cc index e03105c672b..a4467236662 100644 --- a/src/core/ext/filters/client_channel/health/health_check_client.cc +++ b/src/core/ext/filters/client_channel/health/health_check_client.cc @@ -63,7 +63,7 @@ HealthCheckClient::HealthCheckClient( .set_jitter(HEALTH_CHECK_RECONNECT_JITTER) .set_max_backoff(HEALTH_CHECK_RECONNECT_MAX_BACKOFF_SECONDS * 1000)) { - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "created HealthCheckClient %p", this); } GRPC_CLOSURE_INIT(&retry_timer_callback_, OnRetryTimer, this, @@ -72,7 +72,7 @@ HealthCheckClient::HealthCheckClient( } HealthCheckClient::~HealthCheckClient() { - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "destroying HealthCheckClient %p", this); } GRPC_ERROR_UNREF(error_); @@ -99,7 +99,7 @@ void HealthCheckClient::SetHealthStatus(grpc_connectivity_state state, void HealthCheckClient::SetHealthStatusLocked(grpc_connectivity_state state, grpc_error* error) { - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p: setting state=%d error=%s", this, state, grpc_error_string(error)); } @@ -115,7 +115,7 @@ void HealthCheckClient::SetHealthStatusLocked(grpc_connectivity_state state, } void HealthCheckClient::Orphan() { - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p: shutting down", this); } { @@ -145,7 +145,7 @@ void HealthCheckClient::StartCallLocked() { GPR_ASSERT(call_state_ == nullptr); SetHealthStatusLocked(GRPC_CHANNEL_CONNECTING, GRPC_ERROR_NONE); call_state_ = MakeOrphanable(Ref(), interested_parties_); - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p: created CallState %p", this, call_state_.get()); } @@ -159,7 +159,7 @@ void HealthCheckClient::StartRetryTimer() { GRPC_ERROR_CREATE_FROM_STATIC_STRING( "health check call failed; will retry after backoff")); grpc_millis next_try = retry_backoff_.NextAttemptTime(); - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p: health check call lost...", this); grpc_millis timeout = next_try - ExecCtx::Get()->Now(); if (timeout > 0) { @@ -184,7 +184,7 @@ void HealthCheckClient::OnRetryTimer(void* arg, grpc_error* error) { self->retry_timer_callback_pending_ = false; if (!self->shutting_down_ && error == GRPC_ERROR_NONE && self->call_state_ == nullptr) { - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p: restarting health check call", self); } @@ -285,7 +285,7 @@ HealthCheckClient::CallState::CallState( payload_(context_) {} HealthCheckClient::CallState::~CallState() { - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p: destroying CallState %p", health_check_client_.get(), this); } @@ -589,7 +589,7 @@ void HealthCheckClient::CallState::RecvTrailingMetadataReady( status = grpc_get_status_code_from_metadata( self->recv_trailing_metadata_.idx.named.grpc_status->md); } - if (grpc_health_check_client_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_health_check_client_trace)) { gpr_log(GPR_INFO, "HealthCheckClient %p CallState %p: health watch failed with " "status %d", diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc index e28a1495c18..ed6e8de3f21 100644 --- a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc +++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc @@ -640,7 +640,7 @@ void GrpcLb::Helper::UpdateState(grpc_connectivity_state state, // If this request is from the pending child policy, ignore it until // it reports READY, at which point we swap it into place. if (CalledByPendingChild()) { - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p helper %p] pending child policy %p reports state=%s", parent_.get(), this, parent_->pending_child_policy_.get(), @@ -682,7 +682,7 @@ void GrpcLb::Helper::UpdateState(grpc_connectivity_state state, if (parent_->serverlist_ == nullptr || (!parent_->serverlist_->ContainsAllDropEntries() && state != GRPC_CHANNEL_READY)) { - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p helper %p] state=%s passing child picker %p as-is", parent_.get(), this, grpc_connectivity_state_name(state), @@ -692,7 +692,7 @@ void GrpcLb::Helper::UpdateState(grpc_connectivity_state state, return; } // Cases 2 and 3a: wrap picker from the child in our own picker. - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p helper %p] state=%s wrapping child picker %p", parent_.get(), this, grpc_connectivity_state_name(state), picker.get()); @@ -715,7 +715,7 @@ void GrpcLb::Helper::RequestReresolution() { ? parent_->pending_child_policy_.get() : parent_->child_policy_.get(); if (child_ != latest_child_policy) return; - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Re-resolution requested from %schild policy (%p).", parent_.get(), CalledByPendingChild() ? "pending " : "", child_); @@ -802,7 +802,7 @@ void GrpcLb::BalancerCallState::Orphan() { void GrpcLb::BalancerCallState::StartQuery() { GPR_ASSERT(lb_call_ != nullptr); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] lb_calld=%p: Starting LB call %p", grpclb_policy_.get(), this, lb_call_); } @@ -1009,7 +1009,7 @@ void GrpcLb::BalancerCallState::OnBalancerMessageReceivedLocked( lb_calld->client_stats_report_interval_ = GPR_MAX( GPR_MS_PER_SEC, grpc_grpclb_duration_to_millis( &initial_response->client_stats_report_interval)); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] lb_calld=%p: Received initial LB response " "message; client load reporting interval = %" PRId64 @@ -1017,7 +1017,7 @@ void GrpcLb::BalancerCallState::OnBalancerMessageReceivedLocked( grpclb_policy, lb_calld, lb_calld->client_stats_report_interval_); } - } else if (grpc_lb_glb_trace.enabled()) { + } else if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] lb_calld=%p: Received initial LB response message; " "client load reporting NOT enabled", @@ -1030,7 +1030,7 @@ void GrpcLb::BalancerCallState::OnBalancerMessageReceivedLocked( // Have seen initial response, look for serverlist. GPR_ASSERT(lb_calld->lb_call_ != nullptr); auto serverlist_wrapper = MakeRefCounted(serverlist); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { UniquePtr serverlist_text = serverlist_wrapper->AsText(); gpr_log(GPR_INFO, "[grpclb %p] lb_calld=%p: Serverlist with %" PRIuPTR @@ -1051,7 +1051,7 @@ void GrpcLb::BalancerCallState::OnBalancerMessageReceivedLocked( // Check if the serverlist differs from the previous one. if (grpclb_policy->serverlist_ != nullptr && *grpclb_policy->serverlist_ == *serverlist_wrapper) { - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] lb_calld=%p: Incoming server list identical to " "current, ignoring.", @@ -1129,7 +1129,7 @@ void GrpcLb::BalancerCallState::OnBalancerStatusReceivedLocked( BalancerCallState* lb_calld = static_cast(arg); GrpcLb* grpclb_policy = lb_calld->grpclb_policy(); GPR_ASSERT(lb_calld->lb_call_ != nullptr); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { char* status_details = grpc_slice_to_c_string(lb_calld->lb_call_status_details_); gpr_log(GPR_INFO, @@ -1291,7 +1291,7 @@ GrpcLb::GrpcLb(Args args) grpc_uri* uri = grpc_uri_parse(server_uri, true); GPR_ASSERT(uri->path[0] != '\0'); server_name_ = gpr_strdup(uri->path[0] == '/' ? uri->path + 1 : uri->path); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Will use '%s' as the server name for LB request.", this, server_name_); @@ -1535,7 +1535,7 @@ void GrpcLb::StartBalancerCallLocked() { // Init the LB call data. GPR_ASSERT(lb_calld_ == nullptr); lb_calld_ = MakeOrphanable(Ref()); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Query for backends (lb_channel: %p, lb_calld: %p)", this, lb_channel_, lb_calld_.get()); @@ -1545,7 +1545,7 @@ void GrpcLb::StartBalancerCallLocked() { void GrpcLb::StartBalancerCallRetryTimerLocked() { grpc_millis next_try = lb_call_backoff_.NextAttemptTime(); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Connection to LB server lost...", this); grpc_millis timeout = next_try - ExecCtx::Get()->Now(); if (timeout > 0) { @@ -1572,7 +1572,7 @@ void GrpcLb::OnBalancerCallRetryTimerLocked(void* arg, grpc_error* error) { grpclb_policy->retry_timer_callback_pending_ = false; if (!grpclb_policy->shutting_down_ && error == GRPC_ERROR_NONE && grpclb_policy->lb_calld_ == nullptr) { - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Restarting call to LB server", grpclb_policy); } @@ -1656,7 +1656,7 @@ OrphanablePtr GrpcLb::CreateChildPolicyLocked( return nullptr; } helper->set_child(lb_policy.get()); - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Created new child policy %s (%p)", this, name, lb_policy.get()); } @@ -1755,7 +1755,7 @@ void GrpcLb::CreateOrUpdateChildPolicyLocked() { // Cases 1, 2b, and 3b: create a new child policy. // If child_policy_ is null, we set it (case 1), else we set // pending_child_policy_ (cases 2b and 3b). - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Creating new %schild policy %s", this, child_policy_ == nullptr ? "" : "pending ", child_policy_name); } @@ -1779,7 +1779,7 @@ void GrpcLb::CreateOrUpdateChildPolicyLocked() { } GPR_ASSERT(policy_to_update != nullptr); // Update the policy. - if (grpc_lb_glb_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_glb_trace)) { gpr_log(GPR_INFO, "[grpclb %p] Updating %schild policy %p", this, policy_to_update == pending_child_policy_.get() ? "pending " : "", policy_to_update); diff --git a/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc b/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc index 95f6a149720..bc2f6e5efd6 100644 --- a/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc +++ b/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc @@ -160,13 +160,13 @@ class PickFirst : public LoadBalancingPolicy { }; PickFirst::PickFirst(Args args) : LoadBalancingPolicy(std::move(args)) { - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p created.", this); } } PickFirst::~PickFirst() { - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Destroying Pick First %p", this); } GPR_ASSERT(subchannel_list_ == nullptr); @@ -175,7 +175,7 @@ PickFirst::~PickFirst() { void PickFirst::ShutdownLocked() { AutoChildRefsUpdater guard(this); - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p Shutting down", this); } shutdown_ = true; @@ -245,7 +245,7 @@ void PickFirst::UpdateChildRefsLocked() { void PickFirst::UpdateLocked(UpdateArgs args) { AutoChildRefsUpdater guard(this); - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p received update with %" PRIuPTR " addresses", this, args.addresses.size()); @@ -317,7 +317,7 @@ void PickFirst::UpdateLocked(UpdateArgs args) { // We do have a selected subchannel (which means it's READY), so keep // using it until one of the subchannels in the new list reports READY. if (latest_pending_subchannel_list_ != nullptr) { - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p Shutting down latest pending subchannel list " "%p, about to be replaced by newer latest %p", @@ -349,7 +349,7 @@ void PickFirst::PickFirstSubchannelData::ProcessConnectivityChangeLocked( GPR_ASSERT(connectivity_state != GRPC_CHANNEL_SHUTDOWN); // Handle updates for the currently selected subchannel. if (p->selected_ == this) { - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p selected subchannel connectivity changed to %s", p, grpc_connectivity_state_name(connectivity_state)); @@ -358,7 +358,7 @@ void PickFirst::PickFirstSubchannelData::ProcessConnectivityChangeLocked( // pending update, switch to the pending update. if (connectivity_state != GRPC_CHANNEL_READY && p->latest_pending_subchannel_list_ != nullptr) { - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p promoting pending subchannel list %p to " "replace %p", @@ -492,7 +492,7 @@ void PickFirst::PickFirstSubchannelData::ProcessUnselectedReadyLocked() { subchannel_list() == p->latest_pending_subchannel_list_.get()); // Case 2. Promote p->latest_pending_subchannel_list_ to p->subchannel_list_. if (subchannel_list() == p->latest_pending_subchannel_list_.get()) { - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p promoting pending subchannel list %p to " "replace %p", @@ -506,7 +506,7 @@ void PickFirst::PickFirstSubchannelData::ProcessUnselectedReadyLocked() { p->channel_control_helper()->UpdateState( GRPC_CHANNEL_READY, UniquePtr(New(connected_subchannel()->Ref()))); - if (grpc_lb_pick_first_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_pick_first_trace)) { gpr_log(GPR_INFO, "Pick First %p selected subchannel %p", p, subchannel()); } } diff --git a/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc b/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc index 2d4ecd38f96..6d603913d82 100644 --- a/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc +++ b/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc @@ -212,7 +212,7 @@ RoundRobin::Picker::Picker(RoundRobin* parent, // TODO(roth): rand(3) is not thread-safe. This should be replaced with // something better as part of https://github.com/grpc/grpc/issues/17891. last_picked_index_ = rand() % subchannels_.size(); - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p picker %p] created picker from subchannel_list=%p " "with %" PRIuPTR " READY subchannels; last_picked_index_=%" PRIuPTR, @@ -224,7 +224,7 @@ RoundRobin::Picker::Picker(RoundRobin* parent, RoundRobin::PickResult RoundRobin::Picker::Pick(PickArgs* pick, grpc_error** error) { last_picked_index_ = (last_picked_index_ + 1) % subchannels_.size(); - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p picker %p] returning index %" PRIuPTR ", connected_subchannel=%p", @@ -240,13 +240,13 @@ RoundRobin::PickResult RoundRobin::Picker::Pick(PickArgs* pick, // RoundRobin::RoundRobin(Args args) : LoadBalancingPolicy(std::move(args)) { - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p] Created", this); } } RoundRobin::~RoundRobin() { - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p] Destroying Round Robin policy", this); } GPR_ASSERT(subchannel_list_ == nullptr); @@ -255,7 +255,7 @@ RoundRobin::~RoundRobin() { void RoundRobin::ShutdownLocked() { AutoChildRefsUpdater guard(this); - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p] Shutting down", this); } shutdown_ = true; @@ -403,7 +403,7 @@ void RoundRobin::RoundRobinSubchannelList:: // therefore we would not be receiving a notification for them. GPR_ASSERT(p->latest_pending_subchannel_list_.get() == this); GPR_ASSERT(!shutting_down()); - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { const size_t old_num_subchannels = p->subchannel_list_ != nullptr ? p->subchannel_list_->num_subchannels() @@ -424,7 +424,7 @@ void RoundRobin::RoundRobinSubchannelList:: void RoundRobin::RoundRobinSubchannelData::UpdateConnectivityStateLocked( grpc_connectivity_state connectivity_state) { RoundRobin* p = static_cast(subchannel_list()->policy()); - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log( GPR_INFO, "[RR %p] connectivity changed for subchannel %p, subchannel_list %p " @@ -449,7 +449,7 @@ void RoundRobin::RoundRobinSubchannelData::ProcessConnectivityChangeLocked( // when the subchannel list was created, we'd wind up in a constant // loop of re-resolution. if (connectivity_state == GRPC_CHANNEL_TRANSIENT_FAILURE) { - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p] Subchannel %p has gone into TRANSIENT_FAILURE. " "Requesting re-resolution", @@ -467,13 +467,13 @@ void RoundRobin::RoundRobinSubchannelData::ProcessConnectivityChangeLocked( void RoundRobin::UpdateLocked(UpdateArgs args) { AutoChildRefsUpdater guard(this); - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p] received update with %" PRIuPTR " addresses", this, args.addresses.size()); } // Replace latest_pending_subchannel_list_. if (latest_pending_subchannel_list_ != nullptr) { - if (grpc_lb_round_robin_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_round_robin_trace)) { gpr_log(GPR_INFO, "[RR %p] Shutting down previous pending subchannel list %p", this, latest_pending_subchannel_list_.get()); diff --git a/src/core/ext/filters/client_channel/lb_policy/subchannel_list.h b/src/core/ext/filters/client_channel/lb_policy/subchannel_list.h index 004ee04459b..c14f6828e36 100644 --- a/src/core/ext/filters/client_channel/lb_policy/subchannel_list.h +++ b/src/core/ext/filters/client_channel/lb_policy/subchannel_list.h @@ -298,7 +298,7 @@ template void SubchannelData:: UnrefSubchannelLocked(const char* reason) { if (subchannel_ != nullptr) { - if (subchannel_list_->tracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*subchannel_list_->tracer())) { gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR " (subchannel %p): unreffing subchannel", @@ -323,7 +323,7 @@ void SubchannelData void SubchannelData::StartConnectivityWatchLocked() { - if (subchannel_list_->tracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*subchannel_list_->tracer())) { gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR " (subchannel %p): starting watch: requesting connectivity change " @@ -345,7 +345,7 @@ void SubchannelData void SubchannelData::RenewConnectivityWatchLocked() { - if (subchannel_list_->tracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*subchannel_list_->tracer())) { gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR " (subchannel %p): renewing watch: requesting connectivity change " @@ -365,7 +365,7 @@ void SubchannelData void SubchannelData::StopConnectivityWatchLocked() { - if (subchannel_list_->tracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*subchannel_list_->tracer())) { gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR " (subchannel %p): stopping connectivity watch", @@ -381,7 +381,7 @@ void SubchannelData void SubchannelData:: CancelConnectivityWatchLocked(const char* reason) { - if (subchannel_list_->tracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*subchannel_list_->tracer())) { gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR " (subchannel %p): canceling connectivity watch (%s)", @@ -413,7 +413,7 @@ bool SubchannelDatatracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*subchannel_list_->tracer())) { gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR " (subchannel %p): state is READY but connected subchannel is " @@ -436,7 +436,7 @@ template void SubchannelData:: OnConnectivityChangedLocked(void* arg, grpc_error* error) { SubchannelData* sd = static_cast(arg); - if (sd->subchannel_list_->tracer()->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*sd->subchannel_list_->tracer())) { gpr_log( GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR " of %" PRIuPTR @@ -490,7 +490,7 @@ SubchannelList::SubchannelList( policy_(policy), tracer_(tracer), combiner_(GRPC_COMBINER_REF(combiner, "subchannel_list")) { - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "[%s %p] Creating subchannel list %p for %" PRIuPTR " subchannels", tracer_->name(), policy, this, addresses.size()); @@ -524,7 +524,7 @@ SubchannelList::SubchannelList( grpc_channel_args_destroy(new_args); if (subchannel == nullptr) { // Subchannel could not be created. - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { char* address_uri = grpc_sockaddr_to_uri(&addresses[i].address()); gpr_log(GPR_INFO, "[%s %p] could not create subchannel for address uri %s, " @@ -534,7 +534,7 @@ SubchannelList::SubchannelList( } continue; } - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { char* address_uri = grpc_sockaddr_to_uri(&addresses[i].address()); gpr_log(GPR_INFO, "[%s %p] subchannel list %p index %" PRIuPTR @@ -549,7 +549,7 @@ SubchannelList::SubchannelList( template SubchannelList::~SubchannelList() { - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "[%s %p] Destroying subchannel_list %p", tracer_->name(), policy_, this); } @@ -558,7 +558,7 @@ SubchannelList::~SubchannelList() { template void SubchannelList::ShutdownLocked() { - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "[%s %p] Shutting down subchannel_list %p", tracer_->name(), policy_, this); } diff --git a/src/core/ext/filters/client_channel/lb_policy/xds/xds.cc b/src/core/ext/filters/client_channel/lb_policy/xds/xds.cc index f3806bdbde1..3eb94371c71 100644 --- a/src/core/ext/filters/client_channel/lb_policy/xds/xds.cc +++ b/src/core/ext/filters/client_channel/lb_policy/xds/xds.cc @@ -608,7 +608,7 @@ void XdsLb::FallbackHelper::UpdateState(grpc_connectivity_state state, // If this request is from the pending fallback policy, ignore it until // it reports READY, at which point we swap it into place. if (CalledByPendingFallback()) { - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log( GPR_INFO, "[xdslb %p helper %p] pending fallback policy %p reports state=%s", @@ -635,7 +635,7 @@ void XdsLb::FallbackHelper::RequestReresolution() { ? parent_->pending_fallback_policy_.get() : parent_->fallback_policy_.get(); if (child_ != latest_fallback_policy) return; - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Re-resolution requested from the fallback policy (%p).", parent_.get(), child_); @@ -755,7 +755,7 @@ void XdsLb::BalancerChannelState::Orphan() { void XdsLb::BalancerChannelState::StartCallRetryTimerLocked() { grpc_millis next_try = lb_call_backoff_.NextAttemptTime(); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Failed to connect to LB server (lb_chand: %p)...", xdslb_policy_.get(), this); @@ -781,7 +781,7 @@ void XdsLb::BalancerChannelState::OnCallRetryTimerLocked(void* arg, lb_chand->retry_timer_callback_pending_ = false; if (!lb_chand->shutting_down_ && error == GRPC_ERROR_NONE && lb_chand->lb_calld_ == nullptr) { - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Restarting call to LB server (lb_chand: %p)", lb_chand->xdslb_policy_.get(), lb_chand); @@ -796,7 +796,7 @@ void XdsLb::BalancerChannelState::StartCallLocked() { GPR_ASSERT(channel_ != nullptr); GPR_ASSERT(lb_calld_ == nullptr); lb_calld_ = MakeOrphanable(Ref()); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Query for backends (lb_chand: %p, lb_calld: %p)", xdslb_policy_.get(), this, lb_calld_.get()); @@ -932,7 +932,7 @@ void XdsLb::BalancerChannelState::BalancerCallState::Orphan() { void XdsLb::BalancerChannelState::BalancerCallState::StartQuery() { GPR_ASSERT(lb_call_ != nullptr); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Starting LB call (lb_calld: %p, lb_call: %p)", xdslb_policy(), this, lb_call_); } @@ -1121,7 +1121,7 @@ void XdsLb::BalancerChannelState::BalancerCallState:: GPR_MAX(GPR_MS_PER_SEC, interval); } } - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { if (lb_calld->client_stats_report_interval_ != 0) { gpr_log(GPR_INFO, "[xdslb %p] Received initial LB response message; " @@ -1140,7 +1140,7 @@ void XdsLb::BalancerChannelState::BalancerCallState:: response_slice)) != nullptr) { // Have seen initial response, look for serverlist. GPR_ASSERT(lb_calld->lb_call_ != nullptr); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Serverlist with %" PRIuPTR " servers received", xdslb_policy, serverlist->num_servers); @@ -1159,7 +1159,7 @@ void XdsLb::BalancerChannelState::BalancerCallState:: // such channels don't have any current call but we have checked this call // is a current call. if (!lb_calld->lb_chand_->IsCurrentChannel()) { - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Promoting pending LB channel %p to replace " "current LB channel %p", @@ -1180,7 +1180,7 @@ void XdsLb::BalancerChannelState::BalancerCallState:: if (!xdslb_policy->locality_serverlist_.empty() && xds_grpclb_serverlist_equals( xdslb_policy->locality_serverlist_[0]->serverlist, serverlist)) { - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Incoming server list identical to current, " "ignoring.", @@ -1252,7 +1252,7 @@ void XdsLb::BalancerChannelState::BalancerCallState:: XdsLb* xdslb_policy = lb_calld->xdslb_policy(); BalancerChannelState* lb_chand = lb_calld->lb_chand_.get(); GPR_ASSERT(lb_calld->lb_call_ != nullptr); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { char* status_details = grpc_slice_to_c_string(lb_calld->lb_call_status_details_); gpr_log(GPR_INFO, @@ -1271,7 +1271,7 @@ void XdsLb::BalancerChannelState::BalancerCallState:: if (lb_chand != xdslb_policy->LatestLbChannel()) { // This channel must be the current one and there is a pending one. Swap // in the pending one and we are done. - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Promoting pending LB channel %p to replace " "current LB channel %p", @@ -1370,7 +1370,7 @@ XdsLb::XdsLb(Args args) grpc_uri* uri = grpc_uri_parse(server_uri, true); GPR_ASSERT(uri->path[0] != '\0'); server_name_ = gpr_strdup(uri->path[0] == '/' ? uri->path + 1 : uri->path); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Will use '%s' as the server name for LB request.", this, server_name_); @@ -1569,7 +1569,7 @@ void XdsLb::OnFallbackTimerLocked(void* arg, grpc_error* error) { // this callback actually runs, don't fall back. if (xdslb_policy->fallback_at_startup_checks_pending_ && !xdslb_policy->shutting_down_ && error == GRPC_ERROR_NONE) { - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Child policy not ready after fallback timeout; " "entering fallback mode", @@ -1657,7 +1657,7 @@ void XdsLb::UpdateFallbackPolicyLocked() { // Cases 1, 2b, and 3b: create a new child policy. // If child_policy_ is null, we set it (case 1), else we set // pending_child_policy_ (cases 2b and 3b). - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Creating new %sfallback policy %s", this, fallback_policy_ == nullptr ? "" : "pending ", fallback_policy_name); @@ -1681,7 +1681,7 @@ void XdsLb::UpdateFallbackPolicyLocked() { } GPR_ASSERT(policy_to_update != nullptr); // Update the policy. - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log( GPR_INFO, "[xdslb %p] Updating %sfallback policy %p", this, policy_to_update == pending_fallback_policy_.get() ? "pending " : "", @@ -1707,7 +1707,7 @@ OrphanablePtr XdsLb::CreateFallbackPolicyLocked( return nullptr; } helper->set_child(lb_policy.get()); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Created new fallback policy %s (%p)", this, name, lb_policy.get()); } @@ -1829,7 +1829,7 @@ XdsLb::LocalityMap::LocalityEntry::CreateChildPolicyLocked( return nullptr; } helper->set_child(lb_policy.get()); - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Created new child policy %s (%p)", this, name, lb_policy.get()); } @@ -1920,7 +1920,7 @@ void XdsLb::LocalityMap::LocalityEntry::UpdateLocked( // Cases 1, 2b, and 3b: create a new child policy. // If child_policy_ is null, we set it (case 1), else we set // pending_child_policy_ (cases 2b and 3b). - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Creating new %schild policy %s", this, child_policy_ == nullptr ? "" : "pending ", child_policy_name); } @@ -1943,7 +1943,7 @@ void XdsLb::LocalityMap::LocalityEntry::UpdateLocked( } GPR_ASSERT(policy_to_update != nullptr); // Update the policy. - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Updating %schild policy %p", this, policy_to_update == pending_child_policy_.get() ? "pending " : "", policy_to_update); @@ -2029,7 +2029,7 @@ void XdsLb::LocalityMap::LocalityEntry::Helper::UpdateState( // If this request is from the pending child policy, ignore it until // it reports READY, at which point we swap it into place. if (CalledByPendingChild()) { - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p helper %p] pending child policy %p reports state=%s", entry_->parent_.get(), this, entry_->pending_child_policy_.get(), @@ -2129,7 +2129,7 @@ void XdsLb::LocalityMap::LocalityEntry::Helper::RequestReresolution() { if (entry_->pending_child_policy_ != nullptr && !CalledByPendingChild()) { return; } - if (grpc_lb_xds_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_lb_xds_trace)) { gpr_log(GPR_INFO, "[xdslb %p] Re-resolution requested from the internal RR policy " "(%p).", diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc index 97ef61d2f33..5e7921697d4 100644 --- a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc +++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc @@ -101,7 +101,7 @@ static void log_address_sorting_list(const ServerAddressList& addresses, } void grpc_cares_wrapper_address_sorting_sort(ServerAddressList* addresses) { - if (grpc_trace_cares_address_sorting.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cares_address_sorting)) { log_address_sorting_list(*addresses, "input"); } address_sorting_sortable* sortables = (address_sorting_sortable*)gpr_zalloc( @@ -120,7 +120,7 @@ void grpc_cares_wrapper_address_sorting_sort(ServerAddressList* addresses) { } gpr_free(sortables); *addresses = std::move(sorted); - if (grpc_trace_cares_address_sorting.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cares_address_sorting)) { log_address_sorting_list(*addresses, "output"); } } diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h index 28082504565..881dfcdcee6 100644 --- a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h +++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h @@ -33,7 +33,7 @@ extern grpc_core::TraceFlag grpc_trace_cares_address_sorting; extern grpc_core::TraceFlag grpc_trace_cares_resolver; #define GRPC_CARES_TRACE_LOG(format, ...) \ - if (grpc_trace_cares_resolver.enabled()) { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cares_resolver)) { \ gpr_log(GPR_DEBUG, "(c-ares resolver) " format, __VA_ARGS__); \ } diff --git a/src/core/ext/filters/client_channel/resolving_lb_policy.cc b/src/core/ext/filters/client_channel/resolving_lb_policy.cc index 193c9e256ed..6fc80d4fdae 100644 --- a/src/core/ext/filters/client_channel/resolving_lb_policy.cc +++ b/src/core/ext/filters/client_channel/resolving_lb_policy.cc @@ -77,7 +77,7 @@ class ResolvingLoadBalancingPolicy::ResolverResultHandler : parent_(std::move(parent)) {} ~ResolverResultHandler() { - if (parent_->tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*(parent_->tracer_))) { gpr_log(GPR_INFO, "resolving_lb=%p: resolver shutdown complete", parent_.get()); } @@ -125,7 +125,7 @@ class ResolvingLoadBalancingPolicy::ResolvingControlHelper // If this request is from the pending child policy, ignore it until // it reports READY, at which point we swap it into place. if (CalledByPendingChild()) { - if (parent_->tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*(parent_->tracer_))) { gpr_log(GPR_INFO, "resolving_lb=%p helper=%p: pending child policy %p reports " "state=%s", @@ -151,7 +151,7 @@ class ResolvingLoadBalancingPolicy::ResolvingControlHelper if (parent_->pending_lb_policy_ != nullptr && !CalledByPendingChild()) { return; } - if (parent_->tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*(parent_->tracer_))) { gpr_log(GPR_INFO, "resolving_lb=%p: started name re-resolving", parent_.get()); } @@ -241,7 +241,7 @@ void ResolvingLoadBalancingPolicy::ShutdownLocked() { resolver_.reset(); MutexLock lock(&lb_policy_mu_); if (lb_policy_ != nullptr) { - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: shutting down lb_policy=%p", this, lb_policy_.get()); } @@ -250,7 +250,7 @@ void ResolvingLoadBalancingPolicy::ShutdownLocked() { lb_policy_.reset(); } if (pending_lb_policy_ != nullptr) { - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: shutting down pending lb_policy=%p", this, pending_lb_policy_.get()); } @@ -298,7 +298,7 @@ void ResolvingLoadBalancingPolicy::FillChildRefsForChannelz( } void ResolvingLoadBalancingPolicy::StartResolvingLocked() { - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: starting name resolution", this); } GPR_ASSERT(!started_resolving_); @@ -314,7 +314,7 @@ void ResolvingLoadBalancingPolicy::OnResolverError(grpc_error* error) { GRPC_ERROR_UNREF(error); return; } - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: resolver transient failure: %s", this, grpc_error_string(error)); } @@ -398,7 +398,7 @@ void ResolvingLoadBalancingPolicy::CreateOrUpdateLbPolicyLocked( // Cases 1, 2b, and 3b: create a new child policy. // If lb_policy_ is null, we set it (case 1), else we set // pending_lb_policy_ (cases 2b and 3b). - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: Creating new %schild policy %s", this, lb_policy_ == nullptr ? "" : "pending ", lb_policy_name); } @@ -419,7 +419,7 @@ void ResolvingLoadBalancingPolicy::CreateOrUpdateLbPolicyLocked( } GPR_ASSERT(policy_to_update != nullptr); // Update the policy. - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: Updating %schild policy %p", this, policy_to_update == pending_lb_policy_.get() ? "pending " : "", policy_to_update); @@ -458,7 +458,7 @@ ResolvingLoadBalancingPolicy::CreateLbPolicyLocked( return nullptr; } helper->set_child(lb_policy.get()); - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: created new LB policy \"%s\" (%p)", this, lb_policy_name, lb_policy.get()); } @@ -514,7 +514,7 @@ void ResolvingLoadBalancingPolicy::OnResolverResultChangedLocked( Resolver::Result result) { // Handle race conditions. if (resolver_ == nullptr) return; - if (tracer_->enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(*tracer_)) { gpr_log(GPR_INFO, "resolving_lb=%p: got resolver result", this); } // We only want to trace the address resolution in the follow cases: diff --git a/src/core/ext/filters/http/message_compress/message_compress_filter.cc b/src/core/ext/filters/http/message_compress/message_compress_filter.cc index a53d813c3f0..d2b1f6794cd 100644 --- a/src/core/ext/filters/http/message_compress/message_compress_filter.cc +++ b/src/core/ext/filters/http/message_compress/message_compress_filter.cc @@ -249,7 +249,7 @@ static void finish_send_message(grpc_call_element* elem) { bool did_compress = grpc_msg_compress(calld->message_compression_algorithm, &calld->slices, &tmp); if (did_compress) { - if (grpc_compression_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_compression_trace)) { const char* algo_name; const size_t before_size = calld->slices.length; const size_t after_size = tmp.length; @@ -265,7 +265,7 @@ static void finish_send_message(grpc_call_element* elem) { grpc_slice_buffer_swap(&calld->slices, &tmp); send_flags |= GRPC_WRITE_INTERNAL_COMPRESS; } else { - if (grpc_compression_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_compression_trace)) { const char* algo_name; GPR_ASSERT(grpc_message_compression_algorithm_name( calld->message_compression_algorithm, &algo_name)); diff --git a/src/core/ext/transport/chttp2/transport/chttp2_transport.cc b/src/core/ext/transport/chttp2/transport/chttp2_transport.cc index 5ced5829cc1..ccc3077f42b 100644 --- a/src/core/ext/transport/chttp2/transport/chttp2_transport.cc +++ b/src/core/ext/transport/chttp2/transport/chttp2_transport.cc @@ -1249,7 +1249,7 @@ void grpc_chttp2_complete_closure_step(grpc_chttp2_transport* t, return; } closure->next_data.scratch -= CLOSURE_BARRIER_FIRST_REF_BIT; - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { const char* errstr = grpc_error_string(error); gpr_log( GPR_INFO, @@ -1401,7 +1401,7 @@ static void perform_stream_op_locked(void* stream_op, s->context = op->payload->context; s->traced = op->is_traced; - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* str = grpc_transport_stream_op_batch_string(op); gpr_log(GPR_INFO, "perform_stream_op_locked: %s; on_complete = %p", str, op->on_complete); @@ -1705,7 +1705,7 @@ static void perform_stream_op(grpc_transport* gt, grpc_stream* gs, } } - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* str = grpc_transport_stream_op_batch_string(op); gpr_log(GPR_INFO, "perform_stream_op[s=%p]: %s", s, str); gpr_free(str); @@ -1872,7 +1872,7 @@ static void perform_transport_op_locked(void* stream_op, static void perform_transport_op(grpc_transport* gt, grpc_transport_op* op) { grpc_chttp2_transport* t = reinterpret_cast(gt); - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* msg = grpc_transport_op_string(op); gpr_log(GPR_INFO, "perform_transport_op[t=%p]: %s", t, msg); gpr_free(msg); @@ -2619,7 +2619,7 @@ static void schedule_bdp_ping_locked(grpc_chttp2_transport* t) { static void start_bdp_ping_locked(void* tp, grpc_error* error) { grpc_chttp2_transport* t = static_cast(tp); - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "%s: Start BDP ping err=%s", t->peer_string, grpc_error_string(error)); } @@ -2635,7 +2635,7 @@ static void start_bdp_ping_locked(void* tp, grpc_error* error) { static void finish_bdp_ping_locked(void* tp, grpc_error* error) { grpc_chttp2_transport* t = static_cast(tp); - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "%s: Complete BDP ping err=%s", t->peer_string, grpc_error_string(error)); } @@ -2767,7 +2767,7 @@ static void start_keepalive_ping_locked(void* arg, grpc_error* error) { if (t->channelz_socket != nullptr) { t->channelz_socket->RecordKeepaliveSent(); } - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "%s: Start keepalive ping", t->peer_string); } GRPC_CHTTP2_REF_TRANSPORT(t, "keepalive watchdog"); @@ -2780,7 +2780,7 @@ static void finish_keepalive_ping_locked(void* arg, grpc_error* error) { grpc_chttp2_transport* t = static_cast(arg); if (t->keepalive_state == GRPC_CHTTP2_KEEPALIVE_STATE_PINGING) { if (error == GRPC_ERROR_NONE) { - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "%s: Finish keepalive ping", t->peer_string); } t->keepalive_state = GRPC_CHTTP2_KEEPALIVE_STATE_WAITING; @@ -3090,7 +3090,7 @@ static void benign_reclaimer_locked(void* arg, grpc_error* error) { grpc_chttp2_stream_map_size(&t->stream_map) == 0) { /* Channel with no active streams: send a goaway to try and make it * disconnect cleanly */ - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "HTTP2: %s - send goaway to free memory", t->peer_string); } @@ -3098,7 +3098,8 @@ static void benign_reclaimer_locked(void* arg, grpc_error* error) { grpc_error_set_int( GRPC_ERROR_CREATE_FROM_STATIC_STRING("Buffers full"), GRPC_ERROR_INT_HTTP2_ERROR, GRPC_HTTP2_ENHANCE_YOUR_CALM)); - } else if (error == GRPC_ERROR_NONE && grpc_resource_quota_trace.enabled()) { + } else if (error == GRPC_ERROR_NONE && + GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "HTTP2: %s - skip benign reclamation, there are still %" PRIdPTR " streams", @@ -3119,7 +3120,7 @@ static void destructive_reclaimer_locked(void* arg, grpc_error* error) { if (error == GRPC_ERROR_NONE && n > 0) { grpc_chttp2_stream* s = static_cast( grpc_chttp2_stream_map_rand(&t->stream_map)); - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "HTTP2: %s - abandon stream id %d", t->peer_string, s->id); } diff --git a/src/core/ext/transport/chttp2/transport/flow_control.h b/src/core/ext/transport/chttp2/transport/flow_control.h index 120fefc8b79..49e206fca3d 100644 --- a/src/core/ext/transport/chttp2/transport/flow_control.h +++ b/src/core/ext/transport/chttp2/transport/flow_control.h @@ -127,7 +127,7 @@ class FlowControlTrace { StreamFlowControl* sfc); void Finish(); - const bool enabled_ = grpc_flowctl_trace.enabled(); + const bool enabled_ = GRPC_TRACE_FLAG_ENABLED(grpc_flowctl_trace); TransportFlowControl* tfc_; StreamFlowControl* sfc_; diff --git a/src/core/ext/transport/chttp2/transport/frame_settings.cc b/src/core/ext/transport/chttp2/transport/frame_settings.cc index ed1554e2fef..3f84679ec31 100644 --- a/src/core/ext/transport/chttp2/transport/frame_settings.cc +++ b/src/core/ext/transport/chttp2/transport/frame_settings.cc @@ -217,19 +217,20 @@ grpc_error* grpc_chttp2_settings_parser_parse(void* p, grpc_chttp2_transport* t, parser->incoming_settings[id] != parser->value) { t->initial_window_update += static_cast(parser->value) - parser->incoming_settings[id]; - if (grpc_http_trace.enabled() || grpc_flowctl_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace) || + GRPC_TRACE_FLAG_ENABLED(grpc_flowctl_trace)) { gpr_log(GPR_INFO, "%p[%s] adding %d for initial_window change", t, t->is_client ? "cli" : "svr", static_cast(t->initial_window_update)); } } parser->incoming_settings[id] = parser->value; - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "CHTTP2:%s:%s: got setting %s = %d", t->is_client ? "CLI" : "SVR", t->peer_string, sp->name, parser->value); } - } else if (grpc_http_trace.enabled()) { + } else if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_ERROR, "CHTTP2: Ignoring unknown setting %d (value %d)", parser->id, parser->value); } diff --git a/src/core/ext/transport/chttp2/transport/hpack_encoder.cc b/src/core/ext/transport/chttp2/transport/hpack_encoder.cc index 1ae81fe37ff..d2607e97707 100644 --- a/src/core/ext/transport/chttp2/transport/hpack_encoder.cc +++ b/src/core/ext/transport/chttp2/transport/hpack_encoder.cc @@ -461,7 +461,7 @@ static void hpack_enc(grpc_chttp2_hpack_compressor* c, grpc_mdelem elem, "Reserved header (colon-prefixed) happening after regular ones."); } - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* k = grpc_slice_to_c_string(GRPC_MDKEY(elem)); char* v = nullptr; if (grpc_is_binary_header(GRPC_MDKEY(elem))) { @@ -660,7 +660,7 @@ void grpc_chttp2_hpack_compressor_set_max_table_size( } } c->advertise_table_size_change = 1; - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "set max table size from encoder to %d", max_table_size); } } diff --git a/src/core/ext/transport/chttp2/transport/hpack_parser.cc b/src/core/ext/transport/chttp2/transport/hpack_parser.cc index 7a37d37fd10..6e422127511 100644 --- a/src/core/ext/transport/chttp2/transport/hpack_parser.cc +++ b/src/core/ext/transport/chttp2/transport/hpack_parser.cc @@ -624,7 +624,7 @@ static const uint8_t inverse_base64[256] = { /* emission helpers */ static grpc_error* on_hdr(grpc_chttp2_hpack_parser* p, grpc_mdelem md, int add_to_table) { - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* k = grpc_slice_to_c_string(GRPC_MDKEY(md)); char* v = nullptr; if (grpc_is_binary_header(GRPC_MDKEY(md))) { @@ -994,7 +994,7 @@ static grpc_error* parse_lithdr_nvridx_v(grpc_chttp2_hpack_parser* p, /* finish parsing a max table size change */ static grpc_error* finish_max_tbl_size(grpc_chttp2_hpack_parser* p, const uint8_t* cur, const uint8_t* end) { - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "MAX TABLE SIZE: %d", p->index); } grpc_error* err = diff --git a/src/core/ext/transport/chttp2/transport/hpack_table.cc b/src/core/ext/transport/chttp2/transport/hpack_table.cc index fcfb01872bf..16aeb49df4d 100644 --- a/src/core/ext/transport/chttp2/transport/hpack_table.cc +++ b/src/core/ext/transport/chttp2/transport/hpack_table.cc @@ -247,7 +247,7 @@ void grpc_chttp2_hptbl_set_max_bytes(grpc_chttp2_hptbl* tbl, if (tbl->max_bytes == max_bytes) { return; } - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "Update hpack parser max size to %d", max_bytes); } while (tbl->mem_used > max_bytes) { @@ -270,7 +270,7 @@ grpc_error* grpc_chttp2_hptbl_set_current_table_size(grpc_chttp2_hptbl* tbl, gpr_free(msg); return err; } - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_INFO, "Update hpack parser table size to %d", bytes); } while (tbl->mem_used > bytes) { diff --git a/src/core/ext/transport/chttp2/transport/internal.h b/src/core/ext/transport/chttp2/transport/internal.h index 0a55ee111ee..b3a2545a189 100644 --- a/src/core/ext/transport/chttp2/transport/internal.h +++ b/src/core/ext/transport/chttp2/transport/internal.h @@ -771,11 +771,10 @@ void grpc_chttp2_complete_closure_step(grpc_chttp2_transport* t, // extern grpc_core::TraceFlag grpc_http_trace; // extern grpc_core::TraceFlag grpc_flowctl_trace; -#define GRPC_CHTTP2_IF_TRACING(stmt) \ - if (!(grpc_http_trace.enabled())) \ - ; \ - else \ - stmt +#define GRPC_CHTTP2_IF_TRACING(stmt) \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { \ + (stmt); \ + } void grpc_chttp2_fake_status(grpc_chttp2_transport* t, grpc_chttp2_stream* stream, grpc_error* error); diff --git a/src/core/ext/transport/chttp2/transport/parsing.cc b/src/core/ext/transport/chttp2/transport/parsing.cc index 84b2275ebc4..15648c06fcd 100644 --- a/src/core/ext/transport/chttp2/transport/parsing.cc +++ b/src/core/ext/transport/chttp2/transport/parsing.cc @@ -304,7 +304,7 @@ static grpc_error* init_frame_parser(grpc_chttp2_transport* t) { case GRPC_CHTTP2_FRAME_GOAWAY: return init_goaway_parser(t); default: - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { gpr_log(GPR_ERROR, "Unknown frame type %02x", t->incoming_frame_type); } return init_skip_frame_parser(t, 0); @@ -400,7 +400,7 @@ static void on_initial_header(void* tp, grpc_mdelem md) { grpc_chttp2_stream* s = t->incoming_stream; GPR_ASSERT(s != nullptr); - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* key = grpc_slice_to_c_string(GRPC_MDKEY(md)); char* value = grpc_dump_slice(GRPC_MDVALUE(md), GPR_DUMP_HEX | GPR_DUMP_ASCII); @@ -496,7 +496,7 @@ static void on_trailing_header(void* tp, grpc_mdelem md) { grpc_chttp2_stream* s = t->incoming_stream; GPR_ASSERT(s != nullptr); - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { char* key = grpc_slice_to_c_string(GRPC_MDKEY(md)); char* value = grpc_dump_slice(GRPC_MDVALUE(md), GPR_DUMP_HEX | GPR_DUMP_ASCII); @@ -761,7 +761,7 @@ static grpc_error* parse_frame_slice(grpc_chttp2_transport* t, if (GPR_LIKELY(err == GRPC_ERROR_NONE)) { return err; } else if (grpc_error_get_int(err, GRPC_ERROR_INT_STREAM_ID, &unused)) { - if (grpc_http_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { const char* msg = grpc_error_string(err); gpr_log(GPR_ERROR, "%s", msg); } diff --git a/src/core/ext/transport/chttp2/transport/stream_lists.cc b/src/core/ext/transport/chttp2/transport/stream_lists.cc index 6626170a7e4..db79899dedc 100644 --- a/src/core/ext/transport/chttp2/transport/stream_lists.cc +++ b/src/core/ext/transport/chttp2/transport/stream_lists.cc @@ -67,7 +67,7 @@ static bool stream_list_pop(grpc_chttp2_transport* t, s->included[id] = 0; } *stream = s; - if (s && grpc_trace_http2_stream_state.enabled()) { + if (s && GRPC_TRACE_FLAG_ENABLED(grpc_trace_http2_stream_state)) { gpr_log(GPR_INFO, "%p[%d][%s]: pop from %s", t, s->id, t->is_client ? "cli" : "svr", stream_list_id_string(id)); } @@ -89,7 +89,7 @@ static void stream_list_remove(grpc_chttp2_transport* t, grpc_chttp2_stream* s, } else { t->lists[id].tail = s->links[id].prev; } - if (grpc_trace_http2_stream_state.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_http2_stream_state)) { gpr_log(GPR_INFO, "%p[%d][%s]: remove from %s", t, s->id, t->is_client ? "cli" : "svr", stream_list_id_string(id)); } @@ -121,7 +121,7 @@ static void stream_list_add_tail(grpc_chttp2_transport* t, } t->lists[id].tail = s; s->included[id] = 1; - if (grpc_trace_http2_stream_state.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_http2_stream_state)) { gpr_log(GPR_INFO, "%p[%d][%s]: add to %s", t, s->id, t->is_client ? "cli" : "svr", stream_list_id_string(id)); } diff --git a/src/core/ext/transport/chttp2/transport/writing.cc b/src/core/ext/transport/chttp2/transport/writing.cc index 90015bd97ec..f3cb390dc7a 100644 --- a/src/core/ext/transport/chttp2/transport/writing.cc +++ b/src/core/ext/transport/chttp2/transport/writing.cc @@ -53,7 +53,8 @@ static void maybe_initiate_ping(grpc_chttp2_transport* t) { } if (!grpc_closure_list_empty(pq->lists[GRPC_CHTTP2_PCL_INFLIGHT])) { /* ping already in-flight: wait */ - if (grpc_http_trace.enabled() || grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace) || + GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "%s: Ping delayed [%p]: already pinging", t->is_client ? "CLIENT" : "SERVER", t->peer_string); } @@ -62,7 +63,8 @@ static void maybe_initiate_ping(grpc_chttp2_transport* t) { if (t->ping_state.pings_before_data_required == 0 && t->ping_policy.max_pings_without_data != 0) { /* need to receive something of substance before sending a ping again */ - if (grpc_http_trace.enabled() || grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace) || + GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "%s: Ping delayed [%p]: too many recent pings: %d/%d", t->is_client ? "CLIENT" : "SERVER", t->peer_string, t->ping_state.pings_before_data_required, @@ -82,7 +84,8 @@ static void maybe_initiate_ping(grpc_chttp2_transport* t) { if (next_allowed_ping > now) { /* not enough elapsed time between successive pings */ - if (grpc_http_trace.enabled() || grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace) || + GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "%s: Ping delayed [%p]: not enough time elapsed since last ping. " " Last ping %f: Next ping %f: Now %f", @@ -108,7 +111,8 @@ static void maybe_initiate_ping(grpc_chttp2_transport* t) { grpc_chttp2_ping_create(false, pq->inflight_id)); GRPC_STATS_INC_HTTP2_PINGS_SENT(); t->ping_state.last_ping_sent_time = now; - if (grpc_http_trace.enabled() || grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace) || + GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "%s: Ping sent [%s]: %d/%d", t->is_client ? "CLIENT" : "SERVER", t->peer_string, t->ping_state.pings_before_data_required, @@ -141,7 +145,7 @@ static bool update_list(grpc_chttp2_transport* t, grpc_chttp2_stream* s, static void report_stall(grpc_chttp2_transport* t, grpc_chttp2_stream* s, const char* staller) { - if (grpc_flowctl_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_flowctl_trace)) { gpr_log( GPR_DEBUG, "%s:%p stream %d moved to stalled list by %s. This is FULLY expected " diff --git a/src/core/ext/transport/inproc/inproc_transport.cc b/src/core/ext/transport/inproc/inproc_transport.cc index d93e0610e50..8da89851a69 100644 --- a/src/core/ext/transport/inproc/inproc_transport.cc +++ b/src/core/ext/transport/inproc/inproc_transport.cc @@ -35,9 +35,11 @@ #include "src/core/lib/transport/error_utils.h" #include "src/core/lib/transport/transport_impl.h" -#define INPROC_LOG(...) \ - do { \ - if (grpc_inproc_trace.enabled()) gpr_log(__VA_ARGS__); \ +#define INPROC_LOG(...) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_inproc_trace)) { \ + gpr_log(__VA_ARGS__); \ + } \ } while (0) namespace { @@ -296,7 +298,7 @@ grpc_error* fill_in_metadata(inproc_stream* s, const grpc_metadata_batch* metadata, uint32_t flags, grpc_metadata_batch* out_md, uint32_t* outflags, bool* markfilled) { - if (grpc_inproc_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_inproc_trace)) { log_metadata(metadata, s->t->is_client, outflags != nullptr); } @@ -907,7 +909,7 @@ void perform_stream_op(grpc_transport* gt, grpc_stream* gs, gpr_mu* mu = &s->t->mu->mu; // save aside in case s gets closed gpr_mu_lock(mu); - if (grpc_inproc_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_inproc_trace)) { if (op->send_initial_metadata) { log_metadata(op->payload->send_initial_metadata.send_initial_metadata, s->t->is_client, true); diff --git a/src/core/lib/channel/channel_stack.h b/src/core/lib/channel/channel_stack.h index f21dfb5d55e..a7c28d059f3 100644 --- a/src/core/lib/channel/channel_stack.h +++ b/src/core/lib/channel/channel_stack.h @@ -274,7 +274,11 @@ void grpc_call_log_op(const char* file, int line, gpr_log_severity severity, extern grpc_core::TraceFlag grpc_trace_channel; -#define GRPC_CALL_LOG_OP(sev, elem, op) \ - if (grpc_trace_channel.enabled()) grpc_call_log_op(sev, elem, op) +#define GRPC_CALL_LOG_OP(sev, elem, op) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_channel)) { \ + grpc_call_log_op(sev, elem, op); \ + } \ + } while (0) #endif /* GRPC_CORE_LIB_CHANNEL_CHANNEL_STACK_H */ diff --git a/src/core/lib/channel/handshaker.cc b/src/core/lib/channel/handshaker.cc index 6bb05cee24e..27ff25084cb 100644 --- a/src/core/lib/channel/handshaker.cc +++ b/src/core/lib/channel/handshaker.cc @@ -94,7 +94,7 @@ void HandshakeManager::ShutdownAllPending(grpc_error* why) { } void HandshakeManager::Add(RefCountedPtr handshaker) { - if (grpc_handshaker_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_handshaker_trace)) { gpr_log( GPR_INFO, "handshake_manager %p: adding handshaker %s [%p] at index %" PRIuPTR, @@ -125,7 +125,7 @@ void HandshakeManager::Shutdown(grpc_error* why) { // on_handshake_done callback. // Returns true if we've scheduled the on_handshake_done callback. bool HandshakeManager::CallNextHandshakerLocked(grpc_error* error) { - if (grpc_handshaker_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_handshaker_trace)) { char* args_str = HandshakerArgsString(&args_); gpr_log(GPR_INFO, "handshake_manager %p: error=%s shutdown=%d index=%" PRIuPTR @@ -159,7 +159,7 @@ bool HandshakeManager::CallNextHandshakerLocked(grpc_error* error) { args_.read_buffer = nullptr; } } - if (grpc_handshaker_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_handshaker_trace)) { gpr_log(GPR_INFO, "handshake_manager %p: handshaking complete -- scheduling " "on_handshake_done with error=%s", @@ -172,7 +172,7 @@ bool HandshakeManager::CallNextHandshakerLocked(grpc_error* error) { is_shutdown_ = true; } else { auto handshaker = handshakers_[index_]; - if (grpc_handshaker_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_handshaker_trace)) { gpr_log( GPR_INFO, "handshake_manager %p: calling handshaker %s [%p] at index %" PRIuPTR, diff --git a/src/core/lib/debug/trace.h b/src/core/lib/debug/trace.h index 6108fb239bd..72e1a4eded7 100644 --- a/src/core/lib/debug/trace.h +++ b/src/core/lib/debug/trace.h @@ -65,6 +65,8 @@ class TraceFlag { // wrapped language (wr don't want to force recompilation to get tracing). // Internally, however, for performance reasons, we compile them out by // default, since internal build systems make recompiling trivial. +// +// Prefer GRPC_TRACE_FLAG_ENABLED() macro instead of using enabled() directly. #define GRPC_USE_TRACERS // tracers on by default in OSS #if defined(GRPC_USE_TRACERS) || !defined(NDEBUG) bool enabled() { @@ -99,6 +101,8 @@ class TraceFlag { #endif }; +#define GRPC_TRACE_FLAG_ENABLED(f) GPR_UNLIKELY((f).enabled()) + #ifndef NDEBUG typedef TraceFlag DebugOnlyTraceFlag; #else diff --git a/src/core/lib/http/parser.cc b/src/core/lib/http/parser.cc index 7ca1cc9db5f..58608da982c 100644 --- a/src/core/lib/http/parser.cc +++ b/src/core/lib/http/parser.cc @@ -300,7 +300,7 @@ static grpc_error* addbyte(grpc_http_parser* parser, uint8_t byte, case GRPC_HTTP_FIRST_LINE: case GRPC_HTTP_HEADERS: if (parser->cur_line_length >= GRPC_HTTP_PARSER_MAX_HEADER_LENGTH) { - if (grpc_http1_trace.enabled()) + if (GRPC_TRACE_FLAG_ENABLED(grpc_http1_trace)) gpr_log(GPR_ERROR, "HTTP header max line length (%d) exceeded", GRPC_HTTP_PARSER_MAX_HEADER_LENGTH); return GRPC_ERROR_CREATE_FROM_STATIC_STRING( diff --git a/src/core/lib/iomgr/call_combiner.cc b/src/core/lib/iomgr/call_combiner.cc index a245ff04874..6a4e85de4df 100644 --- a/src/core/lib/iomgr/call_combiner.cc +++ b/src/core/lib/iomgr/call_combiner.cc @@ -113,7 +113,7 @@ void CallCombiner::ScheduleClosure(grpc_closure* closure, grpc_error* error) { void CallCombiner::Start(grpc_closure* closure, grpc_error* error, DEBUG_ARGS const char* reason) { GPR_TIMER_SCOPE("CallCombiner::Start", 0); - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "==> CallCombiner::Start() [%p] closure=%p [" DEBUG_FMT_STR "%s] error=%s", @@ -121,7 +121,7 @@ void CallCombiner::Start(grpc_closure* closure, grpc_error* error, } size_t prev_size = static_cast(gpr_atm_full_fetch_add(&size_, (gpr_atm)1)); - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " size: %" PRIdPTR " -> %" PRIdPTR, prev_size, prev_size + 1); } @@ -129,13 +129,13 @@ void CallCombiner::Start(grpc_closure* closure, grpc_error* error, if (prev_size == 0) { GRPC_STATS_INC_CALL_COMBINER_LOCKS_INITIATED(); GPR_TIMER_MARK("call_combiner_initiate", 0); - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " EXECUTING IMMEDIATELY"); } // Queue was empty, so execute this closure immediately. ScheduleClosure(closure, error); } else { - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " QUEUING"); } // Queue was not empty, so add closure to queue. @@ -146,20 +146,20 @@ void CallCombiner::Start(grpc_closure* closure, grpc_error* error, void CallCombiner::Stop(DEBUG_ARGS const char* reason) { GPR_TIMER_SCOPE("CallCombiner::Stop", 0); - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "==> CallCombiner::Stop() [%p] [" DEBUG_FMT_STR "%s]", this DEBUG_FMT_ARGS, reason); } size_t prev_size = static_cast(gpr_atm_full_fetch_add(&size_, (gpr_atm)-1)); - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " size: %" PRIdPTR " -> %" PRIdPTR, prev_size, prev_size - 1); } GPR_ASSERT(prev_size >= 1); if (prev_size > 1) { while (true) { - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " checking queue"); } bool empty; @@ -168,19 +168,19 @@ void CallCombiner::Stop(DEBUG_ARGS const char* reason) { if (closure == nullptr) { // This can happen either due to a race condition within the mpscq // code or because of a race with Start(). - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " queue returned no result; checking again"); } continue; } - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " EXECUTING FROM QUEUE: closure=%p error=%s", closure, grpc_error_string(closure->error_data.error)); } ScheduleClosure(closure, closure->error_data.error); break; } - } else if (grpc_call_combiner_trace.enabled()) { + } else if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, " queue empty"); } } @@ -194,7 +194,7 @@ void CallCombiner::SetNotifyOnCancel(grpc_closure* closure) { // If error is set, invoke the cancellation closure immediately. // Otherwise, store the new closure. if (original_error != GRPC_ERROR_NONE) { - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "call_combiner=%p: scheduling notify_on_cancel callback=%p " "for pre-existing cancellation", @@ -204,7 +204,7 @@ void CallCombiner::SetNotifyOnCancel(grpc_closure* closure) { break; } else { if (gpr_atm_full_cas(&cancel_state_, original_state, (gpr_atm)closure)) { - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "call_combiner=%p: setting notify_on_cancel=%p", this, closure); } @@ -213,7 +213,7 @@ void CallCombiner::SetNotifyOnCancel(grpc_closure* closure) { // up any resources they may be holding for the callback. if (original_state != 0) { closure = (grpc_closure*)original_state; - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "call_combiner=%p: scheduling old cancel callback=%p", this, closure); @@ -240,7 +240,7 @@ void CallCombiner::Cancel(grpc_error* error) { EncodeCancelStateError(error))) { if (original_state != 0) { grpc_closure* notify_on_cancel = (grpc_closure*)original_state; - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "call_combiner=%p: scheduling notify_on_cancel callback=%p", this, notify_on_cancel); diff --git a/src/core/lib/iomgr/call_combiner.h b/src/core/lib/iomgr/call_combiner.h index ffcd0f7fb76..a10b437c15a 100644 --- a/src/core/lib/iomgr/call_combiner.h +++ b/src/core/lib/iomgr/call_combiner.h @@ -169,7 +169,7 @@ class CallCombinerClosureList { GRPC_CALL_COMBINER_START(call_combiner, closure.closure, closure.error, closure.reason); } - if (grpc_call_combiner_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_combiner_trace)) { gpr_log(GPR_INFO, "CallCombinerClosureList executing closure while already " "holding call_combiner %p: closure=%p error=%s reason=%s", diff --git a/src/core/lib/iomgr/ev_epoll1_linux.cc b/src/core/lib/iomgr/ev_epoll1_linux.cc index c2165341964..09e0834818b 100644 --- a/src/core/lib/iomgr/ev_epoll1_linux.cc +++ b/src/core/lib/iomgr/ev_epoll1_linux.cc @@ -351,7 +351,7 @@ static grpc_fd* fd_create(int fd, const char* name, bool track_err) { grpc_iomgr_register_object(&new_fd->iomgr_object, fd_name); fork_fd_list_add_grpc_fd(new_fd); #ifndef NDEBUG - if (grpc_trace_fd_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_fd_refcount)) { gpr_log(GPR_DEBUG, "FD %d %p create %s", fd, new_fd, fd_name); } #endif @@ -724,7 +724,7 @@ static grpc_error* do_epoll_wait(grpc_pollset* ps, grpc_millis deadline) { GRPC_STATS_INC_POLL_EVENTS_RETURNED(r); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "ps: %p poll got %d events", ps, r); } @@ -744,7 +744,7 @@ static bool begin_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, worker->schedule_on_end_work = (grpc_closure_list)GRPC_CLOSURE_LIST_INIT; pollset->begin_refs++; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p BEGIN_STARTS:%p", pollset, worker); } @@ -763,7 +763,7 @@ static bool begin_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, retry_lock_neighborhood: gpr_mu_lock(&neighborhood->mu); gpr_mu_lock(&pollset->mu); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p BEGIN_REORG:%p kick_state=%s is_reassigning=%d", pollset, worker, kick_state_string(worker->state), is_reassigning); @@ -815,7 +815,7 @@ static bool begin_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, worker->initialized_cv = true; gpr_cv_init(&worker->cv); while (worker->state == UNKICKED && !pollset->shutting_down) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p BEGIN_WAIT:%p kick_state=%s shutdown=%d", pollset, worker, kick_state_string(worker->state), pollset->shutting_down); @@ -832,7 +832,7 @@ static bool begin_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, grpc_core::ExecCtx::Get()->InvalidateNow(); } - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p BEGIN_DONE:%p kick_state=%s shutdown=%d " "kicked_without_poller: %d", @@ -875,7 +875,7 @@ static bool check_neighborhood_for_available_poller( case UNKICKED: if (gpr_atm_no_barrier_cas(&g_active_poller, 0, (gpr_atm)inspect_worker)) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. choose next poller to be %p", inspect_worker); } @@ -886,7 +886,7 @@ static bool check_neighborhood_for_available_poller( gpr_cv_signal(&inspect_worker->cv); } } else { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. beaten to choose next poller"); } } @@ -904,7 +904,7 @@ static bool check_neighborhood_for_available_poller( } while (!found_worker && inspect_worker != inspect->root_worker); } if (!found_worker) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. mark pollset %p inactive", inspect); } inspect->seen_inactive = true; @@ -924,7 +924,7 @@ static bool check_neighborhood_for_available_poller( static void end_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, grpc_pollset_worker** worker_hdl) { GPR_TIMER_SCOPE("end_worker", 0); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p END_WORKER:%p", pollset, worker); } if (worker_hdl != nullptr) *worker_hdl = nullptr; @@ -934,7 +934,7 @@ static void end_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, grpc_core::ExecCtx::Get()->closure_list()); if (gpr_atm_no_barrier_load(&g_active_poller) == (gpr_atm)worker) { if (worker->next != worker && worker->next->state == UNKICKED) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. choose next poller to be peer %p", worker); } GPR_ASSERT(worker->next->initialized_cv); @@ -986,7 +986,7 @@ static void end_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, if (worker->initialized_cv) { gpr_cv_destroy(&worker->cv); } - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. remove worker"); } if (EMPTIED == worker_remove(pollset, worker)) { @@ -1055,7 +1055,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, GPR_TIMER_SCOPE("pollset_kick", 0); GRPC_STATS_INC_POLLSET_KICK(); grpc_error* ret_err = GRPC_ERROR_NONE; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_strvec log; gpr_strvec_init(&log); char* tmp; @@ -1088,7 +1088,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, if (root_worker == nullptr) { GRPC_STATS_INC_POLLSET_KICKED_WITHOUT_POLLER(); pollset->kicked_without_poller = true; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kicked_without_poller"); } goto done; @@ -1096,14 +1096,14 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, grpc_pollset_worker* next_worker = root_worker->next; if (root_worker->state == KICKED) { GRPC_STATS_INC_POLLSET_KICKED_AGAIN(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. already kicked %p", root_worker); } SET_KICK_STATE(root_worker, KICKED); goto done; } else if (next_worker->state == KICKED) { GRPC_STATS_INC_POLLSET_KICKED_AGAIN(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. already kicked %p", next_worker); } SET_KICK_STATE(next_worker, KICKED); @@ -1114,7 +1114,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, root_worker == (grpc_pollset_worker*)gpr_atm_no_barrier_load( &g_active_poller)) { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_FD(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kicked %p", root_worker); } SET_KICK_STATE(root_worker, KICKED); @@ -1122,7 +1122,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, goto done; } else if (next_worker->state == UNKICKED) { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_CV(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kicked %p", next_worker); } GPR_ASSERT(next_worker->initialized_cv); @@ -1131,7 +1131,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, goto done; } else if (next_worker->state == DESIGNATED_POLLER) { if (root_worker->state != DESIGNATED_POLLER) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log( GPR_INFO, " .. kicked root non-poller %p (initialized_cv=%d) (poller=%p)", @@ -1145,7 +1145,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, goto done; } else { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_FD(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. non-root poller %p (root=%p)", next_worker, root_worker); } @@ -1161,7 +1161,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, } } else { GRPC_STATS_INC_POLLSET_KICK_OWN_THREAD(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kicked while waking up"); } goto done; @@ -1171,14 +1171,14 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, } if (specific_worker->state == KICKED) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. specific worker already kicked"); } goto done; } else if (gpr_tls_get(&g_current_thread_worker) == (intptr_t)specific_worker) { GRPC_STATS_INC_POLLSET_KICK_OWN_THREAD(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. mark %p kicked", specific_worker); } SET_KICK_STATE(specific_worker, KICKED); @@ -1186,7 +1186,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, } else if (specific_worker == (grpc_pollset_worker*)gpr_atm_no_barrier_load(&g_active_poller)) { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_FD(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kick active poller"); } SET_KICK_STATE(specific_worker, KICKED); @@ -1194,7 +1194,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, goto done; } else if (specific_worker->initialized_cv) { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_CV(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kick waiting worker"); } SET_KICK_STATE(specific_worker, KICKED); @@ -1202,7 +1202,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, goto done; } else { GRPC_STATS_INC_POLLSET_KICKED_AGAIN(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, " .. kick non-waiting worker"); } SET_KICK_STATE(specific_worker, KICKED); diff --git a/src/core/lib/iomgr/ev_epollex_linux.cc b/src/core/lib/iomgr/ev_epollex_linux.cc index c387f8359a0..08116b3ab53 100644 --- a/src/core/lib/iomgr/ev_epollex_linux.cc +++ b/src/core/lib/iomgr/ev_epollex_linux.cc @@ -164,7 +164,7 @@ struct grpc_fd { gpr_asprintf(&fd_name, "%s fd=%d", name, fd); grpc_iomgr_register_object(&iomgr_object, fd_name); #ifndef NDEBUG - if (grpc_trace_fd_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_fd_refcount)) { gpr_log(GPR_DEBUG, "FD %d %p create %s", fd, this, fd_name); } #endif @@ -335,7 +335,7 @@ static gpr_mu fd_freelist_mu; #define UNREF_BY(fd, n, reason) unref_by(fd, n, reason, __FILE__, __LINE__) static void ref_by(grpc_fd* fd, int n, const char* reason, const char* file, int line) { - if (grpc_trace_fd_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_fd_refcount)) { gpr_log(GPR_DEBUG, "FD %d %p ref %d %" PRIdPTR " -> %" PRIdPTR " [%s; %s:%d]", fd->fd, fd, n, gpr_atm_no_barrier_load(&fd->refst), @@ -364,7 +364,7 @@ static void fd_destroy(void* arg, grpc_error* error) { #ifndef NDEBUG static void unref_by(grpc_fd* fd, int n, const char* reason, const char* file, int line) { - if (grpc_trace_fd_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_fd_refcount)) { gpr_log(GPR_DEBUG, "FD %d %p unref %d %" PRIdPTR " -> %" PRIdPTR " [%s; %s:%d]", fd->fd, fd, n, gpr_atm_no_barrier_load(&fd->refst), @@ -586,7 +586,7 @@ static grpc_error* pollable_create(pollable_type type, pollable** p) { static pollable* pollable_ref(pollable* p) { #else static pollable* pollable_ref(pollable* p, int line, const char* reason) { - if (grpc_trace_pollable_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_pollable_refcount)) { int r = static_cast gpr_atm_no_barrier_load(&p->refs.count); gpr_log(__FILE__, line, GPR_LOG_SEVERITY_DEBUG, "POLLABLE:%p ref %d->%d %s", p, r, r + 1, reason); @@ -601,7 +601,7 @@ static void pollable_unref(pollable* p) { #else static void pollable_unref(pollable* p, int line, const char* reason) { if (p == nullptr) return; - if (grpc_trace_pollable_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_pollable_refcount)) { int r = static_cast gpr_atm_no_barrier_load(&p->refs.count); gpr_log(__FILE__, line, GPR_LOG_SEVERITY_DEBUG, "POLLABLE:%p unref %d->%d %s", p, r, r - 1, reason); @@ -621,7 +621,7 @@ static grpc_error* pollable_add_fd(pollable* p, grpc_fd* fd) { grpc_error* error = GRPC_ERROR_NONE; static const char* err_desc = "pollable_add_fd"; const int epfd = p->epfd; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "add fd %p (%d) to pollable %p", fd, fd->fd, p); } @@ -669,7 +669,7 @@ static void pollset_global_shutdown(void) { /* pollset->mu must be held while calling this function */ static void pollset_maybe_finish_shutdown(grpc_pollset* pollset) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p (pollable:%p) maybe_finish_shutdown sc=%p (target:!NULL) " "rw=%p (target:NULL) cpsc=%d (target:0)", @@ -694,14 +694,14 @@ static grpc_error* kick_one_worker(grpc_pollset_worker* specific_worker) { grpc_core::MutexLock lock(&p->mu); GPR_ASSERT(specific_worker != nullptr); if (specific_worker->kicked) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kicked_specific_but_already_kicked", p); } GRPC_STATS_INC_POLLSET_KICKED_AGAIN(); return GRPC_ERROR_NONE; } if (gpr_tls_get(&g_current_thread_worker) == (intptr_t)specific_worker) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kicked_specific_but_awake", p); } GRPC_STATS_INC_POLLSET_KICK_OWN_THREAD(); @@ -710,7 +710,7 @@ static grpc_error* kick_one_worker(grpc_pollset_worker* specific_worker) { } if (specific_worker == p->root_worker) { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_FD(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kicked_specific_via_wakeup_fd", p); } specific_worker->kicked = true; @@ -719,7 +719,7 @@ static grpc_error* kick_one_worker(grpc_pollset_worker* specific_worker) { } if (specific_worker->initialized_cv) { GRPC_STATS_INC_POLLSET_KICK_WAKEUP_CV(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kicked_specific_via_cv", p); } specific_worker->kicked = true; @@ -735,7 +735,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, grpc_pollset_worker* specific_worker) { GPR_TIMER_SCOPE("pollset_kick", 0); GRPC_STATS_INC_POLLSET_KICK(); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kick %p tls_pollset=%p tls_worker=%p pollset.root_worker=%p", pollset, specific_worker, @@ -745,7 +745,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, if (specific_worker == nullptr) { if (gpr_tls_get(&g_current_thread_pollset) != (intptr_t)pollset) { if (pollset->root_worker == nullptr) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kicked_any_without_poller", pollset); } GRPC_STATS_INC_POLLSET_KICKED_WITHOUT_POLLER(); @@ -771,7 +771,7 @@ static grpc_error* pollset_kick(grpc_pollset* pollset, pollset->root_worker->links[PWLINK_POLLSET].next); } } else { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p kicked_any_but_awake", pollset); } GRPC_STATS_INC_POLLSET_KICK_OWN_THREAD(); @@ -891,7 +891,7 @@ static grpc_error* pollable_process_events(grpc_pollset* pollset, struct epoll_event* ev = &pollable_obj->events[n]; void* data_ptr = ev->data.ptr; if (1 & (intptr_t)data_ptr) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p got pollset_wakeup %p", pollset, data_ptr); } append_error(&error, @@ -909,7 +909,7 @@ static grpc_error* pollable_process_events(grpc_pollset* pollset, bool write_ev = (ev->events & EPOLLOUT) != 0; bool err_fallback = error && !track_err; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p got fd %p: cancel=%d read=%d " "write=%d", @@ -941,7 +941,7 @@ static grpc_error* pollable_epoll(pollable* p, grpc_millis deadline) { GPR_TIMER_SCOPE("pollable_epoll", 0); int timeout = poll_deadline_to_millis_timeout(deadline); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { char* desc = pollable_desc(p); gpr_log(GPR_INFO, "POLLABLE:%p[%s] poll for %dms", p, desc, timeout); gpr_free(desc); @@ -961,7 +961,7 @@ static grpc_error* pollable_epoll(pollable* p, grpc_millis deadline) { if (r < 0) return GRPC_OS_ERROR(errno, "epoll_wait"); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "POLLABLE:%p got %d events", p, r); } @@ -1031,7 +1031,7 @@ static bool begin_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, worker->initialized_cv = true; gpr_cv_init(&worker->cv); gpr_mu_unlock(&pollset->mu); - if (grpc_polling_trace.enabled() && + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace) && worker->pollable_obj->root_worker != worker) { gpr_log(GPR_INFO, "PS:%p wait %p w=%p for %dms", pollset, worker->pollable_obj, worker, @@ -1040,18 +1040,18 @@ static bool begin_worker(grpc_pollset* pollset, grpc_pollset_worker* worker, while (do_poll && worker->pollable_obj->root_worker != worker) { if (gpr_cv_wait(&worker->cv, &worker->pollable_obj->mu, grpc_millis_to_timespec(deadline, GPR_CLOCK_REALTIME))) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p timeout_wait %p w=%p", pollset, worker->pollable_obj, worker); } do_poll = false; } else if (worker->kicked) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p wakeup %p w=%p", pollset, worker->pollable_obj, worker); } do_poll = false; - } else if (grpc_polling_trace.enabled() && + } else if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace) && worker->pollable_obj->root_worker != worker) { gpr_log(GPR_INFO, "PS:%p spurious_wakeup %p w=%p", pollset, worker->pollable_obj, worker); @@ -1124,7 +1124,7 @@ static grpc_error* pollset_work(grpc_pollset* pollset, #ifndef NDEBUG WORKER_PTR->originator = gettid(); #endif - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p work hdl=%p worker=%p now=%" PRId64 " deadline=%" PRId64 " kwp=%d pollable=%p", @@ -1165,7 +1165,7 @@ static grpc_error* pollset_transition_pollable_from_empty_to_fd_locked( grpc_pollset* pollset, grpc_fd* fd) { static const char* err_desc = "pollset_transition_pollable_from_empty_to_fd"; grpc_error* error = GRPC_ERROR_NONE; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p add fd %p (%d); transition pollable from empty to fd", pollset, fd, fd->fd); @@ -1181,7 +1181,7 @@ static grpc_error* pollset_transition_pollable_from_fd_to_multi_locked( grpc_pollset* pollset, grpc_fd* and_add_fd) { static const char* err_desc = "pollset_transition_pollable_from_fd_to_multi"; grpc_error* error = GRPC_ERROR_NONE; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log( GPR_INFO, "PS:%p add fd %p (%d); transition pollable from fd %p to multipoller", @@ -1253,7 +1253,7 @@ static grpc_error* pollset_as_multipollable_locked(grpc_pollset* pollset, error = pollable_create(PO_MULTI, &pollset->active_pollable); /* Any workers currently polling on this pollset must now be woked up so * that they can pick up the new active_pollable */ - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PS:%p active pollable transition from empty to multi", pollset); @@ -1357,7 +1357,7 @@ static void pollset_set_unref(grpc_pollset_set* pss) { static void pollset_set_add_fd(grpc_pollset_set* pss, grpc_fd* fd) { GPR_TIMER_SCOPE("pollset_set_add_fd", 0); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PSS:%p: add fd %p (%d)", pss, fd, fd->fd); } grpc_error* error = GRPC_ERROR_NONE; @@ -1381,7 +1381,7 @@ static void pollset_set_add_fd(grpc_pollset_set* pss, grpc_fd* fd) { static void pollset_set_del_fd(grpc_pollset_set* pss, grpc_fd* fd) { GPR_TIMER_SCOPE("pollset_set_del_fd", 0); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PSS:%p: del fd %p", pss, fd); } pss = pss_lock_adam(pss); @@ -1402,7 +1402,7 @@ static void pollset_set_del_fd(grpc_pollset_set* pss, grpc_fd* fd) { static void pollset_set_del_pollset(grpc_pollset_set* pss, grpc_pollset* ps) { GPR_TIMER_SCOPE("pollset_set_del_pollset", 0); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PSS:%p: del pollset %p", pss, ps); } pss = pss_lock_adam(pss); @@ -1454,7 +1454,7 @@ static grpc_error* add_fds_to_pollsets(grpc_fd** fds, size_t fd_count, static void pollset_set_add_pollset(grpc_pollset_set* pss, grpc_pollset* ps) { GPR_TIMER_SCOPE("pollset_set_add_pollset", 0); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PSS:%p: add pollset %p", pss, ps); } grpc_error* error = GRPC_ERROR_NONE; @@ -1491,7 +1491,7 @@ static void pollset_set_add_pollset(grpc_pollset_set* pss, grpc_pollset* ps) { static void pollset_set_add_pollset_set(grpc_pollset_set* a, grpc_pollset_set* b) { GPR_TIMER_SCOPE("pollset_set_add_pollset_set", 0); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PSS: merge (%p, %p)", a, b); } grpc_error* error = GRPC_ERROR_NONE; @@ -1525,7 +1525,7 @@ static void pollset_set_add_pollset_set(grpc_pollset_set* a, if (b_size > a_size) { GPR_SWAP(grpc_pollset_set*, a, b); } - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "PSS: parent %p to %p", b, a); } gpr_ref(&a->refs); diff --git a/src/core/lib/iomgr/ev_poll_posix.cc b/src/core/lib/iomgr/ev_poll_posix.cc index 0c95cb75c6d..cee6dcbe530 100644 --- a/src/core/lib/iomgr/ev_poll_posix.cc +++ b/src/core/lib/iomgr/ev_poll_posix.cc @@ -316,7 +316,7 @@ static void fork_fd_list_add_wakeup_fd(grpc_cached_wakeup_fd* fd) { #define UNREF_BY(fd, n, reason) unref_by(fd, n, reason, __FILE__, __LINE__) static void ref_by(grpc_fd* fd, int n, const char* reason, const char* file, int line) { - if (grpc_trace_fd_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_fd_refcount)) { gpr_log(GPR_DEBUG, "FD %d %p ref %d %" PRIdPTR " -> %" PRIdPTR " [%s; %s:%d]", fd->fd, fd, n, gpr_atm_no_barrier_load(&fd->refst), @@ -333,7 +333,7 @@ static void ref_by(grpc_fd* fd, int n) { #ifndef NDEBUG static void unref_by(grpc_fd* fd, int n, const char* reason, const char* file, int line) { - if (grpc_trace_fd_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_fd_refcount)) { gpr_log(GPR_DEBUG, "FD %d %p unref %d %" PRIdPTR " -> %" PRIdPTR " [%s; %s:%d]", fd->fd, fd, n, gpr_atm_no_barrier_load(&fd->refst), @@ -561,7 +561,7 @@ static void fd_notify_on_write(grpc_fd* fd, grpc_closure* closure) { } static void fd_notify_on_error(grpc_fd* fd, grpc_closure* closure) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_ERROR, "Polling engine does not support tracking errors."); } GRPC_CLOSURE_SCHED(closure, GRPC_ERROR_CANCELLED); @@ -580,7 +580,7 @@ static void fd_set_writable(grpc_fd* fd) { } static void fd_set_error(grpc_fd* fd) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_ERROR, "Polling engine does not support tracking errors."); } } @@ -1012,7 +1012,7 @@ static grpc_error* pollset_work(grpc_pollset* pollset, r = grpc_poll_function(pfds, pfd_count, timeout); GRPC_SCHEDULING_END_BLOCKING_REGION; - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "%p poll=%d", pollset, r); } @@ -1036,7 +1036,7 @@ static grpc_error* pollset_work(grpc_pollset* pollset, } } else { if (pfds[0].revents & POLLIN_CHECK) { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "%p: got_wakeup", pollset); } work_combine_error( @@ -1046,7 +1046,7 @@ static grpc_error* pollset_work(grpc_pollset* pollset, if (watchers[i].fd == nullptr) { fd_end_poll(&watchers[i], 0, 0); } else { - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_INFO, "%p got_event: %d r:%d w:%d [%d]", pollset, pfds[i].fd, (pfds[i].revents & POLLIN_CHECK) != 0, (pfds[i].revents & POLLOUT_CHECK) != 0, pfds[i].revents); diff --git a/src/core/lib/iomgr/ev_posix.cc b/src/core/lib/iomgr/ev_posix.cc index 898686b06c3..47cf5b83b17 100644 --- a/src/core/lib/iomgr/ev_posix.cc +++ b/src/core/lib/iomgr/ev_posix.cc @@ -50,7 +50,7 @@ grpc_core::DebugOnlyTraceFlag grpc_polling_api_trace(false, "polling_api"); // Polling API trace only enabled in debug builds #define GRPC_POLLING_API_TRACE(format, ...) \ - if (grpc_polling_api_trace.enabled()) { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_api_trace)) { \ gpr_log(GPR_INFO, "(polling-api) " format, __VA_ARGS__); \ } #else diff --git a/src/core/lib/iomgr/ev_posix.h b/src/core/lib/iomgr/ev_posix.h index 699173fe255..0ca3a6f82fd 100644 --- a/src/core/lib/iomgr/ev_posix.h +++ b/src/core/lib/iomgr/ev_posix.h @@ -33,7 +33,7 @@ extern grpc_core::TraceFlag grpc_fd_trace; /* Disabled by default */ extern grpc_core::TraceFlag grpc_polling_trace; /* Disabled by default */ #define GRPC_FD_TRACE(format, ...) \ - if (grpc_fd_trace.enabled()) { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_fd_trace)) { \ gpr_log(GPR_INFO, "(fd-trace) " format, __VA_ARGS__); \ } diff --git a/src/core/lib/iomgr/executor.cc b/src/core/lib/iomgr/executor.cc index 47836acacc0..9b967a4fe7a 100644 --- a/src/core/lib/iomgr/executor.cc +++ b/src/core/lib/iomgr/executor.cc @@ -37,13 +37,13 @@ #define MAX_DEPTH 2 #define EXECUTOR_TRACE(format, ...) \ - if (executor_trace.enabled()) { \ + if (GRPC_TRACE_FLAG_ENABLED(executor_trace)) { \ gpr_log(GPR_INFO, "EXECUTOR " format, __VA_ARGS__); \ } -#define EXECUTOR_TRACE0(str) \ - if (executor_trace.enabled()) { \ - gpr_log(GPR_INFO, "EXECUTOR " str); \ +#define EXECUTOR_TRACE0(str) \ + if (GRPC_TRACE_FLAG_ENABLED(executor_trace)) { \ + gpr_log(GPR_INFO, "EXECUTOR " str); \ } namespace grpc_core { diff --git a/src/core/lib/iomgr/lockfree_event.cc b/src/core/lib/iomgr/lockfree_event.cc index 085fea40a40..f0c40b4827d 100644 --- a/src/core/lib/iomgr/lockfree_event.cc +++ b/src/core/lib/iomgr/lockfree_event.cc @@ -94,7 +94,7 @@ void LockfreeEvent::NotifyOn(grpc_closure* closure) { * sure that the shutdown error has been initialized properly before us * referencing it. */ gpr_atm curr = gpr_atm_acq_load(&state_); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_ERROR, "LockfreeEvent::NotifyOn: %p curr=%p closure=%p", this, (void*)curr, closure); } @@ -160,7 +160,7 @@ bool LockfreeEvent::SetShutdown(grpc_error* shutdown_err) { while (true) { gpr_atm curr = gpr_atm_no_barrier_load(&state_); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_ERROR, "LockfreeEvent::SetShutdown: %p curr=%p err=%s", &state_, (void*)curr, grpc_error_string(shutdown_err)); } @@ -209,7 +209,7 @@ void LockfreeEvent::SetReady() { while (true) { gpr_atm curr = gpr_atm_no_barrier_load(&state_); - if (grpc_polling_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_polling_trace)) { gpr_log(GPR_ERROR, "LockfreeEvent::SetReady: %p curr=%p", &state_, (void*)curr); } diff --git a/src/core/lib/iomgr/resource_quota.cc b/src/core/lib/iomgr/resource_quota.cc index 06e19d71e0c..dffac348c52 100644 --- a/src/core/lib/iomgr/resource_quota.cc +++ b/src/core/lib/iomgr/resource_quota.cc @@ -317,7 +317,7 @@ static bool rq_alloc(grpc_resource_quota* resource_quota) { while ((resource_user = rulist_pop_head(resource_quota, GRPC_RULIST_AWAITING_ALLOCATION))) { gpr_mu_lock(&resource_user->mu); - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ: check allocation for user %p shutdown=%" PRIdPTR " free_pool=%" PRId64, @@ -343,14 +343,14 @@ static bool rq_alloc(grpc_resource_quota* resource_quota) { resource_user->free_pool = 0; resource_quota->free_pool -= amt; rq_update_estimate(resource_quota); - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: grant alloc %" PRId64 " bytes; rq_free_pool -> %" PRId64, resource_quota->name, resource_user->name, amt, resource_quota->free_pool); } - } else if (grpc_resource_quota_trace.enabled() && + } else if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace) && resource_user->free_pool >= 0) { gpr_log(GPR_INFO, "RQ %s %s: discard already satisfied alloc request", resource_quota->name, resource_user->name); @@ -382,7 +382,7 @@ static bool rq_reclaim_from_per_user_free_pool( resource_user->free_pool = 0; resource_quota->free_pool += amt; rq_update_estimate(resource_quota); - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: reclaim_from_per_user_free_pool %" PRId64 " bytes; rq_free_pool -> %" PRId64, @@ -392,7 +392,7 @@ static bool rq_reclaim_from_per_user_free_pool( gpr_mu_unlock(&resource_user->mu); return true; } else { - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: failed to reclaim_from_per_user_free_pool; " "free_pool = %" PRId64 "; rq_free_pool = %" PRId64, @@ -412,7 +412,7 @@ static bool rq_reclaim(grpc_resource_quota* resource_quota, bool destructive) { : GRPC_RULIST_RECLAIMER_BENIGN; grpc_resource_user* resource_user = rulist_pop_head(resource_quota, list); if (resource_user == nullptr) return false; - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: initiate %s reclamation", resource_quota->name, resource_user->name, destructive ? "destructive" : "benign"); } @@ -543,7 +543,7 @@ static void ru_post_destructive_reclaimer(void* ru, grpc_error* error) { } static void ru_shutdown(void* ru, grpc_error* error) { - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RU shutdown %p", ru); } grpc_resource_user* resource_user = static_cast(ru); @@ -885,7 +885,7 @@ static void resource_user_alloc_locked(grpc_resource_user* resource_user, grpc_closure* optional_on_done) { ru_ref_by(resource_user, static_cast(size)); resource_user->free_pool -= static_cast(size); - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: alloc %" PRIdPTR "; free_pool -> %" PRId64, resource_user->resource_quota->name, resource_user->name, size, resource_user->free_pool); @@ -944,7 +944,7 @@ void grpc_resource_user_free(grpc_resource_user* resource_user, size_t size) { GPR_ASSERT(prior >= static_cast(size)); bool was_zero_or_negative = resource_user->free_pool <= 0; resource_user->free_pool += static_cast(size); - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: free %" PRIdPTR "; free_pool -> %" PRId64, resource_user->resource_quota->name, resource_user->name, size, resource_user->free_pool); @@ -970,7 +970,7 @@ void grpc_resource_user_post_reclaimer(grpc_resource_user* resource_user, } void grpc_resource_user_finish_reclamation(grpc_resource_user* resource_user) { - if (grpc_resource_quota_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_resource_quota_trace)) { gpr_log(GPR_INFO, "RQ %s %s: reclamation complete", resource_user->resource_quota->name, resource_user->name); } diff --git a/src/core/lib/iomgr/socket_utils_common_posix.cc b/src/core/lib/iomgr/socket_utils_common_posix.cc index ea0adb1f6a6..2101651b33f 100644 --- a/src/core/lib/iomgr/socket_utils_common_posix.cc +++ b/src/core/lib/iomgr/socket_utils_common_posix.cc @@ -292,7 +292,7 @@ grpc_error* grpc_set_socket_tcp_user_timeout( } if (enable) { extern grpc_core::TraceFlag grpc_tcp_trace; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "Enabling TCP_USER_TIMEOUT with a timeout of %d ms", timeout); } @@ -315,7 +315,7 @@ grpc_error* grpc_set_socket_tcp_user_timeout( } #else extern grpc_core::TraceFlag grpc_tcp_trace; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP_USER_TIMEOUT not supported for this platform"); } #endif /* GRPC_HAVE_TCP_USER_TIMEOUT */ diff --git a/src/core/lib/iomgr/tcp_client_custom.cc b/src/core/lib/iomgr/tcp_client_custom.cc index 73344b18d8b..14a8b78dc68 100644 --- a/src/core/lib/iomgr/tcp_client_custom.cc +++ b/src/core/lib/iomgr/tcp_client_custom.cc @@ -64,7 +64,7 @@ static void on_alarm(void* acp, grpc_error* error) { int done; grpc_custom_socket* socket = (grpc_custom_socket*)acp; grpc_custom_tcp_connect* connect = socket->connector; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "CLIENT_CONNECT: %s: on_alarm: error=%s", connect->addr_name, str); @@ -146,7 +146,7 @@ static void tcp_connect(grpc_closure* closure, grpc_endpoint** ep, socket->listener = nullptr; connect->refs = 2; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "CLIENT_CONNECT: %p %s: asynchronously connecting", socket, connect->addr_name); } diff --git a/src/core/lib/iomgr/tcp_client_posix.cc b/src/core/lib/iomgr/tcp_client_posix.cc index 0bff74e88b3..ad9d7798d05 100644 --- a/src/core/lib/iomgr/tcp_client_posix.cc +++ b/src/core/lib/iomgr/tcp_client_posix.cc @@ -108,7 +108,7 @@ done: static void tc_on_alarm(void* acp, grpc_error* error) { int done; async_connect* ac = static_cast(acp); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "CLIENT_CONNECT: %s: on_alarm: error=%s", ac->addr_str, str); @@ -145,7 +145,7 @@ static void on_writable(void* acp, grpc_error* error) { GRPC_ERROR_REF(error); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "CLIENT_CONNECT: %s: on_writable: error=%s", ac->addr_str, str); @@ -328,7 +328,7 @@ void grpc_tcp_client_create_from_prepared_fd( grpc_schedule_on_exec_ctx); ac->channel_args = grpc_channel_args_copy(channel_args); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "CLIENT_CONNECT: %s: asynchronously connecting fd %p", ac->addr_str, fdobj); } diff --git a/src/core/lib/iomgr/tcp_custom.cc b/src/core/lib/iomgr/tcp_custom.cc index f7ad120b026..66df5082e98 100644 --- a/src/core/lib/iomgr/tcp_custom.cc +++ b/src/core/lib/iomgr/tcp_custom.cc @@ -88,7 +88,7 @@ static void tcp_free(grpc_custom_socket* s) { #define TCP_REF(tcp, reason) tcp_ref((tcp), (reason), __FILE__, __LINE__) static void tcp_unref(custom_tcp_endpoint* tcp, const char* reason, const char* file, int line) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_atm val = gpr_atm_no_barrier_load(&tcp->refcount.count); gpr_log(file, line, GPR_LOG_SEVERITY_ERROR, "TCP unref %p : %s %" PRIdPTR " -> %" PRIdPTR, tcp->socket, reason, @@ -101,7 +101,7 @@ static void tcp_unref(custom_tcp_endpoint* tcp, const char* reason, static void tcp_ref(custom_tcp_endpoint* tcp, const char* reason, const char* file, int line) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_atm val = gpr_atm_no_barrier_load(&tcp->refcount.count); gpr_log(file, line, GPR_LOG_SEVERITY_ERROR, "TCP ref %p : %s %" PRIdPTR " -> %" PRIdPTR, tcp->socket, reason, @@ -123,7 +123,7 @@ static void tcp_ref(custom_tcp_endpoint* tcp) { gpr_ref(&tcp->refcount); } static void call_read_cb(custom_tcp_endpoint* tcp, grpc_error* error) { grpc_closure* cb = tcp->read_cb; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p call_cb %p %p:%p", tcp->socket, cb, cb->cb, cb->cb_arg); size_t i; @@ -169,7 +169,7 @@ static void custom_read_callback(grpc_custom_socket* socket, size_t nread, static void tcp_read_allocation_done(void* tcpp, grpc_error* error) { custom_tcp_endpoint* tcp = (custom_tcp_endpoint*)tcpp; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p read_allocation_done: %s", tcp->socket, grpc_error_string(error)); } @@ -185,7 +185,7 @@ static void tcp_read_allocation_done(void* tcpp, grpc_error* error) { grpc_slice_buffer_reset_and_unref_internal(tcp->read_slices); call_read_cb(tcp, GRPC_ERROR_REF(error)); } - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "Initiating read on %p: error=%s", tcp->socket, str); } @@ -211,7 +211,7 @@ static void custom_write_callback(grpc_custom_socket* socket, custom_tcp_endpoint* tcp = (custom_tcp_endpoint*)socket->endpoint; grpc_closure* cb = tcp->write_cb; tcp->write_cb = nullptr; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "write complete on %p: error=%s", tcp->socket, str); } @@ -224,7 +224,7 @@ static void endpoint_write(grpc_endpoint* ep, grpc_slice_buffer* write_slices, custom_tcp_endpoint* tcp = (custom_tcp_endpoint*)ep; GRPC_CUSTOM_IOMGR_ASSERT_SAME_THREAD(); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { size_t j; for (j = 0; j < write_slices->count; j++) { @@ -280,7 +280,7 @@ static void endpoint_delete_from_pollset_set(grpc_endpoint* ep, static void endpoint_shutdown(grpc_endpoint* ep, grpc_error* why) { custom_tcp_endpoint* tcp = (custom_tcp_endpoint*)ep; if (!tcp->shutting_down) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(why); gpr_log(GPR_INFO, "TCP %p shutdown why=%s", tcp->socket, str); } @@ -345,7 +345,7 @@ grpc_endpoint* custom_tcp_endpoint_create(grpc_custom_socket* socket, (custom_tcp_endpoint*)gpr_malloc(sizeof(custom_tcp_endpoint)); grpc_core::ExecCtx exec_ctx; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "Creating TCP endpoint %p", socket); } memset(tcp, 0, sizeof(custom_tcp_endpoint)); diff --git a/src/core/lib/iomgr/tcp_posix.cc b/src/core/lib/iomgr/tcp_posix.cc index b140ae0ccae..b9376b3ed0a 100644 --- a/src/core/lib/iomgr/tcp_posix.cc +++ b/src/core/lib/iomgr/tcp_posix.cc @@ -163,7 +163,7 @@ static void tcp_drop_uncovered_then_handle_write(void* arg /* grpc_tcp */, static void done_poller(void* bp, grpc_error* error_ignored) { backup_poller* p = static_cast(bp); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p destroy", p); } grpc_pollset_destroy(BACKUP_POLLER_POLLSET(p)); @@ -172,7 +172,7 @@ static void done_poller(void* bp, grpc_error* error_ignored) { static void run_poller(void* bp, grpc_error* error_ignored) { backup_poller* p = static_cast(bp); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p run", p); } gpr_mu_lock(p->pollset_mu); @@ -188,18 +188,18 @@ static void run_poller(void* bp, grpc_error* error_ignored) { gpr_atm_full_cas(&g_uncovered_notifications_pending, 1, 0)) { gpr_mu_lock(p->pollset_mu); bool cas_ok = gpr_atm_full_cas(&g_backup_poller, (gpr_atm)p, 0); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p done cas_ok=%d", p, cas_ok); } gpr_mu_unlock(p->pollset_mu); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p shutdown", p); } grpc_pollset_shutdown(BACKUP_POLLER_POLLSET(p), GRPC_CLOSURE_INIT(&p->run_poller, done_poller, p, grpc_schedule_on_exec_ctx)); } else { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p reschedule", p); } GRPC_CLOSURE_SCHED(&p->run_poller, GRPC_ERROR_NONE); @@ -210,7 +210,7 @@ static void drop_uncovered(grpc_tcp* tcp) { backup_poller* p = (backup_poller*)gpr_atm_acq_load(&g_backup_poller); gpr_atm old_count = gpr_atm_full_fetch_add(&g_uncovered_notifications_pending, -1); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p uncover cnt %d->%d", p, static_cast(old_count), static_cast(old_count) - 1); } @@ -228,7 +228,7 @@ static void cover_self(grpc_tcp* tcp) { backup_poller* p; gpr_atm old_count = gpr_atm_no_barrier_fetch_add(&g_uncovered_notifications_pending, 2); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER: cover cnt %d->%d", static_cast(old_count), 2 + static_cast(old_count)); } @@ -236,7 +236,7 @@ static void cover_self(grpc_tcp* tcp) { GRPC_STATS_INC_TCP_BACKUP_POLLERS_CREATED(); p = static_cast( gpr_zalloc(sizeof(*p) + grpc_pollset_size())); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p create", p); } grpc_pollset_init(BACKUP_POLLER_POLLSET(p), &p->pollset_mu); @@ -251,7 +251,7 @@ static void cover_self(grpc_tcp* tcp) { // spin waiting for backup poller } } - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "BACKUP_POLLER:%p add %p", p, tcp); } grpc_pollset_add_fd(BACKUP_POLLER_POLLSET(p), tcp->em_fd); @@ -261,14 +261,14 @@ static void cover_self(grpc_tcp* tcp) { } static void notify_on_read(grpc_tcp* tcp) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p notify_on_read", tcp); } grpc_fd_notify_on_read(tcp->em_fd, &tcp->read_done_closure); } static void notify_on_write(grpc_tcp* tcp) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p notify_on_write", tcp); } if (!grpc_event_engine_run_in_background()) { @@ -278,7 +278,7 @@ static void notify_on_write(grpc_tcp* tcp) { } static void tcp_drop_uncovered_then_handle_write(void* arg, grpc_error* error) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p got_write: %s", arg, grpc_error_string(error)); } drop_uncovered(static_cast(arg)); @@ -363,7 +363,7 @@ static void tcp_free(grpc_tcp* tcp) { #define TCP_REF(tcp, reason) tcp_ref((tcp), (reason), __FILE__, __LINE__) static void tcp_unref(grpc_tcp* tcp, const char* reason, const char* file, int line) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_atm val = gpr_atm_no_barrier_load(&tcp->refcount.count); gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "TCP unref %p : %s %" PRIdPTR " -> %" PRIdPTR, tcp, reason, val, @@ -376,7 +376,7 @@ static void tcp_unref(grpc_tcp* tcp, const char* reason, const char* file, static void tcp_ref(grpc_tcp* tcp, const char* reason, const char* file, int line) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_atm val = gpr_atm_no_barrier_load(&tcp->refcount.count); gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "TCP ref %p : %s %" PRIdPTR " -> %" PRIdPTR, tcp, reason, val, @@ -409,7 +409,7 @@ static void tcp_destroy(grpc_endpoint* ep) { static void call_read_cb(grpc_tcp* tcp, grpc_error* error) { grpc_closure* cb = tcp->read_cb; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p call_cb %p %p:%p", tcp, cb, cb->cb, cb->cb_arg); size_t i; const char* str = grpc_error_string(error); @@ -573,7 +573,7 @@ static void tcp_do_read(grpc_tcp* tcp) { static void tcp_read_allocation_done(void* tcpp, grpc_error* error) { grpc_tcp* tcp = static_cast(tcpp); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p read_allocation_done: %s", tcp, grpc_error_string(error)); } @@ -592,13 +592,13 @@ static void tcp_continue_read(grpc_tcp* tcp) { /* Wait for allocation only when there is no buffer left. */ if (tcp->incoming_buffer->length == 0 && tcp->incoming_buffer->count < MAX_READ_IOVEC) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p alloc_slices", tcp); } grpc_resource_user_alloc_slices(&tcp->slice_allocator, target_read_size, 1, tcp->incoming_buffer); } else { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p do_read", tcp); } tcp_do_read(tcp); @@ -607,7 +607,7 @@ static void tcp_continue_read(grpc_tcp* tcp) { static void tcp_handle_read(void* arg /* grpc_tcp */, grpc_error* error) { grpc_tcp* tcp = static_cast(arg); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p got_read: %s", tcp, grpc_error_string(error)); } @@ -686,7 +686,7 @@ static bool tcp_write_with_timestamps(grpc_tcp* tcp, struct msghdr* msg, if (setsockopt(tcp->fd, SOL_SOCKET, SO_TIMESTAMPING, static_cast(&opt), sizeof(opt)) != 0) { grpc_slice_buffer_reset_and_unref_internal(tcp->outgoing_buffer); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_ERROR, "Failed to set timestamping options on the socket."); } return false; @@ -735,7 +735,7 @@ struct cmsghdr* process_timestamp(grpc_tcp* tcp, msghdr* msg, auto next_cmsg = CMSG_NXTHDR(msg, cmsg); cmsghdr* opt_stats = nullptr; if (next_cmsg == nullptr) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_ERROR, "Received timestamp without extended error"); } return cmsg; @@ -747,7 +747,7 @@ struct cmsghdr* process_timestamp(grpc_tcp* tcp, msghdr* msg, opt_stats = next_cmsg; next_cmsg = CMSG_NXTHDR(msg, opt_stats); if (next_cmsg == nullptr) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_ERROR, "Received timestamp without extended error"); } return opt_stats; @@ -757,7 +757,7 @@ struct cmsghdr* process_timestamp(grpc_tcp* tcp, msghdr* msg, if (!(next_cmsg->cmsg_level == SOL_IP || next_cmsg->cmsg_level == SOL_IPV6) || !(next_cmsg->cmsg_type == IP_RECVERR || next_cmsg->cmsg_type == IPV6_RECVERR)) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_ERROR, "Unexpected control message"); } return cmsg; @@ -839,7 +839,7 @@ static void process_errors(grpc_tcp* tcp) { cmsg->cmsg_type != SCM_TIMESTAMPING) { /* Got a control message that is not a timestamp. Don't know how to * handle this. */ - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "unknown control message cmsg_level:%d cmsg_type:%d", cmsg->cmsg_level, cmsg->cmsg_type); @@ -857,7 +857,7 @@ static void process_errors(grpc_tcp* tcp) { static void tcp_handle_error(void* arg /* grpc_tcp */, grpc_error* error) { grpc_tcp* tcp = static_cast(arg); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "TCP:%p got_error: %s", tcp, grpc_error_string(error)); } @@ -1034,14 +1034,14 @@ static void tcp_handle_write(void* arg /* grpc_tcp */, grpc_error* error) { } if (!tcp_flush(tcp, &error)) { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "write: delayed"); } notify_on_write(tcp); } else { cb = tcp->write_cb; tcp->write_cb = nullptr; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "write: %s", str); } @@ -1056,7 +1056,7 @@ static void tcp_write(grpc_endpoint* ep, grpc_slice_buffer* buf, grpc_tcp* tcp = reinterpret_cast(ep); grpc_error* error = GRPC_ERROR_NONE; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { size_t i; for (i = 0; i < buf->count; i++) { @@ -1091,12 +1091,12 @@ static void tcp_write(grpc_endpoint* ep, grpc_slice_buffer* buf, if (!tcp_flush(tcp, &error)) { TCP_REF(tcp, "write"); tcp->write_cb = cb; - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "write: delayed"); } notify_on_write(tcp); } else { - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { const char* str = grpc_error_string(error); gpr_log(GPR_INFO, "write: %s", str); } diff --git a/src/core/lib/iomgr/tcp_server_custom.cc b/src/core/lib/iomgr/tcp_server_custom.cc index 019b354473b..133847d3cd3 100644 --- a/src/core/lib/iomgr/tcp_server_custom.cc +++ b/src/core/lib/iomgr/tcp_server_custom.cc @@ -220,7 +220,7 @@ static void finish_accept(grpc_tcp_listener* sp, grpc_custom_socket* socket) { GRPC_LOG_IF_ERROR("getpeername error", err); GRPC_ERROR_UNREF(err); } - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { if (peer_name_string) { gpr_log(GPR_INFO, "SERVER_CONNECT: %p accepted connection: %s", sp->server, peer_name_string); @@ -372,7 +372,7 @@ static grpc_error* tcp_server_add_port(grpc_tcp_server* s, addr = &wildcard; } - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { char* port_string; grpc_sockaddr_to_string(&port_string, addr, 0); const char* str = grpc_error_string(error); @@ -418,7 +418,7 @@ static void tcp_server_start(grpc_tcp_server* server, grpc_pollset** pollsets, (void)pollsets; (void)pollset_count; GRPC_CUSTOM_IOMGR_ASSERT_SAME_THREAD(); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "SERVER_START %p", server); } GPR_ASSERT(on_accept_cb); diff --git a/src/core/lib/iomgr/tcp_server_posix.cc b/src/core/lib/iomgr/tcp_server_posix.cc index baef3886530..0133972e5f7 100644 --- a/src/core/lib/iomgr/tcp_server_posix.cc +++ b/src/core/lib/iomgr/tcp_server_posix.cc @@ -235,7 +235,7 @@ static void on_read(void* arg, grpc_error* err) { addr_str = grpc_sockaddr_to_uri(&addr); gpr_asprintf(&name, "tcp-server-connection:%s", addr_str); - if (grpc_tcp_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_tcp_trace)) { gpr_log(GPR_INFO, "SERVER_CONNECT: incoming connection: %s", addr_str); } diff --git a/src/core/lib/iomgr/timer_generic.cc b/src/core/lib/iomgr/timer_generic.cc index 4bf86b79551..00c18789481 100644 --- a/src/core/lib/iomgr/timer_generic.cc +++ b/src/core/lib/iomgr/timer_generic.cc @@ -361,7 +361,7 @@ static void timer_init(grpc_timer* timer, grpc_millis deadline, timer->hash_table_next = nullptr; #endif - if (grpc_timer_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_trace)) { gpr_log(GPR_INFO, "TIMER %p: SET %" PRId64 " now %" PRId64 " call %p[%p]", timer, deadline, grpc_core::ExecCtx::Get()->Now(), closure, closure->cb); @@ -397,7 +397,7 @@ static void timer_init(grpc_timer* timer, grpc_millis deadline, timer->heap_index = INVALID_HEAP_INDEX; list_join(&shard->list, timer); } - if (grpc_timer_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_trace)) { gpr_log(GPR_INFO, " .. add to shard %d with queue_deadline_cap=%" PRId64 " => is_first_timer=%s", @@ -419,7 +419,7 @@ static void timer_init(grpc_timer* timer, grpc_millis deadline, grpc_timer_check. */ if (is_first_timer) { gpr_mu_lock(&g_shared_mutables.mu); - if (grpc_timer_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_trace)) { gpr_log(GPR_INFO, " .. old shard min_deadline=%" PRId64, shard->min_deadline); } @@ -463,7 +463,7 @@ static void timer_cancel(grpc_timer* timer) { timer_shard* shard = &g_shards[GPR_HASH_POINTER(timer, g_num_shards)]; gpr_mu_lock(&shard->mu); - if (grpc_timer_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_trace)) { gpr_log(GPR_INFO, "TIMER %p: CANCEL pending=%s", timer, timer->pending ? "true" : "false"); } @@ -504,7 +504,7 @@ static bool refill_heap(timer_shard* shard, grpc_millis now) { saturating_add(GPR_MAX(now, shard->queue_deadline_cap), static_cast(deadline_delta * 1000.0)); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. shard[%d]->queue_deadline_cap --> %" PRId64, static_cast(shard - g_shards), shard->queue_deadline_cap); } @@ -512,7 +512,7 @@ static bool refill_heap(timer_shard* shard, grpc_millis now) { next = timer->next; if (timer->deadline < shard->queue_deadline_cap) { - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. add timer with deadline %" PRId64 " to heap", timer->deadline); } @@ -529,7 +529,7 @@ static bool refill_heap(timer_shard* shard, grpc_millis now) { static grpc_timer* pop_one(timer_shard* shard, grpc_millis now) { grpc_timer* timer; for (;;) { - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. shard[%d]: heap_empty=%s", static_cast(shard - g_shards), grpc_timer_heap_is_empty(&shard->heap) ? "true" : "false"); @@ -539,13 +539,13 @@ static grpc_timer* pop_one(timer_shard* shard, grpc_millis now) { if (!refill_heap(shard, now)) return nullptr; } timer = grpc_timer_heap_top(&shard->heap); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. check top timer deadline=%" PRId64 " now=%" PRId64, timer->deadline, now); } if (timer->deadline > now) return nullptr; - if (grpc_timer_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_trace)) { gpr_log(GPR_INFO, "TIMER %p: FIRE %" PRId64 "ms late via %s scheduler", timer, now - timer->deadline, timer->closure->scheduler->vtable->name); @@ -569,7 +569,7 @@ static size_t pop_timers(timer_shard* shard, grpc_millis now, } *new_min_deadline = compute_min_deadline(shard); gpr_mu_unlock(&shard->mu); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. shard[%d] popped %" PRIdPTR, static_cast(shard - g_shards), n); } @@ -606,7 +606,7 @@ static grpc_timer_check_result run_some_expired_timers(grpc_millis now, gpr_mu_lock(&g_shared_mutables.mu); result = GRPC_TIMERS_CHECKED_AND_EMPTY; - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. shard[%d]->min_deadline = %" PRId64, static_cast(g_shard_queue[0] - g_shards), g_shard_queue[0]->min_deadline); @@ -624,7 +624,7 @@ static grpc_timer_check_result run_some_expired_timers(grpc_millis now, result = GRPC_TIMERS_FIRED; } - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, " .. result --> %d" ", shard[%d]->min_deadline %" PRId64 " --> %" PRId64 @@ -691,7 +691,7 @@ static grpc_timer_check_result timer_check(grpc_millis* next) { if (next != nullptr) { *next = GPR_MIN(*next, min_timer); } - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "TIMER CHECK SKIP: now=%" PRId64 " min_timer=%" PRId64, now, min_timer); } @@ -704,7 +704,7 @@ static grpc_timer_check_result timer_check(grpc_millis* next) { : GRPC_ERROR_CREATE_FROM_STATIC_STRING("Shutting down timer system"); // tracing - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { char* next_str; if (next == nullptr) { next_str = gpr_strdup("NULL"); @@ -728,7 +728,7 @@ static grpc_timer_check_result timer_check(grpc_millis* next) { grpc_timer_check_result r = run_some_expired_timers(now, next, shutdown_error); // tracing - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { char* next_str; if (next == nullptr) { next_str = gpr_strdup("NULL"); diff --git a/src/core/lib/iomgr/timer_manager.cc b/src/core/lib/iomgr/timer_manager.cc index 4469db70dd0..bdf54909b4c 100644 --- a/src/core/lib/iomgr/timer_manager.cc +++ b/src/core/lib/iomgr/timer_manager.cc @@ -90,7 +90,7 @@ static void start_timer_thread_and_unlock(void) { ++g_waiter_count; ++g_thread_count; gpr_mu_unlock(&g_mu); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "Spawn timer thread"); } completed_thread* ct = @@ -126,7 +126,7 @@ static void run_some_timers() { // if there's no thread waiting with a timeout, kick an existing untimed // waiter so that the next deadline is not missed if (!g_has_timed_waiter) { - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "kick untimed waiter"); } gpr_cv_signal(&g_cv_wait); @@ -134,7 +134,7 @@ static void run_some_timers() { gpr_mu_unlock(&g_mu); } // without our lock, flush the exec_ctx - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "flush exec_ctx"); } grpc_core::ExecCtx::Get()->Flush(); @@ -189,7 +189,7 @@ static bool wait_until(grpc_millis next) { g_has_timed_waiter = true; g_timed_waiter_deadline = next; - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { grpc_millis wait_time = next - grpc_core::ExecCtx::Get()->Now(); gpr_log(GPR_INFO, "sleep for a %" PRId64 " milliseconds", wait_time); } @@ -198,7 +198,8 @@ static bool wait_until(grpc_millis next) { } } - if (grpc_timer_check_trace.enabled() && next == GRPC_MILLIS_INF_FUTURE) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace) && + next == GRPC_MILLIS_INF_FUTURE) { gpr_log(GPR_INFO, "sleep until kicked"); } @@ -210,7 +211,7 @@ static bool wait_until(grpc_millis next) { gpr_cv_wait(&g_cv_wait, &g_mu, grpc_millis_to_timespec(next, GPR_CLOCK_MONOTONIC)); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "wait ended: was_timed:%d kicked:%d", my_timed_waiter_generation == g_timed_waiter_generation, g_kicked); @@ -255,7 +256,7 @@ static void timer_main_loop() { Consequently, we can just sleep forever here and be happy at some saved wakeup cycles. */ - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "timers not checked: expect another thread to"); } next = GRPC_MILLIS_INF_FUTURE; @@ -281,7 +282,7 @@ static void timer_thread_cleanup(completed_thread* ct) { ct->next = g_completed_threads; g_completed_threads = ct; gpr_mu_unlock(&g_mu); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "End timer thread"); } } @@ -327,18 +328,18 @@ void grpc_timer_manager_init(void) { static void stop_threads(void) { gpr_mu_lock(&g_mu); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "stop timer threads: threaded=%d", g_threaded); } if (g_threaded) { g_threaded = false; gpr_cv_broadcast(&g_cv_wait); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "num timer threads: %d", g_thread_count); } while (g_thread_count > 0) { gpr_cv_wait(&g_cv_shutdown, &g_mu, gpr_inf_future(GPR_CLOCK_MONOTONIC)); - if (grpc_timer_check_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_timer_check_trace)) { gpr_log(GPR_INFO, "num timer threads: %d", g_thread_count); } gc_completed_threads(); diff --git a/src/core/lib/security/credentials/jwt/jwt_credentials.cc b/src/core/lib/security/credentials/jwt/jwt_credentials.cc index 70fe45e56dc..df1d05c83b3 100644 --- a/src/core/lib/security/credentials/jwt/jwt_credentials.cc +++ b/src/core/lib/security/credentials/jwt/jwt_credentials.cc @@ -160,7 +160,7 @@ static char* redact_private_key(const char* json_key) { grpc_call_credentials* grpc_service_account_jwt_access_credentials_create( const char* json_key, gpr_timespec token_lifetime, void* reserved) { - if (grpc_api_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace)) { char* clean_json = redact_private_key(json_key); gpr_log(GPR_INFO, "grpc_service_account_jwt_access_credentials_create(" diff --git a/src/core/lib/security/credentials/oauth2/oauth2_credentials.cc b/src/core/lib/security/credentials/oauth2/oauth2_credentials.cc index b9af757d05e..b001868b3d3 100644 --- a/src/core/lib/security/credentials/oauth2/oauth2_credentials.cc +++ b/src/core/lib/security/credentials/oauth2/oauth2_credentials.cc @@ -459,7 +459,7 @@ grpc_call_credentials* grpc_google_refresh_token_credentials_create( const char* json_refresh_token, void* reserved) { grpc_auth_refresh_token token = grpc_auth_refresh_token_create_from_string(json_refresh_token); - if (grpc_api_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace)) { char* loggable_token = create_loggable_refresh_token(&token); gpr_log(GPR_INFO, "grpc_refresh_token_credentials_create(json_refresh_token=%s, " diff --git a/src/core/lib/security/credentials/plugin/plugin_credentials.cc b/src/core/lib/security/credentials/plugin/plugin_credentials.cc index 59fecbca992..333366d0f0a 100644 --- a/src/core/lib/security/credentials/plugin/plugin_credentials.cc +++ b/src/core/lib/security/credentials/plugin/plugin_credentials.cc @@ -119,7 +119,7 @@ static void plugin_md_request_metadata_ready(void* request, GRPC_EXEC_CTX_FLAG_THREAD_RESOURCE_LOOP); grpc_plugin_credentials::pending_request* r = static_cast(request); - if (grpc_plugin_credentials_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: request %p: plugin returned " "asynchronously", @@ -132,7 +132,7 @@ static void plugin_md_request_metadata_ready(void* request, grpc_error* error = process_plugin_result(r, md, num_md, status, error_details); GRPC_CLOSURE_SCHED(r->on_request_metadata, error); - } else if (grpc_plugin_credentials_trace.enabled()) { + } else if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: request %p: plugin was previously " "cancelled", @@ -162,7 +162,7 @@ bool grpc_plugin_credentials::get_request_metadata( pending_requests_ = request; gpr_mu_unlock(&mu_); // Invoke the plugin. The callback holds a ref to us. - if (grpc_plugin_credentials_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: request %p: invoking plugin", this, request); } @@ -174,7 +174,7 @@ bool grpc_plugin_credentials::get_request_metadata( if (!plugin_.get_metadata( plugin_.state, context, plugin_md_request_metadata_ready, request, creds_md, &num_creds_md, &status, &error_details)) { - if (grpc_plugin_credentials_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: request %p: plugin will return " "asynchronously", @@ -189,7 +189,7 @@ bool grpc_plugin_credentials::get_request_metadata( // asynchronously by plugin_cancel_get_request_metadata(), so return // false. Otherwise, process the result. if (request->cancelled) { - if (grpc_plugin_credentials_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: request %p was cancelled, error " "will be returned asynchronously", @@ -197,7 +197,7 @@ bool grpc_plugin_credentials::get_request_metadata( } retval = false; } else { - if (grpc_plugin_credentials_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: request %p: plugin returned " "synchronously", @@ -223,7 +223,7 @@ void grpc_plugin_credentials::cancel_get_request_metadata( for (pending_request* pending_request = pending_requests_; pending_request != nullptr; pending_request = pending_request->next) { if (pending_request->md_array == md_array) { - if (grpc_plugin_credentials_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_plugin_credentials_trace)) { gpr_log(GPR_INFO, "plugin_credentials[%p]: cancelling request %p", this, pending_request); } diff --git a/src/core/lib/security/transport/secure_endpoint.cc b/src/core/lib/security/transport/secure_endpoint.cc index 2a862492bd7..0aac7d8d780 100644 --- a/src/core/lib/security/transport/secure_endpoint.cc +++ b/src/core/lib/security/transport/secure_endpoint.cc @@ -113,7 +113,7 @@ static void destroy(secure_endpoint* ep) { grpc_core::Delete(ep); } secure_endpoint_ref((ep), (reason), __FILE__, __LINE__) static void secure_endpoint_unref(secure_endpoint* ep, const char* reason, const char* file, int line) { - if (grpc_trace_secure_endpoint.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_secure_endpoint)) { gpr_atm val = gpr_atm_no_barrier_load(&ep->ref.count); gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "SECENDP unref %p : %s %" PRIdPTR " -> %" PRIdPTR, ep, reason, val, @@ -126,7 +126,7 @@ static void secure_endpoint_unref(secure_endpoint* ep, const char* reason, static void secure_endpoint_ref(secure_endpoint* ep, const char* reason, const char* file, int line) { - if (grpc_trace_secure_endpoint.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_secure_endpoint)) { gpr_atm val = gpr_atm_no_barrier_load(&ep->ref.count); gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "SECENDP ref %p : %s %" PRIdPTR " -> %" PRIdPTR, ep, reason, val, @@ -155,7 +155,7 @@ static void flush_read_staging_buffer(secure_endpoint* ep, uint8_t** cur, } static void call_read_cb(secure_endpoint* ep, grpc_error* error) { - if (grpc_trace_secure_endpoint.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_secure_endpoint)) { size_t i; for (i = 0; i < ep->read_buffer->count; i++) { char* data = grpc_dump_slice(ep->read_buffer->slices[i], @@ -292,7 +292,7 @@ static void endpoint_write(grpc_endpoint* secure_ep, grpc_slice_buffer* slices, grpc_slice_buffer_reset_and_unref_internal(&ep->output_buffer); - if (grpc_trace_secure_endpoint.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_secure_endpoint)) { for (i = 0; i < slices->count; i++) { char* data = grpc_dump_slice(slices->slices[i], GPR_DUMP_HEX | GPR_DUMP_ASCII); diff --git a/src/core/lib/surface/api_trace.h b/src/core/lib/surface/api_trace.h index 72ed830554f..51d1f522230 100644 --- a/src/core/lib/surface/api_trace.h +++ b/src/core/lib/surface/api_trace.h @@ -45,7 +45,7 @@ extern grpc_core::TraceFlag grpc_api_trace; /* Due to the limitations of C89's preprocessor, the arity of the var-arg list 'nargs' must be specified. */ #define GRPC_API_TRACE(fmt, nargs, args) \ - if (grpc_api_trace.enabled()) { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace)) { \ gpr_log(GPR_INFO, fmt GRPC_API_TRACE_UNWRAP##nargs args); \ } diff --git a/src/core/lib/surface/call.cc b/src/core/lib/surface/call.cc index bd140021c96..d0cddfa5a12 100644 --- a/src/core/lib/surface/call.cc +++ b/src/core/lib/surface/call.cc @@ -723,7 +723,7 @@ static void cancel_with_status(grpc_call* c, grpc_status_code status, } static void set_final_status(grpc_call* call, grpc_error* error) { - if (grpc_call_error_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_call_error_trace)) { gpr_log(GPR_DEBUG, "set_final_status %s", call->is_client ? "CLI" : "SVR"); gpr_log(GPR_DEBUG, "%s", grpc_error_string(error)); } @@ -1280,7 +1280,7 @@ static void receiving_slice_ready(void* bctlp, grpc_error* error) { } if (error != GRPC_ERROR_NONE) { - if (grpc_trace_operation_failures.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures)) { GRPC_LOG_IF_ERROR("receiving_slice_ready", GRPC_ERROR_REF(error)); } call->receiving_stream.reset(); @@ -1404,7 +1404,7 @@ static void validate_filtered_metadata(batch_control* bctl) { GPR_ASSERT(call->encodings_accepted_by_peer != 0); if (!GPR_BITGET(call->encodings_accepted_by_peer, compression_algorithm)) { - if (grpc_compression_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_compression_trace)) { const char* algo_name = nullptr; grpc_compression_algorithm_name(compression_algorithm, &algo_name); gpr_log(GPR_ERROR, diff --git a/src/core/lib/surface/call.h b/src/core/lib/surface/call.h index d9cb97cb31b..11bde0787cb 100644 --- a/src/core/lib/surface/call.h +++ b/src/core/lib/surface/call.h @@ -102,7 +102,8 @@ void grpc_call_context_set(grpc_call* call, grpc_context_index elem, void* grpc_call_context_get(grpc_call* call, grpc_context_index elem); #define GRPC_CALL_LOG_BATCH(sev, call, ops, nops, tag) \ - if (grpc_api_trace.enabled()) grpc_call_log_batch(sev, call, ops, nops, tag) + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace)) \ + grpc_call_log_batch(sev, call, ops, nops, tag) uint8_t grpc_call_is_client(grpc_call* call); diff --git a/src/core/lib/surface/completion_queue.cc b/src/core/lib/surface/completion_queue.cc index 3a32d292a7b..bb5921a6a55 100644 --- a/src/core/lib/surface/completion_queue.cc +++ b/src/core/lib/surface/completion_queue.cc @@ -411,12 +411,13 @@ static const cq_vtable g_cq_vtable[] = { grpc_core::TraceFlag grpc_cq_pluck_trace(false, "queue_pluck"); -#define GRPC_SURFACE_TRACE_RETURNED_EVENT(cq, event) \ - if (grpc_api_trace.enabled() && (grpc_cq_pluck_trace.enabled() || \ - (event)->type != GRPC_QUEUE_TIMEOUT)) { \ - char* _ev = grpc_event_string(event); \ - gpr_log(GPR_INFO, "RETURN_EVENT[%p]: %s", cq, _ev); \ - gpr_free(_ev); \ +#define GRPC_SURFACE_TRACE_RETURNED_EVENT(cq, event) \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace) && \ + (GRPC_TRACE_FLAG_ENABLED(grpc_cq_pluck_trace) || \ + (event)->type != GRPC_QUEUE_TIMEOUT)) { \ + char* _ev = grpc_event_string(event); \ + gpr_log(GPR_INFO, "RETURN_EVENT[%p]: %s", cq, _ev); \ + gpr_free(_ev); \ } static void on_pollset_shutdown_done(void* cq, grpc_error* error); @@ -572,7 +573,7 @@ int grpc_get_cq_poll_num(grpc_completion_queue* cq) { #ifndef NDEBUG void grpc_cq_internal_ref(grpc_completion_queue* cq, const char* reason, const char* file, int line) { - if (grpc_trace_cq_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cq_refcount)) { gpr_atm val = gpr_atm_no_barrier_load(&cq->owning_refs.count); gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "CQ:%p ref %" PRIdPTR " -> %" PRIdPTR " %s", cq, val, val + 1, @@ -592,7 +593,7 @@ static void on_pollset_shutdown_done(void* arg, grpc_error* error) { #ifndef NDEBUG void grpc_cq_internal_unref(grpc_completion_queue* cq, const char* reason, const char* file, int line) { - if (grpc_trace_cq_refcount.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cq_refcount)) { gpr_atm val = gpr_atm_no_barrier_load(&cq->owning_refs.count); gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "CQ:%p unref %" PRIdPTR " -> %" PRIdPTR " %s", cq, val, val - 1, @@ -678,14 +679,16 @@ static void cq_end_op_for_next(grpc_completion_queue* cq, void* tag, void* done_arg, grpc_cq_completion* storage) { GPR_TIMER_SCOPE("cq_end_op_for_next", 0); - if (grpc_api_trace.enabled() || - (grpc_trace_operation_failures.enabled() && error != GRPC_ERROR_NONE)) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace) || + (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures) && + error != GRPC_ERROR_NONE)) { const char* errmsg = grpc_error_string(error); GRPC_API_TRACE( "cq_end_op_for_next(cq=%p, tag=%p, error=%s, " "done=%p, done_arg=%p, storage=%p)", 6, (cq, tag, errmsg, done, done_arg, storage)); - if (grpc_trace_operation_failures.enabled() && error != GRPC_ERROR_NONE) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures) && + error != GRPC_ERROR_NONE) { gpr_log(GPR_ERROR, "Operation failed: tag=%p, error=%s", tag, errmsg); } } @@ -759,14 +762,16 @@ static void cq_end_op_for_pluck(grpc_completion_queue* cq, void* tag, cq_pluck_data* cqd = static_cast DATA_FROM_CQ(cq); int is_success = (error == GRPC_ERROR_NONE); - if (grpc_api_trace.enabled() || - (grpc_trace_operation_failures.enabled() && error != GRPC_ERROR_NONE)) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace) || + (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures) && + error != GRPC_ERROR_NONE)) { const char* errmsg = grpc_error_string(error); GRPC_API_TRACE( "cq_end_op_for_pluck(cq=%p, tag=%p, error=%s, " "done=%p, done_arg=%p, storage=%p)", 6, (cq, tag, errmsg, done, done_arg, storage)); - if (grpc_trace_operation_failures.enabled() && error != GRPC_ERROR_NONE) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures) && + error != GRPC_ERROR_NONE) { gpr_log(GPR_ERROR, "Operation failed: tag=%p, error=%s", tag, errmsg); } } @@ -824,14 +829,16 @@ static void cq_end_op_for_callback( cq_callback_data* cqd = static_cast DATA_FROM_CQ(cq); bool is_success = (error == GRPC_ERROR_NONE); - if (grpc_api_trace.enabled() || - (grpc_trace_operation_failures.enabled() && error != GRPC_ERROR_NONE)) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace) || + (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures) && + error != GRPC_ERROR_NONE)) { const char* errmsg = grpc_error_string(error); GRPC_API_TRACE( "cq_end_op_for_callback(cq=%p, tag=%p, error=%s, " "done=%p, done_arg=%p, storage=%p)", 6, (cq, tag, errmsg, done, done_arg, storage)); - if (grpc_trace_operation_failures.enabled() && error != GRPC_ERROR_NONE) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_operation_failures) && + error != GRPC_ERROR_NONE) { gpr_log(GPR_ERROR, "Operation failed: tag=%p, error=%s", tag, errmsg); } } @@ -906,7 +913,7 @@ class ExecCtxNext : public grpc_core::ExecCtx { #ifndef NDEBUG static void dump_pending_tags(grpc_completion_queue* cq) { - if (!grpc_trace_pending_tags.enabled()) return; + if (!GRPC_TRACE_FLAG_ENABLED(grpc_trace_pending_tags)) return; gpr_strvec v; gpr_strvec_init(&v); @@ -1176,7 +1183,7 @@ static grpc_event cq_pluck(grpc_completion_queue* cq, void* tag, grpc_pollset_worker* worker = nullptr; cq_pluck_data* cqd = static_cast DATA_FROM_CQ(cq); - if (grpc_cq_pluck_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_cq_pluck_trace)) { GRPC_API_TRACE( "grpc_completion_queue_pluck(" "cq=%p, tag=%p, " diff --git a/src/core/lib/surface/server.cc b/src/core/lib/surface/server.cc index 443d199898d..2377c4d8f23 100644 --- a/src/core/lib/surface/server.cc +++ b/src/core/lib/surface/server.cc @@ -464,7 +464,8 @@ static void destroy_channel(channel_data* chand, grpc_error* error) { GRPC_CLOSURE_INIT(&chand->finish_destroy_channel_closure, finish_destroy_channel, chand, grpc_schedule_on_exec_ctx); - if (grpc_server_channel_trace.enabled() && error != GRPC_ERROR_NONE) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_server_channel_trace) && + error != GRPC_ERROR_NONE) { const char* msg = grpc_error_string(error); gpr_log(GPR_INFO, "Disconnected client: %s", msg); } diff --git a/src/core/lib/transport/bdp_estimator.cc b/src/core/lib/transport/bdp_estimator.cc index 8e71f869894..8835e32bcff 100644 --- a/src/core/lib/transport/bdp_estimator.cc +++ b/src/core/lib/transport/bdp_estimator.cc @@ -46,7 +46,7 @@ grpc_millis BdpEstimator::CompletePing() { 1e-9 * static_cast(dt_ts.tv_nsec); double bw = dt > 0 ? (static_cast(accumulator_) / dt) : 0; int start_inter_ping_delay = inter_ping_delay_; - if (grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "bdp[%s]:complete acc=%" PRId64 " est=%" PRId64 " dt=%lf bw=%lfMbs bw_est=%lfMbs", @@ -57,7 +57,7 @@ grpc_millis BdpEstimator::CompletePing() { if (accumulator_ > 2 * estimate_ / 3 && bw > bw_est_) { estimate_ = GPR_MAX(accumulator_, estimate_ * 2); bw_est_ = bw; - if (grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "bdp[%s]: estimate increased to %" PRId64, name_, estimate_); } @@ -74,7 +74,7 @@ grpc_millis BdpEstimator::CompletePing() { } if (start_inter_ping_delay != inter_ping_delay_) { stable_estimate_count_ = 0; - if (grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "bdp[%s]:update_inter_time to %dms", name_, inter_ping_delay_); } diff --git a/src/core/lib/transport/bdp_estimator.h b/src/core/lib/transport/bdp_estimator.h index ab13ae4be4c..6dc4d6bb05e 100644 --- a/src/core/lib/transport/bdp_estimator.h +++ b/src/core/lib/transport/bdp_estimator.h @@ -49,7 +49,7 @@ class BdpEstimator { // grpc_bdp_estimator_add_incoming_bytes once a ping has been scheduled by a // transport (but not necessarily started) void SchedulePing() { - if (grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "bdp[%s]:sched acc=%" PRId64 " est=%" PRId64, name_, accumulator_, estimate_); } @@ -62,7 +62,7 @@ class BdpEstimator { // once // the ping is on the wire void StartPing() { - if (grpc_bdp_estimator_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_bdp_estimator_trace)) { gpr_log(GPR_INFO, "bdp[%s]:start acc=%" PRId64 " est=%" PRId64, name_, accumulator_, estimate_); } diff --git a/src/core/lib/transport/connectivity_state.cc b/src/core/lib/transport/connectivity_state.cc index 5b73085c7f3..bf35fd09def 100644 --- a/src/core/lib/transport/connectivity_state.cc +++ b/src/core/lib/transport/connectivity_state.cc @@ -75,7 +75,7 @@ grpc_connectivity_state grpc_connectivity_state_check( grpc_connectivity_state_tracker* tracker) { grpc_connectivity_state cur = static_cast( gpr_atm_no_barrier_load(&tracker->current_state_atm)); - if (grpc_connectivity_state_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_connectivity_state_trace)) { gpr_log(GPR_INFO, "CONWATCH: %p %s: get %s", tracker, tracker->name, grpc_connectivity_state_name(cur)); } @@ -92,7 +92,7 @@ bool grpc_connectivity_state_notify_on_state_change( grpc_closure* notify) { grpc_connectivity_state cur = static_cast( gpr_atm_no_barrier_load(&tracker->current_state_atm)); - if (grpc_connectivity_state_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_connectivity_state_trace)) { if (current == nullptr) { gpr_log(GPR_INFO, "CONWATCH: %p %s: unsubscribe notify=%p", tracker, tracker->name, notify); @@ -143,7 +143,7 @@ void grpc_connectivity_state_set(grpc_connectivity_state_tracker* tracker, grpc_connectivity_state cur = static_cast( gpr_atm_no_barrier_load(&tracker->current_state_atm)); grpc_connectivity_state_watcher* w; - if (grpc_connectivity_state_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_connectivity_state_trace)) { gpr_log(GPR_INFO, "SET: %p %s: %s --> %s [%s]", tracker, tracker->name, grpc_connectivity_state_name(cur), grpc_connectivity_state_name(state), reason); @@ -156,7 +156,7 @@ void grpc_connectivity_state_set(grpc_connectivity_state_tracker* tracker, while ((w = tracker->watchers) != nullptr) { *w->current = state; tracker->watchers = w->next; - if (grpc_connectivity_state_trace.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(grpc_connectivity_state_trace)) { gpr_log(GPR_INFO, "NOTIFY: %p %s: %p", tracker, tracker->name, w->notify); } GRPC_CLOSURE_SCHED(w->notify, GRPC_ERROR_NONE); diff --git a/src/core/tsi/fake_transport_security.cc b/src/core/tsi/fake_transport_security.cc index 4d4c4950451..fde88dd819b 100644 --- a/src/core/tsi/fake_transport_security.cc +++ b/src/core/tsi/fake_transport_security.cc @@ -585,7 +585,7 @@ static tsi_result fake_handshaker_get_bytes_to_send_to_peer( if (next_message_to_send > TSI_FAKE_HANDSHAKE_MESSAGE_MAX) { next_message_to_send = TSI_FAKE_HANDSHAKE_MESSAGE_MAX; } - if (tsi_tracing_enabled.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(tsi_tracing_enabled)) { gpr_log(GPR_INFO, "%s prepared %s.", impl->is_client ? "Client" : "Server", tsi_fake_handshake_message_to_string(impl->next_message_to_send)); @@ -597,7 +597,7 @@ static tsi_result fake_handshaker_get_bytes_to_send_to_peer( if (!impl->is_client && impl->next_message_to_send == TSI_FAKE_HANDSHAKE_MESSAGE_MAX) { /* We're done. */ - if (tsi_tracing_enabled.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(tsi_tracing_enabled)) { gpr_log(GPR_INFO, "Server is done."); } impl->result = TSI_OK; @@ -636,7 +636,7 @@ static tsi_result fake_handshaker_process_bytes_from_peer( tsi_fake_handshake_message_to_string(received_msg), tsi_fake_handshake_message_to_string(expected_msg)); } - if (tsi_tracing_enabled.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(tsi_tracing_enabled)) { gpr_log(GPR_INFO, "%s received %s.", impl->is_client ? "Client" : "Server", tsi_fake_handshake_message_to_string(received_msg)); } @@ -644,7 +644,7 @@ static tsi_result fake_handshaker_process_bytes_from_peer( impl->needs_incoming_message = 0; if (impl->next_message_to_send == TSI_FAKE_HANDSHAKE_MESSAGE_MAX) { /* We're done. */ - if (tsi_tracing_enabled.enabled()) { + if (GRPC_TRACE_FLAG_ENABLED(tsi_tracing_enabled)) { gpr_log(GPR_INFO, "%s is done.", impl->is_client ? "Client" : "Server"); } impl->result = TSI_OK; diff --git a/src/core/tsi/ssl_transport_security.cc b/src/core/tsi/ssl_transport_security.cc index cbdb4227b31..25ae2cee285 100644 --- a/src/core/tsi/ssl_transport_security.cc +++ b/src/core/tsi/ssl_transport_security.cc @@ -213,7 +213,7 @@ static const char* ssl_error_string(int error) { /* TODO(jboeuf): Remove when we are past the debugging phase with this code. */ static void ssl_log_where_info(const SSL* ssl, int where, int flag, const char* msg) { - if ((where & flag) && tsi_tracing_enabled.enabled()) { + if ((where & flag) && GRPC_TRACE_FLAG_ENABLED(tsi_tracing_enabled)) { gpr_log(GPR_INFO, "%20.20s - %30.30s - %5.10s", msg, SSL_state_string_long(ssl), SSL_state_string(ssl)); } From 3bcae1e3682e45ad528587f47f4c46e5f5d11fbb Mon Sep 17 00:00:00 2001 From: Soheil Hassas Yeganeh Date: Wed, 8 May 2019 12:22:06 -0400 Subject: [PATCH 2/2] Apply do {...} while(0) to the remaining macros. --- .../resolver/dns/c_ares/grpc_ares_wrapper.h | 10 +++++---- .../ext/transport/chttp2/transport/internal.h | 10 +++++---- src/core/lib/iomgr/executor.cc | 22 +++++++++++-------- src/core/lib/surface/call.h | 7 ++++-- src/core/lib/surface/completion_queue.cc | 18 ++++++++------- 5 files changed, 40 insertions(+), 27 deletions(-) diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h index 881dfcdcee6..a707c1ae7c4 100644 --- a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h +++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h @@ -32,10 +32,12 @@ extern grpc_core::TraceFlag grpc_trace_cares_address_sorting; extern grpc_core::TraceFlag grpc_trace_cares_resolver; -#define GRPC_CARES_TRACE_LOG(format, ...) \ - if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cares_resolver)) { \ - gpr_log(GPR_DEBUG, "(c-ares resolver) " format, __VA_ARGS__); \ - } +#define GRPC_CARES_TRACE_LOG(format, ...) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_trace_cares_resolver)) { \ + gpr_log(GPR_DEBUG, "(c-ares resolver) " format, __VA_ARGS__); \ + } \ + } while (0) typedef struct grpc_ares_request grpc_ares_request; diff --git a/src/core/ext/transport/chttp2/transport/internal.h b/src/core/ext/transport/chttp2/transport/internal.h index b3a2545a189..11936dc8cce 100644 --- a/src/core/ext/transport/chttp2/transport/internal.h +++ b/src/core/ext/transport/chttp2/transport/internal.h @@ -771,10 +771,12 @@ void grpc_chttp2_complete_closure_step(grpc_chttp2_transport* t, // extern grpc_core::TraceFlag grpc_http_trace; // extern grpc_core::TraceFlag grpc_flowctl_trace; -#define GRPC_CHTTP2_IF_TRACING(stmt) \ - if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { \ - (stmt); \ - } +#define GRPC_CHTTP2_IF_TRACING(stmt) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_http_trace)) { \ + (stmt); \ + } \ + } while (0) void grpc_chttp2_fake_status(grpc_chttp2_transport* t, grpc_chttp2_stream* stream, grpc_error* error); diff --git a/src/core/lib/iomgr/executor.cc b/src/core/lib/iomgr/executor.cc index 9b967a4fe7a..8adc0902bd1 100644 --- a/src/core/lib/iomgr/executor.cc +++ b/src/core/lib/iomgr/executor.cc @@ -36,15 +36,19 @@ #define MAX_DEPTH 2 -#define EXECUTOR_TRACE(format, ...) \ - if (GRPC_TRACE_FLAG_ENABLED(executor_trace)) { \ - gpr_log(GPR_INFO, "EXECUTOR " format, __VA_ARGS__); \ - } - -#define EXECUTOR_TRACE0(str) \ - if (GRPC_TRACE_FLAG_ENABLED(executor_trace)) { \ - gpr_log(GPR_INFO, "EXECUTOR " str); \ - } +#define EXECUTOR_TRACE(format, ...) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(executor_trace)) { \ + gpr_log(GPR_INFO, "EXECUTOR " format, __VA_ARGS__); \ + } \ + } while (0) + +#define EXECUTOR_TRACE0(str) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(executor_trace)) { \ + gpr_log(GPR_INFO, "EXECUTOR " str); \ + } \ + } while (0) namespace grpc_core { namespace { diff --git a/src/core/lib/surface/call.h b/src/core/lib/surface/call.h index 11bde0787cb..15392fea6dc 100644 --- a/src/core/lib/surface/call.h +++ b/src/core/lib/surface/call.h @@ -102,8 +102,11 @@ void grpc_call_context_set(grpc_call* call, grpc_context_index elem, void* grpc_call_context_get(grpc_call* call, grpc_context_index elem); #define GRPC_CALL_LOG_BATCH(sev, call, ops, nops, tag) \ - if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace)) \ - grpc_call_log_batch(sev, call, ops, nops, tag) + do { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace)) { \ + grpc_call_log_batch(sev, call, ops, nops, tag); \ + } \ + } while (0) uint8_t grpc_call_is_client(grpc_call* call); diff --git a/src/core/lib/surface/completion_queue.cc b/src/core/lib/surface/completion_queue.cc index bb5921a6a55..e796071eedc 100644 --- a/src/core/lib/surface/completion_queue.cc +++ b/src/core/lib/surface/completion_queue.cc @@ -411,14 +411,16 @@ static const cq_vtable g_cq_vtable[] = { grpc_core::TraceFlag grpc_cq_pluck_trace(false, "queue_pluck"); -#define GRPC_SURFACE_TRACE_RETURNED_EVENT(cq, event) \ - if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace) && \ - (GRPC_TRACE_FLAG_ENABLED(grpc_cq_pluck_trace) || \ - (event)->type != GRPC_QUEUE_TIMEOUT)) { \ - char* _ev = grpc_event_string(event); \ - gpr_log(GPR_INFO, "RETURN_EVENT[%p]: %s", cq, _ev); \ - gpr_free(_ev); \ - } +#define GRPC_SURFACE_TRACE_RETURNED_EVENT(cq, event) \ + do { \ + if (GRPC_TRACE_FLAG_ENABLED(grpc_api_trace) && \ + (GRPC_TRACE_FLAG_ENABLED(grpc_cq_pluck_trace) || \ + (event)->type != GRPC_QUEUE_TIMEOUT)) { \ + char* _ev = grpc_event_string(event); \ + gpr_log(GPR_INFO, "RETURN_EVENT[%p]: %s", cq, _ev); \ + gpr_free(_ev); \ + } \ + } while (0) static void on_pollset_shutdown_done(void* cq, grpc_error* error);