Преглед на файлове

Merge pull request #18746 from yashykt/svc_cfg2

Service config changes Part 2
Yash Tibrewal преди 6 години
родител
ревизия
6fac033568
променени са 24 файла, в които са добавени 1937 реда и са изтрити 851 реда
  1. 73 42
      src/core/ext/filters/client_channel/client_channel.cc
  2. 2 0
      src/core/ext/filters/client_channel/client_channel_plugin.cc
  3. 0 26
      src/core/ext/filters/client_channel/lb_policy.cc
  4. 13 24
      src/core/ext/filters/client_channel/lb_policy.h
  5. 57 24
      src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc
  6. 14 0
      src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc
  7. 14 0
      src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc
  8. 112 38
      src/core/ext/filters/client_channel/lb_policy/xds/xds.cc
  9. 4 1
      src/core/ext/filters/client_channel/lb_policy_factory.h
  10. 105 2
      src/core/ext/filters/client_channel/lb_policy_registry.cc
  11. 9 2
      src/core/ext/filters/client_channel/lb_policy_registry.h
  12. 402 187
      src/core/ext/filters/client_channel/resolver_result_parsing.cc
  13. 110 73
      src/core/ext/filters/client_channel/resolver_result_parsing.h
  14. 7 5
      src/core/ext/filters/client_channel/resolving_lb_policy.cc
  15. 14 12
      src/core/ext/filters/client_channel/resolving_lb_policy.h
  16. 25 54
      src/core/ext/filters/client_channel/service_config.cc
  17. 70 203
      src/core/ext/filters/client_channel/service_config.h
  18. 3 34
      src/core/ext/filters/client_channel/subchannel.cc
  19. 116 77
      src/core/ext/filters/message_size/message_size_filter.cc
  20. 33 0
      src/core/ext/filters/message_size/message_size_filter.h
  21. 3 0
      src/core/lib/channel/context.h
  22. 1 0
      src/core/lib/gprpp/optional.h
  23. 23 0
      src/core/lib/iomgr/error.h
  24. 727 47
      test/core/client_channel/service_config_test.cc

+ 73 - 42
src/core/ext/filters/client_channel/client_channel.cc

@@ -66,8 +66,7 @@
 #include "src/core/lib/transport/static_metadata.h"
 #include "src/core/lib/transport/status_metadata.h"
 
-using grpc_core::internal::ClientChannelMethodParams;
-using grpc_core::internal::ClientChannelMethodParamsTable;
+using grpc_core::internal::ClientChannelMethodParsedObject;
 using grpc_core::internal::ProcessedResolverResult;
 using grpc_core::internal::ServerRetryThrottleData;
 
@@ -157,10 +156,8 @@ class ChannelData {
   RefCountedPtr<ServerRetryThrottleData> retry_throttle_data() const {
     return retry_throttle_data_;
   }
-  RefCountedPtr<ClientChannelMethodParams> GetMethodParams(
-      const grpc_slice& path) {
-    if (method_params_table_ == nullptr) return nullptr;
-    return ServiceConfig::MethodConfigTableLookup(*method_params_table_, path);
+  RefCountedPtr<ServiceConfig> service_config() const {
+    return service_config_;
   }
 
   grpc_connectivity_state CheckConnectivityState(bool try_to_connect);
@@ -226,8 +223,8 @@ class ChannelData {
   ~ChannelData();
 
   static bool ProcessResolverResultLocked(
-      void* arg, Resolver::Result* result, const char** lb_policy_name,
-      RefCountedPtr<LoadBalancingPolicy::Config>* lb_policy_config);
+      void* arg, const Resolver::Result& result, const char** lb_policy_name,
+      RefCountedPtr<ParsedLoadBalancingConfig>* lb_policy_config);
 
   grpc_error* DoPingLocked(grpc_transport_op* op);
 
@@ -243,6 +240,7 @@ class ChannelData {
   const size_t per_rpc_retry_buffer_size_;
   grpc_channel_stack* owning_stack_;
   ClientChannelFactory* client_channel_factory_;
+  UniquePtr<char> server_name_;
   // Initialized shortly after construction.
   channelz::ClientChannelNode* channelz_node_ = nullptr;
 
@@ -255,7 +253,7 @@ class ChannelData {
   // Data from service config.
   bool received_service_config_data_ = false;
   RefCountedPtr<ServerRetryThrottleData> retry_throttle_data_;
-  RefCountedPtr<ClientChannelMethodParamsTable> method_params_table_;
+  RefCountedPtr<ServiceConfig> service_config_;
 
   //
   // Fields used in the control plane.  Guarded by combiner.
@@ -266,6 +264,7 @@ class ChannelData {
   OrphanablePtr<LoadBalancingPolicy> resolving_lb_policy_;
   grpc_connectivity_state_tracker state_tracker_;
   ExternalConnectivityWatcher::WatcherList external_connectivity_watcher_list_;
+  UniquePtr<char> health_check_service_name_;
 
   //
   // Fields accessed from both data plane and control plane combiners.
@@ -621,7 +620,8 @@ class CallData {
   grpc_call_context_element* call_context_;
 
   RefCountedPtr<ServerRetryThrottleData> retry_throttle_data_;
-  RefCountedPtr<ClientChannelMethodParams> method_params_;
+  ServiceConfig::CallData service_config_call_data_;
+  const ClientChannelMethodParsedObject* method_params_ = nullptr;
 
   RefCountedPtr<SubchannelCall> subchannel_call_;
 
@@ -764,11 +764,12 @@ class ChannelData::ServiceConfigSetter {
  public:
   ServiceConfigSetter(
       ChannelData* chand,
-      RefCountedPtr<ServerRetryThrottleData> retry_throttle_data,
-      RefCountedPtr<ClientChannelMethodParamsTable> method_params_table)
+      Optional<internal::ClientChannelGlobalParsedObject::RetryThrottling>
+          retry_throttle_data,
+      RefCountedPtr<ServiceConfig> service_config)
       : chand_(chand),
-        retry_throttle_data_(std::move(retry_throttle_data)),
-        method_params_table_(std::move(method_params_table)) {
+        retry_throttle_data_(retry_throttle_data),
+        service_config_(std::move(service_config)) {
     GRPC_CHANNEL_STACK_REF(chand->owning_stack_, "ServiceConfigSetter");
     GRPC_CLOSURE_INIT(&closure_, SetServiceConfigData, this,
                       grpc_combiner_scheduler(chand->data_plane_combiner_));
@@ -781,8 +782,14 @@ class ChannelData::ServiceConfigSetter {
     ChannelData* chand = self->chand_;
     // Update channel state.
     chand->received_service_config_data_ = true;
-    chand->retry_throttle_data_ = std::move(self->retry_throttle_data_);
-    chand->method_params_table_ = std::move(self->method_params_table_);
+    if (self->retry_throttle_data_.has_value()) {
+      chand->retry_throttle_data_ =
+          internal::ServerRetryThrottleMap::GetDataForServer(
+              chand->server_name_.get(),
+              self->retry_throttle_data_.value().max_milli_tokens,
+              self->retry_throttle_data_.value().milli_token_ratio);
+    }
+    chand->service_config_ = std::move(self->service_config_);
     // Apply service config to queued picks.
     for (QueuedPick* pick = chand->queued_picks_; pick != nullptr;
          pick = pick->next) {
@@ -796,8 +803,9 @@ class ChannelData::ServiceConfigSetter {
   }
 
   ChannelData* chand_;
-  RefCountedPtr<ServerRetryThrottleData> retry_throttle_data_;
-  RefCountedPtr<ClientChannelMethodParamsTable> method_params_table_;
+  Optional<internal::ClientChannelGlobalParsedObject::RetryThrottling>
+      retry_throttle_data_;
+  RefCountedPtr<ServiceConfig> service_config_;
   grpc_closure closure_;
 };
 
@@ -934,10 +942,18 @@ class ChannelData::ClientChannelControlHelper
   }
 
   Subchannel* CreateSubchannel(const grpc_channel_args& args) override {
-    grpc_arg arg = SubchannelPoolInterface::CreateChannelArg(
+    grpc_arg args_to_add[2];
+    int num_args_to_add = 0;
+    if (chand_->health_check_service_name_ != nullptr) {
+      args_to_add[0] = grpc_channel_arg_string_create(
+          const_cast<char*>("grpc.temp.health_check"),
+          const_cast<char*>(chand_->health_check_service_name_.get()));
+      num_args_to_add++;
+    }
+    args_to_add[num_args_to_add++] = SubchannelPoolInterface::CreateChannelArg(
         chand_->subchannel_pool_.get());
     grpc_channel_args* new_args =
-        grpc_channel_args_copy_and_add(&args, &arg, 1);
+        grpc_channel_args_copy_and_add(&args, args_to_add, num_args_to_add);
     Subchannel* subchannel =
         chand_->client_channel_factory_->CreateSubchannel(new_args);
     grpc_channel_args_destroy(new_args);
@@ -1050,6 +1066,12 @@ ChannelData::ChannelData(grpc_channel_element_args* args, grpc_error** error)
         "filter");
     return;
   }
+  grpc_uri* uri = grpc_uri_parse(server_uri, true);
+  if (uri != nullptr && uri->path[0] != '\0') {
+    server_name_.reset(
+        gpr_strdup(uri->path[0] == '/' ? uri->path + 1 : uri->path));
+  }
+  grpc_uri_destroy(uri);
   char* proxy_name = nullptr;
   grpc_channel_args* new_args = nullptr;
   grpc_proxy_mappers_map_name(server_uri, args->channel_args, &proxy_name,
@@ -1109,19 +1131,21 @@ ChannelData::~ChannelData() {
 // Synchronous callback from ResolvingLoadBalancingPolicy to process a
 // resolver result update.
 bool ChannelData::ProcessResolverResultLocked(
-    void* arg, Resolver::Result* result, const char** lb_policy_name,
-    RefCountedPtr<LoadBalancingPolicy::Config>* lb_policy_config) {
+    void* arg, const Resolver::Result& result, const char** lb_policy_name,
+    RefCountedPtr<ParsedLoadBalancingConfig>* lb_policy_config) {
   ChannelData* chand = static_cast<ChannelData*>(arg);
-  ProcessedResolverResult resolver_result(result, chand->enable_retries_);
-  UniquePtr<char> service_config_json = resolver_result.service_config_json();
+  ProcessedResolverResult resolver_result(result);
+  char* service_config_json = gpr_strdup(resolver_result.service_config_json());
   if (grpc_client_channel_routing_trace.enabled()) {
     gpr_log(GPR_INFO, "chand=%p: resolver returned service config: \"%s\"",
-            chand, service_config_json.get());
+            chand, service_config_json);
   }
+  chand->health_check_service_name_.reset(
+      gpr_strdup(resolver_result.health_check_service_name()));
   // Create service config setter to update channel state in the data
   // plane combiner.  Destroys itself when done.
   New<ServiceConfigSetter>(chand, resolver_result.retry_throttle_data(),
-                           resolver_result.method_params_table());
+                           resolver_result.service_config());
   // Swap out the data used by GetChannelInfo().
   bool service_config_changed;
   {
@@ -1131,9 +1155,9 @@ bool ChannelData::ProcessResolverResultLocked(
         ((service_config_json == nullptr) !=
          (chand->info_service_config_json_ == nullptr)) ||
         (service_config_json != nullptr &&
-         strcmp(service_config_json.get(),
-                chand->info_service_config_json_.get()) != 0);
-    chand->info_service_config_json_ = std::move(service_config_json);
+         strcmp(service_config_json, chand->info_service_config_json_.get()) !=
+             0);
+    chand->info_service_config_json_.reset(service_config_json);
   }
   // Return results.
   *lb_policy_name = chand->info_lb_policy_name_.get();
@@ -1840,8 +1864,7 @@ void CallData::DoRetry(grpc_call_element* elem,
                        grpc_millis server_pushback_ms) {
   ChannelData* chand = static_cast<ChannelData*>(elem->channel_data);
   GPR_ASSERT(method_params_ != nullptr);
-  const ClientChannelMethodParams::RetryPolicy* retry_policy =
-      method_params_->retry_policy();
+  const auto* retry_policy = method_params_->retry_policy();
   GPR_ASSERT(retry_policy != nullptr);
   // Reset subchannel call and connected subchannel.
   subchannel_call_.reset();
@@ -1849,7 +1872,7 @@ void CallData::DoRetry(grpc_call_element* elem,
   // Compute backoff delay.
   grpc_millis next_attempt_time;
   if (server_pushback_ms >= 0) {
-    next_attempt_time = grpc_core::ExecCtx::Get()->Now() + server_pushback_ms;
+    next_attempt_time = ExecCtx::Get()->Now() + server_pushback_ms;
     last_attempt_got_server_pushback_ = true;
   } else {
     if (num_attempts_completed_ == 1 || last_attempt_got_server_pushback_) {
@@ -1866,7 +1889,7 @@ void CallData::DoRetry(grpc_call_element* elem,
   if (grpc_client_channel_call_trace.enabled()) {
     gpr_log(GPR_INFO,
             "chand=%p calld=%p: retrying failed call in %" PRId64 " ms", chand,
-            this, next_attempt_time - grpc_core::ExecCtx::Get()->Now());
+            this, next_attempt_time - ExecCtx::Get()->Now());
   }
   // Schedule retry after computed delay.
   GRPC_CLOSURE_INIT(&pick_closure_, StartPickLocked, elem,
@@ -1883,8 +1906,7 @@ bool CallData::MaybeRetry(grpc_call_element* elem,
   ChannelData* chand = static_cast<ChannelData*>(elem->channel_data);
   // Get retry policy.
   if (method_params_ == nullptr) return false;
-  const ClientChannelMethodParams::RetryPolicy* retry_policy =
-      method_params_->retry_policy();
+  const auto* retry_policy = method_params_->retry_policy();
   if (retry_policy == nullptr) return false;
   // If we've already dispatched a retry from this call, return true.
   // This catches the case where the batch has multiple callbacks
@@ -3072,8 +3094,19 @@ void CallData::ApplyServiceConfigToCallLocked(grpc_call_element* elem) {
     gpr_log(GPR_INFO, "chand=%p calld=%p: applying service config to call",
             chand, this);
   }
+  // Store a ref to the service_config in service_config_call_data_. Also, save
+  // a pointer to this in the call_context so that all future filters can access
+  // it.
+  service_config_call_data_ =
+      ServiceConfig::CallData(chand->service_config(), path_);
+  if (service_config_call_data_.service_config() != nullptr) {
+    call_context_[GRPC_SERVICE_CONFIG_CALL_DATA].value =
+        &service_config_call_data_;
+    method_params_ = static_cast<ClientChannelMethodParsedObject*>(
+        service_config_call_data_.GetMethodParsedObject(
+            internal::ClientChannelServiceConfigParser::ParserIndex()));
+  }
   retry_throttle_data_ = chand->retry_throttle_data();
-  method_params_ = chand->GetMethodParams(path_);
   if (method_params_ != nullptr) {
     // If the deadline from the service config is shorter than the one
     // from the client API, reset the deadline timer.
@@ -3091,12 +3124,10 @@ void CallData::ApplyServiceConfigToCallLocked(grpc_call_element* elem) {
     uint32_t* send_initial_metadata_flags =
         &pending_batches_[0]
              .batch->payload->send_initial_metadata.send_initial_metadata_flags;
-    if (GPR_UNLIKELY(method_params_->wait_for_ready() !=
-                         ClientChannelMethodParams::WAIT_FOR_READY_UNSET &&
-                     !(*send_initial_metadata_flags &
-                       GRPC_INITIAL_METADATA_WAIT_FOR_READY_EXPLICITLY_SET))) {
-      if (method_params_->wait_for_ready() ==
-          ClientChannelMethodParams::WAIT_FOR_READY_TRUE) {
+    if (method_params_->wait_for_ready().has_value() &&
+        !(*send_initial_metadata_flags &
+          GRPC_INITIAL_METADATA_WAIT_FOR_READY_EXPLICITLY_SET)) {
+      if (method_params_->wait_for_ready().value()) {
         *send_initial_metadata_flags |= GRPC_INITIAL_METADATA_WAIT_FOR_READY;
       } else {
         *send_initial_metadata_flags &= ~GRPC_INITIAL_METADATA_WAIT_FOR_READY;

+ 2 - 0
src/core/ext/filters/client_channel/client_channel_plugin.cc

@@ -32,6 +32,7 @@
 #include "src/core/ext/filters/client_channel/lb_policy_registry.h"
 #include "src/core/ext/filters/client_channel/proxy_mapper_registry.h"
 #include "src/core/ext/filters/client_channel/resolver_registry.h"
+#include "src/core/ext/filters/client_channel/resolver_result_parsing.h"
 #include "src/core/ext/filters/client_channel/retry_throttle.h"
 #include "src/core/lib/surface/channel_init.h"
 
@@ -50,6 +51,7 @@ static bool append_filter(grpc_channel_stack_builder* builder, void* arg) {
 
 void grpc_client_channel_init(void) {
   grpc_core::ServiceConfig::Init();
+  grpc_core::internal::ClientChannelServiceConfigParser::Register();
   grpc_core::LoadBalancingPolicyRegistry::Builder::InitRegistry();
   grpc_core::ResolverRegistry::Builder::InitRegistry();
   grpc_core::internal::ServerRetryThrottleMap::Init();

+ 0 - 26
src/core/ext/filters/client_channel/lb_policy.cc

@@ -62,32 +62,6 @@ void LoadBalancingPolicy::ShutdownAndUnrefLocked(void* arg,
   policy->Unref();
 }
 
-grpc_json* LoadBalancingPolicy::ParseLoadBalancingConfig(
-    const grpc_json* lb_config_array) {
-  if (lb_config_array == nullptr || lb_config_array->type != GRPC_JSON_ARRAY) {
-    return nullptr;
-  }
-  // Find the first LB policy that this client supports.
-  for (const grpc_json* lb_config = lb_config_array->child;
-       lb_config != nullptr; lb_config = lb_config->next) {
-    if (lb_config->type != GRPC_JSON_OBJECT) return nullptr;
-    grpc_json* policy = nullptr;
-    for (grpc_json* field = lb_config->child; field != nullptr;
-         field = field->next) {
-      if (field->key == nullptr || field->type != GRPC_JSON_OBJECT)
-        return nullptr;
-      if (policy != nullptr) return nullptr;  // Violate "oneof" type.
-      policy = field;
-    }
-    if (policy == nullptr) return nullptr;
-    // If we support this policy, then select it.
-    if (LoadBalancingPolicyRegistry::LoadBalancingPolicyExists(policy->key)) {
-      return policy;
-    }
-  }
-  return nullptr;
-}
-
 //
 // LoadBalancingPolicy::UpdateArgs
 //

+ 13 - 24
src/core/ext/filters/client_channel/lb_policy.h

@@ -36,6 +36,18 @@ extern grpc_core::DebugOnlyTraceFlag grpc_trace_lb_policy_refcount;
 
 namespace grpc_core {
 
+/// Interface for parsed forms of load balancing configs found in a service
+/// config.
+class ParsedLoadBalancingConfig : public RefCounted<ParsedLoadBalancingConfig> {
+ public:
+  virtual ~ParsedLoadBalancingConfig() = default;
+
+  // Returns the load balancing policy name
+  virtual const char* name() const GRPC_ABSTRACT;
+
+  GRPC_ABSTRACT_BASE_CLASS;
+};
+
 /// Interface for load balancing policies.
 ///
 /// The following concepts are used here:
@@ -196,30 +208,11 @@ class LoadBalancingPolicy : public InternallyRefCounted<LoadBalancingPolicy> {
     GRPC_ABSTRACT_BASE_CLASS
   };
 
-  /// Configuration for an LB policy instance.
-  // TODO(roth): Find a better JSON representation for this API.
-  class Config : public RefCounted<Config> {
-   public:
-    Config(const grpc_json* lb_config,
-           RefCountedPtr<ServiceConfig> service_config)
-        : json_(lb_config), service_config_(std::move(service_config)) {}
-
-    const char* name() const { return json_->key; }
-    const grpc_json* config() const { return json_->child; }
-    RefCountedPtr<ServiceConfig> service_config() const {
-      return service_config_;
-    }
-
-   private:
-    const grpc_json* json_;
-    RefCountedPtr<ServiceConfig> service_config_;
-  };
-
   /// Data passed to the UpdateLocked() method when new addresses and
   /// config are available.
   struct UpdateArgs {
     ServerAddressList addresses;
-    RefCountedPtr<Config> config;
+    RefCountedPtr<ParsedLoadBalancingConfig> config;
     const grpc_channel_args* args = nullptr;
 
     // TODO(roth): Remove everything below once channel args is
@@ -290,10 +283,6 @@ class LoadBalancingPolicy : public InternallyRefCounted<LoadBalancingPolicy> {
 
   void Orphan() override;
 
-  /// Returns the JSON node of policy (with both policy name and config content)
-  /// given the JSON node of a LoadBalancingConfig array.
-  static grpc_json* ParseLoadBalancingConfig(const grpc_json* lb_config_array);
-
   // A picker that returns PICK_QUEUE for all picks.
   // Also calls the parent LB policy's ExitIdleLocked() method when the
   // first pick is seen.

+ 57 - 24
src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc

@@ -118,6 +118,21 @@ namespace {
 
 constexpr char kGrpclb[] = "grpclb";
 
+class ParsedGrpcLbConfig : public ParsedLoadBalancingConfig {
+ public:
+  explicit ParsedGrpcLbConfig(
+      RefCountedPtr<ParsedLoadBalancingConfig> child_policy)
+      : child_policy_(std::move(child_policy)) {}
+  const char* name() const override { return kGrpclb; }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> child_policy() const {
+    return child_policy_;
+  }
+
+ private:
+  RefCountedPtr<ParsedLoadBalancingConfig> child_policy_;
+};
+
 class GrpcLb : public LoadBalancingPolicy {
  public:
   explicit GrpcLb(Args args);
@@ -302,7 +317,6 @@ class GrpcLb : public LoadBalancingPolicy {
   // Helper functions used in UpdateLocked().
   void ProcessAddressesAndChannelArgsLocked(const ServerAddressList& addresses,
                                             const grpc_channel_args& args);
-  void ParseLbConfig(Config* grpclb_config);
   static void OnBalancerChannelConnectivityChangedLocked(void* arg,
                                                          grpc_error* error);
   void CancelBalancerChannelConnectivityWatchLocked();
@@ -380,7 +394,7 @@ class GrpcLb : public LoadBalancingPolicy {
   // until it reports READY, at which point it will be moved to child_policy_.
   OrphanablePtr<LoadBalancingPolicy> pending_child_policy_;
   // The child policy config.
-  RefCountedPtr<Config> child_policy_config_;
+  RefCountedPtr<ParsedLoadBalancingConfig> child_policy_config_;
   // Child policy in state READY.
   bool child_policy_ready_ = false;
 };
@@ -1373,7 +1387,13 @@ void GrpcLb::FillChildRefsForChannelz(
 
 void GrpcLb::UpdateLocked(UpdateArgs args) {
   const bool is_initial_update = lb_channel_ == nullptr;
-  ParseLbConfig(args.config.get());
+  auto* grpclb_config =
+      static_cast<const ParsedGrpcLbConfig*>(args.config.get());
+  if (grpclb_config != nullptr) {
+    child_policy_config_ = grpclb_config->child_policy();
+  } else {
+    child_policy_config_ = nullptr;
+  }
   ProcessAddressesAndChannelArgsLocked(args.addresses, *args.args);
   // Update the existing child policy.
   if (child_policy_ != nullptr) CreateOrUpdateChildPolicyLocked();
@@ -1462,27 +1482,6 @@ void GrpcLb::ProcessAddressesAndChannelArgsLocked(
   response_generator_->SetResponse(std::move(result));
 }
 
-void GrpcLb::ParseLbConfig(Config* grpclb_config) {
-  const grpc_json* child_policy = nullptr;
-  if (grpclb_config != nullptr) {
-    const grpc_json* grpclb_config_json = grpclb_config->config();
-    for (const grpc_json* field = grpclb_config_json; field != nullptr;
-         field = field->next) {
-      if (field->key == nullptr) return;
-      if (strcmp(field->key, "childPolicy") == 0) {
-        if (child_policy != nullptr) return;  // Duplicate.
-        child_policy = ParseLoadBalancingConfig(field);
-      }
-    }
-  }
-  if (child_policy != nullptr) {
-    child_policy_config_ =
-        MakeRefCounted<Config>(child_policy, grpclb_config->service_config());
-  } else {
-    child_policy_config_.reset();
-  }
-}
-
 void GrpcLb::OnBalancerChannelConnectivityChangedLocked(void* arg,
                                                         grpc_error* error) {
   GrpcLb* self = static_cast<GrpcLb*>(arg);
@@ -1800,6 +1799,40 @@ class GrpcLbFactory : public LoadBalancingPolicyFactory {
   }
 
   const char* name() const override { return kGrpclb; }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> ParseLoadBalancingConfig(
+      const grpc_json* json, grpc_error** error) const override {
+    GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+    if (json == nullptr) {
+      return RefCountedPtr<ParsedLoadBalancingConfig>(
+          New<ParsedGrpcLbConfig>(nullptr));
+    }
+    InlinedVector<grpc_error*, 2> error_list;
+    RefCountedPtr<ParsedLoadBalancingConfig> child_policy;
+    for (const grpc_json* field = json->child; field != nullptr;
+         field = field->next) {
+      if (field->key == nullptr) continue;
+      if (strcmp(field->key, "childPolicy") == 0) {
+        if (child_policy != nullptr) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:childPolicy error:Duplicate entry"));
+        }
+        grpc_error* parse_error = GRPC_ERROR_NONE;
+        child_policy = LoadBalancingPolicyRegistry::ParseLoadBalancingConfig(
+            field, &parse_error);
+        if (parse_error != GRPC_ERROR_NONE) {
+          error_list.push_back(parse_error);
+        }
+      }
+    }
+    if (error_list.empty()) {
+      return RefCountedPtr<ParsedLoadBalancingConfig>(
+          New<ParsedGrpcLbConfig>(std::move(child_policy)));
+    } else {
+      *error = GRPC_ERROR_CREATE_FROM_VECTOR("GrpcLb Parser", &error_list);
+      return nullptr;
+    }
+  }
 };
 
 }  // namespace

+ 14 - 0
src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc

@@ -526,6 +526,11 @@ void PickFirst::PickFirstSubchannelData::
   }
 }
 
+class ParsedPickFirstConfig : public ParsedLoadBalancingConfig {
+ public:
+  const char* name() const override { return kPickFirst; }
+};
+
 //
 // factory
 //
@@ -538,6 +543,15 @@ class PickFirstFactory : public LoadBalancingPolicyFactory {
   }
 
   const char* name() const override { return kPickFirst; }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> ParseLoadBalancingConfig(
+      const grpc_json* json, grpc_error** error) const override {
+    if (json != nullptr) {
+      GPR_DEBUG_ASSERT(strcmp(json->key, name()) == 0);
+    }
+    return RefCountedPtr<ParsedLoadBalancingConfig>(
+        New<ParsedPickFirstConfig>());
+  }
 };
 
 }  // namespace

+ 14 - 0
src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc

@@ -503,6 +503,11 @@ void RoundRobin::UpdateLocked(UpdateArgs args) {
   }
 }
 
+class ParsedRoundRobinConfig : public ParsedLoadBalancingConfig {
+ public:
+  const char* name() const override { return kRoundRobin; }
+};
+
 //
 // factory
 //
@@ -515,6 +520,15 @@ class RoundRobinFactory : public LoadBalancingPolicyFactory {
   }
 
   const char* name() const override { return kRoundRobin; }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> ParseLoadBalancingConfig(
+      const grpc_json* json, grpc_error** error) const override {
+    if (json != nullptr) {
+      GPR_DEBUG_ASSERT(strcmp(json->key, name()) == 0);
+    }
+    return RefCountedPtr<ParsedLoadBalancingConfig>(
+        New<ParsedRoundRobinConfig>());
+  }
 };
 
 }  // namespace

+ 112 - 38
src/core/ext/filters/client_channel/lb_policy/xds/xds.cc

@@ -120,6 +120,33 @@ constexpr char kXds[] = "xds_experimental";
 constexpr char kDefaultLocalityName[] = "xds_default_locality";
 constexpr uint32_t kDefaultLocalityWeight = 3;
 
+class ParsedXdsConfig : public ParsedLoadBalancingConfig {
+ public:
+  ParsedXdsConfig(const char* balancer_name,
+                  RefCountedPtr<ParsedLoadBalancingConfig> child_policy,
+                  RefCountedPtr<ParsedLoadBalancingConfig> fallback_policy)
+      : balancer_name_(balancer_name),
+        child_policy_(std::move(child_policy)),
+        fallback_policy_(std::move(fallback_policy)) {}
+
+  const char* name() const override { return kXds; }
+
+  const char* balancer_name() const { return balancer_name_; };
+
+  RefCountedPtr<ParsedLoadBalancingConfig> child_policy() const {
+    return child_policy_;
+  }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> fallback_policy() const {
+    return fallback_policy_;
+  }
+
+ private:
+  const char* balancer_name_ = nullptr;
+  RefCountedPtr<ParsedLoadBalancingConfig> child_policy_;
+  RefCountedPtr<ParsedLoadBalancingConfig> fallback_policy_;
+};
+
 class XdsLb : public LoadBalancingPolicy {
  public:
   explicit XdsLb(Args args);
@@ -336,7 +363,7 @@ class XdsLb : public LoadBalancingPolicy {
       ~LocalityEntry() = default;
 
       void UpdateLocked(xds_grpclb_serverlist* serverlist,
-                        LoadBalancingPolicy::Config* child_policy_config,
+                        ParsedLoadBalancingConfig* child_policy_config,
                         const grpc_channel_args* args);
       void ShutdownLocked();
       void ResetBackoffLocked();
@@ -383,7 +410,7 @@ class XdsLb : public LoadBalancingPolicy {
     };
 
     void UpdateLocked(const LocalityList& locality_list,
-                      LoadBalancingPolicy::Config* child_policy_config,
+                      ParsedLoadBalancingConfig* child_policy_config,
                       const grpc_channel_args* args, XdsLb* parent);
     void ShutdownLocked();
     void ResetBackoffLocked();
@@ -423,7 +450,7 @@ class XdsLb : public LoadBalancingPolicy {
   // If parsing succeeds, updates \a balancer_name, and updates \a
   // child_policy_config_ and \a fallback_policy_config_ if they are also
   // found. Does nothing upon failure.
-  void ParseLbConfig(Config* xds_config);
+  void ParseLbConfig(const ParsedXdsConfig* xds_config);
 
   BalancerChannelState* LatestLbChannel() const {
     return pending_lb_chand_ != nullptr ? pending_lb_chand_.get()
@@ -480,7 +507,7 @@ class XdsLb : public LoadBalancingPolicy {
   grpc_closure lb_on_fallback_;
 
   // The policy to use for the fallback backends.
-  RefCountedPtr<Config> fallback_policy_config_;
+  RefCountedPtr<ParsedLoadBalancingConfig> fallback_policy_config_;
   // Lock held when modifying the value of fallback_policy_ or
   // pending_fallback_policy_.
   Mutex fallback_policy_mu_;
@@ -489,7 +516,7 @@ class XdsLb : public LoadBalancingPolicy {
   OrphanablePtr<LoadBalancingPolicy> pending_fallback_policy_;
 
   // The policy to use for the backends.
-  RefCountedPtr<Config> child_policy_config_;
+  RefCountedPtr<ParsedLoadBalancingConfig> child_policy_config_;
   // Map of policies to use in the backend
   LocalityMap locality_map_;
   // TODO(mhaidry) : Add support for multiple maps of localities
@@ -1485,41 +1512,17 @@ void XdsLb::ProcessAddressesAndChannelArgsLocked(
   grpc_channel_args_destroy(lb_channel_args);
 }
 
-void XdsLb::ParseLbConfig(Config* xds_config) {
-  const grpc_json* xds_config_json = xds_config->config();
-  const char* balancer_name = nullptr;
-  grpc_json* child_policy = nullptr;
-  grpc_json* fallback_policy = nullptr;
-  for (const grpc_json* field = xds_config_json; field != nullptr;
-       field = field->next) {
-    if (field->key == nullptr) return;
-    if (strcmp(field->key, "balancerName") == 0) {
-      if (balancer_name != nullptr) return;  // Duplicate.
-      if (field->type != GRPC_JSON_STRING) return;
-      balancer_name = field->value;
-    } else if (strcmp(field->key, "childPolicy") == 0) {
-      if (child_policy != nullptr) return;  // Duplicate.
-      child_policy = ParseLoadBalancingConfig(field);
-    } else if (strcmp(field->key, "fallbackPolicy") == 0) {
-      if (fallback_policy != nullptr) return;  // Duplicate.
-      fallback_policy = ParseLoadBalancingConfig(field);
-    }
-  }
-  if (balancer_name == nullptr) return;  // Required field.
-  balancer_name_ = UniquePtr<char>(gpr_strdup(balancer_name));
-  if (child_policy != nullptr) {
-    child_policy_config_ =
-        MakeRefCounted<Config>(child_policy, xds_config->service_config());
-  }
-  if (fallback_policy != nullptr) {
-    fallback_policy_config_ =
-        MakeRefCounted<Config>(fallback_policy, xds_config->service_config());
-  }
+void XdsLb::ParseLbConfig(const ParsedXdsConfig* xds_config) {
+  if (xds_config == nullptr || xds_config->balancer_name() == nullptr) return;
+  // TODO(yashykt) : does this need to be a gpr_strdup
+  balancer_name_ = UniquePtr<char>(gpr_strdup(xds_config->balancer_name()));
+  child_policy_config_ = xds_config->child_policy();
+  fallback_policy_config_ = xds_config->fallback_policy();
 }
 
 void XdsLb::UpdateLocked(UpdateArgs args) {
   const bool is_initial_update = lb_chand_ == nullptr;
-  ParseLbConfig(args.config.get());
+  ParseLbConfig(static_cast<const ParsedXdsConfig*>(args.config.get()));
   if (balancer_name_ == nullptr) {
     gpr_log(GPR_ERROR, "[xdslb %p] LB config parsing fails.", this);
     return;
@@ -1745,7 +1748,7 @@ void XdsLb::LocalityMap::PruneLocalities(const LocalityList& locality_list) {
 
 void XdsLb::LocalityMap::UpdateLocked(
     const LocalityList& locality_serverlist,
-    LoadBalancingPolicy::Config* child_policy_config,
+    ParsedLoadBalancingConfig* child_policy_config,
     const grpc_channel_args* args, XdsLb* parent) {
   if (parent->shutting_down_) return;
   for (size_t i = 0; i < locality_serverlist.size(); i++) {
@@ -1840,7 +1843,7 @@ XdsLb::LocalityMap::LocalityEntry::CreateChildPolicyLocked(
 
 void XdsLb::LocalityMap::LocalityEntry::UpdateLocked(
     xds_grpclb_serverlist* serverlist,
-    LoadBalancingPolicy::Config* child_policy_config,
+    ParsedLoadBalancingConfig* child_policy_config,
     const grpc_channel_args* args_in) {
   if (parent_->shutting_down_) return;
   // Construct update args.
@@ -2155,6 +2158,77 @@ class XdsFactory : public LoadBalancingPolicyFactory {
   }
 
   const char* name() const override { return kXds; }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> ParseLoadBalancingConfig(
+      const grpc_json* json, grpc_error** error) const override {
+    GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+    if (json == nullptr) {
+      // xds was mentioned as a policy in the deprecated loadBalancingPolicy
+      // field or in the client API.
+      *error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+          "field:loadBalancingPolicy error:Xds Parser has required field - "
+          "balancerName. Please use loadBalancingConfig field of service "
+          "config instead.");
+      return nullptr;
+    }
+    GPR_DEBUG_ASSERT(strcmp(json->key, name()) == 0);
+
+    InlinedVector<grpc_error*, 3> error_list;
+    const char* balancer_name = nullptr;
+    RefCountedPtr<ParsedLoadBalancingConfig> child_policy;
+    RefCountedPtr<ParsedLoadBalancingConfig> fallback_policy;
+    for (const grpc_json* field = json->child; field != nullptr;
+         field = field->next) {
+      if (field->key == nullptr) continue;
+      if (strcmp(field->key, "balancerName") == 0) {
+        if (balancer_name != nullptr) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:balancerName error:Duplicate entry"));
+        }
+        if (field->type != GRPC_JSON_STRING) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:balancerName error:type should be string"));
+          continue;
+        }
+        balancer_name = field->value;
+      } else if (strcmp(field->key, "childPolicy") == 0) {
+        if (child_policy != nullptr) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:childPolicy error:Duplicate entry"));
+        }
+        grpc_error* parse_error = GRPC_ERROR_NONE;
+        child_policy = LoadBalancingPolicyRegistry::ParseLoadBalancingConfig(
+            field, &parse_error);
+        if (child_policy == nullptr) {
+          GPR_DEBUG_ASSERT(parse_error != GRPC_ERROR_NONE);
+          error_list.push_back(parse_error);
+        }
+      } else if (strcmp(field->key, "fallbackPolicy") == 0) {
+        if (fallback_policy != nullptr) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:fallbackPolicy error:Duplicate entry"));
+        }
+        grpc_error* parse_error = GRPC_ERROR_NONE;
+        fallback_policy = LoadBalancingPolicyRegistry::ParseLoadBalancingConfig(
+            field, &parse_error);
+        if (fallback_policy == nullptr) {
+          GPR_DEBUG_ASSERT(parse_error != GRPC_ERROR_NONE);
+          error_list.push_back(parse_error);
+        }
+      }
+    }
+    if (balancer_name == nullptr) {
+      error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+          "field:balancerName error:not found"));
+    }
+    if (error_list.empty()) {
+      return RefCountedPtr<ParsedLoadBalancingConfig>(New<ParsedXdsConfig>(
+          balancer_name, std::move(child_policy), std::move(fallback_policy)));
+    } else {
+      *error = GRPC_ERROR_CREATE_FROM_VECTOR("Xds Parser", &error_list);
+      return nullptr;
+    }
+  }
 };
 
 }  // namespace

+ 4 - 1
src/core/ext/filters/client_channel/lb_policy_factory.h

@@ -37,9 +37,12 @@ class LoadBalancingPolicyFactory {
   /// Caller does NOT take ownership of result.
   virtual const char* name() const GRPC_ABSTRACT;
 
+  virtual RefCountedPtr<ParsedLoadBalancingConfig> ParseLoadBalancingConfig(
+      const grpc_json* json, grpc_error** error) const GRPC_ABSTRACT;
+
   virtual ~LoadBalancingPolicyFactory() {}
 
-  GRPC_ABSTRACT_BASE_CLASS
+  GRPC_ABSTRACT_BASE_CLASS;
 };
 
 }  // namespace grpc_core

+ 105 - 2
src/core/ext/filters/client_channel/lb_policy_registry.cc

@@ -94,9 +94,112 @@ LoadBalancingPolicyRegistry::CreateLoadBalancingPolicy(
   return factory->CreateLoadBalancingPolicy(std::move(args));
 }
 
-bool LoadBalancingPolicyRegistry::LoadBalancingPolicyExists(const char* name) {
+bool LoadBalancingPolicyRegistry::LoadBalancingPolicyExists(
+    const char* name, bool* requires_config) {
   GPR_ASSERT(g_state != nullptr);
-  return g_state->GetLoadBalancingPolicyFactory(name) != nullptr;
+  auto* factory = g_state->GetLoadBalancingPolicyFactory(name);
+  if (factory == nullptr) {
+    return false;
+  }
+  if (requires_config != nullptr) {
+    grpc_error* error = GRPC_ERROR_NONE;
+    // Check if the load balancing policy allows an empty config
+    *requires_config =
+        factory->ParseLoadBalancingConfig(nullptr, &error) == nullptr;
+    GRPC_ERROR_UNREF(error);
+  }
+  return true;
+}
+
+namespace {
+// Returns the JSON node of policy (with both policy name and config content)
+// given the JSON node of a LoadBalancingConfig array.
+grpc_json* ParseLoadBalancingConfigHelper(const grpc_json* lb_config_array,
+                                          grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+  char* error_msg;
+  if (lb_config_array == nullptr || lb_config_array->type != GRPC_JSON_ARRAY) {
+    gpr_asprintf(&error_msg, "field:%s error:type should be array",
+                 lb_config_array->key);
+    *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+    gpr_free(error_msg);
+    return nullptr;
+  }
+  const char* field_name = lb_config_array->key;
+  // Find the first LB policy that this client supports.
+  for (const grpc_json* lb_config = lb_config_array->child;
+       lb_config != nullptr; lb_config = lb_config->next) {
+    if (lb_config->type != GRPC_JSON_OBJECT) {
+      gpr_asprintf(&error_msg,
+                   "field:%s error:child entry should be of type object",
+                   field_name);
+      *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+      gpr_free(error_msg);
+      return nullptr;
+    }
+    grpc_json* policy = nullptr;
+    for (grpc_json* field = lb_config->child; field != nullptr;
+         field = field->next) {
+      if (field->key == nullptr || field->type != GRPC_JSON_OBJECT) {
+        gpr_asprintf(&error_msg,
+                     "field:%s error:child entry should be of type object",
+                     field_name);
+        *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+        gpr_free(error_msg);
+        return nullptr;
+      }
+      if (policy != nullptr) {
+        gpr_asprintf(&error_msg, "field:%s error:oneOf violation", field_name);
+        *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+        gpr_free(error_msg);
+        return nullptr;
+      }  // Violate "oneof" type.
+      policy = field;
+    }
+    if (policy == nullptr) {
+      gpr_asprintf(&error_msg, "field:%s error:no policy found in child entry",
+                   field_name);
+      *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+      gpr_free(error_msg);
+      return nullptr;
+    }
+    // If we support this policy, then select it.
+    if (LoadBalancingPolicyRegistry::LoadBalancingPolicyExists(policy->key,
+                                                               nullptr)) {
+      return policy;
+    }
+  }
+  gpr_asprintf(&error_msg, "field:%s error:No known policy", field_name);
+  *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+  gpr_free(error_msg);
+  return nullptr;
+}
+}  // namespace
+
+RefCountedPtr<ParsedLoadBalancingConfig>
+LoadBalancingPolicyRegistry::ParseLoadBalancingConfig(const grpc_json* json,
+                                                      grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+  GPR_ASSERT(g_state != nullptr);
+  const grpc_json* policy = ParseLoadBalancingConfigHelper(json, error);
+  if (policy == nullptr) {
+    return nullptr;
+  } else {
+    GPR_DEBUG_ASSERT(*error == GRPC_ERROR_NONE && json != nullptr);
+    // Find factory.
+    LoadBalancingPolicyFactory* factory =
+        g_state->GetLoadBalancingPolicyFactory(policy->key);
+    if (factory == nullptr) {
+      char* msg;
+      gpr_asprintf(&msg, "field:%s error:Factory not found to create policy",
+                   json->key);
+      *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(msg);
+      gpr_free(msg);
+      return nullptr;
+    }
+    // Parse load balancing config via factory.
+    return factory->ParseLoadBalancingConfig(policy, error);
+  }
 }
 
 }  // namespace grpc_core

+ 9 - 2
src/core/ext/filters/client_channel/lb_policy_registry.h

@@ -49,8 +49,15 @@ class LoadBalancingPolicyRegistry {
       const char* name, LoadBalancingPolicy::Args args);
 
   /// Returns true if the LB policy factory specified by \a name exists in this
-  /// registry.
-  static bool LoadBalancingPolicyExists(const char* name);
+  /// registry. If the load balancing policy requires a config to be specified
+  /// then sets \a requires_config to true.
+  static bool LoadBalancingPolicyExists(const char* name,
+                                        bool* requires_config);
+
+  /// Returns a parsed object of the load balancing policy to be used from a
+  /// LoadBalancingConfig array \a json.
+  static RefCountedPtr<ParsedLoadBalancingConfig> ParseLoadBalancingConfig(
+      const grpc_json* json, grpc_error** error);
 };
 
 }  // namespace grpc_core

+ 402 - 187
src/core/ext/filters/client_channel/resolver_result_parsing.cc

@@ -35,6 +35,7 @@
 #include "src/core/lib/channel/status_util.h"
 #include "src/core/lib/gpr/string.h"
 #include "src/core/lib/gprpp/memory.h"
+#include "src/core/lib/gprpp/optional.h"
 #include "src/core/lib/uri/uri_parser.h"
 
 // As per the retry design, we do not allow more than 5 retry attempts.
@@ -43,80 +44,68 @@
 namespace grpc_core {
 namespace internal {
 
+namespace {
+size_t g_client_channel_service_config_parser_index;
+}
+
+size_t ClientChannelServiceConfigParser::ParserIndex() {
+  return g_client_channel_service_config_parser_index;
+}
+
+void ClientChannelServiceConfigParser::Register() {
+  g_client_channel_service_config_parser_index =
+      ServiceConfig::RegisterParser(UniquePtr<ServiceConfig::Parser>(
+          New<ClientChannelServiceConfigParser>()));
+}
+
 ProcessedResolverResult::ProcessedResolverResult(
-    Resolver::Result* resolver_result, bool parse_retry)
-    : service_config_(resolver_result->service_config) {
+    const Resolver::Result& resolver_result)
+    : service_config_(resolver_result.service_config) {
   // If resolver did not return a service config, use the default
   // specified via the client API.
   if (service_config_ == nullptr) {
     const char* service_config_json = grpc_channel_arg_get_string(
-        grpc_channel_args_find(resolver_result->args, GRPC_ARG_SERVICE_CONFIG));
+        grpc_channel_args_find(resolver_result.args, GRPC_ARG_SERVICE_CONFIG));
     if (service_config_json != nullptr) {
       grpc_error* error = GRPC_ERROR_NONE;
       service_config_ = ServiceConfig::Create(service_config_json, &error);
       // Error is currently unused.
       GRPC_ERROR_UNREF(error);
     }
-  } else {
-    // Add the service config JSON to channel args so that it's
-    // accessible in the subchannel.
-    // TODO(roth): Consider whether there's a better way to pass the
-    // service config down into the subchannel stack, such as maybe via
-    // call context or metadata.  This would avoid the problem of having
-    // to recreate all subchannels whenever the service config changes.
-    // It would also avoid the need to pass in the resolver result in
-    // mutable form, both here and in
-    // ResolvingLoadBalancingPolicy::ProcessResolverResultCallback().
-    grpc_arg arg = grpc_channel_arg_string_create(
-        const_cast<char*>(GRPC_ARG_SERVICE_CONFIG),
-        const_cast<char*>(service_config_->service_config_json()));
-    grpc_channel_args* new_args =
-        grpc_channel_args_copy_and_add(resolver_result->args, &arg, 1);
-    grpc_channel_args_destroy(resolver_result->args);
-    resolver_result->args = new_args;
   }
   // Process service config.
-  ProcessServiceConfig(*resolver_result, parse_retry);
-  // If no LB config was found above, just find the LB policy name then.
-  if (lb_policy_name_ == nullptr) ProcessLbPolicyName(*resolver_result);
+  const ClientChannelGlobalParsedObject* parsed_object = nullptr;
+  if (service_config_ != nullptr) {
+    parsed_object = static_cast<const ClientChannelGlobalParsedObject*>(
+        service_config_->GetParsedGlobalServiceConfigObject(
+            ClientChannelServiceConfigParser::ParserIndex()));
+    ProcessServiceConfig(resolver_result, parsed_object);
+  }
+  ProcessLbPolicy(resolver_result, parsed_object);
 }
 
 void ProcessedResolverResult::ProcessServiceConfig(
-    const Resolver::Result& resolver_result, bool parse_retry) {
-  if (service_config_ == nullptr) return;
-  service_config_json_ =
-      UniquePtr<char>(gpr_strdup(service_config_->service_config_json()));
-  if (parse_retry) {
-    const grpc_arg* channel_arg =
-        grpc_channel_args_find(resolver_result.args, GRPC_ARG_SERVER_URI);
-    const char* server_uri = grpc_channel_arg_get_string(channel_arg);
-    GPR_ASSERT(server_uri != nullptr);
-    grpc_uri* uri = grpc_uri_parse(server_uri, true);
-    GPR_ASSERT(uri->path[0] != '\0');
-    server_name_ = uri->path[0] == '/' ? uri->path + 1 : uri->path;
-    service_config_->ParseGlobalParams(ParseServiceConfig, this);
-    grpc_uri_destroy(uri);
-  } else {
-    service_config_->ParseGlobalParams(ParseServiceConfig, this);
+    const Resolver::Result& resolver_result,
+    const ClientChannelGlobalParsedObject* parsed_object) {
+  health_check_service_name_ = parsed_object->health_check_service_name();
+  service_config_json_ = service_config_->service_config_json();
+  if (parsed_object != nullptr) {
+    retry_throttle_data_ = parsed_object->retry_throttling();
   }
-  method_params_table_ = service_config_->CreateMethodConfigTable(
-      ClientChannelMethodParams::CreateFromJson);
 }
 
-void ProcessedResolverResult::ProcessLbPolicyName(
-    const Resolver::Result& resolver_result) {
-  // Prefer the LB policy name found in the service config. Note that this is
-  // checking the deprecated loadBalancingPolicy field, rather than the new
-  // loadBalancingConfig field.
-  if (service_config_ != nullptr) {
-    lb_policy_name_.reset(
-        gpr_strdup(service_config_->GetLoadBalancingPolicyName()));
-    // Convert to lower-case.
-    if (lb_policy_name_ != nullptr) {
-      char* lb_policy_name = lb_policy_name_.get();
-      for (size_t i = 0; i < strlen(lb_policy_name); ++i) {
-        lb_policy_name[i] = tolower(lb_policy_name[i]);
-      }
+void ProcessedResolverResult::ProcessLbPolicy(
+    const Resolver::Result& resolver_result,
+    const ClientChannelGlobalParsedObject* parsed_object) {
+  // Prefer the LB policy name found in the service config.
+  if (parsed_object != nullptr) {
+    if (parsed_object->parsed_lb_config() != nullptr) {
+      lb_policy_name_.reset(
+          gpr_strdup(parsed_object->parsed_lb_config()->name()));
+      lb_policy_config_ = parsed_object->parsed_lb_config();
+    } else {
+      lb_policy_name_.reset(
+          gpr_strdup(parsed_object->parsed_deprecated_lb_policy()));
     }
   }
   // Otherwise, find the LB policy name set by the client API.
@@ -152,97 +141,8 @@ void ProcessedResolverResult::ProcessLbPolicyName(
   }
 }
 
-void ProcessedResolverResult::ParseServiceConfig(
-    const grpc_json* field, ProcessedResolverResult* parsing_state) {
-  parsing_state->ParseLbConfigFromServiceConfig(field);
-  if (parsing_state->server_name_ != nullptr) {
-    parsing_state->ParseRetryThrottleParamsFromServiceConfig(field);
-  }
-}
-
-void ProcessedResolverResult::ParseLbConfigFromServiceConfig(
-    const grpc_json* field) {
-  if (lb_policy_config_ != nullptr) return;  // Already found.
-  if (field->key == nullptr || strcmp(field->key, "loadBalancingConfig") != 0) {
-    return;  // Not the LB config global parameter.
-  }
-  const grpc_json* policy =
-      LoadBalancingPolicy::ParseLoadBalancingConfig(field);
-  if (policy != nullptr) {
-    lb_policy_name_.reset(gpr_strdup(policy->key));
-    lb_policy_config_ =
-        MakeRefCounted<LoadBalancingPolicy::Config>(policy, service_config_);
-  }
-}
-
-void ProcessedResolverResult::ParseRetryThrottleParamsFromServiceConfig(
-    const grpc_json* field) {
-  if (strcmp(field->key, "retryThrottling") == 0) {
-    if (retry_throttle_data_ != nullptr) return;  // Duplicate.
-    if (field->type != GRPC_JSON_OBJECT) return;
-    int max_milli_tokens = 0;
-    int milli_token_ratio = 0;
-    for (grpc_json* sub_field = field->child; sub_field != nullptr;
-         sub_field = sub_field->next) {
-      if (sub_field->key == nullptr) return;
-      if (strcmp(sub_field->key, "maxTokens") == 0) {
-        if (max_milli_tokens != 0) return;  // Duplicate.
-        if (sub_field->type != GRPC_JSON_NUMBER) return;
-        max_milli_tokens = gpr_parse_nonnegative_int(sub_field->value);
-        if (max_milli_tokens == -1) return;
-        max_milli_tokens *= 1000;
-      } else if (strcmp(sub_field->key, "tokenRatio") == 0) {
-        if (milli_token_ratio != 0) return;  // Duplicate.
-        if (sub_field->type != GRPC_JSON_NUMBER) return;
-        // We support up to 3 decimal digits.
-        size_t whole_len = strlen(sub_field->value);
-        uint32_t multiplier = 1;
-        uint32_t decimal_value = 0;
-        const char* decimal_point = strchr(sub_field->value, '.');
-        if (decimal_point != nullptr) {
-          whole_len = static_cast<size_t>(decimal_point - sub_field->value);
-          multiplier = 1000;
-          size_t decimal_len = strlen(decimal_point + 1);
-          if (decimal_len > 3) decimal_len = 3;
-          if (!gpr_parse_bytes_to_uint32(decimal_point + 1, decimal_len,
-                                         &decimal_value)) {
-            return;
-          }
-          uint32_t decimal_multiplier = 1;
-          for (size_t i = 0; i < (3 - decimal_len); ++i) {
-            decimal_multiplier *= 10;
-          }
-          decimal_value *= decimal_multiplier;
-        }
-        uint32_t whole_value;
-        if (!gpr_parse_bytes_to_uint32(sub_field->value, whole_len,
-                                       &whole_value)) {
-          return;
-        }
-        milli_token_ratio =
-            static_cast<int>((whole_value * multiplier) + decimal_value);
-        if (milli_token_ratio <= 0) return;
-      }
-    }
-    retry_throttle_data_ =
-        grpc_core::internal::ServerRetryThrottleMap::GetDataForServer(
-            server_name_, max_milli_tokens, milli_token_ratio);
-  }
-}
-
 namespace {
 
-bool ParseWaitForReady(
-    grpc_json* field, ClientChannelMethodParams::WaitForReady* wait_for_ready) {
-  if (field->type != GRPC_JSON_TRUE && field->type != GRPC_JSON_FALSE) {
-    return false;
-  }
-  *wait_for_ready = field->type == GRPC_JSON_TRUE
-                        ? ClientChannelMethodParams::WAIT_FOR_READY_TRUE
-                        : ClientChannelMethodParams::WAIT_FOR_READY_FALSE;
-  return true;
-}
-
 // Parses a JSON field of the form generated for a google.proto.Duration
 // proto message, as per:
 //   https://developers.google.com/protocol-buffers/docs/proto3#json
@@ -275,18 +175,36 @@ bool ParseDuration(grpc_json* field, grpc_millis* duration) {
   return true;
 }
 
-UniquePtr<ClientChannelMethodParams::RetryPolicy> ParseRetryPolicy(
-    grpc_json* field) {
-  auto retry_policy = MakeUnique<ClientChannelMethodParams::RetryPolicy>();
-  if (field->type != GRPC_JSON_OBJECT) return nullptr;
+UniquePtr<ClientChannelMethodParsedObject::RetryPolicy> ParseRetryPolicy(
+    grpc_json* field, grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+  auto retry_policy =
+      MakeUnique<ClientChannelMethodParsedObject::RetryPolicy>();
+  if (field->type != GRPC_JSON_OBJECT) {
+    *error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+        "field:retryPolicy error:should be of type object");
+    return nullptr;
+  }
+  InlinedVector<grpc_error*, 4> error_list;
   for (grpc_json* sub_field = field->child; sub_field != nullptr;
        sub_field = sub_field->next) {
-    if (sub_field->key == nullptr) return nullptr;
+    if (sub_field->key == nullptr) continue;
     if (strcmp(sub_field->key, "maxAttempts") == 0) {
-      if (retry_policy->max_attempts != 0) return nullptr;  // Duplicate.
-      if (sub_field->type != GRPC_JSON_NUMBER) return nullptr;
+      if (retry_policy->max_attempts != 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxAttempts error:Duplicate entry"));
+      }  // Duplicate. Continue Parsing
+      if (sub_field->type != GRPC_JSON_NUMBER) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxAttempts error:should be of type number"));
+        continue;
+      }
       retry_policy->max_attempts = gpr_parse_nonnegative_int(sub_field->value);
-      if (retry_policy->max_attempts <= 1) return nullptr;
+      if (retry_policy->max_attempts <= 1) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxAttempts error:should be at least 2"));
+        continue;
+      }
       if (retry_policy->max_attempts > MAX_MAX_RETRY_ATTEMPTS) {
         gpr_log(GPR_ERROR,
                 "service config: clamped retryPolicy.maxAttempts at %d",
@@ -294,78 +212,375 @@ UniquePtr<ClientChannelMethodParams::RetryPolicy> ParseRetryPolicy(
         retry_policy->max_attempts = MAX_MAX_RETRY_ATTEMPTS;
       }
     } else if (strcmp(sub_field->key, "initialBackoff") == 0) {
-      if (retry_policy->initial_backoff > 0) return nullptr;  // Duplicate.
+      if (retry_policy->initial_backoff > 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:initialBackoff error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
       if (!ParseDuration(sub_field, &retry_policy->initial_backoff)) {
-        return nullptr;
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:initialBackoff error:Failed to parse"));
+        continue;
+      }
+      if (retry_policy->initial_backoff == 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:initialBackoff error:must be greater than 0"));
       }
-      if (retry_policy->initial_backoff == 0) return nullptr;
     } else if (strcmp(sub_field->key, "maxBackoff") == 0) {
-      if (retry_policy->max_backoff > 0) return nullptr;  // Duplicate.
+      if (retry_policy->max_backoff > 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxBackoff error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
       if (!ParseDuration(sub_field, &retry_policy->max_backoff)) {
-        return nullptr;
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxBackoff error:failed to parse"));
+        continue;
+      }
+      if (retry_policy->max_backoff == 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxBackoff error:should be greater than 0"));
       }
-      if (retry_policy->max_backoff == 0) return nullptr;
     } else if (strcmp(sub_field->key, "backoffMultiplier") == 0) {
-      if (retry_policy->backoff_multiplier != 0) return nullptr;  // Duplicate.
-      if (sub_field->type != GRPC_JSON_NUMBER) return nullptr;
+      if (retry_policy->backoff_multiplier != 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:backoffMultiplier error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      if (sub_field->type != GRPC_JSON_NUMBER) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:backoffMultiplier error:should be of type number"));
+        continue;
+      }
       if (sscanf(sub_field->value, "%f", &retry_policy->backoff_multiplier) !=
           1) {
-        return nullptr;
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:backoffMultiplier error:failed to parse"));
+        continue;
+      }
+      if (retry_policy->backoff_multiplier <= 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:backoffMultiplier error:should be greater than 0"));
       }
-      if (retry_policy->backoff_multiplier <= 0) return nullptr;
     } else if (strcmp(sub_field->key, "retryableStatusCodes") == 0) {
       if (!retry_policy->retryable_status_codes.Empty()) {
-        return nullptr;  // Duplicate.
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryableStatusCodes error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      if (sub_field->type != GRPC_JSON_ARRAY) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryableStatusCodes error:should be of type array"));
+        continue;
       }
-      if (sub_field->type != GRPC_JSON_ARRAY) return nullptr;
       for (grpc_json* element = sub_field->child; element != nullptr;
            element = element->next) {
-        if (element->type != GRPC_JSON_STRING) return nullptr;
+        if (element->type != GRPC_JSON_STRING) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:retryableStatusCodes error:status codes should be of type "
+              "string"));
+          continue;
+        }
         grpc_status_code status;
         if (!grpc_status_code_from_string(element->value, &status)) {
-          return nullptr;
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:retryableStatusCodes error:failed to parse status code"));
+          continue;
         }
         retry_policy->retryable_status_codes.Add(status);
       }
-      if (retry_policy->retryable_status_codes.Empty()) return nullptr;
+      if (retry_policy->retryable_status_codes.Empty()) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryableStatusCodes error:should be non-empty"));
+      };
     }
   }
   // Make sure required fields are set.
-  if (retry_policy->max_attempts == 0 || retry_policy->initial_backoff == 0 ||
-      retry_policy->max_backoff == 0 || retry_policy->backoff_multiplier == 0 ||
-      retry_policy->retryable_status_codes.Empty()) {
+  if (error_list.empty()) {
+    if (retry_policy->max_attempts == 0 || retry_policy->initial_backoff == 0 ||
+        retry_policy->max_backoff == 0 ||
+        retry_policy->backoff_multiplier == 0 ||
+        retry_policy->retryable_status_codes.Empty()) {
+      *error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+          "field:retryPolicy error:Missing required field(s)");
+      return nullptr;
+    }
+  }
+  *error = GRPC_ERROR_CREATE_FROM_VECTOR("retryPolicy", &error_list);
+  return *error == GRPC_ERROR_NONE ? std::move(retry_policy) : nullptr;
+}
+
+const char* ParseHealthCheckConfig(const grpc_json* field, grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+  const char* service_name = nullptr;
+  GPR_DEBUG_ASSERT(strcmp(field->key, "healthCheckConfig") == 0);
+  if (field->type != GRPC_JSON_OBJECT) {
+    *error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+        "field:healthCheckConfig error:should be of type object");
+    return nullptr;
+  }
+  InlinedVector<grpc_error*, 2> error_list;
+  for (grpc_json* sub_field = field->child; sub_field != nullptr;
+       sub_field = sub_field->next) {
+    if (sub_field->key == nullptr) {
+      GPR_DEBUG_ASSERT(false);
+      continue;
+    }
+    if (strcmp(sub_field->key, "serviceName") == 0) {
+      if (service_name != nullptr) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:serviceName error:Duplicate "
+            "entry"));
+      }  // Duplicate. Continue parsing
+      if (sub_field->type != GRPC_JSON_STRING) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:serviceName error:should be of type string"));
+        continue;
+      }
+      service_name = sub_field->value;
+    }
+  }
+  if (!error_list.empty()) {
     return nullptr;
   }
-  return retry_policy;
+  *error =
+      GRPC_ERROR_CREATE_FROM_VECTOR("field:healthCheckConfig", &error_list);
+  return service_name;
 }
 
 }  // namespace
 
-RefCountedPtr<ClientChannelMethodParams>
-ClientChannelMethodParams::CreateFromJson(const grpc_json* json) {
-  RefCountedPtr<ClientChannelMethodParams> method_params =
-      MakeRefCounted<ClientChannelMethodParams>();
+UniquePtr<ServiceConfig::ParsedConfig>
+ClientChannelServiceConfigParser::ParseGlobalParams(const grpc_json* json,
+                                                    grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+  InlinedVector<grpc_error*, 4> error_list;
+  RefCountedPtr<ParsedLoadBalancingConfig> parsed_lb_config;
+  UniquePtr<char> lb_policy_name;
+  Optional<ClientChannelGlobalParsedObject::RetryThrottling> retry_throttling;
+  const char* health_check_service_name = nullptr;
+  for (grpc_json* field = json->child; field != nullptr; field = field->next) {
+    if (field->key == nullptr) {
+      continue;  // Not the LB config global parameter
+    }
+    // Parsed Load balancing config
+    if (strcmp(field->key, "loadBalancingConfig") == 0) {
+      if (parsed_lb_config != nullptr) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:loadBalancingConfig error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      grpc_error* parse_error = GRPC_ERROR_NONE;
+      parsed_lb_config = LoadBalancingPolicyRegistry::ParseLoadBalancingConfig(
+          field, &parse_error);
+      if (parsed_lb_config == nullptr) {
+        error_list.push_back(parse_error);
+      }
+    }
+    // Parse deprecated loadBalancingPolicy
+    if (strcmp(field->key, "loadBalancingPolicy") == 0) {
+      if (lb_policy_name != nullptr) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:loadBalancingPolicy error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      if (field->type != GRPC_JSON_STRING) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:loadBalancingPolicy error:type should be string"));
+        continue;
+      }
+      lb_policy_name.reset(gpr_strdup(field->value));
+      char* lb_policy = lb_policy_name.get();
+      if (lb_policy != nullptr) {
+        for (size_t i = 0; i < strlen(lb_policy); ++i) {
+          lb_policy[i] = tolower(lb_policy[i]);
+        }
+      }
+      bool requires_config = false;
+      if (!LoadBalancingPolicyRegistry::LoadBalancingPolicyExists(
+              lb_policy, &requires_config)) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:loadBalancingPolicy error:Unknown lb policy"));
+      } else if (requires_config) {
+        char* error_msg;
+        gpr_asprintf(&error_msg,
+                     "field:loadBalancingPolicy error:%s requires a config. "
+                     "Please use loadBalancingConfig instead.",
+                     lb_policy);
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg));
+        gpr_free(error_msg);
+      }
+    }
+    // Parse retry throttling
+    if (strcmp(field->key, "retryThrottling") == 0) {
+      if (retry_throttling.has_value()) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryThrottling error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      if (field->type != GRPC_JSON_OBJECT) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryThrottling error:Type should be object"));
+        continue;
+      }
+      Optional<int> max_milli_tokens;
+      Optional<int> milli_token_ratio;
+      for (grpc_json* sub_field = field->child; sub_field != nullptr;
+           sub_field = sub_field->next) {
+        if (sub_field->key == nullptr) continue;
+        if (strcmp(sub_field->key, "maxTokens") == 0) {
+          if (max_milli_tokens.has_value()) {
+            error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                "field:retryThrottling field:maxTokens error:Duplicate "
+                "entry"));
+          }  // Duplicate, continue parsing.
+          if (sub_field->type != GRPC_JSON_NUMBER) {
+            error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                "field:retryThrottling field:maxTokens error:Type should be "
+                "number"));
+          } else {
+            max_milli_tokens.set(gpr_parse_nonnegative_int(sub_field->value) *
+                                 1000);
+            if (max_milli_tokens.value() <= 0) {
+              error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                  "field:retryThrottling field:maxTokens error:should be "
+                  "greater than zero"));
+            }
+          }
+        } else if (strcmp(sub_field->key, "tokenRatio") == 0) {
+          if (milli_token_ratio.has_value()) {
+            error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                "field:retryThrottling field:tokenRatio error:Duplicate "
+                "entry"));
+          }  // Duplicate, continue parsing.
+          if (sub_field->type != GRPC_JSON_NUMBER) {
+            error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                "field:retryThrottling field:tokenRatio error:type should be "
+                "number"));
+          } else {
+            // We support up to 3 decimal digits.
+            size_t whole_len = strlen(sub_field->value);
+            uint32_t multiplier = 1;
+            uint32_t decimal_value = 0;
+            const char* decimal_point = strchr(sub_field->value, '.');
+            if (decimal_point != nullptr) {
+              whole_len = static_cast<size_t>(decimal_point - sub_field->value);
+              multiplier = 1000;
+              size_t decimal_len = strlen(decimal_point + 1);
+              if (decimal_len > 3) decimal_len = 3;
+              if (!gpr_parse_bytes_to_uint32(decimal_point + 1, decimal_len,
+                                             &decimal_value)) {
+                error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                    "field:retryThrottling field:tokenRatio error:Failed "
+                    "parsing"));
+                continue;
+              }
+              uint32_t decimal_multiplier = 1;
+              for (size_t i = 0; i < (3 - decimal_len); ++i) {
+                decimal_multiplier *= 10;
+              }
+              decimal_value *= decimal_multiplier;
+            }
+            uint32_t whole_value;
+            if (!gpr_parse_bytes_to_uint32(sub_field->value, whole_len,
+                                           &whole_value)) {
+              error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                  "field:retryThrottling field:tokenRatio error:Failed "
+                  "parsing"));
+              continue;
+            }
+            milli_token_ratio.set(
+                static_cast<int>((whole_value * multiplier) + decimal_value));
+            if (milli_token_ratio.value() <= 0) {
+              error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                  "field:retryThrottling field:tokenRatio error:value should "
+                  "be greater than 0"));
+            }
+          }
+        }
+      }
+      ClientChannelGlobalParsedObject::RetryThrottling data;
+      if (!max_milli_tokens.has_value()) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryThrottling field:maxTokens error:Not found"));
+      } else {
+        data.max_milli_tokens = max_milli_tokens.value();
+      }
+      if (!milli_token_ratio.has_value()) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryThrottling field:tokenRatio error:Not found"));
+      } else {
+        data.milli_token_ratio = milli_token_ratio.value();
+      }
+      retry_throttling.set(data);
+    }
+    if (strcmp(field->key, "healthCheckConfig") == 0) {
+      if (health_check_service_name != nullptr) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:healthCheckConfig error:Duplicate entry"));
+      }  // Duplicate continue parsing
+      grpc_error* parsing_error = GRPC_ERROR_NONE;
+      health_check_service_name = ParseHealthCheckConfig(field, &parsing_error);
+      if (parsing_error != GRPC_ERROR_NONE) {
+        error_list.push_back(parsing_error);
+      }
+    }
+  }
+  *error = GRPC_ERROR_CREATE_FROM_VECTOR("Client channel global parser",
+                                         &error_list);
+  if (*error == GRPC_ERROR_NONE) {
+    return UniquePtr<ServiceConfig::ParsedConfig>(
+        New<ClientChannelGlobalParsedObject>(
+            std::move(parsed_lb_config), std::move(lb_policy_name),
+            retry_throttling, health_check_service_name));
+  }
+  return nullptr;
+}
+
+UniquePtr<ServiceConfig::ParsedConfig>
+ClientChannelServiceConfigParser::ParsePerMethodParams(const grpc_json* json,
+                                                       grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
+  InlinedVector<grpc_error*, 4> error_list;
+  Optional<bool> wait_for_ready;
+  grpc_millis timeout = 0;
+  UniquePtr<ClientChannelMethodParsedObject::RetryPolicy> retry_policy;
   for (grpc_json* field = json->child; field != nullptr; field = field->next) {
     if (field->key == nullptr) continue;
     if (strcmp(field->key, "waitForReady") == 0) {
-      if (method_params->wait_for_ready_ != WAIT_FOR_READY_UNSET) {
-        return nullptr;  // Duplicate.
-      }
-      if (!ParseWaitForReady(field, &method_params->wait_for_ready_)) {
-        return nullptr;
+      if (wait_for_ready.has_value()) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:waitForReady error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      if (field->type == GRPC_JSON_TRUE) {
+        wait_for_ready.set(true);
+      } else if (field->type == GRPC_JSON_FALSE) {
+        wait_for_ready.set(false);
+      } else {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:waitForReady error:Type should be true/false"));
       }
     } else if (strcmp(field->key, "timeout") == 0) {
-      if (method_params->timeout_ > 0) return nullptr;  // Duplicate.
-      if (!ParseDuration(field, &method_params->timeout_)) return nullptr;
+      if (timeout > 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:timeout error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      if (!ParseDuration(field, &timeout)) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:timeout error:Failed parsing"));
+      };
     } else if (strcmp(field->key, "retryPolicy") == 0) {
-      if (method_params->retry_policy_ != nullptr) {
-        return nullptr;  // Duplicate.
+      if (retry_policy != nullptr) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:retryPolicy error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
+      grpc_error* error = GRPC_ERROR_NONE;
+      retry_policy = ParseRetryPolicy(field, &error);
+      if (retry_policy == nullptr) {
+        error_list.push_back(error);
       }
-      method_params->retry_policy_ = ParseRetryPolicy(field);
-      if (method_params->retry_policy_ == nullptr) return nullptr;
     }
   }
-  return method_params;
+  *error = GRPC_ERROR_CREATE_FROM_VECTOR("Client channel parser", &error_list);
+  if (*error == GRPC_ERROR_NONE) {
+    return UniquePtr<ServiceConfig::ParsedConfig>(
+        New<ClientChannelMethodParsedObject>(timeout, wait_for_ready,
+                                             std::move(retry_policy)));
+  }
+  return nullptr;
 }
 
 }  // namespace internal

+ 110 - 73
src/core/ext/filters/client_channel/resolver_result_parsing.h

@@ -22,10 +22,12 @@
 #include <grpc/support/port_platform.h>
 
 #include "src/core/ext/filters/client_channel/lb_policy.h"
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
 #include "src/core/ext/filters/client_channel/resolver.h"
 #include "src/core/ext/filters/client_channel/retry_throttle.h"
 #include "src/core/ext/filters/client_channel/service_config.h"
 #include "src/core/lib/channel/status_util.h"
+#include "src/core/lib/gprpp/optional.h"
 #include "src/core/lib/gprpp/ref_counted.h"
 #include "src/core/lib/gprpp/ref_counted_ptr.h"
 #include "src/core/lib/iomgr/exec_ctx.h"  // for grpc_millis
@@ -35,44 +37,123 @@
 namespace grpc_core {
 namespace internal {
 
-class ClientChannelMethodParams;
+class ClientChannelGlobalParsedObject : public ServiceConfig::ParsedConfig {
+ public:
+  struct RetryThrottling {
+    intptr_t max_milli_tokens = 0;
+    intptr_t milli_token_ratio = 0;
+  };
+
+  ClientChannelGlobalParsedObject(
+      RefCountedPtr<ParsedLoadBalancingConfig> parsed_lb_config,
+      UniquePtr<char> parsed_deprecated_lb_policy,
+      const Optional<RetryThrottling>& retry_throttling,
+      const char* health_check_service_name)
+      : parsed_lb_config_(std::move(parsed_lb_config)),
+        parsed_deprecated_lb_policy_(std::move(parsed_deprecated_lb_policy)),
+        retry_throttling_(retry_throttling),
+        health_check_service_name_(health_check_service_name) {}
+
+  Optional<RetryThrottling> retry_throttling() const {
+    return retry_throttling_;
+  }
+
+  RefCountedPtr<ParsedLoadBalancingConfig> parsed_lb_config() const {
+    return parsed_lb_config_;
+  }
+
+  const char* parsed_deprecated_lb_policy() const {
+    return parsed_deprecated_lb_policy_.get();
+  }
+
+  const char* health_check_service_name() const {
+    return health_check_service_name_;
+  }
+
+ private:
+  RefCountedPtr<ParsedLoadBalancingConfig> parsed_lb_config_;
+  UniquePtr<char> parsed_deprecated_lb_policy_;
+  Optional<RetryThrottling> retry_throttling_;
+  const char* health_check_service_name_;
+};
+
+class ClientChannelMethodParsedObject : public ServiceConfig::ParsedConfig {
+ public:
+  struct RetryPolicy {
+    int max_attempts = 0;
+    grpc_millis initial_backoff = 0;
+    grpc_millis max_backoff = 0;
+    float backoff_multiplier = 0;
+    StatusCodeSet retryable_status_codes;
+  };
+
+  ClientChannelMethodParsedObject(grpc_millis timeout,
+                                  const Optional<bool>& wait_for_ready,
+                                  UniquePtr<RetryPolicy> retry_policy)
+      : timeout_(timeout),
+        wait_for_ready_(wait_for_ready),
+        retry_policy_(std::move(retry_policy)) {}
+
+  grpc_millis timeout() const { return timeout_; }
+
+  Optional<bool> wait_for_ready() const { return wait_for_ready_; }
+
+  const RetryPolicy* retry_policy() const { return retry_policy_.get(); }
+
+ private:
+  grpc_millis timeout_ = 0;
+  Optional<bool> wait_for_ready_;
+  UniquePtr<RetryPolicy> retry_policy_;
+};
+
+class ClientChannelServiceConfigParser : public ServiceConfig::Parser {
+ public:
+  UniquePtr<ServiceConfig::ParsedConfig> ParseGlobalParams(
+      const grpc_json* json, grpc_error** error) override;
+
+  UniquePtr<ServiceConfig::ParsedConfig> ParsePerMethodParams(
+      const grpc_json* json, grpc_error** error) override;
 
-// A table mapping from a method name to its method parameters.
-typedef SliceHashTable<RefCountedPtr<ClientChannelMethodParams>>
-    ClientChannelMethodParamsTable;
+  static size_t ParserIndex();
+  static void Register();
+};
 
-// A container of processed fields from the resolver result. Simplifies the
-// usage of resolver result.
+// TODO(yashykt): It would be cleaner to move this logic to the client_channel
+// code. A container of processed fields from the resolver result. Simplifies
+// the usage of resolver result.
 class ProcessedResolverResult {
  public:
   // Processes the resolver result and populates the relative members
-  // for later consumption. Tries to parse retry parameters only if parse_retry
-  // is true.
-  ProcessedResolverResult(Resolver::Result* resolver_result, bool parse_retry);
+  // for later consumption.
+  ProcessedResolverResult(const Resolver::Result& resolver_result);
 
   // Getters. Any managed object's ownership is transferred.
-  UniquePtr<char> service_config_json() {
-    return std::move(service_config_json_);
-  }
-  RefCountedPtr<ServerRetryThrottleData> retry_throttle_data() {
-    return std::move(retry_throttle_data_);
-  }
-  RefCountedPtr<ClientChannelMethodParamsTable> method_params_table() {
-    return std::move(method_params_table_);
-  }
+  const char* service_config_json() { return service_config_json_; }
+
+  RefCountedPtr<ServiceConfig> service_config() { return service_config_; }
+
   UniquePtr<char> lb_policy_name() { return std::move(lb_policy_name_); }
-  RefCountedPtr<LoadBalancingPolicy::Config> lb_policy_config() {
-    return std::move(lb_policy_config_);
+  RefCountedPtr<ParsedLoadBalancingConfig> lb_policy_config() {
+    return lb_policy_config_;
   }
 
+  Optional<ClientChannelGlobalParsedObject::RetryThrottling>
+  retry_throttle_data() {
+    return retry_throttle_data_;
+  }
+
+  const char* health_check_service_name() { return health_check_service_name_; }
+
  private:
   // Finds the service config; extracts LB config and (maybe) retry throttle
   // params from it.
-  void ProcessServiceConfig(const Resolver::Result& resolver_result,
-                            bool parse_retry);
+  void ProcessServiceConfig(
+      const Resolver::Result& resolver_result,
+      const ClientChannelGlobalParsedObject* parsed_object);
 
-  // Finds the LB policy name (when no LB config was found).
-  void ProcessLbPolicyName(const Resolver::Result& resolver_result);
+  // Extracts the LB policy.
+  void ProcessLbPolicy(const Resolver::Result& resolver_result,
+                       const ClientChannelGlobalParsedObject* parsed_object);
 
   // Parses the service config. Intended to be used by
   // ServiceConfig::ParseGlobalParams.
@@ -84,59 +165,15 @@ class ProcessedResolverResult {
   void ParseRetryThrottleParamsFromServiceConfig(const grpc_json* field);
 
   // Service config.
-  UniquePtr<char> service_config_json_;
+  const char* service_config_json_ = nullptr;
   RefCountedPtr<ServiceConfig> service_config_;
   // LB policy.
   UniquePtr<char> lb_policy_name_;
-  RefCountedPtr<LoadBalancingPolicy::Config> lb_policy_config_;
+  RefCountedPtr<ParsedLoadBalancingConfig> lb_policy_config_;
   // Retry throttle data.
-  char* server_name_ = nullptr;
-  RefCountedPtr<ServerRetryThrottleData> retry_throttle_data_;
-  // Method params table.
-  RefCountedPtr<ClientChannelMethodParamsTable> method_params_table_;
-};
-
-// The parameters of a method.
-class ClientChannelMethodParams : public RefCounted<ClientChannelMethodParams> {
- public:
-  enum WaitForReady {
-    WAIT_FOR_READY_UNSET = 0,
-    WAIT_FOR_READY_FALSE,
-    WAIT_FOR_READY_TRUE
-  };
-
-  struct RetryPolicy {
-    int max_attempts = 0;
-    grpc_millis initial_backoff = 0;
-    grpc_millis max_backoff = 0;
-    float backoff_multiplier = 0;
-    StatusCodeSet retryable_status_codes;
-  };
-
-  /// Creates a method_parameters object from \a json.
-  /// Intended for use with ServiceConfig::CreateMethodConfigTable().
-  static RefCountedPtr<ClientChannelMethodParams> CreateFromJson(
-      const grpc_json* json);
-
-  grpc_millis timeout() const { return timeout_; }
-  WaitForReady wait_for_ready() const { return wait_for_ready_; }
-  const RetryPolicy* retry_policy() const { return retry_policy_.get(); }
-
- private:
-  // So New() can call our private ctor.
-  template <typename T, typename... Args>
-  friend T* grpc_core::New(Args&&... args);
-
-  // So Delete() can call our private dtor.
-  template <typename T>
-  friend void grpc_core::Delete(T*);
-
-  ClientChannelMethodParams() {}
-  virtual ~ClientChannelMethodParams() {}
-
-  grpc_millis timeout_ = 0;
-  WaitForReady wait_for_ready_ = WAIT_FOR_READY_UNSET;
-  UniquePtr<RetryPolicy> retry_policy_;
+  Optional<ClientChannelGlobalParsedObject::RetryThrottling>
+      retry_throttle_data_;
+  const char* health_check_service_name_ = nullptr;
 };
 
 }  // namespace internal

+ 7 - 5
src/core/ext/filters/client_channel/resolving_lb_policy.cc

@@ -183,7 +183,8 @@ class ResolvingLoadBalancingPolicy::ResolvingControlHelper
 
 ResolvingLoadBalancingPolicy::ResolvingLoadBalancingPolicy(
     Args args, TraceFlag* tracer, UniquePtr<char> target_uri,
-    UniquePtr<char> child_policy_name, RefCountedPtr<Config> child_lb_config,
+    UniquePtr<char> child_policy_name,
+    RefCountedPtr<ParsedLoadBalancingConfig> child_lb_config,
     grpc_error** error)
     : LoadBalancingPolicy(std::move(args)),
       tracer_(tracer),
@@ -331,7 +332,8 @@ void ResolvingLoadBalancingPolicy::OnResolverError(grpc_error* error) {
 }
 
 void ResolvingLoadBalancingPolicy::CreateOrUpdateLbPolicyLocked(
-    const char* lb_policy_name, RefCountedPtr<Config> lb_policy_config,
+    const char* lb_policy_name,
+    RefCountedPtr<ParsedLoadBalancingConfig> lb_policy_config,
     Resolver::Result result, TraceStringVector* trace_strings) {
   // If the child policy name changes, we need to create a new child
   // policy.  When this happens, we leave child_policy_ as-is and store
@@ -528,11 +530,11 @@ void ResolvingLoadBalancingPolicy::OnResolverResultChangedLocked(
   const bool resolution_contains_addresses = result.addresses.size() > 0;
   // Process the resolver result.
   const char* lb_policy_name = nullptr;
-  RefCountedPtr<Config> lb_policy_config;
+  RefCountedPtr<ParsedLoadBalancingConfig> lb_policy_config;
   bool service_config_changed = false;
   if (process_resolver_result_ != nullptr) {
     service_config_changed =
-        process_resolver_result_(process_resolver_result_user_data_, &result,
+        process_resolver_result_(process_resolver_result_user_data_, result,
                                  &lb_policy_name, &lb_policy_config);
   } else {
     lb_policy_name = child_policy_name_.get();
@@ -540,7 +542,7 @@ void ResolvingLoadBalancingPolicy::OnResolverResultChangedLocked(
   }
   GPR_ASSERT(lb_policy_name != nullptr);
   // Create or update LB policy, as needed.
-  CreateOrUpdateLbPolicyLocked(lb_policy_name, std::move(lb_policy_config),
+  CreateOrUpdateLbPolicyLocked(lb_policy_name, lb_policy_config,
                                std::move(result), &trace_strings);
   // Add channel trace event.
   if (channelz_node() != nullptr) {

+ 14 - 12
src/core/ext/filters/client_channel/resolving_lb_policy.h

@@ -23,6 +23,7 @@
 
 #include "src/core/ext/filters/client_channel/client_channel_channelz.h"
 #include "src/core/ext/filters/client_channel/lb_policy.h"
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
 #include "src/core/ext/filters/client_channel/resolver.h"
 #include "src/core/lib/channel/channel_args.h"
 #include "src/core/lib/channel/channel_stack.h"
@@ -53,11 +54,11 @@ class ResolvingLoadBalancingPolicy : public LoadBalancingPolicy {
  public:
   // If error is set when this returns, then construction failed, and
   // the caller may not use the new object.
-  ResolvingLoadBalancingPolicy(Args args, TraceFlag* tracer,
-                               UniquePtr<char> target_uri,
-                               UniquePtr<char> child_policy_name,
-                               RefCountedPtr<Config> child_lb_config,
-                               grpc_error** error);
+  ResolvingLoadBalancingPolicy(
+      Args args, TraceFlag* tracer, UniquePtr<char> target_uri,
+      UniquePtr<char> child_policy_name,
+      RefCountedPtr<ParsedLoadBalancingConfig> child_lb_config,
+      grpc_error** error);
 
   // Private ctor, to be used by client_channel only!
   //
@@ -65,8 +66,9 @@ class ResolvingLoadBalancingPolicy : public LoadBalancingPolicy {
   // lb_policy_name and lb_policy_config to point to the right data.
   // Returns true if the service config has changed since the last result.
   typedef bool (*ProcessResolverResultCallback)(
-      void* user_data, Resolver::Result* result, const char** lb_policy_name,
-      RefCountedPtr<Config>* lb_policy_config);
+      void* user_data, const Resolver::Result& result,
+      const char** lb_policy_name,
+      RefCountedPtr<ParsedLoadBalancingConfig>* lb_policy_config);
   // If error is set when this returns, then construction failed, and
   // the caller may not use the new object.
   ResolvingLoadBalancingPolicy(
@@ -102,10 +104,10 @@ class ResolvingLoadBalancingPolicy : public LoadBalancingPolicy {
 
   void StartResolvingLocked();
   void OnResolverError(grpc_error* error);
-  void CreateOrUpdateLbPolicyLocked(const char* lb_policy_name,
-                                    RefCountedPtr<Config> lb_policy_config,
-                                    Resolver::Result result,
-                                    TraceStringVector* trace_strings);
+  void CreateOrUpdateLbPolicyLocked(
+      const char* lb_policy_name,
+      RefCountedPtr<ParsedLoadBalancingConfig> lb_policy_config,
+      Resolver::Result result, TraceStringVector* trace_strings);
   OrphanablePtr<LoadBalancingPolicy> CreateLbPolicyLocked(
       const char* lb_policy_name, const grpc_channel_args& args,
       TraceStringVector* trace_strings);
@@ -121,7 +123,7 @@ class ResolvingLoadBalancingPolicy : public LoadBalancingPolicy {
   ProcessResolverResultCallback process_resolver_result_ = nullptr;
   void* process_resolver_result_user_data_ = nullptr;
   UniquePtr<char> child_policy_name_;
-  RefCountedPtr<Config> child_lb_config_;
+  RefCountedPtr<ParsedLoadBalancingConfig> child_lb_config_;
 
   // Resolver and associated state.
   OrphanablePtr<Resolver> resolver_;

+ 25 - 54
src/core/ext/filters/client_channel/service_config.cc

@@ -34,28 +34,10 @@
 namespace grpc_core {
 
 namespace {
-typedef InlinedVector<UniquePtr<ServiceConfigParser>,
+typedef InlinedVector<UniquePtr<ServiceConfig::Parser>,
                       ServiceConfig::kNumPreallocatedParsers>
     ServiceConfigParserList;
-ServiceConfigParserList* registered_parsers;
-
-// Consumes all the errors in the vector and forms a referencing error from
-// them. If the vector is empty, return GRPC_ERROR_NONE.
-template <size_t N>
-grpc_error* CreateErrorFromVector(const char* desc,
-                                  InlinedVector<grpc_error*, N>* error_list) {
-  grpc_error* error = GRPC_ERROR_NONE;
-  if (error_list->size() != 0) {
-    error = GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
-        desc, error_list->data(), error_list->size());
-    // Remove refs to all errors in error_list.
-    for (size_t i = 0; i < error_list->size(); i++) {
-      GRPC_ERROR_UNREF((*error_list)[i]);
-    }
-    error_list->clear();
-  }
-  return error;
-}
+ServiceConfigParserList* g_registered_parsers;
 }  // namespace
 
 RefCountedPtr<ServiceConfig> ServiceConfig::Create(const char* json,
@@ -107,16 +89,16 @@ grpc_error* ServiceConfig::ParseGlobalParams(const grpc_json* json_tree) {
   GPR_DEBUG_ASSERT(json_tree_->type == GRPC_JSON_OBJECT);
   GPR_DEBUG_ASSERT(json_tree_->key == nullptr);
   InlinedVector<grpc_error*, 4> error_list;
-  for (size_t i = 0; i < registered_parsers->size(); i++) {
+  for (size_t i = 0; i < g_registered_parsers->size(); i++) {
     grpc_error* parser_error = GRPC_ERROR_NONE;
     auto parsed_obj =
-        (*registered_parsers)[i]->ParseGlobalParams(json_tree, &parser_error);
+        (*g_registered_parsers)[i]->ParseGlobalParams(json_tree, &parser_error);
     if (parser_error != GRPC_ERROR_NONE) {
       error_list.push_back(parser_error);
     }
     parsed_global_service_config_objects_.push_back(std::move(parsed_obj));
   }
-  return CreateErrorFromVector("Global Params", &error_list);
+  return GRPC_ERROR_CREATE_FROM_VECTOR("Global Params", &error_list);
 }
 
 grpc_error* ServiceConfig::ParseJsonMethodConfigToServiceConfigObjectsTable(
@@ -125,17 +107,20 @@ grpc_error* ServiceConfig::ParseJsonMethodConfigToServiceConfigObjectsTable(
     size_t* idx) {
   auto objs_vector = MakeUnique<ServiceConfigObjectsVector>();
   InlinedVector<grpc_error*, 4> error_list;
-  for (size_t i = 0; i < registered_parsers->size(); i++) {
+  for (size_t i = 0; i < g_registered_parsers->size(); i++) {
     grpc_error* parser_error = GRPC_ERROR_NONE;
     auto parsed_obj =
-        (*registered_parsers)[i]->ParsePerMethodParams(json, &parser_error);
+        (*g_registered_parsers)[i]->ParsePerMethodParams(json, &parser_error);
     if (parser_error != GRPC_ERROR_NONE) {
       error_list.push_back(parser_error);
     }
     objs_vector->push_back(std::move(parsed_obj));
   }
-  const auto* vector_ptr = objs_vector.get();
   service_config_objects_vectors_storage_.push_back(std::move(objs_vector));
+  const auto* vector_ptr =
+      service_config_objects_vectors_storage_
+          [service_config_objects_vectors_storage_.size() - 1]
+              .get();
   // Construct list of paths.
   InlinedVector<UniquePtr<char>, 10> paths;
   for (grpc_json* child = json->child; child != nullptr; child = child->next) {
@@ -169,7 +154,7 @@ grpc_error* ServiceConfig::ParseJsonMethodConfigToServiceConfigObjectsTable(
     ++*idx;
   }
 wrap_error:
-  return CreateErrorFromVector("methodConfig", &error_list);
+  return GRPC_ERROR_CREATE_FROM_VECTOR("methodConfig", &error_list);
 }
 
 grpc_error* ServiceConfig::ParsePerMethodParams(const grpc_json* json_tree) {
@@ -226,28 +211,11 @@ grpc_error* ServiceConfig::ParsePerMethodParams(const grpc_json* json_tree) {
             num_entries, entries, nullptr);
     gpr_free(entries);
   }
-  return CreateErrorFromVector("Method Params", &error_list);
+  return GRPC_ERROR_CREATE_FROM_VECTOR("Method Params", &error_list);
 }
 
 ServiceConfig::~ServiceConfig() { grpc_json_destroy(json_tree_); }
 
-const char* ServiceConfig::GetLoadBalancingPolicyName() const {
-  if (json_tree_->type != GRPC_JSON_OBJECT || json_tree_->key != nullptr) {
-    return nullptr;
-  }
-  const char* lb_policy_name = nullptr;
-  for (grpc_json* field = json_tree_->child; field != nullptr;
-       field = field->next) {
-    if (field->key == nullptr) return nullptr;
-    if (strcmp(field->key, "loadBalancingPolicy") == 0) {
-      if (lb_policy_name != nullptr) return nullptr;  // Duplicate.
-      if (field->type != GRPC_JSON_STRING) return nullptr;
-      lb_policy_name = field->value;
-    }
-  }
-  return lb_policy_name;
-}
-
 int ServiceConfig::CountNamesInMethodConfig(grpc_json* json) {
   int num_names = 0;
   for (grpc_json* field = json->child; field != nullptr; field = field->next) {
@@ -319,8 +287,11 @@ UniquePtr<char> ServiceConfig::ParseJsonMethodName(grpc_json* json,
   return UniquePtr<char>(path);
 }
 
-const ServiceConfig::ServiceConfigObjectsVector* const*
+const ServiceConfig::ServiceConfigObjectsVector*
 ServiceConfig::GetMethodServiceConfigObjectsVector(const grpc_slice& path) {
+  if (parsed_method_service_config_objects_table_.get() == nullptr) {
+    return nullptr;
+  }
   const auto* value = parsed_method_service_config_objects_table_->Get(path);
   // If we didn't find a match for the path, try looking for a wildcard
   // entry (i.e., change "/service/method" to "/service/*").
@@ -339,22 +310,22 @@ ServiceConfig::GetMethodServiceConfigObjectsVector(const grpc_slice& path) {
     gpr_free(path_str);
     if (value == nullptr) return nullptr;
   }
-  return value;
+  return *value;
 }
 
-size_t ServiceConfig::RegisterParser(UniquePtr<ServiceConfigParser> parser) {
-  registered_parsers->push_back(std::move(parser));
-  return registered_parsers->size() - 1;
+size_t ServiceConfig::RegisterParser(UniquePtr<Parser> parser) {
+  g_registered_parsers->push_back(std::move(parser));
+  return g_registered_parsers->size() - 1;
 }
 
 void ServiceConfig::Init() {
-  GPR_ASSERT(registered_parsers == nullptr);
-  registered_parsers = New<ServiceConfigParserList>();
+  GPR_ASSERT(g_registered_parsers == nullptr);
+  g_registered_parsers = New<ServiceConfigParserList>();
 }
 
 void ServiceConfig::Shutdown() {
-  Delete(registered_parsers);
-  registered_parsers = nullptr;
+  Delete(g_registered_parsers);
+  g_registered_parsers = nullptr;
 }
 
 }  // namespace grpc_core

+ 70 - 203
src/core/ext/filters/client_channel/service_config.h

@@ -55,41 +55,73 @@
 
 namespace grpc_core {
 
-/// This is the base class that all service config parsers MUST use to store
-/// parsed service config data.
-class ServiceConfigParsedObject {
+class ServiceConfig : public RefCounted<ServiceConfig> {
  public:
-  virtual ~ServiceConfigParsedObject() = default;
+  /// This is the base class that all service config parsers MUST use to store
+  /// parsed service config data.
+  class ParsedConfig {
+   public:
+    virtual ~ParsedConfig() = default;
+
+    GRPC_ABSTRACT_BASE_CLASS;
+  };
+
+  /// This is the base class that all service config parsers should derive from.
+  class Parser {
+   public:
+    virtual ~Parser() = default;
+
+    virtual UniquePtr<ParsedConfig> ParseGlobalParams(const grpc_json* json,
+                                                      grpc_error** error) {
+      GPR_DEBUG_ASSERT(error != nullptr);
+      return nullptr;
+    }
 
-  GRPC_ABSTRACT_BASE_CLASS;
-};
+    virtual UniquePtr<ParsedConfig> ParsePerMethodParams(const grpc_json* json,
+                                                         grpc_error** error) {
+      GPR_DEBUG_ASSERT(error != nullptr);
+      return nullptr;
+    }
 
-/// This is the base class that all service config parsers should derive from.
-class ServiceConfigParser {
- public:
-  virtual ~ServiceConfigParser() = default;
+    GRPC_ABSTRACT_BASE_CLASS;
+  };
 
-  virtual UniquePtr<ServiceConfigParsedObject> ParseGlobalParams(
-      const grpc_json* json, grpc_error** error) {
-    GPR_DEBUG_ASSERT(error != nullptr);
-    return nullptr;
-  }
+  static constexpr int kNumPreallocatedParsers = 4;
+  typedef InlinedVector<UniquePtr<ParsedConfig>, kNumPreallocatedParsers>
+      ServiceConfigObjectsVector;
 
-  virtual UniquePtr<ServiceConfigParsedObject> ParsePerMethodParams(
-      const grpc_json* json, grpc_error** error) {
-    GPR_DEBUG_ASSERT(error != nullptr);
-    return nullptr;
-  }
+  /// When a service config is applied to a call in the client_channel_filter,
+  /// we create an instance of this object and store it in the call_data for
+  /// client_channel. A pointer to this object is also stored in the
+  /// call_context, so that future filters can easily access method and global
+  /// parameters for the call.
+  class CallData {
+   public:
+    CallData() = default;
+    CallData(RefCountedPtr<ServiceConfig> svc_cfg, const grpc_slice& path)
+        : service_config_(std::move(svc_cfg)) {
+      if (service_config_ != nullptr) {
+        method_params_vector_ =
+            service_config_->GetMethodServiceConfigObjectsVector(path);
+      }
+    }
 
-  GRPC_ABSTRACT_BASE_CLASS;
-};
+    ServiceConfig* service_config() { return service_config_.get(); }
 
-class ServiceConfig : public RefCounted<ServiceConfig> {
- public:
-  static constexpr int kNumPreallocatedParsers = 4;
-  typedef InlinedVector<UniquePtr<ServiceConfigParsedObject>,
-                        kNumPreallocatedParsers>
-      ServiceConfigObjectsVector;
+    ParsedConfig* GetMethodParsedObject(size_t index) const {
+      return method_params_vector_ != nullptr
+                 ? (*method_params_vector_)[index].get()
+                 : nullptr;
+    }
+
+    ParsedConfig* GetGlobalParsedObject(size_t index) const {
+      return service_config_->GetParsedGlobalServiceConfigObject(index);
+    }
+
+   private:
+    RefCountedPtr<ServiceConfig> service_config_;
+    const ServiceConfigObjectsVector* method_params_vector_ = nullptr;
+  };
 
   /// Creates a new service config from parsing \a json_string.
   /// Returns null on parse error.
@@ -100,48 +132,18 @@ class ServiceConfig : public RefCounted<ServiceConfig> {
 
   const char* service_config_json() const { return service_config_json_.get(); }
 
-  /// Invokes \a process_json() for each global parameter in the service
-  /// config.  \a arg is passed as the second argument to \a process_json().
-  template <typename T>
-  using ProcessJson = void (*)(const grpc_json*, T*);
-  template <typename T>
-  void ParseGlobalParams(ProcessJson<T> process_json, T* arg) const;
-
-  /// Gets the LB policy name from \a service_config.
-  /// Returns NULL if no LB policy name was specified.
-  /// Caller does NOT take ownership.
-  const char* GetLoadBalancingPolicyName() const;
-
-  /// Creates a method config table based on the data in \a json.
-  /// The table's keys are request paths.  The table's value type is
-  /// returned by \a create_value(), based on data parsed from the JSON tree.
-  /// Returns null on error.
-  template <typename T>
-  using CreateValue = RefCountedPtr<T> (*)(const grpc_json* method_config_json);
-  template <typename T>
-  RefCountedPtr<SliceHashTable<RefCountedPtr<T>>> CreateMethodConfigTable(
-      CreateValue<T> create_value) const;
-
-  /// A helper function for looking up values in the table returned by
-  /// \a CreateMethodConfigTable().
-  /// Gets the method config for the specified \a path, which should be of
-  /// the form "/service/method".
-  /// Returns null if the method has no config.
-  /// Caller does NOT own a reference to the result.
-  template <typename T>
-  static RefCountedPtr<T> MethodConfigTableLookup(
-      const SliceHashTable<RefCountedPtr<T>>& table, const grpc_slice& path);
-
-  /// Retrieves the parsed global service config object at index \a index.
-  ServiceConfigParsedObject* GetParsedGlobalServiceConfigObject(int index) {
-    GPR_DEBUG_ASSERT(
-        index < static_cast<int>(parsed_global_service_config_objects_.size()));
+  /// Retrieves the parsed global service config object at index \a index. The
+  /// lifetime of the returned object is tied to the lifetime of the
+  /// ServiceConfig object.
+  ParsedConfig* GetParsedGlobalServiceConfigObject(size_t index) {
+    GPR_DEBUG_ASSERT(index < parsed_global_service_config_objects_.size());
     return parsed_global_service_config_objects_[index].get();
   }
 
   /// Retrieves the vector of method service config objects for a given path \a
-  /// path.
-  const ServiceConfigObjectsVector* const* GetMethodServiceConfigObjectsVector(
+  /// path. The lifetime of the returned vector and contained objects is tied to
+  /// the lifetime of the ServiceConfig object.
+  const ServiceConfigObjectsVector* GetMethodServiceConfigObjectsVector(
       const grpc_slice& path);
 
   /// Globally register a service config parser. On successful registration, it
@@ -150,7 +152,7 @@ class ServiceConfig : public RefCounted<ServiceConfig> {
   /// registered parser. Each parser is responsible for reading the service
   /// config json and returning a parsed object. This parsed object can later be
   /// retrieved using the same index that was returned at registration time.
-  static size_t RegisterParser(UniquePtr<ServiceConfigParser> parser);
+  static size_t RegisterParser(UniquePtr<Parser> parser);
 
   static void Init();
 
@@ -178,14 +180,6 @@ class ServiceConfig : public RefCounted<ServiceConfig> {
   static UniquePtr<char> ParseJsonMethodName(grpc_json* json,
                                              grpc_error** error);
 
-  // Parses the method config from \a json.  Adds an entry to \a entries for
-  // each name found, incrementing \a idx for each entry added.
-  // Returns false on error.
-  template <typename T>
-  static bool ParseJsonMethodConfig(
-      grpc_json* json, CreateValue<T> create_value,
-      typename SliceHashTable<RefCountedPtr<T>>::Entry* entries, size_t* idx);
-
   grpc_error* ParseJsonMethodConfigToServiceConfigObjectsTable(
       const grpc_json* json,
       SliceHashTable<const ServiceConfigObjectsVector*>::Entry* entries,
@@ -195,7 +189,7 @@ class ServiceConfig : public RefCounted<ServiceConfig> {
   UniquePtr<char> json_string_;  // Underlying storage for json_tree.
   grpc_json* json_tree_;
 
-  InlinedVector<UniquePtr<ServiceConfigParsedObject>, kNumPreallocatedParsers>
+  InlinedVector<UniquePtr<ParsedConfig>, kNumPreallocatedParsers>
       parsed_global_service_config_objects_;
   // A map from the method name to the service config objects vector. Note that
   // we are using a raw pointer and not a unique pointer so that we can use the
@@ -208,133 +202,6 @@ class ServiceConfig : public RefCounted<ServiceConfig> {
       service_config_objects_vectors_storage_;
 };
 
-//
-// implementation -- no user-serviceable parts below
-//
-
-template <typename T>
-void ServiceConfig::ParseGlobalParams(ProcessJson<T> process_json,
-                                      T* arg) const {
-  if (json_tree_->type != GRPC_JSON_OBJECT || json_tree_->key != nullptr) {
-    return;
-  }
-  for (grpc_json* field = json_tree_->child; field != nullptr;
-       field = field->next) {
-    if (field->key == nullptr) return;
-    if (strcmp(field->key, "methodConfig") == 0) continue;
-    process_json(field, arg);
-  }
-}
-
-template <typename T>
-bool ServiceConfig::ParseJsonMethodConfig(
-    grpc_json* json, CreateValue<T> create_value,
-    typename SliceHashTable<RefCountedPtr<T>>::Entry* entries, size_t* idx) {
-  // Construct value.
-  RefCountedPtr<T> method_config = create_value(json);
-  if (method_config == nullptr) return false;
-  // Construct list of paths.
-  InlinedVector<UniquePtr<char>, 10> paths;
-  for (grpc_json* child = json->child; child != nullptr; child = child->next) {
-    if (child->key == nullptr) continue;
-    if (strcmp(child->key, "name") == 0) {
-      if (child->type != GRPC_JSON_ARRAY) return false;
-      for (grpc_json* name = child->child; name != nullptr; name = name->next) {
-        grpc_error* error = GRPC_ERROR_NONE;
-        UniquePtr<char> path = ParseJsonMethodName(name, &error);
-        // We are not reporting the error here.
-        GRPC_ERROR_UNREF(error);
-        if (path == nullptr) return false;
-        paths.push_back(std::move(path));
-      }
-    }
-  }
-  if (paths.size() == 0) return false;  // No names specified.
-  // Add entry for each path.
-  for (size_t i = 0; i < paths.size(); ++i) {
-    entries[*idx].key = grpc_slice_from_copied_string(paths[i].get());
-    entries[*idx].value = method_config;  // Takes a new ref.
-    ++*idx;
-  }
-  // Success.
-  return true;
-}
-
-template <typename T>
-RefCountedPtr<SliceHashTable<RefCountedPtr<T>>>
-ServiceConfig::CreateMethodConfigTable(CreateValue<T> create_value) const {
-  // Traverse parsed JSON tree.
-  if (json_tree_->type != GRPC_JSON_OBJECT || json_tree_->key != nullptr) {
-    return nullptr;
-  }
-  size_t num_entries = 0;
-  typename SliceHashTable<RefCountedPtr<T>>::Entry* entries = nullptr;
-  for (grpc_json* field = json_tree_->child; field != nullptr;
-       field = field->next) {
-    if (field->key == nullptr) return nullptr;
-    if (strcmp(field->key, "methodConfig") == 0) {
-      if (entries != nullptr) return nullptr;  // Duplicate.
-      if (field->type != GRPC_JSON_ARRAY) return nullptr;
-      // Find number of entries.
-      for (grpc_json* method = field->child; method != nullptr;
-           method = method->next) {
-        int count = CountNamesInMethodConfig(method);
-        if (count <= 0) return nullptr;
-        num_entries += static_cast<size_t>(count);
-      }
-      // Populate method config table entries.
-      entries = static_cast<typename SliceHashTable<RefCountedPtr<T>>::Entry*>(
-          gpr_zalloc(num_entries *
-                     sizeof(typename SliceHashTable<RefCountedPtr<T>>::Entry)));
-      size_t idx = 0;
-      for (grpc_json* method = field->child; method != nullptr;
-           method = method->next) {
-        if (!ParseJsonMethodConfig(method, create_value, entries, &idx)) {
-          for (size_t i = 0; i < idx; ++i) {
-            grpc_slice_unref_internal(entries[i].key);
-            entries[i].value.reset();
-          }
-          gpr_free(entries);
-          return nullptr;
-        }
-      }
-      GPR_ASSERT(idx == num_entries);
-    }
-  }
-  // Instantiate method config table.
-  RefCountedPtr<SliceHashTable<RefCountedPtr<T>>> method_config_table;
-  if (entries != nullptr) {
-    method_config_table =
-        SliceHashTable<RefCountedPtr<T>>::Create(num_entries, entries, nullptr);
-    gpr_free(entries);
-  }
-  return method_config_table;
-}
-
-template <typename T>
-RefCountedPtr<T> ServiceConfig::MethodConfigTableLookup(
-    const SliceHashTable<RefCountedPtr<T>>& table, const grpc_slice& path) {
-  const RefCountedPtr<T>* value = table.Get(path);
-  // If we didn't find a match for the path, try looking for a wildcard
-  // entry (i.e., change "/service/method" to "/service/*").
-  if (value == nullptr) {
-    char* path_str = grpc_slice_to_c_string(path);
-    const char* sep = strrchr(path_str, '/') + 1;
-    const size_t len = (size_t)(sep - path_str);
-    char* buf = (char*)gpr_malloc(len + 2);  // '*' and NUL
-    memcpy(buf, path_str, len);
-    buf[len] = '*';
-    buf[len + 1] = '\0';
-    grpc_slice wildcard_path = grpc_slice_from_copied_string(buf);
-    gpr_free(buf);
-    value = table.Get(wildcard_path);
-    grpc_slice_unref_internal(wildcard_path);
-    gpr_free(path_str);
-    if (value == nullptr) return nullptr;
-  }
-  return RefCountedPtr<T>(*value);
-}
-
 }  // namespace grpc_core
 
 #endif /* GRPC_CORE_EXT_FILTERS_CLIENT_CHANNEL_SERVICE_CONFIG_H */

+ 3 - 34
src/core/ext/filters/client_channel/subchannel.cc

@@ -529,25 +529,6 @@ BackOff::Options ParseArgsForBackoffValues(
       .set_max_backoff(max_backoff_ms);
 }
 
-struct HealthCheckParams {
-  UniquePtr<char> service_name;
-
-  static void Parse(const grpc_json* field, HealthCheckParams* params) {
-    if (strcmp(field->key, "healthCheckConfig") == 0) {
-      if (field->type != GRPC_JSON_OBJECT) return;
-      for (grpc_json* sub_field = field->child; sub_field != nullptr;
-           sub_field = sub_field->next) {
-        if (sub_field->key == nullptr) return;
-        if (strcmp(sub_field->key, "serviceName") == 0) {
-          if (params->service_name != nullptr) return;  // Duplicate.
-          if (sub_field->type != GRPC_JSON_STRING) return;
-          params->service_name.reset(gpr_strdup(sub_field->value));
-        }
-      }
-    }
-  }
-};
-
 }  // namespace
 
 Subchannel::Subchannel(SubchannelKey* key, grpc_connector* connector,
@@ -583,21 +564,9 @@ Subchannel::Subchannel(SubchannelKey* key, grpc_connector* connector,
                                "subchannel");
   grpc_connectivity_state_init(&state_and_health_tracker_, GRPC_CHANNEL_IDLE,
                                "subchannel");
-  // Check whether we should enable health checking.
-  const char* service_config_json = grpc_channel_arg_get_string(
-      grpc_channel_args_find(args_, GRPC_ARG_SERVICE_CONFIG));
-  if (service_config_json != nullptr) {
-    grpc_error* service_config_error = GRPC_ERROR_NONE;
-    RefCountedPtr<ServiceConfig> service_config =
-        ServiceConfig::Create(service_config_json, &service_config_error);
-    // service_config_error is currently unused.
-    GRPC_ERROR_UNREF(service_config_error);
-    if (service_config != nullptr) {
-      HealthCheckParams params;
-      service_config->ParseGlobalParams(HealthCheckParams::Parse, &params);
-      health_check_service_name_ = std::move(params.service_name);
-    }
-  }
+  health_check_service_name_ =
+      UniquePtr<char>(gpr_strdup(grpc_channel_arg_get_string(
+          grpc_channel_args_find(args_, "grpc.temp.health_check"))));
   const grpc_arg* arg = grpc_channel_args_find(args_, GRPC_ARG_ENABLE_CHANNELZ);
   const bool channelz_enabled =
       grpc_channel_arg_get_bool(arg, GRPC_ENABLE_CHANNELZ_DEFAULT);

+ 116 - 77
src/core/ext/filters/message_size/message_size_filter.cc

@@ -32,75 +32,78 @@
 #include "src/core/lib/gpr/string.h"
 #include "src/core/lib/gprpp/ref_counted.h"
 #include "src/core/lib/gprpp/ref_counted_ptr.h"
+#include "src/core/lib/surface/call.h"
 #include "src/core/lib/surface/channel_init.h"
 
-typedef struct {
-  int max_send_size;
-  int max_recv_size;
-} message_size_limits;
+static void recv_message_ready(void* user_data, grpc_error* error);
+static void recv_trailing_metadata_ready(void* user_data, grpc_error* error);
 
 namespace grpc_core {
-namespace {
-
-class MessageSizeLimits : public RefCounted<MessageSizeLimits> {
- public:
-  static RefCountedPtr<MessageSizeLimits> CreateFromJson(const grpc_json* json);
-
-  const message_size_limits& limits() const { return limits_; }
 
- private:
-  // So New() can call our private ctor.
-  template <typename T, typename... Args>
-  friend T* grpc_core::New(Args&&... args);
-
-  MessageSizeLimits(int max_send_size, int max_recv_size) {
-    limits_.max_send_size = max_send_size;
-    limits_.max_recv_size = max_recv_size;
-  }
-
-  message_size_limits limits_;
-};
+namespace {
+size_t g_message_size_parser_index;
+}  // namespace
 
-RefCountedPtr<MessageSizeLimits> MessageSizeLimits::CreateFromJson(
-    const grpc_json* json) {
+UniquePtr<ServiceConfig::ParsedConfig> MessageSizeParser::ParsePerMethodParams(
+    const grpc_json* json, grpc_error** error) {
+  GPR_DEBUG_ASSERT(error != nullptr && *error == GRPC_ERROR_NONE);
   int max_request_message_bytes = -1;
   int max_response_message_bytes = -1;
+  InlinedVector<grpc_error*, 4> error_list;
   for (grpc_json* field = json->child; field != nullptr; field = field->next) {
     if (field->key == nullptr) continue;
     if (strcmp(field->key, "maxRequestMessageBytes") == 0) {
-      if (max_request_message_bytes >= 0) return nullptr;  // Duplicate.
+      if (max_request_message_bytes >= 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxRequestMessageBytes error:Duplicate entry"));
+      }  // Duplicate, continue parsing.
       if (field->type != GRPC_JSON_STRING && field->type != GRPC_JSON_NUMBER) {
-        return nullptr;
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxRequestMessageBytes error:should be of type number"));
+      } else {
+        max_request_message_bytes = gpr_parse_nonnegative_int(field->value);
+        if (max_request_message_bytes == -1) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:maxRequestMessageBytes error:should be non-negative"));
+        }
       }
-      max_request_message_bytes = gpr_parse_nonnegative_int(field->value);
-      if (max_request_message_bytes == -1) return nullptr;
     } else if (strcmp(field->key, "maxResponseMessageBytes") == 0) {
-      if (max_response_message_bytes >= 0) return nullptr;  // Duplicate.
+      if (max_response_message_bytes >= 0) {
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxResponseMessageBytes error:Duplicate entry"));
+      }  // Duplicate, continue parsing
       if (field->type != GRPC_JSON_STRING && field->type != GRPC_JSON_NUMBER) {
-        return nullptr;
+        error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+            "field:maxResponseMessageBytes error:should be of type number"));
+      } else {
+        max_response_message_bytes = gpr_parse_nonnegative_int(field->value);
+        if (max_response_message_bytes == -1) {
+          error_list.push_back(GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+              "field:maxResponseMessageBytes error:should be non-negative"));
+        }
       }
-      max_response_message_bytes = gpr_parse_nonnegative_int(field->value);
-      if (max_response_message_bytes == -1) return nullptr;
     }
   }
-  return MakeRefCounted<MessageSizeLimits>(max_request_message_bytes,
-                                           max_response_message_bytes);
+  if (!error_list.empty()) {
+    *error = GRPC_ERROR_CREATE_FROM_VECTOR("Message size parser", &error_list);
+    return nullptr;
+  }
+  return UniquePtr<ServiceConfig::ParsedConfig>(New<MessageSizeParsedObject>(
+      max_request_message_bytes, max_response_message_bytes));
 }
 
-}  // namespace
-}  // namespace grpc_core
+void MessageSizeParser::Register() {
+  g_message_size_parser_index = ServiceConfig::RegisterParser(
+      UniquePtr<ServiceConfig::Parser>(New<MessageSizeParser>()));
+}
 
-static void recv_message_ready(void* user_data, grpc_error* error);
-static void recv_trailing_metadata_ready(void* user_data, grpc_error* error);
+size_t MessageSizeParser::ParserIndex() { return g_message_size_parser_index; }
+}  // namespace grpc_core
 
 namespace {
-
 struct channel_data {
-  message_size_limits limits;
-  // Maps path names to refcounted_message_size_limits structs.
-  grpc_core::RefCountedPtr<grpc_core::SliceHashTable<
-      grpc_core::RefCountedPtr<grpc_core::MessageSizeLimits>>>
-      method_limit_table;
+  grpc_core::MessageSizeParsedObject::message_size_limits limits;
+  grpc_core::RefCountedPtr<grpc_core::ServiceConfig> svc_cfg;
 };
 
 struct call_data {
@@ -116,21 +119,34 @@ struct call_data {
     // Note: Per-method config is only available on the client, so we
     // apply the max request size to the send limit and the max response
     // size to the receive limit.
-    if (chand.method_limit_table != nullptr) {
-      grpc_core::RefCountedPtr<grpc_core::MessageSizeLimits> limits =
-          grpc_core::ServiceConfig::MethodConfigTableLookup(
-              *chand.method_limit_table, args.path);
-      if (limits != nullptr) {
-        if (limits->limits().max_send_size >= 0 &&
-            (limits->limits().max_send_size < this->limits.max_send_size ||
-             this->limits.max_send_size < 0)) {
-          this->limits.max_send_size = limits->limits().max_send_size;
-        }
-        if (limits->limits().max_recv_size >= 0 &&
-            (limits->limits().max_recv_size < this->limits.max_recv_size ||
-             this->limits.max_recv_size < 0)) {
-          this->limits.max_recv_size = limits->limits().max_recv_size;
-        }
+    const grpc_core::MessageSizeParsedObject* limits = nullptr;
+    grpc_core::ServiceConfig::CallData* svc_cfg_call_data = nullptr;
+    if (args.context != nullptr) {
+      svc_cfg_call_data = static_cast<grpc_core::ServiceConfig::CallData*>(
+          args.context[GRPC_SERVICE_CONFIG_CALL_DATA].value);
+    }
+    if (svc_cfg_call_data != nullptr) {
+      limits = static_cast<const grpc_core::MessageSizeParsedObject*>(
+          svc_cfg_call_data->GetMethodParsedObject(
+              grpc_core::MessageSizeParser::ParserIndex()));
+    } else if (chand.svc_cfg != nullptr) {
+      const auto* objs_vector =
+          chand.svc_cfg->GetMethodServiceConfigObjectsVector(args.path);
+      if (objs_vector != nullptr) {
+        limits = static_cast<const grpc_core::MessageSizeParsedObject*>(
+            (*objs_vector)[grpc_core::MessageSizeParser::ParserIndex()].get());
+      }
+    }
+    if (limits != nullptr) {
+      if (limits->limits().max_send_size >= 0 &&
+          (limits->limits().max_send_size < this->limits.max_send_size ||
+           this->limits.max_send_size < 0)) {
+        this->limits.max_send_size = limits->limits().max_send_size;
+      }
+      if (limits->limits().max_recv_size >= 0 &&
+          (limits->limits().max_recv_size < this->limits.max_recv_size ||
+           this->limits.max_recv_size < 0)) {
+        this->limits.max_recv_size = limits->limits().max_recv_size;
       }
     }
   }
@@ -138,7 +154,7 @@ struct call_data {
   ~call_data() { GRPC_ERROR_UNREF(error); }
 
   grpc_core::CallCombiner* call_combiner;
-  message_size_limits limits;
+  grpc_core::MessageSizeParsedObject::message_size_limits limits;
   // Receive closures are chained: we inject this closure as the
   // recv_message_ready up-call on transport_stream_op, and remember to
   // call our next_recv_message_ready member after handling it.
@@ -284,9 +300,9 @@ static int default_size(const grpc_channel_args* args,
   return without_minimal_stack;
 }
 
-message_size_limits get_message_size_limits(
+grpc_core::MessageSizeParsedObject::message_size_limits get_message_size_limits(
     const grpc_channel_args* channel_args) {
-  message_size_limits lim;
+  grpc_core::MessageSizeParsedObject::message_size_limits lim;
   lim.max_send_size =
       default_size(channel_args, GRPC_DEFAULT_MAX_SEND_MESSAGE_LENGTH);
   lim.max_recv_size =
@@ -313,21 +329,27 @@ static grpc_error* init_channel_elem(grpc_channel_element* elem,
                                      grpc_channel_element_args* args) {
   GPR_ASSERT(!args->is_last);
   channel_data* chand = static_cast<channel_data*>(elem->channel_data);
+  new (chand) channel_data();
   chand->limits = get_message_size_limits(args->channel_args);
-  // Get method config table from channel args.
+  // TODO(yashykt): We only need to read GRPC_ARG_SERVICE_CONFIG in the case of
+  // direct channels. (Service config is otherwise stored in the call_context by
+  // client_channel filter.) If we ever need a second filter that also needs to
+  // parse GRPC_ARG_SERVICE_CONFIG, we should refactor this code and add a
+  // separate filter that reads GRPC_ARG_SERVICE_CONFIG and saves the parsed
+  // config in the call_context.
   const grpc_arg* channel_arg =
       grpc_channel_args_find(args->channel_args, GRPC_ARG_SERVICE_CONFIG);
   const char* service_config_str = grpc_channel_arg_get_string(channel_arg);
   if (service_config_str != nullptr) {
     grpc_error* service_config_error = GRPC_ERROR_NONE;
-    grpc_core::RefCountedPtr<grpc_core::ServiceConfig> service_config =
-        grpc_core::ServiceConfig::Create(service_config_str,
-                                         &service_config_error);
-    GRPC_ERROR_UNREF(service_config_error);
-    if (service_config != nullptr) {
-      chand->method_limit_table = service_config->CreateMethodConfigTable(
-          grpc_core::MessageSizeLimits::CreateFromJson);
+    auto svc_cfg = grpc_core::ServiceConfig::Create(service_config_str,
+                                                    &service_config_error);
+    if (service_config_error == GRPC_ERROR_NONE) {
+      chand->svc_cfg = std::move(svc_cfg);
+    } else {
+      gpr_log(GPR_ERROR, "%s", grpc_error_string(service_config_error));
     }
+    GRPC_ERROR_UNREF(service_config_error);
   }
   return GRPC_ERROR_NONE;
 }
@@ -335,7 +357,7 @@ static grpc_error* init_channel_elem(grpc_channel_element* elem,
 // Destructor for channel_data.
 static void destroy_channel_elem(grpc_channel_element* elem) {
   channel_data* chand = static_cast<channel_data*>(elem->channel_data);
-  chand->method_limit_table.reset();
+  chand->~channel_data();
 }
 
 const grpc_channel_filter grpc_message_size_filter = {
@@ -351,18 +373,34 @@ const grpc_channel_filter grpc_message_size_filter = {
     grpc_channel_next_get_info,
     "message_size"};
 
+// Used for GRPC_CLIENT_SUBCHANNEL
+static bool maybe_add_message_size_filter_subchannel(
+    grpc_channel_stack_builder* builder, void* arg) {
+  const grpc_channel_args* channel_args =
+      grpc_channel_stack_builder_get_channel_arguments(builder);
+  if (grpc_channel_args_want_minimal_stack(channel_args)) {
+    return true;
+  }
+  return grpc_channel_stack_builder_prepend_filter(
+      builder, &grpc_message_size_filter, nullptr, nullptr);
+}
+
+// Used for GRPC_CLIENT_DIRECT_CHANNEL and GRPC_SERVER_CHANNEL. Adds the filter
+// only if message size limits or service config is specified.
 static bool maybe_add_message_size_filter(grpc_channel_stack_builder* builder,
                                           void* arg) {
   const grpc_channel_args* channel_args =
       grpc_channel_stack_builder_get_channel_arguments(builder);
   bool enable = false;
-  message_size_limits lim = get_message_size_limits(channel_args);
+  grpc_core::MessageSizeParsedObject::message_size_limits lim =
+      get_message_size_limits(channel_args);
   if (lim.max_send_size != -1 || lim.max_recv_size != -1) {
     enable = true;
   }
   const grpc_arg* a =
       grpc_channel_args_find(channel_args, GRPC_ARG_SERVICE_CONFIG);
-  if (a != nullptr) {
+  const char* svc_cfg_str = grpc_channel_arg_get_string(a);
+  if (svc_cfg_str != nullptr) {
     enable = true;
   }
   if (enable) {
@@ -374,15 +412,16 @@ static bool maybe_add_message_size_filter(grpc_channel_stack_builder* builder,
 }
 
 void grpc_message_size_filter_init(void) {
-  grpc_channel_init_register_stage(GRPC_CLIENT_SUBCHANNEL,
-                                   GRPC_CHANNEL_INIT_BUILTIN_PRIORITY,
-                                   maybe_add_message_size_filter, nullptr);
+  grpc_channel_init_register_stage(
+      GRPC_CLIENT_SUBCHANNEL, GRPC_CHANNEL_INIT_BUILTIN_PRIORITY,
+      maybe_add_message_size_filter_subchannel, nullptr);
   grpc_channel_init_register_stage(GRPC_CLIENT_DIRECT_CHANNEL,
                                    GRPC_CHANNEL_INIT_BUILTIN_PRIORITY,
                                    maybe_add_message_size_filter, nullptr);
   grpc_channel_init_register_stage(GRPC_SERVER_CHANNEL,
                                    GRPC_CHANNEL_INIT_BUILTIN_PRIORITY,
                                    maybe_add_message_size_filter, nullptr);
+  grpc_core::MessageSizeParser::Register();
 }
 
 void grpc_message_size_filter_shutdown(void) {}

+ 33 - 0
src/core/ext/filters/message_size/message_size_filter.h

@@ -19,8 +19,41 @@
 
 #include <grpc/support/port_platform.h>
 
+#include "src/core/ext/filters/client_channel/service_config.h"
 #include "src/core/lib/channel/channel_stack.h"
 
 extern const grpc_channel_filter grpc_message_size_filter;
 
+namespace grpc_core {
+
+class MessageSizeParsedObject : public ServiceConfig::ParsedConfig {
+ public:
+  struct message_size_limits {
+    int max_send_size;
+    int max_recv_size;
+  };
+
+  MessageSizeParsedObject(int max_send_size, int max_recv_size) {
+    limits_.max_send_size = max_send_size;
+    limits_.max_recv_size = max_recv_size;
+  }
+
+  const message_size_limits& limits() const { return limits_; }
+
+ private:
+  message_size_limits limits_;
+};
+
+class MessageSizeParser : public ServiceConfig::Parser {
+ public:
+  UniquePtr<ServiceConfig::ParsedConfig> ParsePerMethodParams(
+      const grpc_json* json, grpc_error** error) override;
+
+  static void Register();
+
+  static size_t ParserIndex();
+};
+
+}  // namespace grpc_core
+
 #endif /* GRPC_CORE_EXT_FILTERS_MESSAGE_SIZE_MESSAGE_SIZE_FILTER_H */

+ 3 - 0
src/core/lib/channel/context.h

@@ -35,6 +35,9 @@ typedef enum {
   /// Reserved for traffic_class_context.
   GRPC_CONTEXT_TRAFFIC,
 
+  /// Holds a pointer to ServiceConfig::CallData associated with this call.
+  GRPC_SERVICE_CONFIG_CALL_DATA,
+
   GRPC_CONTEXT_COUNT
 } grpc_context_index;
 

+ 1 - 0
src/core/lib/gprpp/optional.h

@@ -26,6 +26,7 @@ namespace grpc_core {
 template <typename T>
 class Optional {
  public:
+  Optional() : value_() {}
   void set(const T& val) {
     value_ = val;
     set_ = true;

+ 23 - 0
src/core/lib/iomgr/error.h

@@ -30,6 +30,7 @@
 #include <grpc/support/time.h>
 
 #include "src/core/lib/debug/trace.h"
+#include "src/core/lib/gprpp/inlined_vector.h"
 
 /// Opaque representation of an error.
 /// See https://github.com/grpc/grpc/blob/master/doc/core/grpc-error.md for a
@@ -165,6 +166,9 @@ grpc_error* grpc_error_create(const char* file, int line,
   grpc_error_create(__FILE__, __LINE__, grpc_slice_from_copied_string(desc), \
                     errs, count)
 
+#define GRPC_ERROR_CREATE_FROM_VECTOR(desc, error_list) \
+  grpc_error_create_from_vector(__FILE__, __LINE__, desc, error_list)
+
 #ifndef NDEBUG
 grpc_error* grpc_error_do_ref(grpc_error* err, const char* file, int line);
 void grpc_error_do_unref(grpc_error* err, const char* file, int line);
@@ -193,6 +197,25 @@ inline void grpc_error_unref(grpc_error* err) {
 #define GRPC_ERROR_UNREF(err) grpc_error_unref(err)
 #endif
 
+// Consumes all the errors in the vector and forms a referencing error from
+// them. If the vector is empty, return GRPC_ERROR_NONE.
+template <size_t N>
+static grpc_error* grpc_error_create_from_vector(
+    const char* file, int line, const char* desc,
+    grpc_core::InlinedVector<grpc_error*, N>* error_list) {
+  grpc_error* error = GRPC_ERROR_NONE;
+  if (error_list->size() != 0) {
+    error = grpc_error_create(file, line, grpc_slice_from_static_string(desc),
+                              error_list->data(), error_list->size());
+    // Remove refs to all errors in error_list.
+    for (size_t i = 0; i < error_list->size(); i++) {
+      GRPC_ERROR_UNREF((*error_list)[i]);
+    }
+    error_list->clear();
+  }
+  return error;
+}
+
 grpc_error* grpc_error_set_int(grpc_error* src, grpc_error_ints which,
                                intptr_t value) GRPC_MUST_USE_RESULT;
 /// It is an error to pass nullptr as `p`. Caller should allocate a dummy

+ 727 - 47
test/core/client_channel/service_config_test.cc

@@ -21,7 +21,9 @@
 #include <gtest/gtest.h>
 
 #include <grpc/grpc.h>
+#include "src/core/ext/filters/client_channel/resolver_result_parsing.h"
 #include "src/core/ext/filters/client_channel/service_config.h"
+#include "src/core/ext/filters/message_size/message_size_filter.h"
 #include "src/core/lib/gpr/string.h"
 #include "test/core/util/port.h"
 #include "test/core/util/test_config.h"
@@ -29,7 +31,7 @@
 namespace grpc_core {
 namespace testing {
 
-class TestParsedObject1 : public ServiceConfigParsedObject {
+class TestParsedObject1 : public ServiceConfig::ParsedConfig {
  public:
   TestParsedObject1(int value) : value_(value) {}
 
@@ -39,9 +41,9 @@ class TestParsedObject1 : public ServiceConfigParsedObject {
   int value_;
 };
 
-class TestParser1 : public ServiceConfigParser {
+class TestParser1 : public ServiceConfig::Parser {
  public:
-  UniquePtr<ServiceConfigParsedObject> ParseGlobalParams(
+  UniquePtr<ServiceConfig::ParsedConfig> ParseGlobalParams(
       const grpc_json* json, grpc_error** error) override {
     GPR_DEBUG_ASSERT(error != nullptr);
     for (grpc_json* field = json->child; field != nullptr;
@@ -58,7 +60,7 @@ class TestParser1 : public ServiceConfigParser {
               GRPC_ERROR_CREATE_FROM_STATIC_STRING(InvalidValueErrorMessage());
           return nullptr;
         }
-        return UniquePtr<ServiceConfigParsedObject>(
+        return UniquePtr<ServiceConfig::ParsedConfig>(
             New<TestParsedObject1>(value));
       }
     }
@@ -74,9 +76,9 @@ class TestParser1 : public ServiceConfigParser {
   }
 };
 
-class TestParser2 : public ServiceConfigParser {
+class TestParser2 : public ServiceConfig::Parser {
  public:
-  UniquePtr<ServiceConfigParsedObject> ParsePerMethodParams(
+  UniquePtr<ServiceConfig::ParsedConfig> ParsePerMethodParams(
       const grpc_json* json, grpc_error** error) override {
     GPR_DEBUG_ASSERT(error != nullptr);
     for (grpc_json* field = json->child; field != nullptr;
@@ -96,7 +98,7 @@ class TestParser2 : public ServiceConfigParser {
               GRPC_ERROR_CREATE_FROM_STATIC_STRING(InvalidValueErrorMessage());
           return nullptr;
         }
-        return UniquePtr<ServiceConfigParsedObject>(
+        return UniquePtr<ServiceConfig::ParsedConfig>(
             New<TestParsedObject1>(value));
       }
     }
@@ -113,16 +115,16 @@ class TestParser2 : public ServiceConfigParser {
 };
 
 // This parser always adds errors
-class ErrorParser : public ServiceConfigParser {
+class ErrorParser : public ServiceConfig::Parser {
  public:
-  UniquePtr<ServiceConfigParsedObject> ParsePerMethodParams(
+  UniquePtr<ServiceConfig::ParsedConfig> ParsePerMethodParams(
       const grpc_json* json, grpc_error** error) override {
     GPR_DEBUG_ASSERT(error != nullptr);
     *error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(MethodError());
     return nullptr;
   }
 
-  UniquePtr<ServiceConfigParsedObject> ParseGlobalParams(
+  UniquePtr<ServiceConfig::ParsedConfig> ParseGlobalParams(
       const grpc_json* json, grpc_error** error) override {
     GPR_DEBUG_ASSERT(error != nullptr);
     *error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(GlobalError());
@@ -134,15 +136,22 @@ class ErrorParser : public ServiceConfigParser {
   static const char* GlobalError() { return "ErrorParser : globalError"; }
 };
 
+void VerifyRegexMatch(grpc_error* error, const std::regex& e) {
+  std::smatch match;
+  std::string s(grpc_error_string(error));
+  EXPECT_TRUE(std::regex_search(s, match, e));
+  GRPC_ERROR_UNREF(error);
+}
+
 class ServiceConfigTest : public ::testing::Test {
  protected:
   void SetUp() override {
     ServiceConfig::Shutdown();
     ServiceConfig::Init();
     EXPECT_TRUE(ServiceConfig::RegisterParser(
-                    UniquePtr<ServiceConfigParser>(New<TestParser1>())) == 0);
+                    UniquePtr<ServiceConfig::Parser>(New<TestParser1>())) == 0);
     EXPECT_TRUE(ServiceConfig::RegisterParser(
-                    UniquePtr<ServiceConfigParser>(New<TestParser2>())) == 1);
+                    UniquePtr<ServiceConfig::Parser>(New<TestParser2>())) == 1);
   }
 };
 
@@ -152,9 +161,8 @@ TEST_F(ServiceConfigTest, ErrorCheck1) {
   auto svc_cfg = ServiceConfig::Create(test_json, &error);
   gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
   ASSERT_TRUE(error != GRPC_ERROR_NONE);
-  EXPECT_TRUE(strstr(grpc_error_string(error),
-                     "failed to parse JSON for service config") != nullptr);
-  GRPC_ERROR_UNREF(error);
+  std::regex e(std::string("failed to parse JSON for service config"));
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ServiceConfigTest, BasicTest1) {
@@ -170,8 +178,13 @@ TEST_F(ServiceConfigTest, ErrorNoNames) {
   auto svc_cfg = ServiceConfig::Create(test_json, &error);
   gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
   ASSERT_TRUE(error != GRPC_ERROR_NONE);
-  EXPECT_TRUE(strstr(grpc_error_string(error), "No names found") != nullptr);
-  GRPC_ERROR_UNREF(error);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Method "
+                  "Params)(.*)(referenced_errors)(.*)(No names "
+                  "found)(.*)(methodConfig)(.*)(referenced_errors)(.*)(No "
+                  "names specified)"));
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ServiceConfigTest, ErrorNoNamesWithMultipleMethodConfigs) {
@@ -181,8 +194,13 @@ TEST_F(ServiceConfigTest, ErrorNoNamesWithMultipleMethodConfigs) {
   auto svc_cfg = ServiceConfig::Create(test_json, &error);
   gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
   ASSERT_TRUE(error != GRPC_ERROR_NONE);
-  EXPECT_TRUE(strstr(grpc_error_string(error), "No names found") != nullptr);
-  GRPC_ERROR_UNREF(error);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Method "
+                  "Params)(.*)(referenced_errors)(.*)(No names "
+                  "found)(.*)(methodConfig)(.*)(referenced_errors)(.*)(No "
+                  "names specified)"));
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ServiceConfigTest, ValidMethodConfig) {
@@ -201,6 +219,9 @@ TEST_F(ServiceConfigTest, Parser1BasicTest1) {
   EXPECT_TRUE((static_cast<TestParsedObject1*>(
                    svc_cfg->GetParsedGlobalServiceConfigObject(0)))
                   ->value() == 5);
+  EXPECT_TRUE(svc_cfg->GetMethodServiceConfigObjectsVector(
+                  grpc_slice_from_static_string("/TestServ/TestMethod")) ==
+              nullptr);
 }
 
 TEST_F(ServiceConfigTest, Parser1BasicTest2) {
@@ -221,12 +242,9 @@ TEST_F(ServiceConfigTest, Parser1ErrorInvalidType) {
   ASSERT_TRUE(error != GRPC_ERROR_NONE);
   std::regex e(std::string("(Service config parsing "
                            "error)(.*)(referenced_errors)(.*)(Global "
-                           "Params)(.*)(referenced_errors)()(.*)") +
+                           "Params)(.*)(referenced_errors)(.*)") +
                TestParser1::InvalidTypeErrorMessage());
-  std::smatch match;
-  std::string s(grpc_error_string(error));
-  EXPECT_TRUE(std::regex_search(s, match, e));
-  GRPC_ERROR_UNREF(error);
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ServiceConfigTest, Parser1ErrorInvalidValue) {
@@ -237,12 +255,9 @@ TEST_F(ServiceConfigTest, Parser1ErrorInvalidValue) {
   ASSERT_TRUE(error != GRPC_ERROR_NONE);
   std::regex e(std::string("(Service config parsing "
                            "error)(.*)(referenced_errors)(.*)(Global "
-                           "Params)(.*)(referenced_errors)()(.*)") +
+                           "Params)(.*)(referenced_errors)(.*)") +
                TestParser1::InvalidValueErrorMessage());
-  std::smatch match;
-  std::string s(grpc_error_string(error));
-  EXPECT_TRUE(std::regex_search(s, match, e));
-  GRPC_ERROR_UNREF(error);
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ServiceConfigTest, Parser2BasicTest) {
@@ -252,11 +267,10 @@ TEST_F(ServiceConfigTest, Parser2BasicTest) {
   grpc_error* error = GRPC_ERROR_NONE;
   auto svc_cfg = ServiceConfig::Create(test_json, &error);
   ASSERT_TRUE(error == GRPC_ERROR_NONE);
-  const auto* const* vector_ptr = svc_cfg->GetMethodServiceConfigObjectsVector(
+  const auto* vector_ptr = svc_cfg->GetMethodServiceConfigObjectsVector(
       grpc_slice_from_static_string("/TestServ/TestMethod"));
   EXPECT_TRUE(vector_ptr != nullptr);
-  const auto* vector = *vector_ptr;
-  auto parsed_object = ((*vector)[1]).get();
+  auto parsed_object = ((*vector_ptr)[1]).get();
   EXPECT_TRUE(static_cast<TestParsedObject1*>(parsed_object)->value() == 5);
 }
 
@@ -270,13 +284,10 @@ TEST_F(ServiceConfigTest, Parser2ErrorInvalidType) {
   gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
   std::regex e(std::string("(Service config parsing "
                            "error)(.*)(referenced_errors\":\\[)(.*)(Method "
-                           "Params)(.*)(referenced_errors)()(.*)(methodConfig)("
+                           "Params)(.*)(referenced_errors)(.*)(methodConfig)("
                            ".*)(referenced_errors)(.*)") +
                TestParser2::InvalidTypeErrorMessage());
-  std::smatch match;
-  std::string s(grpc_error_string(error));
-  EXPECT_TRUE(std::regex_search(s, match, e));
-  GRPC_ERROR_UNREF(error);
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ServiceConfigTest, Parser2ErrorInvalidValue) {
@@ -292,10 +303,7 @@ TEST_F(ServiceConfigTest, Parser2ErrorInvalidValue) {
                            "Params)(.*)(referenced_errors)()(.*)(methodConfig)("
                            ".*)(referenced_errors)(.*)") +
                TestParser2::InvalidValueErrorMessage());
-  std::smatch match;
-  std::string s(grpc_error_string(error));
-  EXPECT_TRUE(std::regex_search(s, match, e));
-  GRPC_ERROR_UNREF(error);
+  VerifyRegexMatch(error, e);
 }
 
 // Test parsing with ErrorParsers which always add errors
@@ -305,9 +313,9 @@ class ErroredParsersScopingTest : public ::testing::Test {
     ServiceConfig::Shutdown();
     ServiceConfig::Init();
     EXPECT_TRUE(ServiceConfig::RegisterParser(
-                    UniquePtr<ServiceConfigParser>(New<ErrorParser>())) == 0);
+                    UniquePtr<ServiceConfig::Parser>(New<ErrorParser>())) == 0);
     EXPECT_TRUE(ServiceConfig::RegisterParser(
-                    UniquePtr<ServiceConfigParser>(New<ErrorParser>())) == 1);
+                    UniquePtr<ServiceConfig::Parser>(New<ErrorParser>())) == 1);
   }
 };
 
@@ -322,10 +330,7 @@ TEST_F(ErroredParsersScopingTest, GlobalParams) {
                            "Params)(.*)(referenced_errors)()(.*)") +
                ErrorParser::GlobalError() + std::string("(.*)") +
                ErrorParser::GlobalError());
-  std::smatch match;
-  std::string s(grpc_error_string(error));
-  EXPECT_TRUE(std::regex_search(s, match, e));
-  GRPC_ERROR_UNREF(error);
+  VerifyRegexMatch(error, e);
 }
 
 TEST_F(ErroredParsersScopingTest, MethodParams) {
@@ -346,12 +351,687 @@ TEST_F(ErroredParsersScopingTest, MethodParams) {
                   "found)(.*)(methodConfig)(.*)(referenced_errors)(.*)") +
       ErrorParser::MethodError() + std::string("(.*)") +
       ErrorParser::MethodError() + std::string("(.*)(No names specified)"));
+  VerifyRegexMatch(error, e);
+}
+
+class ClientChannelParserTest : public ::testing::Test {
+ protected:
+  void SetUp() override {
+    ServiceConfig::Shutdown();
+    ServiceConfig::Init();
+    EXPECT_TRUE(
+        ServiceConfig::RegisterParser(UniquePtr<ServiceConfig::Parser>(
+            New<grpc_core::internal::ClientChannelServiceConfigParser>())) ==
+        0);
+  }
+};
+
+TEST_F(ClientChannelParserTest, ValidLoadBalancingConfigPickFirst) {
+  const char* test_json = "{\"loadBalancingConfig\": [{\"pick_first\":{}}]}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  auto lb_config = parsed_object->parsed_lb_config();
+  EXPECT_TRUE(strcmp(lb_config->name(), "pick_first") == 0);
+}
+
+TEST_F(ClientChannelParserTest, ValidLoadBalancingConfigRoundRobin) {
+  const char* test_json =
+      "{\"loadBalancingConfig\": [{\"round_robin\":{}}, {}]}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  auto parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  auto lb_config = parsed_object->parsed_lb_config();
+  EXPECT_TRUE(strcmp(lb_config->name(), "round_robin") == 0);
+}
+
+TEST_F(ClientChannelParserTest, ValidLoadBalancingConfigGrpclb) {
+  const char* test_json =
+      "{\"loadBalancingConfig\": "
+      "[{\"grpclb\":{\"childPolicy\":[{\"pick_first\":{}}]}}]}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  auto lb_config = parsed_object->parsed_lb_config();
+  EXPECT_TRUE(strcmp(lb_config->name(), "grpclb") == 0);
+}
+
+TEST_F(ClientChannelParserTest, ValidLoadBalancingConfigXds) {
+  const char* test_json =
+      "{\n"
+      "  \"loadBalancingConfig\":[\n"
+      "    { \"does_not_exist\":{} },\n"
+      "    { \"xds_experimental\":{ \"balancerName\": \"fake:///lb\" } }\n"
+      "  ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  auto lb_config = parsed_object->parsed_lb_config();
+  EXPECT_TRUE(strcmp(lb_config->name(), "xds_experimental") == 0);
+}
+
+TEST_F(ClientChannelParserTest, UnknownLoadBalancingConfig) {
+  const char* test_json = "{\"loadBalancingConfig\": [{\"unknown\":{}}]}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(field:"
+                  "loadBalancingConfig error:No known policy)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidGrpclbLoadBalancingConfig) {
+  const char* test_json =
+      "{\"loadBalancingConfig\": "
+      "[{\"grpclb\":{\"childPolicy\":[{\"unknown\":{}}]}}]}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(GrpcLb "
+                  "Parser)(.*)(referenced_errors)(.*)(field:childPolicy "
+                  "error:No known policy)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InalidLoadBalancingConfigXds) {
+  const char* test_json =
+      "{\n"
+      "  \"loadBalancingConfig\":[\n"
+      "    { \"does_not_exist\":{} },\n"
+      "    { \"xds_experimental\":{} }\n"
+      "  ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(Xds "
+                  "Parser)(.*)(referenced_errors)(.*)(field:balancerName "
+                  "error:not found)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, ValidLoadBalancingPolicy) {
+  const char* test_json = "{\"loadBalancingPolicy\":\"pick_first\"}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  const auto* lb_policy = parsed_object->parsed_deprecated_lb_policy();
+  ASSERT_TRUE(lb_policy != nullptr);
+  EXPECT_TRUE(strcmp(lb_policy, "pick_first") == 0);
+}
+
+TEST_F(ClientChannelParserTest, ValidLoadBalancingPolicyAllCaps) {
+  const char* test_json = "{\"loadBalancingPolicy\":\"PICK_FIRST\"}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  const auto* lb_policy = parsed_object->parsed_deprecated_lb_policy();
+  ASSERT_TRUE(lb_policy != nullptr);
+  EXPECT_TRUE(strcmp(lb_policy, "pick_first") == 0);
+}
+
+TEST_F(ClientChannelParserTest, UnknownLoadBalancingPolicy) {
+  const char* test_json = "{\"loadBalancingPolicy\":\"unknown\"}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(field:"
+                  "loadBalancingPolicy error:Unknown lb policy)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, LoadBalancingPolicyXdsNotAllowed) {
+  const char* test_json = "{\"loadBalancingPolicy\":\"xds_experimental\"}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(field:"
+                  "loadBalancingPolicy error:xds_experimental requires a "
+                  "config. Please use loadBalancingConfig instead.)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, ValidRetryThrottling) {
+  const char* test_json =
+      "{\n"
+      "  \"retryThrottling\": {\n"
+      "    \"maxTokens\": 2,\n"
+      "    \"tokenRatio\": 1.0\n"
+      "  }\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  const auto retryThrottling = parsed_object->retry_throttling();
+  ASSERT_TRUE(retryThrottling.has_value());
+  EXPECT_EQ(retryThrottling.value().max_milli_tokens, 2000);
+  EXPECT_EQ(retryThrottling.value().milli_token_ratio, 1000);
+}
+
+TEST_F(ClientChannelParserTest, RetryThrottlingMissingFields) {
+  const char* test_json =
+      "{\n"
+      "  \"retryThrottling\": {\n"
+      "  }\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(field:retryThrottling "
+                  "field:maxTokens error:Not found)(.*)(field:retryThrottling "
+                  "field:tokenRatio error:Not found)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryThrottlingNegativeMaxTokens) {
+  const char* test_json =
+      "{\n"
+      "  \"retryThrottling\": {\n"
+      "    \"maxTokens\": -2,\n"
+      "    \"tokenRatio\": 1.0\n"
+      "  }\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(field:retryThrottling "
+                  "field:maxTokens error:should be greater than zero)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryThrottlingInvalidTokenRatio) {
+  const char* test_json =
+      "{\n"
+      "  \"retryThrottling\": {\n"
+      "    \"maxTokens\": 2,\n"
+      "    \"tokenRatio\": -1\n"
+      "  }\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(Client channel global "
+                  "parser)(.*)(referenced_errors)(.*)(field:retryThrottling "
+                  "field:tokenRatio error:Failed parsing)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, ValidTimeout) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"timeout\": \"5s\"\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* vector_ptr = svc_cfg->GetMethodServiceConfigObjectsVector(
+      grpc_slice_from_static_string("/TestServ/TestMethod"));
+  EXPECT_TRUE(vector_ptr != nullptr);
+  auto parsed_object = ((*vector_ptr)[0]).get();
+  EXPECT_EQ((static_cast<grpc_core::internal::ClientChannelMethodParsedObject*>(
+                 parsed_object))
+                ->timeout(),
+            5000);
+}
+
+TEST_F(ClientChannelParserTest, InvalidTimeout) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"service\", \"method\": \"method\" }\n"
+      "    ],\n"
+      "    \"timeout\": \"5sec\"\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Method "
+                  "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)("
+                  "referenced_errors)(.*)(Client channel "
+                  "parser)(.*)(referenced_errors)(.*)(field:timeout "
+                  "error:Failed parsing)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, ValidWaitForReady) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"waitForReady\": true\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* vector_ptr = svc_cfg->GetMethodServiceConfigObjectsVector(
+      grpc_slice_from_static_string("/TestServ/TestMethod"));
+  EXPECT_TRUE(vector_ptr != nullptr);
+  auto parsed_object = ((*vector_ptr)[0]).get();
+  EXPECT_TRUE(
+      (static_cast<grpc_core::internal::ClientChannelMethodParsedObject*>(
+           parsed_object))
+          ->wait_for_ready()
+          .has_value());
+  EXPECT_TRUE(
+      (static_cast<grpc_core::internal::ClientChannelMethodParsedObject*>(
+           parsed_object))
+          ->wait_for_ready()
+          .value());
+}
+
+TEST_F(ClientChannelParserTest, InvalidWaitForReady) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"service\", \"method\": \"method\" }\n"
+      "    ],\n"
+      "    \"waitForReady\": \"true\"\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Method "
+                  "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)("
+                  "referenced_errors)(.*)(Client channel "
+                  "parser)(.*)(referenced_errors)(.*)(field:waitForReady "
+                  "error:Type should be true/false)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, ValidRetryPolicy) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"retryPolicy\": {\n"
+      "      \"maxAttempts\": 3,\n"
+      "      \"initialBackoff\": \"1s\",\n"
+      "      \"maxBackoff\": \"120s\",\n"
+      "      \"backoffMultiplier\": 1.6,\n"
+      "      \"retryableStatusCodes\": [ \"ABORTED\" ]\n"
+      "    }\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* vector_ptr = svc_cfg->GetMethodServiceConfigObjectsVector(
+      grpc_slice_from_static_string("/TestServ/TestMethod"));
+  EXPECT_TRUE(vector_ptr != nullptr);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelMethodParsedObject*>(
+          ((*vector_ptr)[0]).get());
+  EXPECT_TRUE(parsed_object->retry_policy() != nullptr);
+  EXPECT_EQ(parsed_object->retry_policy()->max_attempts, 3);
+  EXPECT_EQ(parsed_object->retry_policy()->initial_backoff, 1000);
+  EXPECT_EQ(parsed_object->retry_policy()->max_backoff, 120000);
+  EXPECT_EQ(parsed_object->retry_policy()->backoff_multiplier, 1.6f);
+  EXPECT_TRUE(parsed_object->retry_policy()->retryable_status_codes.Contains(
+      GRPC_STATUS_ABORTED));
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryPolicyMaxAttempts) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"retryPolicy\": {\n"
+      "      \"maxAttempts\": 1,\n"
+      "      \"initialBackoff\": \"1s\",\n"
+      "      \"maxBackoff\": \"120s\",\n"
+      "      \"backoffMultiplier\": 1.6,\n"
+      "      \"retryableStatusCodes\": [ \"ABORTED\" ]\n"
+      "    }\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(std::string(
+      "(Service config parsing "
+      "error)(.*)(referenced_errors)(.*)(Method "
+      "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)(referenced_errors)("
+      ".*)(Client channel "
+      "parser)(.*)(referenced_errors)(.*)(retryPolicy)(.*)(referenced_errors)(."
+      "*)(field:maxAttempts error:should be at least 2)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryPolicyInitialBackoff) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"retryPolicy\": {\n"
+      "      \"maxAttempts\": 1,\n"
+      "      \"initialBackoff\": \"1sec\",\n"
+      "      \"maxBackoff\": \"120s\",\n"
+      "      \"backoffMultiplier\": 1.6,\n"
+      "      \"retryableStatusCodes\": [ \"ABORTED\" ]\n"
+      "    }\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(std::string(
+      "(Service config parsing "
+      "error)(.*)(referenced_errors)(.*)(Method "
+      "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)(referenced_errors)("
+      ".*)(Client channel "
+      "parser)(.*)(referenced_errors)(.*)(retryPolicy)(.*)(referenced_errors)(."
+      "*)(field:initialBackoff error:Failed to parse)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryPolicyMaxBackoff) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"retryPolicy\": {\n"
+      "      \"maxAttempts\": 1,\n"
+      "      \"initialBackoff\": \"1s\",\n"
+      "      \"maxBackoff\": \"120sec\",\n"
+      "      \"backoffMultiplier\": 1.6,\n"
+      "      \"retryableStatusCodes\": [ \"ABORTED\" ]\n"
+      "    }\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(std::string(
+      "(Service config parsing "
+      "error)(.*)(referenced_errors)(.*)(Method "
+      "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)(referenced_errors)("
+      ".*)(Client channel "
+      "parser)(.*)(referenced_errors)(.*)(retryPolicy)(.*)(referenced_errors)(."
+      "*)(field:maxBackoff error:failed to parse)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryPolicyBackoffMultiplier) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"retryPolicy\": {\n"
+      "      \"maxAttempts\": 1,\n"
+      "      \"initialBackoff\": \"1s\",\n"
+      "      \"maxBackoff\": \"120s\",\n"
+      "      \"backoffMultiplier\": \"1.6\",\n"
+      "      \"retryableStatusCodes\": [ \"ABORTED\" ]\n"
+      "    }\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(std::string(
+      "(Service config parsing "
+      "error)(.*)(referenced_errors)(.*)(Method "
+      "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)(referenced_errors)("
+      ".*)(Client channel "
+      "parser)(.*)(referenced_errors)(.*)(retryPolicy)(.*)(referenced_errors)(."
+      "*)(field:backoffMultiplier error:should be of type number)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, InvalidRetryPolicyRetryableStatusCodes) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"retryPolicy\": {\n"
+      "      \"maxAttempts\": 1,\n"
+      "      \"initialBackoff\": \"1s\",\n"
+      "      \"maxBackoff\": \"120s\",\n"
+      "      \"backoffMultiplier\": \"1.6\",\n"
+      "      \"retryableStatusCodes\": []\n"
+      "    }\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(std::string(
+      "(Service config parsing "
+      "error)(.*)(referenced_errors)(.*)(Method "
+      "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)(referenced_errors)("
+      ".*)(Client channel "
+      "parser)(.*)(referenced_errors)(.*)(retryPolicy)(.*)(referenced_errors)(."
+      "*)(field:retryableStatusCodes error:should be non-empty)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(ClientChannelParserTest, ValidHealthCheck) {
+  const char* test_json =
+      "{\n"
+      "  \"healthCheckConfig\": {\n"
+      "    \"serviceName\": \"health_check_service_name\"\n"
+      "    }\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* parsed_object =
+      static_cast<grpc_core::internal::ClientChannelGlobalParsedObject*>(
+          svc_cfg->GetParsedGlobalServiceConfigObject(0));
+  ASSERT_TRUE(parsed_object != nullptr);
+  EXPECT_EQ(strcmp(parsed_object->health_check_service_name(),
+                   "health_check_service_name"),
+            0);
+}
+
+TEST_F(ClientChannelParserTest, InvalidHealthCheckMultipleEntries) {
+  const char* test_json =
+      "{\n"
+      "  \"healthCheckConfig\": {\n"
+      "    \"serviceName\": \"health_check_service_name\"\n"
+      "    },\n"
+      "  \"healthCheckConfig\": {\n"
+      "    \"serviceName\": \"health_check_service_name1\"\n"
+      "    }\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Global "
+                  "Params)(.*)(referenced_errors)(.*)(field:healthCheckConfig "
+                  "error:Duplicate entry)"));
   std::smatch match;
   std::string s(grpc_error_string(error));
   EXPECT_TRUE(std::regex_search(s, match, e));
   GRPC_ERROR_UNREF(error);
 }
 
+class MessageSizeParserTest : public ::testing::Test {
+ protected:
+  void SetUp() override {
+    ServiceConfig::Shutdown();
+    ServiceConfig::Init();
+    EXPECT_TRUE(ServiceConfig::RegisterParser(UniquePtr<ServiceConfig::Parser>(
+                    New<MessageSizeParser>())) == 0);
+  }
+};
+
+TEST_F(MessageSizeParserTest, Valid) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"maxRequestMessageBytes\": 1024,\n"
+      "    \"maxResponseMessageBytes\": 1024\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error == GRPC_ERROR_NONE);
+  const auto* vector_ptr = svc_cfg->GetMethodServiceConfigObjectsVector(
+      grpc_slice_from_static_string("/TestServ/TestMethod"));
+  EXPECT_TRUE(vector_ptr != nullptr);
+  auto parsed_object =
+      static_cast<MessageSizeParsedObject*>(((*vector_ptr)[0]).get());
+  ASSERT_TRUE(parsed_object != nullptr);
+  EXPECT_EQ(parsed_object->limits().max_send_size, 1024);
+  EXPECT_EQ(parsed_object->limits().max_recv_size, 1024);
+}
+
+TEST_F(MessageSizeParserTest, InvalidMaxRequestMessageBytes) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"maxRequestMessageBytes\": -1024\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Method "
+                  "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)("
+                  "referenced_errors)(.*)(Message size "
+                  "parser)(.*)(referenced_errors)(.*)(field:"
+                  "maxRequestMessageBytes error:should be non-negative)"));
+  VerifyRegexMatch(error, e);
+}
+
+TEST_F(MessageSizeParserTest, InvalidMaxResponseMessageBytes) {
+  const char* test_json =
+      "{\n"
+      "  \"methodConfig\": [ {\n"
+      "    \"name\": [\n"
+      "      { \"service\": \"TestServ\", \"method\": \"TestMethod\" }\n"
+      "    ],\n"
+      "    \"maxResponseMessageBytes\": {}\n"
+      "  } ]\n"
+      "}";
+  grpc_error* error = GRPC_ERROR_NONE;
+  auto svc_cfg = ServiceConfig::Create(test_json, &error);
+  gpr_log(GPR_ERROR, "%s", grpc_error_string(error));
+  ASSERT_TRUE(error != GRPC_ERROR_NONE);
+  std::regex e(
+      std::string("(Service config parsing "
+                  "error)(.*)(referenced_errors)(.*)(Method "
+                  "Params)(.*)(referenced_errors)(.*)(methodConfig)(.*)("
+                  "referenced_errors)(.*)(Message size "
+                  "parser)(.*)(referenced_errors)(.*)(field:"
+                  "maxResponseMessageBytes error:should be of type number)"));
+  VerifyRegexMatch(error, e);
+}
+
 }  // namespace testing
 }  // namespace grpc_core