You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zw...@apache.org on 2015/12/21 20:44:57 UTC

[06/16] trafficserver git commit: TS-3418: Refactored parent selection to add a secondary parent consistent hash ring.

TS-3418: Refactored parent selection to add a secondary parent consistent hash ring.


Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/0e7975af
Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/0e7975af
Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/0e7975af

Branch: refs/heads/6.1.x
Commit: 0e7975afe4d7400f22fe4c251e3fcf223a0a8295
Parents: 1c64869
Author: John J. Rushford <Jo...@cable.comcast.com>
Authored: Wed Dec 9 22:06:30 2015 +0000
Committer: James Peach <jp...@apache.org>
Committed: Thu Dec 17 20:23:39 2015 -0800

----------------------------------------------------------------------
 doc/admin-guide/files/parent.config.en.rst |   9 +
 iocore/net/Socks.cc                        |   2 +-
 lib/ts/ConsistentHash.cc                   |  22 +-
 lib/ts/ConsistentHash.h                    |   9 +-
 proxy/Makefile.am                          |   6 +
 proxy/ParentConsistentHash.cc              | 340 ++++++++++
 proxy/ParentConsistentHash.h               |  63 ++
 proxy/ParentRoundRobin.cc                  | 274 ++++++++
 proxy/ParentRoundRobin.h                   |  48 ++
 proxy/ParentSelection.cc                   | 816 ++++++++----------------
 proxy/ParentSelection.h                    | 264 ++++----
 proxy/http/HttpSM.cc                       |   2 +-
 proxy/http/HttpTransact.cc                 |  24 +-
 13 files changed, 1200 insertions(+), 679 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/doc/admin-guide/files/parent.config.en.rst
----------------------------------------------------------------------
diff --git a/doc/admin-guide/files/parent.config.en.rst b/doc/admin-guide/files/parent.config.en.rst
index ecd1ad2..e3d8952 100644
--- a/doc/admin-guide/files/parent.config.en.rst
+++ b/doc/admin-guide/files/parent.config.en.rst
@@ -129,6 +129,15 @@ The following list shows the possible actions and their allowed values.
     origin server. You can specify either a hostname or an IP address,
     but; you must specify the port number.
 
+.. _parent-config-format-secondary-parent:
+
+``secondary_parent``
+    An optional ordered list of secondary parent servers.  This optional
+    list may only be used when ``round_robin`` is set to ``consistent_hash``.
+    If the request cannot be handled by a parent server from the ``parent``
+    list, then the request will be re-tried from a server found in this list
+    using a consistent hash of the url.
+
 .. _parent-config-format-round-robin:
 
 ``round_robin``

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/iocore/net/Socks.cc
----------------------------------------------------------------------
diff --git a/iocore/net/Socks.cc b/iocore/net/Socks.cc
index 78ccd9a..a4b0330 100644
--- a/iocore/net/Socks.cc
+++ b/iocore/net/Socks.cc
@@ -148,7 +148,7 @@ SocksEntry::free()
 
 #ifdef SOCKS_WITH_TS
   if (!lerrno && netVConnection && server_result.retry)
-    server_params->recordRetrySuccess(&server_result);
+    server_params->markParentUp(&server_result);
 #endif
 
   if ((action_.cancelled || lerrno) && netVConnection)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/lib/ts/ConsistentHash.cc
----------------------------------------------------------------------
diff --git a/lib/ts/ConsistentHash.cc b/lib/ts/ConsistentHash.cc
index 9acbab6..fe919eb 100644
--- a/lib/ts/ConsistentHash.cc
+++ b/lib/ts/ConsistentHash.cc
@@ -19,7 +19,7 @@
   limitations under the License.
  */
 
-#include "ts/ConsistentHash.h"
+#include "ConsistentHash.h"
 #include <cstring>
 #include <string>
 #include <sstream>
@@ -67,13 +67,19 @@ ATSConsistentHash::insert(ATSConsistentHashNode *node, float weight, ATSHash64 *
 }
 
 ATSConsistentHashNode *
-ATSConsistentHash::lookup(const char *url, ATSConsistentHashIter *i, bool *w, ATSHash64 *h)
+ATSConsistentHash::lookup(const char *url, size_t url_len, ATSConsistentHashIter *i, bool *w, ATSHash64 *h)
 {
   uint64_t url_hash;
   ATSConsistentHashIter NodeMapIterUp, *iter;
   ATSHash64 *thash;
   bool *wptr, wrapped = false;
 
+  if (url_len <= 0 && url) {
+    url_len = strlen(url);
+  } else {
+    url_len = 0;
+  }
+
   if (h) {
     thash = h;
   } else if (hash) {
@@ -95,7 +101,7 @@ ATSConsistentHash::lookup(const char *url, ATSConsistentHashIter *i, bool *w, AT
   }
 
   if (url) {
-    thash->update(url, strlen(url));
+    thash->update(url, url_len);
     thash->final();
     url_hash = thash->get();
     thash->clear();
@@ -123,13 +129,19 @@ ATSConsistentHash::lookup(const char *url, ATSConsistentHashIter *i, bool *w, AT
 }
 
 ATSConsistentHashNode *
-ATSConsistentHash::lookup_available(const char *url, ATSConsistentHashIter *i, bool *w, ATSHash64 *h)
+ATSConsistentHash::lookup_available(const char *url, size_t url_len, ATSConsistentHashIter *i, bool *w, ATSHash64 *h)
 {
   uint64_t url_hash;
   ATSConsistentHashIter NodeMapIterUp, *iter;
   ATSHash64 *thash;
   bool *wptr, wrapped = false;
 
+  if (url_len <= 0 && url) {
+    url_len = strlen(url);
+  } else {
+    url_len = 0;
+  }
+
   if (h) {
     thash = h;
   } else if (hash) {
@@ -151,7 +163,7 @@ ATSConsistentHash::lookup_available(const char *url, ATSConsistentHashIter *i, b
   }
 
   if (url) {
-    thash->update(url, strlen(url));
+    thash->update(url, url_len);
     thash->final();
     url_hash = thash->get();
     thash->clear();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/lib/ts/ConsistentHash.h
----------------------------------------------------------------------
diff --git a/lib/ts/ConsistentHash.h b/lib/ts/ConsistentHash.h
index d1a34ba..49822ad 100644
--- a/lib/ts/ConsistentHash.h
+++ b/lib/ts/ConsistentHash.h
@@ -22,7 +22,7 @@
 #ifndef __CONSISTENT_HASH_H__
 #define __CONSISTENT_HASH_H__
 
-#include "ts/Hash.h"
+#include "Hash.h"
 #include <stdint.h>
 #include <iostream>
 #include <map>
@@ -49,9 +49,10 @@ typedef std::map<uint64_t, ATSConsistentHashNode *>::iterator ATSConsistentHashI
 struct ATSConsistentHash {
   ATSConsistentHash(int r = 1024, ATSHash64 *h = NULL);
   void insert(ATSConsistentHashNode *node, float weight = 1.0, ATSHash64 *h = NULL);
-  ATSConsistentHashNode *lookup(const char *url = NULL, ATSConsistentHashIter *i = NULL, bool *w = NULL, ATSHash64 *h = NULL);
-  ATSConsistentHashNode *lookup_available(const char *url = NULL, ATSConsistentHashIter *i = NULL, bool *w = NULL,
-                                          ATSHash64 *h = NULL);
+  ATSConsistentHashNode *lookup(const char *url = NULL, size_t url_len = 0, ATSConsistentHashIter *i = NULL, bool *w = NULL,
+                                ATSHash64 *h = NULL);
+  ATSConsistentHashNode *lookup_available(const char *url = NULL, size_t url_len = 0, ATSConsistentHashIter *i = NULL,
+                                          bool *w = NULL, ATSHash64 *h = NULL);
   ATSConsistentHashNode *lookup_by_hashval(uint64_t hashval, ATSConsistentHashIter *i = NULL, bool *w = NULL);
   ~ATSConsistentHash();
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/Makefile.am
----------------------------------------------------------------------
diff --git a/proxy/Makefile.am b/proxy/Makefile.am
index 0e49606..2b07237 100644
--- a/proxy/Makefile.am
+++ b/proxy/Makefile.am
@@ -162,6 +162,10 @@ traffic_server_SOURCES = \
   InkIOCoreAPI.cc \
   Main.cc \
   Main.h \
+  ParentConsistentHash.cc \
+  ParentConsistentHash.h \
+  ParentRoundRobin.cc \
+  ParentRoundRobin.h \
   ParentSelection.cc \
   ParentSelection.h \
   Plugin.cc \
@@ -276,6 +280,8 @@ traffic_sac_SOURCES = \
   ICPProcessor.cc \
   ICPStats.cc \
   IPAllow.cc \
+  ParentConsistentHash.cc \
+  ParentRoundRobin.cc \
   ParentSelection.cc \
   ControlBase.cc \
   ControlMatcher.cc \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/ParentConsistentHash.cc
----------------------------------------------------------------------
diff --git a/proxy/ParentConsistentHash.cc b/proxy/ParentConsistentHash.cc
new file mode 100644
index 0000000..48d7037
--- /dev/null
+++ b/proxy/ParentConsistentHash.cc
@@ -0,0 +1,340 @@
+/** @file
+
+  Implementation of Parent Proxy routing
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+#include "ParentConsistentHash.h"
+
+ParentConsistentHash::ParentConsistentHash(ParentRecord *parent_record)
+{
+  int i;
+
+  ink_assert(parent_record->num_parents > 0);
+  parents[PRIMARY] = parent_record->parents;
+  parents[SECONDARY] = parent_record->secondary_parents;
+  ignore_query = parent_record->ignore_query;
+  memset(foundParents, 0, sizeof(foundParents));
+
+  chash[PRIMARY] = new ATSConsistentHash();
+
+  for (i = 0; i < parent_record->num_parents; i++) {
+    chash[PRIMARY]->insert(&(parent_record->parents[i]), parent_record->parents[i].weight, (ATSHash64 *)&hash[PRIMARY]);
+  }
+
+  if (parent_record->num_secondary_parents > 0) {
+    Debug("parent_select", "ParentConsistentHash(): initializing the secondary parents hash.");
+    chash[SECONDARY] = new ATSConsistentHash();
+
+    for (i = 0; i < parent_record->num_secondary_parents; i++) {
+      chash[SECONDARY]->insert(&(parent_record->secondary_parents[i]), parent_record->secondary_parents[i].weight,
+                               (ATSHash64 *)&hash[SECONDARY]);
+    }
+  } else {
+    chash[SECONDARY] = NULL;
+  }
+  Debug("parent_select", "Using a consistent hash parent selection strategy.");
+}
+
+ParentConsistentHash::~ParentConsistentHash()
+{
+  if (chash[PRIMARY]) {
+    delete chash[PRIMARY];
+  }
+  if (chash[SECONDARY]) {
+    delete chash[SECONDARY];
+  }
+}
+
+
+uint64_t
+ParentConsistentHash::getPathHash(HttpRequestData *hrdata, ATSHash64 *h)
+{
+  const char *tmp = NULL;
+  int len;
+  URL *url = hrdata->hdr->url_get();
+
+  // Always hash on '/' because paths returned by ATS are always stripped of it
+  h->update("/", 1);
+
+  tmp = url->path_get(&len);
+  if (tmp) {
+    h->update(tmp, len);
+  }
+
+  if (!ignore_query) {
+    tmp = url->query_get(&len);
+    if (tmp) {
+      h->update("?", 1);
+      h->update(tmp, len);
+    }
+  }
+
+  h->final();
+
+  return h->get();
+}
+
+void
+ParentConsistentHash::selectParent(const ParentSelectionPolicy *policy, bool first_call, ParentResult *result, RequestData *rdata)
+{
+  ATSHash64Sip24 hash;
+  ATSConsistentHash *fhash;
+  HttpRequestData *request_info = static_cast<HttpRequestData *>(rdata);
+  bool firstCall = first_call;
+  bool parentRetry = false;
+  bool wrap_around[2] = {false, false};
+  uint64_t path_hash = 0;
+  uint32_t last_lookup;
+  pRecord *prtmp = NULL, *pRec = NULL;
+
+  Debug("parent_select", "ParentConsistentHash::%s(): Using a consistent hash parent selection strategy.", __func__);
+  ink_assert(numParents(result) > 0 || result->rec->go_direct == true);
+
+  // Should only get into this state if we are supposed to go direct.
+  if (parents[PRIMARY] == NULL && parents[SECONDARY] == NULL) {
+    if (result->rec->go_direct == true) {
+      result->r = PARENT_DIRECT;
+    } else {
+      result->r = PARENT_FAIL;
+    }
+    result->hostname = NULL;
+    result->port = 0;
+    return;
+  }
+
+  // findParent() call if firstCall.
+  if (firstCall) {
+    last_lookup = PRIMARY;
+    path_hash = getPathHash(request_info, (ATSHash64 *)&hash);
+    fhash = chash[PRIMARY];
+    if (path_hash) {
+      prtmp = (pRecord *)fhash->lookup_by_hashval(path_hash, &chashIter[last_lookup], &wrap_around[last_lookup]);
+      if (prtmp)
+        pRec = (parents[last_lookup] + prtmp->idx);
+    }
+    // else called by nextParent().
+  } else {
+    if (chash[SECONDARY] != NULL) {
+      last_lookup = SECONDARY;
+      fhash = chash[SECONDARY];
+      path_hash = getPathHash(request_info, (ATSHash64 *)&hash);
+      if (path_hash) {
+        prtmp = (pRecord *)fhash->lookup_by_hashval(path_hash, &chashIter[last_lookup], &wrap_around[last_lookup]);
+        if (prtmp)
+          pRec = (parents[last_lookup] + prtmp->idx);
+      }
+    } else {
+      last_lookup = PRIMARY;
+      fhash = chash[PRIMARY];
+      do { // search until we've selected a different parent.
+        prtmp = (pRecord *)fhash->lookup(NULL, 0, &chashIter[last_lookup], &wrap_around[last_lookup], &hash);
+        if (prtmp)
+          pRec = (parents[last_lookup] + prtmp->idx);
+      } while (prtmp && strcmp(prtmp->hostname, result->hostname) == 0);
+    }
+  }
+
+  // didn't find a parent or the parent is marked unavailable.
+  if (!pRec || (pRec && !pRec->available)) {
+    do {
+      if (pRec && !pRec->available) {
+        Debug("parent_select", "Parent.failedAt = %u, retry = %u, xact_start = %u", (unsigned int)pRec->failedAt,
+              (unsigned int)policy->ParentRetryTime, (unsigned int)request_info->xact_start);
+        if ((pRec->failedAt + policy->ParentRetryTime) < request_info->xact_start) {
+          parentRetry = true;
+          // make sure that the proper state is recorded in the result structure
+          // so that markParentUp() finds the proper record.
+          result->last_parent = prtmp->idx;
+          result->last_lookup = last_lookup;
+          result->retry = parentRetry;
+          result->r = PARENT_SPECIFIED;
+          markParentUp(result);
+          Debug("parent_select", "Down parent %s is now retryable, marked it available.", pRec->hostname);
+          break;
+        }
+      }
+      Debug("parent_select", "wrap_around[PRIMARY]: %d, wrap_around[SECONDARY]: %d", wrap_around[PRIMARY], wrap_around[SECONDARY]);
+      if (!wrap_around[PRIMARY] || (chash[SECONDARY] != NULL)) {
+        Debug("parent_select", "Selected parent %s is not available, looking up another parent.", pRec->hostname);
+        if (chash[SECONDARY] != NULL && !wrap_around[SECONDARY]) {
+          fhash = chash[SECONDARY];
+          last_lookup = SECONDARY;
+        } else {
+          fhash = chash[PRIMARY];
+          last_lookup = PRIMARY;
+        }
+        if (firstCall) {
+          prtmp = (pRecord *)fhash->lookup_by_hashval(path_hash, &chashIter[last_lookup], &wrap_around[last_lookup]);
+          firstCall = false;
+        } else {
+          prtmp = (pRecord *)fhash->lookup(NULL, 0, &chashIter[last_lookup], &wrap_around[last_lookup], &hash);
+        }
+
+        if (prtmp) {
+          pRec = (parents[last_lookup] + prtmp->idx);
+          Debug("parent_select", "Selected a new parent: %s.", pRec->hostname);
+        }
+      }
+      if (wrap_around[PRIMARY] && chash[SECONDARY] == NULL) {
+        Debug("parent_select", "No available parents.");
+        break;
+      }
+      if (wrap_around[PRIMARY] && chash[SECONDARY] != NULL && wrap_around[SECONDARY]) {
+        Debug("parent_select", "No available parents.");
+        break;
+      }
+    } while (!prtmp || !pRec->available);
+  }
+
+  // use the available parent.
+  if (pRec && pRec->available) {
+    result->r = PARENT_SPECIFIED;
+    result->hostname = pRec->hostname;
+    result->port = pRec->port;
+    result->last_parent = pRec->idx;
+    result->last_lookup = last_lookup;
+    result->retry = parentRetry;
+    ink_assert(result->hostname != NULL);
+    ink_assert(result->port != 0);
+    Debug("parent_select", "Chosen parent: %s.%d", result->hostname, result->port);
+  } else {
+    if (result->rec->go_direct == true) {
+      result->r = PARENT_DIRECT;
+    } else {
+      result->r = PARENT_FAIL;
+    }
+    result->hostname = NULL;
+    result->port = 0;
+    result->retry = false;
+  }
+
+  return;
+}
+
+void
+ParentConsistentHash::markParentDown(const ParentSelectionPolicy *policy, ParentResult *result)
+{
+  time_t now;
+  pRecord *pRec;
+  int new_fail_count = 0;
+
+  Debug("parent_select", "Starting ParentConsistentHash::markParentDown()");
+
+  //  Make sure that we are being called back with with a
+  //   result structure with a parent
+  ink_assert(result->r == PARENT_SPECIFIED);
+  if (result->r != PARENT_SPECIFIED) {
+    return;
+  }
+  // If we were set through the API we currently have not failover
+  //   so just return fail
+  if (result->rec == extApiRecord) {
+    return;
+  }
+
+  ink_assert((result->last_parent) < numParents(result));
+  pRec = parents[result->last_lookup] + result->last_parent;
+
+  // If the parent has already been marked down, just increment
+  //   the failure count.  If this is the first mark down on a
+  //   parent we need to both set the failure time and set
+  //   count to one.  It's possible for the count and time get out
+  //   sync due there being no locks.  Therefore the code should
+  //   handle this condition.  If this was the result of a retry, we
+  //   must update move the failedAt timestamp to now so that we continue
+  //   negative cache the parent
+  if (pRec->failedAt == 0 || result->retry == true) {
+    // Reread the current time.  We want this to be accurate since
+    //   it relates to how long the parent has been down.
+    now = time(NULL);
+
+    // Mark the parent as down
+    ink_atomic_swap(&pRec->failedAt, now);
+
+    // If this is clean mark down and not a failed retry, we
+    //   must set the count to reflect this
+    if (result->retry == false) {
+      new_fail_count = pRec->failCount = 1;
+    }
+
+    Note("Parent %s marked as down %s:%d", (result->retry) ? "retry" : "initially", pRec->hostname, pRec->port);
+
+  } else {
+    int old_count = ink_atomic_increment(&pRec->failCount, 1);
+
+    Debug("parent_select", "Parent fail count increased to %d for %s:%d", old_count + 1, pRec->hostname, pRec->port);
+    new_fail_count = old_count + 1;
+  }
+
+  if (new_fail_count > 0 && new_fail_count == policy->FailThreshold) {
+    Note("Failure threshold met, http parent proxy %s:%d marked down", pRec->hostname, pRec->port);
+    pRec->available = false;
+    Debug("parent_select", "Parent %s:%d marked unavailable, pRec->available=%d", pRec->hostname, pRec->port, pRec->available);
+  }
+}
+
+uint32_t
+ParentConsistentHash::numParents(ParentResult *result)
+{
+  uint32_t n = 0;
+
+  switch (result->last_lookup) {
+  case PRIMARY:
+    n = result->rec->num_parents;
+    break;
+  case SECONDARY:
+    n = result->rec->num_secondary_parents;
+    break;
+  }
+
+  return n;
+}
+
+void
+ParentConsistentHash::markParentUp(ParentResult *result)
+{
+  pRecord *pRec;
+
+  //  Make sure that we are being called back with with a
+  //   result structure with a parent that is being retried
+  ink_release_assert(result->retry == true);
+  ink_assert(result->r == PARENT_SPECIFIED);
+  if (result->r != PARENT_SPECIFIED) {
+    return;
+  }
+  // If we were set through the API we currently have not failover
+  //   so just return fail
+  if (result->rec == extApiRecord) {
+    ink_assert(0);
+    return;
+  }
+
+  ink_assert((result->last_parent) < numParents(result));
+  pRec = parents[result->last_lookup] + result->last_parent;
+  pRec->available = true;
+  Debug("parent_select", "%s:%s(): marked %s:%d available.", __FILE__, __func__, pRec->hostname, pRec->port);
+
+  ink_atomic_swap(&pRec->failedAt, (time_t)0);
+  int old_count = ink_atomic_swap(&pRec->failCount, 0);
+
+  if (old_count > 0) {
+    Note("http parent proxy %s:%d restored", pRec->hostname, pRec->port);
+  }
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/ParentConsistentHash.h
----------------------------------------------------------------------
diff --git a/proxy/ParentConsistentHash.h b/proxy/ParentConsistentHash.h
new file mode 100644
index 0000000..ff09072
--- /dev/null
+++ b/proxy/ParentConsistentHash.h
@@ -0,0 +1,63 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+/*****************************************************************************
+ *
+ *  ParentConsistentHash.h - Interface to Parent Consistent Hash.
+ *
+ ****************************************************************************/
+
+#ifndef _PARENT_CONSISTENT_HASH_H
+#define _PARENT_CONSISTENT_HASH_H
+
+#include "ts/HashSip.h"
+#include "ParentSelection.h"
+
+//
+//  Implementation of round robin based upon consistent hash of the URL,
+//  ParentRR_t = P_CONSISTENT_HASH.
+//
+class ParentConsistentHash : public ParentSelectionStrategy
+{
+  // there are two hashes PRIMARY parents
+  // and SECONDARY parents.
+  ATSHash64Sip24 hash[2];
+  ATSConsistentHash *chash[2];
+  ATSConsistentHashIter chashIter[2];
+  pRecord *parents[2];
+  bool foundParents[2][MAX_PARENTS];
+  bool ignore_query;
+
+public:
+  static const int PRIMARY = 0;
+  static const int SECONDARY = 1;
+  ParentConsistentHash(ParentRecord *_parent_record);
+  ~ParentConsistentHash();
+  uint64_t getPathHash(HttpRequestData *hrdata, ATSHash64 *h);
+  void selectParent(const ParentSelectionPolicy *policy, bool firstCall, ParentResult *result, RequestData *rdata);
+  void markParentDown(const ParentSelectionPolicy *policy, ParentResult *result);
+  uint32_t numParents(ParentResult *result);
+  void markParentUp(ParentResult *result);
+};
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/ParentRoundRobin.cc
----------------------------------------------------------------------
diff --git a/proxy/ParentRoundRobin.cc b/proxy/ParentRoundRobin.cc
new file mode 100644
index 0000000..44c050c
--- /dev/null
+++ b/proxy/ParentRoundRobin.cc
@@ -0,0 +1,274 @@
+/** @file
+
+  Implementation of Parent Proxy routing
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+#include "ParentRoundRobin.h"
+
+ParentRoundRobin::ParentRoundRobin(ParentRecord *parent_record, ParentRR_t _round_robin_type)
+{
+  round_robin_type = _round_robin_type;
+
+  if (is_debug_tag_set("parent_select")) {
+    switch (round_robin_type) {
+    case P_NO_ROUND_ROBIN:
+      Debug("parent_select", "Using a round robin parent selection strategy of type P_NO_ROUND_ROBIN.");
+      break;
+    case P_STRICT_ROUND_ROBIN:
+      Debug("parent_select", "Using a round robin parent selection strategy of type P_STRICT_ROUND_ROBIN.");
+      break;
+    case P_HASH_ROUND_ROBIN:
+      Debug("parent_select", "Using a round robin parent selection strategy of type P_HASH_ROUND_ROBIN.");
+      break;
+    default:
+      // should never see this, there is a problem if you do.
+      Debug("parent_select", "Using a round robin parent selection strategy of type UNKNOWN TYPE.");
+      break;
+    }
+  }
+}
+
+ParentRoundRobin::~ParentRoundRobin() { }
+
+void
+ParentRoundRobin::selectParent(const ParentSelectionPolicy *policy, bool first_call, ParentResult *result, RequestData *rdata)
+{
+  Debug("parent_select", "In ParentRoundRobin::selectParent(): Using a round robin parent selection strategy.");
+  int cur_index = 0;
+  bool parentUp = false;
+  bool parentRetry = false;
+  bool bypass_ok = (result->rec->go_direct == true && policy->DNS_ParentOnly == 0);
+
+  HttpRequestData *request_info = static_cast<HttpRequestData *>(rdata);
+
+  ink_assert(numParents(result) > 0 || result->rec->go_direct == true);
+
+  if (first_call) {
+    if (result->rec->parents == NULL) {
+      // We should only get into this state if
+      //   if we are supposed to go direct
+      ink_assert(result->rec->go_direct == true);
+      // Could not find a parent
+      if (result->rec->go_direct == true) {
+        result->r = PARENT_DIRECT;
+      } else {
+        result->r = PARENT_FAIL;
+      }
+
+      result->hostname = NULL;
+      result->port = 0;
+      return;
+    } else {
+      switch (round_robin_type) {
+      case P_HASH_ROUND_ROBIN:
+        // INKqa12817 - make sure to convert to host byte order
+        // Why was it important to do host order here?  And does this have any
+        // impact with the transition to IPv6?  The IPv4 functionality is
+        // preserved for now anyway as ats_ip_hash returns the 32-bit address in
+        // that case.
+        if (rdata->get_client_ip() != NULL) {
+          cur_index = result->start_parent = ntohl(ats_ip_hash(rdata->get_client_ip())) % result->rec->num_parents;
+        } else {
+          cur_index = 0;
+        }
+        break;
+      case P_STRICT_ROUND_ROBIN:
+        cur_index = ink_atomic_increment((int32_t *)&result->rec->rr_next, 1);
+        cur_index = result->start_parent = cur_index % result->rec->num_parents;
+        break;
+      case P_NO_ROUND_ROBIN:
+        cur_index = result->start_parent = 0;
+        break;
+      default:
+        ink_release_assert(0);
+      }
+    }
+  } else {
+    // Move to next parent due to failure
+    cur_index = (result->last_parent + 1) % result->rec->num_parents;
+
+    // Check to see if we have wrapped around
+    if ((unsigned int)cur_index == result->start_parent) {
+      // We've wrapped around so bypass if we can
+      if (bypass_ok == true) {
+        // Could not find a parent
+        if (result->rec->go_direct == true) {
+          result->r = PARENT_DIRECT;
+        } else {
+          result->r = PARENT_FAIL;
+        }
+        result->hostname = NULL;
+        result->port = 0;
+        return;
+      } else {
+        // Bypass disabled so keep trying, ignoring whether we think
+        //   a parent is down or not
+        result->wrap_around = true;
+      }
+    }
+  }
+  // Loop through the array of parent seeing if any are up or
+  //   should be retried
+  do {
+    Debug("parent_select", "cur_index: %d, result->start_parent: %d", cur_index, result->start_parent);
+    // DNS ParentOnly inhibits bypassing the parent so always return that t
+    if ((result->rec->parents[cur_index].failedAt == 0) ||
+        (result->rec->parents[cur_index].failCount < policy->FailThreshold)) {
+      Debug("parent_select", "FailThreshold = %d", policy->FailThreshold);
+      Debug("parent_select", "Selecting a parent due to little failCount (faileAt: %u failCount: %d)",
+            (unsigned)result->rec->parents[cur_index].failedAt, result->rec->parents[cur_index].failCount);
+      parentUp = true;
+    } else {
+      if ((result->wrap_around) ||
+          ((result->rec->parents[cur_index].failedAt + policy->ParentRetryTime) < request_info->xact_start)) {
+        Debug("parent_select", "Parent[%d].failedAt = %u, retry = %u,xact_start = %" PRId64 " but wrap = %d", cur_index,
+              (unsigned)result->rec->parents[cur_index].failedAt, policy->ParentRetryTime, (int64_t)request_info->xact_start,
+              result->wrap_around);
+        // Reuse the parent
+        parentUp = true;
+        parentRetry = true;
+        Debug("parent_select", "Parent marked for retry %s:%d", result->rec->parents[cur_index].hostname,
+              result->rec->parents[cur_index].port);
+      } else {
+        parentUp = false;
+      }
+    }
+
+    if (parentUp == true) {
+      result->r = PARENT_SPECIFIED;
+      result->hostname = result->rec->parents[cur_index].hostname;
+      result->port = result->rec->parents[cur_index].port;
+      result->last_parent = cur_index;
+      result->retry = parentRetry;
+      ink_assert(result->hostname != NULL);
+      ink_assert(result->port != 0);
+      Debug("parent_select", "Chosen parent = %s.%d", result->hostname, result->port);
+      return;
+    }
+    cur_index = (cur_index + 1) % result->rec->num_parents;
+  } while ((unsigned int)cur_index != result->start_parent);
+
+  if (result->rec->go_direct == true) {
+    result->r = PARENT_DIRECT;
+  } else {
+    result->r = PARENT_FAIL;
+  }
+
+  result->hostname = NULL;
+  result->port = 0;
+}
+
+uint32_t
+ParentRoundRobin::numParents(ParentResult *result)
+{
+  return result->rec->num_parents;
+}
+
+void
+ParentRoundRobin::markParentDown(const ParentSelectionPolicy *policy, ParentResult *result)
+{
+  time_t now;
+  pRecord *pRec;
+  int new_fail_count = 0;
+
+  Debug("parent_select", "Starting ParentRoundRobin::markParentDown()");
+  //  Make sure that we are being called back with with a
+  //   result structure with a parent
+  ink_assert(result->r == PARENT_SPECIFIED);
+  if (result->r != PARENT_SPECIFIED) {
+    return;
+  }
+  // If we were set through the API we currently have not failover
+  //   so just return fail
+  if (result->rec == extApiRecord) {
+    return;
+  }
+
+  ink_assert((int)(result->last_parent) < result->rec->num_parents);
+  pRec = result->rec->parents + result->last_parent;
+
+  // If the parent has already been marked down, just increment
+  //   the failure count.  If this is the first mark down on a
+  //   parent we need to both set the failure time and set
+  //   count to one.  It's possible for the count and time get out
+  //   sync due there being no locks.  Therefore the code should
+  //   handle this condition.  If this was the result of a retry, we
+  //   must update move the failedAt timestamp to now so that we continue
+  //   negative cache the parent
+  if (pRec->failedAt == 0 || result->retry == true) {
+    // Reread the current time.  We want this to be accurate since
+    //   it relates to how long the parent has been down.
+    now = time(NULL);
+
+    // Mark the parent as down
+    ink_atomic_swap(&pRec->failedAt, now);
+
+    // If this is clean mark down and not a failed retry, we
+    //   must set the count to reflect this
+    if (result->retry == false) {
+      new_fail_count = pRec->failCount = 1;
+    }
+
+    Note("Parent %s marked as down %s:%d", (result->retry) ? "retry" : "initially", pRec->hostname, pRec->port);
+
+  } else {
+    int old_count = ink_atomic_increment(&pRec->failCount, 1);
+
+    Debug("parent_select", "Parent fail count increased to %d for %s:%d", old_count + 1, pRec->hostname, pRec->port);
+    new_fail_count = old_count + 1;
+  }
+
+  if (new_fail_count > 0 && new_fail_count == policy->FailThreshold) {
+    Note("Failure threshold met, http parent proxy %s:%d marked down", pRec->hostname, pRec->port);
+    pRec->available = false;
+    Debug("parent_select", "Parent marked unavailable, pRec->available=%d", pRec->available);
+  }
+}
+
+void
+ParentRoundRobin::markParentUp(ParentResult *result)
+{
+  pRecord *pRec;
+
+  //  Make sure that we are being called back with with a
+  //   result structure with a parent that is being retried
+  ink_release_assert(result->retry == true);
+  ink_assert(result->r == PARENT_SPECIFIED);
+  if (result->r != PARENT_SPECIFIED) {
+    return;
+  }
+  // If we were set through the API we currently have not failover
+  //   so just return fail
+  if (result->rec == extApiRecord) {
+    ink_assert(0);
+    return;
+  }
+
+  ink_assert((int)(result->last_parent) < result->rec->num_parents);
+  pRec = result->rec->parents + result->last_parent;
+  pRec->available = true;
+
+  ink_atomic_swap(&pRec->failedAt, (time_t)0);
+  int old_count = ink_atomic_swap(&pRec->failCount, 0);
+
+  if (old_count > 0) {
+    Note("http parent proxy %s:%d restored", pRec->hostname, pRec->port);
+  }
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/ParentRoundRobin.h
----------------------------------------------------------------------
diff --git a/proxy/ParentRoundRobin.h b/proxy/ParentRoundRobin.h
new file mode 100644
index 0000000..e8c2755
--- /dev/null
+++ b/proxy/ParentRoundRobin.h
@@ -0,0 +1,48 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+/*****************************************************************************
+ *
+ *  ParentRoundRobin.h - Implementation of various round robin strategies.
+ *
+ *****************************************************************************/
+
+#ifndef _PARENT_ROUND_ROBIN_H
+#define _PARENT_ROUND_ROBIN_H
+
+#include "ParentSelection.h"
+
+class ParentRoundRobin : public ParentSelectionStrategy
+{
+  ParentRR_t round_robin_type;
+
+public:
+  ParentRoundRobin(ParentRecord *_parent_record, ParentRR_t _round_robin_type);
+  ~ParentRoundRobin();
+  void selectParent(const ParentSelectionPolicy *policy, bool firstCall, ParentResult *result, RequestData *rdata);
+  void markParentDown(const ParentSelectionPolicy *policy, ParentResult *result);
+  uint32_t numParents(ParentResult *result);
+  void markParentUp(ParentResult *result);
+};
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/ParentSelection.cc
----------------------------------------------------------------------
diff --git a/proxy/ParentSelection.cc b/proxy/ParentSelection.cc
index 3367351..aa158e3 100644
--- a/proxy/ParentSelection.cc
+++ b/proxy/ParentSelection.cc
@@ -20,11 +20,10 @@
   See the License for the specific language governing permissions and
   limitations under the License.
  */
-#include "ts/ink_platform.h"
-#include "ts/HashSip.h"
-#include "ts/Tokenizer.h"
 #include "P_EventSystem.h"
 #include "ParentSelection.h"
+#include "ParentConsistentHash.h"
+#include "ParentRoundRobin.h"
 #include "ControlMatcher.h"
 #include "Main.h"
 #include "Error.h"
@@ -50,9 +49,7 @@ static const char *enable_var = "proxy.config.http.parent_proxy_routing_enable";
 static const char *threshold_var = "proxy.config.http.parent_proxy.fail_threshold";
 static const char *dns_parent_only_var = "proxy.config.http.no_dns_just_forward_to_parent";
 
-static const char *ParentResultStr[] = {"Parent_Undefined", "Parent_Direct", "Parent_Specified", "Parent_Failed"};
-
-static const char *ParentRRStr[] = {"false", "strict", "true", "consistent"};
+static const char *ParentResultStr[] = {"PARENT_UNDEFINED", "PARENT_DIRECT", "PARENT_SPECIFIED", "PARENT_AGENT", "PARENT_FAIL"};
 
 //
 //  Config Callback Prototypes
@@ -66,122 +63,41 @@ enum ParentCB_t {
   PARENT_DNS_ONLY_CB,
 };
 
-// If the parent was set by the external customer api,
-//   our HttpRequestData structure told us what parent to
-//   use and we are only called to preserve clean interface
-//   between HttpTransact & the parent selection code.  The following
-ParentRecord *const extApiRecord = (ParentRecord *)0xeeeeffff;
-
-ParentConfigParams::ParentConfigParams()
-  : ParentTable(NULL), DefaultParent(NULL), ParentRetryTime(30), ParentEnable(0), FailThreshold(10), DNS_ParentOnly(0)
-{
-}
-
-ParentConfigParams::~ParentConfigParams()
-{
-  if (ParentTable) {
-    delete ParentTable;
-  }
-
-  if (DefaultParent) {
-    delete DefaultParent;
-  }
-}
-
-int ParentConfig::m_id = 0;
-
-//
-//   Begin API functions
-//
-void
-ParentConfig::startup()
-{
-  parentConfigUpdate = new ConfigUpdateHandler<ParentConfig>();
-
-  // Load the initial configuration
-  reconfigure();
-
-  // Setup the callbacks for reconfiuration
-  //   parent table
-  parentConfigUpdate->attach(file_var);
-  //   default parent
-  parentConfigUpdate->attach(default_var);
-  //   Retry time
-  parentConfigUpdate->attach(retry_var);
-  //   Enable
-  parentConfigUpdate->attach(enable_var);
-
-  //   Fail Threshold
-  parentConfigUpdate->attach(threshold_var);
-
-  //   DNS Parent Only
-  parentConfigUpdate->attach(dns_parent_only_var);
-}
-
-void
-ParentConfig::reconfigure()
+ParentSelectionPolicy::ParentSelectionPolicy()
 {
-  char *default_val = NULL;
-  int retry_time = 30;
-  int enable = 0;
-  int fail_threshold;
-  int dns_parent_only;
-
-  ParentConfigParams *params;
-  params = new ParentConfigParams;
-
-  // Allocate parent table
-  params->ParentTable = new P_table(file_var, modulePrefix, &http_dest_tags);
-
-  // Handle default parent
-  PARENT_ReadConfigStringAlloc(default_val, default_var);
-  params->DefaultParent = createDefaultParent(default_val);
-  ats_free(default_val);
+  bool enable = false;
+  int32_t retry_time = 0;
+  int32_t fail_threshold = 0;
+  int32_t dns_parent_only = 0;
 
   // Handle parent timeout
   PARENT_ReadConfigInteger(retry_time, retry_var);
-  params->ParentRetryTime = retry_time;
+  ParentRetryTime = retry_time;
 
   // Handle parent enable
   PARENT_ReadConfigInteger(enable, enable_var);
-  params->ParentEnable = enable;
+  ParentEnable = enable;
 
   // Handle the fail threshold
   PARENT_ReadConfigInteger(fail_threshold, threshold_var);
-  params->FailThreshold = fail_threshold;
+  FailThreshold = fail_threshold;
 
   // Handle dns parent only
   PARENT_ReadConfigInteger(dns_parent_only, dns_parent_only_var);
-  params->DNS_ParentOnly = dns_parent_only;
-
-  m_id = configProcessor.set(m_id, params);
-
-  if (is_debug_tag_set("parent_config")) {
-    ParentConfig::print();
-  }
+  DNS_ParentOnly = dns_parent_only;
 }
 
-// void ParentConfig::print
-//
-//   Debugging function
-//
-void
-ParentConfig::print()
+ParentConfigParams::ParentConfigParams(P_table *_parent_table)
 {
-  ParentConfigParams *params = ParentConfig::acquire();
+  char *default_val = NULL;
+  parent_table = _parent_table;
 
-  printf("Parent Selection Config\n");
-  printf("\tEnabled %d\tRetryTime %d\tParent DNS Only %d\n", params->ParentEnable, params->ParentRetryTime, params->DNS_ParentOnly);
-  if (params->DefaultParent == NULL) {
-    printf("\tNo Default Parent\n");
-  } else {
-    printf("\tDefault Parent:\n");
-    params->DefaultParent->Print();
-  }
-  printf("  ");
-  params->ParentTable->Print();
+  policy = new ParentSelectionPolicy();
 
-  ParentConfig::release(params);
+  // Handle default parent
+  PARENT_ReadConfigStringAlloc(default_val, default_var);
+  DefaultParent = createDefaultParent(default_val);
+  ats_free(default_val);
 }
 
 bool
@@ -190,31 +106,19 @@ ParentConfigParams::apiParentExists(HttpRequestData *rdata)
   return (rdata->api_info && rdata->api_info->parent_proxy_name != NULL && rdata->api_info->parent_proxy_port > 0);
 }
 
-bool
-ParentConfigParams::parentExists(HttpRequestData *rdata)
-{
-  ParentResult junk;
-
-  findParent(rdata, &junk);
-
-  if (junk.r == PARENT_SPECIFIED) {
-    return true;
-  } else {
-    return false;
-  }
-}
-
 void
 ParentConfigParams::findParent(HttpRequestData *rdata, ParentResult *result)
 {
-  P_table *tablePtr = ParentTable;
+  P_table *tablePtr = parent_table;
   ParentRecord *defaultPtr = DefaultParent;
   ParentRecord *rec;
 
+  Debug("parent_select", "In ParentConfigParams::findParent(): parent_table: %p.", parent_table);
   ink_assert(result->r == PARENT_UNDEFINED);
 
   // Check to see if we are enabled
-  if (ParentEnable == 0) {
+  Debug("parent_select", "policy->ParentEnable: %d", policy->ParentEnable);
+  if (policy->ParentEnable == 0) {
     result->r = PARENT_DIRECT;
     return;
   }
@@ -223,9 +127,6 @@ ParentConfigParams::findParent(HttpRequestData *rdata, ParentResult *result)
   result->epoch = tablePtr;
   result->line_number = 0xffffffff;
   result->wrap_around = false;
-  // if this variabel is not set, we have problems: the code in
-  // FindParent relies on the value of start_parent and when it is not
-  // initialized, the code in FindParent can get into an infinite loop!
   result->start_parent = 0;
   result->last_parent = 0;
 
@@ -254,155 +155,47 @@ ParentConfigParams::findParent(HttpRequestData *rdata, ParentResult *result)
       rec = result->rec = defaultPtr;
     } else {
       result->r = PARENT_DIRECT;
-      Debug("cdn", "Returning PARENT_DIRECT (no parents were found)");
+      Debug("parent_select", "Returning PARENT_DIRECT (no parents were found)");
       return;
     }
   }
-  // Loop through the set of parents to see if any are
-  //   available
-  Debug("cdn", "Calling FindParent from findParent");
-
-  // Bug INKqa08251:
-  // If a parent proxy is set by the API,
-  // no need to call FindParent()
-  if (rec != extApiRecord)
-    rec->FindParent(true, result, rdata, this);
-
-  if (is_debug_tag_set("parent_select") || is_debug_tag_set("cdn")) {
-    switch (result->r) {
-    case PARENT_UNDEFINED:
-      Debug("cdn", "PARENT_UNDEFINED");
-      break;
-    case PARENT_FAIL:
-      Debug("cdn", "PARENT_FAIL");
-      break;
-    case PARENT_DIRECT:
-      Debug("cdn", "PARENT_DIRECT");
-      break;
-    case PARENT_SPECIFIED:
-      Debug("cdn", "PARENT_SPECIFIED");
-      break;
-    default:
-      // Handled here:
-      // PARENT_AGENT
-      break;
-    }
-
-    const char *host = rdata->get_host();
-
-    switch (result->r) {
-    case PARENT_UNDEFINED:
-    case PARENT_FAIL:
-    case PARENT_DIRECT:
-      Debug("parent_select", "Result for %s was %s", host, ParentResultStr[result->r]);
-      break;
-    case PARENT_SPECIFIED:
-      Debug("parent_select", "Result for %s was parent %s:%d", host, result->hostname, result->port);
-      break;
-    default:
-      // Handled here:
-      // PARENT_AGENT
-      break;
-    }
-  }
-}
-
-
-void
-ParentConfigParams::recordRetrySuccess(ParentResult *result)
-{
-  pRecord *pRec;
-
-  //  Make sure that we are being called back with with a
-  //   result structure with a parent that is being retried
-  ink_release_assert(result->retry == true);
-  ink_assert(result->r == PARENT_SPECIFIED);
-  if (result->r != PARENT_SPECIFIED) {
-    return;
-  }
-  // If we were set through the API we currently have not failover
-  //   so just return fail
-  if (result->rec == extApiRecord) {
-    ink_assert(0);
-    return;
-  }
-
-  ink_assert((int)(result->last_parent) < result->rec->num_parents);
-  pRec = result->rec->parents + result->last_parent;
-
-  pRec->available = true;
 
-  ink_atomic_swap(&pRec->failedAt, (time_t)0);
-  int old_count = ink_atomic_swap(&pRec->failCount, 0);
-
-  if (old_count > 0) {
-    Note("http parent proxy %s:%d restored", pRec->hostname, pRec->port);
+  if (rec != extApiRecord) {
+    selectParent(true, result, rdata);
   }
-}
 
-void
-ParentConfigParams::markParentDown(ParentResult *result)
-{
-  time_t now;
-  pRecord *pRec;
-  int new_fail_count = 0;
+  const char *host = rdata->get_host();
 
-  //  Make sure that we are being called back with with a
-  //   result structure with a parent
-  ink_assert(result->r == PARENT_SPECIFIED);
-  if (result->r != PARENT_SPECIFIED) {
-    return;
-  }
-  // If we were set through the API we currently have not failover
-  //   so just return fail
-  if (result->rec == extApiRecord) {
-    return;
-  }
-
-  ink_assert((int)(result->last_parent) < result->rec->num_parents);
-  pRec = result->rec->parents + result->last_parent;
-
-  // If the parent has already been marked down, just increment
-  //   the failure count.  If this is the first mark down on a
-  //   parent we need to both set the failure time and set
-  //   count to one.  It's possible for the count and time get out
-  //   sync due there being no locks.  Therefore the code should
-  //   handle this condition.  If this was the result of a retry, we
-  //   must update move the failedAt timestamp to now so that we continue
-  //   negative cache the parent
-  if (pRec->failedAt == 0 || result->retry == true) {
-    // Reread the current time.  We want this to be accurate since
-    //   it relates to how long the parent has been down.
-    now = time(NULL);
-
-    // Mark the parent as down
-    ink_atomic_swap(&pRec->failedAt, now);
-
-    // If this is clean mark down and not a failed retry, we
-    //   must set the count to reflect this
-    if (result->retry == false) {
-      new_fail_count = pRec->failCount = 1;
-    }
-
-    Debug("parent_select", "Parent %s marked as down %s:%d", (result->retry) ? "retry" : "initially", pRec->hostname, pRec->port);
-
-  } else {
-    int old_count = ink_atomic_increment(&pRec->failCount, 1);
-
-    Debug("parent_select", "Parent fail count increased to %d for %s:%d", old_count + 1, pRec->hostname, pRec->port);
-    new_fail_count = old_count + 1;
-  }
-
-  if (new_fail_count > 0 && new_fail_count == FailThreshold) {
-    Note("http parent proxy %s:%d marked down", pRec->hostname, pRec->port);
-    pRec->available = false;
+  switch (result->r) {
+  case PARENT_UNDEFINED:
+    Debug("parent_select", "PARENT_UNDEFINED");
+    Debug("parent_select", "Result for %s was %s", host, ParentResultStr[result->r]);
+    break;
+  case PARENT_FAIL:
+    Debug("parent_select", "PARENT_FAIL");
+    break;
+  case PARENT_DIRECT:
+    Debug("parent_select", "PARENT_DIRECT");
+    Debug("parent_select", "Result for %s was %s", host, ParentResultStr[result->r]);
+    break;
+  case PARENT_SPECIFIED:
+    Debug("parent_select", "PARENT_SPECIFIED");
+    Debug("parent_select", "Result for %s was parent %s:%d", host, result->hostname, result->port);
+    break;
+  default:
+    // Handled here:
+    // PARENT_AGENT
+    break;
   }
 }
 
 void
 ParentConfigParams::nextParent(HttpRequestData *rdata, ParentResult *result)
 {
-  P_table *tablePtr = ParentTable;
+  P_table *tablePtr = parent_table;
+
+  Debug("parent_select", "ParentConfigParams::nextParent(): parent_table: %p, result->rec: %p, result->epoch: %p", parent_table,
+        result->rec, result->epoch);
 
   //  Make sure that we are being called back with a
   //   result structure with a parent
@@ -418,292 +211,150 @@ ParentConfigParams::nextParent(HttpRequestData *rdata, ParentResult *result)
     result->r = PARENT_FAIL;
     return;
   }
-  // The epoch pointer is a legacy from the time when the tables
-  //  would be swapped and deleted in the future.  I'm using the
-  //  pointer now to ensure that the ParentConfigParams structure
-  //  is properly used.  The table should never change out from
-  //  under the a http transaction
+  Debug("parent_select", "ParentConfigParams::nextParent(): result->r: %d, tablePtr: %p, result->epoch: %p", result->r, tablePtr,
+        result->epoch);
   ink_release_assert(tablePtr == result->epoch);
 
   // Find the next parent in the array
-  Debug("cdn", "Calling FindParent from nextParent");
-  result->rec->FindParent(false, result, rdata, this);
+  Debug("parent_select", "Calling selectParent() from nextParent");
+  selectParent(false, result, rdata);
+
+  const char *host = rdata->get_host();
 
   switch (result->r) {
   case PARENT_UNDEFINED:
-    Debug("cdn", "PARENT_UNDEFINED");
+    Debug("parent_select", "PARENT_UNDEFINED");
+    Debug("parent_select", "Retry result for %s was %s", host, ParentResultStr[result->r]);
     break;
   case PARENT_FAIL:
-    Debug("cdn", "PARENT_FAIL");
+    Debug("parent_select", "PARENT_FAIL");
+    Debug("parent_select", "Retry result for %s was %s", host, ParentResultStr[result->r]);
     break;
   case PARENT_DIRECT:
-    Debug("cdn", "PARENT_DIRECT");
+    Debug("parent_select", "PARENT_DIRECT");
+    Debug("parent_select", "Retry result for %s was %s", host, ParentResultStr[result->r]);
     break;
   case PARENT_SPECIFIED:
-    Debug("cdn", "PARENT_SPECIFIED");
+    Debug("parent_select", "Retry result for %s was parent %s:%d", host, result->hostname, result->port);
     break;
   default:
     // Handled here:
     // PARENT_AGENT
     break;
   }
-
-  if (is_debug_tag_set("parent_select")) {
-    const char *host = rdata->get_host();
-
-    switch (result->r) {
-    case PARENT_UNDEFINED:
-    case PARENT_FAIL:
-    case PARENT_DIRECT:
-      Debug("parent_select", "Retry result for %s was %s", host, ParentResultStr[result->r]);
-      break;
-    case PARENT_SPECIFIED:
-      Debug("parent_select", "Retry result for %s was parent %s:%d", host, result->hostname, result->port);
-      break;
-    default:
-      // Handled here:
-      // PARENT_AGENT
-      break;
-    }
-  }
 }
 
-//
-//   End API functions
-//
-
-uint64_t
-ParentRecord::getPathHash(HttpRequestData *hrdata, ATSHash64 *h)
+bool
+ParentConfigParams::parentExists(HttpRequestData *rdata)
 {
-  const char *tmp = NULL;
-  int len;
-  URL *url = hrdata->hdr->url_get();
+  ParentResult result;
 
-  // Always hash on '/' because paths returned by ATS are always stripped of it
-  h->update("/", 1);
+  findParent(rdata, &result);
 
-  tmp = url->path_get(&len);
-  if (tmp) {
-    h->update(tmp, len);
-  }
-
-  if (!ignore_query) {
-    tmp = url->query_get(&len);
-    if (tmp) {
-      h->update("?", 1);
-      h->update(tmp, len);
-    }
+  if (result.r == PARENT_SPECIFIED) {
+    return true;
+  } else {
+    return false;
   }
-
-  h->final();
-
-  return h->get();
 }
 
+int ParentConfig::m_id = 0;
+
 void
-ParentRecord::FindParent(bool first_call, ParentResult *result, RequestData *rdata, ParentConfigParams *config)
+ParentConfig::startup()
 {
-  Debug("cdn", "Entering FindParent (the inner loop)");
-  int cur_index = 0;
-  bool parentUp = false;
-  bool parentRetry = false;
-  bool bypass_ok = (go_direct == true && config->DNS_ParentOnly == 0);
-  uint64_t path_hash;
-
-  ATSHash64Sip24 hash;
-  pRecord *prtmp = NULL;
-
-  HttpRequestData *request_info = static_cast<HttpRequestData *>(rdata);
-
-  ink_assert(num_parents > 0 || go_direct == true);
-
-  if (first_call) {
-    if (parents == NULL) {
-      // We should only get into this state if
-      //   if we are supposed to go direct
-      ink_assert(go_direct == true);
-      goto NO_PARENTS;
-    } else {
-      switch (round_robin) {
-      case P_HASH_ROUND_ROBIN:
-        // INKqa12817 - make sure to convert to host byte order
-        // Why was it important to do host order here?  And does this have any
-        // impact with the transition to IPv6?  The IPv4 functionality is
-        // preserved for now anyway as ats_ip_hash returns the 32-bit address in
-        // that case.
-        if (rdata->get_client_ip() != NULL) {
-          cur_index = ntohl(ats_ip_hash(rdata->get_client_ip())) % num_parents;
-        } else {
-          cur_index = 0;
-        }
-        break;
-      case P_CONSISTENT_HASH:
-        path_hash = getPathHash(request_info, (ATSHash64 *)&hash);
-        if (path_hash) {
-          prtmp = (pRecord *)chash->lookup_by_hashval(path_hash, &result->chashIter, &result->wrap_around);
-          if (prtmp) {
-            cur_index = prtmp->idx;
-            result->foundParents[cur_index] = true;
-            result->start_parent++;
-            break;
-          } else {
-            Error("Consistent Hash loopup returned NULL");
-          }
-        } else {
-          Error("Could not find path");
-        }
-      // Fall through to round robin
-      case P_STRICT_ROUND_ROBIN:
-        cur_index = ink_atomic_increment((int32_t *)&rr_next, 1);
-        cur_index = result->start_parent = cur_index % num_parents;
-        break;
-      case P_NO_ROUND_ROBIN:
-        cur_index = result->start_parent = 0;
-        break;
-      default:
-        ink_release_assert(0);
-      }
-    }
-  } else {
-    if (round_robin == P_CONSISTENT_HASH) {
-      Debug("parent_select", "result->start_parent=%d, num_parents=%d", result->start_parent, num_parents);
-      if (result->start_parent == (unsigned int)num_parents) {
-        result->wrap_around = true;
-        result->start_parent = 0;
-        memset(result->foundParents, 0, sizeof(result->foundParents));
-      }
+  parentConfigUpdate = new ConfigUpdateHandler<ParentConfig>();
 
-      do {
-        prtmp = (pRecord *)chash->lookup(NULL, &result->chashIter, &result->wrap_around, &hash);
-      } while (prtmp && result->foundParents[prtmp->idx]);
+  // Load the initial configuration
+  reconfigure();
 
-      if (prtmp) {
-        cur_index = prtmp->idx;
-        result->foundParents[cur_index] = true;
-        result->start_parent++;
-      } else {
-        Error("Consistent Hash loopup returned NULL");
-        cur_index = ink_atomic_increment((int32_t *)&rr_next, 1);
-        cur_index = cur_index % num_parents;
-      }
-    } else {
-      // Move to next parent due to failure
-      cur_index = (result->last_parent + 1) % num_parents;
-
-      // Check to see if we have wrapped around
-      if ((unsigned int)cur_index == result->start_parent) {
-        // We've wrapped around so bypass if we can
-        if (bypass_ok == true) {
-          goto NO_PARENTS;
-        } else {
-        // Bypass disabled so keep trying, ignoring whether we think
-        //   a parent is down or not
-        FORCE_WRAP_AROUND:
-          result->wrap_around = true;
-        }
-      }
-    }
-  }
+  // Setup the callbacks for reconfiuration
+  //   parent table
+  parentConfigUpdate->attach(file_var);
+  //   default parent
+  parentConfigUpdate->attach(default_var);
+  //   Retry time
+  parentConfigUpdate->attach(retry_var);
+  //   Enable
+  parentConfigUpdate->attach(enable_var);
 
-  // Loop through the array of parent seeing if any are up or
-  //   should be retried
-  do {
-    // DNS ParentOnly inhibits bypassing the parent so always return that t
-    if ((parents[cur_index].failedAt == 0) || (parents[cur_index].failCount < config->FailThreshold)) {
-      Debug("parent_select", "config->FailThreshold = %d", config->FailThreshold);
-      Debug("parent_select", "Selecting a down parent due to little failCount"
-                             "(faileAt: %u failCount: %d)",
-            (unsigned)parents[cur_index].failedAt, parents[cur_index].failCount);
-      parentUp = true;
-    } else {
-      if ((result->wrap_around) || ((parents[cur_index].failedAt + config->ParentRetryTime) < request_info->xact_start)) {
-        Debug("parent_select", "Parent[%d].failedAt = %u, retry = %u,xact_start = %" PRId64 " but wrap = %d", cur_index,
-              (unsigned)parents[cur_index].failedAt, config->ParentRetryTime, (int64_t)request_info->xact_start,
-              result->wrap_around);
-        // Reuse the parent
-        parentUp = true;
-        parentRetry = true;
-        Debug("parent_select", "Parent marked for retry %s:%d", parents[cur_index].hostname, parents[cur_index].port);
-      } else {
-        parentUp = false;
-      }
-    }
+  //   Fail Threshold
+  parentConfigUpdate->attach(threshold_var);
 
-    if (parentUp == true) {
-      result->r = PARENT_SPECIFIED;
-      result->hostname = parents[cur_index].hostname;
-      result->port = parents[cur_index].port;
-      result->last_parent = cur_index;
-      result->retry = parentRetry;
-      ink_assert(result->hostname != NULL);
-      ink_assert(result->port != 0);
-      Debug("parent_select", "Chosen parent = %s.%d", result->hostname, result->port);
-      return;
-    }
+  //   DNS Parent Only
+  parentConfigUpdate->attach(dns_parent_only_var);
+}
 
-    if (round_robin == P_CONSISTENT_HASH) {
-      if (result->start_parent == (unsigned int)num_parents) {
-        result->wrap_around = false;
-        result->start_parent = 0;
-        memset(result->foundParents, 0, sizeof(result->foundParents));
-      }
+void
+ParentConfig::reconfigure()
+{
+  ParentConfigParams *params = NULL;
 
-      do {
-        prtmp = (pRecord *)chash->lookup(NULL, &(result->chashIter), &result->wrap_around, &hash);
-      } while (prtmp && result->foundParents[prtmp->idx]);
+  // Allocate parent table
+  P_table *pTable = new P_table(file_var, modulePrefix, &http_dest_tags);
 
-      if (prtmp) {
-        cur_index = prtmp->idx;
-        result->foundParents[cur_index] = true;
-        result->start_parent++;
-      }
-    } else {
-      cur_index = (cur_index + 1) % num_parents;
-    }
+  params = new ParentConfigParams(pTable);
+  ink_assert(params != NULL);
 
-  } while ((round_robin == P_CONSISTENT_HASH ? result->wrap_around : ((unsigned int)cur_index != result->start_parent)));
+  m_id = configProcessor.set(m_id, params);
 
-  // We can't bypass so retry, taking any parent that we can
-  if (bypass_ok == false) {
-    goto FORCE_WRAP_AROUND;
+  if (is_debug_tag_set("parent_config")) {
+    ParentConfig::print();
   }
+}
 
-NO_PARENTS:
+// void ParentConfig::print
+//
+//   Debugging function
+//
+void
+ParentConfig::print()
+{
+  ParentConfigParams *params = ParentConfig::acquire();
 
-  // Could not find a parent
-  if (this->go_direct == true) {
-    result->r = PARENT_DIRECT;
+  printf("Parent Selection Config\n");
+  printf("\tEnabled %d\tRetryTime %d\tParent DNS Only %d\n", params->policy->ParentEnable, params->policy->ParentRetryTime,
+         params->policy->DNS_ParentOnly);
+  if (params->DefaultParent == NULL) {
+    printf("\tNo Default Parent\n");
   } else {
-    result->r = PARENT_FAIL;
+    printf("\tDefault Parent:\n");
+    params->DefaultParent->Print();
   }
+  printf("  ");
+  params->parent_table->Print();
 
-  result->hostname = NULL;
-  result->port = 0;
+  ParentConfig::release(params);
 }
 
-// const char* ParentRecord::ProcessParents(char* val)
+// const char* ParentRecord::ProcessParents(char* val, bool isPrimary)
 //
 //   Reads in the value of a "round-robin" or "order"
 //     directive and parses out the individual parents
-//     allocates and builds the this->parents array
+//     allocates and builds the this->parents array or
+//     this->secondary_parents based upon the isPrimary
+//     boolean.
 //
 //   Returns NULL on success and a static error string
 //     on failure
 //
 const char *
-ParentRecord::ProcessParents(char *val)
+ParentRecord::ProcessParents(char *val, bool isPrimary)
 {
   Tokenizer pTok(",; \t\r");
-  int numTok;
-  const char *current;
-  int port;
-  char *tmp, *tmp2;
-  const char *errPtr;
+  int numTok = 0;
+  const char *current = NULL;
+  int port = 0;
+  char *tmp = NULL, *tmp2 = NULL;
+  const char *errPtr = NULL;
   float weight = 1.0;
 
-  if (parents != NULL) {
+  if (parents != NULL && isPrimary == true) {
     return "Can not specify more than one set of parents";
   }
+  if (secondary_parents != NULL && isPrimary == false) {
+    return "Can not specify more than one set of secondary parents";
+  }
 
   numTok = pTok.Initialize(val, SHARE_TOKS);
 
@@ -711,7 +362,11 @@ ParentRecord::ProcessParents(char *val)
     return "No parents specified";
   }
   // Allocate the parents array
-  this->parents = (pRecord *)ats_malloc(sizeof(pRecord) * numTok);
+  if (isPrimary) {
+    this->parents = (pRecord *)ats_malloc(sizeof(pRecord) * numTok);
+  } else {
+    this->secondary_parents = (pRecord *)ats_malloc(sizeof(pRecord) * numTok);
+  }
 
   // Loop through the set of parents specified
   //
@@ -768,18 +423,35 @@ ParentRecord::ProcessParents(char *val)
       goto MERROR;
     }
     // Update the pRecords
-    memcpy(this->parents[i].hostname, current, tmp - current);
-    this->parents[i].hostname[tmp - current] = '\0';
-    this->parents[i].port = port;
-    this->parents[i].failedAt = 0;
-    this->parents[i].scheme = scheme;
-    this->parents[i].idx = i;
-    this->parents[i].name = this->parents[i].hostname;
-    this->parents[i].available = true;
-    this->parents[i].weight = weight;
+    if (isPrimary) {
+      memcpy(this->parents[i].hostname, current, tmp - current);
+      this->parents[i].hostname[tmp - current] = '\0';
+      this->parents[i].port = port;
+      this->parents[i].failedAt = 0;
+      this->parents[i].scheme = scheme;
+      this->parents[i].idx = i;
+      this->parents[i].name = this->parents[i].hostname;
+      this->parents[i].available = true;
+      this->parents[i].weight = weight;
+    } else {
+      memcpy(this->secondary_parents[i].hostname, current, tmp - current);
+      this->secondary_parents[i].hostname[tmp - current] = '\0';
+      this->secondary_parents[i].port = port;
+      this->secondary_parents[i].failedAt = 0;
+      this->secondary_parents[i].scheme = scheme;
+      this->secondary_parents[i].idx = i;
+      this->secondary_parents[i].name = this->secondary_parents[i].hostname;
+      this->secondary_parents[i].available = true;
+      this->secondary_parents[i].weight = weight;
+    }
+  }
+
+  if (isPrimary) {
+    num_parents = numTok;
+  } else {
+    num_secondary_parents = numTok;
   }
 
-  num_parents = numTok;
   return NULL;
 
 MERROR:
@@ -807,10 +479,9 @@ ParentRecord::DefaultInit(char *val)
   bool alarmAlready = false;
 
   this->go_direct = true;
-  this->round_robin = P_NO_ROUND_ROBIN;
   this->ignore_query = false;
   this->scheme = NULL;
-  errPtr = ProcessParents(val);
+  errPtr = ProcessParents(val, true);
 
   if (errPtr != NULL) {
     errBuf = (char *)ats_malloc(1024);
@@ -823,23 +494,6 @@ ParentRecord::DefaultInit(char *val)
   }
 }
 
-void
-ParentRecord::buildConsistentHash(void)
-{
-  ATSHash64Sip24 hash;
-  int i;
-
-  if (chash) {
-    return;
-  }
-
-  chash = new ATSConsistentHash();
-
-  for (i = 0; i < num_parents; i++) {
-    chash->insert(&(this->parents[i]), this->parents[i].weight, (ATSHash64 *)&hash);
-  }
-}
-
 // config_parse_error ParentRecord::Init(matcher_line* line_info)
 //
 //    matcher_line* line_info - contains parsed label/value
@@ -857,6 +511,8 @@ ParentRecord::Init(matcher_line *line_info)
   char *label;
   char *val;
   bool used = false;
+  ParentRR_t round_robin = P_NO_ROUND_ROBIN;
+
 
   this->line_num = line_info->line_num;
   this->scheme = NULL;
@@ -879,20 +535,17 @@ ParentRecord::Init(matcher_line *line_info)
         round_robin = P_NO_ROUND_ROBIN;
       } else if (strcasecmp(val, "consistent_hash") == 0) {
         round_robin = P_CONSISTENT_HASH;
-        if (this->parents != NULL) {
-          buildConsistentHash();
-        }
       } else {
         round_robin = P_NO_ROUND_ROBIN;
         errPtr = "invalid argument to round_robin directive";
       }
       used = true;
-    } else if (strcasecmp(label, "parent") == 0) {
-      errPtr = ProcessParents(val);
+    } else if (strcasecmp(label, "parent") == 0 || strcasecmp(label, "primary_parent") == 0) {
+      errPtr = ProcessParents(val, true);
+      used = true;
+    } else if (strcasecmp(label, "secondary_parent") == 0) {
+      errPtr = ProcessParents(val, false);
       used = true;
-      if (round_robin == P_CONSISTENT_HASH) {
-        buildConsistentHash();
-      }
     } else if (strcasecmp(label, "go_direct") == 0) {
       if (strcasecmp(val, "false") == 0) {
         go_direct = false;
@@ -909,6 +562,7 @@ ParentRecord::Init(matcher_line *line_info)
       } else {
         this->ignore_query = false;
       }
+      used = true;
     }
     // Report errors generated by ProcessParents();
     if (errPtr != NULL) {
@@ -943,6 +597,26 @@ ParentRecord::Init(matcher_line *line_info)
     }
   }
 
+  switch (round_robin) {
+  // ParentRecord.round_robin defaults to P_NO_ROUND_ROBIN when round_robin
+  // is not set in parent.config.  Therefore ParentRoundRobin is the default
+  // strategy.  If setting go_direct to true, there should be no parent list
+  // in parent.config and ParentRoundRobin::lookup will set parent_result->r
+  // to PARENT_DIRECT.
+  case P_NO_ROUND_ROBIN:
+  case P_STRICT_ROUND_ROBIN:
+  case P_HASH_ROUND_ROBIN:
+    TSDebug("parent_select", "allocating ParentRoundRobin() lookup strategy.");
+    selection_strategy = new ParentRoundRobin(this, round_robin);
+    break;
+  case P_CONSISTENT_HASH:
+    TSDebug("parent_select", "allocating ParentConsistentHash() lookup strategy.");
+    selection_strategy = new ParentConsistentHash(this);
+    break;
+  default:
+    ink_release_assert(0);
+  }
+
   return config_parse_error::ok();
 }
 
@@ -964,10 +638,8 @@ ParentRecord::UpdateMatch(ParentResult *result, RequestData *rdata)
 
 ParentRecord::~ParentRecord()
 {
-  if (chash) {
-    delete chash;
-  }
   ats_free(parents);
+  delete selection_strategy;
 }
 
 void
@@ -977,7 +649,7 @@ ParentRecord::Print()
   for (int i = 0; i < num_parents; i++) {
     printf(" %s:%d ", parents[i].hostname, parents[i].port);
   }
-  printf(" rr=%s direct=%s\n", ParentRRStr[round_robin], (go_direct == true) ? "true" : "false");
+  printf(" direct=%s\n", (go_direct == true) ? "true" : "false");
 }
 
 // ParentRecord* createDefaultParent(char* val)
@@ -1050,7 +722,6 @@ setup_socks_servers(ParentRecord *rec_arr, int len)
   return 0;
 }
 
-
 void
 SocksServerConfig::reconfigure()
 {
@@ -1058,11 +729,13 @@ SocksServerConfig::reconfigure()
   int retry_time = 30;
   int fail_threshold;
 
-  ParentConfigParams *params;
-  params = new ParentConfigParams;
+  ParentConfigParams *params = NULL;
 
   // Allocate parent table
-  params->ParentTable = new P_table("proxy.config.socks.socks_config_file", "[Socks Server Selection]", &socks_server_tags);
+  P_table *pTable = new P_table("proxy.config.socks.socks_config_file", "[Socks Server Selection]", &socks_server_tags);
+
+  params = new ParentConfigParams(pTable);
+  ink_assert(params != NULL);
 
   // Handle default parent
   PARENT_ReadConfigStringAlloc(default_val, "proxy.config.socks.default_servers");
@@ -1071,24 +744,24 @@ SocksServerConfig::reconfigure()
 
   if (params->DefaultParent)
     setup_socks_servers(params->DefaultParent, 1);
-  if (params->ParentTable->ipMatch)
-    setup_socks_servers(params->ParentTable->ipMatch->data_array, params->ParentTable->ipMatch->array_len);
+  if (params->parent_table->ipMatch)
+    setup_socks_servers(params->parent_table->ipMatch->data_array, params->parent_table->ipMatch->array_len);
 
   // Handle parent timeout
   PARENT_ReadConfigInteger(retry_time, "proxy.config.socks.server_retry_time");
-  params->ParentRetryTime = retry_time;
+  params->policy->ParentRetryTime = retry_time;
 
   // Handle parent enable
   // enable is always true for use. We will come here only if socks is enabled
-  params->ParentEnable = 1;
+  params->policy->ParentEnable = 1;
 
   // Handle the fail threshold
   PARENT_ReadConfigInteger(fail_threshold, "proxy.config.socks.server_fail_threshold");
-  params->FailThreshold = fail_threshold;
+  params->policy->FailThreshold = fail_threshold;
 
   // Handle dns parent only
   // PARENT_ReadConfigInteger(dns_parent_only, dns_parent_only_var);
-  params->DNS_ParentOnly = 0;
+  params->policy->DNS_ParentOnly = 0;
 
   m_id = configProcessor.set(m_id, params);
 
@@ -1103,7 +776,8 @@ SocksServerConfig::print()
   ParentConfigParams *params = SocksServerConfig::acquire();
 
   printf("Parent Selection Config for Socks Server\n");
-  printf("\tEnabled %d\tRetryTime %d\tParent DNS Only %d\n", params->ParentEnable, params->ParentRetryTime, params->DNS_ParentOnly);
+  printf("\tEnabled %d\tRetryTime %d\tParent DNS Only %d\n", params->policy->ParentEnable, params->policy->ParentRetryTime,
+         params->policy->DNS_ParentOnly);
   if (params->DefaultParent == NULL) {
     printf("\tNo Default Parent\n");
   } else {
@@ -1111,7 +785,7 @@ SocksServerConfig::print()
     params->DefaultParent->Print();
   }
   printf("  ");
-  params->ParentTable->Print();
+  params->parent_table->Print();
 
   SocksServerConfig::release(params);
 }
@@ -1151,18 +825,20 @@ EXCLUSIVE_REGRESSION_TEST(PARENTSELECTION)(RegressionTest * /* t ATS_UNUSED */,
   // first, set everything up
   *pstatus = REGRESSION_TEST_INPROGRESS;
   ParentConfig config;
-  ParentConfigParams *params = new ParentConfigParams();
-  params->FailThreshold = 1;
-  params->ParentRetryTime = 5;
+  P_table *ParentTable;
+  ParentConfigParams *params;
   passes = fails = 0;
   config.startup();
-  params->ParentEnable = true;
   char tbl[2048];
 #define T(x) ink_strlcat(tbl, x, sizeof(tbl));
-#define REBUILD                                                                                                                  \
-  params->ParentTable = new P_table("", "ParentSelection Unit Test Table", &http_dest_tags,                                      \
-                                    ALLOW_HOST_TABLE | ALLOW_REGEX_TABLE | ALLOW_URL_TABLE | ALLOW_IP_TABLE | DONT_BUILD_TABLE); \
-  params->ParentTable->BuildTableFromString(tbl);
+#define REBUILD                                                                                                          \
+  ParentTable = new P_table("", "ParentSelection Unit Test Table", &http_dest_tags,                                      \
+                            ALLOW_HOST_TABLE | ALLOW_REGEX_TABLE | ALLOW_URL_TABLE | ALLOW_IP_TABLE | DONT_BUILD_TABLE); \
+  ParentTable->BuildTableFromString(tbl);                                                                                \
+  params = new ParentConfigParams(ParentTable);                                                                          \
+  params->policy->FailThreshold = 1;                                                                                   \
+  params->policy->ParentEnable = true;                                                                                 \
+  params->policy->ParentRetryTime = 5;
   HttpRequestData *request = NULL;
   ParentResult *result = NULL;
 #define REINIT                            \
@@ -1348,7 +1024,7 @@ EXCLUSIVE_REGRESSION_TEST(PARENTSELECTION)(RegressionTest * /* t ATS_UNUSED */,
   }
 
   // sleep(5); // parents should come back up; they don't
-  sleep(params->ParentRetryTime + 1);
+  sleep(params->policy->ParentRetryTime + 1);
 
   // Fix: The following tests failed because
   // br() should set xact_start correctly instead of 0.
@@ -1370,6 +1046,48 @@ EXCLUSIVE_REGRESSION_TEST(PARENTSELECTION)(RegressionTest * /* t ATS_UNUSED */,
       ink_assert(0);
     }
   }
+
+  // Test 173
+  tbl[0] = '\0';
+  ST(173)
+  T("dest_domain=rabbit.net parent=fuzzy:80|1.0;fluffy:80|1.0 secondary_parent=furry:80|1.0;frisky:80|1.0 "
+    "round_robin=consistent_hash go_direct=false\n")
+  REBUILD
+  REINIT br(request, "i.am.rabbit.net");
+  FP sleep(1);
+  RE(verify(result, PARENT_SPECIFIED, "fuzzy", 80), 173)
+  params->markParentDown(result); // fuzzy is down.
+
+  // Test 174
+  ST(174)
+  REINIT br(request, "i.am.rabbit.net");
+  FP sleep(1);
+  RE(verify(result, PARENT_SPECIFIED, "frisky", 80), 174)
+
+  params->markParentDown(result); // frisky is down.
+
+  // Test 175
+  ST(175)
+  REINIT br(request, "i.am.rabbit.net");
+  FP sleep(1);
+  RE(verify(result, PARENT_SPECIFIED, "furry", 80), 175)
+
+  params->markParentDown(result); // frisky is down.
+
+  // Test 176
+  ST(176)
+  REINIT br(request, "i.am.rabbit.net");
+  FP sleep(1);
+  RE(verify(result, PARENT_SPECIFIED, "fluffy", 80), 176)
+
+  params->markParentDown(result); // all are down now.
+
+  // Test 177
+  ST(177)
+  REINIT br(request, "i.am.rabbit.net");
+  FP sleep(1);
+  RE(verify(result, PARENT_FAIL, NULL, 80), 177)
+
   delete request;
   delete result;
   delete params;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/ParentSelection.h
----------------------------------------------------------------------
diff --git a/proxy/ParentSelection.h b/proxy/ParentSelection.h
index 8dc428c..d04b53b 100644
--- a/proxy/ParentSelection.h
+++ b/proxy/ParentSelection.h
@@ -35,19 +35,18 @@
 #include "ProxyConfig.h"
 #include "ControlBase.h"
 #include "ControlMatcher.h"
-
 #include "P_RecProcess.h"
-
-#include "ts/ink_platform.h"
 #include "ts/ConsistentHash.h"
+#include "ts/Tokenizer.h"
+#include "ts/ink_apidefs.h"
 
 #define MAX_PARENTS 64
 
 struct RequestData;
-
 struct matcher_line;
 struct ParentResult;
 class ParentRecord;
+class ParentSelectionStrategy;
 
 enum ParentResultType {
   PARENT_UNDEFINED,
@@ -57,23 +56,88 @@ enum ParentResultType {
   PARENT_FAIL,
 };
 
+enum ParentRR_t {
+  P_NO_ROUND_ROBIN = 0,
+  P_STRICT_ROUND_ROBIN,
+  P_HASH_ROUND_ROBIN,
+  P_CONSISTENT_HASH,
+};
+
+// struct pRecord
+//
+//    A record for an invidual parent
+//
+struct pRecord : ATSConsistentHashNode {
+  char hostname[MAXDNAME + 1];
+  int port;
+  time_t failedAt;
+  int failCount;
+  int32_t upAt;
+  const char *scheme; // for which parent matches (if any)
+  int idx;
+  float weight;
+};
+
 typedef ControlMatcher<ParentRecord, ParentResult> P_table;
 
+// class ParentRecord : public ControlBase
 //
-// API to outside world
+//   A record for a configuration line in the parent.config
+//    file
 //
+class ParentRecord : public ControlBase
+{
+public:
+  ParentRecord()
+    : parents(NULL), secondary_parents(NULL), num_parents(0), num_secondary_parents(0), ignore_query(false), 
+      rr_next(0), go_direct(true), selection_strategy(NULL)
+  {
+  }
+
+  ~ParentRecord();
+
+  config_parse_error Init(matcher_line *line_info);
+  bool DefaultInit(char *val);
+  void UpdateMatch(ParentResult *result, RequestData *rdata);
+  void Print();
+  pRecord *parents;
+  pRecord *secondary_parents;
+  int num_parents;
+  int num_secondary_parents;
+
+  bool
+  bypass_ok() const
+  {
+    return go_direct;
+  }
+
+  const char *scheme;
+  // private:
+  const char *ProcessParents(char *val, bool isPrimary);
+  bool ignore_query;
+  volatile uint32_t rr_next;
+  bool go_direct;
+  ParentSelectionStrategy *selection_strategy;
+};
+
+// If the parent was set by the external customer api,
+//   our HttpRequestData structure told us what parent to
+//   use and we are only called to preserve clean interface
+//   between HttpTransact & the parent selection code.  The following
+ParentRecord *const extApiRecord = (ParentRecord *)0xeeeeffff;
+
 struct ParentResult {
   ParentResult()
-    : r(PARENT_UNDEFINED), hostname(NULL), port(0), line_number(0), epoch(NULL), rec(NULL), last_parent(0), start_parent(0),
-      wrap_around(false), retry(false)
+    : r(PARENT_UNDEFINED), hostname(NULL), port(0), retry(false), line_number(0), epoch(NULL), rec(NULL), last_parent(0),
+      start_parent(0), wrap_around(false), last_lookup(0)
   {
-    memset(foundParents, 0, sizeof(foundParents));
-  };
+  }
 
   // For outside consumption
   ParentResultType r;
   const char *hostname;
   int port;
+  bool retry;
 
   // Internal use only
   //   Not to be modified by HTTP
@@ -83,149 +147,117 @@ struct ParentResult {
   uint32_t last_parent;
   uint32_t start_parent;
   bool wrap_around;
-  bool retry;
-  // Arena *a;
-  ATSConsistentHashIter chashIter;
-  bool foundParents[MAX_PARENTS];
+  int last_lookup; // state for for consistent hash.
 };
 
-class HttpRequestData;
-
-struct ParentConfigParams : public ConfigInfo {
-  ParentConfigParams();
-  ~ParentConfigParams();
+struct ParentSelectionPolicy {
+  int32_t ParentRetryTime;
+  int32_t ParentEnable;
+  int32_t FailThreshold;
+  int32_t DNS_ParentOnly;
+  ParentSelectionPolicy();
+};
 
-  // void findParent(RequestData* rdata, ParentResult* result)
+//
+// API definition.
+class ParentSelectionStrategy
+{
+public:
+  // void selectParent(const ParentSelectionPolicy, *policy, bool firstCall, ParentResult *result, RequestData *rdata)
   //
-  //   Does initial parent lookup
+  // The implementation parent lookup.
   //
-  inkcoreapi void findParent(HttpRequestData *rdata, ParentResult *result);
+  virtual void selectParent(const ParentSelectionPolicy *policy, bool firstCall, ParentResult *result, RequestData *rdata) = 0;
 
-  // void markParentDown(ParentResult* rsult)
+  // void markParentDown(const ParentSelectionPolicy, *policy, ParentResult* rsult)
   //
   //    Marks the parent pointed to by result as down
   //
-  inkcoreapi void markParentDown(ParentResult *result);
-
-  // void recordRetrySuccess
-  //
-  //    After a successful retry, http calls this function
-  //      to clear the bits indicating the parent is down
-  //
-  void recordRetrySuccess(ParentResult *result);
+  virtual void markParentDown(const ParentSelectionPolicy *policy, ParentResult *result) = 0;
 
-  // void nextParent(RequestData* rdata, ParentResult* result);
+  // uint32_t numParents(ParentResult *result);
   //
-  //    Marks the parent pointed to by result as down and attempts
-  //      to find the next parent
+  // Returns the number of parent records in a strategy.
   //
-  inkcoreapi void nextParent(HttpRequestData *rdata, ParentResult *result);
+  virtual uint32_t numParents(ParentResult *result) = 0;
 
-  // bool parentExists(HttpRequestData* rdata)
+  // void markParentUp
   //
-  //   Returns true if there is a parent matching the request data and
-  //   false otherwise
-  bool parentExists(HttpRequestData *rdata);
-
-  // bool apiParentExists(HttpRequestData* rdata)
+  //    After a successful retry, http calls this function
+  //      to clear the bits indicating the parent is down
   //
-  //   Retures true if a parent has been set through the api
-  bool apiParentExists(HttpRequestData *rdata);
+  virtual void markParentUp(ParentResult *result) = 0;
 
-  P_table *ParentTable;
-  ParentRecord *DefaultParent;
-  int32_t ParentRetryTime;
-  int32_t ParentEnable;
-  int32_t FailThreshold;
-  int32_t DNS_ParentOnly;
+  // virtual destructor.
+  virtual ~ParentSelectionStrategy(){};
 };
 
-struct ParentConfig {
+class ParentConfigParams : public ConfigInfo
+{
 public:
-  static void startup();
-  static void reconfigure();
-  static void print();
+  P_table *parent_table;
+  ParentRecord *DefaultParent;
+  ParentConfigParams(P_table *_parent_table);
+  ParentSelectionPolicy *policy;
+  ~ParentConfigParams(){};
 
-  inkcoreapi static ParentConfigParams *
-  acquire()
+  bool apiParentExists(HttpRequestData *rdata);
+  void findParent(HttpRequestData *rdata, ParentResult *result);
+  void nextParent(HttpRequestData *rdata, ParentResult *result);
+  bool parentExists(HttpRequestData *rdata);
+
+  // implementation of functions from ParentSelectionStrategy.
+  void
+  selectParent(bool firstCall, ParentResult *result, RequestData *rdata)
   {
-    return (ParentConfigParams *)configProcessor.get(ParentConfig::m_id);
+    ink_release_assert(result->rec->selection_strategy != NULL);
+    return result->rec->selection_strategy->selectParent(policy, firstCall, result, rdata);
   }
-  inkcoreapi static void
-  release(ParentConfigParams *params)
+
+  void
+  markParentDown(ParentResult *result)
   {
-    configProcessor.release(ParentConfig::m_id, params);
+    ink_release_assert(result->rec->selection_strategy != NULL);
+    result->rec->selection_strategy->markParentDown(policy, result);
   }
 
+  uint32_t
+  numParents(ParentResult *result)
+  {
+    ink_release_assert(result->rec->selection_strategy != NULL);
+    return result->rec->selection_strategy->numParents(result);
+  }
 
-  static int m_id;
-};
-//
-// End API to outside world
-//
-
-
-// struct pRecord
-//
-//    A record for an invidual parent
-//
-struct pRecord : ATSConsistentHashNode {
-  char hostname[MAXDNAME + 1];
-  int port;
-  time_t failedAt;
-  int failCount;
-  int32_t upAt;
-  const char *scheme; // for which parent matches (if any)
-  int idx;
-  float weight;
+  void
+  markParentUp(ParentResult *result)
+  {
+    ink_release_assert(result != NULL);
+    result->rec->selection_strategy->markParentUp(result);
+  }
 };
 
-enum ParentRR_t {
-  P_NO_ROUND_ROBIN = 0,
-  P_STRICT_ROUND_ROBIN,
-  P_HASH_ROUND_ROBIN,
-  P_CONSISTENT_HASH,
-};
+class HttpRequestData;
 
-// class ParentRecord : public ControlBase
-//
-//   A record for a configuration line in the parent.config
-//    file
-//
-class ParentRecord : public ControlBase
-{
+struct ParentConfig {
 public:
-  ParentRecord()
-    : parents(NULL), num_parents(0), round_robin(P_NO_ROUND_ROBIN), ignore_query(false), rr_next(0), go_direct(true), chash(NULL)
+  static void startup();
+  static void reconfigure();
+  static void print();
+  static void set_parent_table(P_table *pTable, ParentRecord *rec, int num_elements);
+
+  static ParentConfigParams *
+  acquire()
   {
+    return (ParentConfigParams *)configProcessor.get(ParentConfig::m_id);
   }
 
-  ~ParentRecord();
-
-  config_parse_error Init(matcher_line *line_info);
-  bool DefaultInit(char *val);
-  void UpdateMatch(ParentResult *result, RequestData *rdata);
-  void FindParent(bool firstCall, ParentResult *result, RequestData *rdata, ParentConfigParams *config);
-  uint64_t getPathHash(HttpRequestData *hrdata, ATSHash64 *h);
-  void Print();
-  pRecord *parents;
-  int num_parents;
-
-  bool
-  bypass_ok() const
+  static void
+  release(ParentConfigParams *strategy)
   {
-    return go_direct;
+    configProcessor.release(ParentConfig::m_id, strategy);
   }
 
-  const char *scheme;
-  // private:
-  const char *ProcessParents(char *val);
-  void buildConsistentHash(void);
-  ParentRR_t round_robin;
-  bool ignore_query;
-  volatile uint32_t rr_next;
-  bool go_direct;
-  ATSConsistentHash *chash;
+  static int m_id;
 };
 
 // Helper Functions

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 9ec932c..5fae64c 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -363,7 +363,7 @@ HttpSM::init()
   // Added to skip dns if the document is in cache. DNS will be forced if there is a ip based ACL in
   // cache control or parent.config or if the doc_in_cache_skip_dns is disabled or if http caching is disabled
   // TODO: This probably doesn't honor this as a per-transaction overridable config.
-  t_state.force_dns = (ip_rule_in_CacheControlTable() || t_state.parent_params->ParentTable->ipMatch ||
+  t_state.force_dns = (ip_rule_in_CacheControlTable() || t_state.parent_params->parent_table->ipMatch ||
                        !(t_state.txn_conf->doc_in_cache_skip_dns) || !(t_state.txn_conf->cache_http));
 
   http_parser.m_allow_non_http = t_state.http_config_param->parser_allow_non_http;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/0e7975af/proxy/http/HttpTransact.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index 343a0e6..8c7e582 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -1360,7 +1360,7 @@ HttpTransact::HandleRequest(State *s)
   // if the newly added varible doc_in_cache_skip_dns is not enabled
   if (s->dns_info.lookup_name[0] <= '9' && s->dns_info.lookup_name[0] >= '0' &&
       (!s->state_machine->enable_redirection || !s->redirect_info.redirect_in_process) &&
-      s->parent_params->ParentTable->hostMatch) {
+      s->parent_params->parent_table->hostMatch) {
     s->force_dns = 1;
   }
   /* A redirect means we need to check some things again.
@@ -1772,7 +1772,7 @@ HttpTransact::OSDNSLookup(State *s)
     // we've come back after already trying the server to get a better address
     // and finished with all backtracking - return to trying the server.
     TRANSACT_RETURN(how_to_open_connection(s), HttpTransact::HandleResponse);
-  } else if (s->dns_info.lookup_name[0] <= '9' && s->dns_info.lookup_name[0] >= '0' && s->parent_params->ParentTable->hostMatch &&
+  } else if (s->dns_info.lookup_name[0] <= '9' && s->dns_info.lookup_name[0] >= '0' && s->parent_params->parent_table->hostMatch &&
              !s->http_config_param->no_dns_forward_to_parent) {
     // note, broken logic: ACC fudges the OR stmt to always be true,
     // 'AuthHttpAdapter' should do the rev-dns if needed, not here .
@@ -2668,6 +2668,18 @@ HttpTransact::HandleCacheOpenReadHit(State *s)
         update_current_info(&s->current, NULL, UNDEFINED_LOOKUP, 0);
         DebugTxn("http_trans", "CacheOpenReadHit - server_down, returning stale document");
       }
+      // a parent lookup could come back as PARENT_FAIL if in parent.config, go_direct == false and
+      // there are no available parents (all down).
+      else if (s->current.request_to == HOST_NONE && s->parent_result.r == PARENT_FAIL) {
+        if (is_server_negative_cached(s) && response_returnable == true && is_stale_cache_response_returnable(s) == true) {
+          server_up = false;
+          update_current_info(&s->current, NULL, UNDEFINED_LOOKUP, 0);
+          DebugTxn("http_trans", "CacheOpenReadHit - server_down, returning stale document");
+        } else {
+          handle_parent_died(s);
+          return;
+        }
+      }
     }
 
     if (server_up || s->stale_icp_lookup) {
@@ -3104,6 +3116,12 @@ HttpTransact::HandleCacheOpenReadMiss(State *s)
 
   if (!h->is_cache_control_set(HTTP_VALUE_ONLY_IF_CACHED)) {
     find_server_and_update_current_info(s);
+    // a parent lookup could come back as PARENT_FAIL if in parent.config go_direct == false and
+    // there are no available parents (all down).
+    if (s->parent_result.r == PARENT_FAIL) {
+      handle_parent_died(s);
+      return;
+    }
     if (!s->current.server->dst_addr.isValid()) {
       ink_release_assert(s->current.request_to == PARENT_PROXY || s->http_config_param->no_dns_forward_to_parent != 0);
       if (s->current.request_to == PARENT_PROXY) {
@@ -3504,7 +3522,7 @@ HttpTransact::handle_response_from_parent(State *s)
     s->current.server->connect_result = 0;
     SET_VIA_STRING(VIA_DETAIL_PP_CONNECT, VIA_DETAIL_PP_SUCCESS);
     if (s->parent_result.retry) {
-      s->parent_params->recordRetrySuccess(&s->parent_result);
+      s->parent_params->markParentUp(&s->parent_result);
     }
     handle_forward_server_connection_open(s);
     break;