Reorganize ext tree

- filters live under filters
- lb_policy, resolver implementations (being part of client_channel) live under client_channel
diff --git a/src/core/ext/filters/client_channel/README.md b/src/core/ext/filters/client_channel/README.md
new file mode 100644
index 0000000..7c209db
--- /dev/null
+++ b/src/core/ext/filters/client_channel/README.md
@@ -0,0 +1,65 @@
+Client Configuration Support for GRPC
+=====================================
+
+This library provides high level configuration machinery to construct client
+channels and load balance between them.
+
+Each grpc_channel is created with a grpc_resolver. It is the resolver's duty
+to resolve a name into a set of arguments for the channel. Such arguments
+might include:
+
+- a list of (ip, port) addresses to connect to
+- a load balancing policy to decide which server to send a request to
+- a set of filters to mutate outgoing requests (say, by adding metadata)
+
+The resolver provides this data as a stream of grpc_channel_args objects to
+the channel. We represent arguments as a stream so that they can be changed
+by the resolver during execution, by reacting to external events (such as
+new service configuration data being pushed to some store).
+
+
+Load Balancing
+--------------
+
+Load balancing configuration is provided by a grpc_lb_policy object.
+
+The primary job of the load balancing policies is to pick a target server
+given only the initial metadata for a request. It does this by providing
+a grpc_subchannel object to the owning channel.
+
+
+Sub-Channels
+------------
+
+A sub-channel provides a connection to a server for a client channel. It has a
+connectivity state like a regular channel, and so can be connected or
+disconnected. This connectivity state can be used to inform load balancing
+decisions (for example, by avoiding disconnected backends).
+
+Configured sub-channels are fully setup to participate in the grpc data plane.
+Their behavior is specified by a set of grpc channel filters defined at their
+construction. To customize this behavior, resolvers build
+grpc_client_channel_factory objects, which use the decorator pattern to customize
+construction arguments for concrete grpc_subchannel instances.
+
+
+Naming for GRPC
+===============
+
+Names in GRPC are represented by a URI (as defined in
+[RFC 3986](https://tools.ietf.org/html/rfc3986)).
+
+The following schemes are currently supported:
+
+dns:///host:port - dns schemes are currently supported so long as authority is
+                   empty (authority based dns resolution is expected in a future
+                   release)
+
+unix:path        - the unix scheme is used to create and connect to unix domain
+                   sockets - the authority must be empty, and the path
+                   represents the absolute or relative path to the desired
+                   socket
+
+ipv4:host:port   - a pre-resolved ipv4 dotted decimal address/port combination
+
+ipv6:[host]:port - a pre-resolved ipv6 address/port combination
diff --git a/src/core/ext/filters/client_channel/channel_connectivity.c b/src/core/ext/filters/client_channel/channel_connectivity.c
new file mode 100644
index 0000000..62f58fb
--- /dev/null
+++ b/src/core/ext/filters/client_channel/channel_connectivity.c
@@ -0,0 +1,226 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/lib/surface/channel.h"
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/lib/iomgr/timer.h"
+#include "src/core/lib/surface/api_trace.h"
+#include "src/core/lib/surface/completion_queue.h"
+
+grpc_connectivity_state grpc_channel_check_connectivity_state(
+    grpc_channel *channel, int try_to_connect) {
+  /* forward through to the underlying client channel */
+  grpc_channel_element *client_channel_elem =
+      grpc_channel_stack_last_element(grpc_channel_get_channel_stack(channel));
+  grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
+  grpc_connectivity_state state;
+  GRPC_API_TRACE(
+      "grpc_channel_check_connectivity_state(channel=%p, try_to_connect=%d)", 2,
+      (channel, try_to_connect));
+  if (client_channel_elem->filter == &grpc_client_channel_filter) {
+    state = grpc_client_channel_check_connectivity_state(
+        &exec_ctx, client_channel_elem, try_to_connect);
+    grpc_exec_ctx_finish(&exec_ctx);
+    return state;
+  }
+  gpr_log(GPR_ERROR,
+          "grpc_channel_check_connectivity_state called on something that is "
+          "not a client channel, but '%s'",
+          client_channel_elem->filter->name);
+  grpc_exec_ctx_finish(&exec_ctx);
+  return GRPC_CHANNEL_SHUTDOWN;
+}
+
+typedef enum {
+  WAITING,
+  CALLING_BACK,
+  CALLING_BACK_AND_FINISHED,
+  CALLED_BACK
+} callback_phase;
+
+typedef struct {
+  gpr_mu mu;
+  callback_phase phase;
+  grpc_closure on_complete;
+  grpc_closure on_timeout;
+  grpc_timer alarm;
+  grpc_connectivity_state state;
+  grpc_completion_queue *cq;
+  grpc_cq_completion completion_storage;
+  grpc_channel *channel;
+  void *tag;
+} state_watcher;
+
+static void delete_state_watcher(grpc_exec_ctx *exec_ctx, state_watcher *w) {
+  grpc_channel_element *client_channel_elem = grpc_channel_stack_last_element(
+      grpc_channel_get_channel_stack(w->channel));
+  if (client_channel_elem->filter == &grpc_client_channel_filter) {
+    GRPC_CHANNEL_INTERNAL_UNREF(exec_ctx, w->channel,
+                                "watch_channel_connectivity");
+  } else {
+    abort();
+  }
+  gpr_mu_destroy(&w->mu);
+  gpr_free(w);
+}
+
+static void finished_completion(grpc_exec_ctx *exec_ctx, void *pw,
+                                grpc_cq_completion *ignored) {
+  int delete = 0;
+  state_watcher *w = pw;
+  gpr_mu_lock(&w->mu);
+  switch (w->phase) {
+    case WAITING:
+    case CALLED_BACK:
+      GPR_UNREACHABLE_CODE(return );
+    case CALLING_BACK:
+      w->phase = CALLED_BACK;
+      break;
+    case CALLING_BACK_AND_FINISHED:
+      delete = 1;
+      break;
+  }
+  gpr_mu_unlock(&w->mu);
+
+  if (delete) {
+    delete_state_watcher(exec_ctx, w);
+  }
+}
+
+static void partly_done(grpc_exec_ctx *exec_ctx, state_watcher *w,
+                        bool due_to_completion, grpc_error *error) {
+  int delete = 0;
+
+  if (due_to_completion) {
+    grpc_timer_cancel(exec_ctx, &w->alarm);
+  }
+
+  gpr_mu_lock(&w->mu);
+
+  if (due_to_completion) {
+    if (grpc_trace_operation_failures) {
+      GRPC_LOG_IF_ERROR("watch_completion_error", GRPC_ERROR_REF(error));
+    }
+    GRPC_ERROR_UNREF(error);
+    error = GRPC_ERROR_NONE;
+  } else {
+    if (error == GRPC_ERROR_NONE) {
+      error = GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+          "Timed out waiting for connection state change");
+    } else if (error == GRPC_ERROR_CANCELLED) {
+      error = GRPC_ERROR_NONE;
+    }
+  }
+  switch (w->phase) {
+    case WAITING:
+      w->phase = CALLING_BACK;
+      grpc_cq_end_op(exec_ctx, w->cq, w->tag, GRPC_ERROR_REF(error),
+                     finished_completion, w, &w->completion_storage);
+      break;
+    case CALLING_BACK:
+      w->phase = CALLING_BACK_AND_FINISHED;
+      break;
+    case CALLING_BACK_AND_FINISHED:
+      GPR_UNREACHABLE_CODE(return );
+    case CALLED_BACK:
+      delete = 1;
+      break;
+  }
+  gpr_mu_unlock(&w->mu);
+
+  if (delete) {
+    delete_state_watcher(exec_ctx, w);
+  }
+
+  GRPC_ERROR_UNREF(error);
+}
+
+static void watch_complete(grpc_exec_ctx *exec_ctx, void *pw,
+                           grpc_error *error) {
+  partly_done(exec_ctx, pw, true, GRPC_ERROR_REF(error));
+}
+
+static void timeout_complete(grpc_exec_ctx *exec_ctx, void *pw,
+                             grpc_error *error) {
+  partly_done(exec_ctx, pw, false, GRPC_ERROR_REF(error));
+}
+
+void grpc_channel_watch_connectivity_state(
+    grpc_channel *channel, grpc_connectivity_state last_observed_state,
+    gpr_timespec deadline, grpc_completion_queue *cq, void *tag) {
+  grpc_channel_element *client_channel_elem =
+      grpc_channel_stack_last_element(grpc_channel_get_channel_stack(channel));
+  grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
+  state_watcher *w = gpr_malloc(sizeof(*w));
+
+  GRPC_API_TRACE(
+      "grpc_channel_watch_connectivity_state("
+      "channel=%p, last_observed_state=%d, "
+      "deadline=gpr_timespec { tv_sec: %" PRId64
+      ", tv_nsec: %d, clock_type: %d }, "
+      "cq=%p, tag=%p)",
+      7, (channel, (int)last_observed_state, deadline.tv_sec, deadline.tv_nsec,
+          (int)deadline.clock_type, cq, tag));
+
+  grpc_cq_begin_op(cq, tag);
+
+  gpr_mu_init(&w->mu);
+  grpc_closure_init(&w->on_complete, watch_complete, w,
+                    grpc_schedule_on_exec_ctx);
+  grpc_closure_init(&w->on_timeout, timeout_complete, w,
+                    grpc_schedule_on_exec_ctx);
+  w->phase = WAITING;
+  w->state = last_observed_state;
+  w->cq = cq;
+  w->tag = tag;
+  w->channel = channel;
+
+  grpc_timer_init(&exec_ctx, &w->alarm,
+                  gpr_convert_clock_type(deadline, GPR_CLOCK_MONOTONIC),
+                  &w->on_timeout, gpr_now(GPR_CLOCK_MONOTONIC));
+
+  if (client_channel_elem->filter == &grpc_client_channel_filter) {
+    GRPC_CHANNEL_INTERNAL_REF(channel, "watch_channel_connectivity");
+    grpc_client_channel_watch_connectivity_state(&exec_ctx, client_channel_elem,
+                                                 grpc_cq_pollset(cq), &w->state,
+                                                 &w->on_complete);
+  } else {
+    abort();
+  }
+
+  grpc_exec_ctx_finish(&exec_ctx);
+}
diff --git a/src/core/ext/filters/client_channel/client_channel.c b/src/core/ext/filters/client_channel/client_channel.c
new file mode 100644
index 0000000..a84c96b
--- /dev/null
+++ b/src/core/ext/filters/client_channel/client_channel.c
@@ -0,0 +1,1393 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+
+#include <stdbool.h>
+#include <stdio.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/sync.h>
+#include <grpc/support/useful.h>
+
+#include "src/core/ext/filters/client_channel/http_connect_handshaker.h"
+#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/retry_throttle.h"
+#include "src/core/ext/filters/client_channel/subchannel.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/channel/connected_channel.h"
+#include "src/core/lib/channel/deadline_filter.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/iomgr.h"
+#include "src/core/lib/iomgr/polling_entity.h"
+#include "src/core/lib/profiling/timers.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/support/string.h"
+#include "src/core/lib/surface/channel.h"
+#include "src/core/lib/transport/connectivity_state.h"
+#include "src/core/lib/transport/metadata.h"
+#include "src/core/lib/transport/metadata_batch.h"
+#include "src/core/lib/transport/service_config.h"
+#include "src/core/lib/transport/static_metadata.h"
+
+/* Client channel implementation */
+
+/*************************************************************************
+ * METHOD-CONFIG TABLE
+ */
+
+typedef enum {
+  /* zero so it can be default initialized */
+  WAIT_FOR_READY_UNSET = 0,
+  WAIT_FOR_READY_FALSE,
+  WAIT_FOR_READY_TRUE
+} wait_for_ready_value;
+
+typedef struct {
+  gpr_refcount refs;
+  gpr_timespec timeout;
+  wait_for_ready_value wait_for_ready;
+} method_parameters;
+
+static method_parameters *method_parameters_ref(
+    method_parameters *method_params) {
+  gpr_ref(&method_params->refs);
+  return method_params;
+}
+
+static void method_parameters_unref(method_parameters *method_params) {
+  if (gpr_unref(&method_params->refs)) {
+    gpr_free(method_params);
+  }
+}
+
+static void *method_parameters_copy(void *value) {
+  return method_parameters_ref(value);
+}
+
+static void method_parameters_free(grpc_exec_ctx *exec_ctx, void *value) {
+  method_parameters_unref(value);
+}
+
+static const grpc_slice_hash_table_vtable method_parameters_vtable = {
+    method_parameters_free, method_parameters_copy};
+
+static bool parse_wait_for_ready(grpc_json *field,
+                                 wait_for_ready_value *wait_for_ready) {
+  if (field->type != GRPC_JSON_TRUE && field->type != GRPC_JSON_FALSE) {
+    return false;
+  }
+  *wait_for_ready = field->type == GRPC_JSON_TRUE ? WAIT_FOR_READY_TRUE
+                                                  : WAIT_FOR_READY_FALSE;
+  return true;
+}
+
+static bool parse_timeout(grpc_json *field, gpr_timespec *timeout) {
+  if (field->type != GRPC_JSON_STRING) return false;
+  size_t len = strlen(field->value);
+  if (field->value[len - 1] != 's') return false;
+  char *buf = gpr_strdup(field->value);
+  buf[len - 1] = '\0';  // Remove trailing 's'.
+  char *decimal_point = strchr(buf, '.');
+  if (decimal_point != NULL) {
+    *decimal_point = '\0';
+    timeout->tv_nsec = gpr_parse_nonnegative_int(decimal_point + 1);
+    if (timeout->tv_nsec == -1) {
+      gpr_free(buf);
+      return false;
+    }
+    // There should always be exactly 3, 6, or 9 fractional digits.
+    int multiplier = 1;
+    switch (strlen(decimal_point + 1)) {
+      case 9:
+        break;
+      case 6:
+        multiplier *= 1000;
+        break;
+      case 3:
+        multiplier *= 1000000;
+        break;
+      default:  // Unsupported number of digits.
+        gpr_free(buf);
+        return false;
+    }
+    timeout->tv_nsec *= multiplier;
+  }
+  timeout->tv_sec = gpr_parse_nonnegative_int(buf);
+  gpr_free(buf);
+  if (timeout->tv_sec == -1) return false;
+  return true;
+}
+
+static void *method_parameters_create_from_json(const grpc_json *json) {
+  wait_for_ready_value wait_for_ready = WAIT_FOR_READY_UNSET;
+  gpr_timespec timeout = {0, 0, GPR_TIMESPAN};
+  for (grpc_json *field = json->child; field != NULL; field = field->next) {
+    if (field->key == NULL) continue;
+    if (strcmp(field->key, "waitForReady") == 0) {
+      if (wait_for_ready != WAIT_FOR_READY_UNSET) return NULL;  // Duplicate.
+      if (!parse_wait_for_ready(field, &wait_for_ready)) return NULL;
+    } else if (strcmp(field->key, "timeout") == 0) {
+      if (timeout.tv_sec > 0 || timeout.tv_nsec > 0) return NULL;  // Duplicate.
+      if (!parse_timeout(field, &timeout)) return NULL;
+    }
+  }
+  method_parameters *value = gpr_malloc(sizeof(method_parameters));
+  gpr_ref_init(&value->refs, 1);
+  value->timeout = timeout;
+  value->wait_for_ready = wait_for_ready;
+  return value;
+}
+
+/*************************************************************************
+ * CHANNEL-WIDE FUNCTIONS
+ */
+
+typedef struct client_channel_channel_data {
+  /** resolver for this channel */
+  grpc_resolver *resolver;
+  /** have we started resolving this channel */
+  bool started_resolving;
+  /** client channel factory */
+  grpc_client_channel_factory *client_channel_factory;
+
+  /** combiner protecting all variables below in this data structure */
+  grpc_combiner *combiner;
+  /** currently active load balancer */
+  grpc_lb_policy *lb_policy;
+  /** retry throttle data */
+  grpc_server_retry_throttle_data *retry_throttle_data;
+  /** maps method names to method_parameters structs */
+  grpc_slice_hash_table *method_params_table;
+  /** incoming resolver result - set by resolver.next() */
+  grpc_channel_args *resolver_result;
+  /** a list of closures that are all waiting for config to come in */
+  grpc_closure_list waiting_for_config_closures;
+  /** resolver callback */
+  grpc_closure on_resolver_result_changed;
+  /** connectivity state being tracked */
+  grpc_connectivity_state_tracker state_tracker;
+  /** when an lb_policy arrives, should we try to exit idle */
+  bool exit_idle_when_lb_policy_arrives;
+  /** owning stack */
+  grpc_channel_stack *owning_stack;
+  /** interested parties (owned) */
+  grpc_pollset_set *interested_parties;
+
+  /* the following properties are guarded by a mutex since API's require them
+     to be instantaneously available */
+  gpr_mu info_mu;
+  char *info_lb_policy_name;
+  /** service config in JSON form */
+  char *info_service_config_json;
+} channel_data;
+
+/** We create one watcher for each new lb_policy that is returned from a
+    resolver, to watch for state changes from the lb_policy. When a state
+    change is seen, we update the channel, and create a new watcher. */
+typedef struct {
+  channel_data *chand;
+  grpc_closure on_changed;
+  grpc_connectivity_state state;
+  grpc_lb_policy *lb_policy;
+} lb_policy_connectivity_watcher;
+
+static void watch_lb_policy_locked(grpc_exec_ctx *exec_ctx, channel_data *chand,
+                                   grpc_lb_policy *lb_policy,
+                                   grpc_connectivity_state current_state);
+
+static void set_channel_connectivity_state_locked(grpc_exec_ctx *exec_ctx,
+                                                  channel_data *chand,
+                                                  grpc_connectivity_state state,
+                                                  grpc_error *error,
+                                                  const char *reason) {
+  if ((state == GRPC_CHANNEL_TRANSIENT_FAILURE ||
+       state == GRPC_CHANNEL_SHUTDOWN) &&
+      chand->lb_policy != NULL) {
+    /* cancel picks with wait_for_ready=false */
+    grpc_lb_policy_cancel_picks_locked(
+        exec_ctx, chand->lb_policy,
+        /* mask= */ GRPC_INITIAL_METADATA_WAIT_FOR_READY,
+        /* check= */ 0, GRPC_ERROR_REF(error));
+  }
+  grpc_connectivity_state_set(exec_ctx, &chand->state_tracker, state, error,
+                              reason);
+}
+
+static void on_lb_policy_state_changed_locked(grpc_exec_ctx *exec_ctx,
+                                              void *arg, grpc_error *error) {
+  lb_policy_connectivity_watcher *w = arg;
+  grpc_connectivity_state publish_state = w->state;
+  /* check if the notification is for the latest policy */
+  if (w->lb_policy == w->chand->lb_policy) {
+    if (publish_state == GRPC_CHANNEL_SHUTDOWN && w->chand->resolver != NULL) {
+      publish_state = GRPC_CHANNEL_TRANSIENT_FAILURE;
+      grpc_resolver_channel_saw_error_locked(exec_ctx, w->chand->resolver);
+      GRPC_LB_POLICY_UNREF(exec_ctx, w->chand->lb_policy, "channel");
+      w->chand->lb_policy = NULL;
+    }
+    set_channel_connectivity_state_locked(exec_ctx, w->chand, publish_state,
+                                          GRPC_ERROR_REF(error), "lb_changed");
+    if (w->state != GRPC_CHANNEL_SHUTDOWN) {
+      watch_lb_policy_locked(exec_ctx, w->chand, w->lb_policy, w->state);
+    }
+  }
+
+  GRPC_CHANNEL_STACK_UNREF(exec_ctx, w->chand->owning_stack, "watch_lb_policy");
+  gpr_free(w);
+}
+
+static void watch_lb_policy_locked(grpc_exec_ctx *exec_ctx, channel_data *chand,
+                                   grpc_lb_policy *lb_policy,
+                                   grpc_connectivity_state current_state) {
+  lb_policy_connectivity_watcher *w = gpr_malloc(sizeof(*w));
+  GRPC_CHANNEL_STACK_REF(chand->owning_stack, "watch_lb_policy");
+
+  w->chand = chand;
+  grpc_closure_init(&w->on_changed, on_lb_policy_state_changed_locked, w,
+                    grpc_combiner_scheduler(chand->combiner, false));
+  w->state = current_state;
+  w->lb_policy = lb_policy;
+  grpc_lb_policy_notify_on_state_change_locked(exec_ctx, lb_policy, &w->state,
+                                               &w->on_changed);
+}
+
+typedef struct {
+  char *server_name;
+  grpc_server_retry_throttle_data *retry_throttle_data;
+} service_config_parsing_state;
+
+static void parse_retry_throttle_params(const grpc_json *field, void *arg) {
+  service_config_parsing_state *parsing_state = arg;
+  if (strcmp(field->key, "retryThrottling") == 0) {
+    if (parsing_state->retry_throttle_data != NULL) 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 != NULL;
+         sub_field = sub_field->next) {
+      if (sub_field->key == NULL) 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 != NULL) {
+          whole_len = (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 = (int)((whole_value * multiplier) + decimal_value);
+        if (milli_token_ratio <= 0) return;
+      }
+    }
+    parsing_state->retry_throttle_data =
+        grpc_retry_throttle_map_get_data_for_server(
+            parsing_state->server_name, max_milli_tokens, milli_token_ratio);
+  }
+}
+
+static void on_resolver_result_changed_locked(grpc_exec_ctx *exec_ctx,
+                                              void *arg, grpc_error *error) {
+  channel_data *chand = arg;
+  char *lb_policy_name = NULL;
+  grpc_lb_policy *lb_policy = NULL;
+  grpc_lb_policy *old_lb_policy;
+  grpc_slice_hash_table *method_params_table = NULL;
+  grpc_connectivity_state state = GRPC_CHANNEL_TRANSIENT_FAILURE;
+  bool exit_idle = false;
+  grpc_error *state_error =
+      GRPC_ERROR_CREATE_FROM_STATIC_STRING("No load balancing policy");
+  char *service_config_json = NULL;
+  service_config_parsing_state parsing_state;
+  memset(&parsing_state, 0, sizeof(parsing_state));
+
+  if (chand->resolver_result != NULL) {
+    // Find LB policy name.
+    const grpc_arg *channel_arg =
+        grpc_channel_args_find(chand->resolver_result, GRPC_ARG_LB_POLICY_NAME);
+    if (channel_arg != NULL) {
+      GPR_ASSERT(channel_arg->type == GRPC_ARG_STRING);
+      lb_policy_name = channel_arg->value.string;
+    }
+    // Special case: If all of the addresses are balancer addresses,
+    // assume that we should use the grpclb policy, regardless of what the
+    // resolver actually specified.
+    channel_arg =
+        grpc_channel_args_find(chand->resolver_result, GRPC_ARG_LB_ADDRESSES);
+    if (channel_arg != NULL) {
+      GPR_ASSERT(channel_arg->type == GRPC_ARG_POINTER);
+      grpc_lb_addresses *addresses = channel_arg->value.pointer.p;
+      bool found_backend_address = false;
+      for (size_t i = 0; i < addresses->num_addresses; ++i) {
+        if (!addresses->addresses[i].is_balancer) {
+          found_backend_address = true;
+          break;
+        }
+      }
+      if (!found_backend_address) {
+        if (lb_policy_name != NULL && strcmp(lb_policy_name, "grpclb") != 0) {
+          gpr_log(GPR_INFO,
+                  "resolver requested LB policy %s but provided only balancer "
+                  "addresses, no backend addresses -- forcing use of grpclb LB "
+                  "policy",
+                  lb_policy_name);
+        }
+        lb_policy_name = "grpclb";
+      }
+    }
+    // Use pick_first if nothing was specified and we didn't select grpclb
+    // above.
+    if (lb_policy_name == NULL) lb_policy_name = "pick_first";
+    // Instantiate LB policy.
+    grpc_lb_policy_args lb_policy_args;
+    lb_policy_args.args = chand->resolver_result;
+    lb_policy_args.client_channel_factory = chand->client_channel_factory;
+    lb_policy_args.combiner = chand->combiner;
+    lb_policy =
+        grpc_lb_policy_create(exec_ctx, lb_policy_name, &lb_policy_args);
+    if (lb_policy != NULL) {
+      GRPC_LB_POLICY_REF(lb_policy, "config_change");
+      GRPC_ERROR_UNREF(state_error);
+      state = grpc_lb_policy_check_connectivity_locked(exec_ctx, lb_policy,
+                                                       &state_error);
+    }
+    // Find service config.
+    channel_arg =
+        grpc_channel_args_find(chand->resolver_result, GRPC_ARG_SERVICE_CONFIG);
+    if (channel_arg != NULL) {
+      GPR_ASSERT(channel_arg->type == GRPC_ARG_STRING);
+      service_config_json = gpr_strdup(channel_arg->value.string);
+      grpc_service_config *service_config =
+          grpc_service_config_create(service_config_json);
+      if (service_config != NULL) {
+        channel_arg =
+            grpc_channel_args_find(chand->resolver_result, GRPC_ARG_SERVER_URI);
+        GPR_ASSERT(channel_arg != NULL);
+        GPR_ASSERT(channel_arg->type == GRPC_ARG_STRING);
+        grpc_uri *uri =
+            grpc_uri_parse(exec_ctx, channel_arg->value.string, true);
+        GPR_ASSERT(uri->path[0] != '\0');
+        parsing_state.server_name =
+            uri->path[0] == '/' ? uri->path + 1 : uri->path;
+        grpc_service_config_parse_global_params(
+            service_config, parse_retry_throttle_params, &parsing_state);
+        parsing_state.server_name = NULL;
+        grpc_uri_destroy(uri);
+        method_params_table = grpc_service_config_create_method_config_table(
+            exec_ctx, service_config, method_parameters_create_from_json,
+            &method_parameters_vtable);
+        grpc_service_config_destroy(service_config);
+      }
+    }
+    // Before we clean up, save a copy of lb_policy_name, since it might
+    // be pointing to data inside chand->resolver_result.
+    // The copy will be saved in chand->lb_policy_name below.
+    lb_policy_name = gpr_strdup(lb_policy_name);
+    grpc_channel_args_destroy(exec_ctx, chand->resolver_result);
+    chand->resolver_result = NULL;
+  }
+
+  if (lb_policy != NULL) {
+    grpc_pollset_set_add_pollset_set(exec_ctx, lb_policy->interested_parties,
+                                     chand->interested_parties);
+  }
+
+  gpr_mu_lock(&chand->info_mu);
+  if (lb_policy_name != NULL) {
+    gpr_free(chand->info_lb_policy_name);
+    chand->info_lb_policy_name = lb_policy_name;
+  }
+  old_lb_policy = chand->lb_policy;
+  chand->lb_policy = lb_policy;
+  if (service_config_json != NULL) {
+    gpr_free(chand->info_service_config_json);
+    chand->info_service_config_json = service_config_json;
+  }
+  gpr_mu_unlock(&chand->info_mu);
+
+  if (chand->retry_throttle_data != NULL) {
+    grpc_server_retry_throttle_data_unref(chand->retry_throttle_data);
+  }
+  chand->retry_throttle_data = parsing_state.retry_throttle_data;
+  if (chand->method_params_table != NULL) {
+    grpc_slice_hash_table_unref(exec_ctx, chand->method_params_table);
+  }
+  chand->method_params_table = method_params_table;
+  if (lb_policy != NULL) {
+    grpc_closure_list_sched(exec_ctx, &chand->waiting_for_config_closures);
+  } else if (chand->resolver == NULL /* disconnected */) {
+    grpc_closure_list_fail_all(&chand->waiting_for_config_closures,
+                               GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                                   "Channel disconnected", &error, 1));
+    grpc_closure_list_sched(exec_ctx, &chand->waiting_for_config_closures);
+  }
+  if (lb_policy != NULL && chand->exit_idle_when_lb_policy_arrives) {
+    GRPC_LB_POLICY_REF(lb_policy, "exit_idle");
+    exit_idle = true;
+    chand->exit_idle_when_lb_policy_arrives = false;
+  }
+
+  if (error == GRPC_ERROR_NONE && chand->resolver) {
+    set_channel_connectivity_state_locked(
+        exec_ctx, chand, state, GRPC_ERROR_REF(state_error), "new_lb+resolver");
+    if (lb_policy != NULL) {
+      watch_lb_policy_locked(exec_ctx, chand, lb_policy, state);
+    }
+    GRPC_CHANNEL_STACK_REF(chand->owning_stack, "resolver");
+    grpc_resolver_next_locked(exec_ctx, chand->resolver,
+                              &chand->resolver_result,
+                              &chand->on_resolver_result_changed);
+  } else {
+    if (chand->resolver != NULL) {
+      grpc_resolver_shutdown_locked(exec_ctx, chand->resolver);
+      GRPC_RESOLVER_UNREF(exec_ctx, chand->resolver, "channel");
+      chand->resolver = NULL;
+    }
+    grpc_error *refs[] = {error, state_error};
+    set_channel_connectivity_state_locked(
+        exec_ctx, chand, GRPC_CHANNEL_SHUTDOWN,
+        GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+            "Got config after disconnection", refs, GPR_ARRAY_SIZE(refs)),
+        "resolver_gone");
+  }
+
+  if (exit_idle) {
+    grpc_lb_policy_exit_idle_locked(exec_ctx, lb_policy);
+    GRPC_LB_POLICY_UNREF(exec_ctx, lb_policy, "exit_idle");
+  }
+
+  if (old_lb_policy != NULL) {
+    grpc_pollset_set_del_pollset_set(
+        exec_ctx, old_lb_policy->interested_parties, chand->interested_parties);
+    GRPC_LB_POLICY_UNREF(exec_ctx, old_lb_policy, "channel");
+  }
+
+  if (lb_policy != NULL) {
+    GRPC_LB_POLICY_UNREF(exec_ctx, lb_policy, "config_change");
+  }
+
+  GRPC_CHANNEL_STACK_UNREF(exec_ctx, chand->owning_stack, "resolver");
+  GRPC_ERROR_UNREF(state_error);
+}
+
+static void start_transport_op_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                      grpc_error *error_ignored) {
+  grpc_transport_op *op = arg;
+  grpc_channel_element *elem = op->transport_private.args[0];
+  channel_data *chand = elem->channel_data;
+
+  if (op->on_connectivity_state_change != NULL) {
+    grpc_connectivity_state_notify_on_state_change(
+        exec_ctx, &chand->state_tracker, op->connectivity_state,
+        op->on_connectivity_state_change);
+    op->on_connectivity_state_change = NULL;
+    op->connectivity_state = NULL;
+  }
+
+  if (op->send_ping != NULL) {
+    if (chand->lb_policy == NULL) {
+      grpc_closure_sched(
+          exec_ctx, op->send_ping,
+          GRPC_ERROR_CREATE_FROM_STATIC_STRING("Ping with no load balancing"));
+    } else {
+      grpc_lb_policy_ping_one_locked(exec_ctx, chand->lb_policy, op->send_ping);
+      op->bind_pollset = NULL;
+    }
+    op->send_ping = NULL;
+  }
+
+  if (op->disconnect_with_error != GRPC_ERROR_NONE) {
+    if (chand->resolver != NULL) {
+      set_channel_connectivity_state_locked(
+          exec_ctx, chand, GRPC_CHANNEL_SHUTDOWN,
+          GRPC_ERROR_REF(op->disconnect_with_error), "disconnect");
+      grpc_resolver_shutdown_locked(exec_ctx, chand->resolver);
+      GRPC_RESOLVER_UNREF(exec_ctx, chand->resolver, "channel");
+      chand->resolver = NULL;
+      if (!chand->started_resolving) {
+        grpc_closure_list_fail_all(&chand->waiting_for_config_closures,
+                                   GRPC_ERROR_REF(op->disconnect_with_error));
+        grpc_closure_list_sched(exec_ctx, &chand->waiting_for_config_closures);
+      }
+      if (chand->lb_policy != NULL) {
+        grpc_pollset_set_del_pollset_set(exec_ctx,
+                                         chand->lb_policy->interested_parties,
+                                         chand->interested_parties);
+        GRPC_LB_POLICY_UNREF(exec_ctx, chand->lb_policy, "channel");
+        chand->lb_policy = NULL;
+      }
+    }
+    GRPC_ERROR_UNREF(op->disconnect_with_error);
+  }
+  GRPC_CHANNEL_STACK_UNREF(exec_ctx, chand->owning_stack, "start_transport_op");
+
+  grpc_closure_sched(exec_ctx, op->on_consumed, GRPC_ERROR_NONE);
+}
+
+static void cc_start_transport_op(grpc_exec_ctx *exec_ctx,
+                                  grpc_channel_element *elem,
+                                  grpc_transport_op *op) {
+  channel_data *chand = elem->channel_data;
+
+  GPR_ASSERT(op->set_accept_stream == false);
+  if (op->bind_pollset != NULL) {
+    grpc_pollset_set_add_pollset(exec_ctx, chand->interested_parties,
+                                 op->bind_pollset);
+  }
+
+  op->transport_private.args[0] = elem;
+  GRPC_CHANNEL_STACK_REF(chand->owning_stack, "start_transport_op");
+  grpc_closure_sched(
+      exec_ctx, grpc_closure_init(
+                    &op->transport_private.closure, start_transport_op_locked,
+                    op, grpc_combiner_scheduler(chand->combiner, false)),
+      GRPC_ERROR_NONE);
+}
+
+static void cc_get_channel_info(grpc_exec_ctx *exec_ctx,
+                                grpc_channel_element *elem,
+                                const grpc_channel_info *info) {
+  channel_data *chand = elem->channel_data;
+  gpr_mu_lock(&chand->info_mu);
+  if (info->lb_policy_name != NULL) {
+    *info->lb_policy_name = chand->info_lb_policy_name == NULL
+                                ? NULL
+                                : gpr_strdup(chand->info_lb_policy_name);
+  }
+  if (info->service_config_json != NULL) {
+    *info->service_config_json =
+        chand->info_service_config_json == NULL
+            ? NULL
+            : gpr_strdup(chand->info_service_config_json);
+  }
+  gpr_mu_unlock(&chand->info_mu);
+}
+
+/* Constructor for channel_data */
+static grpc_error *cc_init_channel_elem(grpc_exec_ctx *exec_ctx,
+                                        grpc_channel_element *elem,
+                                        grpc_channel_element_args *args) {
+  channel_data *chand = elem->channel_data;
+  GPR_ASSERT(args->is_last);
+  GPR_ASSERT(elem->filter == &grpc_client_channel_filter);
+  // Initialize data members.
+  chand->combiner = grpc_combiner_create(NULL);
+  gpr_mu_init(&chand->info_mu);
+  chand->owning_stack = args->channel_stack;
+  grpc_closure_init(&chand->on_resolver_result_changed,
+                    on_resolver_result_changed_locked, chand,
+                    grpc_combiner_scheduler(chand->combiner, false));
+  chand->interested_parties = grpc_pollset_set_create();
+  grpc_connectivity_state_init(&chand->state_tracker, GRPC_CHANNEL_IDLE,
+                               "client_channel");
+  // Record client channel factory.
+  const grpc_arg *arg = grpc_channel_args_find(args->channel_args,
+                                               GRPC_ARG_CLIENT_CHANNEL_FACTORY);
+  GPR_ASSERT(arg != NULL);
+  GPR_ASSERT(arg->type == GRPC_ARG_POINTER);
+  grpc_client_channel_factory_ref(arg->value.pointer.p);
+  chand->client_channel_factory = arg->value.pointer.p;
+  // Get server name to resolve, using proxy mapper if needed.
+  arg = grpc_channel_args_find(args->channel_args, GRPC_ARG_SERVER_URI);
+  GPR_ASSERT(arg != NULL);
+  GPR_ASSERT(arg->type == GRPC_ARG_STRING);
+  char *proxy_name = NULL;
+  grpc_channel_args *new_args = NULL;
+  grpc_proxy_mappers_map_name(exec_ctx, arg->value.string, args->channel_args,
+                              &proxy_name, &new_args);
+  // Instantiate resolver.
+  chand->resolver = grpc_resolver_create(
+      exec_ctx, proxy_name != NULL ? proxy_name : arg->value.string,
+      new_args != NULL ? new_args : args->channel_args,
+      chand->interested_parties, chand->combiner);
+  if (proxy_name != NULL) gpr_free(proxy_name);
+  if (new_args != NULL) grpc_channel_args_destroy(exec_ctx, new_args);
+  if (chand->resolver == NULL) {
+    return GRPC_ERROR_CREATE_FROM_STATIC_STRING("resolver creation failed");
+  }
+  return GRPC_ERROR_NONE;
+}
+
+static void shutdown_resolver_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                     grpc_error *error) {
+  grpc_resolver *resolver = arg;
+  grpc_resolver_shutdown_locked(exec_ctx, resolver);
+  GRPC_RESOLVER_UNREF(exec_ctx, resolver, "channel");
+}
+
+/* Destructor for channel_data */
+static void cc_destroy_channel_elem(grpc_exec_ctx *exec_ctx,
+                                    grpc_channel_element *elem) {
+  channel_data *chand = elem->channel_data;
+  if (chand->resolver != NULL) {
+    grpc_closure_sched(
+        exec_ctx,
+        grpc_closure_create(shutdown_resolver_locked, chand->resolver,
+                            grpc_combiner_scheduler(chand->combiner, false)),
+        GRPC_ERROR_NONE);
+  }
+  if (chand->client_channel_factory != NULL) {
+    grpc_client_channel_factory_unref(exec_ctx, chand->client_channel_factory);
+  }
+  if (chand->lb_policy != NULL) {
+    grpc_pollset_set_del_pollset_set(exec_ctx,
+                                     chand->lb_policy->interested_parties,
+                                     chand->interested_parties);
+    GRPC_LB_POLICY_UNREF(exec_ctx, chand->lb_policy, "channel");
+  }
+  gpr_free(chand->info_lb_policy_name);
+  gpr_free(chand->info_service_config_json);
+  if (chand->retry_throttle_data != NULL) {
+    grpc_server_retry_throttle_data_unref(chand->retry_throttle_data);
+  }
+  if (chand->method_params_table != NULL) {
+    grpc_slice_hash_table_unref(exec_ctx, chand->method_params_table);
+  }
+  grpc_connectivity_state_destroy(exec_ctx, &chand->state_tracker);
+  grpc_pollset_set_destroy(exec_ctx, chand->interested_parties);
+  GRPC_COMBINER_UNREF(exec_ctx, chand->combiner, "client_channel");
+  gpr_mu_destroy(&chand->info_mu);
+}
+
+/*************************************************************************
+ * PER-CALL FUNCTIONS
+ */
+
+#define GET_CALL(call_data) \
+  ((grpc_subchannel_call *)(gpr_atm_acq_load(&(call_data)->subchannel_call)))
+
+#define CANCELLED_CALL ((grpc_subchannel_call *)1)
+
+typedef enum {
+  /* zero so that it can be default-initialized */
+  GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING = 0,
+  GRPC_SUBCHANNEL_CALL_HOLDER_PICKING_SUBCHANNEL
+} subchannel_creation_phase;
+
+/** Call data.  Holds a pointer to grpc_subchannel_call and the
+    associated machinery to create such a pointer.
+    Handles queueing of stream ops until a call object is ready, waiting
+    for initial metadata before trying to create a call object,
+    and handling cancellation gracefully. */
+typedef struct client_channel_call_data {
+  // State for handling deadlines.
+  // The code in deadline_filter.c requires this to be the first field.
+  // TODO(roth): This is slightly sub-optimal in that grpc_deadline_state
+  // and this struct both independently store a pointer to the call
+  // stack and each has its own mutex.  If/when we have time, find a way
+  // to avoid this without breaking the grpc_deadline_state abstraction.
+  grpc_deadline_state deadline_state;
+
+  grpc_slice path;  // Request path.
+  gpr_timespec call_start_time;
+  gpr_timespec deadline;
+  grpc_server_retry_throttle_data *retry_throttle_data;
+  method_parameters *method_params;
+
+  grpc_error *cancel_error;
+
+  /** either 0 for no call, 1 for cancelled, or a pointer to a
+      grpc_subchannel_call */
+  gpr_atm subchannel_call;
+  gpr_arena *arena;
+
+  subchannel_creation_phase creation_phase;
+  grpc_connected_subchannel *connected_subchannel;
+  grpc_polling_entity *pollent;
+
+  grpc_transport_stream_op **waiting_ops;
+  size_t waiting_ops_count;
+  size_t waiting_ops_capacity;
+
+  grpc_closure next_step;
+
+  grpc_call_stack *owning_call;
+
+  grpc_linked_mdelem lb_token_mdelem;
+
+  grpc_closure on_complete;
+  grpc_closure *original_on_complete;
+} call_data;
+
+grpc_subchannel_call *grpc_client_channel_get_subchannel_call(
+    grpc_call_element *call_elem) {
+  grpc_subchannel_call *scc = GET_CALL((call_data *)call_elem->call_data);
+  return scc == CANCELLED_CALL ? NULL : scc;
+}
+
+static void add_waiting_locked(call_data *calld, grpc_transport_stream_op *op) {
+  GPR_TIMER_BEGIN("add_waiting_locked", 0);
+  if (calld->waiting_ops_count == calld->waiting_ops_capacity) {
+    calld->waiting_ops_capacity = GPR_MAX(3, 2 * calld->waiting_ops_capacity);
+    calld->waiting_ops =
+        gpr_realloc(calld->waiting_ops,
+                    calld->waiting_ops_capacity * sizeof(*calld->waiting_ops));
+  }
+  calld->waiting_ops[calld->waiting_ops_count++] = op;
+  GPR_TIMER_END("add_waiting_locked", 0);
+}
+
+static void fail_locked(grpc_exec_ctx *exec_ctx, call_data *calld,
+                        grpc_error *error) {
+  size_t i;
+  for (i = 0; i < calld->waiting_ops_count; i++) {
+    grpc_transport_stream_op_finish_with_failure(
+        exec_ctx, calld->waiting_ops[i], GRPC_ERROR_REF(error));
+  }
+  calld->waiting_ops_count = 0;
+  GRPC_ERROR_UNREF(error);
+}
+
+static void retry_waiting_locked(grpc_exec_ctx *exec_ctx, call_data *calld) {
+  if (calld->waiting_ops_count == 0) {
+    return;
+  }
+
+  grpc_subchannel_call *call = GET_CALL(calld);
+  grpc_transport_stream_op **ops = calld->waiting_ops;
+  size_t nops = calld->waiting_ops_count;
+  if (call == CANCELLED_CALL) {
+    fail_locked(exec_ctx, calld, GRPC_ERROR_CANCELLED);
+    return;
+  }
+  calld->waiting_ops = NULL;
+  calld->waiting_ops_count = 0;
+  calld->waiting_ops_capacity = 0;
+  for (size_t i = 0; i < nops; i++) {
+    grpc_subchannel_call_process_op(exec_ctx, call, ops[i]);
+  }
+  gpr_free(ops);
+}
+
+// Sets calld->method_params and calld->retry_throttle_data.
+// If the method params specify a timeout, populates
+// *per_method_deadline and returns true.
+static bool set_call_method_params_from_service_config_locked(
+    grpc_exec_ctx *exec_ctx, grpc_call_element *elem,
+    gpr_timespec *per_method_deadline) {
+  channel_data *chand = elem->channel_data;
+  call_data *calld = elem->call_data;
+  if (chand->retry_throttle_data != NULL) {
+    calld->retry_throttle_data =
+        grpc_server_retry_throttle_data_ref(chand->retry_throttle_data);
+  }
+  if (chand->method_params_table != NULL) {
+    calld->method_params = grpc_method_config_table_get(
+        exec_ctx, chand->method_params_table, calld->path);
+    if (calld->method_params != NULL) {
+      method_parameters_ref(calld->method_params);
+      if (gpr_time_cmp(calld->method_params->timeout,
+                       gpr_time_0(GPR_TIMESPAN)) != 0) {
+        *per_method_deadline =
+            gpr_time_add(calld->call_start_time, calld->method_params->timeout);
+        return true;
+      }
+    }
+  }
+  return false;
+}
+
+static void apply_final_configuration_locked(grpc_exec_ctx *exec_ctx,
+                                             grpc_call_element *elem) {
+  /* apply service-config level configuration to the call (now that we're
+   * certain it exists) */
+  call_data *calld = elem->call_data;
+  gpr_timespec per_method_deadline;
+  if (set_call_method_params_from_service_config_locked(exec_ctx, elem,
+                                                        &per_method_deadline)) {
+    // If the deadline from the service config is shorter than the one
+    // from the client API, reset the deadline timer.
+    if (gpr_time_cmp(per_method_deadline, calld->deadline) < 0) {
+      calld->deadline = per_method_deadline;
+      grpc_deadline_state_reset(exec_ctx, elem, calld->deadline);
+    }
+  }
+}
+
+static void subchannel_ready_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                    grpc_error *error) {
+  grpc_call_element *elem = arg;
+  call_data *calld = elem->call_data;
+  channel_data *chand = elem->channel_data;
+  GPR_ASSERT(calld->creation_phase ==
+             GRPC_SUBCHANNEL_CALL_HOLDER_PICKING_SUBCHANNEL);
+  grpc_polling_entity_del_from_pollset_set(exec_ctx, calld->pollent,
+                                           chand->interested_parties);
+  calld->creation_phase = GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING;
+  if (calld->connected_subchannel == NULL) {
+    gpr_atm_no_barrier_store(&calld->subchannel_call, 1);
+    fail_locked(exec_ctx, calld,
+                GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                    "Failed to create subchannel", &error, 1));
+  } else if (GET_CALL(calld) == CANCELLED_CALL) {
+    /* already cancelled before subchannel became ready */
+    grpc_error *cancellation_error =
+        GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+            "Cancelled before creating subchannel", &error, 1);
+    /* if due to deadline, attach the deadline exceeded status to the error */
+    if (gpr_time_cmp(calld->deadline, gpr_now(GPR_CLOCK_MONOTONIC)) < 0) {
+      cancellation_error =
+          grpc_error_set_int(cancellation_error, GRPC_ERROR_INT_GRPC_STATUS,
+                             GRPC_STATUS_DEADLINE_EXCEEDED);
+    }
+    fail_locked(exec_ctx, calld, cancellation_error);
+  } else {
+    /* Create call on subchannel. */
+    grpc_subchannel_call *subchannel_call = NULL;
+    const grpc_connected_subchannel_call_args call_args = {
+        .pollent = calld->pollent,
+        .path = calld->path,
+        .start_time = calld->call_start_time,
+        .deadline = calld->deadline,
+        .arena = calld->arena};
+    grpc_error *new_error = grpc_connected_subchannel_create_call(
+        exec_ctx, calld->connected_subchannel, &call_args, &subchannel_call);
+    if (new_error != GRPC_ERROR_NONE) {
+      new_error = grpc_error_add_child(new_error, error);
+      subchannel_call = CANCELLED_CALL;
+      fail_locked(exec_ctx, calld, new_error);
+    }
+    gpr_atm_rel_store(&calld->subchannel_call,
+                      (gpr_atm)(uintptr_t)subchannel_call);
+    retry_waiting_locked(exec_ctx, calld);
+  }
+  GRPC_CALL_STACK_UNREF(exec_ctx, calld->owning_call, "pick_subchannel");
+}
+
+static char *cc_get_peer(grpc_exec_ctx *exec_ctx, grpc_call_element *elem) {
+  call_data *calld = elem->call_data;
+  grpc_subchannel_call *subchannel_call = GET_CALL(calld);
+  if (subchannel_call == NULL || subchannel_call == CANCELLED_CALL) {
+    return NULL;
+  } else {
+    return grpc_subchannel_call_get_peer(exec_ctx, subchannel_call);
+  }
+}
+
+typedef struct {
+  grpc_metadata_batch *initial_metadata;
+  uint32_t initial_metadata_flags;
+  grpc_connected_subchannel **connected_subchannel;
+  grpc_closure *on_ready;
+  grpc_call_element *elem;
+  grpc_closure closure;
+} continue_picking_args;
+
+/** Return true if subchannel is available immediately (in which case on_ready
+    should not be called), or false otherwise (in which case on_ready should be
+    called when the subchannel is available). */
+static bool pick_subchannel_locked(
+    grpc_exec_ctx *exec_ctx, grpc_call_element *elem,
+    grpc_metadata_batch *initial_metadata, uint32_t initial_metadata_flags,
+    grpc_connected_subchannel **connected_subchannel, grpc_closure *on_ready,
+    grpc_error *error);
+
+static void continue_picking_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                    grpc_error *error) {
+  continue_picking_args *cpa = arg;
+  if (cpa->connected_subchannel == NULL) {
+    /* cancelled, do nothing */
+  } else if (error != GRPC_ERROR_NONE) {
+    grpc_closure_sched(exec_ctx, cpa->on_ready, GRPC_ERROR_REF(error));
+  } else {
+    if (pick_subchannel_locked(exec_ctx, cpa->elem, cpa->initial_metadata,
+                               cpa->initial_metadata_flags,
+                               cpa->connected_subchannel, cpa->on_ready,
+                               GRPC_ERROR_NONE)) {
+      grpc_closure_sched(exec_ctx, cpa->on_ready, GRPC_ERROR_NONE);
+    }
+  }
+  gpr_free(cpa);
+}
+
+static bool pick_subchannel_locked(
+    grpc_exec_ctx *exec_ctx, grpc_call_element *elem,
+    grpc_metadata_batch *initial_metadata, uint32_t initial_metadata_flags,
+    grpc_connected_subchannel **connected_subchannel, grpc_closure *on_ready,
+    grpc_error *error) {
+  GPR_TIMER_BEGIN("pick_subchannel", 0);
+
+  channel_data *chand = elem->channel_data;
+  call_data *calld = elem->call_data;
+  continue_picking_args *cpa;
+  grpc_closure *closure;
+
+  GPR_ASSERT(connected_subchannel);
+
+  if (initial_metadata == NULL) {
+    if (chand->lb_policy != NULL) {
+      grpc_lb_policy_cancel_pick_locked(exec_ctx, chand->lb_policy,
+                                        connected_subchannel,
+                                        GRPC_ERROR_REF(error));
+    }
+    for (closure = chand->waiting_for_config_closures.head; closure != NULL;
+         closure = closure->next_data.next) {
+      cpa = closure->cb_arg;
+      if (cpa->connected_subchannel == connected_subchannel) {
+        cpa->connected_subchannel = NULL;
+        grpc_closure_sched(exec_ctx, cpa->on_ready,
+                           GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                               "Pick cancelled", &error, 1));
+      }
+    }
+    GPR_TIMER_END("pick_subchannel", 0);
+    GRPC_ERROR_UNREF(error);
+    return true;
+  }
+  GPR_ASSERT(error == GRPC_ERROR_NONE);
+  if (chand->lb_policy != NULL) {
+    apply_final_configuration_locked(exec_ctx, elem);
+    grpc_lb_policy *lb_policy = chand->lb_policy;
+    GRPC_LB_POLICY_REF(lb_policy, "pick_subchannel");
+    // If the application explicitly set wait_for_ready, use that.
+    // Otherwise, if the service config specified a value for this
+    // method, use that.
+    const bool wait_for_ready_set_from_api =
+        initial_metadata_flags &
+        GRPC_INITIAL_METADATA_WAIT_FOR_READY_EXPLICITLY_SET;
+    const bool wait_for_ready_set_from_service_config =
+        calld->method_params != NULL &&
+        calld->method_params->wait_for_ready != WAIT_FOR_READY_UNSET;
+    if (!wait_for_ready_set_from_api &&
+        wait_for_ready_set_from_service_config) {
+      if (calld->method_params->wait_for_ready == WAIT_FOR_READY_TRUE) {
+        initial_metadata_flags |= GRPC_INITIAL_METADATA_WAIT_FOR_READY;
+      } else {
+        initial_metadata_flags &= ~GRPC_INITIAL_METADATA_WAIT_FOR_READY;
+      }
+    }
+    const grpc_lb_policy_pick_args inputs = {
+        initial_metadata, initial_metadata_flags, &calld->lb_token_mdelem,
+        gpr_inf_future(GPR_CLOCK_MONOTONIC)};
+    const bool result = grpc_lb_policy_pick_locked(
+        exec_ctx, lb_policy, &inputs, connected_subchannel, NULL, on_ready);
+    GRPC_LB_POLICY_UNREF(exec_ctx, lb_policy, "pick_subchannel");
+    GPR_TIMER_END("pick_subchannel", 0);
+    return result;
+  }
+  if (chand->resolver != NULL && !chand->started_resolving) {
+    chand->started_resolving = true;
+    GRPC_CHANNEL_STACK_REF(chand->owning_stack, "resolver");
+    grpc_resolver_next_locked(exec_ctx, chand->resolver,
+                              &chand->resolver_result,
+                              &chand->on_resolver_result_changed);
+  }
+  if (chand->resolver != NULL) {
+    cpa = gpr_malloc(sizeof(*cpa));
+    cpa->initial_metadata = initial_metadata;
+    cpa->initial_metadata_flags = initial_metadata_flags;
+    cpa->connected_subchannel = connected_subchannel;
+    cpa->on_ready = on_ready;
+    cpa->elem = elem;
+    grpc_closure_init(&cpa->closure, continue_picking_locked, cpa,
+                      grpc_combiner_scheduler(chand->combiner, true));
+    grpc_closure_list_append(&chand->waiting_for_config_closures, &cpa->closure,
+                             GRPC_ERROR_NONE);
+  } else {
+    grpc_closure_sched(exec_ctx, on_ready,
+                       GRPC_ERROR_CREATE_FROM_STATIC_STRING("Disconnected"));
+  }
+
+  GPR_TIMER_END("pick_subchannel", 0);
+  return false;
+}
+
+static void start_transport_stream_op_locked_inner(grpc_exec_ctx *exec_ctx,
+                                                   grpc_transport_stream_op *op,
+                                                   grpc_call_element *elem) {
+  channel_data *chand = elem->channel_data;
+  call_data *calld = elem->call_data;
+  grpc_subchannel_call *call;
+
+  /* need to recheck that another thread hasn't set the call */
+  call = GET_CALL(calld);
+  if (call == CANCELLED_CALL) {
+    grpc_transport_stream_op_finish_with_failure(
+        exec_ctx, op, GRPC_ERROR_REF(calld->cancel_error));
+    /* early out */
+    return;
+  }
+  if (call != NULL) {
+    grpc_subchannel_call_process_op(exec_ctx, call, op);
+    /* early out */
+    return;
+  }
+  /* if this is a cancellation, then we can raise our cancelled flag */
+  if (op->cancel_error != GRPC_ERROR_NONE) {
+    if (!gpr_atm_rel_cas(&calld->subchannel_call, 0,
+                         (gpr_atm)(uintptr_t)CANCELLED_CALL)) {
+      /* recurse to retry */
+      start_transport_stream_op_locked_inner(exec_ctx, op, elem);
+      /* early out */
+      return;
+    } else {
+      /* Stash a copy of cancel_error in our call data, so that we can use
+         it for subsequent operations.  This ensures that if the call is
+         cancelled before any ops are passed down (e.g., if the deadline
+         is in the past when the call starts), we can return the right
+         error to the caller when the first op does get passed down. */
+      calld->cancel_error = GRPC_ERROR_REF(op->cancel_error);
+      switch (calld->creation_phase) {
+        case GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING:
+          fail_locked(exec_ctx, calld, GRPC_ERROR_REF(op->cancel_error));
+          break;
+        case GRPC_SUBCHANNEL_CALL_HOLDER_PICKING_SUBCHANNEL:
+          pick_subchannel_locked(exec_ctx, elem, NULL, 0,
+                                 &calld->connected_subchannel, NULL,
+                                 GRPC_ERROR_REF(op->cancel_error));
+          break;
+      }
+      grpc_transport_stream_op_finish_with_failure(
+          exec_ctx, op, GRPC_ERROR_REF(op->cancel_error));
+      /* early out */
+      return;
+    }
+  }
+  /* if we don't have a subchannel, try to get one */
+  if (calld->creation_phase == GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING &&
+      calld->connected_subchannel == NULL &&
+      op->send_initial_metadata != NULL) {
+    calld->creation_phase = GRPC_SUBCHANNEL_CALL_HOLDER_PICKING_SUBCHANNEL;
+    grpc_closure_init(&calld->next_step, subchannel_ready_locked, elem,
+                      grpc_combiner_scheduler(chand->combiner, true));
+    GRPC_CALL_STACK_REF(calld->owning_call, "pick_subchannel");
+    /* If a subchannel is not available immediately, the polling entity from
+       call_data should be provided to channel_data's interested_parties, so
+       that IO of the lb_policy and resolver could be done under it. */
+    if (pick_subchannel_locked(exec_ctx, elem, op->send_initial_metadata,
+                               op->send_initial_metadata_flags,
+                               &calld->connected_subchannel, &calld->next_step,
+                               GRPC_ERROR_NONE)) {
+      calld->creation_phase = GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING;
+      GRPC_CALL_STACK_UNREF(exec_ctx, calld->owning_call, "pick_subchannel");
+    } else {
+      grpc_polling_entity_add_to_pollset_set(exec_ctx, calld->pollent,
+                                             chand->interested_parties);
+    }
+  }
+  /* if we've got a subchannel, then let's ask it to create a call */
+  if (calld->creation_phase == GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING &&
+      calld->connected_subchannel != NULL) {
+    grpc_subchannel_call *subchannel_call = NULL;
+    const grpc_connected_subchannel_call_args call_args = {
+        .pollent = calld->pollent,
+        .path = calld->path,
+        .start_time = calld->call_start_time,
+        .deadline = calld->deadline,
+        .arena = calld->arena};
+    grpc_error *error = grpc_connected_subchannel_create_call(
+        exec_ctx, calld->connected_subchannel, &call_args, &subchannel_call);
+    if (error != GRPC_ERROR_NONE) {
+      subchannel_call = CANCELLED_CALL;
+      fail_locked(exec_ctx, calld, GRPC_ERROR_REF(error));
+      grpc_transport_stream_op_finish_with_failure(exec_ctx, op, error);
+    }
+    gpr_atm_rel_store(&calld->subchannel_call,
+                      (gpr_atm)(uintptr_t)subchannel_call);
+    retry_waiting_locked(exec_ctx, calld);
+    /* recurse to retry */
+    start_transport_stream_op_locked_inner(exec_ctx, op, elem);
+    /* early out */
+    return;
+  }
+  /* nothing to be done but wait */
+  add_waiting_locked(calld, op);
+}
+
+static void on_complete(grpc_exec_ctx *exec_ctx, void *arg, grpc_error *error) {
+  grpc_call_element *elem = arg;
+  call_data *calld = elem->call_data;
+  if (calld->retry_throttle_data != NULL) {
+    if (error == GRPC_ERROR_NONE) {
+      grpc_server_retry_throttle_data_record_success(
+          calld->retry_throttle_data);
+    } else {
+      // TODO(roth): In a subsequent PR, check the return value here and
+      // decide whether or not to retry.  Note that we should only
+      // record failures whose statuses match the configured retryable
+      // or non-fatal status codes.
+      grpc_server_retry_throttle_data_record_failure(
+          calld->retry_throttle_data);
+    }
+  }
+  grpc_closure_run(exec_ctx, calld->original_on_complete,
+                   GRPC_ERROR_REF(error));
+}
+
+static void start_transport_stream_op_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                             grpc_error *error_ignored) {
+  GPR_TIMER_BEGIN("start_transport_stream_op_locked", 0);
+
+  grpc_transport_stream_op *op = arg;
+  grpc_call_element *elem = op->handler_private.args[0];
+  call_data *calld = elem->call_data;
+
+  if (op->recv_trailing_metadata != NULL) {
+    GPR_ASSERT(op->on_complete != NULL);
+    calld->original_on_complete = op->on_complete;
+    grpc_closure_init(&calld->on_complete, on_complete, elem,
+                      grpc_schedule_on_exec_ctx);
+    op->on_complete = &calld->on_complete;
+  }
+
+  start_transport_stream_op_locked_inner(exec_ctx, op, elem);
+
+  GRPC_CALL_STACK_UNREF(exec_ctx, calld->owning_call,
+                        "start_transport_stream_op");
+  GPR_TIMER_END("start_transport_stream_op_locked", 0);
+}
+
+/* The logic here is fairly complicated, due to (a) the fact that we
+   need to handle the case where we receive the send op before the
+   initial metadata op, and (b) the need for efficiency, especially in
+   the streaming case.
+
+   We use double-checked locking to initially see if initialization has been
+   performed. If it has not, we acquire the combiner and perform initialization.
+   If it has, we proceed on the fast path. */
+static void cc_start_transport_stream_op(grpc_exec_ctx *exec_ctx,
+                                         grpc_call_element *elem,
+                                         grpc_transport_stream_op *op) {
+  call_data *calld = elem->call_data;
+  channel_data *chand = elem->channel_data;
+  GRPC_CALL_LOG_OP(GPR_INFO, elem, op);
+  grpc_deadline_state_client_start_transport_stream_op(exec_ctx, elem, op);
+  /* try to (atomically) get the call */
+  grpc_subchannel_call *call = GET_CALL(calld);
+  GPR_TIMER_BEGIN("cc_start_transport_stream_op", 0);
+  if (call == CANCELLED_CALL) {
+    grpc_transport_stream_op_finish_with_failure(
+        exec_ctx, op, GRPC_ERROR_REF(calld->cancel_error));
+    GPR_TIMER_END("cc_start_transport_stream_op", 0);
+    /* early out */
+    return;
+  }
+  if (call != NULL) {
+    grpc_subchannel_call_process_op(exec_ctx, call, op);
+    GPR_TIMER_END("cc_start_transport_stream_op", 0);
+    /* early out */
+    return;
+  }
+  /* we failed; lock and figure out what to do */
+  GRPC_CALL_STACK_REF(calld->owning_call, "start_transport_stream_op");
+  op->handler_private.args[0] = elem;
+  grpc_closure_sched(
+      exec_ctx,
+      grpc_closure_init(&op->handler_private.closure,
+                        start_transport_stream_op_locked, op,
+                        grpc_combiner_scheduler(chand->combiner, false)),
+      GRPC_ERROR_NONE);
+  GPR_TIMER_END("cc_start_transport_stream_op", 0);
+}
+
+/* Constructor for call_data */
+static grpc_error *cc_init_call_elem(grpc_exec_ctx *exec_ctx,
+                                     grpc_call_element *elem,
+                                     const grpc_call_element_args *args) {
+  call_data *calld = elem->call_data;
+  // Initialize data members.
+  grpc_deadline_state_init(exec_ctx, elem, args->call_stack);
+  calld->path = grpc_slice_ref_internal(args->path);
+  calld->call_start_time = args->start_time;
+  calld->deadline = gpr_convert_clock_type(args->deadline, GPR_CLOCK_MONOTONIC);
+  calld->owning_call = args->call_stack;
+  calld->arena = args->arena;
+  grpc_deadline_state_start(exec_ctx, elem, calld->deadline);
+  return GRPC_ERROR_NONE;
+}
+
+/* Destructor for call_data */
+static void cc_destroy_call_elem(grpc_exec_ctx *exec_ctx,
+                                 grpc_call_element *elem,
+                                 const grpc_call_final_info *final_info,
+                                 grpc_closure *then_schedule_closure) {
+  call_data *calld = elem->call_data;
+  grpc_deadline_state_destroy(exec_ctx, elem);
+  grpc_slice_unref_internal(exec_ctx, calld->path);
+  if (calld->method_params != NULL) {
+    method_parameters_unref(calld->method_params);
+  }
+  GRPC_ERROR_UNREF(calld->cancel_error);
+  grpc_subchannel_call *call = GET_CALL(calld);
+  if (call != NULL && call != CANCELLED_CALL) {
+    grpc_subchannel_call_set_cleanup_closure(call, then_schedule_closure);
+    then_schedule_closure = NULL;
+    GRPC_SUBCHANNEL_CALL_UNREF(exec_ctx, call, "client_channel_destroy_call");
+  }
+  GPR_ASSERT(calld->creation_phase == GRPC_SUBCHANNEL_CALL_HOLDER_NOT_CREATING);
+  GPR_ASSERT(calld->waiting_ops_count == 0);
+  if (calld->connected_subchannel != NULL) {
+    GRPC_CONNECTED_SUBCHANNEL_UNREF(exec_ctx, calld->connected_subchannel,
+                                    "picked");
+  }
+  gpr_free(calld->waiting_ops);
+  grpc_closure_sched(exec_ctx, then_schedule_closure, GRPC_ERROR_NONE);
+}
+
+static void cc_set_pollset_or_pollset_set(grpc_exec_ctx *exec_ctx,
+                                          grpc_call_element *elem,
+                                          grpc_polling_entity *pollent) {
+  call_data *calld = elem->call_data;
+  calld->pollent = pollent;
+}
+
+/*************************************************************************
+ * EXPORTED SYMBOLS
+ */
+
+const grpc_channel_filter grpc_client_channel_filter = {
+    cc_start_transport_stream_op,
+    cc_start_transport_op,
+    sizeof(call_data),
+    cc_init_call_elem,
+    cc_set_pollset_or_pollset_set,
+    cc_destroy_call_elem,
+    sizeof(channel_data),
+    cc_init_channel_elem,
+    cc_destroy_channel_elem,
+    cc_get_peer,
+    cc_get_channel_info,
+    "client-channel",
+};
+
+static void try_to_connect_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                  grpc_error *error_ignored) {
+  channel_data *chand = arg;
+  if (chand->lb_policy != NULL) {
+    grpc_lb_policy_exit_idle_locked(exec_ctx, chand->lb_policy);
+  } else {
+    chand->exit_idle_when_lb_policy_arrives = true;
+    if (!chand->started_resolving && chand->resolver != NULL) {
+      GRPC_CHANNEL_STACK_REF(chand->owning_stack, "resolver");
+      chand->started_resolving = true;
+      grpc_resolver_next_locked(exec_ctx, chand->resolver,
+                                &chand->resolver_result,
+                                &chand->on_resolver_result_changed);
+    }
+  }
+  GRPC_CHANNEL_STACK_UNREF(exec_ctx, chand->owning_stack, "try_to_connect");
+}
+
+grpc_connectivity_state grpc_client_channel_check_connectivity_state(
+    grpc_exec_ctx *exec_ctx, grpc_channel_element *elem, int try_to_connect) {
+  channel_data *chand = elem->channel_data;
+  grpc_connectivity_state out =
+      grpc_connectivity_state_check(&chand->state_tracker);
+  if (out == GRPC_CHANNEL_IDLE && try_to_connect) {
+    GRPC_CHANNEL_STACK_REF(chand->owning_stack, "try_to_connect");
+    grpc_closure_sched(
+        exec_ctx,
+        grpc_closure_create(try_to_connect_locked, chand,
+                            grpc_combiner_scheduler(chand->combiner, false)),
+        GRPC_ERROR_NONE);
+  }
+  return out;
+}
+
+typedef struct {
+  channel_data *chand;
+  grpc_pollset *pollset;
+  grpc_closure *on_complete;
+  grpc_connectivity_state *state;
+  grpc_closure my_closure;
+} external_connectivity_watcher;
+
+static void on_external_watch_complete(grpc_exec_ctx *exec_ctx, void *arg,
+                                       grpc_error *error) {
+  external_connectivity_watcher *w = arg;
+  grpc_closure *follow_up = w->on_complete;
+  grpc_pollset_set_del_pollset(exec_ctx, w->chand->interested_parties,
+                               w->pollset);
+  GRPC_CHANNEL_STACK_UNREF(exec_ctx, w->chand->owning_stack,
+                           "external_connectivity_watcher");
+  gpr_free(w);
+  grpc_closure_run(exec_ctx, follow_up, GRPC_ERROR_REF(error));
+}
+
+static void watch_connectivity_state_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                            grpc_error *error_ignored) {
+  external_connectivity_watcher *w = arg;
+  grpc_closure_init(&w->my_closure, on_external_watch_complete, w,
+                    grpc_schedule_on_exec_ctx);
+  grpc_connectivity_state_notify_on_state_change(
+      exec_ctx, &w->chand->state_tracker, w->state, &w->my_closure);
+}
+
+void grpc_client_channel_watch_connectivity_state(
+    grpc_exec_ctx *exec_ctx, grpc_channel_element *elem, grpc_pollset *pollset,
+    grpc_connectivity_state *state, grpc_closure *on_complete) {
+  channel_data *chand = elem->channel_data;
+  external_connectivity_watcher *w = gpr_malloc(sizeof(*w));
+  w->chand = chand;
+  w->pollset = pollset;
+  w->on_complete = on_complete;
+  w->state = state;
+  grpc_pollset_set_add_pollset(exec_ctx, chand->interested_parties, pollset);
+  GRPC_CHANNEL_STACK_REF(w->chand->owning_stack,
+                         "external_connectivity_watcher");
+  grpc_closure_sched(
+      exec_ctx,
+      grpc_closure_init(&w->my_closure, watch_connectivity_state_locked, w,
+                        grpc_combiner_scheduler(chand->combiner, true)),
+      GRPC_ERROR_NONE);
+}
diff --git a/src/core/ext/filters/client_channel/client_channel.h b/src/core/ext/filters/client_channel/client_channel.h
new file mode 100644
index 0000000..39725ce
--- /dev/null
+++ b/src/core/ext/filters/client_channel/client_channel.h
@@ -0,0 +1,64 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_CLIENT_CHANNEL_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_CLIENT_CHANNEL_H
+
+#include "src/core/ext/filters/client_channel/client_channel_factory.h"
+#include "src/core/ext/filters/client_channel/resolver.h"
+#include "src/core/lib/channel/channel_stack.h"
+
+// Channel arg key for server URI string.
+#define GRPC_ARG_SERVER_URI "grpc.server_uri"
+
+/* A client channel is a channel that begins disconnected, and can connect
+   to some endpoint on demand. If that endpoint disconnects, it will be
+   connected to again later.
+
+   Calls on a disconnected client channel are queued until a connection is
+   established. */
+
+extern const grpc_channel_filter grpc_client_channel_filter;
+
+grpc_connectivity_state grpc_client_channel_check_connectivity_state(
+    grpc_exec_ctx *exec_ctx, grpc_channel_element *elem, int try_to_connect);
+
+void grpc_client_channel_watch_connectivity_state(
+    grpc_exec_ctx *exec_ctx, grpc_channel_element *elem, grpc_pollset *pollset,
+    grpc_connectivity_state *state, grpc_closure *on_complete);
+
+/* Debug helper: pull the subchannel call from a call stack element */
+grpc_subchannel_call *grpc_client_channel_get_subchannel_call(
+    grpc_call_element *elem);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_CLIENT_CHANNEL_H */
diff --git a/src/core/ext/filters/client_channel/client_channel_factory.c b/src/core/ext/filters/client_channel/client_channel_factory.c
new file mode 100644
index 0000000..44e83b5
--- /dev/null
+++ b/src/core/ext/filters/client_channel/client_channel_factory.c
@@ -0,0 +1,87 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/client_channel_factory.h"
+
+void grpc_client_channel_factory_ref(grpc_client_channel_factory* factory) {
+  factory->vtable->ref(factory);
+}
+
+void grpc_client_channel_factory_unref(grpc_exec_ctx* exec_ctx,
+                                       grpc_client_channel_factory* factory) {
+  factory->vtable->unref(exec_ctx, factory);
+}
+
+grpc_subchannel* grpc_client_channel_factory_create_subchannel(
+    grpc_exec_ctx* exec_ctx, grpc_client_channel_factory* factory,
+    const grpc_subchannel_args* args) {
+  return factory->vtable->create_subchannel(exec_ctx, factory, args);
+}
+
+grpc_channel* grpc_client_channel_factory_create_channel(
+    grpc_exec_ctx* exec_ctx, grpc_client_channel_factory* factory,
+    const char* target, grpc_client_channel_type type,
+    const grpc_channel_args* args) {
+  return factory->vtable->create_client_channel(exec_ctx, factory, target, type,
+                                                args);
+}
+
+static void* factory_arg_copy(void* factory) {
+  grpc_client_channel_factory_ref(factory);
+  return factory;
+}
+
+static void factory_arg_destroy(grpc_exec_ctx* exec_ctx, void* factory) {
+  // TODO(roth): Remove local exec_ctx when
+  // https://github.com/grpc/grpc/pull/8705 is merged.
+  grpc_client_channel_factory_unref(exec_ctx, factory);
+}
+
+static int factory_arg_cmp(void* factory1, void* factory2) {
+  if (factory1 < factory2) return -1;
+  if (factory1 > factory2) return 1;
+  return 0;
+}
+
+static const grpc_arg_pointer_vtable factory_arg_vtable = {
+    factory_arg_copy, factory_arg_destroy, factory_arg_cmp};
+
+grpc_arg grpc_client_channel_factory_create_channel_arg(
+    grpc_client_channel_factory* factory) {
+  grpc_arg arg;
+  arg.type = GRPC_ARG_POINTER;
+  arg.key = GRPC_ARG_CLIENT_CHANNEL_FACTORY;
+  arg.value.pointer.p = factory;
+  arg.value.pointer.vtable = &factory_arg_vtable;
+  return arg;
+}
diff --git a/src/core/ext/filters/client_channel/client_channel_factory.h b/src/core/ext/filters/client_channel/client_channel_factory.h
new file mode 100644
index 0000000..0bbb921
--- /dev/null
+++ b/src/core/ext/filters/client_channel/client_channel_factory.h
@@ -0,0 +1,92 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_CLIENT_CHANNEL_FACTORY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_CLIENT_CHANNEL_FACTORY_H
+
+#include <grpc/impl/codegen/grpc_types.h>
+
+#include "src/core/ext/filters/client_channel/subchannel.h"
+#include "src/core/lib/channel/channel_stack.h"
+
+// Channel arg key for client channel factory.
+#define GRPC_ARG_CLIENT_CHANNEL_FACTORY "grpc.client_channel_factory"
+
+typedef struct grpc_client_channel_factory grpc_client_channel_factory;
+typedef struct grpc_client_channel_factory_vtable
+    grpc_client_channel_factory_vtable;
+
+typedef enum {
+  GRPC_CLIENT_CHANNEL_TYPE_REGULAR, /** for the user-level regular calls */
+  GRPC_CLIENT_CHANNEL_TYPE_LOAD_BALANCING, /** for communication with a load
+                                              balancing service */
+} grpc_client_channel_type;
+
+/** Constructor for new configured channels.
+    Creating decorators around this type is encouraged to adapt behavior. */
+struct grpc_client_channel_factory {
+  const grpc_client_channel_factory_vtable *vtable;
+};
+
+struct grpc_client_channel_factory_vtable {
+  void (*ref)(grpc_client_channel_factory *factory);
+  void (*unref)(grpc_exec_ctx *exec_ctx, grpc_client_channel_factory *factory);
+  grpc_subchannel *(*create_subchannel)(grpc_exec_ctx *exec_ctx,
+                                        grpc_client_channel_factory *factory,
+                                        const grpc_subchannel_args *args);
+  grpc_channel *(*create_client_channel)(grpc_exec_ctx *exec_ctx,
+                                         grpc_client_channel_factory *factory,
+                                         const char *target,
+                                         grpc_client_channel_type type,
+                                         const grpc_channel_args *args);
+};
+
+void grpc_client_channel_factory_ref(grpc_client_channel_factory *factory);
+void grpc_client_channel_factory_unref(grpc_exec_ctx *exec_ctx,
+                                       grpc_client_channel_factory *factory);
+
+/** Create a new grpc_subchannel */
+grpc_subchannel *grpc_client_channel_factory_create_subchannel(
+    grpc_exec_ctx *exec_ctx, grpc_client_channel_factory *factory,
+    const grpc_subchannel_args *args);
+
+/** Create a new grpc_channel */
+grpc_channel *grpc_client_channel_factory_create_channel(
+    grpc_exec_ctx *exec_ctx, grpc_client_channel_factory *factory,
+    const char *target, grpc_client_channel_type type,
+    const grpc_channel_args *args);
+
+grpc_arg grpc_client_channel_factory_create_channel_arg(
+    grpc_client_channel_factory *factory);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_CLIENT_CHANNEL_FACTORY_H */
diff --git a/src/core/ext/filters/client_channel/client_channel_plugin.c b/src/core/ext/filters/client_channel/client_channel_plugin.c
new file mode 100644
index 0000000..944af01
--- /dev/null
+++ b/src/core/ext/filters/client_channel/client_channel_plugin.c
@@ -0,0 +1,104 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <limits.h>
+#include <stdbool.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/ext/filters/client_channel/http_connect_handshaker.h"
+#include "src/core/ext/filters/client_channel/http_proxy.h"
+#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/retry_throttle.h"
+#include "src/core/ext/filters/client_channel/subchannel_index.h"
+#include "src/core/lib/surface/channel_init.h"
+
+static bool append_filter(grpc_exec_ctx *exec_ctx,
+                          grpc_channel_stack_builder *builder, void *arg) {
+  return grpc_channel_stack_builder_append_filter(
+      builder, (const grpc_channel_filter *)arg, NULL, NULL);
+}
+
+static bool set_default_host_if_unset(grpc_exec_ctx *exec_ctx,
+                                      grpc_channel_stack_builder *builder,
+                                      void *unused) {
+  const grpc_channel_args *args =
+      grpc_channel_stack_builder_get_channel_arguments(builder);
+  for (size_t i = 0; i < args->num_args; i++) {
+    if (0 == strcmp(args->args[i].key, GRPC_ARG_DEFAULT_AUTHORITY) ||
+        0 == strcmp(args->args[i].key, GRPC_SSL_TARGET_NAME_OVERRIDE_ARG)) {
+      return true;
+    }
+  }
+  char *default_authority = grpc_get_default_authority(
+      exec_ctx, grpc_channel_stack_builder_get_target(builder));
+  if (default_authority != NULL) {
+    grpc_arg arg;
+    arg.type = GRPC_ARG_STRING;
+    arg.key = GRPC_ARG_DEFAULT_AUTHORITY;
+    arg.value.string = default_authority;
+    grpc_channel_args *new_args = grpc_channel_args_copy_and_add(args, &arg, 1);
+    grpc_channel_stack_builder_set_channel_arguments(exec_ctx, builder,
+                                                     new_args);
+    gpr_free(default_authority);
+    grpc_channel_args_destroy(exec_ctx, new_args);
+  }
+  return true;
+}
+
+void grpc_client_channel_init(void) {
+  grpc_lb_policy_registry_init();
+  grpc_resolver_registry_init();
+  grpc_retry_throttle_map_init();
+  grpc_proxy_mapper_registry_init();
+  grpc_register_http_proxy_mapper();
+  grpc_subchannel_index_init();
+  grpc_channel_init_register_stage(GRPC_CLIENT_CHANNEL, INT_MIN,
+                                   set_default_host_if_unset, NULL);
+  grpc_channel_init_register_stage(GRPC_CLIENT_CHANNEL, INT_MAX, append_filter,
+                                   (void *)&grpc_client_channel_filter);
+  grpc_http_connect_register_handshaker_factory();
+}
+
+void grpc_client_channel_shutdown(void) {
+  grpc_subchannel_index_shutdown();
+  grpc_channel_init_shutdown();
+  grpc_proxy_mapper_registry_shutdown();
+  grpc_retry_throttle_map_shutdown();
+  grpc_resolver_registry_shutdown();
+  grpc_lb_policy_registry_shutdown();
+}
diff --git a/src/core/ext/filters/client_channel/connector.c b/src/core/ext/filters/client_channel/connector.c
new file mode 100644
index 0000000..51c1d7e
--- /dev/null
+++ b/src/core/ext/filters/client_channel/connector.c
@@ -0,0 +1,55 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/connector.h"
+
+grpc_connector* grpc_connector_ref(grpc_connector* connector) {
+  connector->vtable->ref(connector);
+  return connector;
+}
+
+void grpc_connector_unref(grpc_exec_ctx* exec_ctx, grpc_connector* connector) {
+  connector->vtable->unref(exec_ctx, connector);
+}
+
+void grpc_connector_connect(grpc_exec_ctx* exec_ctx, grpc_connector* connector,
+                            const grpc_connect_in_args* in_args,
+                            grpc_connect_out_args* out_args,
+                            grpc_closure* notify) {
+  connector->vtable->connect(exec_ctx, connector, in_args, out_args, notify);
+}
+
+void grpc_connector_shutdown(grpc_exec_ctx* exec_ctx, grpc_connector* connector,
+                             grpc_error* why) {
+  connector->vtable->shutdown(exec_ctx, connector, why);
+}
diff --git a/src/core/ext/filters/client_channel/connector.h b/src/core/ext/filters/client_channel/connector.h
new file mode 100644
index 0000000..94b5fb5
--- /dev/null
+++ b/src/core/ext/filters/client_channel/connector.h
@@ -0,0 +1,88 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_CONNECTOR_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_CONNECTOR_H
+
+#include "src/core/lib/channel/channel_stack.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+#include "src/core/lib/transport/transport.h"
+
+typedef struct grpc_connector grpc_connector;
+typedef struct grpc_connector_vtable grpc_connector_vtable;
+
+struct grpc_connector {
+  const grpc_connector_vtable *vtable;
+};
+
+typedef struct {
+  /** set of pollsets interested in this connection */
+  grpc_pollset_set *interested_parties;
+  /** deadline for connection */
+  gpr_timespec deadline;
+  /** channel arguments (to be passed to transport) */
+  const grpc_channel_args *channel_args;
+} grpc_connect_in_args;
+
+typedef struct {
+  /** the connected transport */
+  grpc_transport *transport;
+
+  /** channel arguments (to be passed to the filters) */
+  grpc_channel_args *channel_args;
+} grpc_connect_out_args;
+
+struct grpc_connector_vtable {
+  void (*ref)(grpc_connector *connector);
+  void (*unref)(grpc_exec_ctx *exec_ctx, grpc_connector *connector);
+  /** Implementation of grpc_connector_shutdown */
+  void (*shutdown)(grpc_exec_ctx *exec_ctx, grpc_connector *connector,
+                   grpc_error *why);
+  /** Implementation of grpc_connector_connect */
+  void (*connect)(grpc_exec_ctx *exec_ctx, grpc_connector *connector,
+                  const grpc_connect_in_args *in_args,
+                  grpc_connect_out_args *out_args, grpc_closure *notify);
+};
+
+grpc_connector *grpc_connector_ref(grpc_connector *connector);
+void grpc_connector_unref(grpc_exec_ctx *exec_ctx, grpc_connector *connector);
+/** Connect using the connector: max one outstanding call at a time */
+void grpc_connector_connect(grpc_exec_ctx *exec_ctx, grpc_connector *connector,
+                            const grpc_connect_in_args *in_args,
+                            grpc_connect_out_args *out_args,
+                            grpc_closure *notify);
+/** Cancel any pending connection */
+void grpc_connector_shutdown(grpc_exec_ctx *exec_ctx, grpc_connector *connector,
+                             grpc_error *why);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_CONNECTOR_H */
diff --git a/src/core/ext/filters/client_channel/http_connect_handshaker.c b/src/core/ext/filters/client_channel/http_connect_handshaker.c
new file mode 100644
index 0000000..c09a863
--- /dev/null
+++ b/src/core/ext/filters/client_channel/http_connect_handshaker.c
@@ -0,0 +1,389 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/http_connect_handshaker.h"
+
+#include <string.h>
+
+#include <grpc/slice_buffer.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/ext/filters/client_channel/resolver_registry.h"
+#include "src/core/ext/filters/client_channel/uri_parser.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/channel/handshaker_registry.h"
+#include "src/core/lib/http/format_request.h"
+#include "src/core/lib/http/parser.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/support/env.h"
+#include "src/core/lib/support/string.h"
+
+typedef struct http_connect_handshaker {
+  // Base class.  Must be first.
+  grpc_handshaker base;
+
+  gpr_refcount refcount;
+  gpr_mu mu;
+
+  bool shutdown;
+  // Endpoint and read buffer to destroy after a shutdown.
+  grpc_endpoint* endpoint_to_destroy;
+  grpc_slice_buffer* read_buffer_to_destroy;
+
+  // State saved while performing the handshake.
+  grpc_handshaker_args* args;
+  grpc_closure* on_handshake_done;
+
+  // Objects for processing the HTTP CONNECT request and response.
+  grpc_slice_buffer write_buffer;
+  grpc_closure request_done_closure;
+  grpc_closure response_read_closure;
+  grpc_http_parser http_parser;
+  grpc_http_response http_response;
+} http_connect_handshaker;
+
+// Unref and clean up handshaker.
+static void http_connect_handshaker_unref(grpc_exec_ctx* exec_ctx,
+                                          http_connect_handshaker* handshaker) {
+  if (gpr_unref(&handshaker->refcount)) {
+    gpr_mu_destroy(&handshaker->mu);
+    if (handshaker->endpoint_to_destroy != NULL) {
+      grpc_endpoint_destroy(exec_ctx, handshaker->endpoint_to_destroy);
+    }
+    if (handshaker->read_buffer_to_destroy != NULL) {
+      grpc_slice_buffer_destroy_internal(exec_ctx,
+                                         handshaker->read_buffer_to_destroy);
+      gpr_free(handshaker->read_buffer_to_destroy);
+    }
+    grpc_slice_buffer_destroy_internal(exec_ctx, &handshaker->write_buffer);
+    grpc_http_parser_destroy(&handshaker->http_parser);
+    grpc_http_response_destroy(&handshaker->http_response);
+    gpr_free(handshaker);
+  }
+}
+
+// Set args fields to NULL, saving the endpoint and read buffer for
+// later destruction.
+static void cleanup_args_for_failure_locked(
+    grpc_exec_ctx* exec_ctx, http_connect_handshaker* handshaker) {
+  handshaker->endpoint_to_destroy = handshaker->args->endpoint;
+  handshaker->args->endpoint = NULL;
+  handshaker->read_buffer_to_destroy = handshaker->args->read_buffer;
+  handshaker->args->read_buffer = NULL;
+  grpc_channel_args_destroy(exec_ctx, handshaker->args->args);
+  handshaker->args->args = NULL;
+}
+
+// If the handshake failed or we're shutting down, clean up and invoke the
+// callback with the error.
+static void handshake_failed_locked(grpc_exec_ctx* exec_ctx,
+                                    http_connect_handshaker* handshaker,
+                                    grpc_error* error) {
+  if (error == GRPC_ERROR_NONE) {
+    // If we were shut down after an endpoint operation succeeded but
+    // before the endpoint callback was invoked, we need to generate our
+    // own error.
+    error = GRPC_ERROR_CREATE_FROM_STATIC_STRING("Handshaker shutdown");
+  }
+  if (!handshaker->shutdown) {
+    // TODO(ctiller): It is currently necessary to shutdown endpoints
+    // before destroying them, even if we know that there are no
+    // pending read/write callbacks.  This should be fixed, at which
+    // point this can be removed.
+    grpc_endpoint_shutdown(exec_ctx, handshaker->args->endpoint,
+                           GRPC_ERROR_REF(error));
+    // Not shutting down, so the handshake failed.  Clean up before
+    // invoking the callback.
+    cleanup_args_for_failure_locked(exec_ctx, handshaker);
+    // Set shutdown to true so that subsequent calls to
+    // http_connect_handshaker_shutdown() do nothing.
+    handshaker->shutdown = true;
+  }
+  // Invoke callback.
+  grpc_closure_sched(exec_ctx, handshaker->on_handshake_done, error);
+}
+
+// Callback invoked when finished writing HTTP CONNECT request.
+static void on_write_done(grpc_exec_ctx* exec_ctx, void* arg,
+                          grpc_error* error) {
+  http_connect_handshaker* handshaker = arg;
+  gpr_mu_lock(&handshaker->mu);
+  if (error != GRPC_ERROR_NONE || handshaker->shutdown) {
+    // If the write failed or we're shutting down, clean up and invoke the
+    // callback with the error.
+    handshake_failed_locked(exec_ctx, handshaker, GRPC_ERROR_REF(error));
+    gpr_mu_unlock(&handshaker->mu);
+    http_connect_handshaker_unref(exec_ctx, handshaker);
+  } else {
+    // Otherwise, read the response.
+    // The read callback inherits our ref to the handshaker.
+    grpc_endpoint_read(exec_ctx, handshaker->args->endpoint,
+                       handshaker->args->read_buffer,
+                       &handshaker->response_read_closure);
+    gpr_mu_unlock(&handshaker->mu);
+  }
+}
+
+// Callback invoked for reading HTTP CONNECT response.
+static void on_read_done(grpc_exec_ctx* exec_ctx, void* arg,
+                         grpc_error* error) {
+  http_connect_handshaker* handshaker = arg;
+  gpr_mu_lock(&handshaker->mu);
+  if (error != GRPC_ERROR_NONE || handshaker->shutdown) {
+    // If the read failed or we're shutting down, clean up and invoke the
+    // callback with the error.
+    handshake_failed_locked(exec_ctx, handshaker, GRPC_ERROR_REF(error));
+    goto done;
+  }
+  // Add buffer to parser.
+  for (size_t i = 0; i < handshaker->args->read_buffer->count; ++i) {
+    if (GRPC_SLICE_LENGTH(handshaker->args->read_buffer->slices[i]) > 0) {
+      size_t body_start_offset = 0;
+      error = grpc_http_parser_parse(&handshaker->http_parser,
+                                     handshaker->args->read_buffer->slices[i],
+                                     &body_start_offset);
+      if (error != GRPC_ERROR_NONE) {
+        handshake_failed_locked(exec_ctx, handshaker, error);
+        goto done;
+      }
+      if (handshaker->http_parser.state == GRPC_HTTP_BODY) {
+        // Remove the data we've already read from the read buffer,
+        // leaving only the leftover bytes (if any).
+        grpc_slice_buffer tmp_buffer;
+        grpc_slice_buffer_init(&tmp_buffer);
+        if (body_start_offset <
+            GRPC_SLICE_LENGTH(handshaker->args->read_buffer->slices[i])) {
+          grpc_slice_buffer_add(
+              &tmp_buffer,
+              grpc_slice_split_tail(&handshaker->args->read_buffer->slices[i],
+                                    body_start_offset));
+        }
+        grpc_slice_buffer_addn(&tmp_buffer,
+                               &handshaker->args->read_buffer->slices[i + 1],
+                               handshaker->args->read_buffer->count - i - 1);
+        grpc_slice_buffer_swap(handshaker->args->read_buffer, &tmp_buffer);
+        grpc_slice_buffer_destroy_internal(exec_ctx, &tmp_buffer);
+        break;
+      }
+    }
+  }
+  // If we're not done reading the response, read more data.
+  // TODO(roth): In practice, I suspect that the response to a CONNECT
+  // request will never include a body, in which case this check is
+  // sufficient.  However, the language of RFC-2817 doesn't explicitly
+  // forbid the response from including a body.  If there is a body,
+  // it's possible that we might have parsed part but not all of the
+  // body, in which case this check will cause us to fail to parse the
+  // remainder of the body.  If that ever becomes an issue, we may
+  // need to fix the HTTP parser to understand when the body is
+  // complete (e.g., handling chunked transfer encoding or looking
+  // at the Content-Length: header).
+  if (handshaker->http_parser.state != GRPC_HTTP_BODY) {
+    grpc_slice_buffer_reset_and_unref_internal(exec_ctx,
+                                               handshaker->args->read_buffer);
+    grpc_endpoint_read(exec_ctx, handshaker->args->endpoint,
+                       handshaker->args->read_buffer,
+                       &handshaker->response_read_closure);
+    gpr_mu_unlock(&handshaker->mu);
+    return;
+  }
+  // Make sure we got a 2xx response.
+  if (handshaker->http_response.status < 200 ||
+      handshaker->http_response.status >= 300) {
+    char* msg;
+    gpr_asprintf(&msg, "HTTP proxy returned response code %d",
+                 handshaker->http_response.status);
+    error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(msg);
+    gpr_free(msg);
+    handshake_failed_locked(exec_ctx, handshaker, error);
+    goto done;
+  }
+  // Success.  Invoke handshake-done callback.
+  grpc_closure_sched(exec_ctx, handshaker->on_handshake_done, error);
+done:
+  // Set shutdown to true so that subsequent calls to
+  // http_connect_handshaker_shutdown() do nothing.
+  handshaker->shutdown = true;
+  gpr_mu_unlock(&handshaker->mu);
+  http_connect_handshaker_unref(exec_ctx, handshaker);
+}
+
+//
+// Public handshaker methods
+//
+
+static void http_connect_handshaker_destroy(grpc_exec_ctx* exec_ctx,
+                                            grpc_handshaker* handshaker_in) {
+  http_connect_handshaker* handshaker = (http_connect_handshaker*)handshaker_in;
+  http_connect_handshaker_unref(exec_ctx, handshaker);
+}
+
+static void http_connect_handshaker_shutdown(grpc_exec_ctx* exec_ctx,
+                                             grpc_handshaker* handshaker_in,
+                                             grpc_error* why) {
+  http_connect_handshaker* handshaker = (http_connect_handshaker*)handshaker_in;
+  gpr_mu_lock(&handshaker->mu);
+  if (!handshaker->shutdown) {
+    handshaker->shutdown = true;
+    grpc_endpoint_shutdown(exec_ctx, handshaker->args->endpoint,
+                           GRPC_ERROR_REF(why));
+    cleanup_args_for_failure_locked(exec_ctx, handshaker);
+  }
+  gpr_mu_unlock(&handshaker->mu);
+  GRPC_ERROR_UNREF(why);
+}
+
+static void http_connect_handshaker_do_handshake(
+    grpc_exec_ctx* exec_ctx, grpc_handshaker* handshaker_in,
+    grpc_tcp_server_acceptor* acceptor, grpc_closure* on_handshake_done,
+    grpc_handshaker_args* args) {
+  http_connect_handshaker* handshaker = (http_connect_handshaker*)handshaker_in;
+  // Check for HTTP CONNECT channel arg.
+  // If not found, invoke on_handshake_done without doing anything.
+  const grpc_arg* arg =
+      grpc_channel_args_find(args->args, GRPC_ARG_HTTP_CONNECT_SERVER);
+  if (arg == NULL) {
+    // Set shutdown to true so that subsequent calls to
+    // http_connect_handshaker_shutdown() do nothing.
+    gpr_mu_lock(&handshaker->mu);
+    handshaker->shutdown = true;
+    gpr_mu_unlock(&handshaker->mu);
+    grpc_closure_sched(exec_ctx, on_handshake_done, GRPC_ERROR_NONE);
+    return;
+  }
+  GPR_ASSERT(arg->type == GRPC_ARG_STRING);
+  char* server_name = arg->value.string;
+  // Get headers from channel args.
+  arg = grpc_channel_args_find(args->args, GRPC_ARG_HTTP_CONNECT_HEADERS);
+  grpc_http_header* headers = NULL;
+  size_t num_headers = 0;
+  char** header_strings = NULL;
+  size_t num_header_strings = 0;
+  if (arg != NULL) {
+    GPR_ASSERT(arg->type == GRPC_ARG_STRING);
+    gpr_string_split(arg->value.string, "\n", &header_strings,
+                     &num_header_strings);
+    headers = gpr_malloc(sizeof(grpc_http_header) * num_header_strings);
+    for (size_t i = 0; i < num_header_strings; ++i) {
+      char* sep = strchr(header_strings[i], ':');
+      if (sep == NULL) {
+        gpr_log(GPR_ERROR, "skipping unparseable HTTP CONNECT header: %s",
+                header_strings[i]);
+        continue;
+      }
+      *sep = '\0';
+      headers[num_headers].key = header_strings[i];
+      headers[num_headers].value = sep + 1;
+      ++num_headers;
+    }
+  }
+  // Save state in the handshaker object.
+  gpr_mu_lock(&handshaker->mu);
+  handshaker->args = args;
+  handshaker->on_handshake_done = on_handshake_done;
+  // Log connection via proxy.
+  char* proxy_name = grpc_endpoint_get_peer(args->endpoint);
+  gpr_log(GPR_INFO, "Connecting to server %s via HTTP proxy %s", server_name,
+          proxy_name);
+  gpr_free(proxy_name);
+  // Construct HTTP CONNECT request.
+  grpc_httpcli_request request;
+  memset(&request, 0, sizeof(request));
+  request.host = server_name;
+  request.http.method = "CONNECT";
+  request.http.path = server_name;
+  request.http.hdrs = headers;
+  request.http.hdr_count = num_headers;
+  request.handshaker = &grpc_httpcli_plaintext;
+  grpc_slice request_slice = grpc_httpcli_format_connect_request(&request);
+  grpc_slice_buffer_add(&handshaker->write_buffer, request_slice);
+  // Clean up.
+  gpr_free(headers);
+  for (size_t i = 0; i < num_header_strings; ++i) {
+    gpr_free(header_strings[i]);
+  }
+  gpr_free(header_strings);
+  // Take a new ref to be held by the write callback.
+  gpr_ref(&handshaker->refcount);
+  grpc_endpoint_write(exec_ctx, args->endpoint, &handshaker->write_buffer,
+                      &handshaker->request_done_closure);
+  gpr_mu_unlock(&handshaker->mu);
+}
+
+static const grpc_handshaker_vtable http_connect_handshaker_vtable = {
+    http_connect_handshaker_destroy, http_connect_handshaker_shutdown,
+    http_connect_handshaker_do_handshake};
+
+static grpc_handshaker* grpc_http_connect_handshaker_create() {
+  http_connect_handshaker* handshaker = gpr_malloc(sizeof(*handshaker));
+  memset(handshaker, 0, sizeof(*handshaker));
+  grpc_handshaker_init(&http_connect_handshaker_vtable, &handshaker->base);
+  gpr_mu_init(&handshaker->mu);
+  gpr_ref_init(&handshaker->refcount, 1);
+  grpc_slice_buffer_init(&handshaker->write_buffer);
+  grpc_closure_init(&handshaker->request_done_closure, on_write_done,
+                    handshaker, grpc_schedule_on_exec_ctx);
+  grpc_closure_init(&handshaker->response_read_closure, on_read_done,
+                    handshaker, grpc_schedule_on_exec_ctx);
+  grpc_http_parser_init(&handshaker->http_parser, GRPC_HTTP_RESPONSE,
+                        &handshaker->http_response);
+  return &handshaker->base;
+}
+
+//
+// handshaker factory
+//
+
+static void handshaker_factory_add_handshakers(
+    grpc_exec_ctx* exec_ctx, grpc_handshaker_factory* factory,
+    const grpc_channel_args* args, grpc_handshake_manager* handshake_mgr) {
+  grpc_handshake_manager_add(handshake_mgr,
+                             grpc_http_connect_handshaker_create());
+}
+
+static void handshaker_factory_destroy(grpc_exec_ctx* exec_ctx,
+                                       grpc_handshaker_factory* factory) {}
+
+static const grpc_handshaker_factory_vtable handshaker_factory_vtable = {
+    handshaker_factory_add_handshakers, handshaker_factory_destroy};
+
+static grpc_handshaker_factory handshaker_factory = {
+    &handshaker_factory_vtable};
+
+void grpc_http_connect_register_handshaker_factory() {
+  grpc_handshaker_factory_register(true /* at_start */, HANDSHAKER_CLIENT,
+                                   &handshaker_factory);
+}
diff --git a/src/core/ext/filters/client_channel/http_connect_handshaker.h b/src/core/ext/filters/client_channel/http_connect_handshaker.h
new file mode 100644
index 0000000..3059d55
--- /dev/null
+++ b/src/core/ext/filters/client_channel/http_connect_handshaker.h
@@ -0,0 +1,49 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_HTTP_CONNECT_HANDSHAKER_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_HTTP_CONNECT_HANDSHAKER_H
+
+/// Channel arg indicating the server in HTTP CONNECT request (string).
+/// The presence of this arg triggers the use of HTTP CONNECT.
+#define GRPC_ARG_HTTP_CONNECT_SERVER "grpc.http_connect_server"
+
+/// Channel arg indicating HTTP CONNECT headers (string).
+/// Multiple headers are separated by newlines.  Key/value pairs are
+/// seperated by colons.
+#define GRPC_ARG_HTTP_CONNECT_HEADERS "grpc.http_connect_headers"
+
+/// Registers handshaker factory.
+void grpc_http_connect_register_handshaker_factory();
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_HTTP_CONNECT_HANDSHAKER_H */
diff --git a/src/core/ext/filters/client_channel/http_proxy.c b/src/core/ext/filters/client_channel/http_proxy.c
new file mode 100644
index 0000000..f8a2d06
--- /dev/null
+++ b/src/core/ext/filters/client_channel/http_proxy.c
@@ -0,0 +1,125 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/http_proxy.h"
+
+#include <stdbool.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/http_connect_handshaker.h"
+#include "src/core/ext/filters/client_channel/proxy_mapper_registry.h"
+#include "src/core/ext/filters/client_channel/uri_parser.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/support/env.h"
+
+static char* grpc_get_http_proxy_server(grpc_exec_ctx* exec_ctx) {
+  char* uri_str = gpr_getenv("http_proxy");
+  if (uri_str == NULL) return NULL;
+  grpc_uri* uri =
+      grpc_uri_parse(exec_ctx, uri_str, false /* suppress_errors */);
+  char* proxy_name = NULL;
+  if (uri == NULL || uri->authority == NULL) {
+    gpr_log(GPR_ERROR, "cannot parse value of 'http_proxy' env var");
+    goto done;
+  }
+  if (strcmp(uri->scheme, "http") != 0) {
+    gpr_log(GPR_ERROR, "'%s' scheme not supported in proxy URI", uri->scheme);
+    goto done;
+  }
+  if (strchr(uri->authority, '@') != NULL) {
+    gpr_log(GPR_ERROR, "userinfo not supported in proxy URI");
+    goto done;
+  }
+  proxy_name = gpr_strdup(uri->authority);
+done:
+  gpr_free(uri_str);
+  grpc_uri_destroy(uri);
+  return proxy_name;
+}
+
+static bool proxy_mapper_map_name(grpc_exec_ctx* exec_ctx,
+                                  grpc_proxy_mapper* mapper,
+                                  const char* server_uri,
+                                  const grpc_channel_args* args,
+                                  char** name_to_resolve,
+                                  grpc_channel_args** new_args) {
+  *name_to_resolve = grpc_get_http_proxy_server(exec_ctx);
+  if (*name_to_resolve == NULL) return false;
+  grpc_uri* uri =
+      grpc_uri_parse(exec_ctx, server_uri, false /* suppress_errors */);
+  if (uri == NULL || uri->path[0] == '\0') {
+    gpr_log(GPR_ERROR,
+            "'http_proxy' environment variable set, but cannot "
+            "parse server URI '%s' -- not using proxy",
+            server_uri);
+    if (uri != NULL) grpc_uri_destroy(uri);
+    return false;
+  }
+  if (strcmp(uri->scheme, "unix") == 0) {
+    gpr_log(GPR_INFO, "not using proxy for Unix domain socket '%s'",
+            server_uri);
+    grpc_uri_destroy(uri);
+    return false;
+  }
+  grpc_arg new_arg;
+  new_arg.key = GRPC_ARG_HTTP_CONNECT_SERVER;
+  new_arg.type = GRPC_ARG_STRING;
+  new_arg.value.string = uri->path[0] == '/' ? uri->path + 1 : uri->path;
+  *new_args = grpc_channel_args_copy_and_add(args, &new_arg, 1);
+  grpc_uri_destroy(uri);
+  return true;
+}
+
+static bool proxy_mapper_map_address(grpc_exec_ctx* exec_ctx,
+                                     grpc_proxy_mapper* mapper,
+                                     const grpc_resolved_address* address,
+                                     const grpc_channel_args* args,
+                                     grpc_resolved_address** new_address,
+                                     grpc_channel_args** new_args) {
+  return false;
+}
+
+static void proxy_mapper_destroy(grpc_proxy_mapper* mapper) {}
+
+static const grpc_proxy_mapper_vtable proxy_mapper_vtable = {
+    proxy_mapper_map_name, proxy_mapper_map_address, proxy_mapper_destroy};
+
+static grpc_proxy_mapper proxy_mapper = {&proxy_mapper_vtable};
+
+void grpc_register_http_proxy_mapper() {
+  grpc_proxy_mapper_register(true /* at_start */, &proxy_mapper);
+}
diff --git a/src/core/ext/filters/client_channel/http_proxy.h b/src/core/ext/filters/client_channel/http_proxy.h
new file mode 100644
index 0000000..c8882b1
--- /dev/null
+++ b/src/core/ext/filters/client_channel/http_proxy.h
@@ -0,0 +1,39 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_HTTP_PROXY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_HTTP_PROXY_H
+
+void grpc_register_http_proxy_mapper();
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_HTTP_PROXY_H */
diff --git a/src/core/ext/filters/client_channel/lb_policy.c b/src/core/ext/filters/client_channel/lb_policy.c
new file mode 100644
index 0000000..2d31499
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy.c
@@ -0,0 +1,167 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/lb_policy.h"
+#include "src/core/lib/iomgr/combiner.h"
+
+#define WEAK_REF_BITS 16
+
+void grpc_lb_policy_init(grpc_lb_policy *policy,
+                         const grpc_lb_policy_vtable *vtable,
+                         grpc_combiner *combiner) {
+  policy->vtable = vtable;
+  gpr_atm_no_barrier_store(&policy->ref_pair, 1 << WEAK_REF_BITS);
+  policy->interested_parties = grpc_pollset_set_create();
+  policy->combiner = GRPC_COMBINER_REF(combiner, "lb_policy");
+}
+
+#ifdef GRPC_LB_POLICY_REFCOUNT_DEBUG
+#define REF_FUNC_EXTRA_ARGS , const char *file, int line, const char *reason
+#define REF_MUTATE_EXTRA_ARGS REF_FUNC_EXTRA_ARGS, const char *purpose
+#define REF_FUNC_PASS_ARGS(new_reason) , file, line, new_reason
+#define REF_MUTATE_PASS_ARGS(purpose) , file, line, reason, purpose
+#else
+#define REF_FUNC_EXTRA_ARGS
+#define REF_MUTATE_EXTRA_ARGS
+#define REF_FUNC_PASS_ARGS(new_reason)
+#define REF_MUTATE_PASS_ARGS(x)
+#endif
+
+static gpr_atm ref_mutate(grpc_lb_policy *c, gpr_atm delta,
+                          int barrier REF_MUTATE_EXTRA_ARGS) {
+  gpr_atm old_val = barrier ? gpr_atm_full_fetch_add(&c->ref_pair, delta)
+                            : gpr_atm_no_barrier_fetch_add(&c->ref_pair, delta);
+#ifdef GRPC_LB_POLICY_REFCOUNT_DEBUG
+  gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG,
+          "LB_POLICY: 0x%" PRIxPTR " %12s 0x%" PRIxPTR " -> 0x%" PRIxPTR
+          " [%s]",
+          (intptr_t)c, purpose, old_val, old_val + delta, reason);
+#endif
+  return old_val;
+}
+
+void grpc_lb_policy_ref(grpc_lb_policy *policy REF_FUNC_EXTRA_ARGS) {
+  ref_mutate(policy, 1 << WEAK_REF_BITS, 0 REF_MUTATE_PASS_ARGS("STRONG_REF"));
+}
+
+static void shutdown_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                            grpc_error *error) {
+  grpc_lb_policy *policy = arg;
+  policy->vtable->shutdown_locked(exec_ctx, policy);
+  GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, policy, "strong-unref");
+}
+
+void grpc_lb_policy_unref(grpc_exec_ctx *exec_ctx,
+                          grpc_lb_policy *policy REF_FUNC_EXTRA_ARGS) {
+  gpr_atm old_val =
+      ref_mutate(policy, (gpr_atm)1 - (gpr_atm)(1 << WEAK_REF_BITS),
+                 1 REF_MUTATE_PASS_ARGS("STRONG_UNREF"));
+  gpr_atm mask = ~(gpr_atm)((1 << WEAK_REF_BITS) - 1);
+  gpr_atm check = 1 << WEAK_REF_BITS;
+  if ((old_val & mask) == check) {
+    grpc_closure_sched(
+        exec_ctx,
+        grpc_closure_create(shutdown_locked, policy,
+                            grpc_combiner_scheduler(policy->combiner, false)),
+        GRPC_ERROR_NONE);
+  } else {
+    grpc_lb_policy_weak_unref(exec_ctx,
+                              policy REF_FUNC_PASS_ARGS("strong-unref"));
+  }
+}
+
+void grpc_lb_policy_weak_ref(grpc_lb_policy *policy REF_FUNC_EXTRA_ARGS) {
+  ref_mutate(policy, 1, 0 REF_MUTATE_PASS_ARGS("WEAK_REF"));
+}
+
+void grpc_lb_policy_weak_unref(grpc_exec_ctx *exec_ctx,
+                               grpc_lb_policy *policy REF_FUNC_EXTRA_ARGS) {
+  gpr_atm old_val =
+      ref_mutate(policy, -(gpr_atm)1, 1 REF_MUTATE_PASS_ARGS("WEAK_UNREF"));
+  if (old_val == 1) {
+    grpc_pollset_set_destroy(exec_ctx, policy->interested_parties);
+    grpc_combiner *combiner = policy->combiner;
+    policy->vtable->destroy(exec_ctx, policy);
+    GRPC_COMBINER_UNREF(exec_ctx, combiner, "lb_policy");
+  }
+}
+
+int grpc_lb_policy_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                               const grpc_lb_policy_pick_args *pick_args,
+                               grpc_connected_subchannel **target,
+                               void **user_data, grpc_closure *on_complete) {
+  return policy->vtable->pick_locked(exec_ctx, policy, pick_args, target,
+                                     user_data, on_complete);
+}
+
+void grpc_lb_policy_cancel_pick_locked(grpc_exec_ctx *exec_ctx,
+                                       grpc_lb_policy *policy,
+                                       grpc_connected_subchannel **target,
+                                       grpc_error *error) {
+  policy->vtable->cancel_pick_locked(exec_ctx, policy, target, error);
+}
+
+void grpc_lb_policy_cancel_picks_locked(grpc_exec_ctx *exec_ctx,
+                                        grpc_lb_policy *policy,
+                                        uint32_t initial_metadata_flags_mask,
+                                        uint32_t initial_metadata_flags_eq,
+                                        grpc_error *error) {
+  policy->vtable->cancel_picks_locked(exec_ctx, policy,
+                                      initial_metadata_flags_mask,
+                                      initial_metadata_flags_eq, error);
+}
+
+void grpc_lb_policy_exit_idle_locked(grpc_exec_ctx *exec_ctx,
+                                     grpc_lb_policy *policy) {
+  policy->vtable->exit_idle_locked(exec_ctx, policy);
+}
+
+void grpc_lb_policy_ping_one_locked(grpc_exec_ctx *exec_ctx,
+                                    grpc_lb_policy *policy,
+                                    grpc_closure *closure) {
+  policy->vtable->ping_one_locked(exec_ctx, policy, closure);
+}
+
+void grpc_lb_policy_notify_on_state_change_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+    grpc_connectivity_state *state, grpc_closure *closure) {
+  policy->vtable->notify_on_state_change_locked(exec_ctx, policy, state,
+                                                closure);
+}
+
+grpc_connectivity_state grpc_lb_policy_check_connectivity_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+    grpc_error **connectivity_error) {
+  return policy->vtable->check_connectivity_locked(exec_ctx, policy,
+                                                   connectivity_error);
+}
diff --git a/src/core/ext/filters/client_channel/lb_policy.h b/src/core/ext/filters/client_channel/lb_policy.h
new file mode 100644
index 0000000..3f5fc94
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy.h
@@ -0,0 +1,209 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_H
+
+#include "src/core/ext/filters/client_channel/subchannel.h"
+#include "src/core/lib/iomgr/polling_entity.h"
+#include "src/core/lib/transport/connectivity_state.h"
+
+/** A load balancing policy: specified by a vtable and a struct (which
+    is expected to be extended to contain some parameters) */
+typedef struct grpc_lb_policy grpc_lb_policy;
+typedef struct grpc_lb_policy_vtable grpc_lb_policy_vtable;
+
+typedef void (*grpc_lb_completion)(void *cb_arg, grpc_subchannel *subchannel,
+                                   grpc_status_code status, const char *errmsg);
+
+struct grpc_lb_policy {
+  const grpc_lb_policy_vtable *vtable;
+  gpr_atm ref_pair;
+  /* owned pointer to interested parties in load balancing decisions */
+  grpc_pollset_set *interested_parties;
+  /* combiner under which lb_policy actions take place */
+  grpc_combiner *combiner;
+};
+
+/** Extra arguments for an LB pick */
+typedef struct grpc_lb_policy_pick_args {
+  /** Initial metadata associated with the picking call. */
+  grpc_metadata_batch *initial_metadata;
+  /** Bitmask used for selective cancelling. See \a
+   * grpc_lb_policy_cancel_picks() and \a GRPC_INITIAL_METADATA_* in
+   * grpc_types.h */
+  uint32_t initial_metadata_flags;
+  /** Storage for LB token in \a initial_metadata, or NULL if not used */
+  grpc_linked_mdelem *lb_token_mdelem_storage;
+  /** Deadline for the call to the LB server */
+  gpr_timespec deadline;
+} grpc_lb_policy_pick_args;
+
+struct grpc_lb_policy_vtable {
+  void (*destroy)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy);
+  void (*shutdown_locked)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy);
+
+  /** \see grpc_lb_policy_pick */
+  int (*pick_locked)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                     const grpc_lb_policy_pick_args *pick_args,
+                     grpc_connected_subchannel **target, void **user_data,
+                     grpc_closure *on_complete);
+
+  /** \see grpc_lb_policy_cancel_pick */
+  void (*cancel_pick_locked)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                             grpc_connected_subchannel **target,
+                             grpc_error *error);
+
+  /** \see grpc_lb_policy_cancel_picks */
+  void (*cancel_picks_locked)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                              uint32_t initial_metadata_flags_mask,
+                              uint32_t initial_metadata_flags_eq,
+                              grpc_error *error);
+
+  /** \see grpc_lb_policy_ping_one */
+  void (*ping_one_locked)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                          grpc_closure *closure);
+
+  /** Try to enter a READY connectivity state */
+  void (*exit_idle_locked)(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy);
+
+  /** check the current connectivity of the lb_policy */
+  grpc_connectivity_state (*check_connectivity_locked)(
+      grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+      grpc_error **connectivity_error);
+
+  /** call notify when the connectivity state of a channel changes from *state.
+      Updates *state with the new state of the policy. Calling with a NULL \a
+      state cancels the subscription.  */
+  void (*notify_on_state_change_locked)(grpc_exec_ctx *exec_ctx,
+                                        grpc_lb_policy *policy,
+                                        grpc_connectivity_state *state,
+                                        grpc_closure *closure);
+};
+
+/*#define GRPC_LB_POLICY_REFCOUNT_DEBUG*/
+#ifdef GRPC_LB_POLICY_REFCOUNT_DEBUG
+
+/* Strong references: the policy will shutdown when they reach zero */
+#define GRPC_LB_POLICY_REF(p, r) \
+  grpc_lb_policy_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_LB_POLICY_UNREF(exec_ctx, p, r) \
+  grpc_lb_policy_unref((exec_ctx), (p), __FILE__, __LINE__, (r))
+
+/* Weak references: they don't prevent the shutdown of the LB policy. When no
+ * strong references are left but there are still weak ones, shutdown is called.
+ * Once the weak reference also reaches zero, the LB policy is destroyed. */
+#define GRPC_LB_POLICY_WEAK_REF(p, r) \
+  grpc_lb_policy_weak_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, p, r) \
+  grpc_lb_policy_weak_unref((exec_ctx), (p), __FILE__, __LINE__, (r))
+void grpc_lb_policy_ref(grpc_lb_policy *policy, const char *file, int line,
+                        const char *reason);
+void grpc_lb_policy_unref(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                          const char *file, int line, const char *reason);
+void grpc_lb_policy_weak_ref(grpc_lb_policy *policy, const char *file, int line,
+                             const char *reason);
+void grpc_lb_policy_weak_unref(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                               const char *file, int line, const char *reason);
+#else
+#define GRPC_LB_POLICY_REF(p, r) grpc_lb_policy_ref((p))
+#define GRPC_LB_POLICY_UNREF(cl, p, r) grpc_lb_policy_unref((cl), (p))
+#define GRPC_LB_POLICY_WEAK_REF(p, r) grpc_lb_policy_weak_ref((p))
+#define GRPC_LB_POLICY_WEAK_UNREF(cl, p, r) grpc_lb_policy_weak_unref((cl), (p))
+void grpc_lb_policy_ref(grpc_lb_policy *policy);
+void grpc_lb_policy_unref(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy);
+void grpc_lb_policy_weak_ref(grpc_lb_policy *policy);
+void grpc_lb_policy_weak_unref(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy);
+#endif
+
+/** called by concrete implementations to initialize the base struct */
+void grpc_lb_policy_init(grpc_lb_policy *policy,
+                         const grpc_lb_policy_vtable *vtable,
+                         grpc_combiner *combiner);
+
+/** Finds an appropriate subchannel for a call, based on \a pick_args.
+
+    \a target will be set to the selected subchannel, or NULL on failure.
+    Upon success, \a user_data will be set to whatever opaque information
+    may need to be propagated from the LB policy, or NULL if not needed.
+
+    If the pick succeeds and a result is known immediately, a non-zero
+    value will be returned.  Otherwise, \a on_complete will be invoked
+    once the pick is complete with its error argument set to indicate
+    success or failure.
+
+    Any IO should be done under the \a interested_parties \a grpc_pollset_set
+    in the \a grpc_lb_policy struct. */
+int grpc_lb_policy_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+                               const grpc_lb_policy_pick_args *pick_args,
+                               grpc_connected_subchannel **target,
+                               void **user_data, grpc_closure *on_complete);
+
+/** Perform a connected subchannel ping (see \a grpc_connected_subchannel_ping)
+    against one of the connected subchannels managed by \a policy. */
+void grpc_lb_policy_ping_one_locked(grpc_exec_ctx *exec_ctx,
+                                    grpc_lb_policy *policy,
+                                    grpc_closure *closure);
+
+/** Cancel picks for \a target.
+    The \a on_complete callback of the pending picks will be invoked with \a
+    *target set to NULL. */
+void grpc_lb_policy_cancel_pick_locked(grpc_exec_ctx *exec_ctx,
+                                       grpc_lb_policy *policy,
+                                       grpc_connected_subchannel **target,
+                                       grpc_error *error);
+
+/** Cancel all pending picks for which their \a initial_metadata_flags (as given
+    in the call to \a grpc_lb_policy_pick) matches \a initial_metadata_flags_eq
+    when AND'd with \a initial_metadata_flags_mask */
+void grpc_lb_policy_cancel_picks_locked(grpc_exec_ctx *exec_ctx,
+                                        grpc_lb_policy *policy,
+                                        uint32_t initial_metadata_flags_mask,
+                                        uint32_t initial_metadata_flags_eq,
+                                        grpc_error *error);
+
+/** Try to enter a READY connectivity state */
+void grpc_lb_policy_exit_idle_locked(grpc_exec_ctx *exec_ctx,
+                                     grpc_lb_policy *policy);
+
+/* Call notify when the connectivity state of a channel changes from \a *state.
+ * Updates \a *state with the new state of the policy */
+void grpc_lb_policy_notify_on_state_change_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+    grpc_connectivity_state *state, grpc_closure *closure);
+
+grpc_connectivity_state grpc_lb_policy_check_connectivity_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *policy,
+    grpc_error **connectivity_error);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_H */
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.c b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.c
new file mode 100644
index 0000000..71c7bd9
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.c
@@ -0,0 +1,1417 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+/** Implementation of the gRPC LB policy.
+ *
+ * This policy takes as input a set of resolved addresses {a1..an} for which the
+ * LB set was set (it's the resolver's responsibility to ensure this). That is
+ * to say, {a1..an} represent a collection of LB servers.
+ *
+ * An internal channel (\a glb_lb_policy.lb_channel) is created over {a1..an}.
+ * This channel behaves just like a regular channel. In particular, the
+ * constructed URI over the addresses a1..an will use the default pick first
+ * policy to select from this list of LB server backends.
+ *
+ * The first time the policy gets a request for a pick, a ping, or to exit the
+ * idle state, \a query_for_backends_locked() is called. This function sets up
+ * and initiates the internal communication with the LB server. In particular,
+ * it's responsible for instantiating the internal *streaming* call to the LB
+ * server (whichever address from {a1..an} pick-first chose). This call is
+ * serviced by two callbacks, \a lb_on_server_status_received and \a
+ * lb_on_response_received. The former will be called when the call to the LB
+ * server completes. This can happen if the LB server closes the connection or
+ * if this policy itself cancels the call (for example because it's shutting
+ * down). If the internal call times out, the usual behavior of pick-first
+ * applies, continuing to pick from the list {a1..an}.
+ *
+ * Upon sucesss, the incoming \a LoadBalancingResponse is processed by \a
+ * res_recv. An invalid one results in the termination of the streaming call. A
+ * new streaming call should be created if possible, failing the original call
+ * otherwise. For a valid \a LoadBalancingResponse, the server list of actual
+ * backends is extracted. A Round Robin policy will be created from this list.
+ * There are two possible scenarios:
+ *
+ * 1. This is the first server list received. There was no previous instance of
+ *    the Round Robin policy. \a rr_handover_locked() will instantiate the RR
+ *    policy and perform all the pending operations over it.
+ * 2. There's already a RR policy instance active. We need to introduce the new
+ *    one build from the new serverlist, but taking care not to disrupt the
+ *    operations in progress over the old RR instance. This is done by
+ *    decreasing the reference count on the old policy. The moment no more
+ *    references are held on the old RR policy, it'll be destroyed and \a
+ *    glb_rr_connectivity_changed notified with a \a GRPC_CHANNEL_SHUTDOWN
+ *    state. At this point we can transition to a new RR instance safely, which
+ *    is done once again via \a rr_handover_locked().
+ *
+ *
+ * Once a RR policy instance is in place (and getting updated as described),
+ * calls to for a pick, a ping or a cancellation will be serviced right away by
+ * forwarding them to the RR instance. Any time there's no RR policy available
+ * (ie, right after the creation of the gRPCLB policy, if an empty serverlist is
+ * received, etc), pick/ping requests are added to a list of pending picks/pings
+ * to be flushed and serviced as part of \a rr_handover_locked() the moment the
+ * RR policy instance becomes available.
+ *
+ * \see https://github.com/grpc/grpc/blob/master/doc/load-balancing.md for the
+ * high level design and details. */
+
+/* TODO(dgq):
+ * - Implement LB service forwarding (point 2c. in the doc's diagram).
+ */
+
+/* With the addition of a libuv endpoint, sockaddr.h now includes uv.h when
+   using that endpoint. Because of various transitive includes in uv.h,
+   including windows.h on Windows, uv.h must be included before other system
+   headers. Therefore, sockaddr.h must always be included first */
+#include "src/core/lib/iomgr/sockaddr.h"
+
+#include <errno.h>
+
+#include <string.h>
+
+#include <grpc/byte_buffer_reader.h>
+#include <grpc/grpc.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/host_port.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/time.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/ext/filters/client_channel/client_channel_factory.h"
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+#include "src/core/ext/filters/client_channel/lb_policy_registry.h"
+#include "src/core/ext/filters/client_channel/parse_address.h"
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.h"
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.h"
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/sockaddr.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/iomgr/timer.h"
+#include "src/core/lib/slice/slice_hash_table.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/slice/slice_string_helpers.h"
+#include "src/core/lib/support/backoff.h"
+#include "src/core/lib/support/string.h"
+#include "src/core/lib/surface/call.h"
+#include "src/core/lib/surface/channel.h"
+#include "src/core/lib/transport/static_metadata.h"
+
+#define GRPC_GRPCLB_MIN_CONNECT_TIMEOUT_SECONDS 20
+#define GRPC_GRPCLB_INITIAL_CONNECT_BACKOFF_SECONDS 1
+#define GRPC_GRPCLB_RECONNECT_BACKOFF_MULTIPLIER 1.6
+#define GRPC_GRPCLB_RECONNECT_MAX_BACKOFF_SECONDS 120
+#define GRPC_GRPCLB_RECONNECT_JITTER 0.2
+
+int grpc_lb_glb_trace = 0;
+
+/* add lb_token of selected subchannel (address) to the call's initial
+ * metadata */
+static grpc_error *initial_metadata_add_lb_token(
+    grpc_exec_ctx *exec_ctx, grpc_metadata_batch *initial_metadata,
+    grpc_linked_mdelem *lb_token_mdelem_storage, grpc_mdelem lb_token) {
+  GPR_ASSERT(lb_token_mdelem_storage != NULL);
+  GPR_ASSERT(!GRPC_MDISNULL(lb_token));
+  return grpc_metadata_batch_add_tail(exec_ctx, initial_metadata,
+                                      lb_token_mdelem_storage, lb_token);
+}
+
+typedef struct wrapped_rr_closure_arg {
+  /* the closure instance using this struct as argument */
+  grpc_closure wrapper_closure;
+
+  /* the original closure. Usually a on_complete/notify cb for pick() and ping()
+   * calls against the internal RR instance, respectively. */
+  grpc_closure *wrapped_closure;
+
+  /* the pick's initial metadata, kept in order to append the LB token for the
+   * pick */
+  grpc_metadata_batch *initial_metadata;
+
+  /* the picked target, used to determine which LB token to add to the pick's
+   * initial metadata */
+  grpc_connected_subchannel **target;
+
+  /* the LB token associated with the pick */
+  grpc_mdelem lb_token;
+
+  /* storage for the lb token initial metadata mdelem */
+  grpc_linked_mdelem *lb_token_mdelem_storage;
+
+  /* The RR instance related to the closure */
+  grpc_lb_policy *rr_policy;
+
+  /* heap memory to be freed upon closure execution. */
+  void *free_when_done;
+} wrapped_rr_closure_arg;
+
+/* The \a on_complete closure passed as part of the pick requires keeping a
+ * reference to its associated round robin instance. We wrap this closure in
+ * order to unref the round robin instance upon its invocation */
+static void wrapped_rr_closure(grpc_exec_ctx *exec_ctx, void *arg,
+                               grpc_error *error) {
+  wrapped_rr_closure_arg *wc_arg = arg;
+
+  GPR_ASSERT(wc_arg->wrapped_closure != NULL);
+  grpc_closure_sched(exec_ctx, wc_arg->wrapped_closure, GRPC_ERROR_REF(error));
+
+  if (wc_arg->rr_policy != NULL) {
+    /* if *target is NULL, no pick has been made by the RR policy (eg, all
+     * addresses failed to connect). There won't be any user_data/token
+     * available */
+    if (*wc_arg->target != NULL) {
+      if (!GRPC_MDISNULL(wc_arg->lb_token)) {
+        initial_metadata_add_lb_token(exec_ctx, wc_arg->initial_metadata,
+                                      wc_arg->lb_token_mdelem_storage,
+                                      GRPC_MDELEM_REF(wc_arg->lb_token));
+      } else {
+        gpr_log(GPR_ERROR,
+                "No LB token for connected subchannel pick %p (from RR "
+                "instance %p).",
+                (void *)*wc_arg->target, (void *)wc_arg->rr_policy);
+        abort();
+      }
+    }
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO, "Unreffing RR %p", (void *)wc_arg->rr_policy);
+    }
+    GRPC_LB_POLICY_UNREF(exec_ctx, wc_arg->rr_policy, "wrapped_rr_closure");
+  }
+  GPR_ASSERT(wc_arg->free_when_done != NULL);
+  gpr_free(wc_arg->free_when_done);
+}
+
+/* Linked list of pending pick requests. It stores all information needed to
+ * eventually call (Round Robin's) pick() on them. They mainly stay pending
+ * waiting for the RR policy to be created/updated.
+ *
+ * One particularity is the wrapping of the user-provided \a on_complete closure
+ * (in \a wrapped_on_complete and \a wrapped_on_complete_arg). This is needed in
+ * order to correctly unref the RR policy instance upon completion of the pick.
+ * See \a wrapped_rr_closure for details. */
+typedef struct pending_pick {
+  struct pending_pick *next;
+
+  /* original pick()'s arguments */
+  grpc_lb_policy_pick_args pick_args;
+
+  /* output argument where to store the pick()ed connected subchannel, or NULL
+   * upon error. */
+  grpc_connected_subchannel **target;
+
+  /* args for wrapped_on_complete */
+  wrapped_rr_closure_arg wrapped_on_complete_arg;
+} pending_pick;
+
+static void add_pending_pick(pending_pick **root,
+                             const grpc_lb_policy_pick_args *pick_args,
+                             grpc_connected_subchannel **target,
+                             grpc_closure *on_complete) {
+  pending_pick *pp = gpr_zalloc(sizeof(*pp));
+  pp->next = *root;
+  pp->pick_args = *pick_args;
+  pp->target = target;
+  pp->wrapped_on_complete_arg.wrapped_closure = on_complete;
+  pp->wrapped_on_complete_arg.target = target;
+  pp->wrapped_on_complete_arg.initial_metadata = pick_args->initial_metadata;
+  pp->wrapped_on_complete_arg.lb_token_mdelem_storage =
+      pick_args->lb_token_mdelem_storage;
+  pp->wrapped_on_complete_arg.free_when_done = pp;
+  grpc_closure_init(&pp->wrapped_on_complete_arg.wrapper_closure,
+                    wrapped_rr_closure, &pp->wrapped_on_complete_arg,
+                    grpc_schedule_on_exec_ctx);
+  *root = pp;
+}
+
+/* Same as the \a pending_pick struct but for ping operations */
+typedef struct pending_ping {
+  struct pending_ping *next;
+
+  /* args for wrapped_notify */
+  wrapped_rr_closure_arg wrapped_notify_arg;
+} pending_ping;
+
+static void add_pending_ping(pending_ping **root, grpc_closure *notify) {
+  pending_ping *pping = gpr_zalloc(sizeof(*pping));
+  pping->wrapped_notify_arg.wrapped_closure = notify;
+  pping->wrapped_notify_arg.free_when_done = pping;
+  pping->next = *root;
+  grpc_closure_init(&pping->wrapped_notify_arg.wrapper_closure,
+                    wrapped_rr_closure, &pping->wrapped_notify_arg,
+                    grpc_schedule_on_exec_ctx);
+  *root = pping;
+}
+
+/*
+ * glb_lb_policy
+ */
+typedef struct rr_connectivity_data rr_connectivity_data;
+static const grpc_lb_policy_vtable glb_lb_policy_vtable;
+typedef struct glb_lb_policy {
+  /** base policy: must be first */
+  grpc_lb_policy base;
+
+  /** who the client is trying to communicate with */
+  const char *server_name;
+  grpc_client_channel_factory *cc_factory;
+  grpc_channel_args *args;
+
+  /** deadline for the LB's call */
+  gpr_timespec deadline;
+
+  /** for communicating with the LB server */
+  grpc_channel *lb_channel;
+
+  /** the RR policy to use of the backend servers returned by the LB server */
+  grpc_lb_policy *rr_policy;
+
+  bool started_picking;
+
+  /** our connectivity state tracker */
+  grpc_connectivity_state_tracker state_tracker;
+
+  /** stores the deserialized response from the LB. May be NULL until one such
+   * response has arrived. */
+  grpc_grpclb_serverlist *serverlist;
+
+  /** list of picks that are waiting on RR's policy connectivity */
+  pending_pick *pending_picks;
+
+  /** list of pings that are waiting on RR's policy connectivity */
+  pending_ping *pending_pings;
+
+  bool shutting_down;
+
+  /************************************************************/
+  /*  client data associated with the LB server communication */
+  /************************************************************/
+  /* Status from the LB server has been received. This signals the end of the LB
+   * call. */
+  grpc_closure lb_on_server_status_received;
+
+  /* A response from the LB server has been received. Process it */
+  grpc_closure lb_on_response_received;
+
+  /* LB call retry timer callback. */
+  grpc_closure lb_on_call_retry;
+
+  grpc_call *lb_call; /* streaming call to the LB server, */
+
+  grpc_metadata_array lb_initial_metadata_recv; /* initial MD from LB server */
+  grpc_metadata_array
+      lb_trailing_metadata_recv; /* trailing MD from LB server */
+
+  /* what's being sent to the LB server. Note that its value may vary if the LB
+   * server indicates a redirect. */
+  grpc_byte_buffer *lb_request_payload;
+
+  /* response the LB server, if any. Processed in lb_on_response_received() */
+  grpc_byte_buffer *lb_response_payload;
+
+  /* call status code and details, set in lb_on_server_status_received() */
+  grpc_status_code lb_call_status;
+  grpc_slice lb_call_status_details;
+
+  /** LB call retry backoff state */
+  gpr_backoff lb_call_backoff_state;
+
+  /** LB call retry timer */
+  grpc_timer lb_call_retry_timer;
+} glb_lb_policy;
+
+/* Keeps track and reacts to changes in connectivity of the RR instance */
+struct rr_connectivity_data {
+  grpc_closure on_change;
+  grpc_connectivity_state state;
+  glb_lb_policy *glb_policy;
+};
+
+static bool is_server_valid(const grpc_grpclb_server *server, size_t idx,
+                            bool log) {
+  const grpc_grpclb_ip_address *ip = &server->ip_address;
+  if (server->port >> 16 != 0) {
+    if (log) {
+      gpr_log(GPR_ERROR,
+              "Invalid port '%d' at index %lu of serverlist. Ignoring.",
+              server->port, (unsigned long)idx);
+    }
+    return false;
+  }
+
+  if (ip->size != 4 && ip->size != 16) {
+    if (log) {
+      gpr_log(GPR_ERROR,
+              "Expected IP to be 4 or 16 bytes, got %d at index %lu of "
+              "serverlist. Ignoring",
+              ip->size, (unsigned long)idx);
+    }
+    return false;
+  }
+  return true;
+}
+
+/* vtable for LB tokens in grpc_lb_addresses. */
+static void *lb_token_copy(void *token) {
+  return token == NULL
+             ? NULL
+             : (void *)GRPC_MDELEM_REF((grpc_mdelem){(uintptr_t)token}).payload;
+}
+static void lb_token_destroy(grpc_exec_ctx *exec_ctx, void *token) {
+  if (token != NULL) {
+    GRPC_MDELEM_UNREF(exec_ctx, (grpc_mdelem){(uintptr_t)token});
+  }
+}
+static int lb_token_cmp(void *token1, void *token2) {
+  if (token1 > token2) return 1;
+  if (token1 < token2) return -1;
+  return 0;
+}
+static const grpc_lb_user_data_vtable lb_token_vtable = {
+    lb_token_copy, lb_token_destroy, lb_token_cmp};
+
+static void parse_server(const grpc_grpclb_server *server,
+                         grpc_resolved_address *addr) {
+  const uint16_t netorder_port = htons((uint16_t)server->port);
+  /* the addresses are given in binary format (a in(6)_addr struct) in
+   * server->ip_address.bytes. */
+  const grpc_grpclb_ip_address *ip = &server->ip_address;
+  memset(addr, 0, sizeof(*addr));
+  if (ip->size == 4) {
+    addr->len = sizeof(struct sockaddr_in);
+    struct sockaddr_in *addr4 = (struct sockaddr_in *)&addr->addr;
+    addr4->sin_family = AF_INET;
+    memcpy(&addr4->sin_addr, ip->bytes, ip->size);
+    addr4->sin_port = netorder_port;
+  } else if (ip->size == 16) {
+    addr->len = sizeof(struct sockaddr_in6);
+    struct sockaddr_in6 *addr6 = (struct sockaddr_in6 *)&addr->addr;
+    addr6->sin6_family = AF_INET6;
+    memcpy(&addr6->sin6_addr, ip->bytes, ip->size);
+    addr6->sin6_port = netorder_port;
+  }
+}
+
+/* Returns addresses extracted from \a serverlist. */
+static grpc_lb_addresses *process_serverlist_locked(
+    grpc_exec_ctx *exec_ctx, const grpc_grpclb_serverlist *serverlist) {
+  size_t num_valid = 0;
+  /* first pass: count how many are valid in order to allocate the necessary
+   * memory in a single block */
+  for (size_t i = 0; i < serverlist->num_servers; ++i) {
+    if (is_server_valid(serverlist->servers[i], i, true)) ++num_valid;
+  }
+  if (num_valid == 0) return NULL;
+
+  grpc_lb_addresses *lb_addresses =
+      grpc_lb_addresses_create(num_valid, &lb_token_vtable);
+
+  /* second pass: actually populate the addresses and LB tokens (aka user data
+   * to the outside world) to be read by the RR policy during its creation.
+   * Given that the validity tests are very cheap, they are performed again
+   * instead of marking the valid ones during the first pass, as this would
+   * incurr in an allocation due to the arbitrary number of server */
+  size_t addr_idx = 0;
+  for (size_t sl_idx = 0; sl_idx < serverlist->num_servers; ++sl_idx) {
+    GPR_ASSERT(addr_idx < num_valid);
+    const grpc_grpclb_server *server = serverlist->servers[sl_idx];
+    if (!is_server_valid(serverlist->servers[sl_idx], sl_idx, false)) continue;
+
+    /* address processing */
+    grpc_resolved_address addr;
+    parse_server(server, &addr);
+
+    /* lb token processing */
+    void *user_data;
+    if (server->has_load_balance_token) {
+      const size_t lb_token_max_length =
+          GPR_ARRAY_SIZE(server->load_balance_token);
+      const size_t lb_token_length =
+          strnlen(server->load_balance_token, lb_token_max_length);
+      grpc_slice lb_token_mdstr = grpc_slice_from_copied_buffer(
+          server->load_balance_token, lb_token_length);
+      user_data = (void *)grpc_mdelem_from_slices(exec_ctx, GRPC_MDSTR_LB_TOKEN,
+                                                  lb_token_mdstr)
+                      .payload;
+    } else {
+      char *uri = grpc_sockaddr_to_uri(&addr);
+      gpr_log(GPR_INFO,
+              "Missing LB token for backend address '%s'. The empty token will "
+              "be used instead",
+              uri);
+      gpr_free(uri);
+      user_data = (void *)GRPC_MDELEM_LB_TOKEN_EMPTY.payload;
+    }
+
+    grpc_lb_addresses_set_address(lb_addresses, addr_idx, &addr.addr, addr.len,
+                                  false /* is_balancer */,
+                                  NULL /* balancer_name */, user_data);
+    ++addr_idx;
+  }
+  GPR_ASSERT(addr_idx == num_valid);
+  return lb_addresses;
+}
+
+/* returns true if the new RR policy should replace the current one, if any */
+static bool update_lb_connectivity_status_locked(
+    grpc_exec_ctx *exec_ctx, glb_lb_policy *glb_policy,
+    grpc_connectivity_state new_rr_state, grpc_error *new_rr_state_error) {
+  const grpc_connectivity_state curr_glb_state =
+      grpc_connectivity_state_check(&glb_policy->state_tracker);
+
+  /* The new connectivity status is a function of the previous one and the new
+   * input coming from the status of the RR policy.
+   *
+   *  current state (grpclb's)
+   *  |
+   *  v  || I  |  C  |  R  |  TF  |  SD  |  <- new state (RR's)
+   *  ===++====+=====+=====+======+======+
+   *   I || I  |  C  |  R  | [I]  | [I]  |
+   *  ---++----+-----+-----+------+------+
+   *   C || I  |  C  |  R  | [C]  | [C]  |
+   *  ---++----+-----+-----+------+------+
+   *   R || I  |  C  |  R  | [R]  | [R]  |
+   *  ---++----+-----+-----+------+------+
+   *  TF || I  |  C  |  R  | [TF] | [TF] |
+   *  ---++----+-----+-----+------+------+
+   *  SD || NA |  NA |  NA |  NA  |  NA  | (*)
+   *  ---++----+-----+-----+------+------+
+   *
+   * A [STATE] indicates that the old RR policy is kept. In those cases, STATE
+   * is the current state of grpclb, which is left untouched.
+   *
+   *  In summary, if the new state is TRANSIENT_FAILURE or SHUTDOWN, stick to
+   *  the previous RR instance.
+   *
+   *  Note that the status is never updated to SHUTDOWN as a result of calling
+   *  this function. Only glb_shutdown() has the power to set that state.
+   *
+   *  (*) This function mustn't be called during shutting down. */
+  GPR_ASSERT(curr_glb_state != GRPC_CHANNEL_SHUTDOWN);
+
+  switch (new_rr_state) {
+    case GRPC_CHANNEL_TRANSIENT_FAILURE:
+    case GRPC_CHANNEL_SHUTDOWN:
+      GPR_ASSERT(new_rr_state_error != GRPC_ERROR_NONE);
+      return false; /* don't replace the RR policy */
+    case GRPC_CHANNEL_INIT:
+    case GRPC_CHANNEL_IDLE:
+    case GRPC_CHANNEL_CONNECTING:
+    case GRPC_CHANNEL_READY:
+      GPR_ASSERT(new_rr_state_error == GRPC_ERROR_NONE);
+  }
+
+  if (grpc_lb_glb_trace) {
+    gpr_log(GPR_INFO,
+            "Setting grpclb's state to %s from new RR policy %p state.",
+            grpc_connectivity_state_name(new_rr_state),
+            (void *)glb_policy->rr_policy);
+  }
+  grpc_connectivity_state_set(exec_ctx, &glb_policy->state_tracker,
+                              new_rr_state, GRPC_ERROR_REF(new_rr_state_error),
+                              "update_lb_connectivity_status_locked");
+  return true;
+}
+
+/* perform a pick over \a rr_policy. Given that a pick can return immediately
+ * (ignoring its completion callback) we need to perform the cleanups this
+ * callback would be otherwise resposible for */
+static bool pick_from_internal_rr_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *rr_policy,
+    const grpc_lb_policy_pick_args *pick_args,
+    grpc_connected_subchannel **target, wrapped_rr_closure_arg *wc_arg) {
+  GPR_ASSERT(rr_policy != NULL);
+  const bool pick_done = grpc_lb_policy_pick_locked(
+      exec_ctx, rr_policy, pick_args, target, (void **)&wc_arg->lb_token,
+      &wc_arg->wrapper_closure);
+  if (pick_done) {
+    /* synchronous grpc_lb_policy_pick call. Unref the RR policy. */
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO, "Unreffing RR (0x%" PRIxPTR ")",
+              (intptr_t)wc_arg->rr_policy);
+    }
+    GRPC_LB_POLICY_UNREF(exec_ctx, wc_arg->rr_policy, "glb_pick_sync");
+
+    /* add the load reporting initial metadata */
+    initial_metadata_add_lb_token(exec_ctx, pick_args->initial_metadata,
+                                  pick_args->lb_token_mdelem_storage,
+                                  GRPC_MDELEM_REF(wc_arg->lb_token));
+
+    gpr_free(wc_arg);
+  }
+  /* else, the pending pick will be registered and taken care of by the
+   * pending pick list inside the RR policy (glb_policy->rr_policy).
+   * Eventually, wrapped_on_complete will be called, which will -among other
+   * things- add the LB token to the call's initial metadata */
+  return pick_done;
+}
+
+static grpc_lb_policy *create_rr_locked(
+    grpc_exec_ctx *exec_ctx, const grpc_grpclb_serverlist *serverlist,
+    glb_lb_policy *glb_policy) {
+  GPR_ASSERT(serverlist != NULL && serverlist->num_servers > 0);
+
+  grpc_lb_policy_args args;
+  memset(&args, 0, sizeof(args));
+  args.client_channel_factory = glb_policy->cc_factory;
+  args.combiner = glb_policy->base.combiner;
+  grpc_lb_addresses *addresses =
+      process_serverlist_locked(exec_ctx, serverlist);
+
+  // Replace the LB addresses in the channel args that we pass down to
+  // the subchannel.
+  static const char *keys_to_remove[] = {GRPC_ARG_LB_ADDRESSES};
+  const grpc_arg arg = grpc_lb_addresses_create_channel_arg(addresses);
+  args.args = grpc_channel_args_copy_and_add_and_remove(
+      glb_policy->args, keys_to_remove, GPR_ARRAY_SIZE(keys_to_remove), &arg,
+      1);
+
+  grpc_lb_policy *rr = grpc_lb_policy_create(exec_ctx, "round_robin", &args);
+  GPR_ASSERT(rr != NULL);
+  grpc_lb_addresses_destroy(exec_ctx, addresses);
+  grpc_channel_args_destroy(exec_ctx, args.args);
+  return rr;
+}
+
+static void glb_rr_connectivity_changed_locked(grpc_exec_ctx *exec_ctx,
+                                               void *arg, grpc_error *error);
+/* glb_policy->rr_policy may be NULL (initial handover) */
+static void rr_handover_locked(grpc_exec_ctx *exec_ctx,
+                               glb_lb_policy *glb_policy) {
+  GPR_ASSERT(glb_policy->serverlist != NULL &&
+             glb_policy->serverlist->num_servers > 0);
+
+  if (glb_policy->shutting_down) return;
+
+  grpc_lb_policy *new_rr_policy =
+      create_rr_locked(exec_ctx, glb_policy->serverlist, glb_policy);
+  if (new_rr_policy == NULL) {
+    gpr_log(GPR_ERROR,
+            "Failure creating a RoundRobin policy for serverlist update with "
+            "%lu entries. The previous RR instance (%p), if any, will continue "
+            "to be used. Future updates from the LB will attempt to create new "
+            "instances.",
+            (unsigned long)glb_policy->serverlist->num_servers,
+            (void *)glb_policy->rr_policy);
+    return;
+  }
+
+  grpc_error *new_rr_state_error = NULL;
+  const grpc_connectivity_state new_rr_state =
+      grpc_lb_policy_check_connectivity_locked(exec_ctx, new_rr_policy,
+                                               &new_rr_state_error);
+  /* Connectivity state is a function of the new RR policy just created */
+  const bool replace_old_rr = update_lb_connectivity_status_locked(
+      exec_ctx, glb_policy, new_rr_state, new_rr_state_error);
+
+  if (!replace_old_rr) {
+    /* dispose of the new RR policy that won't be used after all */
+    GRPC_LB_POLICY_UNREF(exec_ctx, new_rr_policy, "rr_handover_no_replace");
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO,
+              "Keeping old RR policy (%p) despite new serverlist: new RR "
+              "policy was in %s connectivity state.",
+              (void *)glb_policy->rr_policy,
+              grpc_connectivity_state_name(new_rr_state));
+    }
+    return;
+  }
+
+  if (grpc_lb_glb_trace) {
+    gpr_log(GPR_INFO, "Created RR policy (%p) to replace old RR (%p)",
+            (void *)new_rr_policy, (void *)glb_policy->rr_policy);
+  }
+
+  if (glb_policy->rr_policy != NULL) {
+    /* if we are phasing out an existing RR instance, unref it. */
+    GRPC_LB_POLICY_UNREF(exec_ctx, glb_policy->rr_policy, "rr_handover");
+  }
+
+  /* Finally update the RR policy to the newly created one */
+  glb_policy->rr_policy = new_rr_policy;
+
+  /* Add the gRPC LB's interested_parties pollset_set to that of the newly
+   * created RR policy. This will make the RR policy progress upon activity on
+   * gRPC LB, which in turn is tied to the application's call */
+  grpc_pollset_set_add_pollset_set(exec_ctx,
+                                   glb_policy->rr_policy->interested_parties,
+                                   glb_policy->base.interested_parties);
+
+  /* Allocate the data for the tracking of the new RR policy's connectivity.
+   * It'll be deallocated in glb_rr_connectivity_changed() */
+  rr_connectivity_data *rr_connectivity =
+      gpr_zalloc(sizeof(rr_connectivity_data));
+  grpc_closure_init(&rr_connectivity->on_change,
+                    glb_rr_connectivity_changed_locked, rr_connectivity,
+                    grpc_combiner_scheduler(glb_policy->base.combiner, false));
+  rr_connectivity->glb_policy = glb_policy;
+  rr_connectivity->state = new_rr_state;
+
+  /* Subscribe to changes to the connectivity of the new RR */
+  GRPC_LB_POLICY_WEAK_REF(&glb_policy->base, "rr_connectivity_cb");
+  grpc_lb_policy_notify_on_state_change_locked(exec_ctx, glb_policy->rr_policy,
+                                               &rr_connectivity->state,
+                                               &rr_connectivity->on_change);
+  grpc_lb_policy_exit_idle_locked(exec_ctx, glb_policy->rr_policy);
+
+  /* Update picks and pings in wait */
+  pending_pick *pp;
+  while ((pp = glb_policy->pending_picks)) {
+    glb_policy->pending_picks = pp->next;
+    GRPC_LB_POLICY_REF(glb_policy->rr_policy, "rr_handover_pending_pick");
+    pp->wrapped_on_complete_arg.rr_policy = glb_policy->rr_policy;
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO, "Pending pick about to PICK from 0x%" PRIxPTR "",
+              (intptr_t)glb_policy->rr_policy);
+    }
+    pick_from_internal_rr_locked(exec_ctx, glb_policy->rr_policy,
+                                 &pp->pick_args, pp->target,
+                                 &pp->wrapped_on_complete_arg);
+  }
+
+  pending_ping *pping;
+  while ((pping = glb_policy->pending_pings)) {
+    glb_policy->pending_pings = pping->next;
+    GRPC_LB_POLICY_REF(glb_policy->rr_policy, "rr_handover_pending_ping");
+    pping->wrapped_notify_arg.rr_policy = glb_policy->rr_policy;
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO, "Pending ping about to PING from 0x%" PRIxPTR "",
+              (intptr_t)glb_policy->rr_policy);
+    }
+    grpc_lb_policy_ping_one_locked(exec_ctx, glb_policy->rr_policy,
+                                   &pping->wrapped_notify_arg.wrapper_closure);
+  }
+}
+
+static void glb_rr_connectivity_changed_locked(grpc_exec_ctx *exec_ctx,
+                                               void *arg, grpc_error *error) {
+  rr_connectivity_data *rr_connectivity = arg;
+  glb_lb_policy *glb_policy = rr_connectivity->glb_policy;
+
+  const bool shutting_down = glb_policy->shutting_down;
+  bool unref_needed = false;
+  GRPC_ERROR_REF(error);
+
+  if (rr_connectivity->state == GRPC_CHANNEL_SHUTDOWN || shutting_down) {
+    /* RR policy shutting down. Don't renew subscription and free the arg of
+     * this callback. In addition  we need to stash away the current policy to
+     * be UNREF'd after releasing the lock. Otherwise, if the UNREF is the last
+     * one, the policy would be destroyed, alongside the lock, which would
+     * result in a use-after-free */
+    unref_needed = true;
+    gpr_free(rr_connectivity);
+  } else { /* rr state != SHUTDOWN && !shutting down: biz as usual */
+    update_lb_connectivity_status_locked(exec_ctx, glb_policy,
+                                         rr_connectivity->state, error);
+    /* Resubscribe. Reuse the "rr_connectivity_cb" weak ref. */
+    grpc_lb_policy_notify_on_state_change_locked(
+        exec_ctx, glb_policy->rr_policy, &rr_connectivity->state,
+        &rr_connectivity->on_change);
+  }
+  if (unref_needed) {
+    GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &glb_policy->base,
+                              "rr_connectivity_cb");
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void destroy_balancer_name(grpc_exec_ctx *exec_ctx,
+                                  void *balancer_name) {
+  gpr_free(balancer_name);
+}
+
+static void *copy_balancer_name(void *balancer_name) {
+  return gpr_strdup(balancer_name);
+}
+
+static grpc_slice_hash_table_entry targets_info_entry_create(
+    const char *address, const char *balancer_name) {
+  static const grpc_slice_hash_table_vtable vtable = {destroy_balancer_name,
+                                                      copy_balancer_name};
+  grpc_slice_hash_table_entry entry;
+  entry.key = grpc_slice_from_copied_string(address);
+  entry.value = (void *)balancer_name;
+  entry.vtable = &vtable;
+  return entry;
+}
+
+/* Returns the target URI for the LB service whose addresses are in \a
+ * addresses.  Using this URI, a bidirectional streaming channel will be created
+ * for the reception of load balancing updates.
+ *
+ * The output argument \a targets_info will be updated to contain a mapping of
+ * "LB server address" to "balancer name", as reported by the naming system.
+ * This mapping will be propagated via the channel arguments of the
+ * aforementioned LB streaming channel, to be used by the security connector for
+ * secure naming checks. The user is responsible for freeing \a targets_info. */
+static char *get_lb_uri_target_addresses(grpc_exec_ctx *exec_ctx,
+                                         const grpc_lb_addresses *addresses,
+                                         grpc_slice_hash_table **targets_info) {
+  size_t num_grpclb_addrs = 0;
+  for (size_t i = 0; i < addresses->num_addresses; ++i) {
+    if (addresses->addresses[i].is_balancer) ++num_grpclb_addrs;
+  }
+  /* All input addresses come from a resolver that claims they are LB services.
+   * It's the resolver's responsibility to make sure this policy is only
+   * instantiated and used in that case. Otherwise, something has gone wrong. */
+  GPR_ASSERT(num_grpclb_addrs > 0);
+
+  grpc_slice_hash_table_entry *targets_info_entries =
+      gpr_malloc(sizeof(*targets_info_entries) * num_grpclb_addrs);
+
+  /* construct a target ipvX://ip1:port1,ip2:port2,... from the addresses in \a
+   * addresses */
+  /* TODO(dgq): support mixed ip version */
+  char **addr_strs = gpr_malloc(sizeof(char *) * num_grpclb_addrs);
+  size_t addr_index = 0;
+
+  for (size_t i = 0; i < addresses->num_addresses; i++) {
+    if (addresses->addresses[i].user_data != NULL) {
+      gpr_log(GPR_ERROR,
+              "This LB policy doesn't support user data. It will be ignored");
+    }
+    if (addresses->addresses[i].is_balancer) {
+      char *addr_str;
+      GPR_ASSERT(grpc_sockaddr_to_string(
+                     &addr_str, &addresses->addresses[i].address, true) > 0);
+      targets_info_entries[addr_index] = targets_info_entry_create(
+          addr_str, addresses->addresses[i].balancer_name);
+      addr_strs[addr_index++] = addr_str;
+    }
+  }
+  GPR_ASSERT(addr_index == num_grpclb_addrs);
+
+  size_t uri_path_len;
+  char *uri_path = gpr_strjoin_sep((const char **)addr_strs, num_grpclb_addrs,
+                                   ",", &uri_path_len);
+  for (size_t i = 0; i < num_grpclb_addrs; i++) gpr_free(addr_strs[i]);
+  gpr_free(addr_strs);
+
+  char *target_uri_str = NULL;
+  /* TODO(dgq): Don't assume all addresses will share the scheme of the first
+   * one */
+  gpr_asprintf(&target_uri_str, "%s:%s",
+               grpc_sockaddr_get_uri_scheme(&addresses->addresses[0].address),
+               uri_path);
+  gpr_free(uri_path);
+
+  *targets_info =
+      grpc_slice_hash_table_create(num_grpclb_addrs, targets_info_entries);
+  for (size_t i = 0; i < num_grpclb_addrs; i++) {
+    grpc_slice_unref_internal(exec_ctx, targets_info_entries[i].key);
+  }
+  gpr_free(targets_info_entries);
+
+  return target_uri_str;
+}
+
+static grpc_lb_policy *glb_create(grpc_exec_ctx *exec_ctx,
+                                  grpc_lb_policy_factory *factory,
+                                  grpc_lb_policy_args *args) {
+  /* Count the number of gRPC-LB addresses. There must be at least one.
+   * TODO(roth): For now, we ignore non-balancer addresses, but in the
+   * future, we may change the behavior such that we fall back to using
+   * the non-balancer addresses if we cannot reach any balancers. At that
+   * time, this should be changed to allow a list with no balancer addresses,
+   * since the resolver might fail to return a balancer address even when
+   * this is the right LB policy to use. */
+  const grpc_arg *arg =
+      grpc_channel_args_find(args->args, GRPC_ARG_LB_ADDRESSES);
+  GPR_ASSERT(arg != NULL && arg->type == GRPC_ARG_POINTER);
+  grpc_lb_addresses *addresses = arg->value.pointer.p;
+  size_t num_grpclb_addrs = 0;
+  for (size_t i = 0; i < addresses->num_addresses; ++i) {
+    if (addresses->addresses[i].is_balancer) ++num_grpclb_addrs;
+  }
+  if (num_grpclb_addrs == 0) return NULL;
+
+  glb_lb_policy *glb_policy = gpr_zalloc(sizeof(*glb_policy));
+
+  /* Get server name. */
+  arg = grpc_channel_args_find(args->args, GRPC_ARG_SERVER_URI);
+  GPR_ASSERT(arg != NULL);
+  GPR_ASSERT(arg->type == GRPC_ARG_STRING);
+  grpc_uri *uri = grpc_uri_parse(exec_ctx, arg->value.string, true);
+  GPR_ASSERT(uri->path[0] != '\0');
+  glb_policy->server_name =
+      gpr_strdup(uri->path[0] == '/' ? uri->path + 1 : uri->path);
+  if (grpc_lb_glb_trace) {
+    gpr_log(GPR_INFO, "Will use '%s' as the server name for LB request.",
+            glb_policy->server_name);
+  }
+  grpc_uri_destroy(uri);
+
+  glb_policy->cc_factory = args->client_channel_factory;
+  glb_policy->args = grpc_channel_args_copy(args->args);
+  GPR_ASSERT(glb_policy->cc_factory != NULL);
+
+  grpc_slice_hash_table *targets_info = NULL;
+  /* Create a client channel over them to communicate with a LB service */
+  char *lb_service_target_addresses =
+      get_lb_uri_target_addresses(exec_ctx, addresses, &targets_info);
+  grpc_channel_args *lb_channel_args =
+      get_lb_channel_args(exec_ctx, targets_info, args->args);
+  glb_policy->lb_channel = grpc_lb_policy_grpclb_create_lb_channel(
+      exec_ctx, lb_service_target_addresses, args->client_channel_factory,
+      lb_channel_args);
+  grpc_slice_hash_table_unref(exec_ctx, targets_info);
+  grpc_channel_args_destroy(exec_ctx, lb_channel_args);
+  gpr_free(lb_service_target_addresses);
+  if (glb_policy->lb_channel == NULL) {
+    gpr_free(glb_policy);
+    return NULL;
+  }
+  grpc_lb_policy_init(&glb_policy->base, &glb_lb_policy_vtable, args->combiner);
+  grpc_connectivity_state_init(&glb_policy->state_tracker, GRPC_CHANNEL_IDLE,
+                               "grpclb");
+  return &glb_policy->base;
+}
+
+static void glb_destroy(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  GPR_ASSERT(glb_policy->pending_picks == NULL);
+  GPR_ASSERT(glb_policy->pending_pings == NULL);
+  gpr_free((void *)glb_policy->server_name);
+  grpc_channel_args_destroy(exec_ctx, glb_policy->args);
+  grpc_channel_destroy(glb_policy->lb_channel);
+  glb_policy->lb_channel = NULL;
+  grpc_connectivity_state_destroy(exec_ctx, &glb_policy->state_tracker);
+  if (glb_policy->serverlist != NULL) {
+    grpc_grpclb_destroy_serverlist(glb_policy->serverlist);
+  }
+  gpr_free(glb_policy);
+}
+
+static void glb_shutdown_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  glb_policy->shutting_down = true;
+
+  pending_pick *pp = glb_policy->pending_picks;
+  glb_policy->pending_picks = NULL;
+  pending_ping *pping = glb_policy->pending_pings;
+  glb_policy->pending_pings = NULL;
+  if (glb_policy->rr_policy) {
+    GRPC_LB_POLICY_UNREF(exec_ctx, glb_policy->rr_policy, "glb_shutdown");
+  }
+  grpc_connectivity_state_set(
+      exec_ctx, &glb_policy->state_tracker, GRPC_CHANNEL_SHUTDOWN,
+      GRPC_ERROR_CREATE_FROM_STATIC_STRING("Channel Shutdown"), "glb_shutdown");
+  /* We need a copy of the lb_call pointer because we can't cancell the call
+   * while holding glb_policy->mu: lb_on_server_status_received, invoked due to
+   * the cancel, needs to acquire that same lock */
+  grpc_call *lb_call = glb_policy->lb_call;
+
+  /* glb_policy->lb_call and this local lb_call must be consistent at this point
+   * because glb_policy->lb_call is only assigned in lb_call_init_locked as part
+   * of query_for_backends_locked, which can only be invoked while
+   * glb_policy->shutting_down is false. */
+  if (lb_call != NULL) {
+    grpc_call_cancel(lb_call, NULL);
+    /* lb_on_server_status_received will pick up the cancel and clean up */
+  }
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    *pp->target = NULL;
+    grpc_closure_sched(exec_ctx, &pp->wrapped_on_complete_arg.wrapper_closure,
+                       GRPC_ERROR_NONE);
+    pp = next;
+  }
+
+  while (pping != NULL) {
+    pending_ping *next = pping->next;
+    grpc_closure_sched(exec_ctx, &pping->wrapped_notify_arg.wrapper_closure,
+                       GRPC_ERROR_NONE);
+    pping = next;
+  }
+}
+
+static void glb_cancel_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                                   grpc_connected_subchannel **target,
+                                   grpc_error *error) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  pending_pick *pp = glb_policy->pending_picks;
+  glb_policy->pending_picks = NULL;
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    if (pp->target == target) {
+      *target = NULL;
+      grpc_closure_sched(exec_ctx, &pp->wrapped_on_complete_arg.wrapper_closure,
+                         GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                             "Pick Cancelled", &error, 1));
+    } else {
+      pp->next = glb_policy->pending_picks;
+      glb_policy->pending_picks = pp;
+    }
+    pp = next;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void glb_cancel_picks_locked(grpc_exec_ctx *exec_ctx,
+                                    grpc_lb_policy *pol,
+                                    uint32_t initial_metadata_flags_mask,
+                                    uint32_t initial_metadata_flags_eq,
+                                    grpc_error *error) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  pending_pick *pp = glb_policy->pending_picks;
+  glb_policy->pending_picks = NULL;
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    if ((pp->pick_args.initial_metadata_flags & initial_metadata_flags_mask) ==
+        initial_metadata_flags_eq) {
+      grpc_closure_sched(exec_ctx, &pp->wrapped_on_complete_arg.wrapper_closure,
+                         GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                             "Pick Cancelled", &error, 1));
+    } else {
+      pp->next = glb_policy->pending_picks;
+      glb_policy->pending_picks = pp;
+    }
+    pp = next;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void query_for_backends_locked(grpc_exec_ctx *exec_ctx,
+                                      glb_lb_policy *glb_policy);
+static void start_picking_locked(grpc_exec_ctx *exec_ctx,
+                                 glb_lb_policy *glb_policy) {
+  glb_policy->started_picking = true;
+  gpr_backoff_reset(&glb_policy->lb_call_backoff_state);
+  query_for_backends_locked(exec_ctx, glb_policy);
+}
+
+static void glb_exit_idle_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  if (!glb_policy->started_picking) {
+    start_picking_locked(exec_ctx, glb_policy);
+  }
+}
+
+static int glb_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                           const grpc_lb_policy_pick_args *pick_args,
+                           grpc_connected_subchannel **target, void **user_data,
+                           grpc_closure *on_complete) {
+  if (pick_args->lb_token_mdelem_storage == NULL) {
+    *target = NULL;
+    grpc_closure_sched(exec_ctx, on_complete,
+                       GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                           "No mdelem storage for the LB token. Load reporting "
+                           "won't work without it. Failing"));
+    return 0;
+  }
+
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  glb_policy->deadline = pick_args->deadline;
+  bool pick_done;
+
+  if (glb_policy->rr_policy != NULL) {
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO, "grpclb %p about to PICK from RR %p",
+              (void *)glb_policy, (void *)glb_policy->rr_policy);
+    }
+    GRPC_LB_POLICY_REF(glb_policy->rr_policy, "glb_pick");
+
+    wrapped_rr_closure_arg *wc_arg = gpr_zalloc(sizeof(wrapped_rr_closure_arg));
+
+    grpc_closure_init(&wc_arg->wrapper_closure, wrapped_rr_closure, wc_arg,
+                      grpc_schedule_on_exec_ctx);
+    wc_arg->rr_policy = glb_policy->rr_policy;
+    wc_arg->target = target;
+    wc_arg->wrapped_closure = on_complete;
+    wc_arg->lb_token_mdelem_storage = pick_args->lb_token_mdelem_storage;
+    wc_arg->initial_metadata = pick_args->initial_metadata;
+    wc_arg->free_when_done = wc_arg;
+    pick_done = pick_from_internal_rr_locked(exec_ctx, glb_policy->rr_policy,
+                                             pick_args, target, wc_arg);
+  } else {
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_DEBUG,
+              "No RR policy in grpclb instance %p. Adding to grpclb's pending "
+              "picks",
+              (void *)(glb_policy));
+    }
+    add_pending_pick(&glb_policy->pending_picks, pick_args, target,
+                     on_complete);
+
+    if (!glb_policy->started_picking) {
+      start_picking_locked(exec_ctx, glb_policy);
+    }
+    pick_done = false;
+  }
+  return pick_done;
+}
+
+static grpc_connectivity_state glb_check_connectivity_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+    grpc_error **connectivity_error) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  return grpc_connectivity_state_get(&glb_policy->state_tracker,
+                                     connectivity_error);
+}
+
+static void glb_ping_one_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                                grpc_closure *closure) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  if (glb_policy->rr_policy) {
+    grpc_lb_policy_ping_one_locked(exec_ctx, glb_policy->rr_policy, closure);
+  } else {
+    add_pending_ping(&glb_policy->pending_pings, closure);
+    if (!glb_policy->started_picking) {
+      start_picking_locked(exec_ctx, glb_policy);
+    }
+  }
+}
+
+static void glb_notify_on_state_change_locked(grpc_exec_ctx *exec_ctx,
+                                              grpc_lb_policy *pol,
+                                              grpc_connectivity_state *current,
+                                              grpc_closure *notify) {
+  glb_lb_policy *glb_policy = (glb_lb_policy *)pol;
+  grpc_connectivity_state_notify_on_state_change(
+      exec_ctx, &glb_policy->state_tracker, current, notify);
+}
+
+static void lb_on_server_status_received_locked(grpc_exec_ctx *exec_ctx,
+                                                void *arg, grpc_error *error);
+static void lb_on_response_received_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                           grpc_error *error);
+static void lb_call_init_locked(grpc_exec_ctx *exec_ctx,
+                                glb_lb_policy *glb_policy) {
+  GPR_ASSERT(glb_policy->server_name != NULL);
+  GPR_ASSERT(glb_policy->server_name[0] != '\0');
+  GPR_ASSERT(!glb_policy->shutting_down);
+
+  /* Note the following LB call progresses every time there's activity in \a
+   * glb_policy->base.interested_parties, which is comprised of the polling
+   * entities from \a client_channel. */
+  grpc_slice host = grpc_slice_from_copied_string(glb_policy->server_name);
+  glb_policy->lb_call = grpc_channel_create_pollset_set_call(
+      exec_ctx, glb_policy->lb_channel, NULL, GRPC_PROPAGATE_DEFAULTS,
+      glb_policy->base.interested_parties,
+      GRPC_MDSTR_SLASH_GRPC_DOT_LB_DOT_V1_DOT_LOADBALANCER_SLASH_BALANCELOAD,
+      &host, glb_policy->deadline, NULL);
+
+  grpc_metadata_array_init(&glb_policy->lb_initial_metadata_recv);
+  grpc_metadata_array_init(&glb_policy->lb_trailing_metadata_recv);
+
+  grpc_grpclb_request *request =
+      grpc_grpclb_request_create(glb_policy->server_name);
+  grpc_slice request_payload_slice = grpc_grpclb_request_encode(request);
+  glb_policy->lb_request_payload =
+      grpc_raw_byte_buffer_create(&request_payload_slice, 1);
+  grpc_slice_unref_internal(exec_ctx, request_payload_slice);
+  grpc_grpclb_request_destroy(request);
+
+  grpc_closure_init(&glb_policy->lb_on_server_status_received,
+                    lb_on_server_status_received_locked, glb_policy,
+                    grpc_combiner_scheduler(glb_policy->base.combiner, false));
+  grpc_closure_init(&glb_policy->lb_on_response_received,
+                    lb_on_response_received_locked, glb_policy,
+                    grpc_combiner_scheduler(glb_policy->base.combiner, false));
+
+  gpr_backoff_init(&glb_policy->lb_call_backoff_state,
+                   GRPC_GRPCLB_INITIAL_CONNECT_BACKOFF_SECONDS,
+                   GRPC_GRPCLB_RECONNECT_BACKOFF_MULTIPLIER,
+                   GRPC_GRPCLB_RECONNECT_JITTER,
+                   GRPC_GRPCLB_MIN_CONNECT_TIMEOUT_SECONDS * 1000,
+                   GRPC_GRPCLB_RECONNECT_MAX_BACKOFF_SECONDS * 1000);
+}
+
+static void lb_call_destroy_locked(grpc_exec_ctx *exec_ctx,
+                                   glb_lb_policy *glb_policy) {
+  GPR_ASSERT(glb_policy->lb_call != NULL);
+  grpc_call_destroy(glb_policy->lb_call);
+  glb_policy->lb_call = NULL;
+
+  grpc_metadata_array_destroy(&glb_policy->lb_initial_metadata_recv);
+  grpc_metadata_array_destroy(&glb_policy->lb_trailing_metadata_recv);
+
+  grpc_byte_buffer_destroy(glb_policy->lb_request_payload);
+  grpc_slice_unref_internal(exec_ctx, glb_policy->lb_call_status_details);
+}
+
+/*
+ * Auxiliary functions and LB client callbacks.
+ */
+static void query_for_backends_locked(grpc_exec_ctx *exec_ctx,
+                                      glb_lb_policy *glb_policy) {
+  GPR_ASSERT(glb_policy->lb_channel != NULL);
+  if (glb_policy->shutting_down) return;
+
+  lb_call_init_locked(exec_ctx, glb_policy);
+
+  if (grpc_lb_glb_trace) {
+    gpr_log(GPR_INFO, "Query for backends (grpclb: %p, lb_call: %p)",
+            (void *)glb_policy, (void *)glb_policy->lb_call);
+  }
+  GPR_ASSERT(glb_policy->lb_call != NULL);
+
+  grpc_call_error call_error;
+  grpc_op ops[4];
+  memset(ops, 0, sizeof(ops));
+
+  grpc_op *op = ops;
+  op->op = GRPC_OP_SEND_INITIAL_METADATA;
+  op->data.send_initial_metadata.count = 0;
+  op->flags = 0;
+  op->reserved = NULL;
+  op++;
+
+  op->op = GRPC_OP_RECV_INITIAL_METADATA;
+  op->data.recv_initial_metadata.recv_initial_metadata =
+      &glb_policy->lb_initial_metadata_recv;
+  op->flags = 0;
+  op->reserved = NULL;
+  op++;
+
+  GPR_ASSERT(glb_policy->lb_request_payload != NULL);
+  op->op = GRPC_OP_SEND_MESSAGE;
+  op->data.send_message.send_message = glb_policy->lb_request_payload;
+  op->flags = 0;
+  op->reserved = NULL;
+  op++;
+
+  op->op = GRPC_OP_RECV_STATUS_ON_CLIENT;
+  op->data.recv_status_on_client.trailing_metadata =
+      &glb_policy->lb_trailing_metadata_recv;
+  op->data.recv_status_on_client.status = &glb_policy->lb_call_status;
+  op->data.recv_status_on_client.status_details =
+      &glb_policy->lb_call_status_details;
+  op->flags = 0;
+  op->reserved = NULL;
+  op++;
+  /* take a weak ref (won't prevent calling of \a glb_shutdown if the strong ref
+   * count goes to zero) to be unref'd in lb_on_server_status_received */
+  GRPC_LB_POLICY_WEAK_REF(&glb_policy->base, "lb_on_server_status_received");
+  call_error = grpc_call_start_batch_and_execute(
+      exec_ctx, glb_policy->lb_call, ops, (size_t)(op - ops),
+      &glb_policy->lb_on_server_status_received);
+  GPR_ASSERT(GRPC_CALL_OK == call_error);
+
+  op = ops;
+  op->op = GRPC_OP_RECV_MESSAGE;
+  op->data.recv_message.recv_message = &glb_policy->lb_response_payload;
+  op->flags = 0;
+  op->reserved = NULL;
+  op++;
+  /* take another weak ref to be unref'd in lb_on_response_received */
+  GRPC_LB_POLICY_WEAK_REF(&glb_policy->base, "lb_on_response_received");
+  call_error = grpc_call_start_batch_and_execute(
+      exec_ctx, glb_policy->lb_call, ops, (size_t)(op - ops),
+      &glb_policy->lb_on_response_received);
+  GPR_ASSERT(GRPC_CALL_OK == call_error);
+}
+
+static void lb_on_response_received_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                           grpc_error *error) {
+  glb_lb_policy *glb_policy = arg;
+
+  grpc_op ops[2];
+  memset(ops, 0, sizeof(ops));
+  grpc_op *op = ops;
+  if (glb_policy->lb_response_payload != NULL) {
+    gpr_backoff_reset(&glb_policy->lb_call_backoff_state);
+    /* Received data from the LB server. Look inside
+     * glb_policy->lb_response_payload, for a serverlist. */
+    grpc_byte_buffer_reader bbr;
+    grpc_byte_buffer_reader_init(&bbr, glb_policy->lb_response_payload);
+    grpc_slice response_slice = grpc_byte_buffer_reader_readall(&bbr);
+    grpc_byte_buffer_destroy(glb_policy->lb_response_payload);
+    grpc_grpclb_serverlist *serverlist =
+        grpc_grpclb_response_parse_serverlist(response_slice);
+    if (serverlist != NULL) {
+      GPR_ASSERT(glb_policy->lb_call != NULL);
+      grpc_slice_unref_internal(exec_ctx, response_slice);
+      if (grpc_lb_glb_trace) {
+        gpr_log(GPR_INFO, "Serverlist with %lu servers received",
+                (unsigned long)serverlist->num_servers);
+        for (size_t i = 0; i < serverlist->num_servers; ++i) {
+          grpc_resolved_address addr;
+          parse_server(serverlist->servers[i], &addr);
+          char *ipport;
+          grpc_sockaddr_to_string(&ipport, &addr, false);
+          gpr_log(GPR_INFO, "Serverlist[%lu]: %s", (unsigned long)i, ipport);
+          gpr_free(ipport);
+        }
+      }
+
+      /* update serverlist */
+      if (serverlist->num_servers > 0) {
+        if (grpc_grpclb_serverlist_equals(glb_policy->serverlist, serverlist)) {
+          if (grpc_lb_glb_trace) {
+            gpr_log(GPR_INFO,
+                    "Incoming server list identical to current, ignoring.");
+          }
+          grpc_grpclb_destroy_serverlist(serverlist);
+        } else { /* new serverlist */
+          if (glb_policy->serverlist != NULL) {
+            /* dispose of the old serverlist */
+            grpc_grpclb_destroy_serverlist(glb_policy->serverlist);
+          }
+          /* and update the copy in the glb_lb_policy instance. This serverlist
+           * instance will be destroyed either upon the next update or in
+           * glb_destroy() */
+          glb_policy->serverlist = serverlist;
+
+          rr_handover_locked(exec_ctx, glb_policy);
+        }
+      } else {
+        if (grpc_lb_glb_trace) {
+          gpr_log(GPR_INFO,
+                  "Received empty server list. Picks will stay pending until a "
+                  "response with > 0 servers is received");
+        }
+      }
+    } else { /* serverlist == NULL */
+      gpr_log(GPR_ERROR, "Invalid LB response received: '%s'. Ignoring.",
+              grpc_dump_slice(response_slice, GPR_DUMP_ASCII | GPR_DUMP_HEX));
+      grpc_slice_unref_internal(exec_ctx, response_slice);
+    }
+
+    if (!glb_policy->shutting_down) {
+      /* keep listening for serverlist updates */
+      op->op = GRPC_OP_RECV_MESSAGE;
+      op->data.recv_message.recv_message = &glb_policy->lb_response_payload;
+      op->flags = 0;
+      op->reserved = NULL;
+      op++;
+      /* reuse the "lb_on_response_received" weak ref taken in
+       * query_for_backends_locked() */
+      const grpc_call_error call_error = grpc_call_start_batch_and_execute(
+          exec_ctx, glb_policy->lb_call, ops, (size_t)(op - ops),
+          &glb_policy->lb_on_response_received); /* loop */
+      GPR_ASSERT(GRPC_CALL_OK == call_error);
+    }
+  } else { /* empty payload: call cancelled. */
+           /* dispose of the "lb_on_response_received" weak ref taken in
+            * query_for_backends_locked() and reused in every reception loop */
+    GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &glb_policy->base,
+                              "lb_on_response_received_empty_payload");
+  }
+}
+
+static void lb_call_on_retry_timer_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                          grpc_error *error) {
+  glb_lb_policy *glb_policy = arg;
+
+  if (!glb_policy->shutting_down) {
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_INFO, "Restaring call to LB server (grpclb %p)",
+              (void *)glb_policy);
+    }
+    GPR_ASSERT(glb_policy->lb_call == NULL);
+    query_for_backends_locked(exec_ctx, glb_policy);
+  }
+  GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &glb_policy->base,
+                            "grpclb_on_retry_timer");
+}
+
+static void lb_on_server_status_received_locked(grpc_exec_ctx *exec_ctx,
+                                                void *arg, grpc_error *error) {
+  glb_lb_policy *glb_policy = arg;
+
+  GPR_ASSERT(glb_policy->lb_call != NULL);
+
+  if (grpc_lb_glb_trace) {
+    char *status_details =
+        grpc_slice_to_c_string(glb_policy->lb_call_status_details);
+    gpr_log(GPR_DEBUG,
+            "Status from LB server received. Status = %d, Details = '%s', "
+            "(call: %p)",
+            glb_policy->lb_call_status, status_details,
+            (void *)glb_policy->lb_call);
+    gpr_free(status_details);
+  }
+
+  /* We need to perform cleanups no matter what. */
+  lb_call_destroy_locked(exec_ctx, glb_policy);
+
+  if (!glb_policy->shutting_down) {
+    /* if we aren't shutting down, restart the LB client call after some time */
+    gpr_timespec now = gpr_now(GPR_CLOCK_MONOTONIC);
+    gpr_timespec next_try =
+        gpr_backoff_step(&glb_policy->lb_call_backoff_state, now);
+    if (grpc_lb_glb_trace) {
+      gpr_log(GPR_DEBUG, "Connection to LB server lost (grpclb: %p)...",
+              (void *)glb_policy);
+      gpr_timespec timeout = gpr_time_sub(next_try, now);
+      if (gpr_time_cmp(timeout, gpr_time_0(timeout.clock_type)) > 0) {
+        gpr_log(GPR_DEBUG, "... retrying in %" PRId64 ".%09d seconds.",
+                timeout.tv_sec, timeout.tv_nsec);
+      } else {
+        gpr_log(GPR_DEBUG, "... retrying immediately.");
+      }
+    }
+    GRPC_LB_POLICY_WEAK_REF(&glb_policy->base, "grpclb_retry_timer");
+    grpc_closure_init(
+        &glb_policy->lb_on_call_retry, lb_call_on_retry_timer_locked,
+        glb_policy, grpc_combiner_scheduler(glb_policy->base.combiner, false));
+    grpc_timer_init(exec_ctx, &glb_policy->lb_call_retry_timer, next_try,
+                    &glb_policy->lb_on_call_retry, now);
+  }
+  GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &glb_policy->base,
+                            "lb_on_server_status_received");
+}
+
+/* Code wiring the policy with the rest of the core */
+static const grpc_lb_policy_vtable glb_lb_policy_vtable = {
+    glb_destroy,
+    glb_shutdown_locked,
+    glb_pick_locked,
+    glb_cancel_pick_locked,
+    glb_cancel_picks_locked,
+    glb_ping_one_locked,
+    glb_exit_idle_locked,
+    glb_check_connectivity_locked,
+    glb_notify_on_state_change_locked};
+
+static void glb_factory_ref(grpc_lb_policy_factory *factory) {}
+
+static void glb_factory_unref(grpc_lb_policy_factory *factory) {}
+
+static const grpc_lb_policy_factory_vtable glb_factory_vtable = {
+    glb_factory_ref, glb_factory_unref, glb_create, "grpclb"};
+
+static grpc_lb_policy_factory glb_lb_policy_factory = {&glb_factory_vtable};
+
+grpc_lb_policy_factory *grpc_glb_lb_factory_create() {
+  return &glb_lb_policy_factory;
+}
+
+/* Plugin registration */
+void grpc_lb_policy_grpclb_init() {
+  grpc_register_lb_policy(grpc_glb_lb_factory_create());
+  grpc_register_tracer("glb", &grpc_lb_glb_trace);
+}
+
+void grpc_lb_policy_grpclb_shutdown() {}
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.h b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.h
new file mode 100644
index 0000000..7bd6b1c
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.h
@@ -0,0 +1,44 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_LB_POLICY_GRPCLB_GRPCLB_H
+#define GRPC_CORE_EXT_LB_POLICY_GRPCLB_GRPCLB_H
+
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+
+/** Returns a load balancing factory for the glb policy, which tries to connect
+ * to a load balancing server to decide the next successfully connected
+ * subchannel to pick. */
+grpc_lb_policy_factory *grpc_glb_lb_factory_create();
+
+#endif /* GRPC_CORE_EXT_LB_POLICY_GRPCLB_GRPCLB_H */
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.c b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.c
new file mode 100644
index 0000000..d6201f2
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.c
@@ -0,0 +1,77 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/support/string.h"
+
+grpc_channel *grpc_lb_policy_grpclb_create_lb_channel(
+    grpc_exec_ctx *exec_ctx, const char *lb_service_target_addresses,
+    grpc_client_channel_factory *client_channel_factory,
+    grpc_channel_args *args) {
+  grpc_channel *lb_channel = grpc_client_channel_factory_create_channel(
+      exec_ctx, client_channel_factory, lb_service_target_addresses,
+      GRPC_CLIENT_CHANNEL_TYPE_LOAD_BALANCING, args);
+  return lb_channel;
+}
+
+grpc_channel_args *get_lb_channel_args(grpc_exec_ctx *exec_ctx,
+                                       grpc_slice_hash_table *targets_info,
+                                       const grpc_channel_args *args) {
+  /* We strip out the channel arg for the LB policy name, since we want
+   * to use the default (pick_first) in this case.
+   *
+   * We also strip out the channel arg for the resolved addresses, since
+   * that will be generated by the name resolver used in the LB channel.
+   * Note that the LB channel will use the sockaddr resolver, so this
+   * won't actually generate a query to DNS (or some other name service).
+   * However, the addresses returned by the sockaddr resolver will have
+   * is_balancer=false, whereas our own addresses have is_balancer=true.
+   * We need the LB channel to return addresses with is_balancer=false
+   * so that it does not wind up recursively using the grpclb LB policy,
+   * as per the special case logic in client_channel.c.
+   *
+   * Lastly, we also strip out the channel arg for the server URI,
+   * since that will be different for the LB channel than for the parent
+   * channel (the client channel factory will re-add this arg with
+   * the right value). */
+  static const char *keys_to_remove[] = {
+      GRPC_ARG_LB_POLICY_NAME, GRPC_ARG_LB_ADDRESSES, GRPC_ARG_SERVER_URI};
+  return grpc_channel_args_copy_and_remove(args, keys_to_remove,
+                                           GPR_ARRAY_SIZE(keys_to_remove));
+}
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.h b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.h
new file mode 100644
index 0000000..b954299
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.h
@@ -0,0 +1,56 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_LB_POLICY_GRPCLB_GRPCLB_CHANNEL_H
+#define GRPC_CORE_EXT_LB_POLICY_GRPCLB_GRPCLB_CHANNEL_H
+
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+#include "src/core/lib/slice/slice_hash_table.h"
+
+/** Create the channel used for communicating with an LB service.
+ * Note that an LB *service* may be comprised of several LB *servers*.
+ *
+ * \a lb_service_target_addresses is the target URI containing the addresses
+ * from resolving the LB service's name (eg, ipv4:10.0.0.1:1234,10.2.3.4:9876).
+ * \a client_channel_factory will be used for the creation of the LB channel,
+ * alongside the channel args passed in \a args. */
+grpc_channel *grpc_lb_policy_grpclb_create_lb_channel(
+    grpc_exec_ctx *exec_ctx, const char *lb_service_target_addresses,
+    grpc_client_channel_factory *client_channel_factory,
+    grpc_channel_args *args);
+
+grpc_channel_args *get_lb_channel_args(grpc_exec_ctx *exec_ctx,
+                                       grpc_slice_hash_table *targets_info,
+                                       const grpc_channel_args *args);
+
+#endif /* GRPC_CORE_EXT_LB_POLICY_GRPCLB_GRPCLB_CHANNEL_H */
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel_secure.c b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel_secure.c
new file mode 100644
index 0000000..a145cba
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel_secure.c
@@ -0,0 +1,107 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/security/credentials/credentials.h"
+#include "src/core/lib/security/transport/lb_targets_info.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/support/string.h"
+
+grpc_channel *grpc_lb_policy_grpclb_create_lb_channel(
+    grpc_exec_ctx *exec_ctx, const char *lb_service_target_addresses,
+    grpc_client_channel_factory *client_channel_factory,
+    grpc_channel_args *args) {
+  grpc_channel_args *new_args = args;
+  grpc_channel_credentials *channel_credentials =
+      grpc_channel_credentials_find_in_args(args);
+  if (channel_credentials != NULL) {
+    /* Substitute the channel credentials with a version without call
+     * credentials: the load balancer is not necessarily trusted to handle
+     * bearer token credentials */
+    static const char *keys_to_remove[] = {GRPC_ARG_CHANNEL_CREDENTIALS};
+    grpc_channel_credentials *creds_sans_call_creds =
+        grpc_channel_credentials_duplicate_without_call_credentials(
+            channel_credentials);
+    GPR_ASSERT(creds_sans_call_creds != NULL);
+    grpc_arg args_to_add[] = {
+        grpc_channel_credentials_to_arg(creds_sans_call_creds)};
+    /* Create the new set of channel args */
+    new_args = grpc_channel_args_copy_and_add_and_remove(
+        args, keys_to_remove, GPR_ARRAY_SIZE(keys_to_remove), args_to_add,
+        GPR_ARRAY_SIZE(args_to_add));
+    grpc_channel_credentials_unref(exec_ctx, creds_sans_call_creds);
+  }
+  grpc_channel *lb_channel = grpc_client_channel_factory_create_channel(
+      exec_ctx, client_channel_factory, lb_service_target_addresses,
+      GRPC_CLIENT_CHANNEL_TYPE_LOAD_BALANCING, new_args);
+  if (channel_credentials != NULL) {
+    grpc_channel_args_destroy(exec_ctx, new_args);
+  }
+  return lb_channel;
+}
+
+grpc_channel_args *get_lb_channel_args(grpc_exec_ctx *exec_ctx,
+                                       grpc_slice_hash_table *targets_info,
+                                       const grpc_channel_args *args) {
+  const grpc_arg targets_info_arg =
+      grpc_lb_targets_info_create_channel_arg(targets_info);
+  /* We strip out the channel arg for the LB policy name, since we want
+   * to use the default (pick_first) in this case.
+   *
+   * We also strip out the channel arg for the resolved addresses, since
+   * that will be generated by the name resolver used in the LB channel.
+   * Note that the LB channel will use the sockaddr resolver, so this
+   * won't actually generate a query to DNS (or some other name service).
+   * However, the addresses returned by the sockaddr resolver will have
+   * is_balancer=false, whereas our own addresses have is_balancer=true.
+   * We need the LB channel to return addresses with is_balancer=false
+   * so that it does not wind up recursively using the grpclb LB policy,
+   * as per the special case logic in client_channel.c.
+   *
+   * Lastly, we also strip out the channel arg for the server URI,
+   * since that will be different for the LB channel than for the parent
+   * channel (the client channel factory will re-add this arg with
+   * the right value). */
+  static const char *keys_to_remove[] = {
+      GRPC_ARG_LB_POLICY_NAME, GRPC_ARG_LB_ADDRESSES, GRPC_ARG_SERVER_URI};
+  /* Add the targets info table to be used for secure naming */
+  return grpc_channel_args_copy_and_add_and_remove(
+      args, keys_to_remove, GPR_ARRAY_SIZE(keys_to_remove), &targets_info_arg,
+      1);
+}
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.c b/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.c
new file mode 100644
index 0000000..10af252
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.c
@@ -0,0 +1,249 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.h"
+#include "third_party/nanopb/pb_decode.h"
+#include "third_party/nanopb/pb_encode.h"
+
+#include <grpc/support/alloc.h>
+
+typedef struct decode_serverlist_arg {
+  /* The first pass counts the number of servers in the server list. The second
+   * one allocates and decodes. */
+  bool first_pass;
+  /* The decoding callback is invoked once per server in serverlist. Remember
+   * which index of the serverlist are we currently decoding */
+  size_t decoding_idx;
+  /* Populated after the first pass. Number of server in the input serverlist */
+  size_t num_servers;
+  /* The decoded serverlist */
+  grpc_grpclb_server **servers;
+} decode_serverlist_arg;
+
+/* invoked once for every Server in ServerList */
+static bool decode_serverlist(pb_istream_t *stream, const pb_field_t *field,
+                              void **arg) {
+  decode_serverlist_arg *dec_arg = *arg;
+  if (dec_arg->first_pass) { /* count how many server do we have */
+    grpc_grpclb_server server;
+    if (!pb_decode(stream, grpc_lb_v1_Server_fields, &server)) {
+      gpr_log(GPR_ERROR, "nanopb error: %s", PB_GET_ERROR(stream));
+      return false;
+    }
+    dec_arg->num_servers++;
+  } else { /* second pass. Actually decode. */
+    grpc_grpclb_server *server = gpr_zalloc(sizeof(grpc_grpclb_server));
+    GPR_ASSERT(dec_arg->num_servers > 0);
+    if (dec_arg->decoding_idx == 0) { /* first iteration of second pass */
+      dec_arg->servers =
+          gpr_malloc(sizeof(grpc_grpclb_server *) * dec_arg->num_servers);
+    }
+    if (!pb_decode(stream, grpc_lb_v1_Server_fields, server)) {
+      gpr_log(GPR_ERROR, "nanopb error: %s", PB_GET_ERROR(stream));
+      return false;
+    }
+    dec_arg->servers[dec_arg->decoding_idx++] = server;
+  }
+
+  return true;
+}
+
+grpc_grpclb_request *grpc_grpclb_request_create(const char *lb_service_name) {
+  grpc_grpclb_request *req = gpr_malloc(sizeof(grpc_grpclb_request));
+
+  req->has_client_stats = 0; /* TODO(dgq): add support for stats once defined */
+  req->has_initial_request = 1;
+  req->initial_request.has_name = 1;
+  strncpy(req->initial_request.name, lb_service_name,
+          GRPC_GRPCLB_SERVICE_NAME_MAX_LENGTH);
+  return req;
+}
+
+grpc_slice grpc_grpclb_request_encode(const grpc_grpclb_request *request) {
+  size_t encoded_length;
+  pb_ostream_t sizestream;
+  pb_ostream_t outputstream;
+  grpc_slice slice;
+  memset(&sizestream, 0, sizeof(pb_ostream_t));
+  pb_encode(&sizestream, grpc_lb_v1_LoadBalanceRequest_fields, request);
+  encoded_length = sizestream.bytes_written;
+
+  slice = grpc_slice_malloc(encoded_length);
+  outputstream =
+      pb_ostream_from_buffer(GRPC_SLICE_START_PTR(slice), encoded_length);
+  GPR_ASSERT(pb_encode(&outputstream, grpc_lb_v1_LoadBalanceRequest_fields,
+                       request) != 0);
+  return slice;
+}
+
+void grpc_grpclb_request_destroy(grpc_grpclb_request *request) {
+  gpr_free(request);
+}
+
+typedef grpc_lb_v1_LoadBalanceResponse grpc_grpclb_response;
+grpc_grpclb_initial_response *grpc_grpclb_initial_response_parse(
+    grpc_slice encoded_grpc_grpclb_response) {
+  pb_istream_t stream =
+      pb_istream_from_buffer(GRPC_SLICE_START_PTR(encoded_grpc_grpclb_response),
+                             GRPC_SLICE_LENGTH(encoded_grpc_grpclb_response));
+  grpc_grpclb_response res;
+  memset(&res, 0, sizeof(grpc_grpclb_response));
+  if (!pb_decode(&stream, grpc_lb_v1_LoadBalanceResponse_fields, &res)) {
+    gpr_log(GPR_ERROR, "nanopb error: %s", PB_GET_ERROR(&stream));
+    return NULL;
+  }
+  grpc_grpclb_initial_response *initial_res =
+      gpr_malloc(sizeof(grpc_grpclb_initial_response));
+  memcpy(initial_res, &res.initial_response,
+         sizeof(grpc_grpclb_initial_response));
+
+  return initial_res;
+}
+
+grpc_grpclb_serverlist *grpc_grpclb_response_parse_serverlist(
+    grpc_slice encoded_grpc_grpclb_response) {
+  bool status;
+  decode_serverlist_arg arg;
+  pb_istream_t stream =
+      pb_istream_from_buffer(GRPC_SLICE_START_PTR(encoded_grpc_grpclb_response),
+                             GRPC_SLICE_LENGTH(encoded_grpc_grpclb_response));
+  pb_istream_t stream_at_start = stream;
+  grpc_grpclb_response res;
+  memset(&res, 0, sizeof(grpc_grpclb_response));
+  memset(&arg, 0, sizeof(decode_serverlist_arg));
+
+  res.server_list.servers.funcs.decode = decode_serverlist;
+  res.server_list.servers.arg = &arg;
+  arg.first_pass = true;
+  status = pb_decode(&stream, grpc_lb_v1_LoadBalanceResponse_fields, &res);
+  if (!status) {
+    gpr_log(GPR_ERROR, "nanopb error: %s", PB_GET_ERROR(&stream));
+    return NULL;
+  }
+
+  arg.first_pass = false;
+  status =
+      pb_decode(&stream_at_start, grpc_lb_v1_LoadBalanceResponse_fields, &res);
+  if (!status) {
+    gpr_log(GPR_ERROR, "nanopb error: %s", PB_GET_ERROR(&stream));
+    return NULL;
+  }
+
+  grpc_grpclb_serverlist *sl = gpr_zalloc(sizeof(grpc_grpclb_serverlist));
+  sl->num_servers = arg.num_servers;
+  sl->servers = arg.servers;
+  if (res.server_list.has_expiration_interval) {
+    sl->expiration_interval = res.server_list.expiration_interval;
+  }
+  return sl;
+}
+
+void grpc_grpclb_destroy_serverlist(grpc_grpclb_serverlist *serverlist) {
+  if (serverlist == NULL) {
+    return;
+  }
+  for (size_t i = 0; i < serverlist->num_servers; i++) {
+    gpr_free(serverlist->servers[i]);
+  }
+  gpr_free(serverlist->servers);
+  gpr_free(serverlist);
+}
+
+grpc_grpclb_serverlist *grpc_grpclb_serverlist_copy(
+    const grpc_grpclb_serverlist *sl) {
+  grpc_grpclb_serverlist *copy = gpr_zalloc(sizeof(grpc_grpclb_serverlist));
+  copy->num_servers = sl->num_servers;
+  memcpy(&copy->expiration_interval, &sl->expiration_interval,
+         sizeof(grpc_grpclb_duration));
+  copy->servers = gpr_malloc(sizeof(grpc_grpclb_server *) * sl->num_servers);
+  for (size_t i = 0; i < sl->num_servers; i++) {
+    copy->servers[i] = gpr_malloc(sizeof(grpc_grpclb_server));
+    memcpy(copy->servers[i], sl->servers[i], sizeof(grpc_grpclb_server));
+  }
+  return copy;
+}
+
+bool grpc_grpclb_serverlist_equals(const grpc_grpclb_serverlist *lhs,
+                                   const grpc_grpclb_serverlist *rhs) {
+  if ((lhs == NULL) || (rhs == NULL)) {
+    return false;
+  }
+  if (lhs->num_servers != rhs->num_servers) {
+    return false;
+  }
+  if (grpc_grpclb_duration_compare(&lhs->expiration_interval,
+                                   &rhs->expiration_interval) != 0) {
+    return false;
+  }
+  for (size_t i = 0; i < lhs->num_servers; i++) {
+    if (!grpc_grpclb_server_equals(lhs->servers[i], rhs->servers[i])) {
+      return false;
+    }
+  }
+  return true;
+}
+
+bool grpc_grpclb_server_equals(const grpc_grpclb_server *lhs,
+                               const grpc_grpclb_server *rhs) {
+  return memcmp(lhs, rhs, sizeof(grpc_grpclb_server)) == 0;
+}
+
+int grpc_grpclb_duration_compare(const grpc_grpclb_duration *lhs,
+                                 const grpc_grpclb_duration *rhs) {
+  GPR_ASSERT(lhs && rhs);
+  if (lhs->has_seconds && rhs->has_seconds) {
+    if (lhs->seconds < rhs->seconds) return -1;
+    if (lhs->seconds > rhs->seconds) return 1;
+  } else if (lhs->has_seconds) {
+    return 1;
+  } else if (rhs->has_seconds) {
+    return -1;
+  }
+
+  GPR_ASSERT(lhs->seconds == rhs->seconds);
+  if (lhs->has_nanos && rhs->has_nanos) {
+    if (lhs->nanos < rhs->nanos) return -1;
+    if (lhs->nanos > rhs->nanos) return 1;
+  } else if (lhs->has_nanos) {
+    return 1;
+  } else if (rhs->has_nanos) {
+    return -1;
+  }
+
+  return 0;
+}
+
+void grpc_grpclb_initial_response_destroy(
+    grpc_grpclb_initial_response *response) {
+  gpr_free(response);
+}
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.h b/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.h
new file mode 100644
index 0000000..9702e65
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.h
@@ -0,0 +1,104 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_LB_POLICY_GRPCLB_LOAD_BALANCER_API_H
+#define GRPC_CORE_EXT_LB_POLICY_GRPCLB_LOAD_BALANCER_API_H
+
+#include <grpc/slice_buffer.h>
+
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#define GRPC_GRPCLB_SERVICE_NAME_MAX_LENGTH 128
+
+typedef grpc_lb_v1_Server_ip_address_t grpc_grpclb_ip_address;
+typedef grpc_lb_v1_LoadBalanceRequest grpc_grpclb_request;
+typedef grpc_lb_v1_InitialLoadBalanceResponse grpc_grpclb_initial_response;
+typedef grpc_lb_v1_Server grpc_grpclb_server;
+typedef grpc_lb_v1_Duration grpc_grpclb_duration;
+typedef struct grpc_grpclb_serverlist {
+  grpc_grpclb_server **servers;
+  size_t num_servers;
+  grpc_grpclb_duration expiration_interval;
+} grpc_grpclb_serverlist;
+
+/** Create a request for a gRPC LB service under \a lb_service_name */
+grpc_grpclb_request *grpc_grpclb_request_create(const char *lb_service_name);
+
+/** Protocol Buffers v3-encode \a request */
+grpc_slice grpc_grpclb_request_encode(const grpc_grpclb_request *request);
+
+/** Destroy \a request */
+void grpc_grpclb_request_destroy(grpc_grpclb_request *request);
+
+/** Parse (ie, decode) the bytes in \a encoded_grpc_grpclb_response as a \a
+ * grpc_grpclb_initial_response */
+grpc_grpclb_initial_response *grpc_grpclb_initial_response_parse(
+    grpc_slice encoded_grpc_grpclb_response);
+
+/** Parse the list of servers from an encoded \a grpc_grpclb_response */
+grpc_grpclb_serverlist *grpc_grpclb_response_parse_serverlist(
+    grpc_slice encoded_grpc_grpclb_response);
+
+/** Return a copy of \a sl. The caller is responsible for calling \a
+ * grpc_grpclb_destroy_serverlist on the returned copy. */
+grpc_grpclb_serverlist *grpc_grpclb_serverlist_copy(
+    const grpc_grpclb_serverlist *sl);
+
+bool grpc_grpclb_serverlist_equals(const grpc_grpclb_serverlist *lhs,
+                                   const grpc_grpclb_serverlist *rhs);
+
+bool grpc_grpclb_server_equals(const grpc_grpclb_server *lhs,
+                               const grpc_grpclb_server *rhs);
+
+/** Destroy \a serverlist */
+void grpc_grpclb_destroy_serverlist(grpc_grpclb_serverlist *serverlist);
+
+/** Compare \a lhs against \a rhs and return 0 if \a lhs and \a rhs are equal,
+ * < 0 if \a lhs represents a duration shorter than \a rhs and > 0 otherwise */
+int grpc_grpclb_duration_compare(const grpc_grpclb_duration *lhs,
+                                 const grpc_grpclb_duration *rhs);
+
+/** Destroy \a initial_response */
+void grpc_grpclb_initial_response_destroy(
+    grpc_grpclb_initial_response *response);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* GRPC_CORE_EXT_LB_POLICY_GRPCLB_LOAD_BALANCER_API_H */
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.c b/src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.c
new file mode 100644
index 0000000..e9adf98
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.c
@@ -0,0 +1,88 @@
+/* Automatically generated nanopb constant definitions */
+/* Generated by nanopb-0.3.7-dev */
+
+#include "src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.h"
+
+/* @@protoc_insertion_point(includes) */
+#if PB_PROTO_HEADER_VERSION != 30
+#error Regenerate this file with the current version of nanopb generator.
+#endif
+
+
+
+const pb_field_t grpc_lb_v1_Duration_fields[3] = {
+    PB_FIELD(  1, INT64   , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_Duration, seconds, seconds, 0),
+    PB_FIELD(  2, INT32   , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_Duration, nanos, seconds, 0),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_LoadBalanceRequest_fields[3] = {
+    PB_FIELD(  1, MESSAGE , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_LoadBalanceRequest, initial_request, initial_request, &grpc_lb_v1_InitialLoadBalanceRequest_fields),
+    PB_FIELD(  2, MESSAGE , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_LoadBalanceRequest, client_stats, initial_request, &grpc_lb_v1_ClientStats_fields),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_InitialLoadBalanceRequest_fields[2] = {
+    PB_FIELD(  1, STRING  , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_InitialLoadBalanceRequest, name, name, 0),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_ClientStats_fields[4] = {
+    PB_FIELD(  1, INT64   , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_ClientStats, total_requests, total_requests, 0),
+    PB_FIELD(  2, INT64   , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_ClientStats, client_rpc_errors, total_requests, 0),
+    PB_FIELD(  3, INT64   , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_ClientStats, dropped_requests, client_rpc_errors, 0),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_LoadBalanceResponse_fields[3] = {
+    PB_FIELD(  1, MESSAGE , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_LoadBalanceResponse, initial_response, initial_response, &grpc_lb_v1_InitialLoadBalanceResponse_fields),
+    PB_FIELD(  2, MESSAGE , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_LoadBalanceResponse, server_list, initial_response, &grpc_lb_v1_ServerList_fields),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_InitialLoadBalanceResponse_fields[3] = {
+    PB_FIELD(  1, STRING  , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_InitialLoadBalanceResponse, load_balancer_delegate, load_balancer_delegate, 0),
+    PB_FIELD(  2, MESSAGE , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_InitialLoadBalanceResponse, client_stats_report_interval, load_balancer_delegate, &grpc_lb_v1_Duration_fields),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_ServerList_fields[3] = {
+    PB_FIELD(  1, MESSAGE , REPEATED, CALLBACK, FIRST, grpc_lb_v1_ServerList, servers, servers, &grpc_lb_v1_Server_fields),
+    PB_FIELD(  3, MESSAGE , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_ServerList, expiration_interval, servers, &grpc_lb_v1_Duration_fields),
+    PB_LAST_FIELD
+};
+
+const pb_field_t grpc_lb_v1_Server_fields[5] = {
+    PB_FIELD(  1, BYTES   , OPTIONAL, STATIC  , FIRST, grpc_lb_v1_Server, ip_address, ip_address, 0),
+    PB_FIELD(  2, INT32   , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_Server, port, ip_address, 0),
+    PB_FIELD(  3, STRING  , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_Server, load_balance_token, port, 0),
+    PB_FIELD(  4, BOOL    , OPTIONAL, STATIC  , OTHER, grpc_lb_v1_Server, drop_request, load_balance_token, 0),
+    PB_LAST_FIELD
+};
+
+
+/* Check that field information fits in pb_field_t */
+#if !defined(PB_FIELD_32BIT)
+/* If you get an error here, it means that you need to define PB_FIELD_32BIT
+ * compile-time option. You can do that in pb.h or on compiler command line.
+ * 
+ * The reason you need to do this is that some of your messages contain tag
+ * numbers or field sizes that are larger than what can fit in 8 or 16 bit
+ * field descriptors.
+ */
+PB_STATIC_ASSERT((pb_membersize(grpc_lb_v1_LoadBalanceRequest, initial_request) < 65536 && pb_membersize(grpc_lb_v1_LoadBalanceRequest, client_stats) < 65536 && pb_membersize(grpc_lb_v1_LoadBalanceResponse, initial_response) < 65536 && pb_membersize(grpc_lb_v1_LoadBalanceResponse, server_list) < 65536 && pb_membersize(grpc_lb_v1_InitialLoadBalanceResponse, client_stats_report_interval) < 65536 && pb_membersize(grpc_lb_v1_ServerList, servers) < 65536 && pb_membersize(grpc_lb_v1_ServerList, expiration_interval) < 65536), YOU_MUST_DEFINE_PB_FIELD_32BIT_FOR_MESSAGES_grpc_lb_v1_Duration_grpc_lb_v1_LoadBalanceRequest_grpc_lb_v1_InitialLoadBalanceRequest_grpc_lb_v1_ClientStats_grpc_lb_v1_LoadBalanceResponse_grpc_lb_v1_InitialLoadBalanceResponse_grpc_lb_v1_ServerList_grpc_lb_v1_Server)
+#endif
+
+#if !defined(PB_FIELD_16BIT) && !defined(PB_FIELD_32BIT)
+/* If you get an error here, it means that you need to define PB_FIELD_16BIT
+ * compile-time option. You can do that in pb.h or on compiler command line.
+ * 
+ * The reason you need to do this is that some of your messages contain tag
+ * numbers or field sizes that are larger than what can fit in the default
+ * 8 bit descriptors.
+ */
+PB_STATIC_ASSERT((pb_membersize(grpc_lb_v1_LoadBalanceRequest, initial_request) < 256 && pb_membersize(grpc_lb_v1_LoadBalanceRequest, client_stats) < 256 && pb_membersize(grpc_lb_v1_LoadBalanceResponse, initial_response) < 256 && pb_membersize(grpc_lb_v1_LoadBalanceResponse, server_list) < 256 && pb_membersize(grpc_lb_v1_InitialLoadBalanceResponse, client_stats_report_interval) < 256 && pb_membersize(grpc_lb_v1_ServerList, servers) < 256 && pb_membersize(grpc_lb_v1_ServerList, expiration_interval) < 256), YOU_MUST_DEFINE_PB_FIELD_16BIT_FOR_MESSAGES_grpc_lb_v1_Duration_grpc_lb_v1_LoadBalanceRequest_grpc_lb_v1_InitialLoadBalanceRequest_grpc_lb_v1_ClientStats_grpc_lb_v1_LoadBalanceResponse_grpc_lb_v1_InitialLoadBalanceResponse_grpc_lb_v1_ServerList_grpc_lb_v1_Server)
+#endif
+
+
+/* @@protoc_insertion_point(eof) */
diff --git a/src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.h b/src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.h
new file mode 100644
index 0000000..725aa7e
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/grpclb/proto/grpc/lb/v1/load_balancer.pb.h
@@ -0,0 +1,158 @@
+/* Automatically generated nanopb header */
+/* Generated by nanopb-0.3.7-dev */
+
+#ifndef PB_GRPC_LB_V1_LOAD_BALANCER_PB_H_INCLUDED
+#define PB_GRPC_LB_V1_LOAD_BALANCER_PB_H_INCLUDED
+#include "third_party/nanopb/pb.h"
+/* @@protoc_insertion_point(includes) */
+#if PB_PROTO_HEADER_VERSION != 30
+#error Regenerate this file with the current version of nanopb generator.
+#endif
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/* Struct definitions */
+typedef struct _grpc_lb_v1_ClientStats {
+    bool has_total_requests;
+    int64_t total_requests;
+    bool has_client_rpc_errors;
+    int64_t client_rpc_errors;
+    bool has_dropped_requests;
+    int64_t dropped_requests;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_ClientStats) */
+} grpc_lb_v1_ClientStats;
+
+typedef struct _grpc_lb_v1_Duration {
+    bool has_seconds;
+    int64_t seconds;
+    bool has_nanos;
+    int32_t nanos;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_Duration) */
+} grpc_lb_v1_Duration;
+
+typedef struct _grpc_lb_v1_InitialLoadBalanceRequest {
+    bool has_name;
+    char name[128];
+/* @@protoc_insertion_point(struct:grpc_lb_v1_InitialLoadBalanceRequest) */
+} grpc_lb_v1_InitialLoadBalanceRequest;
+
+typedef PB_BYTES_ARRAY_T(16) grpc_lb_v1_Server_ip_address_t;
+typedef struct _grpc_lb_v1_Server {
+    bool has_ip_address;
+    grpc_lb_v1_Server_ip_address_t ip_address;
+    bool has_port;
+    int32_t port;
+    bool has_load_balance_token;
+    char load_balance_token[50];
+    bool has_drop_request;
+    bool drop_request;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_Server) */
+} grpc_lb_v1_Server;
+
+typedef struct _grpc_lb_v1_InitialLoadBalanceResponse {
+    bool has_load_balancer_delegate;
+    char load_balancer_delegate[64];
+    bool has_client_stats_report_interval;
+    grpc_lb_v1_Duration client_stats_report_interval;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_InitialLoadBalanceResponse) */
+} grpc_lb_v1_InitialLoadBalanceResponse;
+
+typedef struct _grpc_lb_v1_LoadBalanceRequest {
+    bool has_initial_request;
+    grpc_lb_v1_InitialLoadBalanceRequest initial_request;
+    bool has_client_stats;
+    grpc_lb_v1_ClientStats client_stats;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_LoadBalanceRequest) */
+} grpc_lb_v1_LoadBalanceRequest;
+
+typedef struct _grpc_lb_v1_ServerList {
+    pb_callback_t servers;
+    bool has_expiration_interval;
+    grpc_lb_v1_Duration expiration_interval;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_ServerList) */
+} grpc_lb_v1_ServerList;
+
+typedef struct _grpc_lb_v1_LoadBalanceResponse {
+    bool has_initial_response;
+    grpc_lb_v1_InitialLoadBalanceResponse initial_response;
+    bool has_server_list;
+    grpc_lb_v1_ServerList server_list;
+/* @@protoc_insertion_point(struct:grpc_lb_v1_LoadBalanceResponse) */
+} grpc_lb_v1_LoadBalanceResponse;
+
+/* Default values for struct fields */
+
+/* Initializer values for message structs */
+#define grpc_lb_v1_Duration_init_default         {false, 0, false, 0}
+#define grpc_lb_v1_LoadBalanceRequest_init_default {false, grpc_lb_v1_InitialLoadBalanceRequest_init_default, false, grpc_lb_v1_ClientStats_init_default}
+#define grpc_lb_v1_InitialLoadBalanceRequest_init_default {false, ""}
+#define grpc_lb_v1_ClientStats_init_default      {false, 0, false, 0, false, 0}
+#define grpc_lb_v1_LoadBalanceResponse_init_default {false, grpc_lb_v1_InitialLoadBalanceResponse_init_default, false, grpc_lb_v1_ServerList_init_default}
+#define grpc_lb_v1_InitialLoadBalanceResponse_init_default {false, "", false, grpc_lb_v1_Duration_init_default}
+#define grpc_lb_v1_ServerList_init_default       {{{NULL}, NULL}, false, grpc_lb_v1_Duration_init_default}
+#define grpc_lb_v1_Server_init_default           {false, {0, {0}}, false, 0, false, "", false, 0}
+#define grpc_lb_v1_Duration_init_zero            {false, 0, false, 0}
+#define grpc_lb_v1_LoadBalanceRequest_init_zero  {false, grpc_lb_v1_InitialLoadBalanceRequest_init_zero, false, grpc_lb_v1_ClientStats_init_zero}
+#define grpc_lb_v1_InitialLoadBalanceRequest_init_zero {false, ""}
+#define grpc_lb_v1_ClientStats_init_zero         {false, 0, false, 0, false, 0}
+#define grpc_lb_v1_LoadBalanceResponse_init_zero {false, grpc_lb_v1_InitialLoadBalanceResponse_init_zero, false, grpc_lb_v1_ServerList_init_zero}
+#define grpc_lb_v1_InitialLoadBalanceResponse_init_zero {false, "", false, grpc_lb_v1_Duration_init_zero}
+#define grpc_lb_v1_ServerList_init_zero          {{{NULL}, NULL}, false, grpc_lb_v1_Duration_init_zero}
+#define grpc_lb_v1_Server_init_zero              {false, {0, {0}}, false, 0, false, "", false, 0}
+
+/* Field tags (for use in manual encoding/decoding) */
+#define grpc_lb_v1_ClientStats_total_requests_tag 1
+#define grpc_lb_v1_ClientStats_client_rpc_errors_tag 2
+#define grpc_lb_v1_ClientStats_dropped_requests_tag 3
+#define grpc_lb_v1_Duration_seconds_tag          1
+#define grpc_lb_v1_Duration_nanos_tag            2
+#define grpc_lb_v1_InitialLoadBalanceRequest_name_tag 1
+#define grpc_lb_v1_Server_ip_address_tag         1
+#define grpc_lb_v1_Server_port_tag               2
+#define grpc_lb_v1_Server_load_balance_token_tag 3
+#define grpc_lb_v1_Server_drop_request_tag       4
+#define grpc_lb_v1_InitialLoadBalanceResponse_load_balancer_delegate_tag 1
+#define grpc_lb_v1_InitialLoadBalanceResponse_client_stats_report_interval_tag 2
+#define grpc_lb_v1_LoadBalanceRequest_initial_request_tag 1
+#define grpc_lb_v1_LoadBalanceRequest_client_stats_tag 2
+#define grpc_lb_v1_ServerList_servers_tag        1
+#define grpc_lb_v1_ServerList_expiration_interval_tag 3
+#define grpc_lb_v1_LoadBalanceResponse_initial_response_tag 1
+#define grpc_lb_v1_LoadBalanceResponse_server_list_tag 2
+
+/* Struct field encoding specification for nanopb */
+extern const pb_field_t grpc_lb_v1_Duration_fields[3];
+extern const pb_field_t grpc_lb_v1_LoadBalanceRequest_fields[3];
+extern const pb_field_t grpc_lb_v1_InitialLoadBalanceRequest_fields[2];
+extern const pb_field_t grpc_lb_v1_ClientStats_fields[4];
+extern const pb_field_t grpc_lb_v1_LoadBalanceResponse_fields[3];
+extern const pb_field_t grpc_lb_v1_InitialLoadBalanceResponse_fields[3];
+extern const pb_field_t grpc_lb_v1_ServerList_fields[3];
+extern const pb_field_t grpc_lb_v1_Server_fields[5];
+
+/* Maximum encoded size of messages (where known) */
+#define grpc_lb_v1_Duration_size                 22
+#define grpc_lb_v1_LoadBalanceRequest_size       169
+#define grpc_lb_v1_InitialLoadBalanceRequest_size 131
+#define grpc_lb_v1_ClientStats_size              33
+#define grpc_lb_v1_LoadBalanceResponse_size      (98 + grpc_lb_v1_ServerList_size)
+#define grpc_lb_v1_InitialLoadBalanceResponse_size 90
+/* grpc_lb_v1_ServerList_size depends on runtime parameters */
+#define grpc_lb_v1_Server_size                   83
+
+/* Message IDs (where set with "msgid" option) */
+#ifdef PB_MSGID
+
+#define LOAD_BALANCER_MESSAGES \
+
+
+#endif
+
+#ifdef __cplusplus
+} /* extern "C" */
+#endif
+/* @@protoc_insertion_point(eof) */
+
+#endif
diff --git a/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.c b/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.c
new file mode 100644
index 0000000..6e04eda
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.c
@@ -0,0 +1,474 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+
+#include "src/core/ext/filters/client_channel/lb_policy_registry.h"
+#include "src/core/ext/filters/client_channel/subchannel.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/transport/connectivity_state.h"
+
+typedef struct pending_pick {
+  struct pending_pick *next;
+  uint32_t initial_metadata_flags;
+  grpc_connected_subchannel **target;
+  grpc_closure *on_complete;
+} pending_pick;
+
+typedef struct {
+  /** base policy: must be first */
+  grpc_lb_policy base;
+  /** all our subchannels */
+  grpc_subchannel **subchannels;
+  size_t num_subchannels;
+
+  grpc_closure connectivity_changed;
+
+  /** remaining members are protected by the combiner */
+
+  /** the selected channel */
+  grpc_connected_subchannel *selected;
+
+  /** have we started picking? */
+  int started_picking;
+  /** are we shut down? */
+  int shutdown;
+  /** which subchannel are we watching? */
+  size_t checking_subchannel;
+  /** what is the connectivity of that channel? */
+  grpc_connectivity_state checking_connectivity;
+  /** list of picks that are waiting on connectivity */
+  pending_pick *pending_picks;
+
+  /** our connectivity state tracker */
+  grpc_connectivity_state_tracker state_tracker;
+} pick_first_lb_policy;
+
+static void pf_destroy(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  size_t i;
+  GPR_ASSERT(p->pending_picks == NULL);
+  for (i = 0; i < p->num_subchannels; i++) {
+    GRPC_SUBCHANNEL_UNREF(exec_ctx, p->subchannels[i], "pick_first");
+  }
+  if (p->selected != NULL) {
+    GRPC_CONNECTED_SUBCHANNEL_UNREF(exec_ctx, p->selected, "picked_first");
+  }
+  grpc_connectivity_state_destroy(exec_ctx, &p->state_tracker);
+  gpr_free(p->subchannels);
+  gpr_free(p);
+}
+
+static void pf_shutdown_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  pending_pick *pp;
+  p->shutdown = 1;
+  pp = p->pending_picks;
+  p->pending_picks = NULL;
+  grpc_connectivity_state_set(
+      exec_ctx, &p->state_tracker, GRPC_CHANNEL_SHUTDOWN,
+      GRPC_ERROR_CREATE_FROM_STATIC_STRING("Channel shutdown"), "shutdown");
+  /* cancel subscription */
+  if (p->selected != NULL) {
+    grpc_connected_subchannel_notify_on_state_change(
+        exec_ctx, p->selected, NULL, NULL, &p->connectivity_changed);
+  } else if (p->num_subchannels > 0) {
+    grpc_subchannel_notify_on_state_change(
+        exec_ctx, p->subchannels[p->checking_subchannel], NULL, NULL,
+        &p->connectivity_changed);
+  }
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    *pp->target = NULL;
+    grpc_closure_sched(exec_ctx, pp->on_complete, GRPC_ERROR_NONE);
+    gpr_free(pp);
+    pp = next;
+  }
+}
+
+static void pf_cancel_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                                  grpc_connected_subchannel **target,
+                                  grpc_error *error) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  pending_pick *pp;
+  pp = p->pending_picks;
+  p->pending_picks = NULL;
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    if (pp->target == target) {
+      *target = NULL;
+      grpc_closure_sched(exec_ctx, pp->on_complete,
+                         GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                             "Pick Cancelled", &error, 1));
+      gpr_free(pp);
+    } else {
+      pp->next = p->pending_picks;
+      p->pending_picks = pp;
+    }
+    pp = next;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void pf_cancel_picks_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                                   uint32_t initial_metadata_flags_mask,
+                                   uint32_t initial_metadata_flags_eq,
+                                   grpc_error *error) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  pending_pick *pp;
+  pp = p->pending_picks;
+  p->pending_picks = NULL;
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    if ((pp->initial_metadata_flags & initial_metadata_flags_mask) ==
+        initial_metadata_flags_eq) {
+      grpc_closure_sched(exec_ctx, pp->on_complete,
+                         GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                             "Pick Cancelled", &error, 1));
+      gpr_free(pp);
+    } else {
+      pp->next = p->pending_picks;
+      p->pending_picks = pp;
+    }
+    pp = next;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void start_picking(grpc_exec_ctx *exec_ctx, pick_first_lb_policy *p) {
+  p->started_picking = 1;
+  p->checking_subchannel = 0;
+  p->checking_connectivity = GRPC_CHANNEL_IDLE;
+  GRPC_LB_POLICY_WEAK_REF(&p->base, "pick_first_connectivity");
+  grpc_subchannel_notify_on_state_change(
+      exec_ctx, p->subchannels[p->checking_subchannel],
+      p->base.interested_parties, &p->checking_connectivity,
+      &p->connectivity_changed);
+}
+
+static void pf_exit_idle_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  if (!p->started_picking) {
+    start_picking(exec_ctx, p);
+  }
+}
+
+static int pf_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                          const grpc_lb_policy_pick_args *pick_args,
+                          grpc_connected_subchannel **target, void **user_data,
+                          grpc_closure *on_complete) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  pending_pick *pp;
+
+  /* Check atomically for a selected channel */
+  if (p->selected != NULL) {
+    *target = GRPC_CONNECTED_SUBCHANNEL_REF(p->selected, "picked");
+    return 1;
+  }
+
+  /* No subchannel selected yet, so try again */
+  if (!p->started_picking) {
+    start_picking(exec_ctx, p);
+  }
+  pp = gpr_malloc(sizeof(*pp));
+  pp->next = p->pending_picks;
+  pp->target = target;
+  pp->initial_metadata_flags = pick_args->initial_metadata_flags;
+  pp->on_complete = on_complete;
+  p->pending_picks = pp;
+  return 0;
+}
+
+static void destroy_subchannels_locked(grpc_exec_ctx *exec_ctx,
+                                       pick_first_lb_policy *p) {
+  size_t i;
+  size_t num_subchannels = p->num_subchannels;
+  grpc_subchannel **subchannels;
+
+  subchannels = p->subchannels;
+  p->num_subchannels = 0;
+  p->subchannels = NULL;
+  GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &p->base, "destroy_subchannels");
+
+  for (i = 0; i < num_subchannels; i++) {
+    GRPC_SUBCHANNEL_UNREF(exec_ctx, subchannels[i], "pick_first");
+  }
+
+  gpr_free(subchannels);
+}
+
+static void pf_connectivity_changed_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                           grpc_error *error) {
+  pick_first_lb_policy *p = arg;
+  grpc_subchannel *selected_subchannel;
+  pending_pick *pp;
+
+  GRPC_ERROR_REF(error);
+
+  if (p->shutdown) {
+    GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &p->base, "pick_first_connectivity");
+    GRPC_ERROR_UNREF(error);
+    return;
+  } else if (p->selected != NULL) {
+    if (p->checking_connectivity == GRPC_CHANNEL_TRANSIENT_FAILURE) {
+      /* if the selected channel goes bad, we're done */
+      p->checking_connectivity = GRPC_CHANNEL_SHUTDOWN;
+    }
+    grpc_connectivity_state_set(exec_ctx, &p->state_tracker,
+                                p->checking_connectivity, GRPC_ERROR_REF(error),
+                                "selected_changed");
+    if (p->checking_connectivity != GRPC_CHANNEL_SHUTDOWN) {
+      grpc_connected_subchannel_notify_on_state_change(
+          exec_ctx, p->selected, p->base.interested_parties,
+          &p->checking_connectivity, &p->connectivity_changed);
+    } else {
+      GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &p->base, "pick_first_connectivity");
+    }
+  } else {
+  loop:
+    switch (p->checking_connectivity) {
+      case GRPC_CHANNEL_INIT:
+        GPR_UNREACHABLE_CODE(return );
+      case GRPC_CHANNEL_READY:
+        grpc_connectivity_state_set(exec_ctx, &p->state_tracker,
+                                    GRPC_CHANNEL_READY, GRPC_ERROR_NONE,
+                                    "connecting_ready");
+        selected_subchannel = p->subchannels[p->checking_subchannel];
+        p->selected = GRPC_CONNECTED_SUBCHANNEL_REF(
+            grpc_subchannel_get_connected_subchannel(selected_subchannel),
+            "picked_first");
+        /* drop the pick list: we are connected now */
+        GRPC_LB_POLICY_WEAK_REF(&p->base, "destroy_subchannels");
+        destroy_subchannels_locked(exec_ctx, p);
+        /* update any calls that were waiting for a pick */
+        while ((pp = p->pending_picks)) {
+          p->pending_picks = pp->next;
+          *pp->target = GRPC_CONNECTED_SUBCHANNEL_REF(p->selected, "picked");
+          grpc_closure_sched(exec_ctx, pp->on_complete, GRPC_ERROR_NONE);
+          gpr_free(pp);
+        }
+        grpc_connected_subchannel_notify_on_state_change(
+            exec_ctx, p->selected, p->base.interested_parties,
+            &p->checking_connectivity, &p->connectivity_changed);
+        break;
+      case GRPC_CHANNEL_TRANSIENT_FAILURE:
+        p->checking_subchannel =
+            (p->checking_subchannel + 1) % p->num_subchannels;
+        if (p->checking_subchannel == 0) {
+          /* only trigger transient failure when we've tried all alternatives */
+          grpc_connectivity_state_set(
+              exec_ctx, &p->state_tracker, GRPC_CHANNEL_TRANSIENT_FAILURE,
+              GRPC_ERROR_REF(error), "connecting_transient_failure");
+        }
+        GRPC_ERROR_UNREF(error);
+        p->checking_connectivity = grpc_subchannel_check_connectivity(
+            p->subchannels[p->checking_subchannel], &error);
+        if (p->checking_connectivity == GRPC_CHANNEL_TRANSIENT_FAILURE) {
+          grpc_subchannel_notify_on_state_change(
+              exec_ctx, p->subchannels[p->checking_subchannel],
+              p->base.interested_parties, &p->checking_connectivity,
+              &p->connectivity_changed);
+        } else {
+          goto loop;
+        }
+        break;
+      case GRPC_CHANNEL_CONNECTING:
+      case GRPC_CHANNEL_IDLE:
+        grpc_connectivity_state_set(
+            exec_ctx, &p->state_tracker, GRPC_CHANNEL_CONNECTING,
+            GRPC_ERROR_REF(error), "connecting_changed");
+        grpc_subchannel_notify_on_state_change(
+            exec_ctx, p->subchannels[p->checking_subchannel],
+            p->base.interested_parties, &p->checking_connectivity,
+            &p->connectivity_changed);
+        break;
+      case GRPC_CHANNEL_SHUTDOWN:
+        p->num_subchannels--;
+        GPR_SWAP(grpc_subchannel *, p->subchannels[p->checking_subchannel],
+                 p->subchannels[p->num_subchannels]);
+        GRPC_SUBCHANNEL_UNREF(exec_ctx, p->subchannels[p->num_subchannels],
+                              "pick_first");
+        if (p->num_subchannels == 0) {
+          grpc_connectivity_state_set(
+              exec_ctx, &p->state_tracker, GRPC_CHANNEL_SHUTDOWN,
+              GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                  "Pick first exhausted channels", &error, 1),
+              "no_more_channels");
+          while ((pp = p->pending_picks)) {
+            p->pending_picks = pp->next;
+            *pp->target = NULL;
+            grpc_closure_sched(exec_ctx, pp->on_complete, GRPC_ERROR_NONE);
+            gpr_free(pp);
+          }
+          GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &p->base,
+                                    "pick_first_connectivity");
+        } else {
+          grpc_connectivity_state_set(
+              exec_ctx, &p->state_tracker, GRPC_CHANNEL_TRANSIENT_FAILURE,
+              GRPC_ERROR_REF(error), "subchannel_failed");
+          p->checking_subchannel %= p->num_subchannels;
+          GRPC_ERROR_UNREF(error);
+          p->checking_connectivity = grpc_subchannel_check_connectivity(
+              p->subchannels[p->checking_subchannel], &error);
+          goto loop;
+        }
+    }
+  }
+
+  GRPC_ERROR_UNREF(error);
+}
+
+static grpc_connectivity_state pf_check_connectivity_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol, grpc_error **error) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  return grpc_connectivity_state_get(&p->state_tracker, error);
+}
+
+static void pf_notify_on_state_change_locked(grpc_exec_ctx *exec_ctx,
+                                             grpc_lb_policy *pol,
+                                             grpc_connectivity_state *current,
+                                             grpc_closure *notify) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  grpc_connectivity_state_notify_on_state_change(exec_ctx, &p->state_tracker,
+                                                 current, notify);
+}
+
+static void pf_ping_one_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                               grpc_closure *closure) {
+  pick_first_lb_policy *p = (pick_first_lb_policy *)pol;
+  if (p->selected) {
+    grpc_connected_subchannel_ping(exec_ctx, p->selected, closure);
+  } else {
+    grpc_closure_sched(exec_ctx, closure,
+                       GRPC_ERROR_CREATE_FROM_STATIC_STRING("Not connected"));
+  }
+}
+
+static const grpc_lb_policy_vtable pick_first_lb_policy_vtable = {
+    pf_destroy,
+    pf_shutdown_locked,
+    pf_pick_locked,
+    pf_cancel_pick_locked,
+    pf_cancel_picks_locked,
+    pf_ping_one_locked,
+    pf_exit_idle_locked,
+    pf_check_connectivity_locked,
+    pf_notify_on_state_change_locked};
+
+static void pick_first_factory_ref(grpc_lb_policy_factory *factory) {}
+
+static void pick_first_factory_unref(grpc_lb_policy_factory *factory) {}
+
+static grpc_lb_policy *create_pick_first(grpc_exec_ctx *exec_ctx,
+                                         grpc_lb_policy_factory *factory,
+                                         grpc_lb_policy_args *args) {
+  GPR_ASSERT(args->client_channel_factory != NULL);
+
+  /* Find the number of backend addresses. We ignore balancer
+   * addresses, since we don't know how to handle them. */
+  const grpc_arg *arg =
+      grpc_channel_args_find(args->args, GRPC_ARG_LB_ADDRESSES);
+  GPR_ASSERT(arg != NULL && arg->type == GRPC_ARG_POINTER);
+  grpc_lb_addresses *addresses = arg->value.pointer.p;
+  size_t num_addrs = 0;
+  for (size_t i = 0; i < addresses->num_addresses; i++) {
+    if (!addresses->addresses[i].is_balancer) ++num_addrs;
+  }
+  if (num_addrs == 0) return NULL;
+
+  pick_first_lb_policy *p = gpr_zalloc(sizeof(*p));
+
+  p->subchannels = gpr_zalloc(sizeof(grpc_subchannel *) * num_addrs);
+  grpc_subchannel_args sc_args;
+  size_t subchannel_idx = 0;
+  for (size_t i = 0; i < addresses->num_addresses; i++) {
+    /* Skip balancer addresses, since we only know how to handle backends. */
+    if (addresses->addresses[i].is_balancer) continue;
+
+    if (addresses->addresses[i].user_data != NULL) {
+      gpr_log(GPR_ERROR,
+              "This LB policy doesn't support user data. It will be ignored");
+    }
+
+    static const char *keys_to_remove[] = {GRPC_ARG_SUBCHANNEL_ADDRESS};
+    memset(&sc_args, 0, sizeof(grpc_subchannel_args));
+    grpc_arg addr_arg =
+        grpc_create_subchannel_address_arg(&addresses->addresses[i].address);
+    grpc_channel_args *new_args = grpc_channel_args_copy_and_add_and_remove(
+        args->args, keys_to_remove, GPR_ARRAY_SIZE(keys_to_remove), &addr_arg,
+        1);
+    gpr_free(addr_arg.value.string);
+    sc_args.args = new_args;
+    grpc_subchannel *subchannel = grpc_client_channel_factory_create_subchannel(
+        exec_ctx, args->client_channel_factory, &sc_args);
+    grpc_channel_args_destroy(exec_ctx, new_args);
+
+    if (subchannel != NULL) {
+      p->subchannels[subchannel_idx++] = subchannel;
+    }
+  }
+  if (subchannel_idx == 0) {
+    gpr_free(p->subchannels);
+    gpr_free(p);
+    return NULL;
+  }
+  p->num_subchannels = subchannel_idx;
+
+  grpc_lb_policy_init(&p->base, &pick_first_lb_policy_vtable, args->combiner);
+  grpc_closure_init(&p->connectivity_changed, pf_connectivity_changed_locked, p,
+                    grpc_combiner_scheduler(args->combiner, false));
+  return &p->base;
+}
+
+static const grpc_lb_policy_factory_vtable pick_first_factory_vtable = {
+    pick_first_factory_ref, pick_first_factory_unref, create_pick_first,
+    "pick_first"};
+
+static grpc_lb_policy_factory pick_first_lb_policy_factory = {
+    &pick_first_factory_vtable};
+
+static grpc_lb_policy_factory *pick_first_lb_factory_create() {
+  return &pick_first_lb_policy_factory;
+}
+
+/* Plugin registration */
+
+void grpc_lb_policy_pick_first_init() {
+  grpc_register_lb_policy(pick_first_lb_factory_create());
+}
+
+void grpc_lb_policy_pick_first_shutdown() {}
diff --git a/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.c b/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.c
new file mode 100644
index 0000000..dd7b3d1
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.c
@@ -0,0 +1,793 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+/** Round Robin Policy.
+ *
+ * This policy keeps:
+ * - A circular list of ready (connected) subchannels, the *readylist*. An empty
+ *   readylist consists solely of its root (dummy) node.
+ * - A pointer to the last element picked from the readylist, the *lastpick*.
+ *   Initially set to point to the readylist's root.
+ *
+ * Behavior:
+ * - When a subchannel connects, it's *prepended* to the readylist's root node.
+ *   Ie, if readylist = A <-> B <-> ROOT <-> C
+ *                      ^                    ^
+ *                      |____________________|
+ *   and subchannel D becomes connected, the addition of D to the readylist
+ *   results in  readylist = A <-> B <-> D <-> ROOT <-> C
+ *                           ^                          ^
+ *                           |__________________________|
+ * - When a subchannel disconnects, it's removed from the readylist. If the
+ *   subchannel being removed was the most recently picked, the *lastpick*
+ *   pointer moves to the removed node's previous element. Note that if the
+ *   readylist only had one element, this is still legal, as the lastpick would
+ *   point to the dummy root node, for an empty readylist.
+ * - Upon picking, *lastpick* is updated to point to the returned (connected)
+ *   subchannel. Note that it's possible that the selected subchannel becomes
+ *   disconnected in the interim between the selection and the actual usage of
+ *   the subchannel by the caller.
+ */
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+
+#include "src/core/ext/filters/client_channel/lb_policy_registry.h"
+#include "src/core/ext/filters/client_channel/subchannel.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/debug/trace.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/transport/connectivity_state.h"
+#include "src/core/lib/transport/static_metadata.h"
+
+typedef struct round_robin_lb_policy round_robin_lb_policy;
+
+int grpc_lb_round_robin_trace = 0;
+
+/** List of entities waiting for a pick.
+ *
+ * Once a pick is available, \a target is updated and \a on_complete called. */
+typedef struct pending_pick {
+  struct pending_pick *next;
+
+  /* output argument where to store the pick()ed user_data. It'll be NULL if no
+   * such data is present or there's an error (the definite test for errors is
+   * \a target being NULL). */
+  void **user_data;
+
+  /* bitmask passed to pick() and used for selective cancelling. See
+   * grpc_lb_policy_cancel_picks() */
+  uint32_t initial_metadata_flags;
+
+  /* output argument where to store the pick()ed connected subchannel, or NULL
+   * upon error. */
+  grpc_connected_subchannel **target;
+
+  /* to be invoked once the pick() has completed (regardless of success) */
+  grpc_closure *on_complete;
+} pending_pick;
+
+/** List of subchannels in a connectivity READY state */
+typedef struct ready_list {
+  grpc_subchannel *subchannel;
+  /* references namesake entry in subchannel_data */
+  void *user_data;
+  struct ready_list *next;
+  struct ready_list *prev;
+} ready_list;
+
+typedef struct {
+  /** index within policy->subchannels */
+  size_t index;
+  /** backpointer to owning policy */
+  round_robin_lb_policy *policy;
+  /** subchannel itself */
+  grpc_subchannel *subchannel;
+  /** notification that connectivity has changed on subchannel */
+  grpc_closure connectivity_changed_closure;
+  /** this subchannels current position in subchannel->ready_list */
+  ready_list *ready_list_node;
+  /** last observed connectivity. Not updated by
+   * \a grpc_subchannel_notify_on_state_change. Used to determine the previous
+   * state while processing the new state in \a rr_connectivity_changed */
+  grpc_connectivity_state prev_connectivity_state;
+  /** current connectivity state. Updated by \a
+   * grpc_subchannel_notify_on_state_change */
+  grpc_connectivity_state curr_connectivity_state;
+  /** the subchannel's target user data */
+  void *user_data;
+  /** vtable to operate over \a user_data */
+  const grpc_lb_user_data_vtable *user_data_vtable;
+} subchannel_data;
+
+struct round_robin_lb_policy {
+  /** base policy: must be first */
+  grpc_lb_policy base;
+
+  /** total number of addresses received at creation time */
+  size_t num_addresses;
+
+  /** all our subchannels */
+  size_t num_subchannels;
+  subchannel_data **subchannels;
+
+  /** how many subchannels are in TRANSIENT_FAILURE */
+  size_t num_transient_failures;
+  /** how many subchannels are IDLE */
+  size_t num_idle;
+
+  /** have we started picking? */
+  int started_picking;
+  /** are we shutting down? */
+  int shutdown;
+  /** List of picks that are waiting on connectivity */
+  pending_pick *pending_picks;
+
+  /** our connectivity state tracker */
+  grpc_connectivity_state_tracker state_tracker;
+
+  /** (Dummy) root of the doubly linked list containing READY subchannels */
+  ready_list ready_list;
+  /** Last pick from the ready list. */
+  ready_list *ready_list_last_pick;
+};
+
+/** Returns the next subchannel from the connected list or NULL if the list is
+ * empty.
+ *
+ * Note that this function does *not* advance p->ready_list_last_pick. Use \a
+ * advance_last_picked_locked() for that. */
+static ready_list *peek_next_connected_locked(const round_robin_lb_policy *p) {
+  ready_list *selected;
+  selected = p->ready_list_last_pick->next;
+
+  while (selected != NULL) {
+    if (selected == &p->ready_list) {
+      GPR_ASSERT(selected->subchannel == NULL);
+      /* skip dummy root */
+      selected = selected->next;
+    } else {
+      GPR_ASSERT(selected->subchannel != NULL);
+      return selected;
+    }
+  }
+  return NULL;
+}
+
+/** Advance the \a ready_list picking head. */
+static void advance_last_picked_locked(round_robin_lb_policy *p) {
+  if (p->ready_list_last_pick->next != NULL) { /* non-empty list */
+    p->ready_list_last_pick = p->ready_list_last_pick->next;
+    if (p->ready_list_last_pick == &p->ready_list) {
+      /* skip dummy root */
+      p->ready_list_last_pick = p->ready_list_last_pick->next;
+    }
+  } else { /* should be an empty list */
+    GPR_ASSERT(p->ready_list_last_pick == &p->ready_list);
+  }
+
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_DEBUG,
+            "[READYLIST, RR: %p] ADVANCED LAST PICK. NOW AT NODE %p (SC %p, "
+            "CSC %p)",
+            (void *)p, (void *)p->ready_list_last_pick,
+            (void *)p->ready_list_last_pick->subchannel,
+            (void *)grpc_subchannel_get_connected_subchannel(
+                p->ready_list_last_pick->subchannel));
+  }
+}
+
+/** Prepends (relative to the root at p->ready_list) the connected subchannel \a
+ * csc to the list of ready subchannels. */
+static ready_list *add_connected_sc_locked(round_robin_lb_policy *p,
+                                           subchannel_data *sd) {
+  ready_list *new_elem = gpr_zalloc(sizeof(ready_list));
+  new_elem->subchannel = sd->subchannel;
+  new_elem->user_data = sd->user_data;
+  if (p->ready_list.prev == NULL) {
+    /* first element */
+    new_elem->next = &p->ready_list;
+    new_elem->prev = &p->ready_list;
+    p->ready_list.next = new_elem;
+    p->ready_list.prev = new_elem;
+  } else {
+    new_elem->next = &p->ready_list;
+    new_elem->prev = p->ready_list.prev;
+    p->ready_list.prev->next = new_elem;
+    p->ready_list.prev = new_elem;
+  }
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_DEBUG, "[READYLIST] ADDING NODE %p (Conn. SC %p)",
+            (void *)new_elem, (void *)sd->subchannel);
+  }
+  return new_elem;
+}
+
+/** Removes \a node from the list of connected subchannels */
+static void remove_disconnected_sc_locked(round_robin_lb_policy *p,
+                                          ready_list *node) {
+  if (node == NULL) {
+    return;
+  }
+  if (node == p->ready_list_last_pick) {
+    p->ready_list_last_pick = p->ready_list_last_pick->prev;
+  }
+
+  /* removing last item */
+  if (node->next == &p->ready_list && node->prev == &p->ready_list) {
+    GPR_ASSERT(p->ready_list.next == node);
+    GPR_ASSERT(p->ready_list.prev == node);
+    p->ready_list.next = NULL;
+    p->ready_list.prev = NULL;
+  } else {
+    node->prev->next = node->next;
+    node->next->prev = node->prev;
+  }
+
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_DEBUG, "[READYLIST] REMOVED NODE %p (SC %p)", (void *)node,
+            (void *)node->subchannel);
+  }
+
+  node->next = NULL;
+  node->prev = NULL;
+  node->subchannel = NULL;
+
+  gpr_free(node);
+}
+
+static bool is_ready_list_empty(round_robin_lb_policy *p) {
+  return p->ready_list.prev == NULL;
+}
+
+static void rr_destroy(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  ready_list *elem;
+
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_DEBUG, "Destroying Round Robin policy at %p", (void *)pol);
+  }
+
+  for (size_t i = 0; i < p->num_subchannels; i++) {
+    subchannel_data *sd = p->subchannels[i];
+    GRPC_SUBCHANNEL_UNREF(exec_ctx, sd->subchannel, "rr_destroy");
+    if (sd->user_data != NULL) {
+      GPR_ASSERT(sd->user_data_vtable != NULL);
+      sd->user_data_vtable->destroy(exec_ctx, sd->user_data);
+    }
+    gpr_free(sd);
+  }
+
+  grpc_connectivity_state_destroy(exec_ctx, &p->state_tracker);
+  gpr_free(p->subchannels);
+
+  elem = p->ready_list.next;
+  while (elem != NULL && elem != &p->ready_list) {
+    ready_list *tmp;
+    tmp = elem->next;
+    elem->next = NULL;
+    elem->prev = NULL;
+    elem->subchannel = NULL;
+    gpr_free(elem);
+    elem = tmp;
+  }
+
+  gpr_free(p);
+}
+
+static void rr_shutdown_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  pending_pick *pp;
+  size_t i;
+
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_DEBUG, "Shutting down Round Robin policy at %p", (void *)pol);
+  }
+
+  p->shutdown = 1;
+  while ((pp = p->pending_picks)) {
+    p->pending_picks = pp->next;
+    *pp->target = NULL;
+    grpc_closure_sched(
+        exec_ctx, pp->on_complete,
+        GRPC_ERROR_CREATE_FROM_STATIC_STRING("Channel Shutdown"));
+    gpr_free(pp);
+  }
+  grpc_connectivity_state_set(
+      exec_ctx, &p->state_tracker, GRPC_CHANNEL_SHUTDOWN,
+      GRPC_ERROR_CREATE_FROM_STATIC_STRING("Channel Shutdown"), "rr_shutdown");
+  for (i = 0; i < p->num_subchannels; i++) {
+    subchannel_data *sd = p->subchannels[i];
+    grpc_subchannel_notify_on_state_change(exec_ctx, sd->subchannel, NULL, NULL,
+                                           &sd->connectivity_changed_closure);
+  }
+}
+
+static void rr_cancel_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                                  grpc_connected_subchannel **target,
+                                  grpc_error *error) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  pending_pick *pp;
+  pp = p->pending_picks;
+  p->pending_picks = NULL;
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    if (pp->target == target) {
+      *target = NULL;
+      grpc_closure_sched(exec_ctx, pp->on_complete,
+                         GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                             "Pick cancelled", &error, 1));
+      gpr_free(pp);
+    } else {
+      pp->next = p->pending_picks;
+      p->pending_picks = pp;
+    }
+    pp = next;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void rr_cancel_picks_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                                   uint32_t initial_metadata_flags_mask,
+                                   uint32_t initial_metadata_flags_eq,
+                                   grpc_error *error) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  pending_pick *pp;
+  pp = p->pending_picks;
+  p->pending_picks = NULL;
+  while (pp != NULL) {
+    pending_pick *next = pp->next;
+    if ((pp->initial_metadata_flags & initial_metadata_flags_mask) ==
+        initial_metadata_flags_eq) {
+      *pp->target = NULL;
+      grpc_closure_sched(exec_ctx, pp->on_complete,
+                         GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                             "Pick cancelled", &error, 1));
+      gpr_free(pp);
+    } else {
+      pp->next = p->pending_picks;
+      p->pending_picks = pp;
+    }
+    pp = next;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void start_picking_locked(grpc_exec_ctx *exec_ctx,
+                                 round_robin_lb_policy *p) {
+  size_t i;
+  p->started_picking = 1;
+
+  for (i = 0; i < p->num_subchannels; i++) {
+    subchannel_data *sd = p->subchannels[i];
+    /* use some sentinel value outside of the range of grpc_connectivity_state
+     * to signal an undefined previous state. We won't be referring to this
+     * value again and it'll be overwritten after the first call to
+     * rr_connectivity_changed */
+    sd->prev_connectivity_state = GRPC_CHANNEL_INIT;
+    sd->curr_connectivity_state = GRPC_CHANNEL_IDLE;
+    GRPC_LB_POLICY_WEAK_REF(&p->base, "rr_connectivity");
+    grpc_subchannel_notify_on_state_change(
+        exec_ctx, sd->subchannel, p->base.interested_parties,
+        &sd->curr_connectivity_state, &sd->connectivity_changed_closure);
+  }
+}
+
+static void rr_exit_idle_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  if (!p->started_picking) {
+    start_picking_locked(exec_ctx, p);
+  }
+}
+
+static int rr_pick_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                          const grpc_lb_policy_pick_args *pick_args,
+                          grpc_connected_subchannel **target, void **user_data,
+                          grpc_closure *on_complete) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  pending_pick *pp;
+  ready_list *selected;
+
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_INFO, "Round Robin %p trying to pick", (void *)pol);
+  }
+
+  if ((selected = peek_next_connected_locked(p))) {
+    /* readily available, report right away */
+    *target = GRPC_CONNECTED_SUBCHANNEL_REF(
+        grpc_subchannel_get_connected_subchannel(selected->subchannel),
+        "rr_picked");
+
+    if (user_data != NULL) {
+      *user_data = selected->user_data;
+    }
+    if (grpc_lb_round_robin_trace) {
+      gpr_log(GPR_DEBUG,
+              "[RR PICK] TARGET <-- CONNECTED SUBCHANNEL %p (NODE %p)",
+              (void *)*target, (void *)selected);
+    }
+    /* only advance the last picked pointer if the selection was used */
+    advance_last_picked_locked(p);
+    return 1;
+  } else {
+    /* no pick currently available. Save for later in list of pending picks */
+    if (!p->started_picking) {
+      start_picking_locked(exec_ctx, p);
+    }
+    pp = gpr_malloc(sizeof(*pp));
+    pp->next = p->pending_picks;
+    pp->target = target;
+    pp->on_complete = on_complete;
+    pp->initial_metadata_flags = pick_args->initial_metadata_flags;
+    pp->user_data = user_data;
+    p->pending_picks = pp;
+    return 0;
+  }
+}
+
+static void update_state_counters(subchannel_data *sd) {
+  round_robin_lb_policy *p = sd->policy;
+
+  /* update p->num_transient_failures (resp. p->num_idle): if the previous
+   * state was TRANSIENT_FAILURE (resp. IDLE), decrement
+   * p->num_transient_failures (resp. p->num_idle). */
+  if (sd->prev_connectivity_state == GRPC_CHANNEL_TRANSIENT_FAILURE) {
+    GPR_ASSERT(p->num_transient_failures > 0);
+    --p->num_transient_failures;
+  } else if (sd->prev_connectivity_state == GRPC_CHANNEL_IDLE) {
+    GPR_ASSERT(p->num_idle > 0);
+    --p->num_idle;
+  }
+}
+
+/* sd is the subchannel_data associted with the updated subchannel.
+ * shutdown_error will only be used upon policy transition to TRANSIENT_FAILURE
+ * or SHUTDOWN */
+static grpc_connectivity_state update_lb_connectivity_status(
+    grpc_exec_ctx *exec_ctx, subchannel_data *sd, grpc_error *error) {
+  /* In priority order. The first rule to match terminates the search (ie, if we
+   * are on rule n, all previous rules were unfulfilled).
+   *
+   * 1) RULE: ANY subchannel is READY => policy is READY.
+   *    CHECK: At least one subchannel is ready iff p->ready_list is NOT empty.
+   *
+   * 2) RULE: ANY subchannel is CONNECTING => policy is CONNECTING.
+   *    CHECK: sd->curr_connectivity_state == CONNECTING.
+   *
+   * 3) RULE: ALL subchannels are SHUTDOWN => policy is SHUTDOWN.
+   *    CHECK: p->num_subchannels = 0.
+   *
+   * 4) RULE: ALL subchannels are TRANSIENT_FAILURE => policy is
+   *    TRANSIENT_FAILURE.
+   *    CHECK: p->num_transient_failures == p->num_subchannels.
+   *
+   * 5) RULE: ALL subchannels are IDLE => policy is IDLE.
+   *    CHECK: p->num_idle == p->num_subchannels.
+   */
+  round_robin_lb_policy *p = sd->policy;
+  if (!is_ready_list_empty(p)) { /* 1) READY */
+    grpc_connectivity_state_set(exec_ctx, &p->state_tracker, GRPC_CHANNEL_READY,
+                                GRPC_ERROR_NONE, "rr_ready");
+    return GRPC_CHANNEL_READY;
+  } else if (sd->curr_connectivity_state ==
+             GRPC_CHANNEL_CONNECTING) { /* 2) CONNECTING */
+    grpc_connectivity_state_set(exec_ctx, &p->state_tracker,
+                                GRPC_CHANNEL_CONNECTING, GRPC_ERROR_NONE,
+                                "rr_connecting");
+    return GRPC_CHANNEL_CONNECTING;
+  } else if (p->num_subchannels == 0) { /* 3) SHUTDOWN */
+    grpc_connectivity_state_set(exec_ctx, &p->state_tracker,
+                                GRPC_CHANNEL_SHUTDOWN, GRPC_ERROR_REF(error),
+                                "rr_shutdown");
+    return GRPC_CHANNEL_SHUTDOWN;
+  } else if (p->num_transient_failures ==
+             p->num_subchannels) { /* 4) TRANSIENT_FAILURE */
+    grpc_connectivity_state_set(exec_ctx, &p->state_tracker,
+                                GRPC_CHANNEL_TRANSIENT_FAILURE,
+                                GRPC_ERROR_REF(error), "rr_transient_failure");
+    return GRPC_CHANNEL_TRANSIENT_FAILURE;
+  } else if (p->num_idle == p->num_subchannels) { /* 5) IDLE */
+    grpc_connectivity_state_set(exec_ctx, &p->state_tracker, GRPC_CHANNEL_IDLE,
+                                GRPC_ERROR_NONE, "rr_idle");
+    return GRPC_CHANNEL_IDLE;
+  }
+  /* no change */
+  return sd->curr_connectivity_state;
+}
+
+static void rr_connectivity_changed_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                           grpc_error *error) {
+  subchannel_data *sd = arg;
+  round_robin_lb_policy *p = sd->policy;
+  pending_pick *pp;
+
+  GRPC_ERROR_REF(error);
+
+  if (p->shutdown) {
+    GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &p->base, "rr_connectivity");
+    GRPC_ERROR_UNREF(error);
+    return;
+  }
+  switch (sd->curr_connectivity_state) {
+    case GRPC_CHANNEL_INIT:
+      GPR_UNREACHABLE_CODE(return );
+    case GRPC_CHANNEL_READY:
+      /* add the newly connected subchannel to the list of connected ones.
+       * Note that it goes to the "end of the line". */
+      sd->ready_list_node = add_connected_sc_locked(p, sd);
+      /* at this point we know there's at least one suitable subchannel. Go
+       * ahead and pick one and notify the pending suitors in
+       * p->pending_picks. This preemtively replicates rr_pick()'s actions. */
+      ready_list *selected = peek_next_connected_locked(p);
+      GPR_ASSERT(selected != NULL);
+      if (p->pending_picks != NULL) {
+        /* if the selected subchannel is going to be used for the pending
+         * picks, update the last picked pointer */
+        advance_last_picked_locked(p);
+      }
+      while ((pp = p->pending_picks)) {
+        p->pending_picks = pp->next;
+        *pp->target = GRPC_CONNECTED_SUBCHANNEL_REF(
+            grpc_subchannel_get_connected_subchannel(selected->subchannel),
+            "rr_picked");
+        if (pp->user_data != NULL) {
+          *pp->user_data = selected->user_data;
+        }
+        if (grpc_lb_round_robin_trace) {
+          gpr_log(GPR_DEBUG,
+                  "[RR CONN CHANGED] TARGET <-- SUBCHANNEL %p (NODE %p)",
+                  (void *)selected->subchannel, (void *)selected);
+        }
+        grpc_closure_sched(exec_ctx, pp->on_complete, GRPC_ERROR_NONE);
+        gpr_free(pp);
+      }
+      update_lb_connectivity_status(exec_ctx, sd, error);
+      sd->prev_connectivity_state = sd->curr_connectivity_state;
+      /* renew notification: reuses the "rr_connectivity" weak ref */
+      grpc_subchannel_notify_on_state_change(
+          exec_ctx, sd->subchannel, p->base.interested_parties,
+          &sd->curr_connectivity_state, &sd->connectivity_changed_closure);
+      break;
+    case GRPC_CHANNEL_IDLE:
+      ++p->num_idle;
+    /* fallthrough */
+    case GRPC_CHANNEL_CONNECTING:
+      update_state_counters(sd);
+      update_lb_connectivity_status(exec_ctx, sd, error);
+      sd->prev_connectivity_state = sd->curr_connectivity_state;
+      /* renew notification: reuses the "rr_connectivity" weak ref */
+      grpc_subchannel_notify_on_state_change(
+          exec_ctx, sd->subchannel, p->base.interested_parties,
+          &sd->curr_connectivity_state, &sd->connectivity_changed_closure);
+      break;
+    case GRPC_CHANNEL_TRANSIENT_FAILURE:
+      ++p->num_transient_failures;
+      /* remove from ready list if still present */
+      if (sd->ready_list_node != NULL) {
+        remove_disconnected_sc_locked(p, sd->ready_list_node);
+        sd->ready_list_node = NULL;
+      }
+      update_lb_connectivity_status(exec_ctx, sd, error);
+      sd->prev_connectivity_state = sd->curr_connectivity_state;
+      /* renew notification: reuses the "rr_connectivity" weak ref */
+      grpc_subchannel_notify_on_state_change(
+          exec_ctx, sd->subchannel, p->base.interested_parties,
+          &sd->curr_connectivity_state, &sd->connectivity_changed_closure);
+      break;
+    case GRPC_CHANNEL_SHUTDOWN:
+      update_state_counters(sd);
+      if (sd->ready_list_node != NULL) {
+        remove_disconnected_sc_locked(p, sd->ready_list_node);
+        sd->ready_list_node = NULL;
+      }
+      --p->num_subchannels;
+      GPR_SWAP(subchannel_data *, p->subchannels[sd->index],
+               p->subchannels[p->num_subchannels]);
+      GRPC_SUBCHANNEL_UNREF(exec_ctx, sd->subchannel, "rr_subchannel_shutdown");
+      p->subchannels[sd->index]->index = sd->index;
+      if (update_lb_connectivity_status(exec_ctx, sd, error) ==
+          GRPC_CHANNEL_SHUTDOWN) {
+        /* the policy is shutting down. Flush all the pending picks... */
+        while ((pp = p->pending_picks)) {
+          p->pending_picks = pp->next;
+          *pp->target = NULL;
+          grpc_closure_sched(exec_ctx, pp->on_complete, GRPC_ERROR_NONE);
+          gpr_free(pp);
+        }
+      }
+      gpr_free(sd);
+      /* unref the "rr_connectivity" weak ref from start_picking */
+      GRPC_LB_POLICY_WEAK_UNREF(exec_ctx, &p->base, "rr_connectivity");
+      break;
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static grpc_connectivity_state rr_check_connectivity_locked(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol, grpc_error **error) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  return grpc_connectivity_state_get(&p->state_tracker, error);
+}
+
+static void rr_notify_on_state_change_locked(grpc_exec_ctx *exec_ctx,
+                                             grpc_lb_policy *pol,
+                                             grpc_connectivity_state *current,
+                                             grpc_closure *notify) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  grpc_connectivity_state_notify_on_state_change(exec_ctx, &p->state_tracker,
+                                                 current, notify);
+}
+
+static void rr_ping_one_locked(grpc_exec_ctx *exec_ctx, grpc_lb_policy *pol,
+                               grpc_closure *closure) {
+  round_robin_lb_policy *p = (round_robin_lb_policy *)pol;
+  ready_list *selected;
+  grpc_connected_subchannel *target;
+  if ((selected = peek_next_connected_locked(p))) {
+    target = GRPC_CONNECTED_SUBCHANNEL_REF(
+        grpc_subchannel_get_connected_subchannel(selected->subchannel),
+        "rr_picked");
+    grpc_connected_subchannel_ping(exec_ctx, target, closure);
+    GRPC_CONNECTED_SUBCHANNEL_UNREF(exec_ctx, target, "rr_picked");
+  } else {
+    grpc_closure_sched(exec_ctx, closure, GRPC_ERROR_CREATE_FROM_STATIC_STRING(
+                                              "Round Robin not connected"));
+  }
+}
+
+static const grpc_lb_policy_vtable round_robin_lb_policy_vtable = {
+    rr_destroy,
+    rr_shutdown_locked,
+    rr_pick_locked,
+    rr_cancel_pick_locked,
+    rr_cancel_picks_locked,
+    rr_ping_one_locked,
+    rr_exit_idle_locked,
+    rr_check_connectivity_locked,
+    rr_notify_on_state_change_locked};
+
+static void round_robin_factory_ref(grpc_lb_policy_factory *factory) {}
+
+static void round_robin_factory_unref(grpc_lb_policy_factory *factory) {}
+
+static grpc_lb_policy *round_robin_create(grpc_exec_ctx *exec_ctx,
+                                          grpc_lb_policy_factory *factory,
+                                          grpc_lb_policy_args *args) {
+  GPR_ASSERT(args->client_channel_factory != NULL);
+
+  /* Find the number of backend addresses. We ignore balancer
+   * addresses, since we don't know how to handle them. */
+  const grpc_arg *arg =
+      grpc_channel_args_find(args->args, GRPC_ARG_LB_ADDRESSES);
+  GPR_ASSERT(arg != NULL && arg->type == GRPC_ARG_POINTER);
+  grpc_lb_addresses *addresses = arg->value.pointer.p;
+  size_t num_addrs = 0;
+  for (size_t i = 0; i < addresses->num_addresses; i++) {
+    if (!addresses->addresses[i].is_balancer) ++num_addrs;
+  }
+  if (num_addrs == 0) return NULL;
+
+  round_robin_lb_policy *p = gpr_zalloc(sizeof(*p));
+
+  p->num_addresses = num_addrs;
+  p->subchannels = gpr_zalloc(sizeof(*p->subchannels) * num_addrs);
+
+  grpc_subchannel_args sc_args;
+  size_t subchannel_idx = 0;
+  for (size_t i = 0; i < addresses->num_addresses; i++) {
+    /* Skip balancer addresses, since we only know how to handle backends. */
+    if (addresses->addresses[i].is_balancer) continue;
+
+    static const char *keys_to_remove[] = {GRPC_ARG_SUBCHANNEL_ADDRESS};
+    memset(&sc_args, 0, sizeof(grpc_subchannel_args));
+    grpc_arg addr_arg =
+        grpc_create_subchannel_address_arg(&addresses->addresses[i].address);
+    grpc_channel_args *new_args = grpc_channel_args_copy_and_add_and_remove(
+        args->args, keys_to_remove, GPR_ARRAY_SIZE(keys_to_remove), &addr_arg,
+        1);
+    gpr_free(addr_arg.value.string);
+    sc_args.args = new_args;
+    grpc_subchannel *subchannel = grpc_client_channel_factory_create_subchannel(
+        exec_ctx, args->client_channel_factory, &sc_args);
+    if (grpc_lb_round_robin_trace) {
+      char *address_uri =
+          grpc_sockaddr_to_uri(&addresses->addresses[i].address);
+      gpr_log(GPR_DEBUG, "Created subchannel %p for address uri %s",
+              (void *)subchannel, address_uri);
+      gpr_free(address_uri);
+    }
+    grpc_channel_args_destroy(exec_ctx, new_args);
+
+    if (subchannel != NULL) {
+      subchannel_data *sd = gpr_zalloc(sizeof(*sd));
+      p->subchannels[subchannel_idx] = sd;
+      sd->policy = p;
+      sd->index = subchannel_idx;
+      sd->subchannel = subchannel;
+      sd->user_data_vtable = addresses->user_data_vtable;
+      if (sd->user_data_vtable != NULL) {
+        sd->user_data =
+            sd->user_data_vtable->copy(addresses->addresses[i].user_data);
+      }
+      ++subchannel_idx;
+      grpc_closure_init(&sd->connectivity_changed_closure,
+                        rr_connectivity_changed_locked, sd,
+                        grpc_combiner_scheduler(args->combiner, false));
+    }
+  }
+  if (subchannel_idx == 0) {
+    /* couldn't create any subchannel. Bail out */
+    gpr_free(p->subchannels);
+    gpr_free(p);
+    return NULL;
+  }
+  p->num_subchannels = subchannel_idx;
+
+  /* The (dummy node) root of the ready list */
+  p->ready_list.subchannel = NULL;
+  p->ready_list.prev = NULL;
+  p->ready_list.next = NULL;
+  p->ready_list_last_pick = &p->ready_list;
+
+  grpc_lb_policy_init(&p->base, &round_robin_lb_policy_vtable, args->combiner);
+  grpc_connectivity_state_init(&p->state_tracker, GRPC_CHANNEL_IDLE,
+                               "round_robin");
+
+  if (grpc_lb_round_robin_trace) {
+    gpr_log(GPR_DEBUG, "Created RR policy at %p with %lu subchannels",
+            (void *)p, (unsigned long)p->num_subchannels);
+  }
+  return &p->base;
+}
+
+static const grpc_lb_policy_factory_vtable round_robin_factory_vtable = {
+    round_robin_factory_ref, round_robin_factory_unref, round_robin_create,
+    "round_robin"};
+
+static grpc_lb_policy_factory round_robin_lb_policy_factory = {
+    &round_robin_factory_vtable};
+
+static grpc_lb_policy_factory *round_robin_lb_factory_create() {
+  return &round_robin_lb_policy_factory;
+}
+
+/* Plugin registration */
+
+void grpc_lb_policy_round_robin_init() {
+  grpc_register_lb_policy(round_robin_lb_factory_create());
+  grpc_register_tracer("round_robin", &grpc_lb_round_robin_trace);
+}
+
+void grpc_lb_policy_round_robin_shutdown() {}
diff --git a/src/core/ext/filters/client_channel/lb_policy_factory.c b/src/core/ext/filters/client_channel/lb_policy_factory.c
new file mode 100644
index 0000000..e2af216
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy_factory.c
@@ -0,0 +1,163 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+
+grpc_lb_addresses* grpc_lb_addresses_create(
+    size_t num_addresses, const grpc_lb_user_data_vtable* user_data_vtable) {
+  grpc_lb_addresses* addresses = gpr_malloc(sizeof(grpc_lb_addresses));
+  addresses->num_addresses = num_addresses;
+  addresses->user_data_vtable = user_data_vtable;
+  const size_t addresses_size = sizeof(grpc_lb_address) * num_addresses;
+  addresses->addresses = gpr_malloc(addresses_size);
+  memset(addresses->addresses, 0, addresses_size);
+  return addresses;
+}
+
+grpc_lb_addresses* grpc_lb_addresses_copy(const grpc_lb_addresses* addresses) {
+  grpc_lb_addresses* new_addresses = grpc_lb_addresses_create(
+      addresses->num_addresses, addresses->user_data_vtable);
+  memcpy(new_addresses->addresses, addresses->addresses,
+         sizeof(grpc_lb_address) * addresses->num_addresses);
+  for (size_t i = 0; i < addresses->num_addresses; ++i) {
+    if (new_addresses->addresses[i].balancer_name != NULL) {
+      new_addresses->addresses[i].balancer_name =
+          gpr_strdup(new_addresses->addresses[i].balancer_name);
+    }
+    if (new_addresses->addresses[i].user_data != NULL) {
+      new_addresses->addresses[i].user_data = addresses->user_data_vtable->copy(
+          new_addresses->addresses[i].user_data);
+    }
+  }
+  return new_addresses;
+}
+
+void grpc_lb_addresses_set_address(grpc_lb_addresses* addresses, size_t index,
+                                   void* address, size_t address_len,
+                                   bool is_balancer, char* balancer_name,
+                                   void* user_data) {
+  GPR_ASSERT(index < addresses->num_addresses);
+  if (user_data != NULL) GPR_ASSERT(addresses->user_data_vtable != NULL);
+  grpc_lb_address* target = &addresses->addresses[index];
+  memcpy(target->address.addr, address, address_len);
+  target->address.len = address_len;
+  target->is_balancer = is_balancer;
+  target->balancer_name = balancer_name;
+  target->user_data = user_data;
+}
+
+int grpc_lb_addresses_cmp(const grpc_lb_addresses* addresses1,
+                          const grpc_lb_addresses* addresses2) {
+  if (addresses1->num_addresses > addresses2->num_addresses) return 1;
+  if (addresses1->num_addresses < addresses2->num_addresses) return -1;
+  if (addresses1->user_data_vtable > addresses2->user_data_vtable) return 1;
+  if (addresses1->user_data_vtable < addresses2->user_data_vtable) return -1;
+  for (size_t i = 0; i < addresses1->num_addresses; ++i) {
+    const grpc_lb_address* target1 = &addresses1->addresses[i];
+    const grpc_lb_address* target2 = &addresses2->addresses[i];
+    if (target1->address.len > target2->address.len) return 1;
+    if (target1->address.len < target2->address.len) return -1;
+    int retval = memcmp(target1->address.addr, target2->address.addr,
+                        target1->address.len);
+    if (retval != 0) return retval;
+    if (target1->is_balancer > target2->is_balancer) return 1;
+    if (target1->is_balancer < target2->is_balancer) return -1;
+    const char* balancer_name1 =
+        target1->balancer_name != NULL ? target1->balancer_name : "";
+    const char* balancer_name2 =
+        target2->balancer_name != NULL ? target2->balancer_name : "";
+    retval = strcmp(balancer_name1, balancer_name2);
+    if (retval != 0) return retval;
+    if (addresses1->user_data_vtable != NULL) {
+      retval = addresses1->user_data_vtable->cmp(target1->user_data,
+                                                 target2->user_data);
+      if (retval != 0) return retval;
+    }
+  }
+  return 0;
+}
+
+void grpc_lb_addresses_destroy(grpc_exec_ctx* exec_ctx,
+                               grpc_lb_addresses* addresses) {
+  for (size_t i = 0; i < addresses->num_addresses; ++i) {
+    gpr_free(addresses->addresses[i].balancer_name);
+    if (addresses->addresses[i].user_data != NULL) {
+      addresses->user_data_vtable->destroy(exec_ctx,
+                                           addresses->addresses[i].user_data);
+    }
+  }
+  gpr_free(addresses->addresses);
+  gpr_free(addresses);
+}
+
+static void* lb_addresses_copy(void* addresses) {
+  return grpc_lb_addresses_copy(addresses);
+}
+static void lb_addresses_destroy(grpc_exec_ctx* exec_ctx, void* addresses) {
+  grpc_lb_addresses_destroy(exec_ctx, addresses);
+}
+static int lb_addresses_cmp(void* addresses1, void* addresses2) {
+  return grpc_lb_addresses_cmp(addresses1, addresses2);
+}
+static const grpc_arg_pointer_vtable lb_addresses_arg_vtable = {
+    lb_addresses_copy, lb_addresses_destroy, lb_addresses_cmp};
+
+grpc_arg grpc_lb_addresses_create_channel_arg(
+    const grpc_lb_addresses* addresses) {
+  grpc_arg arg;
+  arg.type = GRPC_ARG_POINTER;
+  arg.key = GRPC_ARG_LB_ADDRESSES;
+  arg.value.pointer.p = (void*)addresses;
+  arg.value.pointer.vtable = &lb_addresses_arg_vtable;
+  return arg;
+}
+
+void grpc_lb_policy_factory_ref(grpc_lb_policy_factory* factory) {
+  factory->vtable->ref(factory);
+}
+
+void grpc_lb_policy_factory_unref(grpc_lb_policy_factory* factory) {
+  factory->vtable->unref(factory);
+}
+
+grpc_lb_policy* grpc_lb_policy_factory_create_lb_policy(
+    grpc_exec_ctx* exec_ctx, grpc_lb_policy_factory* factory,
+    grpc_lb_policy_args* args) {
+  if (factory == NULL) return NULL;
+  return factory->vtable->create_lb_policy(exec_ctx, factory, args);
+}
diff --git a/src/core/ext/filters/client_channel/lb_policy_factory.h b/src/core/ext/filters/client_channel/lb_policy_factory.h
new file mode 100644
index 0000000..709de99
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy_factory.h
@@ -0,0 +1,134 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_FACTORY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_FACTORY_H
+
+#include "src/core/ext/filters/client_channel/client_channel_factory.h"
+#include "src/core/ext/filters/client_channel/lb_policy.h"
+
+#include "src/core/lib/iomgr/exec_ctx.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+
+// Channel arg key for grpc_lb_addresses.
+#define GRPC_ARG_LB_ADDRESSES "grpc.lb_addresses"
+
+typedef struct grpc_lb_policy_factory grpc_lb_policy_factory;
+typedef struct grpc_lb_policy_factory_vtable grpc_lb_policy_factory_vtable;
+
+struct grpc_lb_policy_factory {
+  const grpc_lb_policy_factory_vtable *vtable;
+};
+
+/** A resolved address alongside any LB related information associated with it.
+ * \a user_data, if not NULL, contains opaque data meant to be consumed by the
+ * gRPC LB policy. Note that no all LB policies support \a user_data as input.
+ * Those who don't will simply ignore it and will correspondingly return NULL in
+ * their namesake pick() output argument. */
+typedef struct grpc_lb_address {
+  grpc_resolved_address address;
+  bool is_balancer;
+  char *balancer_name; /* For secure naming. */
+  void *user_data;
+} grpc_lb_address;
+
+typedef struct grpc_lb_user_data_vtable {
+  void *(*copy)(void *);
+  void (*destroy)(grpc_exec_ctx *exec_ctx, void *);
+  int (*cmp)(void *, void *);
+} grpc_lb_user_data_vtable;
+
+typedef struct grpc_lb_addresses {
+  size_t num_addresses;
+  grpc_lb_address *addresses;
+  const grpc_lb_user_data_vtable *user_data_vtable;
+} grpc_lb_addresses;
+
+/** Returns a grpc_addresses struct with enough space for
+    \a num_addresses addresses.  The \a user_data_vtable argument may be
+    NULL if no user data will be added. */
+grpc_lb_addresses *grpc_lb_addresses_create(
+    size_t num_addresses, const grpc_lb_user_data_vtable *user_data_vtable);
+
+/** Creates a copy of \a addresses. */
+grpc_lb_addresses *grpc_lb_addresses_copy(const grpc_lb_addresses *addresses);
+
+/** Sets the value of the address at index \a index of \a addresses.
+ * \a address is a socket address of length \a address_len.
+ * Takes ownership of \a balancer_name. */
+void grpc_lb_addresses_set_address(grpc_lb_addresses *addresses, size_t index,
+                                   void *address, size_t address_len,
+                                   bool is_balancer, char *balancer_name,
+                                   void *user_data);
+
+/** Compares \a addresses1 and \a addresses2. */
+int grpc_lb_addresses_cmp(const grpc_lb_addresses *addresses1,
+                          const grpc_lb_addresses *addresses2);
+
+/** Destroys \a addresses. */
+void grpc_lb_addresses_destroy(grpc_exec_ctx *exec_ctx,
+                               grpc_lb_addresses *addresses);
+
+/** Returns a channel arg containing \a addresses. */
+grpc_arg grpc_lb_addresses_create_channel_arg(
+    const grpc_lb_addresses *addresses);
+
+/** Arguments passed to LB policies. */
+typedef struct grpc_lb_policy_args {
+  grpc_client_channel_factory *client_channel_factory;
+  grpc_channel_args *args;
+  grpc_combiner *combiner;
+} grpc_lb_policy_args;
+
+struct grpc_lb_policy_factory_vtable {
+  void (*ref)(grpc_lb_policy_factory *factory);
+  void (*unref)(grpc_lb_policy_factory *factory);
+
+  /** Implementation of grpc_lb_policy_factory_create_lb_policy */
+  grpc_lb_policy *(*create_lb_policy)(grpc_exec_ctx *exec_ctx,
+                                      grpc_lb_policy_factory *factory,
+                                      grpc_lb_policy_args *args);
+
+  /** Name for the LB policy this factory implements */
+  const char *name;
+};
+
+void grpc_lb_policy_factory_ref(grpc_lb_policy_factory *factory);
+void grpc_lb_policy_factory_unref(grpc_lb_policy_factory *factory);
+
+/** Create a lb_policy instance. */
+grpc_lb_policy *grpc_lb_policy_factory_create_lb_policy(
+    grpc_exec_ctx *exec_ctx, grpc_lb_policy_factory *factory,
+    grpc_lb_policy_args *args);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_FACTORY_H */
diff --git a/src/core/ext/filters/client_channel/lb_policy_registry.c b/src/core/ext/filters/client_channel/lb_policy_registry.c
new file mode 100644
index 0000000..1376673
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy_registry.c
@@ -0,0 +1,85 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/lb_policy_registry.h"
+
+#include <string.h>
+
+#include "src/core/lib/support/string.h"
+
+#define MAX_POLICIES 10
+
+static grpc_lb_policy_factory *g_all_of_the_lb_policies[MAX_POLICIES];
+static int g_number_of_lb_policies = 0;
+
+void grpc_lb_policy_registry_init(void) { g_number_of_lb_policies = 0; }
+
+void grpc_lb_policy_registry_shutdown(void) {
+  int i;
+  for (i = 0; i < g_number_of_lb_policies; i++) {
+    grpc_lb_policy_factory_unref(g_all_of_the_lb_policies[i]);
+  }
+}
+
+void grpc_register_lb_policy(grpc_lb_policy_factory *factory) {
+  int i;
+  for (i = 0; i < g_number_of_lb_policies; i++) {
+    GPR_ASSERT(0 != gpr_stricmp(factory->vtable->name,
+                                g_all_of_the_lb_policies[i]->vtable->name));
+  }
+  GPR_ASSERT(g_number_of_lb_policies != MAX_POLICIES);
+  grpc_lb_policy_factory_ref(factory);
+  g_all_of_the_lb_policies[g_number_of_lb_policies++] = factory;
+}
+
+static grpc_lb_policy_factory *lookup_factory(const char *name) {
+  int i;
+
+  if (name == NULL) return NULL;
+
+  for (i = 0; i < g_number_of_lb_policies; i++) {
+    if (0 == gpr_stricmp(name, g_all_of_the_lb_policies[i]->vtable->name)) {
+      return g_all_of_the_lb_policies[i];
+    }
+  }
+
+  return NULL;
+}
+
+grpc_lb_policy *grpc_lb_policy_create(grpc_exec_ctx *exec_ctx, const char *name,
+                                      grpc_lb_policy_args *args) {
+  grpc_lb_policy_factory *factory = lookup_factory(name);
+  grpc_lb_policy *lb_policy =
+      grpc_lb_policy_factory_create_lb_policy(exec_ctx, factory, args);
+  return lb_policy;
+}
diff --git a/src/core/ext/filters/client_channel/lb_policy_registry.h b/src/core/ext/filters/client_channel/lb_policy_registry.h
new file mode 100644
index 0000000..9374e01
--- /dev/null
+++ b/src/core/ext/filters/client_channel/lb_policy_registry.h
@@ -0,0 +1,55 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_REGISTRY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_REGISTRY_H
+
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+#include "src/core/lib/iomgr/exec_ctx.h"
+
+/** Initialize the registry and set \a default_factory as the factory to be
+ * returned when no name is provided in a lookup */
+void grpc_lb_policy_registry_init(void);
+void grpc_lb_policy_registry_shutdown(void);
+
+/** Register a LB policy factory. */
+void grpc_register_lb_policy(grpc_lb_policy_factory *factory);
+
+/** Create a \a grpc_lb_policy instance.
+ *
+ * If \a name is NULL, the default factory from \a grpc_lb_policy_registry_init
+ * will be returned. */
+grpc_lb_policy *grpc_lb_policy_create(grpc_exec_ctx *exec_ctx, const char *name,
+                                      grpc_lb_policy_args *args);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_LB_POLICY_REGISTRY_H */
diff --git a/src/core/ext/filters/client_channel/parse_address.c b/src/core/ext/filters/client_channel/parse_address.c
new file mode 100644
index 0000000..0c97062
--- /dev/null
+++ b/src/core/ext/filters/client_channel/parse_address.c
@@ -0,0 +1,170 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/parse_address.h"
+#include "src/core/lib/iomgr/sockaddr.h"
+
+#include <stdio.h>
+#include <string.h>
+#ifdef GRPC_HAVE_UNIX_SOCKET
+#include <sys/un.h>
+#endif
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/host_port.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+#include "src/core/lib/support/string.h"
+
+#ifdef GRPC_HAVE_UNIX_SOCKET
+
+int parse_unix(grpc_uri *uri, grpc_resolved_address *resolved_addr) {
+  struct sockaddr_un *un = (struct sockaddr_un *)resolved_addr->addr;
+  const size_t maxlen = sizeof(un->sun_path);
+  const size_t path_len = strnlen(uri->path, maxlen);
+  if (path_len == maxlen) return 0;
+  un->sun_family = AF_UNIX;
+  strcpy(un->sun_path, uri->path);
+  resolved_addr->len = sizeof(*un);
+  return 1;
+}
+
+#else /* GRPC_HAVE_UNIX_SOCKET */
+
+int parse_unix(grpc_uri *uri, grpc_resolved_address *resolved_addr) { abort(); }
+
+#endif /* GRPC_HAVE_UNIX_SOCKET */
+
+int parse_ipv4(grpc_uri *uri, grpc_resolved_address *resolved_addr) {
+  const char *host_port = uri->path;
+  char *host;
+  char *port;
+  int port_num;
+  int result = 0;
+  struct sockaddr_in *in = (struct sockaddr_in *)resolved_addr->addr;
+
+  if (*host_port == '/') ++host_port;
+  if (!gpr_split_host_port(host_port, &host, &port)) {
+    return 0;
+  }
+
+  memset(resolved_addr, 0, sizeof(grpc_resolved_address));
+  resolved_addr->len = sizeof(struct sockaddr_in);
+  in->sin_family = AF_INET;
+  if (inet_pton(AF_INET, host, &in->sin_addr) == 0) {
+    gpr_log(GPR_ERROR, "invalid ipv4 address: '%s'", host);
+    goto done;
+  }
+
+  if (port != NULL) {
+    if (sscanf(port, "%d", &port_num) != 1 || port_num < 0 ||
+        port_num > 65535) {
+      gpr_log(GPR_ERROR, "invalid ipv4 port: '%s'", port);
+      goto done;
+    }
+    in->sin_port = htons((uint16_t)port_num);
+  } else {
+    gpr_log(GPR_ERROR, "no port given for ipv4 scheme");
+    goto done;
+  }
+
+  result = 1;
+done:
+  gpr_free(host);
+  gpr_free(port);
+  return result;
+}
+
+int parse_ipv6(grpc_uri *uri, grpc_resolved_address *resolved_addr) {
+  const char *host_port = uri->path;
+  char *host;
+  char *port;
+  int port_num;
+  int result = 0;
+  struct sockaddr_in6 *in6 = (struct sockaddr_in6 *)resolved_addr->addr;
+
+  if (*host_port == '/') ++host_port;
+  if (!gpr_split_host_port(host_port, &host, &port)) {
+    return 0;
+  }
+
+  memset(in6, 0, sizeof(*in6));
+  resolved_addr->len = sizeof(*in6);
+  in6->sin6_family = AF_INET6;
+
+  /* Handle the RFC6874 syntax for IPv6 zone identifiers. */
+  char *host_end = (char *)gpr_memrchr(host, '%', strlen(host));
+  if (host_end != NULL) {
+    GPR_ASSERT(host_end >= host);
+    char host_without_scope[INET6_ADDRSTRLEN];
+    size_t host_without_scope_len = (size_t)(host_end - host);
+    uint32_t sin6_scope_id = 0;
+    strncpy(host_without_scope, host, host_without_scope_len);
+    host_without_scope[host_without_scope_len] = '\0';
+    if (inet_pton(AF_INET6, host_without_scope, &in6->sin6_addr) == 0) {
+      gpr_log(GPR_ERROR, "invalid ipv6 address: '%s'", host_without_scope);
+      goto done;
+    }
+    if (gpr_parse_bytes_to_uint32(host_end + 1,
+                                  strlen(host) - host_without_scope_len - 1,
+                                  &sin6_scope_id) == 0) {
+      gpr_log(GPR_ERROR, "invalid ipv6 scope id: '%s'", host_end + 1);
+      goto done;
+    }
+    // Handle "sin6_scope_id" being type "u_long". See grpc issue ##10027.
+    in6->sin6_scope_id = sin6_scope_id;
+  } else {
+    if (inet_pton(AF_INET6, host, &in6->sin6_addr) == 0) {
+      gpr_log(GPR_ERROR, "invalid ipv6 address: '%s'", host);
+      goto done;
+    }
+  }
+
+  if (port != NULL) {
+    if (sscanf(port, "%d", &port_num) != 1 || port_num < 0 ||
+        port_num > 65535) {
+      gpr_log(GPR_ERROR, "invalid ipv6 port: '%s'", port);
+      goto done;
+    }
+    in6->sin6_port = htons((uint16_t)port_num);
+  } else {
+    gpr_log(GPR_ERROR, "no port given for ipv6 scheme");
+    goto done;
+  }
+
+  result = 1;
+done:
+  gpr_free(host);
+  gpr_free(port);
+  return result;
+}
diff --git a/src/core/ext/filters/client_channel/parse_address.h b/src/core/ext/filters/client_channel/parse_address.h
new file mode 100644
index 0000000..275a060
--- /dev/null
+++ b/src/core/ext/filters/client_channel/parse_address.h
@@ -0,0 +1,54 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_PARSE_ADDRESS_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_PARSE_ADDRESS_H
+
+#include <stddef.h>
+
+#include "src/core/ext/filters/client_channel/uri_parser.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+
+/** Populate \a addr and \a len from \a uri, whose path is expected to contain a
+ * unix socket path. Returns true upon success. */
+int parse_unix(grpc_uri *uri, grpc_resolved_address *resolved_addr);
+
+/** Populate /a addr and \a len from \a uri, whose path is expected to contain a
+ * host:port pair. Returns true upon success. */
+int parse_ipv4(grpc_uri *uri, grpc_resolved_address *resolved_addr);
+
+/** Populate /a addr and \a len from \a uri, whose path is expected to contain a
+ * host:port pair. Returns true upon success. */
+int parse_ipv6(grpc_uri *uri, grpc_resolved_address *resolved_addr);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_PARSE_ADDRESS_H */
diff --git a/src/core/ext/filters/client_channel/proxy_mapper.c b/src/core/ext/filters/client_channel/proxy_mapper.c
new file mode 100644
index 0000000..6dddd3c
--- /dev/null
+++ b/src/core/ext/filters/client_channel/proxy_mapper.c
@@ -0,0 +1,63 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/proxy_mapper.h"
+
+void grpc_proxy_mapper_init(const grpc_proxy_mapper_vtable* vtable,
+                            grpc_proxy_mapper* mapper) {
+  mapper->vtable = vtable;
+}
+
+bool grpc_proxy_mapper_map_name(grpc_exec_ctx* exec_ctx,
+                                grpc_proxy_mapper* mapper,
+                                const char* server_uri,
+                                const grpc_channel_args* args,
+                                char** name_to_resolve,
+                                grpc_channel_args** new_args) {
+  return mapper->vtable->map_name(exec_ctx, mapper, server_uri, args,
+                                  name_to_resolve, new_args);
+}
+
+bool grpc_proxy_mapper_map_address(grpc_exec_ctx* exec_ctx,
+                                   grpc_proxy_mapper* mapper,
+                                   const grpc_resolved_address* address,
+                                   const grpc_channel_args* args,
+                                   grpc_resolved_address** new_address,
+                                   grpc_channel_args** new_args) {
+  return mapper->vtable->map_address(exec_ctx, mapper, address, args,
+                                     new_address, new_args);
+}
+
+void grpc_proxy_mapper_destroy(grpc_proxy_mapper* mapper) {
+  mapper->vtable->destroy(mapper);
+}
diff --git a/src/core/ext/filters/client_channel/proxy_mapper.h b/src/core/ext/filters/client_channel/proxy_mapper.h
new file mode 100644
index 0000000..6e4607f
--- /dev/null
+++ b/src/core/ext/filters/client_channel/proxy_mapper.h
@@ -0,0 +1,89 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_PROXY_MAPPER_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_PROXY_MAPPER_H
+
+#include <stdbool.h>
+
+#include <grpc/impl/codegen/grpc_types.h>
+
+#include "src/core/lib/iomgr/resolve_address.h"
+
+typedef struct grpc_proxy_mapper grpc_proxy_mapper;
+
+typedef struct {
+  /// Determines the proxy name to resolve for \a server_uri.
+  /// If no proxy is needed, returns false.
+  /// Otherwise, sets \a name_to_resolve, optionally sets \a new_args,
+  /// and returns true.
+  bool (*map_name)(grpc_exec_ctx* exec_ctx, grpc_proxy_mapper* mapper,
+                   const char* server_uri, const grpc_channel_args* args,
+                   char** name_to_resolve, grpc_channel_args** new_args);
+  /// Determines the proxy address to use to contact \a address.
+  /// If no proxy is needed, returns false.
+  /// Otherwise, sets \a new_address, optionally sets \a new_args, and
+  /// returns true.
+  bool (*map_address)(grpc_exec_ctx* exec_ctx, grpc_proxy_mapper* mapper,
+                      const grpc_resolved_address* address,
+                      const grpc_channel_args* args,
+                      grpc_resolved_address** new_address,
+                      grpc_channel_args** new_args);
+  /// Destroys \a mapper.
+  void (*destroy)(grpc_proxy_mapper* mapper);
+} grpc_proxy_mapper_vtable;
+
+struct grpc_proxy_mapper {
+  const grpc_proxy_mapper_vtable* vtable;
+};
+
+void grpc_proxy_mapper_init(const grpc_proxy_mapper_vtable* vtable,
+                            grpc_proxy_mapper* mapper);
+
+bool grpc_proxy_mapper_map_name(grpc_exec_ctx* exec_ctx,
+                                grpc_proxy_mapper* mapper,
+                                const char* server_uri,
+                                const grpc_channel_args* args,
+                                char** name_to_resolve,
+                                grpc_channel_args** new_args);
+
+bool grpc_proxy_mapper_map_address(grpc_exec_ctx* exec_ctx,
+                                   grpc_proxy_mapper* mapper,
+                                   const grpc_resolved_address* address,
+                                   const grpc_channel_args* args,
+                                   grpc_resolved_address** new_address,
+                                   grpc_channel_args** new_args);
+
+void grpc_proxy_mapper_destroy(grpc_proxy_mapper* mapper);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_PROXY_MAPPER_H */
diff --git a/src/core/ext/filters/client_channel/proxy_mapper_registry.c b/src/core/ext/filters/client_channel/proxy_mapper_registry.c
new file mode 100644
index 0000000..7a39289
--- /dev/null
+++ b/src/core/ext/filters/client_channel/proxy_mapper_registry.c
@@ -0,0 +1,139 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/proxy_mapper_registry.h"
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+
+//
+// grpc_proxy_mapper_list
+//
+
+typedef struct {
+  grpc_proxy_mapper** list;
+  size_t num_mappers;
+} grpc_proxy_mapper_list;
+
+static void grpc_proxy_mapper_list_register(grpc_proxy_mapper_list* list,
+                                            bool at_start,
+                                            grpc_proxy_mapper* mapper) {
+  list->list = gpr_realloc(
+      list->list, (list->num_mappers + 1) * sizeof(grpc_proxy_mapper*));
+  if (at_start) {
+    memmove(list->list + 1, list->list,
+            sizeof(grpc_proxy_mapper*) * list->num_mappers);
+    list->list[0] = mapper;
+  } else {
+    list->list[list->num_mappers] = mapper;
+  }
+  ++list->num_mappers;
+}
+
+static bool grpc_proxy_mapper_list_map_name(grpc_exec_ctx* exec_ctx,
+                                            grpc_proxy_mapper_list* list,
+                                            const char* server_uri,
+                                            const grpc_channel_args* args,
+                                            char** name_to_resolve,
+                                            grpc_channel_args** new_args) {
+  for (size_t i = 0; i < list->num_mappers; ++i) {
+    if (grpc_proxy_mapper_map_name(exec_ctx, list->list[i], server_uri, args,
+                                   name_to_resolve, new_args)) {
+      return true;
+    }
+  }
+  return false;
+}
+
+static bool grpc_proxy_mapper_list_map_address(
+    grpc_exec_ctx* exec_ctx, grpc_proxy_mapper_list* list,
+    const grpc_resolved_address* address, const grpc_channel_args* args,
+    grpc_resolved_address** new_address, grpc_channel_args** new_args) {
+  for (size_t i = 0; i < list->num_mappers; ++i) {
+    if (grpc_proxy_mapper_map_address(exec_ctx, list->list[i], address, args,
+                                      new_address, new_args)) {
+      return true;
+    }
+  }
+  return false;
+}
+
+static void grpc_proxy_mapper_list_destroy(grpc_proxy_mapper_list* list) {
+  for (size_t i = 0; i < list->num_mappers; ++i) {
+    grpc_proxy_mapper_destroy(list->list[i]);
+  }
+  gpr_free(list->list);
+  // Clean up in case we re-initialze later.
+  // TODO(ctiller): This should ideally live in
+  // grpc_proxy_mapper_registry_init().  However, if we did this there,
+  // then we would do it AFTER we start registering proxy mappers from
+  // third-party plugins, so they'd never show up (and would leak memory).
+  // We probably need some sort of dependency system for plugins to fix
+  // this.
+  memset(list, 0, sizeof(*list));
+}
+
+//
+// plugin
+//
+
+static grpc_proxy_mapper_list g_proxy_mapper_list;
+
+void grpc_proxy_mapper_registry_init() {}
+
+void grpc_proxy_mapper_registry_shutdown() {
+  grpc_proxy_mapper_list_destroy(&g_proxy_mapper_list);
+}
+
+void grpc_proxy_mapper_register(bool at_start, grpc_proxy_mapper* mapper) {
+  grpc_proxy_mapper_list_register(&g_proxy_mapper_list, at_start, mapper);
+}
+
+bool grpc_proxy_mappers_map_name(grpc_exec_ctx* exec_ctx,
+                                 const char* server_uri,
+                                 const grpc_channel_args* args,
+                                 char** name_to_resolve,
+                                 grpc_channel_args** new_args) {
+  return grpc_proxy_mapper_list_map_name(exec_ctx, &g_proxy_mapper_list,
+                                         server_uri, args, name_to_resolve,
+                                         new_args);
+}
+bool grpc_proxy_mappers_map_address(grpc_exec_ctx* exec_ctx,
+                                    const grpc_resolved_address* address,
+                                    const grpc_channel_args* args,
+                                    grpc_resolved_address** new_address,
+                                    grpc_channel_args** new_args) {
+  return grpc_proxy_mapper_list_map_address(
+      exec_ctx, &g_proxy_mapper_list, address, args, new_address, new_args);
+}
diff --git a/src/core/ext/filters/client_channel/proxy_mapper_registry.h b/src/core/ext/filters/client_channel/proxy_mapper_registry.h
new file mode 100644
index 0000000..ab9168f
--- /dev/null
+++ b/src/core/ext/filters/client_channel/proxy_mapper_registry.h
@@ -0,0 +1,59 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_PROXY_MAPPER_REGISTRY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_PROXY_MAPPER_REGISTRY_H
+
+#include "src/core/ext/filters/client_channel/proxy_mapper.h"
+
+void grpc_proxy_mapper_registry_init();
+void grpc_proxy_mapper_registry_shutdown();
+
+/// Registers a new proxy mapper.  Takes ownership.
+/// If \a at_start is true, the new mapper will be at the beginning of
+/// the list.  Otherwise, it will be added to the end.
+void grpc_proxy_mapper_register(bool at_start, grpc_proxy_mapper* mapper);
+
+bool grpc_proxy_mappers_map_name(grpc_exec_ctx* exec_ctx,
+                                 const char* server_uri,
+                                 const grpc_channel_args* args,
+                                 char** name_to_resolve,
+                                 grpc_channel_args** new_args);
+
+bool grpc_proxy_mappers_map_address(grpc_exec_ctx* exec_ctx,
+                                    const grpc_resolved_address* address,
+                                    const grpc_channel_args* args,
+                                    grpc_resolved_address** new_address,
+                                    grpc_channel_args** new_args);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_PROXY_MAPPER_REGISTRY_H */
diff --git a/src/core/ext/filters/client_channel/resolver.c b/src/core/ext/filters/client_channel/resolver.c
new file mode 100644
index 0000000..cafa283
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver.c
@@ -0,0 +1,88 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/resolver.h"
+#include "src/core/lib/iomgr/combiner.h"
+
+void grpc_resolver_init(grpc_resolver *resolver,
+                        const grpc_resolver_vtable *vtable,
+                        grpc_combiner *combiner) {
+  resolver->vtable = vtable;
+  resolver->combiner = GRPC_COMBINER_REF(combiner, "resolver");
+  gpr_ref_init(&resolver->refs, 1);
+}
+
+#ifdef GRPC_RESOLVER_REFCOUNT_DEBUG
+void grpc_resolver_ref(grpc_resolver *resolver, grpc_closure_list *closure_list,
+                       const char *file, int line, const char *reason) {
+  gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "RESOLVER:%p   ref %d -> %d %s",
+          resolver, (int)resolver->refs.count, (int)resolver->refs.count + 1,
+          reason);
+#else
+void grpc_resolver_ref(grpc_resolver *resolver) {
+#endif
+  gpr_ref(&resolver->refs);
+}
+
+#ifdef GRPC_RESOLVER_REFCOUNT_DEBUG
+void grpc_resolver_unref(grpc_resolver *resolver,
+                         grpc_closure_list *closure_list, const char *file,
+                         int line, const char *reason) {
+  gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "RESOLVER:%p unref %d -> %d %s",
+          resolver, (int)resolver->refs.count, (int)resolver->refs.count - 1,
+          reason);
+#else
+void grpc_resolver_unref(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver) {
+#endif
+  if (gpr_unref(&resolver->refs)) {
+    grpc_combiner *combiner = resolver->combiner;
+    resolver->vtable->destroy(exec_ctx, resolver);
+    GRPC_COMBINER_UNREF(exec_ctx, combiner, "resolver");
+  }
+}
+
+void grpc_resolver_shutdown_locked(grpc_exec_ctx *exec_ctx,
+                                   grpc_resolver *resolver) {
+  resolver->vtable->shutdown_locked(exec_ctx, resolver);
+}
+
+void grpc_resolver_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                            grpc_resolver *resolver) {
+  resolver->vtable->channel_saw_error_locked(exec_ctx, resolver);
+}
+
+void grpc_resolver_next_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver,
+                               grpc_channel_args **result,
+                               grpc_closure *on_complete) {
+  resolver->vtable->next_locked(exec_ctx, resolver, result, on_complete);
+}
diff --git a/src/core/ext/filters/client_channel/resolver.h b/src/core/ext/filters/client_channel/resolver.h
new file mode 100644
index 0000000..31cfa32
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver.h
@@ -0,0 +1,101 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_H
+
+#include "src/core/ext/filters/client_channel/subchannel.h"
+#include "src/core/lib/iomgr/iomgr.h"
+
+typedef struct grpc_resolver grpc_resolver;
+typedef struct grpc_resolver_vtable grpc_resolver_vtable;
+
+/** \a grpc_resolver provides \a grpc_channel_args objects to its caller */
+struct grpc_resolver {
+  const grpc_resolver_vtable *vtable;
+  gpr_refcount refs;
+  grpc_combiner *combiner;
+};
+
+struct grpc_resolver_vtable {
+  void (*destroy)(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver);
+  void (*shutdown_locked)(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver);
+  void (*channel_saw_error_locked)(grpc_exec_ctx *exec_ctx,
+                                   grpc_resolver *resolver);
+  void (*next_locked)(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver,
+                      grpc_channel_args **result, grpc_closure *on_complete);
+};
+
+#ifdef GRPC_RESOLVER_REFCOUNT_DEBUG
+#define GRPC_RESOLVER_REF(p, r) grpc_resolver_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_RESOLVER_UNREF(cl, p, r) \
+  grpc_resolver_unref((cl), (p), __FILE__, __LINE__, (r))
+void grpc_resolver_ref(grpc_resolver *policy, const char *file, int line,
+                       const char *reason);
+void grpc_resolver_unref(grpc_resolver *policy, grpc_closure_list *closure_list,
+                         const char *file, int line, const char *reason);
+#else
+#define GRPC_RESOLVER_REF(p, r) grpc_resolver_ref((p))
+#define GRPC_RESOLVER_UNREF(cl, p, r) grpc_resolver_unref((cl), (p))
+void grpc_resolver_ref(grpc_resolver *policy);
+void grpc_resolver_unref(grpc_exec_ctx *exec_ctx, grpc_resolver *policy);
+#endif
+
+void grpc_resolver_init(grpc_resolver *resolver,
+                        const grpc_resolver_vtable *vtable,
+                        grpc_combiner *combiner);
+
+void grpc_resolver_shutdown_locked(grpc_exec_ctx *exec_ctx,
+                                   grpc_resolver *resolver);
+
+/** Notification that the channel has seen an error on some address.
+    Can be used as a hint that re-resolution is desirable soon.
+
+    Must be called from the combiner passed as a resolver_arg at construction
+    time.*/
+void grpc_resolver_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                            grpc_resolver *resolver);
+
+/** Get the next result from the resolver.  Expected to set \a *result with
+    new channel args and then schedule \a on_complete for execution.
+
+    If resolution is fatally broken, set \a *result to NULL and
+    schedule \a on_complete.
+
+    Must be called from the combiner passed as a resolver_arg at construction
+    time.*/
+void grpc_resolver_next_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver,
+                               grpc_channel_args **result,
+                               grpc_closure *on_complete);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_H */
diff --git a/src/core/ext/filters/client_channel/resolver/README.md b/src/core/ext/filters/client_channel/resolver/README.md
new file mode 100644
index 0000000..b0e234e
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/README.md
@@ -0,0 +1,4 @@
+# Resolver
+
+Implementations of various name resolution schemes.
+See the [naming spec](/doc/naming.md).
diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.c b/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.c
new file mode 100644
index 0000000..80db49f
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.c
@@ -0,0 +1,350 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <grpc/support/port_platform.h>
+#if GRPC_ARES == 1 && !defined(GRPC_UV)
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/host_port.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/http_connect_handshaker.h"
+#include "src/core/ext/filters/client_channel/lb_policy_registry.h"
+#include "src/core/ext/filters/client_channel/resolver_registry.h"
+#include "src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+#include "src/core/lib/iomgr/timer.h"
+#include "src/core/lib/support/backoff.h"
+#include "src/core/lib/support/env.h"
+#include "src/core/lib/support/string.h"
+
+#define GRPC_DNS_MIN_CONNECT_TIMEOUT_SECONDS 1
+#define GRPC_DNS_INITIAL_CONNECT_BACKOFF_SECONDS 1
+#define GRPC_DNS_RECONNECT_BACKOFF_MULTIPLIER 1.6
+#define GRPC_DNS_RECONNECT_MAX_BACKOFF_SECONDS 120
+#define GRPC_DNS_RECONNECT_JITTER 0.2
+
+typedef struct {
+  /** base class: must be first */
+  grpc_resolver base;
+  /** name to resolve (usually the same as target_name) */
+  char *name_to_resolve;
+  /** default port to use */
+  char *default_port;
+  /** channel args. */
+  grpc_channel_args *channel_args;
+  /** pollset_set to drive the name resolution process */
+  grpc_pollset_set *interested_parties;
+
+  /** Closures used by the combiner */
+  grpc_closure dns_ares_on_retry_timer_locked;
+  grpc_closure dns_ares_on_resolved_locked;
+
+  /** Combiner guarding the rest of the state */
+  grpc_combiner *combiner;
+  /** are we currently resolving? */
+  bool resolving;
+  /** which version of the result have we published? */
+  int published_version;
+  /** which version of the result is current? */
+  int resolved_version;
+  /** pending next completion, or NULL */
+  grpc_closure *next_completion;
+  /** target result address for next completion */
+  grpc_channel_args **target_result;
+  /** current (fully resolved) result */
+  grpc_channel_args *resolved_result;
+  /** retry timer */
+  bool have_retry_timer;
+  grpc_timer retry_timer;
+  /** retry backoff state */
+  gpr_backoff backoff_state;
+
+  /** currently resolving addresses */
+  grpc_resolved_addresses *addresses;
+} ares_dns_resolver;
+
+static void dns_ares_destroy(grpc_exec_ctx *exec_ctx, grpc_resolver *r);
+
+static void dns_ares_start_resolving_locked(grpc_exec_ctx *exec_ctx,
+                                            ares_dns_resolver *r);
+static void dns_ares_maybe_finish_next_locked(grpc_exec_ctx *exec_ctx,
+                                              ares_dns_resolver *r);
+
+static void dns_ares_shutdown_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *r);
+static void dns_ares_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                              grpc_resolver *r);
+static void dns_ares_next_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *r,
+                                 grpc_channel_args **target_result,
+                                 grpc_closure *on_complete);
+
+static const grpc_resolver_vtable dns_ares_resolver_vtable = {
+    dns_ares_destroy, dns_ares_shutdown_locked,
+    dns_ares_channel_saw_error_locked, dns_ares_next_locked};
+
+static void dns_ares_shutdown_locked(grpc_exec_ctx *exec_ctx,
+                                     grpc_resolver *resolver) {
+  ares_dns_resolver *r = (ares_dns_resolver *)resolver;
+  if (r->have_retry_timer) {
+    grpc_timer_cancel(exec_ctx, &r->retry_timer);
+  }
+  if (r->next_completion != NULL) {
+    *r->target_result = NULL;
+    grpc_closure_sched(
+        exec_ctx, r->next_completion,
+        GRPC_ERROR_CREATE_FROM_STATIC_STRING("Resolver Shutdown"));
+    r->next_completion = NULL;
+  }
+}
+
+static void dns_ares_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                              grpc_resolver *resolver) {
+  ares_dns_resolver *r = (ares_dns_resolver *)resolver;
+  if (!r->resolving) {
+    gpr_backoff_reset(&r->backoff_state);
+    dns_ares_start_resolving_locked(exec_ctx, r);
+  }
+}
+
+static void dns_ares_on_retry_timer_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                           grpc_error *error) {
+  ares_dns_resolver *r = arg;
+  r->have_retry_timer = false;
+  if (error == GRPC_ERROR_NONE) {
+    if (!r->resolving) {
+      dns_ares_start_resolving_locked(exec_ctx, r);
+    }
+  }
+  GRPC_RESOLVER_UNREF(exec_ctx, &r->base, "retry-timer");
+}
+
+static void dns_ares_on_resolved_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                        grpc_error *error) {
+  ares_dns_resolver *r = arg;
+  grpc_channel_args *result = NULL;
+  GPR_ASSERT(r->resolving);
+  r->resolving = false;
+  if (r->addresses != NULL) {
+    grpc_lb_addresses *addresses = grpc_lb_addresses_create(
+        r->addresses->naddrs, NULL /* user_data_vtable */);
+    for (size_t i = 0; i < r->addresses->naddrs; ++i) {
+      grpc_lb_addresses_set_address(
+          addresses, i, &r->addresses->addrs[i].addr,
+          r->addresses->addrs[i].len, false /* is_balancer */,
+          NULL /* balancer_name */, NULL /* user_data */);
+    }
+    grpc_arg new_arg = grpc_lb_addresses_create_channel_arg(addresses);
+    result = grpc_channel_args_copy_and_add(r->channel_args, &new_arg, 1);
+    grpc_resolved_addresses_destroy(r->addresses);
+    grpc_lb_addresses_destroy(exec_ctx, addresses);
+  } else {
+    gpr_timespec now = gpr_now(GPR_CLOCK_MONOTONIC);
+    gpr_timespec next_try = gpr_backoff_step(&r->backoff_state, now);
+    gpr_timespec timeout = gpr_time_sub(next_try, now);
+    gpr_log(GPR_INFO, "dns resolution failed (will retry): %s",
+            grpc_error_string(error));
+    GPR_ASSERT(!r->have_retry_timer);
+    r->have_retry_timer = true;
+    GRPC_RESOLVER_REF(&r->base, "retry-timer");
+    if (gpr_time_cmp(timeout, gpr_time_0(timeout.clock_type)) > 0) {
+      gpr_log(GPR_DEBUG, "retrying in %" PRId64 ".%09d seconds", timeout.tv_sec,
+              timeout.tv_nsec);
+    } else {
+      gpr_log(GPR_DEBUG, "retrying immediately");
+    }
+    grpc_timer_init(exec_ctx, &r->retry_timer, next_try,
+                    &r->dns_ares_on_retry_timer_locked, now);
+  }
+  if (r->resolved_result != NULL) {
+    grpc_channel_args_destroy(exec_ctx, r->resolved_result);
+  }
+  r->resolved_result = result;
+  r->resolved_version++;
+  dns_ares_maybe_finish_next_locked(exec_ctx, r);
+  GRPC_RESOLVER_UNREF(exec_ctx, &r->base, "dns-resolving");
+}
+
+static void dns_ares_next_locked(grpc_exec_ctx *exec_ctx,
+                                 grpc_resolver *resolver,
+                                 grpc_channel_args **target_result,
+                                 grpc_closure *on_complete) {
+  gpr_log(GPR_DEBUG, "dns_ares_next is called.");
+  ares_dns_resolver *r = (ares_dns_resolver *)resolver;
+  GPR_ASSERT(!r->next_completion);
+  r->next_completion = on_complete;
+  r->target_result = target_result;
+  if (r->resolved_version == 0 && !r->resolving) {
+    gpr_backoff_reset(&r->backoff_state);
+    dns_ares_start_resolving_locked(exec_ctx, r);
+  } else {
+    dns_ares_maybe_finish_next_locked(exec_ctx, r);
+  }
+}
+
+static void dns_ares_start_resolving_locked(grpc_exec_ctx *exec_ctx,
+                                            ares_dns_resolver *r) {
+  GRPC_RESOLVER_REF(&r->base, "dns-resolving");
+  GPR_ASSERT(!r->resolving);
+  r->resolving = true;
+  r->addresses = NULL;
+  grpc_resolve_address(exec_ctx, r->name_to_resolve, r->default_port,
+                       r->interested_parties, &r->dns_ares_on_resolved_locked,
+                       &r->addresses);
+}
+
+static void dns_ares_maybe_finish_next_locked(grpc_exec_ctx *exec_ctx,
+                                              ares_dns_resolver *r) {
+  if (r->next_completion != NULL &&
+      r->resolved_version != r->published_version) {
+    *r->target_result = r->resolved_result == NULL
+                            ? NULL
+                            : grpc_channel_args_copy(r->resolved_result);
+    grpc_closure_sched(exec_ctx, r->next_completion, GRPC_ERROR_NONE);
+    r->next_completion = NULL;
+    r->published_version = r->resolved_version;
+  }
+}
+
+static void dns_ares_destroy(grpc_exec_ctx *exec_ctx, grpc_resolver *gr) {
+  gpr_log(GPR_DEBUG, "dns_ares_destroy");
+  ares_dns_resolver *r = (ares_dns_resolver *)gr;
+  if (r->resolved_result != NULL) {
+    grpc_channel_args_destroy(exec_ctx, r->resolved_result);
+  }
+  grpc_pollset_set_destroy(exec_ctx, r->interested_parties);
+  gpr_free(r->name_to_resolve);
+  gpr_free(r->default_port);
+  grpc_channel_args_destroy(exec_ctx, r->channel_args);
+  gpr_free(r);
+}
+
+static grpc_resolver *dns_ares_create(grpc_exec_ctx *exec_ctx,
+                                      grpc_resolver_args *args,
+                                      const char *default_port) {
+  // Get name from args.
+  const char *path = args->uri->path;
+  if (0 != strcmp(args->uri->authority, "")) {
+    gpr_log(GPR_ERROR, "authority based dns uri's not supported");
+    return NULL;
+  }
+  if (path[0] == '/') ++path;
+  // Create resolver.
+  ares_dns_resolver *r = gpr_zalloc(sizeof(ares_dns_resolver));
+  grpc_resolver_init(&r->base, &dns_ares_resolver_vtable, args->combiner);
+  r->name_to_resolve = gpr_strdup(path);
+  r->default_port = gpr_strdup(default_port);
+  r->channel_args = grpc_channel_args_copy(args->args);
+  r->interested_parties = grpc_pollset_set_create();
+  if (args->pollset_set != NULL) {
+    grpc_pollset_set_add_pollset_set(exec_ctx, r->interested_parties,
+                                     args->pollset_set);
+  }
+  gpr_backoff_init(&r->backoff_state, GRPC_DNS_INITIAL_CONNECT_BACKOFF_SECONDS,
+                   GRPC_DNS_RECONNECT_BACKOFF_MULTIPLIER,
+                   GRPC_DNS_RECONNECT_JITTER,
+                   GRPC_DNS_MIN_CONNECT_TIMEOUT_SECONDS * 1000,
+                   GRPC_DNS_RECONNECT_MAX_BACKOFF_SECONDS * 1000);
+  grpc_closure_init(&r->dns_ares_on_retry_timer_locked,
+                    dns_ares_on_retry_timer_locked, r,
+                    grpc_combiner_scheduler(r->base.combiner, false));
+  grpc_closure_init(&r->dns_ares_on_resolved_locked,
+                    dns_ares_on_resolved_locked, r,
+                    grpc_combiner_scheduler(r->base.combiner, false));
+  return &r->base;
+}
+
+/*
+ * FACTORY
+ */
+
+static void dns_ares_factory_ref(grpc_resolver_factory *factory) {}
+
+static void dns_ares_factory_unref(grpc_resolver_factory *factory) {}
+
+static grpc_resolver *dns_factory_create_resolver(
+    grpc_exec_ctx *exec_ctx, grpc_resolver_factory *factory,
+    grpc_resolver_args *args) {
+  return dns_ares_create(exec_ctx, args, "https");
+}
+
+static char *dns_ares_factory_get_default_host_name(
+    grpc_resolver_factory *factory, grpc_uri *uri) {
+  const char *path = uri->path;
+  if (path[0] == '/') ++path;
+  return gpr_strdup(path);
+}
+
+static const grpc_resolver_factory_vtable dns_ares_factory_vtable = {
+    dns_ares_factory_ref, dns_ares_factory_unref, dns_factory_create_resolver,
+    dns_ares_factory_get_default_host_name, "dns"};
+static grpc_resolver_factory dns_resolver_factory = {&dns_ares_factory_vtable};
+
+static grpc_resolver_factory *dns_ares_resolver_factory_create() {
+  return &dns_resolver_factory;
+}
+
+void grpc_resolver_dns_ares_init(void) {
+  char *resolver = gpr_getenv("GRPC_DNS_RESOLVER");
+  /* TODO(zyc): Turn on c-ares based resolver by default after the address
+     sorter and the CNAME support are added. */
+  if (resolver != NULL && gpr_stricmp(resolver, "ares") == 0) {
+    grpc_error *error = grpc_ares_init();
+    if (error != GRPC_ERROR_NONE) {
+      GRPC_LOG_IF_ERROR("ares_library_init() failed", error);
+      return;
+    }
+    grpc_resolve_address = grpc_resolve_address_ares;
+    grpc_register_resolver_type(dns_ares_resolver_factory_create());
+  }
+  gpr_free(resolver);
+}
+
+void grpc_resolver_dns_ares_shutdown(void) {
+  char *resolver = gpr_getenv("GRPC_DNS_RESOLVER");
+  if (resolver != NULL && gpr_stricmp(resolver, "ares") == 0) {
+    grpc_ares_cleanup();
+  }
+  gpr_free(resolver);
+}
+
+#else /* GRPC_ARES == 1 && !defined(GRPC_UV) */
+
+void grpc_resolver_dns_ares_init(void) {}
+
+void grpc_resolver_dns_ares_shutdown(void) {}
+
+#endif /* GRPC_ARES == 1 && !defined(GRPC_UV) */
diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.h b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.h
new file mode 100644
index 0000000..334feaa
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.h
@@ -0,0 +1,65 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_RESOLVER_DNS_C_ARES_GRPC_ARES_EV_DRIVER_H
+#define GRPC_CORE_EXT_RESOLVER_DNS_C_ARES_GRPC_ARES_EV_DRIVER_H
+
+#include <ares.h>
+
+#include "src/core/lib/iomgr/exec_ctx.h"
+#include "src/core/lib/iomgr/pollset_set.h"
+
+typedef struct grpc_ares_ev_driver grpc_ares_ev_driver;
+
+/* Start \a ev_driver. It will keep working until all IO on its ares_channel is
+   done, or grpc_ares_ev_driver_destroy() is called. It may notify the callbacks
+   bound to its ares_channel when necessary. */
+void grpc_ares_ev_driver_start(grpc_exec_ctx *exec_ctx,
+                               grpc_ares_ev_driver *ev_driver);
+
+/* Returns the ares_channel owned by \a ev_driver. To bind a c-ares query to
+   \a ev_driver, use the ares_channel owned by \a ev_driver as the arg of the
+   query. */
+ares_channel *grpc_ares_ev_driver_get_channel(grpc_ares_ev_driver *ev_driver);
+
+/* Creates a new grpc_ares_ev_driver. Returns GRPC_ERROR_NONE if \a ev_driver is
+   created successfully. */
+grpc_error *grpc_ares_ev_driver_create(grpc_ares_ev_driver **ev_driver,
+                                       grpc_pollset_set *pollset_set);
+
+/* Destroys \a ev_driver asynchronously. Pending lookups made on \a ev_driver
+   will be cancelled and their on_done callbacks will be invoked with a status
+   of ARES_ECANCELLED. */
+void grpc_ares_ev_driver_destroy(grpc_ares_ev_driver *ev_driver);
+
+#endif /* GRPC_CORE_EXT_RESOLVER_DNS_C_ARES_GRPC_ARES_EV_DRIVER_H */
diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.c b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.c
new file mode 100644
index 0000000..1e4f3eb
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.c
@@ -0,0 +1,319 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+#include <grpc/support/port_platform.h>
+#include "src/core/lib/iomgr/port.h"
+#if GRPC_ARES == 1 && defined(GRPC_POSIX_SOCKET)
+
+#include "src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.h"
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/time.h>
+#include <grpc/support/useful.h>
+#include "src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h"
+#include "src/core/lib/iomgr/ev_posix.h"
+#include "src/core/lib/iomgr/iomgr_internal.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/support/string.h"
+
+typedef struct fd_node {
+  /** the owner of this fd node */
+  grpc_ares_ev_driver *ev_driver;
+  /** the grpc_fd owned by this fd node */
+  grpc_fd *grpc_fd;
+  /** a closure wrapping on_readable_cb, which should be invoked when the
+      grpc_fd in this node becomes readable. */
+  grpc_closure read_closure;
+  /** a closure wrapping on_writable_cb, which should be invoked when the
+      grpc_fd in this node becomes writable. */
+  grpc_closure write_closure;
+  /** next fd node in the list */
+  struct fd_node *next;
+
+  /** mutex guarding the rest of the state */
+  gpr_mu mu;
+  /** if the readable closure has been registered */
+  bool readable_registered;
+  /** if the writable closure has been registered */
+  bool writable_registered;
+} fd_node;
+
+struct grpc_ares_ev_driver {
+  /** the ares_channel owned by this event driver */
+  ares_channel channel;
+  /** pollset set for driving the IO events of the channel */
+  grpc_pollset_set *pollset_set;
+  /** refcount of the event driver */
+  gpr_refcount refs;
+
+  /** mutex guarding the rest of the state */
+  gpr_mu mu;
+  /** a list of grpc_fd that this event driver is currently using. */
+  fd_node *fds;
+  /** is this event driver currently working? */
+  bool working;
+  /** is this event driver being shut down */
+  bool shutting_down;
+};
+
+static void grpc_ares_notify_on_event_locked(grpc_exec_ctx *exec_ctx,
+                                             grpc_ares_ev_driver *ev_driver);
+
+static grpc_ares_ev_driver *grpc_ares_ev_driver_ref(
+    grpc_ares_ev_driver *ev_driver) {
+  gpr_log(GPR_DEBUG, "Ref ev_driver %" PRIuPTR, (uintptr_t)ev_driver);
+  gpr_ref(&ev_driver->refs);
+  return ev_driver;
+}
+
+static void grpc_ares_ev_driver_unref(grpc_ares_ev_driver *ev_driver) {
+  gpr_log(GPR_DEBUG, "Unref ev_driver %" PRIuPTR, (uintptr_t)ev_driver);
+  if (gpr_unref(&ev_driver->refs)) {
+    gpr_log(GPR_DEBUG, "destroy ev_driver %" PRIuPTR, (uintptr_t)ev_driver);
+    GPR_ASSERT(ev_driver->fds == NULL);
+    gpr_mu_destroy(&ev_driver->mu);
+    ares_destroy(ev_driver->channel);
+    gpr_free(ev_driver);
+  }
+}
+
+static void fd_node_destroy(grpc_exec_ctx *exec_ctx, fd_node *fdn) {
+  gpr_log(GPR_DEBUG, "delete fd: %d", grpc_fd_wrapped_fd(fdn->grpc_fd));
+  GPR_ASSERT(!fdn->readable_registered);
+  GPR_ASSERT(!fdn->writable_registered);
+  gpr_mu_destroy(&fdn->mu);
+  grpc_pollset_set_del_fd(exec_ctx, fdn->ev_driver->pollset_set, fdn->grpc_fd);
+  grpc_fd_shutdown(exec_ctx, fdn->grpc_fd,
+                   GRPC_ERROR_CREATE_FROM_STATIC_STRING("fd node destroyed"));
+  grpc_fd_orphan(exec_ctx, fdn->grpc_fd, NULL, NULL, "c-ares query finished");
+  gpr_free(fdn);
+}
+
+grpc_error *grpc_ares_ev_driver_create(grpc_ares_ev_driver **ev_driver,
+                                       grpc_pollset_set *pollset_set) {
+  *ev_driver = gpr_malloc(sizeof(grpc_ares_ev_driver));
+  int status = ares_init(&(*ev_driver)->channel);
+  gpr_log(GPR_DEBUG, "grpc_ares_ev_driver_create");
+  if (status != ARES_SUCCESS) {
+    char *err_msg;
+    gpr_asprintf(&err_msg, "Failed to init ares channel. C-ares error: %s",
+                 ares_strerror(status));
+    grpc_error *err = GRPC_ERROR_CREATE_FROM_COPIED_STRING(err_msg);
+    gpr_free(err_msg);
+    gpr_free(*ev_driver);
+    return err;
+  }
+  gpr_mu_init(&(*ev_driver)->mu);
+  gpr_ref_init(&(*ev_driver)->refs, 1);
+  (*ev_driver)->pollset_set = pollset_set;
+  (*ev_driver)->fds = NULL;
+  (*ev_driver)->working = false;
+  (*ev_driver)->shutting_down = false;
+  return GRPC_ERROR_NONE;
+}
+
+void grpc_ares_ev_driver_destroy(grpc_ares_ev_driver *ev_driver) {
+  // It's not safe to shut down remaining fds here directly, becauses
+  // ares_host_callback does not provide an exec_ctx. We mark the event driver
+  // as being shut down. If the event driver is working,
+  // grpc_ares_notify_on_event_locked will shut down the fds; if it's not
+  // working, there are no fds to shut down.
+  gpr_mu_lock(&ev_driver->mu);
+  ev_driver->shutting_down = true;
+  gpr_mu_unlock(&ev_driver->mu);
+  grpc_ares_ev_driver_unref(ev_driver);
+}
+
+// Search fd in the fd_node list head. This is an O(n) search, the max possible
+// value of n is ARES_GETSOCK_MAXNUM (16). n is typically 1 - 2 in our tests.
+static fd_node *pop_fd_node(fd_node **head, int fd) {
+  fd_node dummy_head;
+  dummy_head.next = *head;
+  fd_node *node = &dummy_head;
+  while (node->next != NULL) {
+    if (grpc_fd_wrapped_fd(node->next->grpc_fd) == fd) {
+      fd_node *ret = node->next;
+      node->next = node->next->next;
+      *head = dummy_head.next;
+      return ret;
+    }
+    node = node->next;
+  }
+  return NULL;
+}
+
+static void on_readable_cb(grpc_exec_ctx *exec_ctx, void *arg,
+                           grpc_error *error) {
+  fd_node *fdn = arg;
+  grpc_ares_ev_driver *ev_driver = fdn->ev_driver;
+  gpr_mu_lock(&fdn->mu);
+  fdn->readable_registered = false;
+  gpr_mu_unlock(&fdn->mu);
+
+  gpr_log(GPR_DEBUG, "readable on %d", grpc_fd_wrapped_fd(fdn->grpc_fd));
+  if (error == GRPC_ERROR_NONE) {
+    ares_process_fd(ev_driver->channel, grpc_fd_wrapped_fd(fdn->grpc_fd),
+                    ARES_SOCKET_BAD);
+  } else {
+    // If error is not GRPC_ERROR_NONE, it means the fd has been shutdown or
+    // timed out. The pending lookups made on this ev_driver will be cancelled
+    // by the following ares_cancel() and the on_done callbacks will be invoked
+    // with a status of ARES_ECANCELLED. The remaining file descriptors in this
+    // ev_driver will be cleaned up in the follwing
+    // grpc_ares_notify_on_event_locked().
+    ares_cancel(ev_driver->channel);
+  }
+  gpr_mu_lock(&ev_driver->mu);
+  grpc_ares_notify_on_event_locked(exec_ctx, ev_driver);
+  gpr_mu_unlock(&ev_driver->mu);
+  grpc_ares_ev_driver_unref(ev_driver);
+}
+
+static void on_writable_cb(grpc_exec_ctx *exec_ctx, void *arg,
+                           grpc_error *error) {
+  fd_node *fdn = arg;
+  grpc_ares_ev_driver *ev_driver = fdn->ev_driver;
+  gpr_mu_lock(&fdn->mu);
+  fdn->writable_registered = false;
+  gpr_mu_unlock(&fdn->mu);
+
+  gpr_log(GPR_DEBUG, "writable on %d", grpc_fd_wrapped_fd(fdn->grpc_fd));
+  if (error == GRPC_ERROR_NONE) {
+    ares_process_fd(ev_driver->channel, ARES_SOCKET_BAD,
+                    grpc_fd_wrapped_fd(fdn->grpc_fd));
+  } else {
+    // If error is not GRPC_ERROR_NONE, it means the fd has been shutdown or
+    // timed out. The pending lookups made on this ev_driver will be cancelled
+    // by the following ares_cancel() and the on_done callbacks will be invoked
+    // with a status of ARES_ECANCELLED. The remaining file descriptors in this
+    // ev_driver will be cleaned up in the follwing
+    // grpc_ares_notify_on_event_locked().
+    ares_cancel(ev_driver->channel);
+  }
+  gpr_mu_lock(&ev_driver->mu);
+  grpc_ares_notify_on_event_locked(exec_ctx, ev_driver);
+  gpr_mu_unlock(&ev_driver->mu);
+  grpc_ares_ev_driver_unref(ev_driver);
+}
+
+ares_channel *grpc_ares_ev_driver_get_channel(grpc_ares_ev_driver *ev_driver) {
+  return &ev_driver->channel;
+}
+
+// Get the file descriptors used by the ev_driver's ares channel, register
+// driver_closure with these filedescriptors.
+static void grpc_ares_notify_on_event_locked(grpc_exec_ctx *exec_ctx,
+                                             grpc_ares_ev_driver *ev_driver) {
+  fd_node *new_list = NULL;
+  if (!ev_driver->shutting_down) {
+    ares_socket_t socks[ARES_GETSOCK_MAXNUM];
+    int socks_bitmask =
+        ares_getsock(ev_driver->channel, socks, ARES_GETSOCK_MAXNUM);
+    for (size_t i = 0; i < ARES_GETSOCK_MAXNUM; i++) {
+      if (ARES_GETSOCK_READABLE(socks_bitmask, i) ||
+          ARES_GETSOCK_WRITABLE(socks_bitmask, i)) {
+        fd_node *fdn = pop_fd_node(&ev_driver->fds, socks[i]);
+        // Create a new fd_node if sock[i] is not in the fd_node list.
+        if (fdn == NULL) {
+          char *fd_name;
+          gpr_asprintf(&fd_name, "ares_ev_driver-%" PRIuPTR, i);
+          fdn = gpr_malloc(sizeof(fd_node));
+          gpr_log(GPR_DEBUG, "new fd: %d", socks[i]);
+          fdn->grpc_fd = grpc_fd_create(socks[i], fd_name);
+          fdn->ev_driver = ev_driver;
+          fdn->readable_registered = false;
+          fdn->writable_registered = false;
+          gpr_mu_init(&fdn->mu);
+          grpc_closure_init(&fdn->read_closure, on_readable_cb, fdn,
+                            grpc_schedule_on_exec_ctx);
+          grpc_closure_init(&fdn->write_closure, on_writable_cb, fdn,
+                            grpc_schedule_on_exec_ctx);
+          grpc_pollset_set_add_fd(exec_ctx, ev_driver->pollset_set,
+                                  fdn->grpc_fd);
+          gpr_free(fd_name);
+        }
+        fdn->next = new_list;
+        new_list = fdn;
+        gpr_mu_lock(&fdn->mu);
+        // Register read_closure if the socket is readable and read_closure has
+        // not been registered with this socket.
+        if (ARES_GETSOCK_READABLE(socks_bitmask, i) &&
+            !fdn->readable_registered) {
+          grpc_ares_ev_driver_ref(ev_driver);
+          gpr_log(GPR_DEBUG, "notify read on: %d",
+                  grpc_fd_wrapped_fd(fdn->grpc_fd));
+          grpc_fd_notify_on_read(exec_ctx, fdn->grpc_fd, &fdn->read_closure);
+          fdn->readable_registered = true;
+        }
+        // Register write_closure if the socket is writable and write_closure
+        // has not been registered with this socket.
+        if (ARES_GETSOCK_WRITABLE(socks_bitmask, i) &&
+            !fdn->writable_registered) {
+          gpr_log(GPR_DEBUG, "notify write on: %d",
+                  grpc_fd_wrapped_fd(fdn->grpc_fd));
+          grpc_ares_ev_driver_ref(ev_driver);
+          grpc_fd_notify_on_write(exec_ctx, fdn->grpc_fd, &fdn->write_closure);
+          fdn->writable_registered = true;
+        }
+        gpr_mu_unlock(&fdn->mu);
+      }
+    }
+  }
+  // Any remaining fds in ev_driver->fds were not returned by ares_getsock() and
+  // are therefore no longer in use, so they can be shut down and removed from
+  // the list.
+  while (ev_driver->fds != NULL) {
+    fd_node *cur = ev_driver->fds;
+    ev_driver->fds = ev_driver->fds->next;
+    fd_node_destroy(exec_ctx, cur);
+  }
+  ev_driver->fds = new_list;
+  // If the ev driver has no working fd, all the tasks are done.
+  if (new_list == NULL) {
+    ev_driver->working = false;
+    gpr_log(GPR_DEBUG, "ev driver stop working");
+  }
+}
+
+void grpc_ares_ev_driver_start(grpc_exec_ctx *exec_ctx,
+                               grpc_ares_ev_driver *ev_driver) {
+  gpr_mu_lock(&ev_driver->mu);
+  if (!ev_driver->working) {
+    ev_driver->working = true;
+    grpc_ares_notify_on_event_locked(exec_ctx, ev_driver);
+  }
+  gpr_mu_unlock(&ev_driver->mu);
+}
+
+#endif /* GRPC_ARES == 1 && defined(GRPC_POSIX_SOCKET) */
diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.c b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.c
new file mode 100644
index 0000000..09c46a6
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.c
@@ -0,0 +1,289 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <grpc/support/port_platform.h>
+#if GRPC_ARES == 1 && !defined(GRPC_UV)
+
+#include "src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h"
+#include "src/core/lib/iomgr/sockaddr.h"
+#include "src/core/lib/iomgr/socket_utils_posix.h"
+
+#include <string.h>
+#include <sys/types.h>
+
+#include <ares.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/host_port.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/time.h>
+#include <grpc/support/useful.h>
+#include "src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.h"
+#include "src/core/lib/iomgr/executor.h"
+#include "src/core/lib/iomgr/iomgr_internal.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/support/string.h"
+
+static gpr_once g_basic_init = GPR_ONCE_INIT;
+static gpr_mu g_init_mu;
+
+typedef struct grpc_ares_request {
+  /** following members are set in grpc_resolve_address_ares_impl */
+  /** host to resolve, parsed from the name to resolve */
+  char *host;
+  /** port to fill in sockaddr_in, parsed from the name to resolve */
+  char *port;
+  /** default port to use */
+  char *default_port;
+  /** closure to call when the request completes */
+  grpc_closure *on_done;
+  /** the pointer to receive the resolved addresses */
+  grpc_resolved_addresses **addrs_out;
+  /** the evernt driver used by this request */
+  grpc_ares_ev_driver *ev_driver;
+  /** number of ongoing queries */
+  gpr_refcount pending_queries;
+
+  /** mutex guarding the rest of the state */
+  gpr_mu mu;
+  /** is there at least one successful query, set in on_done_cb */
+  bool success;
+  /** the errors explaining the request failure, set in on_done_cb */
+  grpc_error *error;
+} grpc_ares_request;
+
+static void do_basic_init(void) { gpr_mu_init(&g_init_mu); }
+
+static uint16_t strhtons(const char *port) {
+  if (strcmp(port, "http") == 0) {
+    return htons(80);
+  } else if (strcmp(port, "https") == 0) {
+    return htons(443);
+  }
+  return htons((unsigned short)atoi(port));
+}
+
+static void grpc_ares_request_unref(grpc_exec_ctx *exec_ctx,
+                                    grpc_ares_request *r) {
+  /* If there are no pending queries, invoke on_done callback and destroy the
+     request */
+  if (gpr_unref(&r->pending_queries)) {
+    /* TODO(zyc): Sort results with RFC6724 before invoking on_done. */
+    if (exec_ctx == NULL) {
+      /* A new exec_ctx is created here, as the c-ares interface does not
+         provide one in ares_host_callback. It's safe to schedule on_done with
+         the newly created exec_ctx, since the caller has been warned not to
+         acquire locks in on_done. ares_dns_resolver is using combiner to
+         protect resources needed by on_done. */
+      grpc_exec_ctx new_exec_ctx = GRPC_EXEC_CTX_INIT;
+      grpc_closure_sched(&new_exec_ctx, r->on_done, r->error);
+      grpc_exec_ctx_finish(&new_exec_ctx);
+    } else {
+      grpc_closure_sched(exec_ctx, r->on_done, r->error);
+    }
+    gpr_mu_destroy(&r->mu);
+    grpc_ares_ev_driver_destroy(r->ev_driver);
+    gpr_free(r->host);
+    gpr_free(r->port);
+    gpr_free(r->default_port);
+    gpr_free(r);
+  }
+}
+
+static void on_done_cb(void *arg, int status, int timeouts,
+                       struct hostent *hostent) {
+  grpc_ares_request *r = (grpc_ares_request *)arg;
+  gpr_mu_lock(&r->mu);
+  if (status == ARES_SUCCESS) {
+    GRPC_ERROR_UNREF(r->error);
+    r->error = GRPC_ERROR_NONE;
+    r->success = true;
+    grpc_resolved_addresses **addresses = r->addrs_out;
+    if (*addresses == NULL) {
+      *addresses = gpr_malloc(sizeof(grpc_resolved_addresses));
+      (*addresses)->naddrs = 0;
+      (*addresses)->addrs = NULL;
+    }
+    size_t prev_naddr = (*addresses)->naddrs;
+    size_t i;
+    for (i = 0; hostent->h_addr_list[i] != NULL; i++) {
+    }
+    (*addresses)->naddrs += i;
+    (*addresses)->addrs =
+        gpr_realloc((*addresses)->addrs,
+                    sizeof(grpc_resolved_address) * (*addresses)->naddrs);
+    for (i = prev_naddr; i < (*addresses)->naddrs; i++) {
+      memset(&(*addresses)->addrs[i], 0, sizeof(grpc_resolved_address));
+      if (hostent->h_addrtype == AF_INET6) {
+        (*addresses)->addrs[i].len = sizeof(struct sockaddr_in6);
+        struct sockaddr_in6 *addr =
+            (struct sockaddr_in6 *)&(*addresses)->addrs[i].addr;
+        addr->sin6_family = (sa_family_t)hostent->h_addrtype;
+        addr->sin6_port = strhtons(r->port);
+
+        char output[INET6_ADDRSTRLEN];
+        memcpy(&addr->sin6_addr, hostent->h_addr_list[i - prev_naddr],
+               sizeof(struct in6_addr));
+        ares_inet_ntop(AF_INET6, &addr->sin6_addr, output, INET6_ADDRSTRLEN);
+        gpr_log(GPR_DEBUG,
+                "c-ares resolver gets a AF_INET6 result: \n"
+                "  addr: %s\n  port: %s\n  sin6_scope_id: %d\n",
+                output, r->port, addr->sin6_scope_id);
+      } else {
+        (*addresses)->addrs[i].len = sizeof(struct sockaddr_in);
+        struct sockaddr_in *addr =
+            (struct sockaddr_in *)&(*addresses)->addrs[i].addr;
+        memcpy(&addr->sin_addr, hostent->h_addr_list[i - prev_naddr],
+               sizeof(struct in_addr));
+        addr->sin_family = (sa_family_t)hostent->h_addrtype;
+        addr->sin_port = strhtons(r->port);
+
+        char output[INET_ADDRSTRLEN];
+        ares_inet_ntop(AF_INET, &addr->sin_addr, output, INET_ADDRSTRLEN);
+        gpr_log(GPR_DEBUG,
+                "c-ares resolver gets a AF_INET result: \n"
+                "  addr: %s\n  port: %s\n",
+                output, r->port);
+      }
+    }
+  } else if (!r->success) {
+    char *error_msg;
+    gpr_asprintf(&error_msg, "C-ares status is not ARES_SUCCESS: %s",
+                 ares_strerror(status));
+    grpc_error *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+    gpr_free(error_msg);
+    if (r->error == GRPC_ERROR_NONE) {
+      r->error = error;
+    } else {
+      r->error = grpc_error_add_child(error, r->error);
+    }
+  }
+  gpr_mu_unlock(&r->mu);
+  grpc_ares_request_unref(NULL, r);
+}
+
+void grpc_resolve_address_ares_impl(grpc_exec_ctx *exec_ctx, const char *name,
+                                    const char *default_port,
+                                    grpc_pollset_set *interested_parties,
+                                    grpc_closure *on_done,
+                                    grpc_resolved_addresses **addrs) {
+  /* TODO(zyc): Enable tracing after #9603 is checked in */
+  /* if (grpc_dns_trace) {
+      gpr_log(GPR_DEBUG, "resolve_address (blocking): name=%s, default_port=%s",
+              name, default_port);
+     } */
+
+  /* parse name, splitting it into host and port parts */
+  char *host;
+  char *port;
+  gpr_split_host_port(name, &host, &port);
+  if (host == NULL) {
+    grpc_error *err = grpc_error_set_str(
+        GRPC_ERROR_CREATE_FROM_STATIC_STRING("unparseable host:port"),
+        GRPC_ERROR_STR_TARGET_ADDRESS, grpc_slice_from_copied_string(name));
+    grpc_closure_sched(exec_ctx, on_done, err);
+    goto error_cleanup;
+  } else if (port == NULL) {
+    if (default_port == NULL) {
+      grpc_error *err = grpc_error_set_str(
+          GRPC_ERROR_CREATE_FROM_STATIC_STRING("no port in name"),
+          GRPC_ERROR_STR_TARGET_ADDRESS, grpc_slice_from_copied_string(name));
+      grpc_closure_sched(exec_ctx, on_done, err);
+      goto error_cleanup;
+    }
+    port = gpr_strdup(default_port);
+  }
+
+  grpc_ares_ev_driver *ev_driver;
+  grpc_error *err = grpc_ares_ev_driver_create(&ev_driver, interested_parties);
+  if (err != GRPC_ERROR_NONE) {
+    GRPC_LOG_IF_ERROR("grpc_ares_ev_driver_create() failed", err);
+    goto error_cleanup;
+  }
+
+  grpc_ares_request *r = gpr_malloc(sizeof(grpc_ares_request));
+  gpr_mu_init(&r->mu);
+  r->ev_driver = ev_driver;
+  r->on_done = on_done;
+  r->addrs_out = addrs;
+  r->default_port = gpr_strdup(default_port);
+  r->port = port;
+  r->host = host;
+  r->success = false;
+  r->error = GRPC_ERROR_NONE;
+  ares_channel *channel = grpc_ares_ev_driver_get_channel(r->ev_driver);
+  gpr_ref_init(&r->pending_queries, 2);
+  if (grpc_ipv6_loopback_available()) {
+    gpr_ref(&r->pending_queries);
+    ares_gethostbyname(*channel, r->host, AF_INET6, on_done_cb, r);
+  }
+  ares_gethostbyname(*channel, r->host, AF_INET, on_done_cb, r);
+  /* TODO(zyc): Handle CNAME records here. */
+  grpc_ares_ev_driver_start(exec_ctx, r->ev_driver);
+  grpc_ares_request_unref(exec_ctx, r);
+  return;
+
+error_cleanup:
+  gpr_free(host);
+  gpr_free(port);
+}
+
+void (*grpc_resolve_address_ares)(
+    grpc_exec_ctx *exec_ctx, const char *name, const char *default_port,
+    grpc_pollset_set *interested_parties, grpc_closure *on_done,
+    grpc_resolved_addresses **addrs) = grpc_resolve_address_ares_impl;
+
+grpc_error *grpc_ares_init(void) {
+  gpr_once_init(&g_basic_init, do_basic_init);
+  gpr_mu_lock(&g_init_mu);
+  int status = ares_library_init(ARES_LIB_INIT_ALL);
+  gpr_mu_unlock(&g_init_mu);
+
+  if (status != ARES_SUCCESS) {
+    char *error_msg;
+    gpr_asprintf(&error_msg, "ares_library_init failed: %s",
+                 ares_strerror(status));
+    grpc_error *error = GRPC_ERROR_CREATE_FROM_COPIED_STRING(error_msg);
+    gpr_free(error_msg);
+    return error;
+  }
+  return GRPC_ERROR_NONE;
+}
+
+void grpc_ares_cleanup(void) {
+  gpr_mu_lock(&g_init_mu);
+  ares_library_cleanup();
+  gpr_mu_unlock(&g_init_mu);
+}
+
+#endif /* GRPC_ARES == 1 && !defined(GRPC_UV) */
diff --git a/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h
new file mode 100644
index 0000000..ab00a26
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.h
@@ -0,0 +1,63 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_RESOLVER_DNS_C_ARES_GRPC_ARES_WRAPPER_H
+#define GRPC_CORE_EXT_RESOLVER_DNS_C_ARES_GRPC_ARES_WRAPPER_H
+
+#include "src/core/lib/iomgr/exec_ctx.h"
+#include "src/core/lib/iomgr/iomgr.h"
+#include "src/core/lib/iomgr/polling_entity.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+
+/* Asynchronously resolve addr. Use \a default_port if a port isn't designated
+   in addr, otherwise use the port in addr. grpc_ares_init() must be called at
+   least once before this function. \a on_done may be called directly in this
+   function without being scheduled with \a exec_ctx, it must not try to acquire
+   locks that are being held by the caller. */
+extern void (*grpc_resolve_address_ares)(grpc_exec_ctx *exec_ctx,
+                                         const char *addr,
+                                         const char *default_port,
+                                         grpc_pollset_set *interested_parties,
+                                         grpc_closure *on_done,
+                                         grpc_resolved_addresses **addresses);
+
+/* Initialize gRPC ares wrapper. Must be called at least once before
+   grpc_resolve_address_ares(). */
+grpc_error *grpc_ares_init(void);
+
+/* Uninitialized gRPC ares wrapper. If there was more than one previous call to
+   grpc_ares_init(), this function uninitializes the gRPC ares wrapper only if
+   it has been called the same number of times as grpc_ares_init(). */
+void grpc_ares_cleanup(void);
+
+#endif /* GRPC_CORE_EXT_RESOLVER_DNS_C_ARES_GRPC_ARES_WRAPPER_H */
diff --git a/src/core/ext/filters/client_channel/resolver/dns/native/README.md b/src/core/ext/filters/client_channel/resolver/dns/native/README.md
new file mode 100644
index 0000000..695de47
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/native/README.md
@@ -0,0 +1,2 @@
+dns: scheme name resolution, using getaddrbyname
+(or other OS specific implementation)
diff --git a/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.c b/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.c
new file mode 100644
index 0000000..ebe6a07
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.c
@@ -0,0 +1,325 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/host_port.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/lb_policy_registry.h"
+#include "src/core/ext/filters/client_channel/resolver_registry.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+#include "src/core/lib/iomgr/timer.h"
+#include "src/core/lib/support/backoff.h"
+#include "src/core/lib/support/env.h"
+#include "src/core/lib/support/string.h"
+
+#define GRPC_DNS_MIN_CONNECT_TIMEOUT_SECONDS 1
+#define GRPC_DNS_INITIAL_CONNECT_BACKOFF_SECONDS 1
+#define GRPC_DNS_RECONNECT_BACKOFF_MULTIPLIER 1.6
+#define GRPC_DNS_RECONNECT_MAX_BACKOFF_SECONDS 120
+#define GRPC_DNS_RECONNECT_JITTER 0.2
+
+typedef struct {
+  /** base class: must be first */
+  grpc_resolver base;
+  /** name to resolve */
+  char *name_to_resolve;
+  /** default port to use */
+  char *default_port;
+  /** channel args. */
+  grpc_channel_args *channel_args;
+  /** pollset_set to drive the name resolution process */
+  grpc_pollset_set *interested_parties;
+
+  /** are we currently resolving? */
+  bool resolving;
+  /** which version of the result have we published? */
+  int published_version;
+  /** which version of the result is current? */
+  int resolved_version;
+  /** pending next completion, or NULL */
+  grpc_closure *next_completion;
+  /** target result address for next completion */
+  grpc_channel_args **target_result;
+  /** current (fully resolved) result */
+  grpc_channel_args *resolved_result;
+  /** retry timer */
+  bool have_retry_timer;
+  grpc_timer retry_timer;
+  grpc_closure on_retry;
+  /** retry backoff state */
+  gpr_backoff backoff_state;
+
+  /** currently resolving addresses */
+  grpc_resolved_addresses *addresses;
+} dns_resolver;
+
+static void dns_destroy(grpc_exec_ctx *exec_ctx, grpc_resolver *r);
+
+static void dns_start_resolving_locked(grpc_exec_ctx *exec_ctx,
+                                       dns_resolver *r);
+static void dns_maybe_finish_next_locked(grpc_exec_ctx *exec_ctx,
+                                         dns_resolver *r);
+
+static void dns_shutdown_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *r);
+static void dns_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                         grpc_resolver *r);
+static void dns_next_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *r,
+                            grpc_channel_args **target_result,
+                            grpc_closure *on_complete);
+
+static const grpc_resolver_vtable dns_resolver_vtable = {
+    dns_destroy, dns_shutdown_locked, dns_channel_saw_error_locked,
+    dns_next_locked};
+
+static void dns_shutdown_locked(grpc_exec_ctx *exec_ctx,
+                                grpc_resolver *resolver) {
+  dns_resolver *r = (dns_resolver *)resolver;
+  if (r->have_retry_timer) {
+    grpc_timer_cancel(exec_ctx, &r->retry_timer);
+  }
+  if (r->next_completion != NULL) {
+    *r->target_result = NULL;
+    grpc_closure_sched(
+        exec_ctx, r->next_completion,
+        GRPC_ERROR_CREATE_FROM_STATIC_STRING("Resolver Shutdown"));
+    r->next_completion = NULL;
+  }
+}
+
+static void dns_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                         grpc_resolver *resolver) {
+  dns_resolver *r = (dns_resolver *)resolver;
+  if (!r->resolving) {
+    gpr_backoff_reset(&r->backoff_state);
+    dns_start_resolving_locked(exec_ctx, r);
+  }
+}
+
+static void dns_next_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *resolver,
+                            grpc_channel_args **target_result,
+                            grpc_closure *on_complete) {
+  dns_resolver *r = (dns_resolver *)resolver;
+  GPR_ASSERT(!r->next_completion);
+  r->next_completion = on_complete;
+  r->target_result = target_result;
+  if (r->resolved_version == 0 && !r->resolving) {
+    gpr_backoff_reset(&r->backoff_state);
+    dns_start_resolving_locked(exec_ctx, r);
+  } else {
+    dns_maybe_finish_next_locked(exec_ctx, r);
+  }
+}
+
+static void dns_on_retry_timer_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                      grpc_error *error) {
+  dns_resolver *r = arg;
+
+  r->have_retry_timer = false;
+  if (error == GRPC_ERROR_NONE) {
+    if (!r->resolving) {
+      dns_start_resolving_locked(exec_ctx, r);
+    }
+  }
+
+  GRPC_RESOLVER_UNREF(exec_ctx, &r->base, "retry-timer");
+}
+
+static void dns_on_resolved_locked(grpc_exec_ctx *exec_ctx, void *arg,
+                                   grpc_error *error) {
+  dns_resolver *r = arg;
+  grpc_channel_args *result = NULL;
+  GPR_ASSERT(r->resolving);
+  r->resolving = false;
+  if (r->addresses != NULL) {
+    grpc_lb_addresses *addresses = grpc_lb_addresses_create(
+        r->addresses->naddrs, NULL /* user_data_vtable */);
+    for (size_t i = 0; i < r->addresses->naddrs; ++i) {
+      grpc_lb_addresses_set_address(
+          addresses, i, &r->addresses->addrs[i].addr,
+          r->addresses->addrs[i].len, false /* is_balancer */,
+          NULL /* balancer_name */, NULL /* user_data */);
+    }
+    grpc_arg new_arg = grpc_lb_addresses_create_channel_arg(addresses);
+    result = grpc_channel_args_copy_and_add(r->channel_args, &new_arg, 1);
+    grpc_resolved_addresses_destroy(r->addresses);
+    grpc_lb_addresses_destroy(exec_ctx, addresses);
+  } else {
+    gpr_timespec now = gpr_now(GPR_CLOCK_MONOTONIC);
+    gpr_timespec next_try = gpr_backoff_step(&r->backoff_state, now);
+    gpr_timespec timeout = gpr_time_sub(next_try, now);
+    gpr_log(GPR_INFO, "dns resolution failed (will retry): %s",
+            grpc_error_string(error));
+    GPR_ASSERT(!r->have_retry_timer);
+    r->have_retry_timer = true;
+    GRPC_RESOLVER_REF(&r->base, "retry-timer");
+    if (gpr_time_cmp(timeout, gpr_time_0(timeout.clock_type)) > 0) {
+      gpr_log(GPR_DEBUG, "retrying in %" PRId64 ".%09d seconds", timeout.tv_sec,
+              timeout.tv_nsec);
+    } else {
+      gpr_log(GPR_DEBUG, "retrying immediately");
+    }
+    grpc_closure_init(&r->on_retry, dns_on_retry_timer_locked, r,
+                      grpc_combiner_scheduler(r->base.combiner, false));
+    grpc_timer_init(exec_ctx, &r->retry_timer, next_try, &r->on_retry, now);
+  }
+  if (r->resolved_result != NULL) {
+    grpc_channel_args_destroy(exec_ctx, r->resolved_result);
+  }
+  r->resolved_result = result;
+  r->resolved_version++;
+  dns_maybe_finish_next_locked(exec_ctx, r);
+
+  GRPC_RESOLVER_UNREF(exec_ctx, &r->base, "dns-resolving");
+}
+
+static void dns_start_resolving_locked(grpc_exec_ctx *exec_ctx,
+                                       dns_resolver *r) {
+  GRPC_RESOLVER_REF(&r->base, "dns-resolving");
+  GPR_ASSERT(!r->resolving);
+  r->resolving = true;
+  r->addresses = NULL;
+  grpc_resolve_address(
+      exec_ctx, r->name_to_resolve, r->default_port, r->interested_parties,
+      grpc_closure_create(dns_on_resolved_locked, r,
+                          grpc_combiner_scheduler(r->base.combiner, false)),
+      &r->addresses);
+}
+
+static void dns_maybe_finish_next_locked(grpc_exec_ctx *exec_ctx,
+                                         dns_resolver *r) {
+  if (r->next_completion != NULL &&
+      r->resolved_version != r->published_version) {
+    *r->target_result = r->resolved_result == NULL
+                            ? NULL
+                            : grpc_channel_args_copy(r->resolved_result);
+    grpc_closure_sched(exec_ctx, r->next_completion, GRPC_ERROR_NONE);
+    r->next_completion = NULL;
+    r->published_version = r->resolved_version;
+  }
+}
+
+static void dns_destroy(grpc_exec_ctx *exec_ctx, grpc_resolver *gr) {
+  dns_resolver *r = (dns_resolver *)gr;
+  if (r->resolved_result != NULL) {
+    grpc_channel_args_destroy(exec_ctx, r->resolved_result);
+  }
+  grpc_pollset_set_destroy(exec_ctx, r->interested_parties);
+  gpr_free(r->name_to_resolve);
+  gpr_free(r->default_port);
+  grpc_channel_args_destroy(exec_ctx, r->channel_args);
+  gpr_free(r);
+}
+
+static grpc_resolver *dns_create(grpc_exec_ctx *exec_ctx,
+                                 grpc_resolver_args *args,
+                                 const char *default_port) {
+  if (0 != strcmp(args->uri->authority, "")) {
+    gpr_log(GPR_ERROR, "authority based dns uri's not supported");
+    return NULL;
+  }
+  // Get name from args.
+  char *path = args->uri->path;
+  if (path[0] == '/') ++path;
+  // Create resolver.
+  dns_resolver *r = gpr_zalloc(sizeof(dns_resolver));
+  grpc_resolver_init(&r->base, &dns_resolver_vtable, args->combiner);
+  r->name_to_resolve = gpr_strdup(path);
+  r->default_port = gpr_strdup(default_port);
+  r->channel_args = grpc_channel_args_copy(args->args);
+  r->interested_parties = grpc_pollset_set_create();
+  if (args->pollset_set != NULL) {
+    grpc_pollset_set_add_pollset_set(exec_ctx, r->interested_parties,
+                                     args->pollset_set);
+  }
+  gpr_backoff_init(&r->backoff_state, GRPC_DNS_INITIAL_CONNECT_BACKOFF_SECONDS,
+                   GRPC_DNS_RECONNECT_BACKOFF_MULTIPLIER,
+                   GRPC_DNS_RECONNECT_JITTER,
+                   GRPC_DNS_MIN_CONNECT_TIMEOUT_SECONDS * 1000,
+                   GRPC_DNS_RECONNECT_MAX_BACKOFF_SECONDS * 1000);
+  return &r->base;
+}
+
+/*
+ * FACTORY
+ */
+
+static void dns_factory_ref(grpc_resolver_factory *factory) {}
+
+static void dns_factory_unref(grpc_resolver_factory *factory) {}
+
+static grpc_resolver *dns_factory_create_resolver(
+    grpc_exec_ctx *exec_ctx, grpc_resolver_factory *factory,
+    grpc_resolver_args *args) {
+  return dns_create(exec_ctx, args, "https");
+}
+
+static char *dns_factory_get_default_host_name(grpc_resolver_factory *factory,
+                                               grpc_uri *uri) {
+  const char *path = uri->path;
+  if (path[0] == '/') ++path;
+  return gpr_strdup(path);
+}
+
+static const grpc_resolver_factory_vtable dns_factory_vtable = {
+    dns_factory_ref, dns_factory_unref, dns_factory_create_resolver,
+    dns_factory_get_default_host_name, "dns"};
+static grpc_resolver_factory dns_resolver_factory = {&dns_factory_vtable};
+
+static grpc_resolver_factory *dns_resolver_factory_create() {
+  return &dns_resolver_factory;
+}
+
+void grpc_resolver_dns_native_init(void) {
+  char *resolver = gpr_getenv("GRPC_DNS_RESOLVER");
+  if (resolver != NULL && gpr_stricmp(resolver, "native") == 0) {
+    gpr_log(GPR_DEBUG, "Using native dns resolver");
+    grpc_register_resolver_type(dns_resolver_factory_create());
+  } else {
+    grpc_resolver_factory *existing_factory =
+        grpc_resolver_factory_lookup("dns");
+    if (existing_factory == NULL) {
+      gpr_log(GPR_DEBUG, "Using native dns resolver");
+      grpc_register_resolver_type(dns_resolver_factory_create());
+    } else {
+      grpc_resolver_factory_unref(existing_factory);
+    }
+  }
+  gpr_free(resolver);
+}
+
+void grpc_resolver_dns_native_shutdown(void) {}
diff --git a/src/core/ext/filters/client_channel/resolver/sockaddr/README.md b/src/core/ext/filters/client_channel/resolver/sockaddr/README.md
new file mode 100644
index 0000000..e307ba8
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/sockaddr/README.md
@@ -0,0 +1 @@
+Support for resolving ipv4:, ipv6:, unix: schemes
diff --git a/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.c b/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.c
new file mode 100644
index 0000000..54f020d
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.c
@@ -0,0 +1,234 @@
+/*
+ *
+ * Copyright 2015-2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <stdbool.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/host_port.h>
+#include <grpc/support/port_platform.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/lb_policy_factory.h"
+#include "src/core/ext/filters/client_channel/parse_address.h"
+#include "src/core/ext/filters/client_channel/resolver_registry.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/iomgr/combiner.h"
+#include "src/core/lib/iomgr/resolve_address.h"
+#include "src/core/lib/iomgr/unix_sockets_posix.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/slice/slice_string_helpers.h"
+#include "src/core/lib/support/string.h"
+
+typedef struct {
+  /** base class: must be first */
+  grpc_resolver base;
+  /** the addresses that we've 'resolved' */
+  grpc_lb_addresses *addresses;
+  /** channel args */
+  grpc_channel_args *channel_args;
+  /** have we published? */
+  bool published;
+  /** pending next completion, or NULL */
+  grpc_closure *next_completion;
+  /** target result address for next completion */
+  grpc_channel_args **target_result;
+} sockaddr_resolver;
+
+static void sockaddr_destroy(grpc_exec_ctx *exec_ctx, grpc_resolver *r);
+
+static void sockaddr_maybe_finish_next_locked(grpc_exec_ctx *exec_ctx,
+                                              sockaddr_resolver *r);
+
+static void sockaddr_shutdown_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *r);
+static void sockaddr_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                              grpc_resolver *r);
+static void sockaddr_next_locked(grpc_exec_ctx *exec_ctx, grpc_resolver *r,
+                                 grpc_channel_args **target_result,
+                                 grpc_closure *on_complete);
+
+static const grpc_resolver_vtable sockaddr_resolver_vtable = {
+    sockaddr_destroy, sockaddr_shutdown_locked,
+    sockaddr_channel_saw_error_locked, sockaddr_next_locked};
+
+static void sockaddr_shutdown_locked(grpc_exec_ctx *exec_ctx,
+                                     grpc_resolver *resolver) {
+  sockaddr_resolver *r = (sockaddr_resolver *)resolver;
+  if (r->next_completion != NULL) {
+    *r->target_result = NULL;
+    grpc_closure_sched(exec_ctx, r->next_completion, GRPC_ERROR_NONE);
+    r->next_completion = NULL;
+  }
+}
+
+static void sockaddr_channel_saw_error_locked(grpc_exec_ctx *exec_ctx,
+                                              grpc_resolver *resolver) {
+  sockaddr_resolver *r = (sockaddr_resolver *)resolver;
+  r->published = false;
+  sockaddr_maybe_finish_next_locked(exec_ctx, r);
+}
+
+static void sockaddr_next_locked(grpc_exec_ctx *exec_ctx,
+                                 grpc_resolver *resolver,
+                                 grpc_channel_args **target_result,
+                                 grpc_closure *on_complete) {
+  sockaddr_resolver *r = (sockaddr_resolver *)resolver;
+  GPR_ASSERT(!r->next_completion);
+  r->next_completion = on_complete;
+  r->target_result = target_result;
+  sockaddr_maybe_finish_next_locked(exec_ctx, r);
+}
+
+static void sockaddr_maybe_finish_next_locked(grpc_exec_ctx *exec_ctx,
+                                              sockaddr_resolver *r) {
+  if (r->next_completion != NULL && !r->published) {
+    r->published = true;
+    grpc_arg arg = grpc_lb_addresses_create_channel_arg(r->addresses);
+    *r->target_result =
+        grpc_channel_args_copy_and_add(r->channel_args, &arg, 1);
+    grpc_closure_sched(exec_ctx, r->next_completion, GRPC_ERROR_NONE);
+    r->next_completion = NULL;
+  }
+}
+
+static void sockaddr_destroy(grpc_exec_ctx *exec_ctx, grpc_resolver *gr) {
+  sockaddr_resolver *r = (sockaddr_resolver *)gr;
+  grpc_lb_addresses_destroy(exec_ctx, r->addresses);
+  grpc_channel_args_destroy(exec_ctx, r->channel_args);
+  gpr_free(r);
+}
+
+static char *ip_get_default_authority(grpc_uri *uri) {
+  const char *path = uri->path;
+  if (path[0] == '/') ++path;
+  return gpr_strdup(path);
+}
+
+static char *ipv4_get_default_authority(grpc_resolver_factory *factory,
+                                        grpc_uri *uri) {
+  return ip_get_default_authority(uri);
+}
+
+static char *ipv6_get_default_authority(grpc_resolver_factory *factory,
+                                        grpc_uri *uri) {
+  return ip_get_default_authority(uri);
+}
+
+#ifdef GRPC_HAVE_UNIX_SOCKET
+char *unix_get_default_authority(grpc_resolver_factory *factory,
+                                 grpc_uri *uri) {
+  return gpr_strdup("localhost");
+}
+#endif
+
+static void do_nothing(void *ignored) {}
+
+static grpc_resolver *sockaddr_create(grpc_exec_ctx *exec_ctx,
+                                      grpc_resolver_args *args,
+                                      int parse(grpc_uri *uri,
+                                                grpc_resolved_address *dst)) {
+  if (0 != strcmp(args->uri->authority, "")) {
+    gpr_log(GPR_ERROR, "authority based uri's not supported by the %s scheme",
+            args->uri->scheme);
+    return NULL;
+  }
+  /* Construct addresses. */
+  grpc_slice path_slice =
+      grpc_slice_new(args->uri->path, strlen(args->uri->path), do_nothing);
+  grpc_slice_buffer path_parts;
+  grpc_slice_buffer_init(&path_parts);
+  grpc_slice_split(path_slice, ",", &path_parts);
+  grpc_lb_addresses *addresses =
+      grpc_lb_addresses_create(path_parts.count, NULL /* user_data_vtable */);
+  bool errors_found = false;
+  for (size_t i = 0; i < addresses->num_addresses; i++) {
+    grpc_uri ith_uri = *args->uri;
+    char *part_str = grpc_slice_to_c_string(path_parts.slices[i]);
+    ith_uri.path = part_str;
+    if (!parse(&ith_uri, &addresses->addresses[i].address)) {
+      errors_found = true; /* GPR_TRUE */
+    }
+    gpr_free(part_str);
+    if (errors_found) break;
+  }
+  grpc_slice_buffer_destroy_internal(exec_ctx, &path_parts);
+  grpc_slice_unref_internal(exec_ctx, path_slice);
+  if (errors_found) {
+    grpc_lb_addresses_destroy(exec_ctx, addresses);
+    return NULL;
+  }
+  /* Instantiate resolver. */
+  sockaddr_resolver *r = gpr_zalloc(sizeof(sockaddr_resolver));
+  r->addresses = addresses;
+  r->channel_args = grpc_channel_args_copy(args->args);
+  grpc_resolver_init(&r->base, &sockaddr_resolver_vtable, args->combiner);
+  return &r->base;
+}
+
+/*
+ * FACTORY
+ */
+
+static void sockaddr_factory_ref(grpc_resolver_factory *factory) {}
+
+static void sockaddr_factory_unref(grpc_resolver_factory *factory) {}
+
+#define DECL_FACTORY(name)                                                  \
+  static grpc_resolver *name##_factory_create_resolver(                     \
+      grpc_exec_ctx *exec_ctx, grpc_resolver_factory *factory,              \
+      grpc_resolver_args *args) {                                           \
+    return sockaddr_create(exec_ctx, args, parse_##name);                   \
+  }                                                                         \
+  static const grpc_resolver_factory_vtable name##_factory_vtable = {       \
+      sockaddr_factory_ref, sockaddr_factory_unref,                         \
+      name##_factory_create_resolver, name##_get_default_authority, #name}; \
+  static grpc_resolver_factory name##_resolver_factory = {                  \
+      &name##_factory_vtable}
+
+#ifdef GRPC_HAVE_UNIX_SOCKET
+DECL_FACTORY(unix);
+#endif
+DECL_FACTORY(ipv4);
+DECL_FACTORY(ipv6);
+
+void grpc_resolver_sockaddr_init(void) {
+  grpc_register_resolver_type(&ipv4_resolver_factory);
+  grpc_register_resolver_type(&ipv6_resolver_factory);
+#ifdef GRPC_HAVE_UNIX_SOCKET
+  grpc_register_resolver_type(&unix_resolver_factory);
+#endif
+}
+
+void grpc_resolver_sockaddr_shutdown(void) {}
diff --git a/src/core/ext/filters/client_channel/resolver_factory.c b/src/core/ext/filters/client_channel/resolver_factory.c
new file mode 100644
index 0000000..07f9e4c
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver_factory.c
@@ -0,0 +1,56 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/resolver_factory.h"
+
+void grpc_resolver_factory_ref(grpc_resolver_factory* factory) {
+  factory->vtable->ref(factory);
+}
+
+void grpc_resolver_factory_unref(grpc_resolver_factory* factory) {
+  factory->vtable->unref(factory);
+}
+
+/** Create a resolver instance for a name */
+grpc_resolver* grpc_resolver_factory_create_resolver(
+    grpc_exec_ctx* exec_ctx, grpc_resolver_factory* factory,
+    grpc_resolver_args* args) {
+  if (factory == NULL) return NULL;
+  return factory->vtable->create_resolver(exec_ctx, factory, args);
+}
+
+char* grpc_resolver_factory_get_default_authority(
+    grpc_resolver_factory* factory, grpc_uri* uri) {
+  if (factory == NULL) return NULL;
+  return factory->vtable->get_default_authority(factory, uri);
+}
diff --git a/src/core/ext/filters/client_channel/resolver_factory.h b/src/core/ext/filters/client_channel/resolver_factory.h
new file mode 100644
index 0000000..b9cac1d
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver_factory.h
@@ -0,0 +1,85 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_FACTORY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_FACTORY_H
+
+#include "src/core/ext/filters/client_channel/client_channel_factory.h"
+#include "src/core/ext/filters/client_channel/resolver.h"
+#include "src/core/ext/filters/client_channel/uri_parser.h"
+#include "src/core/lib/iomgr/pollset_set.h"
+
+typedef struct grpc_resolver_factory grpc_resolver_factory;
+typedef struct grpc_resolver_factory_vtable grpc_resolver_factory_vtable;
+
+struct grpc_resolver_factory {
+  const grpc_resolver_factory_vtable *vtable;
+};
+
+typedef struct grpc_resolver_args {
+  grpc_uri *uri;
+  const grpc_channel_args *args;
+  grpc_pollset_set *pollset_set;
+  grpc_combiner *combiner;
+} grpc_resolver_args;
+
+struct grpc_resolver_factory_vtable {
+  void (*ref)(grpc_resolver_factory *factory);
+  void (*unref)(grpc_resolver_factory *factory);
+
+  /** Implementation of grpc_resolver_factory_create_resolver */
+  grpc_resolver *(*create_resolver)(grpc_exec_ctx *exec_ctx,
+                                    grpc_resolver_factory *factory,
+                                    grpc_resolver_args *args);
+
+  /** Implementation of grpc_resolver_factory_get_default_authority */
+  char *(*get_default_authority)(grpc_resolver_factory *factory, grpc_uri *uri);
+
+  /** URI scheme that this factory implements */
+  const char *scheme;
+};
+
+void grpc_resolver_factory_ref(grpc_resolver_factory *resolver);
+void grpc_resolver_factory_unref(grpc_resolver_factory *resolver);
+
+/** Create a resolver instance for a name */
+grpc_resolver *grpc_resolver_factory_create_resolver(
+    grpc_exec_ctx *exec_ctx, grpc_resolver_factory *factory,
+    grpc_resolver_args *args);
+
+/** Return a (freshly allocated with gpr_malloc) string representing
+    the default authority to use for this scheme. */
+char *grpc_resolver_factory_get_default_authority(
+    grpc_resolver_factory *factory, grpc_uri *uri);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_FACTORY_H */
diff --git a/src/core/ext/filters/client_channel/resolver_registry.c b/src/core/ext/filters/client_channel/resolver_registry.c
new file mode 100644
index 0000000..fa997bd
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver_registry.c
@@ -0,0 +1,174 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/resolver_registry.h"
+
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+
+#define MAX_RESOLVERS 10
+#define DEFAULT_RESOLVER_PREFIX_MAX_LENGTH 32
+
+static grpc_resolver_factory *g_all_of_the_resolvers[MAX_RESOLVERS];
+static int g_number_of_resolvers = 0;
+
+static char g_default_resolver_prefix[DEFAULT_RESOLVER_PREFIX_MAX_LENGTH] =
+    "dns:///";
+
+void grpc_resolver_registry_init() {}
+
+void grpc_resolver_registry_shutdown(void) {
+  for (int i = 0; i < g_number_of_resolvers; i++) {
+    grpc_resolver_factory_unref(g_all_of_the_resolvers[i]);
+  }
+  // FIXME(ctiller): this should live in grpc_resolver_registry_init,
+  // however that would have the client_channel plugin call this AFTER we start
+  // registering resolvers from third party plugins, and so they'd never show
+  // up.
+  // We likely need some kind of dependency system for plugins.... what form
+  // that takes is TBD.
+  g_number_of_resolvers = 0;
+}
+
+void grpc_resolver_registry_set_default_prefix(
+    const char *default_resolver_prefix) {
+  const size_t len = strlen(default_resolver_prefix);
+  GPR_ASSERT(len < DEFAULT_RESOLVER_PREFIX_MAX_LENGTH &&
+             "default resolver prefix too long");
+  GPR_ASSERT(len > 0 && "default resolver prefix can't be empty");
+  // By the previous assert, default_resolver_prefix is safe to be copied with a
+  // plain strcpy.
+  strcpy(g_default_resolver_prefix, default_resolver_prefix);
+}
+
+void grpc_register_resolver_type(grpc_resolver_factory *factory) {
+  int i;
+  for (i = 0; i < g_number_of_resolvers; i++) {
+    GPR_ASSERT(0 != strcmp(factory->vtable->scheme,
+                           g_all_of_the_resolvers[i]->vtable->scheme));
+  }
+  GPR_ASSERT(g_number_of_resolvers != MAX_RESOLVERS);
+  grpc_resolver_factory_ref(factory);
+  g_all_of_the_resolvers[g_number_of_resolvers++] = factory;
+}
+
+static grpc_resolver_factory *lookup_factory(const char *name) {
+  int i;
+
+  for (i = 0; i < g_number_of_resolvers; i++) {
+    if (0 == strcmp(name, g_all_of_the_resolvers[i]->vtable->scheme)) {
+      return g_all_of_the_resolvers[i];
+    }
+  }
+  return NULL;
+}
+
+grpc_resolver_factory *grpc_resolver_factory_lookup(const char *name) {
+  grpc_resolver_factory *f = lookup_factory(name);
+  if (f) grpc_resolver_factory_ref(f);
+  return f;
+}
+
+static grpc_resolver_factory *lookup_factory_by_uri(grpc_uri *uri) {
+  if (!uri) return NULL;
+  return lookup_factory(uri->scheme);
+}
+
+static grpc_resolver_factory *resolve_factory(grpc_exec_ctx *exec_ctx,
+                                              const char *target,
+                                              grpc_uri **uri,
+                                              char **canonical_target) {
+  grpc_resolver_factory *factory = NULL;
+
+  GPR_ASSERT(uri != NULL);
+  *uri = grpc_uri_parse(exec_ctx, target, 1);
+  factory = lookup_factory_by_uri(*uri);
+  if (factory == NULL) {
+    grpc_uri_destroy(*uri);
+    gpr_asprintf(canonical_target, "%s%s", g_default_resolver_prefix, target);
+    *uri = grpc_uri_parse(exec_ctx, *canonical_target, 1);
+    factory = lookup_factory_by_uri(*uri);
+    if (factory == NULL) {
+      grpc_uri_destroy(grpc_uri_parse(exec_ctx, target, 0));
+      grpc_uri_destroy(grpc_uri_parse(exec_ctx, *canonical_target, 0));
+      gpr_log(GPR_ERROR, "don't know how to resolve '%s' or '%s'", target,
+              *canonical_target);
+    }
+  }
+  return factory;
+}
+
+grpc_resolver *grpc_resolver_create(grpc_exec_ctx *exec_ctx, const char *target,
+                                    const grpc_channel_args *args,
+                                    grpc_pollset_set *pollset_set,
+                                    grpc_combiner *combiner) {
+  grpc_uri *uri = NULL;
+  char *canonical_target = NULL;
+  grpc_resolver_factory *factory =
+      resolve_factory(exec_ctx, target, &uri, &canonical_target);
+  grpc_resolver *resolver;
+  grpc_resolver_args resolver_args;
+  memset(&resolver_args, 0, sizeof(resolver_args));
+  resolver_args.uri = uri;
+  resolver_args.args = args;
+  resolver_args.pollset_set = pollset_set;
+  resolver_args.combiner = combiner;
+  resolver =
+      grpc_resolver_factory_create_resolver(exec_ctx, factory, &resolver_args);
+  grpc_uri_destroy(uri);
+  gpr_free(canonical_target);
+  return resolver;
+}
+
+char *grpc_get_default_authority(grpc_exec_ctx *exec_ctx, const char *target) {
+  grpc_uri *uri = NULL;
+  char *canonical_target = NULL;
+  grpc_resolver_factory *factory =
+      resolve_factory(exec_ctx, target, &uri, &canonical_target);
+  char *authority = grpc_resolver_factory_get_default_authority(factory, uri);
+  grpc_uri_destroy(uri);
+  gpr_free(canonical_target);
+  return authority;
+}
+
+char *grpc_resolver_factory_add_default_prefix_if_needed(
+    grpc_exec_ctx *exec_ctx, const char *target) {
+  grpc_uri *uri = NULL;
+  char *canonical_target = NULL;
+  resolve_factory(exec_ctx, target, &uri, &canonical_target);
+  grpc_uri_destroy(uri);
+  return canonical_target == NULL ? gpr_strdup(target) : canonical_target;
+}
diff --git a/src/core/ext/filters/client_channel/resolver_registry.h b/src/core/ext/filters/client_channel/resolver_registry.h
new file mode 100644
index 0000000..8d95fd8
--- /dev/null
+++ b/src/core/ext/filters/client_channel/resolver_registry.h
@@ -0,0 +1,84 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_REGISTRY_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_REGISTRY_H
+
+#include "src/core/ext/filters/client_channel/resolver_factory.h"
+#include "src/core/lib/iomgr/pollset_set.h"
+
+void grpc_resolver_registry_init();
+void grpc_resolver_registry_shutdown(void);
+
+/** Set the default URI prefix to \a default_prefix. */
+void grpc_resolver_registry_set_default_prefix(const char *default_prefix);
+
+/** Register a resolver type.
+    URI's of \a scheme will be resolved with the given resolver.
+    If \a priority is greater than zero, then the resolver will be eligible
+    to resolve names that are passed in with no scheme. Higher priority
+    resolvers will be tried before lower priority schemes. */
+void grpc_register_resolver_type(grpc_resolver_factory *factory);
+
+/** Create a resolver given \a target.
+    First tries to parse \a target as a URI. If this succeeds, tries
+    to locate a registered resolver factory based on the URI scheme.
+    If parsing or location fails, prefixes default_prefix from
+    grpc_resolver_registry_init to target, and tries again (if default_prefix
+    was not NULL).
+    If a resolver factory was found, use it to instantiate a resolver and
+    return it.
+    If a resolver factory was not found, return NULL.
+    \a args is a set of channel arguments to be included in the result
+    (typically the set of arguments passed in from the client API).
+    \a pollset_set is used to drive IO in the name resolution process, it
+    should not be NULL. */
+grpc_resolver *grpc_resolver_create(grpc_exec_ctx *exec_ctx, const char *target,
+                                    const grpc_channel_args *args,
+                                    grpc_pollset_set *pollset_set,
+                                    grpc_combiner *combiner);
+
+/** Find a resolver factory given a name and return an (owned-by-the-caller)
+ *  reference to it */
+grpc_resolver_factory *grpc_resolver_factory_lookup(const char *name);
+
+/** Given a target, return a (freshly allocated with gpr_malloc) string
+    representing the default authority to pass from a client. */
+char *grpc_get_default_authority(grpc_exec_ctx *exec_ctx, const char *target);
+
+/** Returns a newly allocated string containing \a target, adding the
+    default prefix if needed. */
+char *grpc_resolver_factory_add_default_prefix_if_needed(
+    grpc_exec_ctx *exec_ctx, const char *target);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_RESOLVER_REGISTRY_H */
diff --git a/src/core/ext/filters/client_channel/retry_throttle.c b/src/core/ext/filters/client_channel/retry_throttle.c
new file mode 100644
index 0000000..dd78a17
--- /dev/null
+++ b/src/core/ext/filters/client_channel/retry_throttle.c
@@ -0,0 +1,210 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/retry_throttle.h"
+
+#include <limits.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/atm.h>
+#include <grpc/support/avl.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/sync.h>
+
+//
+// server_retry_throttle_data
+//
+
+struct grpc_server_retry_throttle_data {
+  gpr_refcount refs;
+  int max_milli_tokens;
+  int milli_token_ratio;
+  gpr_atm milli_tokens;
+  // A pointer to the replacement for this grpc_server_retry_throttle_data
+  // entry.  If non-NULL, then this entry is stale and must not be used.
+  // We hold a reference to the replacement.
+  gpr_atm replacement;
+};
+
+static void get_replacement_throttle_data_if_needed(
+    grpc_server_retry_throttle_data** throttle_data) {
+  while (true) {
+    grpc_server_retry_throttle_data* new_throttle_data =
+        (grpc_server_retry_throttle_data*)gpr_atm_acq_load(
+            &(*throttle_data)->replacement);
+    if (new_throttle_data == NULL) return;
+    *throttle_data = new_throttle_data;
+  }
+}
+
+bool grpc_server_retry_throttle_data_record_failure(
+    grpc_server_retry_throttle_data* throttle_data) {
+  // First, check if we are stale and need to be replaced.
+  get_replacement_throttle_data_if_needed(&throttle_data);
+  // We decrement milli_tokens by 1000 (1 token) for each failure.
+  const int new_value = (int)gpr_atm_no_barrier_clamped_add(
+      &throttle_data->milli_tokens, (gpr_atm)-1000, (gpr_atm)0,
+      (gpr_atm)throttle_data->max_milli_tokens);
+  // Retries are allowed as long as the new value is above the threshold
+  // (max_milli_tokens / 2).
+  return new_value > throttle_data->max_milli_tokens / 2;
+}
+
+void grpc_server_retry_throttle_data_record_success(
+    grpc_server_retry_throttle_data* throttle_data) {
+  // First, check if we are stale and need to be replaced.
+  get_replacement_throttle_data_if_needed(&throttle_data);
+  // We increment milli_tokens by milli_token_ratio for each success.
+  gpr_atm_no_barrier_clamped_add(
+      &throttle_data->milli_tokens, (gpr_atm)throttle_data->milli_token_ratio,
+      (gpr_atm)0, (gpr_atm)throttle_data->max_milli_tokens);
+}
+
+grpc_server_retry_throttle_data* grpc_server_retry_throttle_data_ref(
+    grpc_server_retry_throttle_data* throttle_data) {
+  gpr_ref(&throttle_data->refs);
+  return throttle_data;
+}
+
+void grpc_server_retry_throttle_data_unref(
+    grpc_server_retry_throttle_data* throttle_data) {
+  if (gpr_unref(&throttle_data->refs)) {
+    grpc_server_retry_throttle_data* replacement =
+        (grpc_server_retry_throttle_data*)gpr_atm_acq_load(
+            &throttle_data->replacement);
+    if (replacement != NULL) {
+      grpc_server_retry_throttle_data_unref(replacement);
+    }
+    gpr_free(throttle_data);
+  }
+}
+
+static grpc_server_retry_throttle_data* grpc_server_retry_throttle_data_create(
+    int max_milli_tokens, int milli_token_ratio,
+    grpc_server_retry_throttle_data* old_throttle_data) {
+  grpc_server_retry_throttle_data* throttle_data =
+      gpr_malloc(sizeof(*throttle_data));
+  memset(throttle_data, 0, sizeof(*throttle_data));
+  gpr_ref_init(&throttle_data->refs, 1);
+  throttle_data->max_milli_tokens = max_milli_tokens;
+  throttle_data->milli_token_ratio = milli_token_ratio;
+  int initial_milli_tokens = max_milli_tokens;
+  // If there was a pre-existing entry for this server name, initialize
+  // the token count by scaling proportionately to the old data.  This
+  // ensures that if we're already throttling retries on the old scale,
+  // we will start out doing the same thing on the new one.
+  if (old_throttle_data != NULL) {
+    double token_fraction =
+        (int)gpr_atm_acq_load(&old_throttle_data->milli_tokens) /
+        (double)old_throttle_data->max_milli_tokens;
+    initial_milli_tokens = (int)(token_fraction * max_milli_tokens);
+  }
+  gpr_atm_rel_store(&throttle_data->milli_tokens,
+                    (gpr_atm)initial_milli_tokens);
+  // If there was a pre-existing entry, mark it as stale and give it a
+  // pointer to the new entry, which is its replacement.
+  if (old_throttle_data != NULL) {
+    grpc_server_retry_throttle_data_ref(throttle_data);
+    gpr_atm_rel_store(&old_throttle_data->replacement, (gpr_atm)throttle_data);
+  }
+  return throttle_data;
+}
+
+//
+// avl vtable for string -> server_retry_throttle_data map
+//
+
+static void* copy_server_name(void* key) { return gpr_strdup(key); }
+
+static long compare_server_name(void* key1, void* key2) {
+  return strcmp(key1, key2);
+}
+
+static void destroy_server_retry_throttle_data(void* value) {
+  grpc_server_retry_throttle_data* throttle_data = value;
+  grpc_server_retry_throttle_data_unref(throttle_data);
+}
+
+static void* copy_server_retry_throttle_data(void* value) {
+  grpc_server_retry_throttle_data* throttle_data = value;
+  return grpc_server_retry_throttle_data_ref(throttle_data);
+}
+
+static const gpr_avl_vtable avl_vtable = {
+    gpr_free /* destroy_key */, copy_server_name, compare_server_name,
+    destroy_server_retry_throttle_data, copy_server_retry_throttle_data};
+
+//
+// server_retry_throttle_map
+//
+
+static gpr_mu g_mu;
+static gpr_avl g_avl;
+
+void grpc_retry_throttle_map_init() {
+  gpr_mu_init(&g_mu);
+  g_avl = gpr_avl_create(&avl_vtable);
+}
+
+void grpc_retry_throttle_map_shutdown() {
+  gpr_mu_destroy(&g_mu);
+  gpr_avl_unref(g_avl);
+}
+
+grpc_server_retry_throttle_data* grpc_retry_throttle_map_get_data_for_server(
+    const char* server_name, int max_milli_tokens, int milli_token_ratio) {
+  gpr_mu_lock(&g_mu);
+  grpc_server_retry_throttle_data* throttle_data =
+      gpr_avl_get(g_avl, (char*)server_name);
+  if (throttle_data == NULL) {
+    // Entry not found.  Create a new one.
+    throttle_data = grpc_server_retry_throttle_data_create(
+        max_milli_tokens, milli_token_ratio, NULL);
+    g_avl = gpr_avl_add(g_avl, (char*)server_name, throttle_data);
+  } else {
+    if (throttle_data->max_milli_tokens != max_milli_tokens ||
+        throttle_data->milli_token_ratio != milli_token_ratio) {
+      // Entry found but with old parameters.  Create a new one based on
+      // the original one.
+      throttle_data = grpc_server_retry_throttle_data_create(
+          max_milli_tokens, milli_token_ratio, throttle_data);
+      g_avl = gpr_avl_add(g_avl, (char*)server_name, throttle_data);
+    } else {
+      // Entry found.  Increase refcount.
+      grpc_server_retry_throttle_data_ref(throttle_data);
+    }
+  }
+  gpr_mu_unlock(&g_mu);
+  return throttle_data;
+}
diff --git a/src/core/ext/filters/client_channel/retry_throttle.h b/src/core/ext/filters/client_channel/retry_throttle.h
new file mode 100644
index 0000000..f9971fa
--- /dev/null
+++ b/src/core/ext/filters/client_channel/retry_throttle.h
@@ -0,0 +1,65 @@
+/*
+ *
+ * Copyright 2017, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_RETRY_THROTTLE_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_RETRY_THROTTLE_H
+
+#include <stdbool.h>
+
+/// Tracks retry throttling data for an individual server name.
+typedef struct grpc_server_retry_throttle_data grpc_server_retry_throttle_data;
+
+/// Records a failure.  Returns true if it's okay to send a retry.
+bool grpc_server_retry_throttle_data_record_failure(
+    grpc_server_retry_throttle_data* throttle_data);
+/// Records a success.
+void grpc_server_retry_throttle_data_record_success(
+    grpc_server_retry_throttle_data* throttle_data);
+
+grpc_server_retry_throttle_data* grpc_server_retry_throttle_data_ref(
+    grpc_server_retry_throttle_data* throttle_data);
+void grpc_server_retry_throttle_data_unref(
+    grpc_server_retry_throttle_data* throttle_data);
+
+/// Initializes global map of failure data for each server name.
+void grpc_retry_throttle_map_init();
+/// Shuts down global map of failure data for each server name.
+void grpc_retry_throttle_map_shutdown();
+
+/// Returns a reference to the failure data for \a server_name, creating
+/// a new entry if needed.
+/// Caller must eventually unref via \a grpc_server_retry_throttle_data_unref().
+grpc_server_retry_throttle_data* grpc_retry_throttle_map_get_data_for_server(
+    const char* server_name, int max_milli_tokens, int milli_token_ratio);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_RETRY_THROTTLE_H */
diff --git a/src/core/ext/filters/client_channel/subchannel.c b/src/core/ext/filters/client_channel/subchannel.c
new file mode 100644
index 0000000..ac86989
--- /dev/null
+++ b/src/core/ext/filters/client_channel/subchannel.c
@@ -0,0 +1,839 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/subchannel.h"
+
+#include <limits.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/avl.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/ext/filters/client_channel/client_channel.h"
+#include "src/core/ext/filters/client_channel/parse_address.h"
+#include "src/core/ext/filters/client_channel/proxy_mapper_registry.h"
+#include "src/core/ext/filters/client_channel/subchannel_index.h"
+#include "src/core/ext/filters/client_channel/uri_parser.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/channel/connected_channel.h"
+#include "src/core/lib/iomgr/sockaddr_utils.h"
+#include "src/core/lib/iomgr/timer.h"
+#include "src/core/lib/profiling/timers.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/support/backoff.h"
+#include "src/core/lib/surface/channel.h"
+#include "src/core/lib/surface/channel_init.h"
+#include "src/core/lib/transport/connectivity_state.h"
+
+#define INTERNAL_REF_BITS 16
+#define STRONG_REF_MASK (~(gpr_atm)((1 << INTERNAL_REF_BITS) - 1))
+
+#define GRPC_SUBCHANNEL_MIN_CONNECT_TIMEOUT_SECONDS 20
+#define GRPC_SUBCHANNEL_INITIAL_CONNECT_BACKOFF_SECONDS 1
+#define GRPC_SUBCHANNEL_RECONNECT_BACKOFF_MULTIPLIER 1.6
+#define GRPC_SUBCHANNEL_RECONNECT_MAX_BACKOFF_SECONDS 120
+#define GRPC_SUBCHANNEL_RECONNECT_JITTER 0.2
+
+#define GET_CONNECTED_SUBCHANNEL(subchannel, barrier)      \
+  ((grpc_connected_subchannel *)(gpr_atm_##barrier##_load( \
+      &(subchannel)->connected_subchannel)))
+
+typedef struct {
+  grpc_closure closure;
+  grpc_subchannel *subchannel;
+  grpc_connectivity_state connectivity_state;
+} state_watcher;
+
+typedef struct external_state_watcher {
+  grpc_subchannel *subchannel;
+  grpc_pollset_set *pollset_set;
+  grpc_closure *notify;
+  grpc_closure closure;
+  struct external_state_watcher *next;
+  struct external_state_watcher *prev;
+} external_state_watcher;
+
+struct grpc_subchannel {
+  grpc_connector *connector;
+
+  /** refcount
+      - lower INTERNAL_REF_BITS bits are for internal references:
+        these do not keep the subchannel open.
+      - upper remaining bits are for public references: these do
+        keep the subchannel open */
+  gpr_atm ref_pair;
+
+  /** non-transport related channel filters */
+  const grpc_channel_filter **filters;
+  size_t num_filters;
+  /** channel arguments */
+  grpc_channel_args *args;
+
+  grpc_subchannel_key *key;
+
+  /** set during connection */
+  grpc_connect_out_args connecting_result;
+
+  /** callback for connection finishing */
+  grpc_closure connected;
+
+  /** callback for our alarm */
+  grpc_closure on_alarm;
+
+  /** pollset_set tracking who's interested in a connection
+      being setup */
+  grpc_pollset_set *pollset_set;
+
+  /** active connection, or null; of type grpc_connected_subchannel */
+  gpr_atm connected_subchannel;
+
+  /** mutex protecting remaining elements */
+  gpr_mu mu;
+
+  /** have we seen a disconnection? */
+  bool disconnected;
+  /** are we connecting */
+  bool connecting;
+  /** connectivity state tracking */
+  grpc_connectivity_state_tracker state_tracker;
+
+  external_state_watcher root_external_state_watcher;
+
+  /** next connect attempt time */
+  gpr_timespec next_attempt;
+  /** backoff state */
+  gpr_backoff backoff_state;
+  /** do we have an active alarm? */
+  bool have_alarm;
+  /** have we started the backoff loop */
+  bool backoff_begun;
+  /** our alarm */
+  grpc_timer alarm;
+};
+
+struct grpc_subchannel_call {
+  grpc_connected_subchannel *connection;
+  grpc_closure *schedule_closure_after_destroy;
+};
+
+#define SUBCHANNEL_CALL_TO_CALL_STACK(call) ((grpc_call_stack *)((call) + 1))
+#define CHANNEL_STACK_FROM_CONNECTION(con) ((grpc_channel_stack *)(con))
+#define CALLSTACK_TO_SUBCHANNEL_CALL(callstack) \
+  (((grpc_subchannel_call *)(callstack)) - 1)
+
+static void subchannel_connected(grpc_exec_ctx *exec_ctx, void *subchannel,
+                                 grpc_error *error);
+
+#ifdef GRPC_STREAM_REFCOUNT_DEBUG
+#define REF_REASON reason
+#define REF_LOG(name, p)                                                  \
+  gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "%s: %p   ref %d -> %d %s", \
+          (name), (p), (p)->refs.count, (p)->refs.count + 1, reason)
+#define UNREF_LOG(name, p)                                                \
+  gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG, "%s: %p unref %d -> %d %s", \
+          (name), (p), (p)->refs.count, (p)->refs.count - 1, reason)
+#define REF_MUTATE_EXTRA_ARGS \
+  GRPC_SUBCHANNEL_REF_EXTRA_ARGS, const char *purpose
+#define REF_MUTATE_PURPOSE(x) , file, line, reason, x
+#else
+#define REF_REASON ""
+#define REF_LOG(name, p) \
+  do {                   \
+  } while (0)
+#define UNREF_LOG(name, p) \
+  do {                     \
+  } while (0)
+#define REF_MUTATE_EXTRA_ARGS
+#define REF_MUTATE_PURPOSE(x)
+#endif
+
+/*
+ * connection implementation
+ */
+
+static void connection_destroy(grpc_exec_ctx *exec_ctx, void *arg,
+                               grpc_error *error) {
+  grpc_connected_subchannel *c = arg;
+  grpc_channel_stack_destroy(exec_ctx, CHANNEL_STACK_FROM_CONNECTION(c));
+  gpr_free(c);
+}
+
+grpc_connected_subchannel *grpc_connected_subchannel_ref(
+    grpc_connected_subchannel *c GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  GRPC_CHANNEL_STACK_REF(CHANNEL_STACK_FROM_CONNECTION(c), REF_REASON);
+  return c;
+}
+
+void grpc_connected_subchannel_unref(grpc_exec_ctx *exec_ctx,
+                                     grpc_connected_subchannel *c
+                                         GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  GRPC_CHANNEL_STACK_UNREF(exec_ctx, CHANNEL_STACK_FROM_CONNECTION(c),
+                           REF_REASON);
+}
+
+/*
+ * grpc_subchannel implementation
+ */
+
+static void subchannel_destroy(grpc_exec_ctx *exec_ctx, void *arg,
+                               grpc_error *error) {
+  grpc_subchannel *c = arg;
+  gpr_free((void *)c->filters);
+  grpc_channel_args_destroy(exec_ctx, c->args);
+  grpc_connectivity_state_destroy(exec_ctx, &c->state_tracker);
+  grpc_connector_unref(exec_ctx, c->connector);
+  grpc_pollset_set_destroy(exec_ctx, c->pollset_set);
+  grpc_subchannel_key_destroy(exec_ctx, c->key);
+  gpr_free(c);
+}
+
+static gpr_atm ref_mutate(grpc_subchannel *c, gpr_atm delta,
+                          int barrier REF_MUTATE_EXTRA_ARGS) {
+  gpr_atm old_val = barrier ? gpr_atm_full_fetch_add(&c->ref_pair, delta)
+                            : gpr_atm_no_barrier_fetch_add(&c->ref_pair, delta);
+#ifdef GRPC_STREAM_REFCOUNT_DEBUG
+  gpr_log(file, line, GPR_LOG_SEVERITY_DEBUG,
+          "SUBCHANNEL: %p %s 0x%08" PRIxPTR " -> 0x%08" PRIxPTR " [%s]", c,
+          purpose, old_val, old_val + delta, reason);
+#endif
+  return old_val;
+}
+
+grpc_subchannel *grpc_subchannel_ref(
+    grpc_subchannel *c GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  gpr_atm old_refs;
+  old_refs = ref_mutate(c, (1 << INTERNAL_REF_BITS),
+                        0 REF_MUTATE_PURPOSE("STRONG_REF"));
+  GPR_ASSERT((old_refs & STRONG_REF_MASK) != 0);
+  return c;
+}
+
+grpc_subchannel *grpc_subchannel_weak_ref(
+    grpc_subchannel *c GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  gpr_atm old_refs;
+  old_refs = ref_mutate(c, 1, 0 REF_MUTATE_PURPOSE("WEAK_REF"));
+  GPR_ASSERT(old_refs != 0);
+  return c;
+}
+
+grpc_subchannel *grpc_subchannel_ref_from_weak_ref(
+    grpc_subchannel *c GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  if (!c) return NULL;
+  for (;;) {
+    gpr_atm old_refs = gpr_atm_acq_load(&c->ref_pair);
+    if (old_refs >= (1 << INTERNAL_REF_BITS)) {
+      gpr_atm new_refs = old_refs + (1 << INTERNAL_REF_BITS);
+      if (gpr_atm_rel_cas(&c->ref_pair, old_refs, new_refs)) {
+        return c;
+      }
+    } else {
+      return NULL;
+    }
+  }
+}
+
+static void disconnect(grpc_exec_ctx *exec_ctx, grpc_subchannel *c) {
+  grpc_connected_subchannel *con;
+  grpc_subchannel_index_unregister(exec_ctx, c->key, c);
+  gpr_mu_lock(&c->mu);
+  GPR_ASSERT(!c->disconnected);
+  c->disconnected = true;
+  grpc_connector_shutdown(
+      exec_ctx, c->connector,
+      GRPC_ERROR_CREATE_FROM_STATIC_STRING("Subchannel disconnected"));
+  con = GET_CONNECTED_SUBCHANNEL(c, no_barrier);
+  if (con != NULL) {
+    GRPC_CONNECTED_SUBCHANNEL_UNREF(exec_ctx, con, "connection");
+    gpr_atm_no_barrier_store(&c->connected_subchannel, (gpr_atm)0xdeadbeef);
+  }
+  gpr_mu_unlock(&c->mu);
+}
+
+void grpc_subchannel_unref(grpc_exec_ctx *exec_ctx,
+                           grpc_subchannel *c GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  gpr_atm old_refs;
+  old_refs = ref_mutate(c, (gpr_atm)1 - (gpr_atm)(1 << INTERNAL_REF_BITS),
+                        1 REF_MUTATE_PURPOSE("STRONG_UNREF"));
+  if ((old_refs & STRONG_REF_MASK) == (1 << INTERNAL_REF_BITS)) {
+    disconnect(exec_ctx, c);
+  }
+  GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "strong-unref");
+}
+
+void grpc_subchannel_weak_unref(grpc_exec_ctx *exec_ctx,
+                                grpc_subchannel *c
+                                    GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  gpr_atm old_refs;
+  old_refs = ref_mutate(c, -(gpr_atm)1, 1 REF_MUTATE_PURPOSE("WEAK_UNREF"));
+  if (old_refs == 1) {
+    grpc_closure_sched(exec_ctx, grpc_closure_create(subchannel_destroy, c,
+                                                     grpc_schedule_on_exec_ctx),
+                       GRPC_ERROR_NONE);
+  }
+}
+
+grpc_subchannel *grpc_subchannel_create(grpc_exec_ctx *exec_ctx,
+                                        grpc_connector *connector,
+                                        const grpc_subchannel_args *args) {
+  grpc_subchannel_key *key = grpc_subchannel_key_create(connector, args);
+  grpc_subchannel *c = grpc_subchannel_index_find(exec_ctx, key);
+  if (c) {
+    grpc_subchannel_key_destroy(exec_ctx, key);
+    return c;
+  }
+
+  c = gpr_zalloc(sizeof(*c));
+  c->key = key;
+  gpr_atm_no_barrier_store(&c->ref_pair, 1 << INTERNAL_REF_BITS);
+  c->connector = connector;
+  grpc_connector_ref(c->connector);
+  c->num_filters = args->filter_count;
+  if (c->num_filters > 0) {
+    c->filters = gpr_malloc(sizeof(grpc_channel_filter *) * c->num_filters);
+    memcpy((void *)c->filters, args->filters,
+           sizeof(grpc_channel_filter *) * c->num_filters);
+  } else {
+    c->filters = NULL;
+  }
+  c->pollset_set = grpc_pollset_set_create();
+  grpc_resolved_address *addr = gpr_malloc(sizeof(*addr));
+  grpc_get_subchannel_address_arg(exec_ctx, args->args, addr);
+  grpc_resolved_address *new_address = NULL;
+  grpc_channel_args *new_args = NULL;
+  if (grpc_proxy_mappers_map_address(exec_ctx, addr, args->args, &new_address,
+                                     &new_args)) {
+    GPR_ASSERT(new_address != NULL);
+    gpr_free(addr);
+    addr = new_address;
+  }
+  static const char *keys_to_remove[] = {GRPC_ARG_SUBCHANNEL_ADDRESS};
+  grpc_arg new_arg = grpc_create_subchannel_address_arg(addr);
+  gpr_free(addr);
+  c->args = grpc_channel_args_copy_and_add_and_remove(
+      new_args != NULL ? new_args : args->args, keys_to_remove,
+      GPR_ARRAY_SIZE(keys_to_remove), &new_arg, 1);
+  gpr_free(new_arg.value.string);
+  if (new_args != NULL) grpc_channel_args_destroy(exec_ctx, new_args);
+  c->root_external_state_watcher.next = c->root_external_state_watcher.prev =
+      &c->root_external_state_watcher;
+  grpc_closure_init(&c->connected, subchannel_connected, c,
+                    grpc_schedule_on_exec_ctx);
+  grpc_connectivity_state_init(&c->state_tracker, GRPC_CHANNEL_IDLE,
+                               "subchannel");
+  int initial_backoff_ms =
+      GRPC_SUBCHANNEL_INITIAL_CONNECT_BACKOFF_SECONDS * 1000;
+  int max_backoff_ms = GRPC_SUBCHANNEL_RECONNECT_MAX_BACKOFF_SECONDS * 1000;
+  int min_backoff_ms = GRPC_SUBCHANNEL_MIN_CONNECT_TIMEOUT_SECONDS * 1000;
+  bool fixed_reconnect_backoff = false;
+  if (c->args) {
+    for (size_t i = 0; i < c->args->num_args; i++) {
+      if (0 == strcmp(c->args->args[i].key,
+                      "grpc.testing.fixed_reconnect_backoff_ms")) {
+        GPR_ASSERT(c->args->args[i].type == GRPC_ARG_INTEGER);
+        fixed_reconnect_backoff = true;
+        initial_backoff_ms = min_backoff_ms = max_backoff_ms =
+            grpc_channel_arg_get_integer(
+                &c->args->args[i],
+                (grpc_integer_options){initial_backoff_ms, 100, INT_MAX});
+      } else if (0 == strcmp(c->args->args[i].key,
+                             GRPC_ARG_MAX_RECONNECT_BACKOFF_MS)) {
+        fixed_reconnect_backoff = false;
+        max_backoff_ms = grpc_channel_arg_get_integer(
+            &c->args->args[i],
+            (grpc_integer_options){max_backoff_ms, 100, INT_MAX});
+      } else if (0 == strcmp(c->args->args[i].key,
+                             GRPC_ARG_INITIAL_RECONNECT_BACKOFF_MS)) {
+        fixed_reconnect_backoff = false;
+        initial_backoff_ms = grpc_channel_arg_get_integer(
+            &c->args->args[i],
+            (grpc_integer_options){initial_backoff_ms, 100, INT_MAX});
+      }
+    }
+  }
+  gpr_backoff_init(
+      &c->backoff_state, initial_backoff_ms,
+      fixed_reconnect_backoff ? 1.0
+                              : GRPC_SUBCHANNEL_RECONNECT_BACKOFF_MULTIPLIER,
+      fixed_reconnect_backoff ? 0.0 : GRPC_SUBCHANNEL_RECONNECT_JITTER,
+      min_backoff_ms, max_backoff_ms);
+  gpr_mu_init(&c->mu);
+
+  return grpc_subchannel_index_register(exec_ctx, key, c);
+}
+
+static void continue_connect_locked(grpc_exec_ctx *exec_ctx,
+                                    grpc_subchannel *c) {
+  grpc_connect_in_args args;
+
+  args.interested_parties = c->pollset_set;
+  args.deadline = c->next_attempt;
+  args.channel_args = c->args;
+
+  grpc_connectivity_state_set(exec_ctx, &c->state_tracker,
+                              GRPC_CHANNEL_CONNECTING, GRPC_ERROR_NONE,
+                              "state_change");
+  grpc_connector_connect(exec_ctx, c->connector, &args, &c->connecting_result,
+                         &c->connected);
+}
+
+grpc_connectivity_state grpc_subchannel_check_connectivity(grpc_subchannel *c,
+                                                           grpc_error **error) {
+  grpc_connectivity_state state;
+  gpr_mu_lock(&c->mu);
+  state = grpc_connectivity_state_get(&c->state_tracker, error);
+  gpr_mu_unlock(&c->mu);
+  return state;
+}
+
+static void on_external_state_watcher_done(grpc_exec_ctx *exec_ctx, void *arg,
+                                           grpc_error *error) {
+  external_state_watcher *w = arg;
+  grpc_closure *follow_up = w->notify;
+  if (w->pollset_set != NULL) {
+    grpc_pollset_set_del_pollset_set(exec_ctx, w->subchannel->pollset_set,
+                                     w->pollset_set);
+  }
+  gpr_mu_lock(&w->subchannel->mu);
+  w->next->prev = w->prev;
+  w->prev->next = w->next;
+  gpr_mu_unlock(&w->subchannel->mu);
+  GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, w->subchannel, "external_state_watcher");
+  gpr_free(w);
+  grpc_closure_run(exec_ctx, follow_up, GRPC_ERROR_REF(error));
+}
+
+static void on_alarm(grpc_exec_ctx *exec_ctx, void *arg, grpc_error *error) {
+  grpc_subchannel *c = arg;
+  gpr_mu_lock(&c->mu);
+  c->have_alarm = false;
+  if (c->disconnected) {
+    error = GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING("Disconnected",
+                                                             &error, 1);
+  } else {
+    GRPC_ERROR_REF(error);
+  }
+  if (error == GRPC_ERROR_NONE) {
+    gpr_log(GPR_INFO, "Failed to connect to channel, retrying");
+    c->next_attempt =
+        gpr_backoff_step(&c->backoff_state, gpr_now(GPR_CLOCK_MONOTONIC));
+    continue_connect_locked(exec_ctx, c);
+    gpr_mu_unlock(&c->mu);
+  } else {
+    gpr_mu_unlock(&c->mu);
+    GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "connecting");
+  }
+  GRPC_ERROR_UNREF(error);
+}
+
+static void maybe_start_connecting_locked(grpc_exec_ctx *exec_ctx,
+                                          grpc_subchannel *c) {
+  if (c->disconnected) {
+    /* Don't try to connect if we're already disconnected */
+    return;
+  }
+
+  if (c->connecting) {
+    /* Already connecting: don't restart */
+    return;
+  }
+
+  if (GET_CONNECTED_SUBCHANNEL(c, no_barrier) != NULL) {
+    /* Already connected: don't restart */
+    return;
+  }
+
+  if (!grpc_connectivity_state_has_watchers(&c->state_tracker)) {
+    /* Nobody is interested in connecting: so don't just yet */
+    return;
+  }
+
+  c->connecting = true;
+  GRPC_SUBCHANNEL_WEAK_REF(c, "connecting");
+
+  gpr_timespec now = gpr_now(GPR_CLOCK_MONOTONIC);
+  if (!c->backoff_begun) {
+    c->backoff_begun = true;
+    c->next_attempt = gpr_backoff_begin(&c->backoff_state, now);
+    continue_connect_locked(exec_ctx, c);
+  } else {
+    GPR_ASSERT(!c->have_alarm);
+    c->have_alarm = true;
+    gpr_timespec time_til_next = gpr_time_sub(c->next_attempt, now);
+    if (gpr_time_cmp(time_til_next, gpr_time_0(time_til_next.clock_type)) <=
+        0) {
+      gpr_log(GPR_INFO, "Retry immediately");
+    } else {
+      gpr_log(GPR_INFO, "Retry in %" PRId64 ".%09d seconds",
+              time_til_next.tv_sec, time_til_next.tv_nsec);
+    }
+    grpc_closure_init(&c->on_alarm, on_alarm, c, grpc_schedule_on_exec_ctx);
+    grpc_timer_init(exec_ctx, &c->alarm, c->next_attempt, &c->on_alarm, now);
+  }
+}
+
+void grpc_subchannel_notify_on_state_change(
+    grpc_exec_ctx *exec_ctx, grpc_subchannel *c,
+    grpc_pollset_set *interested_parties, grpc_connectivity_state *state,
+    grpc_closure *notify) {
+  external_state_watcher *w;
+
+  if (state == NULL) {
+    gpr_mu_lock(&c->mu);
+    for (w = c->root_external_state_watcher.next;
+         w != &c->root_external_state_watcher; w = w->next) {
+      if (w->notify == notify) {
+        grpc_connectivity_state_notify_on_state_change(
+            exec_ctx, &c->state_tracker, NULL, &w->closure);
+      }
+    }
+    gpr_mu_unlock(&c->mu);
+  } else {
+    w = gpr_malloc(sizeof(*w));
+    w->subchannel = c;
+    w->pollset_set = interested_parties;
+    w->notify = notify;
+    grpc_closure_init(&w->closure, on_external_state_watcher_done, w,
+                      grpc_schedule_on_exec_ctx);
+    if (interested_parties != NULL) {
+      grpc_pollset_set_add_pollset_set(exec_ctx, c->pollset_set,
+                                       interested_parties);
+    }
+    GRPC_SUBCHANNEL_WEAK_REF(c, "external_state_watcher");
+    gpr_mu_lock(&c->mu);
+    w->next = &c->root_external_state_watcher;
+    w->prev = w->next->prev;
+    w->next->prev = w->prev->next = w;
+    grpc_connectivity_state_notify_on_state_change(exec_ctx, &c->state_tracker,
+                                                   state, &w->closure);
+    maybe_start_connecting_locked(exec_ctx, c);
+    gpr_mu_unlock(&c->mu);
+  }
+}
+
+void grpc_connected_subchannel_process_transport_op(
+    grpc_exec_ctx *exec_ctx, grpc_connected_subchannel *con,
+    grpc_transport_op *op) {
+  grpc_channel_stack *channel_stack = CHANNEL_STACK_FROM_CONNECTION(con);
+  grpc_channel_element *top_elem = grpc_channel_stack_element(channel_stack, 0);
+  top_elem->filter->start_transport_op(exec_ctx, top_elem, op);
+}
+
+static void subchannel_on_child_state_changed(grpc_exec_ctx *exec_ctx, void *p,
+                                              grpc_error *error) {
+  state_watcher *sw = p;
+  grpc_subchannel *c = sw->subchannel;
+  gpr_mu *mu = &c->mu;
+
+  gpr_mu_lock(mu);
+
+  /* if we failed just leave this closure */
+  if (sw->connectivity_state == GRPC_CHANNEL_TRANSIENT_FAILURE) {
+    /* any errors on a subchannel ==> we're done, create a new one */
+    sw->connectivity_state = GRPC_CHANNEL_SHUTDOWN;
+  }
+  grpc_connectivity_state_set(exec_ctx, &c->state_tracker,
+                              sw->connectivity_state, GRPC_ERROR_REF(error),
+                              "reflect_child");
+  if (sw->connectivity_state != GRPC_CHANNEL_SHUTDOWN) {
+    grpc_connected_subchannel_notify_on_state_change(
+        exec_ctx, GET_CONNECTED_SUBCHANNEL(c, no_barrier), NULL,
+        &sw->connectivity_state, &sw->closure);
+    GRPC_SUBCHANNEL_WEAK_REF(c, "state_watcher");
+    sw = NULL;
+  }
+
+  gpr_mu_unlock(mu);
+  GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "state_watcher");
+  gpr_free(sw);
+}
+
+static void connected_subchannel_state_op(grpc_exec_ctx *exec_ctx,
+                                          grpc_connected_subchannel *con,
+                                          grpc_pollset_set *interested_parties,
+                                          grpc_connectivity_state *state,
+                                          grpc_closure *closure) {
+  grpc_transport_op *op = grpc_make_transport_op(NULL);
+  grpc_channel_element *elem;
+  op->connectivity_state = state;
+  op->on_connectivity_state_change = closure;
+  op->bind_pollset_set = interested_parties;
+  elem = grpc_channel_stack_element(CHANNEL_STACK_FROM_CONNECTION(con), 0);
+  elem->filter->start_transport_op(exec_ctx, elem, op);
+}
+
+void grpc_connected_subchannel_notify_on_state_change(
+    grpc_exec_ctx *exec_ctx, grpc_connected_subchannel *con,
+    grpc_pollset_set *interested_parties, grpc_connectivity_state *state,
+    grpc_closure *closure) {
+  connected_subchannel_state_op(exec_ctx, con, interested_parties, state,
+                                closure);
+}
+
+void grpc_connected_subchannel_ping(grpc_exec_ctx *exec_ctx,
+                                    grpc_connected_subchannel *con,
+                                    grpc_closure *closure) {
+  grpc_transport_op *op = grpc_make_transport_op(NULL);
+  grpc_channel_element *elem;
+  op->send_ping = closure;
+  elem = grpc_channel_stack_element(CHANNEL_STACK_FROM_CONNECTION(con), 0);
+  elem->filter->start_transport_op(exec_ctx, elem, op);
+}
+
+static void publish_transport_locked(grpc_exec_ctx *exec_ctx,
+                                     grpc_subchannel *c) {
+  grpc_connected_subchannel *con;
+  grpc_channel_stack *stk;
+  state_watcher *sw_subchannel;
+
+  /* construct channel stack */
+  grpc_channel_stack_builder *builder = grpc_channel_stack_builder_create();
+  grpc_channel_stack_builder_set_channel_arguments(
+      exec_ctx, builder, c->connecting_result.channel_args);
+  grpc_channel_stack_builder_set_transport(builder,
+                                           c->connecting_result.transport);
+
+  if (!grpc_channel_init_create_stack(exec_ctx, builder,
+                                      GRPC_CLIENT_SUBCHANNEL)) {
+    grpc_channel_stack_builder_destroy(exec_ctx, builder);
+    abort(); /* TODO(ctiller): what to do here (previously we just crashed) */
+  }
+  grpc_error *error = grpc_channel_stack_builder_finish(
+      exec_ctx, builder, 0, 1, connection_destroy, NULL, (void **)&con);
+  if (error != GRPC_ERROR_NONE) {
+    gpr_log(GPR_ERROR, "error initializing subchannel stack: %s",
+            grpc_error_string(error));
+    GRPC_ERROR_UNREF(error);
+    abort(); /* TODO(ctiller): what to do here? */
+  }
+  stk = CHANNEL_STACK_FROM_CONNECTION(con);
+  memset(&c->connecting_result, 0, sizeof(c->connecting_result));
+
+  /* initialize state watcher */
+  sw_subchannel = gpr_malloc(sizeof(*sw_subchannel));
+  sw_subchannel->subchannel = c;
+  sw_subchannel->connectivity_state = GRPC_CHANNEL_READY;
+  grpc_closure_init(&sw_subchannel->closure, subchannel_on_child_state_changed,
+                    sw_subchannel, grpc_schedule_on_exec_ctx);
+
+  if (c->disconnected) {
+    gpr_free(sw_subchannel);
+    grpc_channel_stack_destroy(exec_ctx, stk);
+    gpr_free(con);
+    GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "connecting");
+    return;
+  }
+
+  /* publish */
+  /* TODO(ctiller): this full barrier seems to clear up a TSAN failure.
+                    I'd have expected the rel_cas below to be enough, but
+                    seemingly it's not.
+                    Re-evaluate if we really need this. */
+  gpr_atm_full_barrier();
+  GPR_ASSERT(gpr_atm_rel_cas(&c->connected_subchannel, 0, (gpr_atm)con));
+
+  /* setup subchannel watching connected subchannel for changes; subchannel
+     ref for connecting is donated to the state watcher */
+  GRPC_SUBCHANNEL_WEAK_REF(c, "state_watcher");
+  GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "connecting");
+  grpc_connected_subchannel_notify_on_state_change(
+      exec_ctx, con, c->pollset_set, &sw_subchannel->connectivity_state,
+      &sw_subchannel->closure);
+
+  /* signal completion */
+  grpc_connectivity_state_set(exec_ctx, &c->state_tracker, GRPC_CHANNEL_READY,
+                              GRPC_ERROR_NONE, "connected");
+}
+
+static void subchannel_connected(grpc_exec_ctx *exec_ctx, void *arg,
+                                 grpc_error *error) {
+  grpc_subchannel *c = arg;
+  grpc_channel_args *delete_channel_args = c->connecting_result.channel_args;
+
+  GRPC_SUBCHANNEL_WEAK_REF(c, "connected");
+  gpr_mu_lock(&c->mu);
+  c->connecting = false;
+  if (c->connecting_result.transport != NULL) {
+    publish_transport_locked(exec_ctx, c);
+  } else if (c->disconnected) {
+    GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "connecting");
+  } else {
+    grpc_connectivity_state_set(
+        exec_ctx, &c->state_tracker, GRPC_CHANNEL_TRANSIENT_FAILURE,
+        grpc_error_set_int(GRPC_ERROR_CREATE_REFERENCING_FROM_STATIC_STRING(
+                               "Connect Failed", &error, 1),
+                           GRPC_ERROR_INT_GRPC_STATUS, GRPC_STATUS_UNAVAILABLE),
+        "connect_failed");
+
+    const char *errmsg = grpc_error_string(error);
+    gpr_log(GPR_INFO, "Connect failed: %s", errmsg);
+
+    maybe_start_connecting_locked(exec_ctx, c);
+    GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "connecting");
+  }
+  gpr_mu_unlock(&c->mu);
+  GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, c, "connected");
+  grpc_channel_args_destroy(exec_ctx, delete_channel_args);
+}
+
+/*
+ * grpc_subchannel_call implementation
+ */
+
+static void subchannel_call_destroy(grpc_exec_ctx *exec_ctx, void *call,
+                                    grpc_error *error) {
+  grpc_subchannel_call *c = call;
+  GPR_ASSERT(c->schedule_closure_after_destroy != NULL);
+  GPR_TIMER_BEGIN("grpc_subchannel_call_unref.destroy", 0);
+  grpc_connected_subchannel *connection = c->connection;
+  grpc_call_stack_destroy(exec_ctx, SUBCHANNEL_CALL_TO_CALL_STACK(c), NULL,
+                          c->schedule_closure_after_destroy);
+  GRPC_CONNECTED_SUBCHANNEL_UNREF(exec_ctx, connection, "subchannel_call");
+  GPR_TIMER_END("grpc_subchannel_call_unref.destroy", 0);
+}
+
+void grpc_subchannel_call_set_cleanup_closure(grpc_subchannel_call *call,
+                                              grpc_closure *closure) {
+  GPR_ASSERT(call->schedule_closure_after_destroy == NULL);
+  GPR_ASSERT(closure != NULL);
+  call->schedule_closure_after_destroy = closure;
+}
+
+void grpc_subchannel_call_ref(
+    grpc_subchannel_call *c GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  GRPC_CALL_STACK_REF(SUBCHANNEL_CALL_TO_CALL_STACK(c), REF_REASON);
+}
+
+void grpc_subchannel_call_unref(grpc_exec_ctx *exec_ctx,
+                                grpc_subchannel_call *c
+                                    GRPC_SUBCHANNEL_REF_EXTRA_ARGS) {
+  GRPC_CALL_STACK_UNREF(exec_ctx, SUBCHANNEL_CALL_TO_CALL_STACK(c), REF_REASON);
+}
+
+char *grpc_subchannel_call_get_peer(grpc_exec_ctx *exec_ctx,
+                                    grpc_subchannel_call *call) {
+  grpc_call_stack *call_stack = SUBCHANNEL_CALL_TO_CALL_STACK(call);
+  grpc_call_element *top_elem = grpc_call_stack_element(call_stack, 0);
+  return top_elem->filter->get_peer(exec_ctx, top_elem);
+}
+
+void grpc_subchannel_call_process_op(grpc_exec_ctx *exec_ctx,
+                                     grpc_subchannel_call *call,
+                                     grpc_transport_stream_op *op) {
+  GPR_TIMER_BEGIN("grpc_subchannel_call_process_op", 0);
+  grpc_call_stack *call_stack = SUBCHANNEL_CALL_TO_CALL_STACK(call);
+  grpc_call_element *top_elem = grpc_call_stack_element(call_stack, 0);
+  top_elem->filter->start_transport_stream_op(exec_ctx, top_elem, op);
+  GPR_TIMER_END("grpc_subchannel_call_process_op", 0);
+}
+
+grpc_connected_subchannel *grpc_subchannel_get_connected_subchannel(
+    grpc_subchannel *c) {
+  return GET_CONNECTED_SUBCHANNEL(c, acq);
+}
+
+grpc_error *grpc_connected_subchannel_create_call(
+    grpc_exec_ctx *exec_ctx, grpc_connected_subchannel *con,
+    const grpc_connected_subchannel_call_args *args,
+    grpc_subchannel_call **call) {
+  grpc_channel_stack *chanstk = CHANNEL_STACK_FROM_CONNECTION(con);
+  *call = gpr_arena_alloc(
+      args->arena, sizeof(grpc_subchannel_call) + chanstk->call_stack_size);
+  grpc_call_stack *callstk = SUBCHANNEL_CALL_TO_CALL_STACK(*call);
+  (*call)->connection = con;  // Ref is added below.
+  const grpc_call_element_args call_args = {.call_stack = callstk,
+                                            .server_transport_data = NULL,
+                                            .context = NULL,
+                                            .path = args->path,
+                                            .start_time = args->start_time,
+                                            .deadline = args->deadline,
+                                            .arena = args->arena};
+  grpc_error *error = grpc_call_stack_init(
+      exec_ctx, chanstk, 1, subchannel_call_destroy, *call, &call_args);
+  if (error != GRPC_ERROR_NONE) {
+    const char *error_string = grpc_error_string(error);
+    gpr_log(GPR_ERROR, "error: %s", error_string);
+
+    gpr_free(*call);
+    return error;
+  }
+  GRPC_CONNECTED_SUBCHANNEL_REF(con, "subchannel_call");
+  grpc_call_stack_set_pollset_or_pollset_set(exec_ctx, callstk, args->pollent);
+  return GRPC_ERROR_NONE;
+}
+
+grpc_call_stack *grpc_subchannel_call_get_call_stack(
+    grpc_subchannel_call *subchannel_call) {
+  return SUBCHANNEL_CALL_TO_CALL_STACK(subchannel_call);
+}
+
+static void grpc_uri_to_sockaddr(grpc_exec_ctx *exec_ctx, const char *uri_str,
+                                 grpc_resolved_address *addr) {
+  grpc_uri *uri = grpc_uri_parse(exec_ctx, uri_str, 0 /* suppress_errors */);
+  GPR_ASSERT(uri != NULL);
+  if (strcmp(uri->scheme, "ipv4") == 0) {
+    GPR_ASSERT(parse_ipv4(uri, addr));
+  } else if (strcmp(uri->scheme, "ipv6") == 0) {
+    GPR_ASSERT(parse_ipv6(uri, addr));
+  } else {
+    GPR_ASSERT(parse_unix(uri, addr));
+  }
+  grpc_uri_destroy(uri);
+}
+
+void grpc_get_subchannel_address_arg(grpc_exec_ctx *exec_ctx,
+                                     const grpc_channel_args *args,
+                                     grpc_resolved_address *addr) {
+  const char *addr_uri_str = grpc_get_subchannel_address_uri_arg(args);
+  memset(addr, 0, sizeof(*addr));
+  if (*addr_uri_str != '\0') {
+    grpc_uri_to_sockaddr(exec_ctx, addr_uri_str, addr);
+  }
+}
+
+const char *grpc_get_subchannel_address_uri_arg(const grpc_channel_args *args) {
+  const grpc_arg *addr_arg =
+      grpc_channel_args_find(args, GRPC_ARG_SUBCHANNEL_ADDRESS);
+  GPR_ASSERT(addr_arg != NULL);  // Should have been set by LB policy.
+  GPR_ASSERT(addr_arg->type == GRPC_ARG_STRING);
+  return addr_arg->value.string;
+}
+
+grpc_arg grpc_create_subchannel_address_arg(const grpc_resolved_address *addr) {
+  grpc_arg new_arg;
+  new_arg.key = GRPC_ARG_SUBCHANNEL_ADDRESS;
+  new_arg.type = GRPC_ARG_STRING;
+  new_arg.value.string =
+      addr->len > 0 ? grpc_sockaddr_to_uri(addr) : gpr_strdup("");
+  return new_arg;
+}
diff --git a/src/core/ext/filters/client_channel/subchannel.h b/src/core/ext/filters/client_channel/subchannel.h
new file mode 100644
index 0000000..4d904e2
--- /dev/null
+++ b/src/core/ext/filters/client_channel/subchannel.h
@@ -0,0 +1,203 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_SUBCHANNEL_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_SUBCHANNEL_H
+
+#include "src/core/ext/filters/client_channel/connector.h"
+#include "src/core/lib/channel/channel_stack.h"
+#include "src/core/lib/iomgr/polling_entity.h"
+#include "src/core/lib/support/arena.h"
+#include "src/core/lib/transport/connectivity_state.h"
+#include "src/core/lib/transport/metadata.h"
+
+// Channel arg containing a grpc_resolved_address to connect to.
+#define GRPC_ARG_SUBCHANNEL_ADDRESS "grpc.subchannel_address"
+
+/** A (sub-)channel that knows how to connect to exactly one target
+    address. Provides a target for load balancing. */
+typedef struct grpc_subchannel grpc_subchannel;
+typedef struct grpc_connected_subchannel grpc_connected_subchannel;
+typedef struct grpc_subchannel_call grpc_subchannel_call;
+typedef struct grpc_subchannel_args grpc_subchannel_args;
+
+#ifdef GRPC_STREAM_REFCOUNT_DEBUG
+#define GRPC_SUBCHANNEL_REF(p, r) \
+  grpc_subchannel_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_REF_FROM_WEAK_REF(p, r) \
+  grpc_subchannel_ref_from_weak_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_UNREF(cl, p, r) \
+  grpc_subchannel_unref((cl), (p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_WEAK_REF(p, r) \
+  grpc_subchannel_weak_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_WEAK_UNREF(cl, p, r) \
+  grpc_subchannel_weak_unref((cl), (p), __FILE__, __LINE__, (r))
+#define GRPC_CONNECTED_SUBCHANNEL_REF(p, r) \
+  grpc_connected_subchannel_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_CONNECTED_SUBCHANNEL_UNREF(cl, p, r) \
+  grpc_connected_subchannel_unref((cl), (p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_CALL_REF(p, r) \
+  grpc_subchannel_call_ref((p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_CALL_UNREF(cl, p, r) \
+  grpc_subchannel_call_unref((cl), (p), __FILE__, __LINE__, (r))
+#define GRPC_SUBCHANNEL_REF_EXTRA_ARGS \
+  , const char *file, int line, const char *reason
+#else
+#define GRPC_SUBCHANNEL_REF(p, r) grpc_subchannel_ref((p))
+#define GRPC_SUBCHANNEL_REF_FROM_WEAK_REF(p, r) \
+  grpc_subchannel_ref_from_weak_ref((p))
+#define GRPC_SUBCHANNEL_UNREF(cl, p, r) grpc_subchannel_unref((cl), (p))
+#define GRPC_SUBCHANNEL_WEAK_REF(p, r) grpc_subchannel_weak_ref((p))
+#define GRPC_SUBCHANNEL_WEAK_UNREF(cl, p, r) \
+  grpc_subchannel_weak_unref((cl), (p))
+#define GRPC_CONNECTED_SUBCHANNEL_REF(p, r) grpc_connected_subchannel_ref((p))
+#define GRPC_CONNECTED_SUBCHANNEL_UNREF(cl, p, r) \
+  grpc_connected_subchannel_unref((cl), (p))
+#define GRPC_SUBCHANNEL_CALL_REF(p, r) grpc_subchannel_call_ref((p))
+#define GRPC_SUBCHANNEL_CALL_UNREF(cl, p, r) \
+  grpc_subchannel_call_unref((cl), (p))
+#define GRPC_SUBCHANNEL_REF_EXTRA_ARGS
+#endif
+
+grpc_subchannel *grpc_subchannel_ref(
+    grpc_subchannel *channel GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+grpc_subchannel *grpc_subchannel_ref_from_weak_ref(
+    grpc_subchannel *channel GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+void grpc_subchannel_unref(grpc_exec_ctx *exec_ctx,
+                           grpc_subchannel *channel
+                               GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+grpc_subchannel *grpc_subchannel_weak_ref(
+    grpc_subchannel *channel GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+void grpc_subchannel_weak_unref(grpc_exec_ctx *exec_ctx,
+                                grpc_subchannel *channel
+                                    GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+grpc_connected_subchannel *grpc_connected_subchannel_ref(
+    grpc_connected_subchannel *channel GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+void grpc_connected_subchannel_unref(grpc_exec_ctx *exec_ctx,
+                                     grpc_connected_subchannel *channel
+                                         GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+void grpc_subchannel_call_ref(
+    grpc_subchannel_call *call GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+void grpc_subchannel_call_unref(grpc_exec_ctx *exec_ctx,
+                                grpc_subchannel_call *call
+                                    GRPC_SUBCHANNEL_REF_EXTRA_ARGS);
+
+/** construct a subchannel call */
+typedef struct {
+  grpc_polling_entity *pollent;
+  grpc_slice path;
+  gpr_timespec start_time;
+  gpr_timespec deadline;
+  gpr_arena *arena;
+} grpc_connected_subchannel_call_args;
+
+grpc_error *grpc_connected_subchannel_create_call(
+    grpc_exec_ctx *exec_ctx, grpc_connected_subchannel *connected_subchannel,
+    const grpc_connected_subchannel_call_args *args,
+    grpc_subchannel_call **subchannel_call);
+
+/** process a transport level op */
+void grpc_connected_subchannel_process_transport_op(
+    grpc_exec_ctx *exec_ctx, grpc_connected_subchannel *subchannel,
+    grpc_transport_op *op);
+
+/** poll the current connectivity state of a channel */
+grpc_connectivity_state grpc_subchannel_check_connectivity(
+    grpc_subchannel *channel, grpc_error **error);
+
+/** call notify when the connectivity state of a channel changes from *state.
+    Updates *state with the new state of the channel */
+void grpc_subchannel_notify_on_state_change(
+    grpc_exec_ctx *exec_ctx, grpc_subchannel *channel,
+    grpc_pollset_set *interested_parties, grpc_connectivity_state *state,
+    grpc_closure *notify);
+void grpc_connected_subchannel_notify_on_state_change(
+    grpc_exec_ctx *exec_ctx, grpc_connected_subchannel *channel,
+    grpc_pollset_set *interested_parties, grpc_connectivity_state *state,
+    grpc_closure *notify);
+void grpc_connected_subchannel_ping(grpc_exec_ctx *exec_ctx,
+                                    grpc_connected_subchannel *channel,
+                                    grpc_closure *notify);
+
+/** retrieve the grpc_connected_subchannel - or NULL if called before
+    the subchannel becomes connected */
+grpc_connected_subchannel *grpc_subchannel_get_connected_subchannel(
+    grpc_subchannel *subchannel);
+
+/** continue processing a transport op */
+void grpc_subchannel_call_process_op(grpc_exec_ctx *exec_ctx,
+                                     grpc_subchannel_call *subchannel_call,
+                                     grpc_transport_stream_op *op);
+
+/** continue querying for peer */
+char *grpc_subchannel_call_get_peer(grpc_exec_ctx *exec_ctx,
+                                    grpc_subchannel_call *subchannel_call);
+
+/** Must be called once per call. Sets the 'then_schedule_closure' argument for
+    call stack destruction. */
+void grpc_subchannel_call_set_cleanup_closure(
+    grpc_subchannel_call *subchannel_call, grpc_closure *closure);
+
+grpc_call_stack *grpc_subchannel_call_get_call_stack(
+    grpc_subchannel_call *subchannel_call);
+
+struct grpc_subchannel_args {
+  /* When updating this struct, also update subchannel_index.c */
+
+  /** Channel filters for this channel - wrapped factories will likely
+      want to mutate this */
+  const grpc_channel_filter **filters;
+  /** The number of filters in the above array */
+  size_t filter_count;
+  /** Channel arguments to be supplied to the newly created channel */
+  const grpc_channel_args *args;
+};
+
+/** create a subchannel given a connector */
+grpc_subchannel *grpc_subchannel_create(grpc_exec_ctx *exec_ctx,
+                                        grpc_connector *connector,
+                                        const grpc_subchannel_args *args);
+
+/// Sets \a addr from \a args.
+void grpc_get_subchannel_address_arg(grpc_exec_ctx *exec_ctx,
+                                     const grpc_channel_args *args,
+                                     grpc_resolved_address *addr);
+
+/// Returns the URI string for the address to connect to.
+const char *grpc_get_subchannel_address_uri_arg(const grpc_channel_args *args);
+
+/// Returns a new channel arg encoding the subchannel address as a string.
+/// Caller is responsible for freeing the string.
+grpc_arg grpc_create_subchannel_address_arg(const grpc_resolved_address *addr);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_SUBCHANNEL_H */
diff --git a/src/core/ext/filters/client_channel/subchannel_index.c b/src/core/ext/filters/client_channel/subchannel_index.c
new file mode 100644
index 0000000..f6ef4a8
--- /dev/null
+++ b/src/core/ext/filters/client_channel/subchannel_index.c
@@ -0,0 +1,262 @@
+//
+//
+// Copyright 2016, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+//
+
+#include "src/core/ext/filters/client_channel/subchannel_index.h"
+
+#include <stdbool.h>
+#include <string.h>
+
+#include <grpc/support/alloc.h>
+#include <grpc/support/avl.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/tls.h>
+
+#include "src/core/lib/channel/channel_args.h"
+
+// a map of subchannel_key --> subchannel, used for detecting connections
+// to the same destination in order to share them
+static gpr_avl g_subchannel_index;
+
+static gpr_mu g_mu;
+
+struct grpc_subchannel_key {
+  grpc_connector *connector;
+  grpc_subchannel_args args;
+};
+
+GPR_TLS_DECL(subchannel_index_exec_ctx);
+
+static void enter_ctx(grpc_exec_ctx *exec_ctx) {
+  GPR_ASSERT(gpr_tls_get(&subchannel_index_exec_ctx) == 0);
+  gpr_tls_set(&subchannel_index_exec_ctx, (intptr_t)exec_ctx);
+}
+
+static void leave_ctx(grpc_exec_ctx *exec_ctx) {
+  GPR_ASSERT(gpr_tls_get(&subchannel_index_exec_ctx) == (intptr_t)exec_ctx);
+  gpr_tls_set(&subchannel_index_exec_ctx, 0);
+}
+
+static grpc_exec_ctx *current_ctx() {
+  grpc_exec_ctx *c = (grpc_exec_ctx *)gpr_tls_get(&subchannel_index_exec_ctx);
+  GPR_ASSERT(c != NULL);
+  return c;
+}
+
+static grpc_subchannel_key *create_key(
+    grpc_connector *connector, const grpc_subchannel_args *args,
+    grpc_channel_args *(*copy_channel_args)(const grpc_channel_args *args)) {
+  grpc_subchannel_key *k = gpr_malloc(sizeof(*k));
+  k->connector = grpc_connector_ref(connector);
+  k->args.filter_count = args->filter_count;
+  if (k->args.filter_count > 0) {
+    k->args.filters =
+        gpr_malloc(sizeof(*k->args.filters) * k->args.filter_count);
+    memcpy((grpc_channel_filter *)k->args.filters, args->filters,
+           sizeof(*k->args.filters) * k->args.filter_count);
+  } else {
+    k->args.filters = NULL;
+  }
+  k->args.args = copy_channel_args(args->args);
+  return k;
+}
+
+grpc_subchannel_key *grpc_subchannel_key_create(
+    grpc_connector *connector, const grpc_subchannel_args *args) {
+  return create_key(connector, args, grpc_channel_args_normalize);
+}
+
+static grpc_subchannel_key *subchannel_key_copy(grpc_subchannel_key *k) {
+  return create_key(k->connector, &k->args, grpc_channel_args_copy);
+}
+
+static int subchannel_key_compare(grpc_subchannel_key *a,
+                                  grpc_subchannel_key *b) {
+  int c = GPR_ICMP(a->connector, b->connector);
+  if (c != 0) return c;
+  c = GPR_ICMP(a->args.filter_count, b->args.filter_count);
+  if (c != 0) return c;
+  if (a->args.filter_count > 0) {
+    c = memcmp(a->args.filters, b->args.filters,
+               a->args.filter_count * sizeof(*a->args.filters));
+    if (c != 0) return c;
+  }
+  return grpc_channel_args_compare(a->args.args, b->args.args);
+}
+
+void grpc_subchannel_key_destroy(grpc_exec_ctx *exec_ctx,
+                                 grpc_subchannel_key *k) {
+  grpc_connector_unref(exec_ctx, k->connector);
+  gpr_free((grpc_channel_args *)k->args.filters);
+  grpc_channel_args_destroy(exec_ctx, (grpc_channel_args *)k->args.args);
+  gpr_free(k);
+}
+
+static void sck_avl_destroy(void *p) {
+  grpc_subchannel_key_destroy(current_ctx(), p);
+}
+
+static void *sck_avl_copy(void *p) { return subchannel_key_copy(p); }
+
+static long sck_avl_compare(void *a, void *b) {
+  return subchannel_key_compare(a, b);
+}
+
+static void scv_avl_destroy(void *p) {
+  GRPC_SUBCHANNEL_WEAK_UNREF(current_ctx(), p, "subchannel_index");
+}
+
+static void *scv_avl_copy(void *p) {
+  GRPC_SUBCHANNEL_WEAK_REF(p, "subchannel_index");
+  return p;
+}
+
+static const gpr_avl_vtable subchannel_avl_vtable = {
+    .destroy_key = sck_avl_destroy,
+    .copy_key = sck_avl_copy,
+    .compare_keys = sck_avl_compare,
+    .destroy_value = scv_avl_destroy,
+    .copy_value = scv_avl_copy};
+
+void grpc_subchannel_index_init(void) {
+  g_subchannel_index = gpr_avl_create(&subchannel_avl_vtable);
+  gpr_mu_init(&g_mu);
+  gpr_tls_init(&subchannel_index_exec_ctx);
+}
+
+void grpc_subchannel_index_shutdown(void) {
+  gpr_mu_destroy(&g_mu);
+  gpr_avl_unref(g_subchannel_index);
+  gpr_tls_destroy(&subchannel_index_exec_ctx);
+}
+
+grpc_subchannel *grpc_subchannel_index_find(grpc_exec_ctx *exec_ctx,
+                                            grpc_subchannel_key *key) {
+  enter_ctx(exec_ctx);
+
+  // Lock, and take a reference to the subchannel index.
+  // We don't need to do the search under a lock as avl's are immutable.
+  gpr_mu_lock(&g_mu);
+  gpr_avl index = gpr_avl_ref(g_subchannel_index);
+  gpr_mu_unlock(&g_mu);
+
+  grpc_subchannel *c =
+      GRPC_SUBCHANNEL_REF_FROM_WEAK_REF(gpr_avl_get(index, key), "index_find");
+  gpr_avl_unref(index);
+
+  leave_ctx(exec_ctx);
+  return c;
+}
+
+grpc_subchannel *grpc_subchannel_index_register(grpc_exec_ctx *exec_ctx,
+                                                grpc_subchannel_key *key,
+                                                grpc_subchannel *constructed) {
+  enter_ctx(exec_ctx);
+
+  grpc_subchannel *c = NULL;
+
+  while (c == NULL) {
+    // Compare and swap loop:
+    // - take a reference to the current index
+    gpr_mu_lock(&g_mu);
+    gpr_avl index = gpr_avl_ref(g_subchannel_index);
+    gpr_mu_unlock(&g_mu);
+
+    // - Check to see if a subchannel already exists
+    c = gpr_avl_get(index, key);
+    if (c != NULL) {
+      // yes -> we're done
+      GRPC_SUBCHANNEL_WEAK_UNREF(exec_ctx, constructed, "index_register");
+    } else {
+      // no -> update the avl and compare/swap
+      gpr_avl updated =
+          gpr_avl_add(gpr_avl_ref(index), subchannel_key_copy(key),
+                      GRPC_SUBCHANNEL_WEAK_REF(constructed, "index_register"));
+
+      // it may happen (but it's expected to be unlikely)
+      // that some other thread has changed the index:
+      // compare/swap here to check that, and retry as necessary
+      gpr_mu_lock(&g_mu);
+      if (index.root == g_subchannel_index.root) {
+        GPR_SWAP(gpr_avl, updated, g_subchannel_index);
+        c = constructed;
+      }
+      gpr_mu_unlock(&g_mu);
+
+      gpr_avl_unref(updated);
+    }
+    gpr_avl_unref(index);
+  }
+
+  leave_ctx(exec_ctx);
+
+  return c;
+}
+
+void grpc_subchannel_index_unregister(grpc_exec_ctx *exec_ctx,
+                                      grpc_subchannel_key *key,
+                                      grpc_subchannel *constructed) {
+  enter_ctx(exec_ctx);
+
+  bool done = false;
+  while (!done) {
+    // Compare and swap loop:
+    // - take a reference to the current index
+    gpr_mu_lock(&g_mu);
+    gpr_avl index = gpr_avl_ref(g_subchannel_index);
+    gpr_mu_unlock(&g_mu);
+
+    // Check to see if this key still refers to the previously
+    // registered subchannel
+    grpc_subchannel *c = gpr_avl_get(index, key);
+    if (c != constructed) {
+      gpr_avl_unref(index);
+      break;
+    }
+
+    // compare and swap the update (some other thread may have
+    // mutated the index behind us)
+    gpr_avl updated = gpr_avl_remove(gpr_avl_ref(index), key);
+
+    gpr_mu_lock(&g_mu);
+    if (index.root == g_subchannel_index.root) {
+      GPR_SWAP(gpr_avl, updated, g_subchannel_index);
+      done = true;
+    }
+    gpr_mu_unlock(&g_mu);
+
+    gpr_avl_unref(updated);
+    gpr_avl_unref(index);
+  }
+
+  leave_ctx(exec_ctx);
+}
diff --git a/src/core/ext/filters/client_channel/subchannel_index.h b/src/core/ext/filters/client_channel/subchannel_index.h
new file mode 100644
index 0000000..83813ca
--- /dev/null
+++ b/src/core/ext/filters/client_channel/subchannel_index.h
@@ -0,0 +1,77 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_SUBCHANNEL_INDEX_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_SUBCHANNEL_INDEX_H
+
+#include "src/core/ext/filters/client_channel/connector.h"
+#include "src/core/ext/filters/client_channel/subchannel.h"
+
+/** \file Provides an index of active subchannels so that they can be
+    shared amongst channels */
+
+typedef struct grpc_subchannel_key grpc_subchannel_key;
+
+/** Create a key that can be used to uniquely identify a subchannel */
+grpc_subchannel_key *grpc_subchannel_key_create(
+    grpc_connector *con, const grpc_subchannel_args *args);
+
+/** Destroy a subchannel key */
+void grpc_subchannel_key_destroy(grpc_exec_ctx *exec_ctx,
+                                 grpc_subchannel_key *key);
+
+/** Given a subchannel key, find the subchannel registered for it.
+    Returns NULL if no such channel exists.
+    Thread-safe. */
+grpc_subchannel *grpc_subchannel_index_find(grpc_exec_ctx *exec_ctx,
+                                            grpc_subchannel_key *key);
+
+/** Register a subchannel against a key.
+    Takes ownership of \a constructed.
+    Returns the registered subchannel. This may be different from
+    \a constructed in the case of a registration race. */
+grpc_subchannel *grpc_subchannel_index_register(grpc_exec_ctx *exec_ctx,
+                                                grpc_subchannel_key *key,
+                                                grpc_subchannel *constructed);
+
+/** Remove \a constructed as the registered subchannel for \a key. */
+void grpc_subchannel_index_unregister(grpc_exec_ctx *exec_ctx,
+                                      grpc_subchannel_key *key,
+                                      grpc_subchannel *constructed);
+
+/** Initialize the subchannel index (global) */
+void grpc_subchannel_index_init(void);
+/** Shutdown the subchannel index (global) */
+void grpc_subchannel_index_shutdown(void);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_SUBCHANNEL_INDEX_H */
diff --git a/src/core/ext/filters/client_channel/uri_parser.c b/src/core/ext/filters/client_channel/uri_parser.c
new file mode 100644
index 0000000..01b9991
--- /dev/null
+++ b/src/core/ext/filters/client_channel/uri_parser.c
@@ -0,0 +1,315 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include "src/core/ext/filters/client_channel/uri_parser.h"
+
+#include <string.h>
+
+#include <grpc/slice_buffer.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/port_platform.h>
+#include <grpc/support/string_util.h>
+
+#include "src/core/lib/slice/percent_encoding.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/slice/slice_string_helpers.h"
+#include "src/core/lib/support/string.h"
+
+/** a size_t default value... maps to all 1's */
+#define NOT_SET (~(size_t)0)
+
+static grpc_uri *bad_uri(const char *uri_text, size_t pos, const char *section,
+                         int suppress_errors) {
+  char *line_prefix;
+  size_t pfx_len;
+
+  if (!suppress_errors) {
+    gpr_asprintf(&line_prefix, "bad uri.%s: '", section);
+    pfx_len = strlen(line_prefix) + pos;
+    gpr_log(GPR_ERROR, "%s%s'", line_prefix, uri_text);
+    gpr_free(line_prefix);
+
+    line_prefix = gpr_malloc(pfx_len + 1);
+    memset(line_prefix, ' ', pfx_len);
+    line_prefix[pfx_len] = 0;
+    gpr_log(GPR_ERROR, "%s^ here", line_prefix);
+    gpr_free(line_prefix);
+  }
+
+  return NULL;
+}
+
+/** Returns a copy of percent decoded \a src[begin, end) */
+static char *decode_and_copy_component(grpc_exec_ctx *exec_ctx, const char *src,
+                                       size_t begin, size_t end) {
+  grpc_slice component =
+      grpc_slice_from_copied_buffer(src + begin, end - begin);
+  grpc_slice decoded_component =
+      grpc_permissive_percent_decode_slice(component);
+  char *out = grpc_dump_slice(decoded_component, GPR_DUMP_ASCII);
+  grpc_slice_unref_internal(exec_ctx, component);
+  grpc_slice_unref_internal(exec_ctx, decoded_component);
+  return out;
+}
+
+/** Returns how many chars to advance if \a uri_text[i] begins a valid \a pchar
+ * production. If \a uri_text[i] introduces an invalid \a pchar (such as percent
+ * sign not followed by two hex digits), NOT_SET is returned. */
+static size_t parse_pchar(const char *uri_text, size_t i) {
+  /* pchar = unreserved / pct-encoded / sub-delims / ":" / "@"
+   * unreserved = ALPHA / DIGIT / "-" / "." / "_" / "~"
+   * pct-encoded = "%" HEXDIG HEXDIG
+   * sub-delims = "!" / "$" / "&" / "'" / "(" / ")"
+   / "*" / "+" / "," / ";" / "=" */
+  char c = uri_text[i];
+  if (((c >= 'A') && (c <= 'Z')) || ((c >= 'a') && (c <= 'z')) ||
+      ((c >= '0') && (c <= '9')) ||
+      (c == '-' || c == '.' || c == '_' || c == '~') || /* unreserved */
+      (c == '!' || c == '$' || c == '&' || c == '\'' || c == '$' || c == '&' ||
+       c == '(' || c == ')' || c == '*' || c == '+' || c == ',' || c == ';' ||
+       c == '=') /* sub-delims */) {
+    return 1;
+  }
+  if (c == '%') { /* pct-encoded */
+    size_t j;
+    if (uri_text[i + 1] == 0 || uri_text[i + 2] == 0) {
+      return NOT_SET;
+    }
+    for (j = i + 1; j < 2; j++) {
+      c = uri_text[j];
+      if (!(((c >= '0') && (c <= '9')) || ((c >= 'a') && (c <= 'f')) ||
+            ((c >= 'A') && (c <= 'F')))) {
+        return NOT_SET;
+      }
+    }
+    return 2;
+  }
+  return 0;
+}
+
+/* *( pchar / "?" / "/" ) */
+static int parse_fragment_or_query(const char *uri_text, size_t *i) {
+  char c;
+  while ((c = uri_text[*i]) != 0) {
+    const size_t advance = parse_pchar(uri_text, *i); /* pchar */
+    switch (advance) {
+      case 0: /* uri_text[i] isn't in pchar */
+        /* maybe it's ? or / */
+        if (uri_text[*i] == '?' || uri_text[*i] == '/') {
+          (*i)++;
+          break;
+        } else {
+          return 1;
+        }
+        GPR_UNREACHABLE_CODE(return 0);
+      default:
+        (*i) += advance;
+        break;
+      case NOT_SET: /* uri_text[i] introduces an invalid URI */
+        return 0;
+    }
+  }
+  /* *i is the first uri_text position past the \a query production, maybe \0 */
+  return 1;
+}
+
+static void parse_query_parts(grpc_uri *uri) {
+  static const char *QUERY_PARTS_SEPARATOR = "&";
+  static const char *QUERY_PARTS_VALUE_SEPARATOR = "=";
+  GPR_ASSERT(uri->query != NULL);
+  if (uri->query[0] == '\0') {
+    uri->query_parts = NULL;
+    uri->query_parts_values = NULL;
+    uri->num_query_parts = 0;
+    return;
+  }
+
+  gpr_string_split(uri->query, QUERY_PARTS_SEPARATOR, &uri->query_parts,
+                   &uri->num_query_parts);
+  uri->query_parts_values = gpr_malloc(uri->num_query_parts * sizeof(char **));
+  for (size_t i = 0; i < uri->num_query_parts; i++) {
+    char **query_param_parts;
+    size_t num_query_param_parts;
+    char *full = uri->query_parts[i];
+    gpr_string_split(full, QUERY_PARTS_VALUE_SEPARATOR, &query_param_parts,
+                     &num_query_param_parts);
+    GPR_ASSERT(num_query_param_parts > 0);
+    uri->query_parts[i] = query_param_parts[0];
+    if (num_query_param_parts > 1) {
+      /* TODO(dgq): only the first value after the separator is considered.
+       * Perhaps all chars after the first separator for the query part should
+       * be included, even if they include the separator. */
+      uri->query_parts_values[i] = query_param_parts[1];
+    } else {
+      uri->query_parts_values[i] = NULL;
+    }
+    for (size_t j = 2; j < num_query_param_parts; j++) {
+      gpr_free(query_param_parts[j]);
+    }
+    gpr_free(query_param_parts);
+    gpr_free(full);
+  }
+}
+
+grpc_uri *grpc_uri_parse(grpc_exec_ctx *exec_ctx, const char *uri_text,
+                         int suppress_errors) {
+  grpc_uri *uri;
+  size_t scheme_begin = 0;
+  size_t scheme_end = NOT_SET;
+  size_t authority_begin = NOT_SET;
+  size_t authority_end = NOT_SET;
+  size_t path_begin = NOT_SET;
+  size_t path_end = NOT_SET;
+  size_t query_begin = NOT_SET;
+  size_t query_end = NOT_SET;
+  size_t fragment_begin = NOT_SET;
+  size_t fragment_end = NOT_SET;
+  size_t i;
+
+  for (i = scheme_begin; uri_text[i] != 0; i++) {
+    if (uri_text[i] == ':') {
+      scheme_end = i;
+      break;
+    }
+    if (uri_text[i] >= 'a' && uri_text[i] <= 'z') continue;
+    if (uri_text[i] >= 'A' && uri_text[i] <= 'Z') continue;
+    if (i != scheme_begin) {
+      if (uri_text[i] >= '0' && uri_text[i] <= '9') continue;
+      if (uri_text[i] == '+') continue;
+      if (uri_text[i] == '-') continue;
+      if (uri_text[i] == '.') continue;
+    }
+    break;
+  }
+  if (scheme_end == NOT_SET) {
+    return bad_uri(uri_text, i, "scheme", suppress_errors);
+  }
+
+  if (uri_text[scheme_end + 1] == '/' && uri_text[scheme_end + 2] == '/') {
+    authority_begin = scheme_end + 3;
+    for (i = authority_begin; uri_text[i] != 0 && authority_end == NOT_SET;
+         i++) {
+      if (uri_text[i] == '/' || uri_text[i] == '?' || uri_text[i] == '#') {
+        authority_end = i;
+      }
+    }
+    if (authority_end == NOT_SET && uri_text[i] == 0) {
+      authority_end = i;
+    }
+    if (authority_end == NOT_SET) {
+      return bad_uri(uri_text, i, "authority", suppress_errors);
+    }
+    /* TODO(ctiller): parse the authority correctly */
+    path_begin = authority_end;
+  } else {
+    path_begin = scheme_end + 1;
+  }
+
+  for (i = path_begin; uri_text[i] != 0; i++) {
+    if (uri_text[i] == '?' || uri_text[i] == '#') {
+      path_end = i;
+      break;
+    }
+  }
+  if (path_end == NOT_SET && uri_text[i] == 0) {
+    path_end = i;
+  }
+  if (path_end == NOT_SET) {
+    return bad_uri(uri_text, i, "path", suppress_errors);
+  }
+
+  if (uri_text[i] == '?') {
+    query_begin = ++i;
+    if (!parse_fragment_or_query(uri_text, &i)) {
+      return bad_uri(uri_text, i, "query", suppress_errors);
+    } else if (uri_text[i] != 0 && uri_text[i] != '#') {
+      /* We must be at the end or at the beginning of a fragment */
+      return bad_uri(uri_text, i, "query", suppress_errors);
+    }
+    query_end = i;
+  }
+  if (uri_text[i] == '#') {
+    fragment_begin = ++i;
+    if (!parse_fragment_or_query(uri_text, &i)) {
+      return bad_uri(uri_text, i - fragment_end, "fragment", suppress_errors);
+    } else if (uri_text[i] != 0) {
+      /* We must be at the end */
+      return bad_uri(uri_text, i, "fragment", suppress_errors);
+    }
+    fragment_end = i;
+  }
+
+  uri = gpr_zalloc(sizeof(*uri));
+  uri->scheme =
+      decode_and_copy_component(exec_ctx, uri_text, scheme_begin, scheme_end);
+  uri->authority = decode_and_copy_component(exec_ctx, uri_text,
+                                             authority_begin, authority_end);
+  uri->path =
+      decode_and_copy_component(exec_ctx, uri_text, path_begin, path_end);
+  uri->query =
+      decode_and_copy_component(exec_ctx, uri_text, query_begin, query_end);
+  uri->fragment = decode_and_copy_component(exec_ctx, uri_text, fragment_begin,
+                                            fragment_end);
+  parse_query_parts(uri);
+
+  return uri;
+}
+
+const char *grpc_uri_get_query_arg(const grpc_uri *uri, const char *key) {
+  GPR_ASSERT(key != NULL);
+  if (key[0] == '\0') return NULL;
+
+  for (size_t i = 0; i < uri->num_query_parts; ++i) {
+    if (0 == strcmp(key, uri->query_parts[i])) {
+      return uri->query_parts_values[i];
+    }
+  }
+  return NULL;
+}
+
+void grpc_uri_destroy(grpc_uri *uri) {
+  if (!uri) return;
+  gpr_free(uri->scheme);
+  gpr_free(uri->authority);
+  gpr_free(uri->path);
+  gpr_free(uri->query);
+  for (size_t i = 0; i < uri->num_query_parts; ++i) {
+    gpr_free(uri->query_parts[i]);
+    gpr_free(uri->query_parts_values[i]);
+  }
+  gpr_free(uri->query_parts);
+  gpr_free(uri->query_parts_values);
+  gpr_free(uri->fragment);
+  gpr_free(uri);
+}
diff --git a/src/core/ext/filters/client_channel/uri_parser.h b/src/core/ext/filters/client_channel/uri_parser.h
new file mode 100644
index 0000000..efd4302
--- /dev/null
+++ b/src/core/ext/filters/client_channel/uri_parser.h
@@ -0,0 +1,65 @@
+/*
+ *
+ * Copyright 2015, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_CLIENT_CHANNEL_URI_PARSER_H
+#define GRPC_CORE_EXT_CLIENT_CHANNEL_URI_PARSER_H
+
+#include <stddef.h>
+#include "src/core/lib/iomgr/exec_ctx.h"
+
+typedef struct {
+  char *scheme;
+  char *authority;
+  char *path;
+  char *query;
+  /** Query substrings separated by '&' */
+  char **query_parts;
+  /** Number of elements in \a query_parts and \a query_parts_values */
+  size_t num_query_parts;
+  /** Split each query part by '='. NULL if not present. */
+  char **query_parts_values;
+  char *fragment;
+} grpc_uri;
+
+/** parse a uri, return NULL on failure */
+grpc_uri *grpc_uri_parse(grpc_exec_ctx *exec_ctx, const char *uri_text,
+                         int suppress_errors);
+
+/** return the part of a query string after the '=' in "?key=xxx&...", or NULL
+ * if key is not present */
+const char *grpc_uri_get_query_arg(const grpc_uri *uri, const char *key);
+
+/** destroy a uri */
+void grpc_uri_destroy(grpc_uri *uri);
+
+#endif /* GRPC_CORE_EXT_CLIENT_CHANNEL_URI_PARSER_H */
diff --git a/src/core/ext/filters/load_reporting/load_reporting.c b/src/core/ext/filters/load_reporting/load_reporting.c
new file mode 100644
index 0000000..9fb33ba
--- /dev/null
+++ b/src/core/ext/filters/load_reporting/load_reporting.c
@@ -0,0 +1,104 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <limits.h>
+#include <string.h>
+
+#include <grpc/load_reporting.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/sync.h>
+
+#include "src/core/ext/filters/load_reporting/load_reporting.h"
+#include "src/core/ext/filters/load_reporting/load_reporting_filter.h"
+#include "src/core/lib/channel/channel_stack_builder.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/surface/call.h"
+#include "src/core/lib/surface/channel_init.h"
+
+static void destroy_lr_cost_context(void *c) {
+  grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
+  grpc_load_reporting_cost_context *cost_ctx = c;
+  for (size_t i = 0; i < cost_ctx->values_count; ++i) {
+    grpc_slice_unref_internal(&exec_ctx, cost_ctx->values[i]);
+  }
+  grpc_exec_ctx_finish(&exec_ctx);
+  gpr_free(cost_ctx->values);
+  gpr_free(cost_ctx);
+}
+
+void grpc_call_set_load_reporting_cost_context(
+    grpc_call *call, grpc_load_reporting_cost_context *ctx) {
+  grpc_call_context_set(call, GRPC_CONTEXT_LR_COST, ctx,
+                        destroy_lr_cost_context);
+}
+
+static bool is_load_reporting_enabled(const grpc_channel_args *a) {
+  if (a == NULL) return false;
+  for (size_t i = 0; i < a->num_args; i++) {
+    if (0 == strcmp(a->args[i].key, GRPC_ARG_ENABLE_LOAD_REPORTING)) {
+      return a->args[i].type == GRPC_ARG_INTEGER &&
+             a->args[i].value.integer != 0;
+    }
+  }
+  return false;
+}
+
+static bool maybe_add_load_reporting_filter(grpc_exec_ctx *exec_ctx,
+                                            grpc_channel_stack_builder *builder,
+                                            void *arg) {
+  const grpc_channel_args *args =
+      grpc_channel_stack_builder_get_channel_arguments(builder);
+  if (is_load_reporting_enabled(args)) {
+    return grpc_channel_stack_builder_prepend_filter(
+        builder, (const grpc_channel_filter *)arg, NULL, NULL);
+  }
+  return true;
+}
+
+grpc_arg grpc_load_reporting_enable_arg() {
+  grpc_arg arg;
+  arg.type = GRPC_ARG_INTEGER;
+  arg.key = GRPC_ARG_ENABLE_LOAD_REPORTING;
+  arg.value.integer = 1;
+  return arg;
+}
+
+/* Plugin registration */
+
+void grpc_load_reporting_plugin_init(void) {
+  grpc_channel_init_register_stage(GRPC_SERVER_CHANNEL, INT_MAX,
+                                   maybe_add_load_reporting_filter,
+                                   (void *)&grpc_load_reporting_filter);
+}
+
+void grpc_load_reporting_plugin_shutdown() {}
diff --git a/src/core/ext/filters/load_reporting/load_reporting.h b/src/core/ext/filters/load_reporting/load_reporting.h
new file mode 100644
index 0000000..22859a5
--- /dev/null
+++ b/src/core/ext/filters/load_reporting/load_reporting.h
@@ -0,0 +1,73 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_LOAD_REPORTING_LOAD_REPORTING_H
+#define GRPC_CORE_EXT_LOAD_REPORTING_LOAD_REPORTING_H
+
+#include <grpc/impl/codegen/grpc_types.h>
+
+#include "src/core/lib/channel/channel_stack.h"
+
+/** Identifiers for the invocation point of the users LR callback */
+typedef enum grpc_load_reporting_source {
+  GRPC_LR_POINT_UNKNOWN = 0,
+  GRPC_LR_POINT_CHANNEL_CREATION,
+  GRPC_LR_POINT_CHANNEL_DESTRUCTION,
+  GRPC_LR_POINT_CALL_CREATION,
+  GRPC_LR_POINT_CALL_DESTRUCTION
+} grpc_load_reporting_source;
+
+/** Call information to be passed to the provided LR callback. */
+typedef struct grpc_load_reporting_call_data {
+  const grpc_load_reporting_source source; /**< point of last data update. */
+
+  /** Unique identifier for the channel associated with the data */
+  intptr_t channel_id;
+
+  /** Unique identifier for the call associated with the data. If the call
+   * hasn't been created yet, it'll have a value of zero. */
+  intptr_t call_id;
+
+  /** Only valid when \a source is \a GRPC_LR_POINT_CALL_DESTRUCTION, that is,
+   * once the call has completed */
+  const grpc_call_final_info *final_info;
+
+  const char *initial_md_string;  /**< value string for LR's initial md key */
+  const char *trailing_md_string; /**< value string for LR's trailing md key */
+  const char *method_name;        /**< Corresponds to :path header */
+} grpc_load_reporting_call_data;
+
+/** Return a \a grpc_arg enabling load reporting */
+grpc_arg grpc_load_reporting_enable_arg();
+
+#endif /* GRPC_CORE_EXT_LOAD_REPORTING_LOAD_REPORTING_H */
diff --git a/src/core/ext/filters/load_reporting/load_reporting_filter.c b/src/core/ext/filters/load_reporting/load_reporting_filter.c
new file mode 100644
index 0000000..7870ec0
--- /dev/null
+++ b/src/core/ext/filters/load_reporting/load_reporting_filter.c
@@ -0,0 +1,215 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#include <string.h>
+
+#include <grpc/load_reporting.h>
+#include <grpc/support/alloc.h>
+#include <grpc/support/log.h>
+#include <grpc/support/string_util.h>
+#include <grpc/support/sync.h>
+
+#include "src/core/ext/filters/load_reporting/load_reporting.h"
+#include "src/core/ext/filters/load_reporting/load_reporting_filter.h"
+#include "src/core/lib/channel/channel_args.h"
+#include "src/core/lib/profiling/timers.h"
+#include "src/core/lib/slice/slice_internal.h"
+#include "src/core/lib/transport/static_metadata.h"
+
+typedef struct call_data {
+  intptr_t id; /**< an id unique to the call */
+  bool have_initial_md_string;
+  grpc_slice initial_md_string;
+  bool have_service_method;
+  grpc_slice service_method;
+
+  /* stores the recv_initial_metadata op's ready closure, which we wrap with our
+   * own (on_initial_md_ready) in order to capture the incoming initial metadata
+   * */
+  grpc_closure *ops_recv_initial_metadata_ready;
+
+  /* to get notified of the availability of the incoming initial metadata. */
+  grpc_closure on_initial_md_ready;
+  grpc_metadata_batch *recv_initial_metadata;
+} call_data;
+
+typedef struct channel_data {
+  intptr_t id; /**< an id unique to the channel */
+} channel_data;
+
+static void on_initial_md_ready(grpc_exec_ctx *exec_ctx, void *user_data,
+                                grpc_error *err) {
+  grpc_call_element *elem = user_data;
+  call_data *calld = elem->call_data;
+
+  if (err == GRPC_ERROR_NONE) {
+    if (calld->recv_initial_metadata->idx.named.path != NULL) {
+      calld->service_method = grpc_slice_ref_internal(
+          GRPC_MDVALUE(calld->recv_initial_metadata->idx.named.path->md));
+      calld->have_service_method = true;
+    } else {
+      err = grpc_error_add_child(
+          err, GRPC_ERROR_CREATE_FROM_STATIC_STRING("Missing :path header"));
+    }
+    if (calld->recv_initial_metadata->idx.named.lb_token != NULL) {
+      calld->initial_md_string = grpc_slice_ref_internal(
+          GRPC_MDVALUE(calld->recv_initial_metadata->idx.named.lb_token->md));
+      calld->have_initial_md_string = true;
+      grpc_metadata_batch_remove(
+          exec_ctx, calld->recv_initial_metadata,
+          calld->recv_initial_metadata->idx.named.lb_token);
+    }
+  } else {
+    GRPC_ERROR_REF(err);
+  }
+  calld->ops_recv_initial_metadata_ready->cb(
+      exec_ctx, calld->ops_recv_initial_metadata_ready->cb_arg, err);
+  GRPC_ERROR_UNREF(err);
+}
+
+/* Constructor for call_data */
+static grpc_error *init_call_elem(grpc_exec_ctx *exec_ctx,
+                                  grpc_call_element *elem,
+                                  const grpc_call_element_args *args) {
+  call_data *calld = elem->call_data;
+  calld->id = (intptr_t)args->call_stack;
+  grpc_closure_init(&calld->on_initial_md_ready, on_initial_md_ready, elem,
+                    grpc_schedule_on_exec_ctx);
+
+  /* TODO(dgq): do something with the data
+  channel_data *chand = elem->channel_data;
+  grpc_load_reporting_call_data lr_call_data = {GRPC_LR_POINT_CALL_CREATION,
+                                                (intptr_t)chand->id,
+                                                (intptr_t)calld->id,
+                                                NULL,
+                                                NULL,
+                                                NULL,
+                                                NULL};
+  */
+
+  return GRPC_ERROR_NONE;
+}
+
+/* Destructor for call_data */
+static void destroy_call_elem(grpc_exec_ctx *exec_ctx, grpc_call_element *elem,
+                              const grpc_call_final_info *final_info,
+                              grpc_closure *ignored) {
+  call_data *calld = elem->call_data;
+
+  /* TODO(dgq): do something with the data
+  channel_data *chand = elem->channel_data;
+  grpc_load_reporting_call_data lr_call_data = {GRPC_LR_POINT_CALL_DESTRUCTION,
+                                                (intptr_t)chand->id,
+                                                (intptr_t)calld->id,
+                                                final_info,
+                                                calld->initial_md_string,
+                                                calld->trailing_md_string,
+                                                calld->service_method};
+  */
+
+  if (calld->have_initial_md_string) {
+    grpc_slice_unref_internal(exec_ctx, calld->initial_md_string);
+  }
+  if (calld->have_service_method) {
+    grpc_slice_unref_internal(exec_ctx, calld->service_method);
+  }
+}
+
+/* Constructor for channel_data */
+static grpc_error *init_channel_elem(grpc_exec_ctx *exec_ctx,
+                                     grpc_channel_element *elem,
+                                     grpc_channel_element_args *args) {
+  GPR_ASSERT(!args->is_last);
+
+  channel_data *chand = elem->channel_data;
+  chand->id = (intptr_t)args->channel_stack;
+
+  /* TODO(dgq): do something with the data
+  grpc_load_reporting_call_data lr_call_data = {GRPC_LR_POINT_CHANNEL_CREATION,
+                                                (intptr_t)chand,
+                                                0,
+                                                NULL,
+                                                NULL,
+                                                NULL,
+                                                NULL};
+                                                */
+
+  return GRPC_ERROR_NONE;
+}
+
+/* Destructor for channel data */
+static void destroy_channel_elem(grpc_exec_ctx *exec_ctx,
+                                 grpc_channel_element *elem) {
+  /* TODO(dgq): do something with the data
+  channel_data *chand = elem->channel_data;
+  grpc_load_reporting_call_data lr_call_data = {
+      GRPC_LR_POINT_CHANNEL_DESTRUCTION,
+      (intptr_t)chand->id,
+      0,
+      NULL,
+      NULL,
+      NULL,
+      NULL};
+  */
+}
+
+static void lr_start_transport_stream_op(grpc_exec_ctx *exec_ctx,
+                                         grpc_call_element *elem,
+                                         grpc_transport_stream_op *op) {
+  GPR_TIMER_BEGIN("lr_start_transport_stream_op", 0);
+  call_data *calld = elem->call_data;
+
+  if (op->recv_initial_metadata) {
+    calld->recv_initial_metadata = op->recv_initial_metadata;
+    /* substitute our callback for the higher callback */
+    calld->ops_recv_initial_metadata_ready = op->recv_initial_metadata_ready;
+    op->recv_initial_metadata_ready = &calld->on_initial_md_ready;
+  }
+  grpc_call_next_op(exec_ctx, elem, op);
+
+  GPR_TIMER_END("lr_start_transport_stream_op", 0);
+}
+
+const grpc_channel_filter grpc_load_reporting_filter = {
+    lr_start_transport_stream_op,
+    grpc_channel_next_op,
+    sizeof(call_data),
+    init_call_elem,
+    grpc_call_stack_ignore_set_pollset_or_pollset_set,
+    destroy_call_elem,
+    sizeof(channel_data),
+    init_channel_elem,
+    destroy_channel_elem,
+    grpc_call_next_get_peer,
+    grpc_channel_next_get_info,
+    "load_reporting"};
diff --git a/src/core/ext/filters/load_reporting/load_reporting_filter.h b/src/core/ext/filters/load_reporting/load_reporting_filter.h
new file mode 100644
index 0000000..a6ce21e
--- /dev/null
+++ b/src/core/ext/filters/load_reporting/load_reporting_filter.h
@@ -0,0 +1,42 @@
+/*
+ *
+ * Copyright 2016, Google Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are
+ * met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above
+ * copyright notice, this list of conditions and the following disclaimer
+ * in the documentation and/or other materials provided with the
+ * distribution.
+ *     * Neither the name of Google Inc. nor the names of its
+ * contributors may be used to endorse or promote products derived from
+ * this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+#ifndef GRPC_CORE_EXT_LOAD_REPORTING_LOAD_REPORTING_FILTER_H
+#define GRPC_CORE_EXT_LOAD_REPORTING_LOAD_REPORTING_FILTER_H
+
+#include "src/core/ext/filters/load_reporting/load_reporting.h"
+#include "src/core/lib/channel/channel_stack.h"
+
+extern const grpc_channel_filter grpc_load_reporting_filter;
+
+#endif /* GRPC_CORE_EXT_LOAD_REPORTING_LOAD_REPORTING_FILTER_H */