mirror of https://github.com/grpc/grpc.git
Implement xDS client-side fault injection filter (#24354)
* Refactored with dynamic filters * Error-tolerance tuned * Fix leak of grpc_error and data race of canceller * Adopt the latest xDS HTTP filter framework * Fix fault injection tests' conflict with router filter * Test alternative setup (override, no-override) without copy * Refactor file strcutures of fault injection filter * Rewrite the Json parsing/assembling logic again * Added logic for aborting streaming RPC && resolve commentspull/25661/head
parent
5d3fe59a5f
commit
3b067c9f3f
40 changed files with 3064 additions and 37 deletions
@ -0,0 +1,495 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/ext/filters/fault_injection/fault_injection_filter.h" |
||||
|
||||
#include "absl/strings/numbers.h" |
||||
|
||||
#include <grpc/support/alloc.h> |
||||
#include <grpc/support/log.h> |
||||
|
||||
#include "src/core/ext/filters/client_channel/service_config.h" |
||||
#include "src/core/ext/filters/client_channel/service_config_call_data.h" |
||||
#include "src/core/ext/filters/fault_injection/service_config_parser.h" |
||||
#include "src/core/lib/channel/channel_stack.h" |
||||
#include "src/core/lib/channel/status_util.h" |
||||
#include "src/core/lib/gprpp/atomic.h" |
||||
#include "src/core/lib/gprpp/sync.h" |
||||
#include "src/core/lib/iomgr/closure.h" |
||||
#include "src/core/lib/iomgr/timer.h" |
||||
#include "src/core/lib/transport/status_conversion.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
TraceFlag grpc_fault_injection_filter_trace(false, "fault_injection_filter"); |
||||
|
||||
namespace { |
||||
|
||||
Atomic<uint32_t> g_active_faults{0}; |
||||
static_assert( |
||||
std::is_trivially_destructible<Atomic<uint32_t>>::value, |
||||
"the active fault counter needs to have a trivially destructible type"); |
||||
|
||||
inline int GetLinkedMetadatumValueInt(grpc_linked_mdelem* md) { |
||||
int res; |
||||
if (absl::SimpleAtoi(StringViewFromSlice(GRPC_MDVALUE(md->md)), &res)) { |
||||
return res; |
||||
} else { |
||||
return -1; |
||||
} |
||||
} |
||||
|
||||
inline uint32_t GetLinkedMetadatumValueUnsignedInt(grpc_linked_mdelem* md) { |
||||
uint32_t res; |
||||
if (absl::SimpleAtoi(StringViewFromSlice(GRPC_MDVALUE(md->md)), &res)) { |
||||
return res; |
||||
} else { |
||||
return -1; |
||||
} |
||||
} |
||||
|
||||
inline int64_t GetLinkedMetadatumValueInt64(grpc_linked_mdelem* md) { |
||||
int64_t res; |
||||
if (absl::SimpleAtoi(StringViewFromSlice(GRPC_MDVALUE(md->md)), &res)) { |
||||
return res; |
||||
} else { |
||||
return -1; |
||||
} |
||||
} |
||||
|
||||
inline bool UnderFraction(const uint32_t numerator, |
||||
const uint32_t denominator) { |
||||
if (numerator <= 0) return false; |
||||
if (numerator >= denominator) return true; |
||||
// Generate a random number in [0, denominator).
|
||||
const uint32_t random_number = rand() % denominator; |
||||
return random_number < numerator; |
||||
} |
||||
|
||||
class ChannelData { |
||||
public: |
||||
static grpc_error* Init(grpc_channel_element* elem, |
||||
grpc_channel_element_args* args); |
||||
static void Destroy(grpc_channel_element* elem); |
||||
|
||||
int index() const { return index_; } |
||||
|
||||
private: |
||||
ChannelData(grpc_channel_element* elem, grpc_channel_element_args* args); |
||||
~ChannelData() = default; |
||||
|
||||
// The relative index of instances of the same filter.
|
||||
int index_; |
||||
}; |
||||
|
||||
class CallData { |
||||
public: |
||||
static grpc_error* Init(grpc_call_element* elem, |
||||
const grpc_call_element_args* args); |
||||
|
||||
static void Destroy(grpc_call_element* elem, |
||||
const grpc_call_final_info* /*final_info*/, |
||||
grpc_closure* /*then_schedule_closure*/); |
||||
|
||||
static void StartTransportStreamOpBatch( |
||||
grpc_call_element* elem, grpc_transport_stream_op_batch* batch); |
||||
|
||||
private: |
||||
class ResumeBatchCanceller; |
||||
|
||||
CallData(grpc_call_element* elem, const grpc_call_element_args* args); |
||||
~CallData(); |
||||
|
||||
void DecideWhetherToInjectFaults(grpc_metadata_batch* initial_metadata); |
||||
|
||||
// Checks if current active faults exceed the allowed max faults.
|
||||
bool HaveActiveFaultsQuota(bool increment); |
||||
|
||||
// Returns true if this RPC needs to be delayed. If so, this call will be
|
||||
// counted as an active fault.
|
||||
bool MaybeDelay(); |
||||
|
||||
// Returns the aborted RPC status if this RPC needs to be aborted. If so,
|
||||
// this call will be counted as an active fault. Otherwise, it returns
|
||||
// GRPC_ERROR_NONE.
|
||||
// If this call is already been delay injected, skip the active faults
|
||||
// quota check.
|
||||
grpc_error* MaybeAbort(); |
||||
|
||||
// Delays the stream operations batch.
|
||||
void DelayBatch(grpc_call_element* elem, |
||||
grpc_transport_stream_op_batch* batch); |
||||
|
||||
// Cancels the delay timer.
|
||||
void CancelDelayTimer() { grpc_timer_cancel(&delay_timer_); } |
||||
|
||||
// Finishes the fault injection, should only be called once.
|
||||
void FaultInjectionFinished() { |
||||
g_active_faults.FetchSub(1, MemoryOrder::RELAXED); |
||||
} |
||||
|
||||
// This is a callback that will be invoked after the delay timer is up.
|
||||
static void ResumeBatch(void* arg, grpc_error* error); |
||||
|
||||
// This is a callback invoked upon completion of recv_trailing_metadata.
|
||||
// Injects the abort_error_ to the recv_trailing_metadata batch if needed.
|
||||
static void HijackedRecvTrailingMetadataReady(void* arg, grpc_error*); |
||||
|
||||
// Used to track the policy structs that needs to be destroyed in dtor.
|
||||
bool fi_policy_owned_ = false; |
||||
const FaultInjectionMethodParsedConfig::FaultInjectionPolicy* fi_policy_; |
||||
grpc_call_stack* owning_call_; |
||||
Arena* arena_; |
||||
CallCombiner* call_combiner_; |
||||
|
||||
// Indicates whether we are doing a delay and/or an abort for this call.
|
||||
bool delay_request_ = false; |
||||
bool abort_request_ = false; |
||||
|
||||
// Delay states
|
||||
grpc_timer delay_timer_ ABSL_GUARDED_BY(delay_mu_); |
||||
ResumeBatchCanceller* resume_batch_canceller_ ABSL_GUARDED_BY(delay_mu_); |
||||
grpc_transport_stream_op_batch* delayed_batch_ ABSL_GUARDED_BY(delay_mu_); |
||||
// Abort states
|
||||
grpc_error* abort_error_ = GRPC_ERROR_NONE; |
||||
grpc_closure recv_trailing_metadata_ready_; |
||||
grpc_closure* original_recv_trailing_metadata_ready_; |
||||
// Protects the asynchronous delay, resume, and cancellation.
|
||||
Mutex delay_mu_; |
||||
}; |
||||
|
||||
// ChannelData
|
||||
|
||||
grpc_error* ChannelData::Init(grpc_channel_element* elem, |
||||
grpc_channel_element_args* args) { |
||||
GPR_ASSERT(elem->filter == &FaultInjectionFilterVtable); |
||||
new (elem->channel_data) ChannelData(elem, args); |
||||
return GRPC_ERROR_NONE; |
||||
} |
||||
|
||||
void ChannelData::Destroy(grpc_channel_element* elem) { |
||||
auto* chand = static_cast<ChannelData*>(elem->channel_data); |
||||
chand->~ChannelData(); |
||||
} |
||||
|
||||
ChannelData::ChannelData(grpc_channel_element* elem, |
||||
grpc_channel_element_args* args) |
||||
: index_(grpc_channel_stack_filter_instance_number(args->channel_stack, |
||||
elem)) {} |
||||
|
||||
// CallData::ResumeBatchCanceller
|
||||
|
||||
class CallData::ResumeBatchCanceller { |
||||
public: |
||||
explicit ResumeBatchCanceller(grpc_call_element* elem) : elem_(elem) { |
||||
auto* calld = static_cast<CallData*>(elem->call_data); |
||||
GRPC_CALL_STACK_REF(calld->owning_call_, "ResumeBatchCanceller"); |
||||
GRPC_CLOSURE_INIT(&closure_, &Cancel, this, grpc_schedule_on_exec_ctx); |
||||
calld->call_combiner_->SetNotifyOnCancel(&closure_); |
||||
} |
||||
|
||||
private: |
||||
static void Cancel(void* arg, grpc_error* error) { |
||||
auto* self = static_cast<ResumeBatchCanceller*>(arg); |
||||
auto* chand = static_cast<ChannelData*>(self->elem_->channel_data); |
||||
auto* calld = static_cast<CallData*>(self->elem_->call_data); |
||||
MutexLock lock(&calld->delay_mu_); |
||||
if (GRPC_TRACE_FLAG_ENABLED(grpc_fault_injection_filter_trace)) { |
||||
gpr_log(GPR_INFO, |
||||
"chand=%p calld=%p: cancelling schdueled pick: " |
||||
"error=%s self=%p calld->resume_batch_canceller_=%p", |
||||
chand, calld, grpc_error_string(error), self, |
||||
calld->resume_batch_canceller_); |
||||
} |
||||
if (error != GRPC_ERROR_NONE && calld->resume_batch_canceller_ == self) { |
||||
// Cancel the delayed pick.
|
||||
calld->CancelDelayTimer(); |
||||
calld->FaultInjectionFinished(); |
||||
// Fail pending batches on the call.
|
||||
grpc_transport_stream_op_batch_finish_with_failure( |
||||
calld->delayed_batch_, GRPC_ERROR_REF(error), calld->call_combiner_); |
||||
} |
||||
GRPC_CALL_STACK_UNREF(calld->owning_call_, "ResumeBatchCanceller"); |
||||
delete self; |
||||
} |
||||
|
||||
grpc_call_element* elem_; |
||||
grpc_closure closure_; |
||||
}; |
||||
|
||||
// CallData
|
||||
|
||||
grpc_error* CallData::Init(grpc_call_element* elem, |
||||
const grpc_call_element_args* args) { |
||||
auto* calld = new (elem->call_data) CallData(elem, args); |
||||
if (calld->fi_policy_ == nullptr) { |
||||
return GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"failed to find fault injection policy"); |
||||
} |
||||
return GRPC_ERROR_NONE; |
||||
} |
||||
|
||||
void CallData::Destroy(grpc_call_element* elem, |
||||
const grpc_call_final_info* /*final_info*/, |
||||
grpc_closure* /*then_schedule_closure*/) { |
||||
auto* calld = static_cast<CallData*>(elem->call_data); |
||||
calld->~CallData(); |
||||
} |
||||
|
||||
void CallData::StartTransportStreamOpBatch( |
||||
grpc_call_element* elem, grpc_transport_stream_op_batch* batch) { |
||||
auto* calld = static_cast<CallData*>(elem->call_data); |
||||
// There should only be one send_initial_metdata op, and fault injection also
|
||||
// only need to be enforced once.
|
||||
if (batch->send_initial_metadata) { |
||||
calld->DecideWhetherToInjectFaults( |
||||
batch->payload->send_initial_metadata.send_initial_metadata); |
||||
if (GRPC_TRACE_FLAG_ENABLED(grpc_fault_injection_filter_trace)) { |
||||
gpr_log(GPR_INFO, |
||||
"chand=%p calld=%p: Fault injection triggered delay=%d abort=%d", |
||||
elem->channel_data, calld, calld->delay_request_, |
||||
calld->abort_request_); |
||||
} |
||||
if (calld->MaybeDelay()) { |
||||
// Delay the batch, and pass down the batch in the scheduled closure.
|
||||
calld->DelayBatch(elem, batch); |
||||
return; |
||||
} |
||||
grpc_error* abort_error = calld->MaybeAbort(); |
||||
if (abort_error != GRPC_ERROR_NONE) { |
||||
calld->abort_error_ = abort_error; |
||||
grpc_transport_stream_op_batch_finish_with_failure( |
||||
batch, GRPC_ERROR_REF(calld->abort_error_), calld->call_combiner_); |
||||
return; |
||||
} |
||||
} else { |
||||
if (batch->recv_trailing_metadata) { |
||||
// Intercept recv_trailing_metadata callback so that we can inject the
|
||||
// failure when aborting streaming calls, because their
|
||||
// recv_trailing_metatdata op may not be on the same batch as the
|
||||
// send_initial_metadata op.
|
||||
calld->original_recv_trailing_metadata_ready_ = |
||||
batch->payload->recv_trailing_metadata.recv_trailing_metadata_ready; |
||||
batch->payload->recv_trailing_metadata.recv_trailing_metadata_ready = |
||||
&calld->recv_trailing_metadata_ready_; |
||||
} |
||||
if (calld->abort_error_ != GRPC_ERROR_NONE) { |
||||
// If we already decided to abort, then immediately fail this batch.
|
||||
grpc_transport_stream_op_batch_finish_with_failure( |
||||
batch, GRPC_ERROR_REF(calld->abort_error_), calld->call_combiner_); |
||||
return; |
||||
} |
||||
} |
||||
// Chain to the next filter.
|
||||
grpc_call_next_op(elem, batch); |
||||
} |
||||
|
||||
CallData::CallData(grpc_call_element* elem, const grpc_call_element_args* args) |
||||
: owning_call_(args->call_stack), |
||||
arena_(args->arena), |
||||
call_combiner_(args->call_combiner) { |
||||
auto* chand = static_cast<ChannelData*>(elem->channel_data); |
||||
// Fetch the fault injection policy from the service config, based on the
|
||||
// relative index for which policy should this CallData use.
|
||||
auto* service_config_call_data = static_cast<ServiceConfigCallData*>( |
||||
args->context[GRPC_CONTEXT_SERVICE_CONFIG_CALL_DATA].value); |
||||
auto* method_params = static_cast<FaultInjectionMethodParsedConfig*>( |
||||
service_config_call_data->GetMethodParsedConfig( |
||||
FaultInjectionServiceConfigParser::ParserIndex())); |
||||
if (method_params != nullptr) { |
||||
fi_policy_ = method_params->fault_injection_policy(chand->index()); |
||||
} |
||||
GRPC_CLOSURE_INIT(&recv_trailing_metadata_ready_, |
||||
HijackedRecvTrailingMetadataReady, elem, |
||||
grpc_schedule_on_exec_ctx); |
||||
} |
||||
|
||||
CallData::~CallData() { |
||||
if (fi_policy_owned_) { |
||||
fi_policy_->~FaultInjectionPolicy(); |
||||
} |
||||
GRPC_ERROR_UNREF(abort_error_); |
||||
} |
||||
|
||||
void CallData::DecideWhetherToInjectFaults( |
||||
grpc_metadata_batch* initial_metadata) { |
||||
FaultInjectionMethodParsedConfig::FaultInjectionPolicy* copied_policy = |
||||
nullptr; |
||||
// Update the policy with values in initial metadata.
|
||||
if (!fi_policy_->abort_code_header.empty() || |
||||
!fi_policy_->abort_percentage_header.empty() || |
||||
!fi_policy_->delay_header.empty() || |
||||
!fi_policy_->delay_percentage_header.empty()) { |
||||
// Defer the actual copy until the first matched header.
|
||||
auto maybe_copy_policy_func = [this, &copied_policy]() { |
||||
if (copied_policy == nullptr) { |
||||
copied_policy = |
||||
arena_->New<FaultInjectionMethodParsedConfig::FaultInjectionPolicy>( |
||||
*fi_policy_); |
||||
} |
||||
}; |
||||
for (grpc_linked_mdelem* md = initial_metadata->list.head; md != nullptr; |
||||
md = md->next) { |
||||
absl::string_view key = StringViewFromSlice(GRPC_MDKEY(md->md)); |
||||
// Only perform string comparison if:
|
||||
// 1. Needs to check this header;
|
||||
// 2. The value is not been filled before.
|
||||
if (!fi_policy_->abort_code_header.empty() && |
||||
(copied_policy == nullptr || |
||||
copied_policy->abort_code == GRPC_STATUS_OK) && |
||||
key == fi_policy_->abort_code_header) { |
||||
maybe_copy_policy_func(); |
||||
grpc_status_code_from_int(GetLinkedMetadatumValueInt(md), |
||||
&copied_policy->abort_code); |
||||
} |
||||
if (!fi_policy_->abort_percentage_header.empty() && |
||||
key == fi_policy_->abort_percentage_header) { |
||||
maybe_copy_policy_func(); |
||||
copied_policy->abort_percentage_numerator = |
||||
GPR_MIN(GetLinkedMetadatumValueUnsignedInt(md), |
||||
fi_policy_->abort_percentage_numerator); |
||||
} |
||||
if (!fi_policy_->delay_header.empty() && |
||||
(copied_policy == nullptr || copied_policy->delay == 0) && |
||||
key == fi_policy_->delay_header) { |
||||
maybe_copy_policy_func(); |
||||
copied_policy->delay = static_cast<grpc_millis>( |
||||
GPR_MAX(GetLinkedMetadatumValueInt64(md), 0)); |
||||
} |
||||
if (!fi_policy_->delay_percentage_header.empty() && |
||||
key == fi_policy_->delay_percentage_header) { |
||||
maybe_copy_policy_func(); |
||||
copied_policy->delay_percentage_numerator = |
||||
GPR_MIN(GetLinkedMetadatumValueUnsignedInt(md), |
||||
fi_policy_->delay_percentage_numerator); |
||||
} |
||||
} |
||||
if (copied_policy != nullptr) fi_policy_ = copied_policy; |
||||
} |
||||
// Roll the dice
|
||||
delay_request_ = fi_policy_->delay != 0 && |
||||
UnderFraction(fi_policy_->delay_percentage_numerator, |
||||
fi_policy_->abort_percentage_denominator); |
||||
abort_request_ = fi_policy_->abort_code != GRPC_STATUS_OK && |
||||
UnderFraction(fi_policy_->abort_percentage_numerator, |
||||
fi_policy_->abort_percentage_denominator); |
||||
if (!delay_request_ && !abort_request_) { |
||||
if (copied_policy != nullptr) copied_policy->~FaultInjectionPolicy(); |
||||
// No fault injection for this call
|
||||
} else { |
||||
fi_policy_owned_ = copied_policy != nullptr; |
||||
} |
||||
} |
||||
|
||||
bool CallData::HaveActiveFaultsQuota(bool increment) { |
||||
if (g_active_faults.Load(MemoryOrder::ACQUIRE) >= fi_policy_->max_faults) { |
||||
return false; |
||||
} |
||||
if (increment) g_active_faults.FetchAdd(1, MemoryOrder::RELAXED); |
||||
return true; |
||||
} |
||||
|
||||
bool CallData::MaybeDelay() { |
||||
if (delay_request_) { |
||||
return HaveActiveFaultsQuota(true); |
||||
} |
||||
return false; |
||||
} |
||||
|
||||
grpc_error* CallData::MaybeAbort() { |
||||
if (abort_request_ && (delay_request_ || HaveActiveFaultsQuota(false))) { |
||||
return grpc_error_set_int( |
||||
GRPC_ERROR_CREATE_FROM_COPIED_STRING(fi_policy_->abort_message.c_str()), |
||||
GRPC_ERROR_INT_GRPC_STATUS, fi_policy_->abort_code); |
||||
} |
||||
return GRPC_ERROR_NONE; |
||||
} |
||||
|
||||
void CallData::DelayBatch(grpc_call_element* elem, |
||||
grpc_transport_stream_op_batch* batch) { |
||||
MutexLock lock(&delay_mu_); |
||||
delayed_batch_ = batch; |
||||
resume_batch_canceller_ = new ResumeBatchCanceller(elem); |
||||
grpc_millis resume_time = ExecCtx::Get()->Now() + fi_policy_->delay; |
||||
GRPC_CLOSURE_INIT(&batch->handler_private.closure, ResumeBatch, elem, |
||||
grpc_schedule_on_exec_ctx); |
||||
grpc_timer_init(&delay_timer_, resume_time, &batch->handler_private.closure); |
||||
} |
||||
|
||||
void CallData::ResumeBatch(void* arg, grpc_error* error) { |
||||
grpc_call_element* elem = static_cast<grpc_call_element*>(arg); |
||||
auto* calld = static_cast<CallData*>(elem->call_data); |
||||
MutexLock lock(&calld->delay_mu_); |
||||
// Cancelled or canceller has already run
|
||||
if (error == GRPC_ERROR_CANCELLED || |
||||
calld->resume_batch_canceller_ == nullptr) { |
||||
return; |
||||
} |
||||
if (GRPC_TRACE_FLAG_ENABLED(grpc_fault_injection_filter_trace)) { |
||||
gpr_log(GPR_INFO, "chand=%p calld=%p: Resuming delayed stream op batch %p", |
||||
elem->channel_data, calld, calld->delayed_batch_); |
||||
} |
||||
// Lame the canceller
|
||||
calld->resume_batch_canceller_ = nullptr; |
||||
// Finish fault injection.
|
||||
calld->FaultInjectionFinished(); |
||||
// Abort if needed.
|
||||
error = calld->MaybeAbort(); |
||||
if (error != GRPC_ERROR_NONE) { |
||||
grpc_transport_stream_op_batch_finish_with_failure( |
||||
calld->delayed_batch_, error, calld->call_combiner_); |
||||
return; |
||||
} |
||||
// Chain to the next filter.
|
||||
grpc_call_next_op(elem, calld->delayed_batch_); |
||||
} |
||||
|
||||
void CallData::HijackedRecvTrailingMetadataReady(void* arg, grpc_error* error) { |
||||
grpc_call_element* elem = static_cast<grpc_call_element*>(arg); |
||||
auto* calld = static_cast<CallData*>(elem->call_data); |
||||
if (calld->abort_error_ != GRPC_ERROR_NONE) { |
||||
error = grpc_error_add_child(GRPC_ERROR_REF(error), |
||||
GRPC_ERROR_REF(calld->abort_error_)); |
||||
Closure::Run(DEBUG_LOCATION, calld->original_recv_trailing_metadata_ready_, |
||||
error); |
||||
} |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
extern const grpc_channel_filter FaultInjectionFilterVtable = { |
||||
CallData::StartTransportStreamOpBatch, |
||||
grpc_channel_next_op, |
||||
sizeof(CallData), |
||||
CallData::Init, |
||||
grpc_call_stack_ignore_set_pollset_or_pollset_set, |
||||
CallData::Destroy, |
||||
sizeof(ChannelData), |
||||
ChannelData::Init, |
||||
ChannelData::Destroy, |
||||
grpc_channel_next_get_info, |
||||
"fault_injection_filter", |
||||
}; |
||||
|
||||
void FaultInjectionFilterInit(void) { |
||||
grpc_core::FaultInjectionServiceConfigParser::Register(); |
||||
} |
||||
|
||||
void FaultInjectionFilterShutdown(void) {} |
||||
|
||||
} // namespace grpc_core
|
@ -0,0 +1,39 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#ifndef GRPC_CORE_EXT_FILTERS_FAULT_INJECTION_FAULT_INJECTION_FILTER_H |
||||
#define GRPC_CORE_EXT_FILTERS_FAULT_INJECTION_FAULT_INJECTION_FILTER_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/ext/filters/fault_injection/service_config_parser.h" |
||||
#include "src/core/lib/channel/channel_stack.h" |
||||
|
||||
// Channel arg key for enabling parsing fault injection via method config.
|
||||
#define GRPC_ARG_PARSE_FAULT_INJECTION_METHOD_CONFIG \ |
||||
"grpc.parse_fault_injection_method_config" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
// This channel filter is intended to be used by the dynamic filters, instead
|
||||
// of the ordinary channel stack. The fault injection filter fetches fault
|
||||
// injection policy from the method config of service config returned by the
|
||||
// resolver, and enforces the fault injection policy.
|
||||
extern const grpc_channel_filter FaultInjectionFilterVtable; |
||||
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_CORE_EXT_FILTERS_FAULT_INJECTION_FAULT_INJECTION_FILTER_H
|
@ -0,0 +1,189 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/ext/filters/fault_injection/service_config_parser.h" |
||||
|
||||
#include <vector> |
||||
|
||||
#include "absl/strings/str_cat.h" |
||||
#include "absl/strings/string_view.h" |
||||
#include "src/core/ext/filters/client_channel/service_config.h" |
||||
#include "src/core/ext/filters/fault_injection/fault_injection_filter.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/channel/status_util.h" |
||||
#include "src/core/lib/gpr/string.h" |
||||
#include "src/core/lib/json/json_util.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
namespace { |
||||
|
||||
size_t g_fault_injection_parser_index; |
||||
|
||||
std::vector<FaultInjectionMethodParsedConfig::FaultInjectionPolicy> |
||||
ParseFaultInjectionPolicy(const Json::Array& policies_json_array, |
||||
std::vector<grpc_error*>* error_list) { |
||||
std::vector<FaultInjectionMethodParsedConfig::FaultInjectionPolicy> policies; |
||||
for (size_t i = 0; i < policies_json_array.size(); i++) { |
||||
FaultInjectionMethodParsedConfig::FaultInjectionPolicy |
||||
fault_injection_policy; |
||||
std::vector<grpc_error*> sub_error_list; |
||||
if (policies_json_array[i].type() != Json::Type::OBJECT) { |
||||
error_list->push_back(GRPC_ERROR_CREATE_FROM_COPIED_STRING( |
||||
absl::StrCat("faultInjectionPolicy index ", i, |
||||
" is not a JSON object") |
||||
.c_str())); |
||||
continue; |
||||
} |
||||
const Json::Object& json_object = policies_json_array[i].object_value(); |
||||
// Parse abort_code
|
||||
std::string abort_code_string; |
||||
if (ParseJsonObjectField(json_object, "abortCode", &abort_code_string, |
||||
&sub_error_list, false)) { |
||||
if (!grpc_status_code_from_string(abort_code_string.c_str(), |
||||
&(fault_injection_policy.abort_code))) { |
||||
sub_error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:abortCode error:failed to parse status code")); |
||||
} |
||||
} |
||||
// Parse abort_message
|
||||
if (!ParseJsonObjectField(json_object, "abortMessage", |
||||
&fault_injection_policy.abort_message, |
||||
&sub_error_list, false)) { |
||||
fault_injection_policy.abort_message = "Fault injected"; |
||||
} |
||||
// Parse abort_code_header
|
||||
ParseJsonObjectField(json_object, "abortCodeHeader", |
||||
&fault_injection_policy.abort_code_header, |
||||
&sub_error_list, false); |
||||
// Parse abort_percentage_header
|
||||
ParseJsonObjectField(json_object, "abortPercentageHeader", |
||||
&fault_injection_policy.abort_percentage_header, |
||||
&sub_error_list, false); |
||||
// Parse abort_percentage_numerator
|
||||
ParseJsonObjectField(json_object, "abortPercentageNumerator", |
||||
&fault_injection_policy.abort_percentage_numerator, |
||||
&sub_error_list, false); |
||||
// Parse abort_percentage_denominator
|
||||
if (ParseJsonObjectField( |
||||
json_object, "abortPercentageDenominator", |
||||
&fault_injection_policy.abort_percentage_denominator, |
||||
&sub_error_list, false)) { |
||||
if (fault_injection_policy.abort_percentage_denominator != 100 && |
||||
fault_injection_policy.abort_percentage_denominator != 10000 && |
||||
fault_injection_policy.abort_percentage_denominator != 1000000) { |
||||
sub_error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:abortPercentageDenominator error:Denominator can only be " |
||||
"one of " |
||||
"100, 10000, 1000000")); |
||||
} |
||||
} |
||||
// Parse delay
|
||||
ParseJsonObjectFieldAsDuration(json_object, "delay", |
||||
&fault_injection_policy.delay, |
||||
&sub_error_list, false); |
||||
// Parse delay_header
|
||||
ParseJsonObjectField(json_object, "delayHeader", |
||||
&fault_injection_policy.delay_header, &sub_error_list, |
||||
false); |
||||
// Parse delay_percentage_header
|
||||
ParseJsonObjectField(json_object, "delayPercentageHeader", |
||||
&fault_injection_policy.delay_percentage_header, |
||||
&sub_error_list, false); |
||||
// Parse delay_percentage_numerator
|
||||
ParseJsonObjectField(json_object, "delayPercentageNumerator", |
||||
&fault_injection_policy.delay_percentage_numerator, |
||||
&sub_error_list, false); |
||||
// Parse delay_percentage_denominator
|
||||
if (ParseJsonObjectField( |
||||
json_object, "delayPercentageDenominator", |
||||
&fault_injection_policy.delay_percentage_denominator, |
||||
&sub_error_list, false)) { |
||||
if (fault_injection_policy.delay_percentage_denominator != 100 && |
||||
fault_injection_policy.delay_percentage_denominator != 10000 && |
||||
fault_injection_policy.delay_percentage_denominator != 1000000) { |
||||
sub_error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:delayPercentageDenominator error:Denominator can only be " |
||||
"one of " |
||||
"100, 10000, 1000000")); |
||||
} |
||||
} |
||||
// Parse max_faults
|
||||
if (ParseJsonObjectField(json_object, "maxFaults", |
||||
&fault_injection_policy.max_faults, |
||||
&sub_error_list, false)) { |
||||
if (fault_injection_policy.max_faults < 0) { |
||||
sub_error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING( |
||||
"field:maxFaults error:should be zero or positive")); |
||||
} |
||||
} |
||||
if (!sub_error_list.empty()) { |
||||
// Can't use GRPC_ERROR_CREATE_FROM_VECTOR() here, because the error
|
||||
// string is not static in this case.
|
||||
grpc_error* error = GRPC_ERROR_CREATE_FROM_COPIED_STRING( |
||||
absl::StrCat("failed to parse faultInjectionPolicy index ", i) |
||||
.c_str()); |
||||
for (size_t i = 0; i < sub_error_list.size(); ++i) { |
||||
error = grpc_error_add_child(error, sub_error_list[i]); |
||||
} |
||||
error_list->push_back(error); |
||||
} |
||||
policies.push_back(std::move(fault_injection_policy)); |
||||
} |
||||
return policies; |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
std::unique_ptr<ServiceConfigParser::ParsedConfig> |
||||
FaultInjectionServiceConfigParser::ParsePerMethodParams( |
||||
const grpc_channel_args* args, const Json& json, grpc_error** error) { |
||||
GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE); |
||||
// Only parse fault injection policy if the following channel arg is present.
|
||||
if (!grpc_channel_args_find_bool( |
||||
args, GRPC_ARG_PARSE_FAULT_INJECTION_METHOD_CONFIG, false)) { |
||||
return nullptr; |
||||
} |
||||
// Parse fault injection policy from given Json
|
||||
std::vector<FaultInjectionMethodParsedConfig::FaultInjectionPolicy> |
||||
fault_injection_policies; |
||||
std::vector<grpc_error*> error_list; |
||||
const Json::Array* policies_json_array; |
||||
if (ParseJsonObjectField(json.object_value(), "faultInjectionPolicy", |
||||
&policies_json_array, &error_list)) { |
||||
fault_injection_policies = |
||||
ParseFaultInjectionPolicy(*policies_json_array, &error_list); |
||||
} |
||||
*error = GRPC_ERROR_CREATE_FROM_VECTOR("Fault injection parser", &error_list); |
||||
if (*error != GRPC_ERROR_NONE || fault_injection_policies.empty()) { |
||||
return nullptr; |
||||
} |
||||
return absl::make_unique<FaultInjectionMethodParsedConfig>( |
||||
std::move(fault_injection_policies)); |
||||
} |
||||
|
||||
void FaultInjectionServiceConfigParser::Register() { |
||||
g_fault_injection_parser_index = ServiceConfigParser::RegisterParser( |
||||
absl::make_unique<FaultInjectionServiceConfigParser>()); |
||||
} |
||||
|
||||
size_t FaultInjectionServiceConfigParser::ParserIndex() { |
||||
return g_fault_injection_parser_index; |
||||
} |
||||
|
||||
} // namespace grpc_core
|
@ -0,0 +1,85 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#ifndef GRPC_CORE_EXT_FILTERS_FAULT_INJECTION_SERVICE_CONFIG_PARSER_H |
||||
#define GRPC_CORE_EXT_FILTERS_FAULT_INJECTION_SERVICE_CONFIG_PARSER_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <vector> |
||||
|
||||
#include "src/core/ext/filters/client_channel/service_config.h" |
||||
#include "src/core/lib/iomgr/exec_ctx.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
class FaultInjectionMethodParsedConfig |
||||
: public ServiceConfigParser::ParsedConfig { |
||||
public: |
||||
struct FaultInjectionPolicy { |
||||
grpc_status_code abort_code = GRPC_STATUS_OK; |
||||
std::string abort_message; |
||||
std::string abort_code_header; |
||||
std::string abort_percentage_header; |
||||
uint32_t abort_percentage_numerator = 0; |
||||
uint32_t abort_percentage_denominator = 100; |
||||
|
||||
grpc_millis delay = 0; |
||||
std::string delay_header; |
||||
std::string delay_percentage_header; |
||||
uint32_t delay_percentage_numerator = 0; |
||||
uint32_t delay_percentage_denominator = 100; |
||||
|
||||
// By default, the max allowed active faults are unlimited.
|
||||
uint32_t max_faults = std::numeric_limits<uint32_t>::max(); |
||||
}; |
||||
|
||||
explicit FaultInjectionMethodParsedConfig( |
||||
std::vector<FaultInjectionPolicy> fault_injection_policies) |
||||
: fault_injection_policies_(std::move(fault_injection_policies)) {} |
||||
|
||||
// Returns the fault injection policy at certain index.
|
||||
// There might be multiple fault injection policies functioning at the same
|
||||
// time. The order between the policies are stable, and an index is used to
|
||||
// keep track of their relative positions. The FaultInjectionFilter uses this
|
||||
// method to access the parsed fault injection policy in service config,
|
||||
// whether it came from xDS resolver or directly from service config
|
||||
const FaultInjectionPolicy* fault_injection_policy(int index) const { |
||||
if (static_cast<size_t>(index) >= fault_injection_policies_.size()) { |
||||
return nullptr; |
||||
} |
||||
return &fault_injection_policies_[index]; |
||||
} |
||||
|
||||
private: |
||||
std::vector<FaultInjectionPolicy> fault_injection_policies_; |
||||
}; |
||||
|
||||
class FaultInjectionServiceConfigParser : public ServiceConfigParser::Parser { |
||||
public: |
||||
// Parses the per-method service config for fault injection filter.
|
||||
std::unique_ptr<ServiceConfigParser::ParsedConfig> ParsePerMethodParams( |
||||
const grpc_channel_args* args, const Json& json, |
||||
grpc_error** error) override; |
||||
// Returns the parser index for FaultInjectionServiceConfigParser.
|
||||
static size_t ParserIndex(); |
||||
// Registers FaultInjectionServiceConfigParser to ServiceConfigParser.
|
||||
static void Register(); |
||||
}; |
||||
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif // GRPC_CORE_EXT_FILTERS_FAULT_INJECTION_SERVICE_CONFIG_PARSER_H
|
@ -0,0 +1,79 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/common/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#include <stddef.h> |
||||
#include "upb/msg.h" |
||||
#include "envoy/extensions/filters/common/fault/v3/fault.upb.h" |
||||
#include "envoy/type/v3/percent.upb.h" |
||||
#include "google/protobuf/duration.upb.h" |
||||
#include "udpa/annotations/status.upb.h" |
||||
#include "udpa/annotations/versioning.upb.h" |
||||
#include "validate/validate.upb.h" |
||||
|
||||
#include "upb/port_def.inc" |
||||
|
||||
static const upb_msglayout *const envoy_extensions_filters_common_fault_v3_FaultDelay_submsgs[3] = { |
||||
&envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, |
||||
&envoy_type_v3_FractionalPercent_msginit, |
||||
&google_protobuf_Duration_msginit, |
||||
}; |
||||
|
||||
static const upb_msglayout_field envoy_extensions_filters_common_fault_v3_FaultDelay__fields[3] = { |
||||
{3, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 2, 11, 1}, |
||||
{4, UPB_SIZE(4, 8), 1, 1, 11, 1}, |
||||
{5, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 0, 11, 1}, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_msginit = { |
||||
&envoy_extensions_filters_common_fault_v3_FaultDelay_submsgs[0], |
||||
&envoy_extensions_filters_common_fault_v3_FaultDelay__fields[0], |
||||
UPB_SIZE(16, 32), 3, false, 255, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit = { |
||||
NULL, |
||||
NULL, |
||||
UPB_SIZE(0, 0), 0, false, 255, |
||||
}; |
||||
|
||||
static const upb_msglayout *const envoy_extensions_filters_common_fault_v3_FaultRateLimit_submsgs[3] = { |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, |
||||
&envoy_type_v3_FractionalPercent_msginit, |
||||
}; |
||||
|
||||
static const upb_msglayout_field envoy_extensions_filters_common_fault_v3_FaultRateLimit__fields[3] = { |
||||
{1, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 0, 11, 1}, |
||||
{2, UPB_SIZE(4, 8), 1, 2, 11, 1}, |
||||
{3, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 1, 11, 1}, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit = { |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_submsgs[0], |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit__fields[0], |
||||
UPB_SIZE(16, 32), 3, false, 255, |
||||
}; |
||||
|
||||
static const upb_msglayout_field envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit__fields[1] = { |
||||
{1, UPB_SIZE(0, 0), 0, 0, 4, 1}, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit = { |
||||
NULL, |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit__fields[0], |
||||
UPB_SIZE(8, 8), 1, false, 255, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit = { |
||||
NULL, |
||||
NULL, |
||||
UPB_SIZE(0, 0), 0, false, 255, |
||||
}; |
||||
|
||||
#include "upb/port_undef.inc" |
||||
|
@ -0,0 +1,268 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/common/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#ifndef ENVOY_EXTENSIONS_FILTERS_COMMON_FAULT_V3_FAULT_PROTO_UPB_H_ |
||||
#define ENVOY_EXTENSIONS_FILTERS_COMMON_FAULT_V3_FAULT_PROTO_UPB_H_ |
||||
|
||||
#include "upb/msg.h" |
||||
#include "upb/decode.h" |
||||
#include "upb/decode_fast.h" |
||||
#include "upb/encode.h" |
||||
|
||||
#include "upb/port_def.inc" |
||||
|
||||
#ifdef __cplusplus |
||||
extern "C" { |
||||
#endif |
||||
|
||||
struct envoy_extensions_filters_common_fault_v3_FaultDelay; |
||||
struct envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay; |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit; |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit; |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit; |
||||
typedef struct envoy_extensions_filters_common_fault_v3_FaultDelay envoy_extensions_filters_common_fault_v3_FaultDelay; |
||||
typedef struct envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay; |
||||
typedef struct envoy_extensions_filters_common_fault_v3_FaultRateLimit envoy_extensions_filters_common_fault_v3_FaultRateLimit; |
||||
typedef struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit; |
||||
typedef struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit; |
||||
struct envoy_type_v3_FractionalPercent; |
||||
struct google_protobuf_Duration; |
||||
extern const upb_msglayout envoy_type_v3_FractionalPercent_msginit; |
||||
extern const upb_msglayout google_protobuf_Duration_msginit; |
||||
|
||||
typedef enum { |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_FIXED = 0 |
||||
} envoy_extensions_filters_common_fault_v3_FaultDelay_FaultDelayType; |
||||
|
||||
|
||||
/* envoy.extensions.filters.common.fault.v3.FaultDelay */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay *envoy_extensions_filters_common_fault_v3_FaultDelay_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_common_fault_v3_FaultDelay *)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay *envoy_extensions_filters_common_fault_v3_FaultDelay_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay *ret = envoy_extensions_filters_common_fault_v3_FaultDelay_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay *envoy_extensions_filters_common_fault_v3_FaultDelay_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay *ret = envoy_extensions_filters_common_fault_v3_FaultDelay_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_common_fault_v3_FaultDelay_serialize(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, arena, len); |
||||
} |
||||
|
||||
typedef enum { |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_fixed_delay = 3, |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_header_delay = 5, |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_NOT_SET = 0 |
||||
} envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_oneofcases; |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_oneofcases envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_case(const envoy_extensions_filters_common_fault_v3_FaultDelay* msg) { return (envoy_extensions_filters_common_fault_v3_FaultDelay_fault_delay_secifier_oneofcases)*UPB_PTR_AT(msg, UPB_SIZE(12, 24), int32_t); } |
||||
|
||||
UPB_INLINE bool envoy_extensions_filters_common_fault_v3_FaultDelay_has_fixed_delay(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 3; } |
||||
UPB_INLINE const struct google_protobuf_Duration* envoy_extensions_filters_common_fault_v3_FaultDelay_fixed_delay(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg) { return UPB_READ_ONEOF(msg, const struct google_protobuf_Duration*, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 3, NULL); } |
||||
UPB_INLINE bool envoy_extensions_filters_common_fault_v3_FaultDelay_has_percentage(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg) { return _upb_hasbit(msg, 1); } |
||||
UPB_INLINE const struct envoy_type_v3_FractionalPercent* envoy_extensions_filters_common_fault_v3_FaultDelay_percentage(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(4, 8), const struct envoy_type_v3_FractionalPercent*); } |
||||
UPB_INLINE bool envoy_extensions_filters_common_fault_v3_FaultDelay_has_header_delay(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 5; } |
||||
UPB_INLINE const envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay* envoy_extensions_filters_common_fault_v3_FaultDelay_header_delay(const envoy_extensions_filters_common_fault_v3_FaultDelay *msg) { return UPB_READ_ONEOF(msg, const envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay*, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 5, NULL); } |
||||
|
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultDelay_set_fixed_delay(envoy_extensions_filters_common_fault_v3_FaultDelay *msg, struct google_protobuf_Duration* value) { |
||||
UPB_WRITE_ONEOF(msg, struct google_protobuf_Duration*, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 3); |
||||
} |
||||
UPB_INLINE struct google_protobuf_Duration* envoy_extensions_filters_common_fault_v3_FaultDelay_mutable_fixed_delay(envoy_extensions_filters_common_fault_v3_FaultDelay *msg, upb_arena *arena) { |
||||
struct google_protobuf_Duration* sub = (struct google_protobuf_Duration*)envoy_extensions_filters_common_fault_v3_FaultDelay_fixed_delay(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct google_protobuf_Duration*)_upb_msg_new(&google_protobuf_Duration_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_set_fixed_delay(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultDelay_set_percentage(envoy_extensions_filters_common_fault_v3_FaultDelay *msg, struct envoy_type_v3_FractionalPercent* value) { |
||||
_upb_sethas(msg, 1); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(4, 8), struct envoy_type_v3_FractionalPercent*) = value; |
||||
} |
||||
UPB_INLINE struct envoy_type_v3_FractionalPercent* envoy_extensions_filters_common_fault_v3_FaultDelay_mutable_percentage(envoy_extensions_filters_common_fault_v3_FaultDelay *msg, upb_arena *arena) { |
||||
struct envoy_type_v3_FractionalPercent* sub = (struct envoy_type_v3_FractionalPercent*)envoy_extensions_filters_common_fault_v3_FaultDelay_percentage(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_type_v3_FractionalPercent*)_upb_msg_new(&envoy_type_v3_FractionalPercent_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_set_percentage(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultDelay_set_header_delay(envoy_extensions_filters_common_fault_v3_FaultDelay *msg, envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay* value) { |
||||
UPB_WRITE_ONEOF(msg, envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay*, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 5); |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay* envoy_extensions_filters_common_fault_v3_FaultDelay_mutable_header_delay(envoy_extensions_filters_common_fault_v3_FaultDelay *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay* sub = (struct envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay*)envoy_extensions_filters_common_fault_v3_FaultDelay_header_delay(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay*)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_set_header_delay(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
|
||||
/* envoy.extensions.filters.common.fault.v3.FaultDelay.HeaderDelay */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *ret = envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *ret = envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_serialize(const envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, arena, len); |
||||
} |
||||
|
||||
|
||||
|
||||
/* envoy.extensions.filters.common.fault.v3.FaultRateLimit */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_common_fault_v3_FaultRateLimit *)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit *ret = envoy_extensions_filters_common_fault_v3_FaultRateLimit_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit *ret = envoy_extensions_filters_common_fault_v3_FaultRateLimit_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_common_fault_v3_FaultRateLimit_serialize(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, arena, len); |
||||
} |
||||
|
||||
typedef enum { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_fixed_limit = 1, |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_header_limit = 3, |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_NOT_SET = 0 |
||||
} envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_oneofcases; |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_oneofcases envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_case(const envoy_extensions_filters_common_fault_v3_FaultRateLimit* msg) { return (envoy_extensions_filters_common_fault_v3_FaultRateLimit_limit_type_oneofcases)*UPB_PTR_AT(msg, UPB_SIZE(12, 24), int32_t); } |
||||
|
||||
UPB_INLINE bool envoy_extensions_filters_common_fault_v3_FaultRateLimit_has_fixed_limit(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 1; } |
||||
UPB_INLINE const envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit* envoy_extensions_filters_common_fault_v3_FaultRateLimit_fixed_limit(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg) { return UPB_READ_ONEOF(msg, const envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit*, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 1, NULL); } |
||||
UPB_INLINE bool envoy_extensions_filters_common_fault_v3_FaultRateLimit_has_percentage(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg) { return _upb_hasbit(msg, 1); } |
||||
UPB_INLINE const struct envoy_type_v3_FractionalPercent* envoy_extensions_filters_common_fault_v3_FaultRateLimit_percentage(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(4, 8), const struct envoy_type_v3_FractionalPercent*); } |
||||
UPB_INLINE bool envoy_extensions_filters_common_fault_v3_FaultRateLimit_has_header_limit(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 3; } |
||||
UPB_INLINE const envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit* envoy_extensions_filters_common_fault_v3_FaultRateLimit_header_limit(const envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg) { return UPB_READ_ONEOF(msg, const envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit*, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 3, NULL); } |
||||
|
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultRateLimit_set_fixed_limit(envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit* value) { |
||||
UPB_WRITE_ONEOF(msg, envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit*, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 1); |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit* envoy_extensions_filters_common_fault_v3_FaultRateLimit_mutable_fixed_limit(envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit* sub = (struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit*)envoy_extensions_filters_common_fault_v3_FaultRateLimit_fixed_limit(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit*)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_set_fixed_limit(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultRateLimit_set_percentage(envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, struct envoy_type_v3_FractionalPercent* value) { |
||||
_upb_sethas(msg, 1); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(4, 8), struct envoy_type_v3_FractionalPercent*) = value; |
||||
} |
||||
UPB_INLINE struct envoy_type_v3_FractionalPercent* envoy_extensions_filters_common_fault_v3_FaultRateLimit_mutable_percentage(envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, upb_arena *arena) { |
||||
struct envoy_type_v3_FractionalPercent* sub = (struct envoy_type_v3_FractionalPercent*)envoy_extensions_filters_common_fault_v3_FaultRateLimit_percentage(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_type_v3_FractionalPercent*)_upb_msg_new(&envoy_type_v3_FractionalPercent_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_set_percentage(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultRateLimit_set_header_limit(envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit* value) { |
||||
UPB_WRITE_ONEOF(msg, envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit*, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 3); |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit* envoy_extensions_filters_common_fault_v3_FaultRateLimit_mutable_header_limit(envoy_extensions_filters_common_fault_v3_FaultRateLimit *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit* sub = (struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit*)envoy_extensions_filters_common_fault_v3_FaultRateLimit_header_limit(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit*)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_set_header_limit(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
|
||||
/* envoy.extensions.filters.common.fault.v3.FaultRateLimit.FixedLimit */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *ret = envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *ret = envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_serialize(const envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, arena, len); |
||||
} |
||||
|
||||
UPB_INLINE uint64_t envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_limit_kbps(const envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(0, 0), uint64_t); } |
||||
|
||||
UPB_INLINE void envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_set_limit_kbps(envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit *msg, uint64_t value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(0, 0), uint64_t) = value; |
||||
} |
||||
|
||||
/* envoy.extensions.filters.common.fault.v3.FaultRateLimit.HeaderLimit */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *ret = envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *ret = envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_serialize(const envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, arena, len); |
||||
} |
||||
|
||||
|
||||
|
||||
#ifdef __cplusplus |
||||
} /* extern "C" */ |
||||
#endif |
||||
|
||||
#include "upb/port_undef.inc" |
||||
|
||||
#endif /* ENVOY_EXTENSIONS_FILTERS_COMMON_FAULT_V3_FAULT_PROTO_UPB_H_ */ |
@ -0,0 +1,78 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/http/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#include <stddef.h> |
||||
#include "upb/msg.h" |
||||
#include "envoy/extensions/filters/http/fault/v3/fault.upb.h" |
||||
#include "envoy/config/route/v3/route_components.upb.h" |
||||
#include "envoy/extensions/filters/common/fault/v3/fault.upb.h" |
||||
#include "envoy/type/v3/percent.upb.h" |
||||
#include "google/protobuf/wrappers.upb.h" |
||||
#include "udpa/annotations/status.upb.h" |
||||
#include "udpa/annotations/versioning.upb.h" |
||||
#include "validate/validate.upb.h" |
||||
|
||||
#include "upb/port_def.inc" |
||||
|
||||
static const upb_msglayout *const envoy_extensions_filters_http_fault_v3_FaultAbort_submsgs[2] = { |
||||
&envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, |
||||
&envoy_type_v3_FractionalPercent_msginit, |
||||
}; |
||||
|
||||
static const upb_msglayout_field envoy_extensions_filters_http_fault_v3_FaultAbort__fields[4] = { |
||||
{2, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 0, 13, 1}, |
||||
{3, UPB_SIZE(4, 8), 1, 1, 11, 1}, |
||||
{4, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 0, 11, 1}, |
||||
{5, UPB_SIZE(8, 16), UPB_SIZE(-13, -25), 0, 13, 1}, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_http_fault_v3_FaultAbort_msginit = { |
||||
&envoy_extensions_filters_http_fault_v3_FaultAbort_submsgs[0], |
||||
&envoy_extensions_filters_http_fault_v3_FaultAbort__fields[0], |
||||
UPB_SIZE(16, 32), 4, false, 255, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit = { |
||||
NULL, |
||||
NULL, |
||||
UPB_SIZE(0, 0), 0, false, 255, |
||||
}; |
||||
|
||||
static const upb_msglayout *const envoy_extensions_filters_http_fault_v3_HTTPFault_submsgs[5] = { |
||||
&envoy_config_route_v3_HeaderMatcher_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, |
||||
&envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, |
||||
&google_protobuf_UInt32Value_msginit, |
||||
}; |
||||
|
||||
static const upb_msglayout_field envoy_extensions_filters_http_fault_v3_HTTPFault__fields[14] = { |
||||
{1, UPB_SIZE(68, 136), 1, 1, 11, 1}, |
||||
{2, UPB_SIZE(72, 144), 2, 3, 11, 1}, |
||||
{3, UPB_SIZE(4, 8), 0, 0, 9, 1}, |
||||
{4, UPB_SIZE(84, 168), 0, 0, 11, 3}, |
||||
{5, UPB_SIZE(88, 176), 0, 0, 9, 3}, |
||||
{6, UPB_SIZE(76, 152), 3, 4, 11, 1}, |
||||
{7, UPB_SIZE(80, 160), 4, 2, 11, 1}, |
||||
{8, UPB_SIZE(12, 24), 0, 0, 9, 1}, |
||||
{9, UPB_SIZE(20, 40), 0, 0, 9, 1}, |
||||
{10, UPB_SIZE(28, 56), 0, 0, 9, 1}, |
||||
{11, UPB_SIZE(36, 72), 0, 0, 9, 1}, |
||||
{12, UPB_SIZE(44, 88), 0, 0, 9, 1}, |
||||
{13, UPB_SIZE(52, 104), 0, 0, 9, 1}, |
||||
{14, UPB_SIZE(60, 120), 0, 0, 9, 1}, |
||||
}; |
||||
|
||||
const upb_msglayout envoy_extensions_filters_http_fault_v3_HTTPFault_msginit = { |
||||
&envoy_extensions_filters_http_fault_v3_HTTPFault_submsgs[0], |
||||
&envoy_extensions_filters_http_fault_v3_HTTPFault__fields[0], |
||||
UPB_SIZE(96, 192), 14, false, 255, |
||||
}; |
||||
|
||||
#include "upb/port_undef.inc" |
||||
|
@ -0,0 +1,281 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/http/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#ifndef ENVOY_EXTENSIONS_FILTERS_HTTP_FAULT_V3_FAULT_PROTO_UPB_H_ |
||||
#define ENVOY_EXTENSIONS_FILTERS_HTTP_FAULT_V3_FAULT_PROTO_UPB_H_ |
||||
|
||||
#include "upb/msg.h" |
||||
#include "upb/decode.h" |
||||
#include "upb/decode_fast.h" |
||||
#include "upb/encode.h" |
||||
|
||||
#include "upb/port_def.inc" |
||||
|
||||
#ifdef __cplusplus |
||||
extern "C" { |
||||
#endif |
||||
|
||||
struct envoy_extensions_filters_http_fault_v3_FaultAbort; |
||||
struct envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort; |
||||
struct envoy_extensions_filters_http_fault_v3_HTTPFault; |
||||
typedef struct envoy_extensions_filters_http_fault_v3_FaultAbort envoy_extensions_filters_http_fault_v3_FaultAbort; |
||||
typedef struct envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort; |
||||
typedef struct envoy_extensions_filters_http_fault_v3_HTTPFault envoy_extensions_filters_http_fault_v3_HTTPFault; |
||||
extern const upb_msglayout envoy_extensions_filters_http_fault_v3_FaultAbort_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_http_fault_v3_HTTPFault_msginit; |
||||
struct envoy_config_route_v3_HeaderMatcher; |
||||
struct envoy_extensions_filters_common_fault_v3_FaultDelay; |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit; |
||||
struct envoy_type_v3_FractionalPercent; |
||||
struct google_protobuf_UInt32Value; |
||||
extern const upb_msglayout envoy_config_route_v3_HeaderMatcher_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit; |
||||
extern const upb_msglayout envoy_type_v3_FractionalPercent_msginit; |
||||
extern const upb_msglayout google_protobuf_UInt32Value_msginit; |
||||
|
||||
|
||||
/* envoy.extensions.filters.http.fault.v3.FaultAbort */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort *envoy_extensions_filters_http_fault_v3_FaultAbort_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_http_fault_v3_FaultAbort *)_upb_msg_new(&envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort *envoy_extensions_filters_http_fault_v3_FaultAbort_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort *ret = envoy_extensions_filters_http_fault_v3_FaultAbort_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort *envoy_extensions_filters_http_fault_v3_FaultAbort_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort *ret = envoy_extensions_filters_http_fault_v3_FaultAbort_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_http_fault_v3_FaultAbort_serialize(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, arena, len); |
||||
} |
||||
|
||||
typedef enum { |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_http_status = 2, |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_grpc_status = 5, |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_header_abort = 4, |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_NOT_SET = 0 |
||||
} envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_oneofcases; |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_oneofcases envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_case(const envoy_extensions_filters_http_fault_v3_FaultAbort* msg) { return (envoy_extensions_filters_http_fault_v3_FaultAbort_error_type_oneofcases)*UPB_PTR_AT(msg, UPB_SIZE(12, 24), int32_t); } |
||||
|
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_FaultAbort_has_http_status(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 2; } |
||||
UPB_INLINE uint32_t envoy_extensions_filters_http_fault_v3_FaultAbort_http_status(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return UPB_READ_ONEOF(msg, uint32_t, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 2, 0); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_FaultAbort_has_percentage(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return _upb_hasbit(msg, 1); } |
||||
UPB_INLINE const struct envoy_type_v3_FractionalPercent* envoy_extensions_filters_http_fault_v3_FaultAbort_percentage(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(4, 8), const struct envoy_type_v3_FractionalPercent*); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_FaultAbort_has_header_abort(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 4; } |
||||
UPB_INLINE const envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort* envoy_extensions_filters_http_fault_v3_FaultAbort_header_abort(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return UPB_READ_ONEOF(msg, const envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort*, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 4, NULL); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_FaultAbort_has_grpc_status(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return _upb_getoneofcase(msg, UPB_SIZE(12, 24)) == 5; } |
||||
UPB_INLINE uint32_t envoy_extensions_filters_http_fault_v3_FaultAbort_grpc_status(const envoy_extensions_filters_http_fault_v3_FaultAbort *msg) { return UPB_READ_ONEOF(msg, uint32_t, UPB_SIZE(8, 16), UPB_SIZE(12, 24), 5, 0); } |
||||
|
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_FaultAbort_set_http_status(envoy_extensions_filters_http_fault_v3_FaultAbort *msg, uint32_t value) { |
||||
UPB_WRITE_ONEOF(msg, uint32_t, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 2); |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_FaultAbort_set_percentage(envoy_extensions_filters_http_fault_v3_FaultAbort *msg, struct envoy_type_v3_FractionalPercent* value) { |
||||
_upb_sethas(msg, 1); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(4, 8), struct envoy_type_v3_FractionalPercent*) = value; |
||||
} |
||||
UPB_INLINE struct envoy_type_v3_FractionalPercent* envoy_extensions_filters_http_fault_v3_FaultAbort_mutable_percentage(envoy_extensions_filters_http_fault_v3_FaultAbort *msg, upb_arena *arena) { |
||||
struct envoy_type_v3_FractionalPercent* sub = (struct envoy_type_v3_FractionalPercent*)envoy_extensions_filters_http_fault_v3_FaultAbort_percentage(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_type_v3_FractionalPercent*)_upb_msg_new(&envoy_type_v3_FractionalPercent_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_set_percentage(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_FaultAbort_set_header_abort(envoy_extensions_filters_http_fault_v3_FaultAbort *msg, envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort* value) { |
||||
UPB_WRITE_ONEOF(msg, envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort*, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 4); |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort* envoy_extensions_filters_http_fault_v3_FaultAbort_mutable_header_abort(envoy_extensions_filters_http_fault_v3_FaultAbort *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort* sub = (struct envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort*)envoy_extensions_filters_http_fault_v3_FaultAbort_header_abort(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort*)_upb_msg_new(&envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_set_header_abort(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_FaultAbort_set_grpc_status(envoy_extensions_filters_http_fault_v3_FaultAbort *msg, uint32_t value) { |
||||
UPB_WRITE_ONEOF(msg, uint32_t, UPB_SIZE(8, 16), value, UPB_SIZE(12, 24), 5); |
||||
} |
||||
|
||||
/* envoy.extensions.filters.http.fault.v3.FaultAbort.HeaderAbort */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *)_upb_msg_new(&envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *ret = envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *ret = envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_serialize(const envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, arena, len); |
||||
} |
||||
|
||||
|
||||
|
||||
/* envoy.extensions.filters.http.fault.v3.HTTPFault */ |
||||
|
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_HTTPFault *envoy_extensions_filters_http_fault_v3_HTTPFault_new(upb_arena *arena) { |
||||
return (envoy_extensions_filters_http_fault_v3_HTTPFault *)_upb_msg_new(&envoy_extensions_filters_http_fault_v3_HTTPFault_msginit, arena); |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_HTTPFault *envoy_extensions_filters_http_fault_v3_HTTPFault_parse(const char *buf, size_t size, |
||||
upb_arena *arena) { |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault *ret = envoy_extensions_filters_http_fault_v3_HTTPFault_new(arena); |
||||
return (ret && upb_decode(buf, size, ret, &envoy_extensions_filters_http_fault_v3_HTTPFault_msginit, arena)) ? ret : NULL; |
||||
} |
||||
UPB_INLINE envoy_extensions_filters_http_fault_v3_HTTPFault *envoy_extensions_filters_http_fault_v3_HTTPFault_parse_ex(const char *buf, size_t size, |
||||
upb_arena *arena, int options) { |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault *ret = envoy_extensions_filters_http_fault_v3_HTTPFault_new(arena); |
||||
return (ret && _upb_decode(buf, size, ret, &envoy_extensions_filters_http_fault_v3_HTTPFault_msginit, arena, options)) |
||||
? ret : NULL; |
||||
} |
||||
UPB_INLINE char *envoy_extensions_filters_http_fault_v3_HTTPFault_serialize(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_arena *arena, size_t *len) { |
||||
return upb_encode(msg, &envoy_extensions_filters_http_fault_v3_HTTPFault_msginit, arena, len); |
||||
} |
||||
|
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_HTTPFault_has_delay(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return _upb_hasbit(msg, 1); } |
||||
UPB_INLINE const struct envoy_extensions_filters_common_fault_v3_FaultDelay* envoy_extensions_filters_http_fault_v3_HTTPFault_delay(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(68, 136), const struct envoy_extensions_filters_common_fault_v3_FaultDelay*); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_HTTPFault_has_abort(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return _upb_hasbit(msg, 2); } |
||||
UPB_INLINE const envoy_extensions_filters_http_fault_v3_FaultAbort* envoy_extensions_filters_http_fault_v3_HTTPFault_abort(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(72, 144), const envoy_extensions_filters_http_fault_v3_FaultAbort*); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_upstream_cluster(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(4, 8), upb_strview); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_HTTPFault_has_headers(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return _upb_has_submsg_nohasbit(msg, UPB_SIZE(84, 168)); } |
||||
UPB_INLINE const struct envoy_config_route_v3_HeaderMatcher* const* envoy_extensions_filters_http_fault_v3_HTTPFault_headers(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg, size_t *len) { return (const struct envoy_config_route_v3_HeaderMatcher* const*)_upb_array_accessor(msg, UPB_SIZE(84, 168), len); } |
||||
UPB_INLINE upb_strview const* envoy_extensions_filters_http_fault_v3_HTTPFault_downstream_nodes(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg, size_t *len) { return (upb_strview const*)_upb_array_accessor(msg, UPB_SIZE(88, 176), len); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_HTTPFault_has_max_active_faults(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return _upb_hasbit(msg, 3); } |
||||
UPB_INLINE const struct google_protobuf_UInt32Value* envoy_extensions_filters_http_fault_v3_HTTPFault_max_active_faults(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(76, 152), const struct google_protobuf_UInt32Value*); } |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_HTTPFault_has_response_rate_limit(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return _upb_hasbit(msg, 4); } |
||||
UPB_INLINE const struct envoy_extensions_filters_common_fault_v3_FaultRateLimit* envoy_extensions_filters_http_fault_v3_HTTPFault_response_rate_limit(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(80, 160), const struct envoy_extensions_filters_common_fault_v3_FaultRateLimit*); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_delay_percent_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(12, 24), upb_strview); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_abort_percent_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(20, 40), upb_strview); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_delay_duration_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(28, 56), upb_strview); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_abort_http_status_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(36, 72), upb_strview); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_max_active_faults_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(44, 88), upb_strview); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_response_rate_limit_percent_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(52, 104), upb_strview); } |
||||
UPB_INLINE upb_strview envoy_extensions_filters_http_fault_v3_HTTPFault_abort_grpc_status_runtime(const envoy_extensions_filters_http_fault_v3_HTTPFault *msg) { return *UPB_PTR_AT(msg, UPB_SIZE(60, 120), upb_strview); } |
||||
|
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_delay(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, struct envoy_extensions_filters_common_fault_v3_FaultDelay* value) { |
||||
_upb_sethas(msg, 1); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(68, 136), struct envoy_extensions_filters_common_fault_v3_FaultDelay*) = value; |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_common_fault_v3_FaultDelay* envoy_extensions_filters_http_fault_v3_HTTPFault_mutable_delay(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_common_fault_v3_FaultDelay* sub = (struct envoy_extensions_filters_common_fault_v3_FaultDelay*)envoy_extensions_filters_http_fault_v3_HTTPFault_delay(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_common_fault_v3_FaultDelay*)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_set_delay(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_abort(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, envoy_extensions_filters_http_fault_v3_FaultAbort* value) { |
||||
_upb_sethas(msg, 2); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(72, 144), envoy_extensions_filters_http_fault_v3_FaultAbort*) = value; |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_http_fault_v3_FaultAbort* envoy_extensions_filters_http_fault_v3_HTTPFault_mutable_abort(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_http_fault_v3_FaultAbort* sub = (struct envoy_extensions_filters_http_fault_v3_FaultAbort*)envoy_extensions_filters_http_fault_v3_HTTPFault_abort(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_http_fault_v3_FaultAbort*)_upb_msg_new(&envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_set_abort(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_upstream_cluster(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(4, 8), upb_strview) = value; |
||||
} |
||||
UPB_INLINE struct envoy_config_route_v3_HeaderMatcher** envoy_extensions_filters_http_fault_v3_HTTPFault_mutable_headers(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, size_t *len) { |
||||
return (struct envoy_config_route_v3_HeaderMatcher**)_upb_array_mutable_accessor(msg, UPB_SIZE(84, 168), len); |
||||
} |
||||
UPB_INLINE struct envoy_config_route_v3_HeaderMatcher** envoy_extensions_filters_http_fault_v3_HTTPFault_resize_headers(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, size_t len, upb_arena *arena) { |
||||
return (struct envoy_config_route_v3_HeaderMatcher**)_upb_array_resize_accessor2(msg, UPB_SIZE(84, 168), len, UPB_SIZE(2, 3), arena); |
||||
} |
||||
UPB_INLINE struct envoy_config_route_v3_HeaderMatcher* envoy_extensions_filters_http_fault_v3_HTTPFault_add_headers(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_arena *arena) { |
||||
struct envoy_config_route_v3_HeaderMatcher* sub = (struct envoy_config_route_v3_HeaderMatcher*)_upb_msg_new(&envoy_config_route_v3_HeaderMatcher_msginit, arena); |
||||
bool ok = _upb_array_append_accessor2( |
||||
msg, UPB_SIZE(84, 168), UPB_SIZE(2, 3), &sub, arena); |
||||
if (!ok) return NULL; |
||||
return sub; |
||||
} |
||||
UPB_INLINE upb_strview* envoy_extensions_filters_http_fault_v3_HTTPFault_mutable_downstream_nodes(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, size_t *len) { |
||||
return (upb_strview*)_upb_array_mutable_accessor(msg, UPB_SIZE(88, 176), len); |
||||
} |
||||
UPB_INLINE upb_strview* envoy_extensions_filters_http_fault_v3_HTTPFault_resize_downstream_nodes(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, size_t len, upb_arena *arena) { |
||||
return (upb_strview*)_upb_array_resize_accessor2(msg, UPB_SIZE(88, 176), len, UPB_SIZE(3, 4), arena); |
||||
} |
||||
UPB_INLINE bool envoy_extensions_filters_http_fault_v3_HTTPFault_add_downstream_nodes(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview val, upb_arena *arena) { |
||||
return _upb_array_append_accessor2(msg, UPB_SIZE(88, 176), UPB_SIZE(3, 4), &val, |
||||
arena); |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_max_active_faults(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, struct google_protobuf_UInt32Value* value) { |
||||
_upb_sethas(msg, 3); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(76, 152), struct google_protobuf_UInt32Value*) = value; |
||||
} |
||||
UPB_INLINE struct google_protobuf_UInt32Value* envoy_extensions_filters_http_fault_v3_HTTPFault_mutable_max_active_faults(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_arena *arena) { |
||||
struct google_protobuf_UInt32Value* sub = (struct google_protobuf_UInt32Value*)envoy_extensions_filters_http_fault_v3_HTTPFault_max_active_faults(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct google_protobuf_UInt32Value*)_upb_msg_new(&google_protobuf_UInt32Value_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_set_max_active_faults(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_response_rate_limit(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, struct envoy_extensions_filters_common_fault_v3_FaultRateLimit* value) { |
||||
_upb_sethas(msg, 4); |
||||
*UPB_PTR_AT(msg, UPB_SIZE(80, 160), struct envoy_extensions_filters_common_fault_v3_FaultRateLimit*) = value; |
||||
} |
||||
UPB_INLINE struct envoy_extensions_filters_common_fault_v3_FaultRateLimit* envoy_extensions_filters_http_fault_v3_HTTPFault_mutable_response_rate_limit(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_arena *arena) { |
||||
struct envoy_extensions_filters_common_fault_v3_FaultRateLimit* sub = (struct envoy_extensions_filters_common_fault_v3_FaultRateLimit*)envoy_extensions_filters_http_fault_v3_HTTPFault_response_rate_limit(msg); |
||||
if (sub == NULL) { |
||||
sub = (struct envoy_extensions_filters_common_fault_v3_FaultRateLimit*)_upb_msg_new(&envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, arena); |
||||
if (!sub) return NULL; |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_set_response_rate_limit(msg, sub); |
||||
} |
||||
return sub; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_delay_percent_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(12, 24), upb_strview) = value; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_abort_percent_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(20, 40), upb_strview) = value; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_delay_duration_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(28, 56), upb_strview) = value; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_abort_http_status_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(36, 72), upb_strview) = value; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_max_active_faults_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(44, 88), upb_strview) = value; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_response_rate_limit_percent_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(52, 104), upb_strview) = value; |
||||
} |
||||
UPB_INLINE void envoy_extensions_filters_http_fault_v3_HTTPFault_set_abort_grpc_status_runtime(envoy_extensions_filters_http_fault_v3_HTTPFault *msg, upb_strview value) { |
||||
*UPB_PTR_AT(msg, UPB_SIZE(60, 120), upb_strview) = value; |
||||
} |
||||
|
||||
#ifdef __cplusplus |
||||
} /* extern "C" */ |
||||
#endif |
||||
|
||||
#include "upb/port_undef.inc" |
||||
|
||||
#endif /* ENVOY_EXTENSIONS_FILTERS_HTTP_FAULT_V3_FAULT_PROTO_UPB_H_ */ |
@ -0,0 +1,102 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/common/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#include "upb/def.h" |
||||
#include "envoy/extensions/filters/common/fault/v3/fault.upbdefs.h" |
||||
|
||||
extern upb_def_init envoy_type_v3_percent_proto_upbdefinit; |
||||
extern upb_def_init google_protobuf_duration_proto_upbdefinit; |
||||
extern upb_def_init udpa_annotations_status_proto_upbdefinit; |
||||
extern upb_def_init udpa_annotations_versioning_proto_upbdefinit; |
||||
extern upb_def_init validate_validate_proto_upbdefinit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit; |
||||
|
||||
static const upb_msglayout *layouts[5] = { |
||||
&envoy_extensions_filters_common_fault_v3_FaultDelay_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_msginit, |
||||
&envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_msginit, |
||||
}; |
||||
|
||||
static const char descriptor[1354] = {'\n', '4', 'e', 'n', 'v', 'o', 'y', '/', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '/', 'f', 'i', 'l', 't', 'e', 'r',
|
||||
's', '/', 'c', 'o', 'm', 'm', 'o', 'n', '/', 'f', 'a', 'u', 'l', 't', '/', 'v', '3', '/', 'f', 'a', 'u', 'l', 't', '.', 'p',
|
||||
'r', 'o', 't', 'o', '\022', '(', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i',
|
||||
'l', 't', 'e', 'r', 's', '.', 'c', 'o', 'm', 'm', 'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', '\032', '\033', 'e', 'n',
|
||||
'v', 'o', 'y', '/', 't', 'y', 'p', 'e', '/', 'v', '3', '/', 'p', 'e', 'r', 'c', 'e', 'n', 't', '.', 'p', 'r', 'o', 't', 'o',
|
||||
'\032', '\036', 'g', 'o', 'o', 'g', 'l', 'e', '/', 'p', 'r', 'o', 't', 'o', 'b', 'u', 'f', '/', 'd', 'u', 'r', 'a', 't', 'i', 'o',
|
||||
'n', '.', 'p', 'r', 'o', 't', 'o', '\032', '\035', 'u', 'd', 'p', 'a', '/', 'a', 'n', 'n', 'o', 't', 'a', 't', 'i', 'o', 'n', 's',
|
||||
'/', 's', 't', 'a', 't', 'u', 's', '.', 'p', 'r', 'o', 't', 'o', '\032', '!', 'u', 'd', 'p', 'a', '/', 'a', 'n', 'n', 'o', 't',
|
||||
'a', 't', 'i', 'o', 'n', 's', '/', 'v', 'e', 'r', 's', 'i', 'o', 'n', 'i', 'n', 'g', '.', 'p', 'r', 'o', 't', 'o', '\032', '\027',
|
||||
'v', 'a', 'l', 'i', 'd', 'a', 't', 'e', '/', 'v', 'a', 'l', 'i', 'd', 'a', 't', 'e', '.', 'p', 'r', 'o', 't', 'o', '\"', '\304',
|
||||
'\003', '\n', '\n', 'F', 'a', 'u', 'l', 't', 'D', 'e', 'l', 'a', 'y', '\022', 'F', '\n', '\013', 'f', 'i', 'x', 'e', 'd', '_', 'd', 'e',
|
||||
'l', 'a', 'y', '\030', '\003', ' ', '\001', '(', '\013', '2', '\031', '.', 'g', 'o', 'o', 'g', 'l', 'e', '.', 'p', 'r', 'o', 't', 'o', 'b',
|
||||
'u', 'f', '.', 'D', 'u', 'r', 'a', 't', 'i', 'o', 'n', 'B', '\010', '\372', 'B', '\005', '\252', '\001', '\002', '*', '\000', 'H', '\000', 'R', '\n',
|
||||
'f', 'i', 'x', 'e', 'd', 'D', 'e', 'l', 'a', 'y', '\022', 'e', '\n', '\014', 'h', 'e', 'a', 'd', 'e', 'r', '_', 'd', 'e', 'l', 'a',
|
||||
'y', '\030', '\005', ' ', '\001', '(', '\013', '2', '@', '.', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n',
|
||||
's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'c', 'o', 'm', 'm', 'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3',
|
||||
'.', 'F', 'a', 'u', 'l', 't', 'D', 'e', 'l', 'a', 'y', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'D', 'e', 'l', 'a', 'y', 'H', '\000',
|
||||
'R', '\013', 'h', 'e', 'a', 'd', 'e', 'r', 'D', 'e', 'l', 'a', 'y', '\022', '@', '\n', '\n', 'p', 'e', 'r', 'c', 'e', 'n', 't', 'a',
|
||||
'g', 'e', '\030', '\004', ' ', '\001', '(', '\013', '2', ' ', '.', 'e', 'n', 'v', 'o', 'y', '.', 't', 'y', 'p', 'e', '.', 'v', '3', '.',
|
||||
'F', 'r', 'a', 'c', 't', 'i', 'o', 'n', 'a', 'l', 'P', 'e', 'r', 'c', 'e', 'n', 't', 'R', '\n', 'p', 'e', 'r', 'c', 'e', 'n',
|
||||
't', 'a', 'g', 'e', '\032', 'I', '\n', '\013', 'H', 'e', 'a', 'd', 'e', 'r', 'D', 'e', 'l', 'a', 'y', ':', ':', '\232', '\305', '\210', '\036',
|
||||
'5', '\n', '3', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o', 'n', 'f', 'i', 'g', '.', 'f', 'i', 'l', 't', 'e', 'r', '.', 'f', 'a',
|
||||
'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l', 't', 'D', 'e', 'l', 'a', 'y', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'D',
|
||||
'e', 'l', 'a', 'y', '\"', '\033', '\n', '\016', 'F', 'a', 'u', 'l', 't', 'D', 'e', 'l', 'a', 'y', 'T', 'y', 'p', 'e', '\022', '\t', '\n',
|
||||
'\005', 'F', 'I', 'X', 'E', 'D', '\020', '\000', ':', '.', '\232', '\305', '\210', '\036', ')', '\n', '\'', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o',
|
||||
'n', 'f', 'i', 'g', '.', 'f', 'i', 'l', 't', 'e', 'r', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l',
|
||||
't', 'D', 'e', 'l', 'a', 'y', 'B', '\033', '\n', '\024', 'f', 'a', 'u', 'l', 't', '_', 'd', 'e', 'l', 'a', 'y', '_', 's', 'e', 'c',
|
||||
'i', 'f', 'i', 'e', 'r', '\022', '\003', '\370', 'B', '\001', 'J', '\004', '\010', '\002', '\020', '\003', 'J', '\004', '\010', '\001', '\020', '\002', 'R', '\004', 't',
|
||||
'y', 'p', 'e', '\"', '\260', '\004', '\n', '\016', 'F', 'a', 'u', 'l', 't', 'R', 'a', 't', 'e', 'L', 'i', 'm', 'i', 't', '\022', 'f', '\n',
|
||||
'\013', 'f', 'i', 'x', 'e', 'd', '_', 'l', 'i', 'm', 'i', 't', '\030', '\001', ' ', '\001', '(', '\013', '2', 'C', '.', 'e', 'n', 'v', 'o',
|
||||
'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'c', 'o', 'm', 'm',
|
||||
'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', '.', 'F', 'a', 'u', 'l', 't', 'R', 'a', 't', 'e', 'L', 'i', 'm', 'i',
|
||||
't', '.', 'F', 'i', 'x', 'e', 'd', 'L', 'i', 'm', 'i', 't', 'H', '\000', 'R', '\n', 'f', 'i', 'x', 'e', 'd', 'L', 'i', 'm', 'i',
|
||||
't', '\022', 'i', '\n', '\014', 'h', 'e', 'a', 'd', 'e', 'r', '_', 'l', 'i', 'm', 'i', 't', '\030', '\003', ' ', '\001', '(', '\013', '2', 'D',
|
||||
'.', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's',
|
||||
'.', 'c', 'o', 'm', 'm', 'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', '.', 'F', 'a', 'u', 'l', 't', 'R', 'a', 't',
|
||||
'e', 'L', 'i', 'm', 'i', 't', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'L', 'i', 'm', 'i', 't', 'H', '\000', 'R', '\013', 'h', 'e', 'a',
|
||||
'd', 'e', 'r', 'L', 'i', 'm', 'i', 't', '\022', '@', '\n', '\n', 'p', 'e', 'r', 'c', 'e', 'n', 't', 'a', 'g', 'e', '\030', '\002', ' ',
|
||||
'\001', '(', '\013', '2', ' ', '.', 'e', 'n', 'v', 'o', 'y', '.', 't', 'y', 'p', 'e', '.', 'v', '3', '.', 'F', 'r', 'a', 'c', 't',
|
||||
'i', 'o', 'n', 'a', 'l', 'P', 'e', 'r', 'c', 'e', 'n', 't', 'R', '\n', 'p', 'e', 'r', 'c', 'e', 'n', 't', 'a', 'g', 'e', '\032',
|
||||
's', '\n', '\n', 'F', 'i', 'x', 'e', 'd', 'L', 'i', 'm', 'i', 't', '\022', '&', '\n', '\n', 'l', 'i', 'm', 'i', 't', '_', 'k', 'b',
|
||||
'p', 's', '\030', '\001', ' ', '\001', '(', '\004', 'B', '\007', '\372', 'B', '\004', '2', '\002', '(', '\001', 'R', '\t', 'l', 'i', 'm', 'i', 't', 'K',
|
||||
'b', 'p', 's', ':', '=', '\232', '\305', '\210', '\036', '8', '\n', '6', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o', 'n', 'f', 'i', 'g', '.',
|
||||
'f', 'i', 'l', 't', 'e', 'r', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l', 't', 'R', 'a', 't', 'e',
|
||||
'L', 'i', 'm', 'i', 't', '.', 'F', 'i', 'x', 'e', 'd', 'L', 'i', 'm', 'i', 't', '\032', 'M', '\n', '\013', 'H', 'e', 'a', 'd', 'e',
|
||||
'r', 'L', 'i', 'm', 'i', 't', ':', '>', '\232', '\305', '\210', '\036', '9', '\n', '7', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o', 'n', 'f',
|
||||
'i', 'g', '.', 'f', 'i', 'l', 't', 'e', 'r', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l', 't', 'R',
|
||||
'a', 't', 'e', 'L', 'i', 'm', 'i', 't', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'L', 'i', 'm', 'i', 't', ':', '2', '\232', '\305', '\210',
|
||||
'\036', '-', '\n', '+', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o', 'n', 'f', 'i', 'g', '.', 'f', 'i', 'l', 't', 'e', 'r', '.', 'f',
|
||||
'a', 'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l', 't', 'R', 'a', 't', 'e', 'L', 'i', 'm', 'i', 't', 'B', '\021', '\n',
|
||||
'\n', 'l', 'i', 'm', 'i', 't', '_', 't', 'y', 'p', 'e', '\022', '\003', '\370', 'B', '\001', 'B', 'N', '\n', '6', 'i', 'o', '.', 'e', 'n',
|
||||
'v', 'o', 'y', 'p', 'r', 'o', 'x', 'y', '.', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's',
|
||||
'.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'c', 'o', 'm', 'm', 'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', 'B',
|
||||
'\n', 'F', 'a', 'u', 'l', 't', 'P', 'r', 'o', 't', 'o', 'P', '\001', '\272', '\200', '\310', '\321', '\006', '\002', '\020', '\002', 'b', '\006', 'p', 'r',
|
||||
'o', 't', 'o', '3',
|
||||
}; |
||||
|
||||
static upb_def_init *deps[6] = { |
||||
&envoy_type_v3_percent_proto_upbdefinit, |
||||
&google_protobuf_duration_proto_upbdefinit, |
||||
&udpa_annotations_status_proto_upbdefinit, |
||||
&udpa_annotations_versioning_proto_upbdefinit, |
||||
&validate_validate_proto_upbdefinit, |
||||
NULL |
||||
}; |
||||
|
||||
upb_def_init envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit = { |
||||
deps, |
||||
layouts, |
||||
"envoy/extensions/filters/common/fault/v3/fault.proto", |
||||
UPB_STRVIEW_INIT(descriptor, 1354) |
||||
}; |
@ -0,0 +1,55 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/common/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#ifndef ENVOY_EXTENSIONS_FILTERS_COMMON_FAULT_V3_FAULT_PROTO_UPBDEFS_H_ |
||||
#define ENVOY_EXTENSIONS_FILTERS_COMMON_FAULT_V3_FAULT_PROTO_UPBDEFS_H_ |
||||
|
||||
#include "upb/def.h" |
||||
#include "upb/port_def.inc" |
||||
#ifdef __cplusplus |
||||
extern "C" { |
||||
#endif |
||||
|
||||
#include "upb/def.h" |
||||
|
||||
#include "upb/port_def.inc" |
||||
|
||||
extern upb_def_init envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit; |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_common_fault_v3_FaultDelay_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.common.fault.v3.FaultDelay"); |
||||
} |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_common_fault_v3_FaultDelay_HeaderDelay_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.common.fault.v3.FaultDelay.HeaderDelay"); |
||||
} |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_common_fault_v3_FaultRateLimit_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.common.fault.v3.FaultRateLimit"); |
||||
} |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_common_fault_v3_FaultRateLimit_FixedLimit_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.common.fault.v3.FaultRateLimit.FixedLimit"); |
||||
} |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_common_fault_v3_FaultRateLimit_HeaderLimit_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.common.fault.v3.FaultRateLimit.HeaderLimit"); |
||||
} |
||||
|
||||
#ifdef __cplusplus |
||||
} /* extern "C" */ |
||||
#endif |
||||
|
||||
#include "upb/port_undef.inc" |
||||
|
||||
#endif /* ENVOY_EXTENSIONS_FILTERS_COMMON_FAULT_V3_FAULT_PROTO_UPBDEFS_H_ */ |
@ -0,0 +1,120 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/http/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#include "upb/def.h" |
||||
#include "envoy/extensions/filters/http/fault/v3/fault.upbdefs.h" |
||||
|
||||
extern upb_def_init envoy_config_route_v3_route_components_proto_upbdefinit; |
||||
extern upb_def_init envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit; |
||||
extern upb_def_init envoy_type_v3_percent_proto_upbdefinit; |
||||
extern upb_def_init google_protobuf_wrappers_proto_upbdefinit; |
||||
extern upb_def_init udpa_annotations_status_proto_upbdefinit; |
||||
extern upb_def_init udpa_annotations_versioning_proto_upbdefinit; |
||||
extern upb_def_init validate_validate_proto_upbdefinit; |
||||
extern const upb_msglayout envoy_extensions_filters_http_fault_v3_FaultAbort_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit; |
||||
extern const upb_msglayout envoy_extensions_filters_http_fault_v3_HTTPFault_msginit; |
||||
|
||||
static const upb_msglayout *layouts[3] = { |
||||
&envoy_extensions_filters_http_fault_v3_FaultAbort_msginit, |
||||
&envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_msginit, |
||||
&envoy_extensions_filters_http_fault_v3_HTTPFault_msginit, |
||||
}; |
||||
|
||||
static const char descriptor[1812] = {'\n', '2', 'e', 'n', 'v', 'o', 'y', '/', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '/', 'f', 'i', 'l', 't', 'e', 'r',
|
||||
's', '/', 'h', 't', 't', 'p', '/', 'f', 'a', 'u', 'l', 't', '/', 'v', '3', '/', 'f', 'a', 'u', 'l', 't', '.', 'p', 'r', 'o',
|
||||
't', 'o', '\022', '&', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't',
|
||||
'e', 'r', 's', '.', 'h', 't', 't', 'p', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', '\032', ',', 'e', 'n', 'v', 'o', 'y', '/',
|
||||
'c', 'o', 'n', 'f', 'i', 'g', '/', 'r', 'o', 'u', 't', 'e', '/', 'v', '3', '/', 'r', 'o', 'u', 't', 'e', '_', 'c', 'o', 'm',
|
||||
'p', 'o', 'n', 'e', 'n', 't', 's', '.', 'p', 'r', 'o', 't', 'o', '\032', '4', 'e', 'n', 'v', 'o', 'y', '/', 'e', 'x', 't', 'e',
|
||||
'n', 's', 'i', 'o', 'n', 's', '/', 'f', 'i', 'l', 't', 'e', 'r', 's', '/', 'c', 'o', 'm', 'm', 'o', 'n', '/', 'f', 'a', 'u',
|
||||
'l', 't', '/', 'v', '3', '/', 'f', 'a', 'u', 'l', 't', '.', 'p', 'r', 'o', 't', 'o', '\032', '\033', 'e', 'n', 'v', 'o', 'y', '/',
|
||||
't', 'y', 'p', 'e', '/', 'v', '3', '/', 'p', 'e', 'r', 'c', 'e', 'n', 't', '.', 'p', 'r', 'o', 't', 'o', '\032', '\036', 'g', 'o',
|
||||
'o', 'g', 'l', 'e', '/', 'p', 'r', 'o', 't', 'o', 'b', 'u', 'f', '/', 'w', 'r', 'a', 'p', 'p', 'e', 'r', 's', '.', 'p', 'r',
|
||||
'o', 't', 'o', '\032', '\035', 'u', 'd', 'p', 'a', '/', 'a', 'n', 'n', 'o', 't', 'a', 't', 'i', 'o', 'n', 's', '/', 's', 't', 'a',
|
||||
't', 'u', 's', '.', 'p', 'r', 'o', 't', 'o', '\032', '!', 'u', 'd', 'p', 'a', '/', 'a', 'n', 'n', 'o', 't', 'a', 't', 'i', 'o',
|
||||
'n', 's', '/', 'v', 'e', 'r', 's', 'i', 'o', 'n', 'i', 'n', 'g', '.', 'p', 'r', 'o', 't', 'o', '\032', '\027', 'v', 'a', 'l', 'i',
|
||||
'd', 'a', 't', 'e', '/', 'v', 'a', 'l', 'i', 'd', 'a', 't', 'e', '.', 'p', 'r', 'o', 't', 'o', '\"', '\244', '\003', '\n', '\n', 'F',
|
||||
'a', 'u', 'l', 't', 'A', 'b', 'o', 'r', 't', '\022', '.', '\n', '\013', 'h', 't', 't', 'p', '_', 's', 't', 'a', 't', 'u', 's', '\030',
|
||||
'\002', ' ', '\001', '(', '\r', 'B', '\013', '\372', 'B', '\010', '*', '\006', '\020', '\330', '\004', '(', '\310', '\001', 'H', '\000', 'R', '\n', 'h', 't', 't',
|
||||
'p', 'S', 't', 'a', 't', 'u', 's', '\022', '!', '\n', '\013', 'g', 'r', 'p', 'c', '_', 's', 't', 'a', 't', 'u', 's', '\030', '\005', ' ',
|
||||
'\001', '(', '\r', 'H', '\000', 'R', '\n', 'g', 'r', 'p', 'c', 'S', 't', 'a', 't', 'u', 's', '\022', 'c', '\n', '\014', 'h', 'e', 'a', 'd',
|
||||
'e', 'r', '_', 'a', 'b', 'o', 'r', 't', '\030', '\004', ' ', '\001', '(', '\013', '2', '>', '.', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x',
|
||||
't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'h', 't', 't', 'p', '.', 'f', 'a', 'u',
|
||||
'l', 't', '.', 'v', '3', '.', 'F', 'a', 'u', 'l', 't', 'A', 'b', 'o', 'r', 't', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'A', 'b',
|
||||
'o', 'r', 't', 'H', '\000', 'R', '\013', 'h', 'e', 'a', 'd', 'e', 'r', 'A', 'b', 'o', 'r', 't', '\022', '@', '\n', '\n', 'p', 'e', 'r',
|
||||
'c', 'e', 'n', 't', 'a', 'g', 'e', '\030', '\003', ' ', '\001', '(', '\013', '2', ' ', '.', 'e', 'n', 'v', 'o', 'y', '.', 't', 'y', 'p',
|
||||
'e', '.', 'v', '3', '.', 'F', 'r', 'a', 'c', 't', 'i', 'o', 'n', 'a', 'l', 'P', 'e', 'r', 'c', 'e', 'n', 't', 'R', '\n', 'p',
|
||||
'e', 'r', 'c', 'e', 'n', 't', 'a', 'g', 'e', '\032', 'N', '\n', '\013', 'H', 'e', 'a', 'd', 'e', 'r', 'A', 'b', 'o', 'r', 't', ':',
|
||||
'?', '\232', '\305', '\210', '\036', ':', '\n', '8', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o', 'n', 'f', 'i', 'g', '.', 'f', 'i', 'l', 't',
|
||||
'e', 'r', '.', 'h', 't', 't', 'p', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l', 't', 'A', 'b', 'o',
|
||||
'r', 't', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'A', 'b', 'o', 'r', 't', ':', '3', '\232', '\305', '\210', '\036', '.', '\n', ',', 'e', 'n',
|
||||
'v', 'o', 'y', '.', 'c', 'o', 'n', 'f', 'i', 'g', '.', 'f', 'i', 'l', 't', 'e', 'r', '.', 'h', 't', 't', 'p', '.', 'f', 'a',
|
||||
'u', 'l', 't', '.', 'v', '2', '.', 'F', 'a', 'u', 'l', 't', 'A', 'b', 'o', 'r', 't', 'B', '\021', '\n', '\n', 'e', 'r', 'r', 'o',
|
||||
'r', '_', 't', 'y', 'p', 'e', '\022', '\003', '\370', 'B', '\001', 'J', '\004', '\010', '\001', '\020', '\002', '\"', '\274', '\007', '\n', '\t', 'H', 'T', 'T',
|
||||
'P', 'F', 'a', 'u', 'l', 't', '\022', 'J', '\n', '\005', 'd', 'e', 'l', 'a', 'y', '\030', '\001', ' ', '\001', '(', '\013', '2', '4', '.', 'e',
|
||||
'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'c',
|
||||
'o', 'm', 'm', 'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', '.', 'F', 'a', 'u', 'l', 't', 'D', 'e', 'l', 'a', 'y',
|
||||
'R', '\005', 'd', 'e', 'l', 'a', 'y', '\022', 'H', '\n', '\005', 'a', 'b', 'o', 'r', 't', '\030', '\002', ' ', '\001', '(', '\013', '2', '2', '.',
|
||||
'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.',
|
||||
'h', 't', 't', 'p', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '3', '.', 'F', 'a', 'u', 'l', 't', 'A', 'b', 'o', 'r', 't', 'R',
|
||||
'\005', 'a', 'b', 'o', 'r', 't', '\022', ')', '\n', '\020', 'u', 'p', 's', 't', 'r', 'e', 'a', 'm', '_', 'c', 'l', 'u', 's', 't', 'e',
|
||||
'r', '\030', '\003', ' ', '\001', '(', '\t', 'R', '\017', 'u', 'p', 's', 't', 'r', 'e', 'a', 'm', 'C', 'l', 'u', 's', 't', 'e', 'r', '\022',
|
||||
'>', '\n', '\007', 'h', 'e', 'a', 'd', 'e', 'r', 's', '\030', '\004', ' ', '\003', '(', '\013', '2', '$', '.', 'e', 'n', 'v', 'o', 'y', '.',
|
||||
'c', 'o', 'n', 'f', 'i', 'g', '.', 'r', 'o', 'u', 't', 'e', '.', 'v', '3', '.', 'H', 'e', 'a', 'd', 'e', 'r', 'M', 'a', 't',
|
||||
'c', 'h', 'e', 'r', 'R', '\007', 'h', 'e', 'a', 'd', 'e', 'r', 's', '\022', ')', '\n', '\020', 'd', 'o', 'w', 'n', 's', 't', 'r', 'e',
|
||||
'a', 'm', '_', 'n', 'o', 'd', 'e', 's', '\030', '\005', ' ', '\003', '(', '\t', 'R', '\017', 'd', 'o', 'w', 'n', 's', 't', 'r', 'e', 'a',
|
||||
'm', 'N', 'o', 'd', 'e', 's', '\022', 'H', '\n', '\021', 'm', 'a', 'x', '_', 'a', 'c', 't', 'i', 'v', 'e', '_', 'f', 'a', 'u', 'l',
|
||||
't', 's', '\030', '\006', ' ', '\001', '(', '\013', '2', '\034', '.', 'g', 'o', 'o', 'g', 'l', 'e', '.', 'p', 'r', 'o', 't', 'o', 'b', 'u',
|
||||
'f', '.', 'U', 'I', 'n', 't', '3', '2', 'V', 'a', 'l', 'u', 'e', 'R', '\017', 'm', 'a', 'x', 'A', 'c', 't', 'i', 'v', 'e', 'F',
|
||||
'a', 'u', 'l', 't', 's', '\022', 'h', '\n', '\023', 'r', 'e', 's', 'p', 'o', 'n', 's', 'e', '_', 'r', 'a', 't', 'e', '_', 'l', 'i',
|
||||
'm', 'i', 't', '\030', '\007', ' ', '\001', '(', '\013', '2', '8', '.', 'e', 'n', 'v', 'o', 'y', '.', 'e', 'x', 't', 'e', 'n', 's', 'i',
|
||||
'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'c', 'o', 'm', 'm', 'o', 'n', '.', 'f', 'a', 'u', 'l', 't', '.',
|
||||
'v', '3', '.', 'F', 'a', 'u', 'l', 't', 'R', 'a', 't', 'e', 'L', 'i', 'm', 'i', 't', 'R', '\021', 'r', 'e', 's', 'p', 'o', 'n',
|
||||
's', 'e', 'R', 'a', 't', 'e', 'L', 'i', 'm', 'i', 't', '\022', '2', '\n', '\025', 'd', 'e', 'l', 'a', 'y', '_', 'p', 'e', 'r', 'c',
|
||||
'e', 'n', 't', '_', 'r', 'u', 'n', 't', 'i', 'm', 'e', '\030', '\010', ' ', '\001', '(', '\t', 'R', '\023', 'd', 'e', 'l', 'a', 'y', 'P',
|
||||
'e', 'r', 'c', 'e', 'n', 't', 'R', 'u', 'n', 't', 'i', 'm', 'e', '\022', '2', '\n', '\025', 'a', 'b', 'o', 'r', 't', '_', 'p', 'e',
|
||||
'r', 'c', 'e', 'n', 't', '_', 'r', 'u', 'n', 't', 'i', 'm', 'e', '\030', '\t', ' ', '\001', '(', '\t', 'R', '\023', 'a', 'b', 'o', 'r',
|
||||
't', 'P', 'e', 'r', 'c', 'e', 'n', 't', 'R', 'u', 'n', 't', 'i', 'm', 'e', '\022', '4', '\n', '\026', 'd', 'e', 'l', 'a', 'y', '_',
|
||||
'd', 'u', 'r', 'a', 't', 'i', 'o', 'n', '_', 'r', 'u', 'n', 't', 'i', 'm', 'e', '\030', '\n', ' ', '\001', '(', '\t', 'R', '\024', 'd',
|
||||
'e', 'l', 'a', 'y', 'D', 'u', 'r', 'a', 't', 'i', 'o', 'n', 'R', 'u', 'n', 't', 'i', 'm', 'e', '\022', '9', '\n', '\031', 'a', 'b',
|
||||
'o', 'r', 't', '_', 'h', 't', 't', 'p', '_', 's', 't', 'a', 't', 'u', 's', '_', 'r', 'u', 'n', 't', 'i', 'm', 'e', '\030', '\013',
|
||||
' ', '\001', '(', '\t', 'R', '\026', 'a', 'b', 'o', 'r', 't', 'H', 't', 't', 'p', 'S', 't', 'a', 't', 'u', 's', 'R', 'u', 'n', 't',
|
||||
'i', 'm', 'e', '\022', '9', '\n', '\031', 'm', 'a', 'x', '_', 'a', 'c', 't', 'i', 'v', 'e', '_', 'f', 'a', 'u', 'l', 't', 's', '_',
|
||||
'r', 'u', 'n', 't', 'i', 'm', 'e', '\030', '\014', ' ', '\001', '(', '\t', 'R', '\026', 'm', 'a', 'x', 'A', 'c', 't', 'i', 'v', 'e', 'F',
|
||||
'a', 'u', 'l', 't', 's', 'R', 'u', 'n', 't', 'i', 'm', 'e', '\022', 'L', '\n', '#', 'r', 'e', 's', 'p', 'o', 'n', 's', 'e', '_',
|
||||
'r', 'a', 't', 'e', '_', 'l', 'i', 'm', 'i', 't', '_', 'p', 'e', 'r', 'c', 'e', 'n', 't', '_', 'r', 'u', 'n', 't', 'i', 'm',
|
||||
'e', '\030', '\r', ' ', '\001', '(', '\t', 'R', '\037', 'r', 'e', 's', 'p', 'o', 'n', 's', 'e', 'R', 'a', 't', 'e', 'L', 'i', 'm', 'i',
|
||||
't', 'P', 'e', 'r', 'c', 'e', 'n', 't', 'R', 'u', 'n', 't', 'i', 'm', 'e', '\022', '9', '\n', '\031', 'a', 'b', 'o', 'r', 't', '_',
|
||||
'g', 'r', 'p', 'c', '_', 's', 't', 'a', 't', 'u', 's', '_', 'r', 'u', 'n', 't', 'i', 'm', 'e', '\030', '\016', ' ', '\001', '(', '\t',
|
||||
'R', '\026', 'a', 'b', 'o', 'r', 't', 'G', 'r', 'p', 'c', 'S', 't', 'a', 't', 'u', 's', 'R', 'u', 'n', 't', 'i', 'm', 'e', ':',
|
||||
'2', '\232', '\305', '\210', '\036', '-', '\n', '+', 'e', 'n', 'v', 'o', 'y', '.', 'c', 'o', 'n', 'f', 'i', 'g', '.', 'f', 'i', 'l', 't',
|
||||
'e', 'r', '.', 'h', 't', 't', 'p', '.', 'f', 'a', 'u', 'l', 't', '.', 'v', '2', '.', 'H', 'T', 'T', 'P', 'F', 'a', 'u', 'l',
|
||||
't', 'B', 'L', '\n', '4', 'i', 'o', '.', 'e', 'n', 'v', 'o', 'y', 'p', 'r', 'o', 'x', 'y', '.', 'e', 'n', 'v', 'o', 'y', '.',
|
||||
'e', 'x', 't', 'e', 'n', 's', 'i', 'o', 'n', 's', '.', 'f', 'i', 'l', 't', 'e', 'r', 's', '.', 'h', 't', 't', 'p', '.', 'f',
|
||||
'a', 'u', 'l', 't', '.', 'v', '3', 'B', '\n', 'F', 'a', 'u', 'l', 't', 'P', 'r', 'o', 't', 'o', 'P', '\001', '\272', '\200', '\310', '\321',
|
||||
'\006', '\002', '\020', '\002', 'b', '\006', 'p', 'r', 'o', 't', 'o', '3',
|
||||
}; |
||||
|
||||
static upb_def_init *deps[8] = { |
||||
&envoy_config_route_v3_route_components_proto_upbdefinit, |
||||
&envoy_extensions_filters_common_fault_v3_fault_proto_upbdefinit, |
||||
&envoy_type_v3_percent_proto_upbdefinit, |
||||
&google_protobuf_wrappers_proto_upbdefinit, |
||||
&udpa_annotations_status_proto_upbdefinit, |
||||
&udpa_annotations_versioning_proto_upbdefinit, |
||||
&validate_validate_proto_upbdefinit, |
||||
NULL |
||||
}; |
||||
|
||||
upb_def_init envoy_extensions_filters_http_fault_v3_fault_proto_upbdefinit = { |
||||
deps, |
||||
layouts, |
||||
"envoy/extensions/filters/http/fault/v3/fault.proto", |
||||
UPB_STRVIEW_INIT(descriptor, 1812) |
||||
}; |
@ -0,0 +1,45 @@ |
||||
/* This file was generated by upbc (the upb compiler) from the input
|
||||
* file: |
||||
* |
||||
* envoy/extensions/filters/http/fault/v3/fault.proto |
||||
* |
||||
* Do not edit -- your changes will be discarded when the file is |
||||
* regenerated. */ |
||||
|
||||
#ifndef ENVOY_EXTENSIONS_FILTERS_HTTP_FAULT_V3_FAULT_PROTO_UPBDEFS_H_ |
||||
#define ENVOY_EXTENSIONS_FILTERS_HTTP_FAULT_V3_FAULT_PROTO_UPBDEFS_H_ |
||||
|
||||
#include "upb/def.h" |
||||
#include "upb/port_def.inc" |
||||
#ifdef __cplusplus |
||||
extern "C" { |
||||
#endif |
||||
|
||||
#include "upb/def.h" |
||||
|
||||
#include "upb/port_def.inc" |
||||
|
||||
extern upb_def_init envoy_extensions_filters_http_fault_v3_fault_proto_upbdefinit; |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_http_fault_v3_FaultAbort_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_http_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.http.fault.v3.FaultAbort"); |
||||
} |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_http_fault_v3_FaultAbort_HeaderAbort_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_http_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.http.fault.v3.FaultAbort.HeaderAbort"); |
||||
} |
||||
|
||||
UPB_INLINE const upb_msgdef *envoy_extensions_filters_http_fault_v3_HTTPFault_getmsgdef(upb_symtab *s) { |
||||
_upb_symtab_loaddefinit(s, &envoy_extensions_filters_http_fault_v3_fault_proto_upbdefinit); |
||||
return upb_symtab_lookupmsg(s, "envoy.extensions.filters.http.fault.v3.HTTPFault"); |
||||
} |
||||
|
||||
#ifdef __cplusplus |
||||
} /* extern "C" */ |
||||
#endif |
||||
|
||||
#include "upb/port_undef.inc" |
||||
|
||||
#endif /* ENVOY_EXTENSIONS_FILTERS_HTTP_FAULT_V3_FAULT_PROTO_UPBDEFS_H_ */ |
@ -0,0 +1,226 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include "src/core/ext/xds/xds_http_fault_filter.h" |
||||
|
||||
#include <grpc/grpc.h> |
||||
|
||||
#include <string> |
||||
|
||||
#include "absl/status/statusor.h" |
||||
#include "absl/strings/str_cat.h" |
||||
#include "absl/strings/str_format.h" |
||||
#include "absl/strings/string_view.h" |
||||
#include "envoy/extensions/filters/common/fault/v3/fault.upb.h" |
||||
#include "envoy/extensions/filters/http/fault/v3/fault.upb.h" |
||||
#include "envoy/extensions/filters/http/fault/v3/fault.upbdefs.h" |
||||
#include "envoy/type/v3/percent.upb.h" |
||||
#include "google/protobuf/any.upb.h" |
||||
#include "google/protobuf/duration.upb.h" |
||||
#include "google/protobuf/wrappers.upb.h" |
||||
#include "src/core/ext/filters/fault_injection/fault_injection_filter.h" |
||||
#include "src/core/ext/xds/xds_http_filters.h" |
||||
#include "src/core/lib/channel/channel_args.h" |
||||
#include "src/core/lib/channel/channel_stack.h" |
||||
#include "src/core/lib/channel/status_util.h" |
||||
#include "src/core/lib/json/json.h" |
||||
#include "src/core/lib/transport/status_conversion.h" |
||||
#include "upb/def.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
const char* kXdsHttpFaultFilterConfigName = |
||||
"envoy.extensions.filters.http.fault.v3.HTTPFault"; |
||||
|
||||
namespace { |
||||
|
||||
uint32_t GetDenominator(const envoy_type_v3_FractionalPercent* fraction) { |
||||
if (fraction != nullptr) { |
||||
const auto denominator = |
||||
static_cast<envoy_type_v3_FractionalPercent_DenominatorType>( |
||||
envoy_type_v3_FractionalPercent_denominator(fraction)); |
||||
switch (denominator) { |
||||
case envoy_type_v3_FractionalPercent_MILLION: |
||||
return 1000000; |
||||
case envoy_type_v3_FractionalPercent_TEN_THOUSAND: |
||||
return 10000; |
||||
case envoy_type_v3_FractionalPercent_HUNDRED: |
||||
default: |
||||
return 100; |
||||
} |
||||
} |
||||
// Use 100 as the default denominator
|
||||
return 100; |
||||
} |
||||
|
||||
absl::StatusOr<Json> ParseHttpFaultIntoJson(upb_strview serialized_http_fault, |
||||
upb_arena* arena) { |
||||
auto* http_fault = envoy_extensions_filters_http_fault_v3_HTTPFault_parse( |
||||
serialized_http_fault.data, serialized_http_fault.size, arena); |
||||
if (http_fault == nullptr) { |
||||
return absl::InvalidArgumentError( |
||||
"could not parse fault injection filter config"); |
||||
} |
||||
// NOTE(lidiz): Here, we are manually translating the upb messages into the
|
||||
// JSON form of the filter config as part of method config, which will be
|
||||
// directly used later by service config. In this way, we can validate the
|
||||
// filter configs, and NACK if needed. It also allows the service config to
|
||||
// function independently without xDS, but not the other way around.
|
||||
// NOTE(lidiz): please refer to FaultInjectionPolicy for ground truth
|
||||
// definitions, located at:
|
||||
// src/core/ext/filters/fault_injection/service_config_parser.h
|
||||
Json::Object fault_injection_policy_json; |
||||
// Section 1: Parse the abort injection config
|
||||
const auto* fault_abort = |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_abort(http_fault); |
||||
if (fault_abort != nullptr) { |
||||
grpc_status_code abort_grpc_status_code = GRPC_STATUS_OK; |
||||
// Try if gRPC status code is set first
|
||||
int abort_grpc_status_code_raw = |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_grpc_status( |
||||
fault_abort); |
||||
if (abort_grpc_status_code_raw != 0) { |
||||
if (!grpc_status_code_from_int(abort_grpc_status_code_raw, |
||||
&abort_grpc_status_code)) { |
||||
return absl::InvalidArgumentError(absl::StrCat( |
||||
"invalid gRPC status code: ", abort_grpc_status_code_raw)); |
||||
} |
||||
} else { |
||||
// if gRPC status code is empty, check http status
|
||||
int abort_http_status_code = |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_http_status( |
||||
fault_abort); |
||||
if (abort_http_status_code != 0 and abort_http_status_code != 200) { |
||||
abort_grpc_status_code = |
||||
grpc_http2_status_to_grpc_status(abort_http_status_code); |
||||
} |
||||
} |
||||
// Set the abort_code, even if it's OK
|
||||
fault_injection_policy_json["abortCode"] = |
||||
grpc_status_code_to_string(abort_grpc_status_code); |
||||
// Set the headers if we enabled header abort injection control
|
||||
if (envoy_extensions_filters_http_fault_v3_FaultAbort_has_header_abort( |
||||
fault_abort)) { |
||||
fault_injection_policy_json["abortCodeHeader"] = |
||||
"x-envoy-fault-abort-grpc-request"; |
||||
fault_injection_policy_json["abortPercentageHeader"] = |
||||
"x-envoy-fault-abort-percentage"; |
||||
} |
||||
// Set the fraction percent
|
||||
auto* percent = |
||||
envoy_extensions_filters_http_fault_v3_FaultAbort_percentage( |
||||
fault_abort); |
||||
fault_injection_policy_json["abortPercentageNumerator"] = |
||||
Json(envoy_type_v3_FractionalPercent_numerator(percent)); |
||||
fault_injection_policy_json["abortPercentageDenominator"] = |
||||
Json(GetDenominator(percent)); |
||||
} |
||||
// Section 2: Parse the delay injection config
|
||||
const auto* fault_delay = |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_delay(http_fault); |
||||
if (fault_delay != nullptr) { |
||||
// Parse the delay duration
|
||||
const auto* delay_duration = |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_fixed_delay( |
||||
fault_delay); |
||||
if (delay_duration != nullptr) { |
||||
fault_injection_policy_json["delay"] = absl::StrFormat( |
||||
"%d.%09ds", google_protobuf_Duration_seconds(delay_duration), |
||||
google_protobuf_Duration_nanos(delay_duration)); |
||||
} |
||||
// Set the headers if we enabled header delay injection control
|
||||
if (envoy_extensions_filters_common_fault_v3_FaultDelay_has_header_delay( |
||||
fault_delay)) { |
||||
fault_injection_policy_json["delayHeader"] = |
||||
"x-envoy-fault-delay-request"; |
||||
fault_injection_policy_json["delayPercentageHeader"] = |
||||
"x-envoy-fault-delay-request-percentage"; |
||||
} |
||||
// Set the fraction percent
|
||||
auto* percent = |
||||
envoy_extensions_filters_common_fault_v3_FaultDelay_percentage( |
||||
fault_delay); |
||||
fault_injection_policy_json["delayPercentageNumerator"] = |
||||
Json(envoy_type_v3_FractionalPercent_numerator(percent)); |
||||
fault_injection_policy_json["delayPercentageDenominator"] = |
||||
Json(GetDenominator(percent)); |
||||
} |
||||
// Section 3: Parse the maximum active faults
|
||||
const auto* max_fault_wrapper = |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_max_active_faults( |
||||
http_fault); |
||||
if (max_fault_wrapper != nullptr) { |
||||
fault_injection_policy_json["maxFaults"] = |
||||
google_protobuf_UInt32Value_value(max_fault_wrapper); |
||||
} |
||||
return fault_injection_policy_json; |
||||
} |
||||
|
||||
} // namespace
|
||||
|
||||
void XdsHttpFaultFilter::PopulateSymtab(upb_symtab* symtab) const { |
||||
envoy_extensions_filters_http_fault_v3_HTTPFault_getmsgdef(symtab); |
||||
} |
||||
|
||||
absl::StatusOr<XdsHttpFilterImpl::FilterConfig> |
||||
XdsHttpFaultFilter::GenerateFilterConfig(upb_strview serialized_filter_config, |
||||
upb_arena* arena) const { |
||||
absl::StatusOr<Json> parse_result = |
||||
ParseHttpFaultIntoJson(serialized_filter_config, arena); |
||||
if (!parse_result.ok()) { |
||||
return parse_result.status(); |
||||
} |
||||
return FilterConfig{kXdsHttpFaultFilterConfigName, std::move(*parse_result)}; |
||||
} |
||||
|
||||
absl::StatusOr<XdsHttpFilterImpl::FilterConfig> |
||||
XdsHttpFaultFilter::GenerateFilterConfigOverride( |
||||
upb_strview serialized_filter_config, upb_arena* arena) const { |
||||
// HTTPFault filter has the same message type in HTTP connection manager's
|
||||
// filter config and in overriding filter config field.
|
||||
return GenerateFilterConfig(serialized_filter_config, arena); |
||||
} |
||||
|
||||
const grpc_channel_filter* XdsHttpFaultFilter::channel_filter() const { |
||||
return &FaultInjectionFilterVtable; |
||||
} |
||||
|
||||
grpc_channel_args* XdsHttpFaultFilter::ModifyChannelArgs( |
||||
grpc_channel_args* args) const { |
||||
grpc_arg args_to_add = grpc_channel_arg_integer_create( |
||||
const_cast<char*>(GRPC_ARG_PARSE_FAULT_INJECTION_METHOD_CONFIG), 1); |
||||
grpc_channel_args* new_args = |
||||
grpc_channel_args_copy_and_add(args, &args_to_add, 1); |
||||
// Since this function takes the ownership of the channel args, it needs to
|
||||
// deallocate the old ones to prevent leak.
|
||||
grpc_channel_args_destroy(args); |
||||
return new_args; |
||||
} |
||||
|
||||
absl::StatusOr<XdsHttpFilterImpl::ServiceConfigJsonEntry> |
||||
XdsHttpFaultFilter::GenerateServiceConfig( |
||||
const FilterConfig& hcm_filter_config, |
||||
const FilterConfig* filter_config_override) const { |
||||
Json policy_json = filter_config_override != nullptr |
||||
? filter_config_override->config |
||||
: hcm_filter_config.config; |
||||
// The policy JSON may be empty, that's allowed.
|
||||
return ServiceConfigJsonEntry{"faultInjectionPolicy", policy_json.Dump()}; |
||||
} |
||||
|
||||
} // namespace grpc_core
|
@ -0,0 +1,59 @@ |
||||
//
|
||||
// Copyright 2021 gRPC authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
//
|
||||
|
||||
#ifndef GRPC_CORE_EXT_XDS_XDS_HTTP_FAULT_FILTER_H |
||||
#define GRPC_CORE_EXT_XDS_XDS_HTTP_FAULT_FILTER_H |
||||
|
||||
#include <grpc/support/port_platform.h> |
||||
|
||||
#include <grpc/grpc.h> |
||||
|
||||
#include "absl/status/statusor.h" |
||||
#include "src/core/ext/xds/xds_http_filters.h" |
||||
#include "upb/def.h" |
||||
|
||||
namespace grpc_core { |
||||
|
||||
extern const char* kXdsHttpFaultFilterConfigName; |
||||
|
||||
class XdsHttpFaultFilter : public XdsHttpFilterImpl { |
||||
public: |
||||
// Overrides the PopulateSymtab method
|
||||
void PopulateSymtab(upb_symtab* symtab) const override; |
||||
|
||||
// Overrides the GenerateFilterConfig method
|
||||
absl::StatusOr<FilterConfig> GenerateFilterConfig( |
||||
upb_strview serialized_filter_config, upb_arena* arena) const override; |
||||
|
||||
// Overrides the GenerateFilterConfigOverride method
|
||||
absl::StatusOr<FilterConfig> GenerateFilterConfigOverride( |
||||
upb_strview serialized_filter_config, upb_arena* arena) const override; |
||||
|
||||
// Overrides the channel_filter method
|
||||
const grpc_channel_filter* channel_filter() const override; |
||||
|
||||
// Overrides the ModifyChannelArgs method
|
||||
grpc_channel_args* ModifyChannelArgs(grpc_channel_args* args) const override; |
||||
|
||||
// Overrides the GenerateServiceConfig method
|
||||
absl::StatusOr<ServiceConfigJsonEntry> GenerateServiceConfig( |
||||
const FilterConfig& hcm_filter_config, |
||||
const FilterConfig* filter_config_override) const override; |
||||
}; |
||||
|
||||
} // namespace grpc_core
|
||||
|
||||
#endif /* GRPC_CORE_EXT_XDS_XDS_HTTP_FAULT_FILTER_H */ |
@ -0,0 +1,91 @@ |
||||
// Copyright 2020 The gRPC Authors |
||||
// |
||||
// Licensed under the Apache License, Version 2.0 (the "License"); |
||||
// you may not use this file except in compliance with the License. |
||||
// You may obtain a copy of the License at |
||||
// |
||||
// http://www.apache.org/licenses/LICENSE-2.0 |
||||
// |
||||
// Unless required by applicable law or agreed to in writing, software |
||||
// distributed under the License is distributed on an "AS IS" BASIS, |
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
// See the License for the specific language governing permissions and |
||||
// limitations under the License. |
||||
|
||||
// Local copy of Envoy xDS proto file, used for testing only. |
||||
|
||||
syntax = "proto3"; |
||||
|
||||
package envoy.extensions.filters.http.fault.v3; |
||||
|
||||
import "src/proto/grpc/testing/xds/v3/fault_common.proto"; |
||||
import "src/proto/grpc/testing/xds/v3/route.proto"; |
||||
import "src/proto/grpc/testing/xds/v3/percent.proto"; |
||||
|
||||
import "google/protobuf/wrappers.proto"; |
||||
|
||||
// [#protodoc-title: Fault Injection] |
||||
// Fault Injection :ref:`configuration overview <config_http_filters_fault_injection>`. |
||||
// [#extension: envoy.filters.http.fault] |
||||
|
||||
// [#next-free-field: 6] |
||||
message FaultAbort { |
||||
// Fault aborts are controlled via an HTTP header (if applicable). See the |
||||
// :ref:`HTTP fault filter <config_http_filters_fault_injection_http_header>` documentation for |
||||
// more information. |
||||
message HeaderAbort { |
||||
} |
||||
|
||||
reserved 1; |
||||
|
||||
oneof error_type { |
||||
// HTTP status code to use to abort the HTTP request. |
||||
uint32 http_status = 2; |
||||
|
||||
// gRPC status code to use to abort the gRPC request. |
||||
uint32 grpc_status = 5; |
||||
|
||||
// Fault aborts are controlled via an HTTP header (if applicable). |
||||
HeaderAbort header_abort = 4; |
||||
} |
||||
|
||||
// The percentage of requests/operations/connections that will be aborted with the error code |
||||
// provided. |
||||
type.v3.FractionalPercent percentage = 3; |
||||
} |
||||
|
||||
// [#next-free-field: 15] |
||||
message HTTPFault { |
||||
// If specified, the filter will inject delays based on the values in the |
||||
// object. |
||||
common.fault.v3.FaultDelay delay = 1; |
||||
|
||||
// If specified, the filter will abort requests based on the values in |
||||
// the object. At least *abort* or *delay* must be specified. |
||||
FaultAbort abort = 2; |
||||
|
||||
// Specifies a set of headers that the filter should match on. The fault |
||||
// injection filter can be applied selectively to requests that match a set of |
||||
// headers specified in the fault filter config. The chances of actual fault |
||||
// injection further depend on the value of the :ref:`percentage |
||||
// <envoy_api_field_extensions.filters.http.fault.v3.FaultAbort.percentage>` field. |
||||
// The filter will check the request's headers against all the specified |
||||
// headers in the filter config. A match will happen if all the headers in the |
||||
// config are present in the request with the same values (or based on |
||||
// presence if the *value* field is not in the config). |
||||
repeated config.route.v3.HeaderMatcher headers = 4; |
||||
|
||||
// The maximum number of faults that can be active at a single time via the configured fault |
||||
// filter. Note that because this setting can be overridden at the route level, it's possible |
||||
// for the number of active faults to be greater than this value (if injected via a different |
||||
// route). If not specified, defaults to unlimited. This setting can be overridden via |
||||
// `runtime <config_http_filters_fault_injection_runtime>` and any faults that are not injected |
||||
// due to overflow will be indicated via the `faults_overflow |
||||
// <config_http_filters_fault_injection_stats>` stat. |
||||
// |
||||
// .. attention:: |
||||
// Like other :ref:`circuit breakers <arch_overview_circuit_break>` in Envoy, this is a fuzzy |
||||
// limit. It's possible for the number of active faults to rise slightly above the configured |
||||
// amount due to the implementation details. |
||||
google.protobuf.UInt32Value max_active_faults = 6; |
||||
} |
@ -0,0 +1,49 @@ |
||||
// Copyright 2020 The gRPC Authors |
||||
// |
||||
// Licensed under the Apache License, Version 2.0 (the "License"); |
||||
// you may not use this file except in compliance with the License. |
||||
// You may obtain a copy of the License at |
||||
// |
||||
// http://www.apache.org/licenses/LICENSE-2.0 |
||||
// |
||||
// Unless required by applicable law or agreed to in writing, software |
||||
// distributed under the License is distributed on an "AS IS" BASIS, |
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
// See the License for the specific language governing permissions and |
||||
// limitations under the License. |
||||
|
||||
// Local copy of Envoy xDS proto file, used for testing only. |
||||
|
||||
syntax = "proto3"; |
||||
|
||||
package envoy.extensions.filters.common.fault.v3; |
||||
|
||||
import "src/proto/grpc/testing/xds/v3/percent.proto"; |
||||
|
||||
import "google/protobuf/duration.proto"; |
||||
|
||||
// Delay specification is used to inject latency into the |
||||
// HTTP/gRPC/Mongo/Redis operation or delay proxying of TCP connections. |
||||
message FaultDelay { |
||||
// Fault delays are controlled via an HTTP header (if applicable). See the |
||||
// :ref:`HTTP fault filter <config_http_filters_fault_injection_http_header>` |
||||
// documentation for more information. |
||||
message HeaderDelay {} |
||||
|
||||
oneof fault_delay_secifier { |
||||
// Add a fixed delay before forwarding the operation upstream. See |
||||
// https://developers.google.com/protocol-buffers/docs/proto3#json for |
||||
// the JSON/YAML Duration mapping. For HTTP/Mongo/Redis, the specified |
||||
// delay will be injected before a new request/operation. For TCP |
||||
// connections, the proxying of the connection upstream will be delayed |
||||
// for the specified period. This is required if type is FIXED. |
||||
google.protobuf.Duration fixed_delay = 3; |
||||
|
||||
// Fault delays are controlled via an HTTP header (if applicable). |
||||
HeaderDelay header_delay = 5; |
||||
} |
||||
|
||||
// The percentage of operations/connections/requests on which the delay will |
||||
// be injected. |
||||
type.v3.FractionalPercent percentage = 4; |
||||
} |
Loading…
Reference in new issue