You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zy...@apache.org on 2013/04/02 05:16:07 UTC

git commit: TS-1713: SRV support refine

Updated Branches:
  refs/heads/master c4b7b61d0 -> 457b231b6


TS-1713: SRV support refine

the currenty SRV supporting in the DNS is not complete, and here is
a full refine of the old codes, as a side affect we also bump the
HostDB database version to 3.0, and increase the disk spaces from
32MB to 200MB:
CONFIG proxy.config.hostdb.storage_size INT 200M

*******CAUTION******
YOU ARE WARNED THAT FAIL TO ADJUST THE hostdb.storage_size CAN NOT
START THE SERVER PROCESS PROPERLY!!


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

Branch: refs/heads/master
Commit: 457b231b6c6c3f53c7c7e5264460c2dc3ad8f9d5
Parents: c4b7b61
Author: weijin <ta...@taobao.com>
Authored: Tue Apr 2 10:52:54 2013 +0800
Committer: Zhao Yongming <mi...@gmail.com>
Committed: Tue Apr 2 11:14:02 2013 +0800

----------------------------------------------------------------------
 iocore/dns/DNS.cc                      |   39 +++--
 iocore/dns/Makefile.am                 |    1 -
 iocore/dns/SRV.cc                      |  161 ------------------
 iocore/dns/SRV.h                       |  110 ++-----------
 iocore/hostdb/HostDB.cc                |  238 +++++++++++++++++----------
 iocore/hostdb/I_HostDBProcessor.h      |  123 +++++++--------
 iocore/hostdb/P_HostDBProcessor.h      |   80 +++++++++-
 proxy/config/records.config.default.in |    2 +-
 proxy/http/HttpSM.cc                   |  163 +++++++++++++------
 proxy/http/HttpTransact.cc             |  140 ++--------------
 proxy/http/HttpTransact.h              |   15 ++-
 11 files changed, 460 insertions(+), 612 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/dns/DNS.cc
----------------------------------------------------------------------
diff --git a/iocore/dns/DNS.cc b/iocore/dns/DNS.cc
index a6f686d..8c85481 100644
--- a/iocore/dns/DNS.cc
+++ b/iocore/dns/DNS.cc
@@ -794,7 +794,6 @@ DNSHandler::recv_dns(int event, Event *e)
         if (dnsc->num == name_server)
           received_one(name_server);
       }
-      hostent_cache = protect_hostent.to_ptr();
     }
   }
 }
@@ -1358,8 +1357,10 @@ dns_process(DNSHandler *handler, HostEnt *buf, int len)
     u_char *cp = ((u_char *) h) + HFIXEDSZ;
     u_char *eom = (u_char *) h + len;
     int n;
-    unsigned char *srv[50];
-    int num_srv = 0;
+    ink_debug_assert(buf->srv_hosts.srv_host_count == 0 && buf->srv_hosts.srv_hosts_length == 0);
+    buf->srv_hosts.srv_host_count = 0;
+    buf->srv_hosts.srv_hosts_length = 0;
+    int &num_srv = buf->srv_hosts.srv_host_count;
     int rname_len = -1;
 
     //
@@ -1518,31 +1519,37 @@ dns_process(DNSHandler *handler, HostEnt *buf, int len)
           buflen -= n;
         }
       } else if (type == T_SRV) {
+        if (num_srv >= HOST_DB_MAX_ROUND_ROBIN_INFO)
+          break;
         cp = here;              /* hack */
         int strlen = dn_skipname(cp, eom);
         cp += strlen;
-        srv[num_srv] = cp;
+        const unsigned char *srv_off = cp;
         cp += SRV_FIXEDSZ;
         cp += dn_skipname(cp, eom);
         here = cp;              /* hack */
-        char srvname[MAXDNAME];
-        int r = ink_ns_name_ntop(srv[num_srv] + SRV_SERVER, srvname, MAXDNAME);
+        SRV *srv = &buf->srv_hosts.hosts[num_srv];
+        int r = ink_ns_name_ntop(srv_off + SRV_SERVER, srv->host, MAXDNAME);
         if (r <= 0) {
           /* FIXME: is this really an error? or just a continue; */
           ++error;
           goto Lerror;
         }
         Debug("dns_srv", "Discovered SRV record [from NS lookup] with cost:%d weight:%d port:%d with host:%s",
-              ink_get16(srv[num_srv] + SRV_COST),
-              ink_get16(srv[num_srv] + SRV_WEIGHT), ink_get16(srv[num_srv] + SRV_PORT), srvname);
-
-        SRV *s = SRVAllocator.alloc();
-        s->setPort(ink_get16(srv[num_srv] + SRV_PORT));
-        s->setPriority(ink_get16(srv[num_srv] + SRV_COST));
-        s->setWeight(ink_get16(srv[num_srv] + SRV_WEIGHT));
-        s->setHost(srvname);
-
-        buf->srv_hosts.insert(s);
+            ink_get16(srv_off + SRV_COST),
+            ink_get16(srv_off + SRV_WEIGHT), ink_get16(srv_off + SRV_PORT), srv->host);
+
+        srv->port = ink_get16(srv_off + SRV_PORT);
+        srv->priority = ink_get16(srv_off + SRV_COST);
+        srv->weight = ink_get16(srv_off + SRV_WEIGHT);
+        srv->host_len = r;
+        srv->host[r-1] = '\0';
+        srv->key = makeHostHash(srv->host);
+
+        if (srv->host[0] != '\0')
+          buf->srv_hosts.srv_hosts_length += r;
+        else
+          continue;
         ++num_srv;
       } else if (is_addr_query(type)) {
         if (answer) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/dns/Makefile.am
----------------------------------------------------------------------
diff --git a/iocore/dns/Makefile.am b/iocore/dns/Makefile.am
index c203dfb..300af79 100644
--- a/iocore/dns/Makefile.am
+++ b/iocore/dns/Makefile.am
@@ -52,7 +52,6 @@ libinkdns_a_SOURCES = \
   P_SplitDNS.h \
   P_SplitDNSProcessor.h \
   SplitDNS.cc \
-  SRV.cc \
   SRV.h \
   Inline.cc
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/dns/SRV.cc
----------------------------------------------------------------------
diff --git a/iocore/dns/SRV.cc b/iocore/dns/SRV.cc
deleted file mode 100644
index 37dd480..0000000
--- a/iocore/dns/SRV.cc
+++ /dev/null
@@ -1,161 +0,0 @@
-/** @file
-
-  Support for SRV records
-
-  @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.
-
-  @section description
-        Support for SRV records
-
-        http://www.faqs.org/rfcs/rfc2782.html
-        http://www.nongnu.org/ruli/
-        http://libsrv.cvs.sourceforge.net/libsrv/libsrv/src/libsrv.c
- */
-
-
-#include "P_DNS.h"
-
-struct HostDBRoundRobin;
-
-ClassAllocator<SRV> SRVAllocator("SRVAllocator");
-
-/*
-To select a target to be contacted next, arrange all SRV RRs
-(that have not been ordered yet) in any order, except that all
-those with weight 0 are placed at the beginning of the list.
-
-Compute the sum of the weights of those RRs, and with each RR
-associate the running sum in the selected order. Then choose a
-uniform random number between 0 and the sum computed
-(inclusive), and select the RR whose running sum value is the
-first in the selected order which is greater than or equal to
-the random number selected. The target host specified in the
-selected SRV RR is the next one to be contacted by the client.
-Remove this SRV RR from the set of the unordered SRV RRs and
-apply the described algorithm to the unordered SRV RRs to select
-the next target host.  Continue the ordering process until there
-are no unordered SRV RRs.  This process is repeated for each
-Priority.
-*/
-
-static InkRand SRVRand(55378008);
-
-void
-SRVHosts::getWeightedHost(char *ret_val)
-{
-  int a_prev;
-  int k = 0;
-  int accum = 0;
-  unsigned int pri = 0;
-  SRV *i;
-  //InkRand x(time(NULL));
-  int tmp[1024];
-  int j = 0;
-  int v;
-  uint32_t xx;
-
-  if (hosts.empty() || getCount() == 0) {
-    goto err;
-  }
-
-  /* Step 1/2 Sort based on 'priority': handled by operator<
-   */
-
-  hosts.sort();
-
-  /*
-   * Step 2/2: Select SRV RRs by random weighted order
-   */
-
-  //get lowest priority (now sorted)
-  i = hosts.head;
-
-  if (!i) {
-    goto err;
-  }
-  //Get current priority
-  pri = i->getPriority();
-  //Accumulate weight sum for priority
-
-  while (i != NULL && pri == i->getPriority()) {
-    a_prev = accum;
-    accum += i->getWeight();
-    for (j = a_prev; j < accum; j++) {
-      tmp[j] = k;
-    }
-    i = i->link.next;
-    k++;
-  }
-
-  Debug("dns_srv", "accum=%d for priority=%d", accum, pri);
-
-  if (!accum) {
-    Debug("dns_srv", "Accumulator was 0. eek.");
-    goto err;
-  }
-  //Pick random number: 0..accum
-  xx = SRVRand.random() % accum;
-
-  Debug("dns_srv", "picked %d as a random number", xx);
-
-  i = hosts.head;
-  v = tmp[xx];
-  j = 0;
-  while (j < v) {
-    i = i->link.next;
-    j++;
-  }
-  Debug("dns_srv", "using SRV record of: pri: %d, wei: %d, port: %d, host: %s",
-        i->getPriority(), i->getWeight(), i->getPort(), i->getHost());
-
-  ink_strlcpy(ret_val, i->getHost(), MAXDNAME);
-  if (strcmp(ret_val, "") == 0 || strcmp(ret_val, ".") == 0) {
-    goto err;
-  }
-  return;
-err:
-  Debug("dns_srv", "there was a problem figuring out getWeightedHost() -- we are returning a blank SRV host");
-  ret_val[0] = '\0';
-  return;
-}
-
-SRVHosts::SRVHosts(HostDBInfo * info)
-{
-  hosts.clear();
-  srv_host_count = 0;
-  if (!info)
-    return;
-  HostDBRoundRobin *rr_data = info->rr();
-
-  if (!rr_data) {
-    return;
-  }
-
-  for (unsigned int i = 0; i < info->srv_count; i++) {
-    /* get the RR data */
-    SRV *s = SRVAllocator.alloc();
-    HostDBInfo nfo = rr_data->info[i];
-    s->setPort(nfo.srv_port);
-    s->setPriority(nfo.srv_priority);
-    s->setWeight(nfo.srv_weight);
-    s->setHost(&rr_data->rr_srv_hosts[i][0]);
-    insert(s);
-  }
-  return;
-}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/dns/SRV.h
----------------------------------------------------------------------
diff --git a/iocore/dns/SRV.h b/iocore/dns/SRV.h
index b0f5191..4393d7e 100644
--- a/iocore/dns/SRV.h
+++ b/iocore/dns/SRV.h
@@ -29,128 +29,46 @@
 
 struct HostDBInfo;
 
+#define HOST_DB_MAX_ROUND_ROBIN_INFO  16
 #define RAND_INV_RANGE(r) ((int) ((RAND_MAX + 1) / (r)))
 
-class SRV
+struct SRV
 {
-private:
   unsigned int weight;
   unsigned int port;
   unsigned int priority;
   unsigned int ttl;
+  unsigned int host_len;
+  unsigned int key;
   char host[MAXDNAME];
 
-public:
-  LINK(SRV, link);
-  SRV():weight(0), port(0), priority(0), ttl(0)
+  SRV():weight(0), port(0), priority(0), ttl(0), host_len(0), key(0)
   {
-    memset(host, 0, MAXDNAME);
+    host[0] = '\0';
   }
-
-  unsigned int getWeight()
-  {
-    return weight;
-  }
-  unsigned int getPriority() const
-  {
-    return priority;
-  }
-  unsigned int getPort()
-  {
-    return port;
-  }
-  unsigned int getTTL()
-  {
-    return ttl;
-  }
-  char *getHost()
-  {
-    return &host[0];
-  }
-
-  void setWeight(int w)
-  {
-    weight = w;
-  }
-  void setTTL(int t)
-  {
-    ttl = t;
-  }
-  void setPort(int p)
-  {
-    port = p;
-  }
-  void setPriority(int p)
-  {
-    priority = p;
-  }
-  void setHost(const char *h)
-  {
-    if (!h) {
-      Debug("dns_srv", "SRV::setHost() was passed a NULL host -- better check your code)");
-      host[0] = '\0';
-      return;
-    }
-    if (*h == '\0') {
-      Debug("dns_srv", "SRV::setHost() was passed a blank host -- better check what might have happened.");
-      host[0] = '\0';
-      return;
-    }
-    ink_strlcpy(host, h, sizeof(host));
-  }
-
 };
 
-TS_INLINE bool
-operator<(const SRV & left, const SRV & right)
+inline bool
+operator<(const SRV &left, const SRV &right)
 {
-  return (left.getPriority() < right.getPriority());    /* lower priorities first :) */
+  // lower priorities first, then the key
+  return (left.priority == right.priority) ? (left.key < right.key) : (left.priority < right.priority);
 }
 
-extern ClassAllocator<SRV> SRVAllocator;
 
-class SRVHosts
+struct SRVHosts
 {
-private:
-  SortableQueue<SRV> hosts;
   int srv_host_count;
+  int srv_hosts_length;
+  SRV hosts[HOST_DB_MAX_ROUND_ROBIN_INFO];
 
-public:
    ~SRVHosts()
   {
-    SRV *i;
-    while ((i = hosts.dequeue())) {
-      Debug("dns_srv", "freeing srv entry inside SRVHosts::~SRVHosts");
-      SRVAllocator.free(i);
-    }
   }
 
-  SRVHosts():srv_host_count(0)
+  SRVHosts():srv_host_count(0), srv_hosts_length(0)
   {
-    hosts.clear();
-  }
-
-  SortableQueue<SRV> *getHosts() {
-    return &hosts;
   }
-
-  void getWeightedHost(char *);
-
-  bool insert(SRV * rec)
-  {
-    hosts.enqueue(rec);
-    srv_host_count++;
-    return true;
-  }
-
-  int getCount()
-  {
-    return srv_host_count;
-  }
-
-  /* convert this HostDBInfo to an SRVHosts */
-  SRVHosts(HostDBInfo * info);
-
 };
 
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/hostdb/HostDB.cc
----------------------------------------------------------------------
diff --git a/iocore/hostdb/HostDB.cc b/iocore/hostdb/HostDB.cc
index 1d9ef72..8ba573c 100644
--- a/iocore/hostdb/HostDB.cc
+++ b/iocore/hostdb/HostDB.cc
@@ -78,6 +78,15 @@ HostDBCache hostDB;
 static  Queue <HostDBContinuation > remoteHostDBQueue[MULTI_CACHE_PARTITIONS];
 #endif
 
+char *
+HostDBInfo::srvname(HostDBRoundRobin *rr)
+{
+  if (!is_srv || !data.srv.srv_offset)
+    return NULL;
+  ink_assert(this - rr->info >= 0 && this - rr->info < rr->n && data.srv.srv_offset < rr->length);
+  return (char *) rr + data.srv.srv_offset;
+}
+
 static inline int
 corrupt_debugging_callout(HostDBInfo * e, RebuildMC & r)
 {
@@ -562,24 +571,17 @@ make_md5(INK_MD5 & md5, const char *hostname, int len, int port, char const* pDN
 
 
 static bool
-reply_to_cont(Continuation * cont, HostDBInfo * ar)
+reply_to_cont(Continuation * cont, HostDBInfo * ar, bool is_srv = false)
 {
   const char *reason = "none";
   HostDBInfo *r = ar;
 
-  if (r == NULL) {
-    cont->handleEvent(EVENT_HOST_DB_LOOKUP, NULL);
+  if (r == NULL || r->is_srv != is_srv || r->failed()) {
+    cont->handleEvent(is_srv ? EVENT_SRV_LOOKUP : EVENT_HOST_DB_LOOKUP, NULL);
     return false;
   }
 
-  if (r->failed()) {
-    if (r->is_srv && r->srv_count) {
-      cont->handleEvent(EVENT_SRV_LOOKUP, NULL);
-      return false;
-    }
-    cont->handleEvent(EVENT_HOST_DB_LOOKUP, NULL);
-    return false;
-  } else {
+  {
     if (r->reverse_dns) {
       if (!r->hostname()) {
         reason = "missing hostname";
@@ -588,7 +590,8 @@ reply_to_cont(Continuation * cont, HostDBInfo * ar)
       }
       Debug("hostdb", "hostname = %s", r->hostname());
     }
-    if (r->round_robin) {
+
+    if (!r->is_srv && r->round_robin) {
       if (!r->rr()) {
         reason = "missing round-robin";
         ink_assert(!"missing round-robin");
@@ -597,26 +600,15 @@ reply_to_cont(Continuation * cont, HostDBInfo * ar)
       ip_text_buffer ipb;
       Debug("hostdb", "RR of %d with %d good, 1st IP = %s", r->rr()->n, r->rr()->good, ats_ip_ntop(r->ip(), ipb, sizeof ipb));
     }
-    if (r->is_srv && r->srv_count) {
-      cont->handleEvent(EVENT_SRV_LOOKUP, r);
-      if (!r->full)
-        goto Ldelete;
-      return true;
-    } else if (r->is_srv) {
-      /* failure case where this is an SRV lookup, but we got no records back  -- this is handled properly in process_srv_info */
-      cont->handleEvent(EVENT_SRV_LOOKUP, r);
-      return true;
-    }
-    cont->handleEvent(EVENT_HOST_DB_LOOKUP, r);
+
+    cont->handleEvent(is_srv ? EVENT_SRV_LOOKUP : EVENT_HOST_DB_LOOKUP, r);
+
     if (!r->full)
       goto Ldelete;
     return true;
   }
 Lerror:
-  if (r->is_srv && r->srv_count) {
-    cont->handleEvent(EVENT_SRV_LOOKUP, r);
-  }
-  cont->handleEvent(EVENT_HOST_DB_LOOKUP, NULL);
+  cont->handleEvent(is_srv ? EVENT_SRV_LOOKUP : EVENT_HOST_DB_LOOKUP, NULL);
 Ldelete:
   Warning("bogus entry deleted from HostDB: %s", reason);
   hostDB.delete_block(ar);
@@ -904,7 +896,7 @@ HostDBProcessor::getSRVbyname_imm(Continuation * cont, process_srv_info_pfn proc
 
   md5.host_name = hostname;
   md5.host_len = hostname ? (len ? len : strlen(hostname)) : 0;
-  md5.port = opt.port;
+  md5.port = 0;
   md5.db_mark = HOSTDB_MARK_SRV;
   md5.refresh();
 
@@ -1040,20 +1032,33 @@ HostDBProcessor::getbyname_imm(Continuation * cont, process_hostdb_info_pfn proc
 
 
 static void
-do_setby(HostDBInfo * r, HostDBApplicationInfo * app, const char *hostname, IpAddr const& ip)
+do_setby(HostDBInfo * r, HostDBApplicationInfo * app, const char *hostname, IpAddr const& ip, bool is_srv = false)
 {
   HostDBRoundRobin *rr = r->rr();
 
+  if (is_srv && (!r->is_srv || !rr))
+    return;
+
   if (rr) {
-    ink_assert(hostname);
-    for (int i = 0; i < rr->n; i++) {
-      if (rr->info[i].ip() == ip) {
-        Debug("hostdb", "immediate setby for %s", hostname ? hostname : "<addr>");
-        rr->info[i].app.allotment.application1 = app->allotment.application1;
-        rr->info[i].app.allotment.application2 = app->allotment.application2;
-        return;
+    if (is_srv) {
+      uint32_t key = makeHostHash(hostname);
+      for (int i = 0; i < rr->n; i++) {
+        if (key == rr->info[i].data.srv.key && !strcmp(hostname, rr->info[i].srvname(rr))) {
+          Debug("hostdb", "immediate setby for %s", hostname);
+          rr->info[i].app.allotment.application1 = app->allotment.application1;
+          rr->info[i].app.allotment.application2 = app->allotment.application2;
+          return;
+        }
+      }
+    } else
+      for (int i = 0; i < rr->n; i++) {
+        if (rr->info[i].ip() == ip) {
+          Debug("hostdb", "immediate setby for %s", hostname ? hostname : "<addr>");
+          rr->info[i].app.allotment.application1 = app->allotment.application1;
+          rr->info[i].app.allotment.application2 = app->allotment.application2;
+          return;
+        }
       }
-    }
   } else {
     if (r->reverse_dns || (!r->round_robin && ip == r->ip())) {
       Debug("hostdb", "immediate setby for %s", hostname ? hostname : "<addr>");
@@ -1100,7 +1105,30 @@ HostDBProcessor::setby(const char *hostname, int len, sockaddr const* ip, HostDB
   thread->schedule_in(c, MUTEX_RETRY_DELAY);
 }
 
+void
+HostDBProcessor::setby_srv(const char *hostname, int len, const char *target, HostDBApplicationInfo * app)
+{
+  if (!hostdb_enable || !hostname || !target)
+      return;
 
+  HostDBMD5 md5;
+  md5.host_name = hostname;
+  md5.host_len = len ? len : strlen(hostname);
+  md5.port = 0;
+  md5.db_mark = HOSTDB_MARK_SRV;
+  md5.refresh();
+
+  // Create a continuation to do a deaper probe in the background
+
+  HostDBContinuation *c = hostDBContAllocator.alloc();
+  c->init(md5);
+  strncpy(c->srv_target_name, target, MAXDNAME);
+  c->app.allotment.application1 = app->allotment.application1;
+  c->app.allotment.application2 = app->allotment.application2;
+  SET_CONTINUATION_HANDLER(c,
+      (HostDBContHandler) & HostDBContinuation::setbyEvent);
+  eventProcessor.schedule_imm(c);
+}
 int
 HostDBContinuation::setbyEvent(int event, Event * e)
 {
@@ -1109,7 +1137,7 @@ HostDBContinuation::setbyEvent(int event, Event * e)
   HostDBInfo *r = probe(mutex, md5, false);
 
   if (r)
-    do_setby(r, &app, md5.host_name, md5.ip);
+    do_setby(r, &app, md5.host_name, md5.ip, is_srv());
 
   hostdb_cont_free(this);
   return EVENT_DONE;
@@ -1254,7 +1282,9 @@ HostDBContinuation::lookup_done(IpAddr const& ip, char const* aname, bool around
     }
     i = insert(hostdb_ip_fail_timeout_interval);        // currently ... 0
     i->round_robin = false;
+    i->is_srv = is_srv();
     i->reverse_dns = !is_byname() && !is_srv();
+
     i->set_failed();
   } else {
     switch (hostdb_ttl_mode) {
@@ -1289,22 +1319,12 @@ HostDBContinuation::lookup_done(IpAddr const& ip, char const* aname, bool around
       }
       i->is_srv = false;
     } else if (is_srv()) {
-      ats_ip_set(i->ip(), ip);
-      i->reverse_dns = false;
-
-      if (srv) {                //failed case: srv == NULL
-        i->srv_count = srv->getCount();
-      } else {
-        i->srv_count = 0;
-      }
-
-      if (i->srv_count <= 0) {
-        i->round_robin = false;
-      } else {
-        i->round_robin = true;
-      }
+      ink_debug_assert(srv && srv->srv_host_count > 0 && srv->srv_host_count <= 16 && around_robin);
 
+      i->data.srv.srv_offset = srv->srv_host_count;
+      i->reverse_dns = false;
       i->is_srv = true;
+      i->round_robin = around_robin;
 
       if (md5.host_name != aname) {
         ink_strlcpy(md5_host_name_store, aname, sizeof(md5_host_name_store));
@@ -1420,7 +1440,7 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
     pending_action = NULL;
 
     if (is_srv()) {
-      rr = !failed && (e->srv_hosts.getCount() > 0);
+      rr = !failed && (e->srv_hosts.srv_host_count > 0);
     } else if (!failed) {
       rr = 0 != e->ent.h_addr_list[1];
     } else {
@@ -1434,13 +1454,22 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
     if (old_r)
       old_info = *old_r;
     HostDBRoundRobin *old_rr_data = old_r ? old_r->rr() : NULL;
-
+#ifdef DEBUG
+    if (old_rr_data) {
+      for (int i = 0; i < old_rr_data->n; ++i) {
+        if (old_r->md5_high != old_rr_data->info[i].md5_high ||
+            old_r->md5_low != old_rr_data->info[i].md5_low ||
+            old_r->md5_low_low != old_rr_data->info[i].md5_low_low)
+          ink_assert(0);
+      }
+    }
+#endif
     int n = 0, nn = 0;
     void* first = 0;
     uint8_t af = e ? e->ent.h_addrtype : AF_UNSPEC; // address family
     if (rr) {
       if (is_srv() && !failed) {
-        n = e->srv_hosts.getCount();
+        n = e->srv_hosts.srv_host_count;
       } else {
         void* ptr; // tmp for current entry.
         for (
@@ -1474,7 +1503,8 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
       if (first) ip_addr_set(tip, af, first);
       r = lookup_done(tip, md5.host_name, rr, ttl_seconds, failed ? 0 : &e->srv_hosts);
     } else if (is_srv()) {
-      if (first) ip_addr_set(tip, af, first);
+      if (!failed)
+        tip._family = AF_INET; // force the tip valid, or else the srv will fail
       r = lookup_done(tip,  /* junk: FIXME: is the code in lookup_done() wrong to NEED this? */
                       md5.host_name,     /* hostname */
                       rr,       /* is round robin, doesnt matter for SRV since we recheck getCount() inside lookup_done() */
@@ -1486,41 +1516,79 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
       r = lookup_done(md5.ip, e->ent.h_name, false, ttl_seconds, &e->srv_hosts);
     }
 
+    ink_debug_assert(!r || (r->app.allotment.application1 == 0 && r->app.allotment.application2 == 0));
+
     if (rr) {
-      int s = HostDBRoundRobin::size(n, is_srv());
+      int s = HostDBRoundRobin::size(n, e->srv_hosts.srv_hosts_length);
       HostDBRoundRobin *rr_data = (HostDBRoundRobin *) hostDB.alloc(&r->app.rr.offset, s);
       Debug("hostdb", "allocating %d bytes for %d RR at %p %d", s, n, rr_data, r->app.rr.offset);
       if (rr_data) {
+        rr_data->length = s;
         int i = 0, ii = 0;
         if (is_srv()) {
-          SortableQueue<SRV> *q = e->srv_hosts.getHosts();
-          if (q) {
-            for (i = 0; i < n; ++i) {
-              SRV *t = q->dequeue();
-              HostDBInfo& item = rr_data->info[i];
+          int skip = 0;
+          char *pos = (char *) rr_data + sizeof(HostDBRoundRobin) + n * sizeof(HostDBInfo);
+          SRV *q[HOST_DB_MAX_ROUND_ROBIN_INFO];
+          ink_debug_assert(n <= HOST_DB_MAX_ROUND_ROBIN_INFO);
+          // sort
+          for (i = 0; i < n; ++i) {
+            q[i] = &e->srv_hosts.hosts[i];
+          }
+          for (i = 0; i < n; ++i) {
+            for (ii = i + 1; ii < n; ++ii) {
+              if (*q[ii] < *q[i]) {
+                SRV *tmp = q[i];
+                q[i] = q[ii];
+                q[ii] = tmp;
+              }
+            }
+          }
 
-              ats_ip_invalidate(item.ip());
-              item.round_robin = 0;
-              item.reverse_dns = 0;
+          for (i = 0; i < n; ++i) {
+            SRV *t = q[i];
+            HostDBInfo& item = rr_data->info[i];
+            item.round_robin = 0;
+            item.reverse_dns = 0;
+            item.is_srv = 1;
+            item.data.srv.srv_weight = t->weight;
+            item.data.srv.srv_priority = t->priority;
+            item.data.srv.srv_port = t->port;
+            item.data.srv.key = t->key;
 
-              item.srv_weight = t->getWeight();
-              item.srv_priority = t->getPriority();
-              item.srv_port = t->getPort();
+            ink_debug_assert (skip + (int) t->host_len <= e->srv_hosts.srv_hosts_length);
 
-              ink_strlcpy(rr_data->rr_srv_hosts[i], t->getHost(), MAXDNAME);
-              rr_data->rr_srv_hosts[i][MAXDNAME - 1] = '\0';
-              item.is_srv = true;
+            memcpy(pos + skip, t->host, t->host_len);
+            item.data.srv.srv_offset = (pos - (char *) rr_data) + skip;
 
-              item.full = 1;
-              item.md5_high = r->md5_high;
-              item.md5_low = r->md5_low;
-              item.md5_low_low = r->md5_low_low;
-              SRVAllocator.free(t);
-              Debug("dns_srv", "inserted SRV RR record into HostDB with TTL: %d seconds", ttl_seconds);
+            skip += t->host_len;
+
+            item.md5_high = r->md5_high;
+            item.md5_low = r->md5_low;
+            item.md5_low_low = r->md5_low_low;
+            item.full = 1;
+
+            item.app.allotment.application1 = 0;
+            item.app.allotment.application2 = 0;
+            Debug("dns_srv", "inserted SRV RR record [%s] into HostDB with TTL: %d seconds", t->host, ttl_seconds);
+          }
+          rr_data->good = rr_data->n = n;
+          rr_data->current = 0;
+
+          // restore
+          if (old_rr_data) {
+            for (i = 0; i < rr_data->n; ++i) {
+              for (ii = 0; ii < old_rr_data->n; ++ii) {
+                if (rr_data->info[i].data.srv.key == old_rr_data->info[ii].data.srv.key) {
+                  char *new_host = rr_data->info[i].srvname(rr_data);
+                  char *old_host = old_rr_data->info[ii].srvname(old_rr_data);
+                  if (!strcmp(new_host, old_host))
+                    rr_data->info[i].app = old_rr_data->info[ii].app;
+                }
+              }
             }
           }
         } else {
-          for (; ii < nn; ++ii) {
+          for (ii = 0; ii < nn; ++ii) {
             if (is_addr_valid(af, e->ent.h_addr_list[ii])) {
               HostDBInfo& item = rr_data->info[i];
               ip_addr_set(item.ip(), af, e->ent.h_addr_list[ii]);
@@ -1528,6 +1596,7 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
               item.full = 1;
               item.round_robin = 0;
               item.reverse_dns = 0;
+              item.is_srv = 0;
               item.md5_high = r->md5_high;
               item.md5_low = r->md5_low;
               item.md5_low_low = r->md5_low_low;
@@ -1538,16 +1607,16 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
               ++i;
             }
           }
+          rr_data->good = rr_data->n = n;
+          rr_data->current = 0;
         }
-        rr_data->good = rr_data->n = n;
-        rr_data->current = 0;
       } else {
         ink_assert(!"out of room in hostdb data area");
         Warning("out of room in hostdb for round-robin DNS data");
         r->round_robin = 0;
       }
     }
-    if (!failed && !rr)
+    if (!failed && !rr && !is_srv())
       restore_info(r, old_r, old_info, old_rr_data);
     ink_assert(!r || !r->round_robin || !r->reverse_dns);
     ink_assert(failed || !r->round_robin || r->app.rr.offset);
@@ -1579,7 +1648,7 @@ HostDBContinuation::dnsEvent(int event, HostEnt * e)
         return EVENT_CONT;
       }
       if (!action.cancelled)
-        reply_to_cont(action.continuation, r);
+        reply_to_cont(action.continuation, r, is_srv());
     }
     // wake up everyone else who is waiting
     remove_trigger_pending_dns();
@@ -1738,7 +1807,7 @@ HostDBContinuation::do_put_response(ClusterMachine * m, HostDBInfo * r, Continua
 {
   // don't remote fill round-robin DNS entries
   // if configured not to cluster them
-  if (!c && r->round_robin && !hostdb_cluster_round_robin)
+  if (!hostdb_cluster || (!c && r->round_robin && !hostdb_cluster_round_robin))
     return;
 
   HostDB_put_message msg;
@@ -2181,8 +2250,7 @@ HostDBInfo::heap_size()
     HostDBRoundRobin *r = rr();
 
     if (r)
-      // this is a bit conservative, we might want to resurect them later
-      return HostDBRoundRobin::size(r->n, this->is_srv);
+      return r->length;
   }
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/hostdb/I_HostDBProcessor.h
----------------------------------------------------------------------
diff --git a/iocore/hostdb/I_HostDBProcessor.h b/iocore/hostdb/I_HostDBProcessor.h
index 65e1418..1408810 100644
--- a/iocore/hostdb/I_HostDBProcessor.h
+++ b/iocore/hostdb/I_HostDBProcessor.h
@@ -38,6 +38,7 @@
 
 #define HOST_DB_MAX_ROUND_ROBIN_INFO         16
 
+#define HOST_DB_SRV_PREFIX "_http._tcp."
 //
 // Data
 //
@@ -59,6 +60,26 @@ extern unsigned int hostdb_ip_fail_timeout_interval;
 extern unsigned int hostdb_serve_stale_but_revalidate;
 
 
+static inline unsigned int
+makeHostHash(const char *string)
+{
+  ink_debug_assert(string && *string);
+  if (!string || *string == 0)
+    return 0;
+
+  const uint32_t InitialFNV = 2166136261U;
+  const int32_t FNVMultiple = 16777619;
+
+  uint64_t hash = InitialFNV;
+  uint32_t *p = (uint32_t *) &hash;
+  while(*string)  {
+    p[0] = p[0] ^ (toupper(*string));
+    hash = (p[1] ^ p[0]) * FNVMultiple;
+    ++string;
+  }
+  return (p[1] ^ p[0]);
+}
+
 //
 // Types
 //
@@ -116,6 +137,15 @@ union HostDBApplicationInfo
 
 struct HostDBRoundRobin;
 
+struct SRVInfo
+{
+  unsigned int srv_offset:16;
+  unsigned int srv_weight:16;
+  unsigned int srv_priority:16;
+  unsigned int srv_port:16;
+  unsigned int key;
+};
+
 struct HostDBInfo
 {
   /** Internal IP address data.
@@ -125,36 +155,12 @@ struct HostDBInfo
   sockaddr const* ip() const { return &data.ip.sa; }
 
   char *hostname();
-  char *srvname();
+  char *srvname(HostDBRoundRobin *rr);
   HostDBRoundRobin *rr();
 
   /** Indicate that the HostDBInfo is BAD and should be deleted. */
   void bad() { full = 0; }
 
-  /** Check the HostDBInfo or selected RR entry of a HostDBInfo is ok. */
-  int ok(bool byname, HostDBInfo * rr = NULL) {
-    if (rr) {
-      if (!byname ||
-          rr->md5_high != md5_high ||
-          rr->md5_low != md5_low || rr->md5_low_low != md5_low_low || rr->reverse_dns || !rr->ip())
-        goto Lbad;
-    } else if (byname) {
-      if (reverse_dns)
-        goto Lbad;
-      if (!ats_is_ip(ip()))
-        goto Lbad;
-    } else {
-      if (!reverse_dns)
-        goto Lbad;
-      if (!hostname())
-        goto Lbad;
-    }
-    return 1;
-  Lbad:
-    bad();
-    return 0;
-  }
-
   /**
     Application specific data. NOTE: We need an integral number of these
     per block. This structure is 32 bytes. (at 200k hosts = 8 Meg). Which
@@ -221,8 +227,10 @@ struct HostDBInfo
     memcpy(&data, &that.data, sizeof data);
     ip_timestamp = that.ip_timestamp;
     ip_timeout_interval = that.ip_timeout_interval;
+    is_srv = that.is_srv;
     round_robin = that.round_robin;
     reverse_dns = that.reverse_dns;
+
     app.allotment.application1 = that.app.allotment.application1;
     app.allotment.application2 = that.app.allotment.application2;
   }
@@ -235,23 +243,19 @@ struct HostDBInfo
   union {
     IpEndpoint ip; ///< IP address / port data.
     int hostname_offset; ///< Some hostname thing.
+    SRVInfo srv;
   } data;
 
-  unsigned int srv_weight:16;
-  unsigned int srv_priority:16;
-  unsigned int srv_port:16;
-  unsigned int srv_count:15;
-  unsigned int is_srv:1;
-
   unsigned int ip_timestamp;
   // limited to 0x1FFFFF (24 days)
-  unsigned int ip_timeout_interval;
+  unsigned int ip_timeout_interval:31;
 
   unsigned int full:1;
   unsigned int backed:1;        // duplicated in lower level
   unsigned int deleted:1;
   unsigned int hits:3;
 
+  unsigned int is_srv:1; // steal a bit from ip_timeout_interval
   unsigned int round_robin:1;
   unsigned int reverse_dns:1;
 
@@ -261,11 +265,15 @@ struct HostDBInfo
   uint64_t md5_high;
 
   bool failed() {
-    return !((reverse_dns && data.hostname_offset) || ats_is_ip(ip()));
+    return !((is_srv && data.srv.srv_offset) || (reverse_dns && data.hostname_offset) || ats_is_ip(ip()));
   }
   void set_failed() {
-    reverse_dns = false;
-    ats_ip_invalidate(ip());
+    if (is_srv)
+      data.srv.srv_offset = 0;
+    else if (reverse_dns)
+      data.hostname_offset = 0;
+    else
+      ats_ip_invalidate(ip());
   }
 
   void set_deleted() { deleted = 1; }
@@ -279,11 +287,6 @@ struct HostDBInfo
     md5_high = 0;
     md5_low = 0;
     md5_low_low = 0;
-    is_srv = 0;
-    srv_weight = 0;
-    srv_priority = 0;
-    srv_port = 0;
-    srv_count = 0;
   }
 
   void set_full(uint64_t folded_md5, int buckets)
@@ -307,6 +310,7 @@ struct HostDBInfo
     hits = 0;
     round_robin = 0;
     reverse_dns = 0;
+    is_srv = 0;
   }
 
   uint64_t tag() {
@@ -319,17 +323,13 @@ struct HostDBInfo
   int *heap_offset_ptr();
 
 HostDBInfo()
-  : srv_weight(0)
-  , srv_priority(0)
-  , srv_port(0)
-  , srv_count(0)
-  , is_srv(0)
-  , ip_timestamp(0)
+  : ip_timestamp(0)
   , ip_timeout_interval(0)
   , full(0)
   , backed(0)
   , deleted(0)
   , hits(0)
+  , is_srv(0)
   , round_robin(0)
   , reverse_dns(0)
   , md5_low_low(0)
@@ -352,28 +352,14 @@ struct HostDBRoundRobin
   short good;
 
   unsigned short current;
+  unsigned short length;
   ink_time_t timed_rr_ctime;
 
-  HostDBInfo info[HOST_DB_MAX_ROUND_ROBIN_INFO];
-  char rr_srv_hosts[HOST_DB_MAX_ROUND_ROBIN_INFO][MAXDNAME];
+  HostDBInfo info[];
 
-  static int size(int nn, bool using_srv)
+  static int size(int nn, int srv_len = 0)
   {
-    if (using_srv) {
-      /*     sizeof this struct
-         minus
-         unused round-robin entries [info]
-         minus
-         unused srv host data [rr_srv_hosts]
-       */
-      return (int) ((sizeof(HostDBRoundRobin)) -
-                    (sizeof(HostDBInfo) * (HOST_DB_MAX_ROUND_ROBIN_INFO - nn)) -
-                    (sizeof(char) * MAXDNAME * (HOST_DB_MAX_ROUND_ROBIN_INFO - nn)));
-    } else {
-      return (int) (sizeof(HostDBRoundRobin) -
-                    sizeof(HostDBInfo) * (HOST_DB_MAX_ROUND_ROBIN_INFO - nn) -
-                    sizeof(char) * MAXDNAME * HOST_DB_MAX_ROUND_ROBIN_INFO);
-    }
+    return INK_ALIGN((int) (sizeof(HostDBRoundRobin) + nn * sizeof(HostDBInfo) + srv_len), 8);
   }
 
   /** Find the index of @a addr in member @a info.
@@ -381,15 +367,17 @@ struct HostDBRoundRobin
   */
   int index_of(sockaddr const* addr);
   HostDBInfo *find_ip(sockaddr const* addr);
+  // Find the srv target
+  HostDBInfo *find_target(const char *target);
   /** Select the next entry after @a addr.
       @note If @a addr isn't an address in the round robin nothing is updated.
       @return The selected entry or @c NULL if @a addr wasn't present.
    */
   HostDBInfo* select_next(sockaddr const* addr);
   HostDBInfo *select_best_http(sockaddr const* client_ip, ink_time_t now, int32_t fail_window);
-
+  HostDBInfo *select_best_srv(char *target, InkRand *rand, ink_time_t now, int32_t fail_window);
   HostDBRoundRobin()
-    : n(0), good(0), current(0), timed_rr_ctime(0)
+    : n(0), good(0), current(0), length(0), timed_rr_ctime(0)
   { }
 
 };
@@ -530,6 +518,9 @@ public:
     HostDBApplicationInfo * app ///< I don't know.
   );
 
+  void setby_srv(const char *hostname, int len, const char *target,
+      HostDBApplicationInfo * app);
+
 };
 
 void run_HostDBTest();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/iocore/hostdb/P_HostDBProcessor.h
----------------------------------------------------------------------
diff --git a/iocore/hostdb/P_HostDBProcessor.h b/iocore/hostdb/P_HostDBProcessor.h
index 3b934d0..387d6da 100644
--- a/iocore/hostdb/P_HostDBProcessor.h
+++ b/iocore/hostdb/P_HostDBProcessor.h
@@ -78,8 +78,8 @@ inline unsigned int HOSTDB_CLIENT_IP_HASH(
 #define CONFIGURATION_HISTORY_PROBE_DEPTH   1
 
 // Bump this any time hostdb format is changed
-#define HOST_DB_CACHE_MAJOR_VERSION         2
-#define HOST_DB_CACHE_MINOR_VERSION         2
+#define HOST_DB_CACHE_MAJOR_VERSION         3
+#define HOST_DB_CACHE_MINOR_VERSION         0
 // 2.2: IP family split 2.1 : IPv6
 
 #define DEFAULT_HOST_DB_FILENAME             "host.db"
@@ -175,7 +175,7 @@ struct HostDBCache: public MultiCache<HostDBInfo>
   }
 
   // This accounts for an average of 2 HostDBInfo per DNS cache (for round-robin etc.)
-  virtual size_t estimated_heap_bytes_per_entry() const { return sizeof(HostDBInfo) * 2; }
+  virtual size_t estimated_heap_bytes_per_entry() const { return sizeof(HostDBInfo) * 2 + 512; }
 
   Queue<HostDBContinuation, Continuation::Link_link> pending_dns[MULTI_CACHE_PARTITIONS];
   Queue<HostDBContinuation, Continuation::Link_link> &pending_dns_for_hash(INK_MD5 & md5);
@@ -219,6 +219,22 @@ HostDBRoundRobin::select_next(sockaddr const* ip) {
 }
 
 inline HostDBInfo *
+HostDBRoundRobin::find_target(const char *target) {
+  bool bad = (n <= 0 || n > HOST_DB_MAX_ROUND_ROBIN_INFO || good <= 0 || good > HOST_DB_MAX_ROUND_ROBIN_INFO);
+  if (bad) {
+    ink_assert(!"bad round robin size");
+    return NULL;
+  }
+
+  uint32_t key = makeHostHash(target);
+  for (int i = 0; i < good; i++) {
+    if (info[i].data.srv.key == key && !strcmp(target, info[i].srvname(this)))
+      return &info[i];
+  }
+  return NULL;
+}
+
+inline HostDBInfo *
 HostDBRoundRobin::select_best_http(sockaddr const* client_ip, ink_time_t now, int32_t fail_window)
 {
   bool bad = (n <= 0 || n > HOST_DB_MAX_ROUND_ROBIN_INFO || good <= 0 || good > HOST_DB_MAX_ROUND_ROBIN_INFO);
@@ -295,6 +311,63 @@ HostDBRoundRobin::select_best_http(sockaddr const* client_ip, ink_time_t now, in
   }
 }
 
+inline HostDBInfo *
+HostDBRoundRobin::select_best_srv(char *target, InkRand *rand, ink_time_t now, int32_t fail_window)
+{
+  bool bad = (n <= 0 || n > HOST_DB_MAX_ROUND_ROBIN_INFO || good <= 0 || good > HOST_DB_MAX_ROUND_ROBIN_INFO);
+
+  if (bad) {
+    ink_assert(!"bad round robin size");
+    return NULL;
+  }
+
+#ifdef DEBUG
+  for (int i = 1; i < good; ++i) {
+    ink_debug_assert(info[i].data.srv.srv_priority >= info[i-1].data.srv.srv_priority);
+  }
+#endif
+
+  int i = 0, len = 0;
+  uint32_t weight = 0, p = INT32_MAX;
+  HostDBInfo *result = NULL;
+  HostDBInfo *infos[HOST_DB_MAX_ROUND_ROBIN_INFO];
+
+  do {
+    if (info[i].app.http_data.last_failure != 0 &&
+        (uint32_t) (now - fail_window) < info[i].app.http_data.last_failure) {
+      continue;
+    }
+
+    if (info[i].app.http_data.last_failure)
+      info[i].app.http_data.last_failure = 0;
+
+    if (info[i].data.srv.srv_priority <= p) {
+      p = info[i].data.srv.srv_priority;
+      weight += info[i].data.srv.srv_weight;
+      infos[len++] = &info[i];
+    } else
+      break;
+  } while (++i < good);
+
+  if (len == 0) { // all failed
+    result = &info[current++ % good];
+  } else if (weight == 0) { // srv weight is 0
+    result = &info[current++ % len];
+  } else {
+    uint32_t xx = rand->random() % weight;
+    for (i = 0; i < len && xx >= infos[i]->data.srv.srv_weight; ++i)
+      xx -= infos[i]->data.srv.srv_weight;
+
+    result = infos[i];
+  }
+
+  if (result) {
+    strcpy(target, result->srvname(this));
+    return result;
+  }
+  return NULL;
+}
+
 //
 // Types
 //
@@ -351,6 +424,7 @@ struct HostDBContinuation: public Continuation
   //  char name[MAXDNAME];
   //  int namelen;
   char md5_host_name_store[MAXDNAME+1]; // used as backing store for @a md5
+  char srv_target_name[MAXDNAME];
   void *m_pDS;
   Action *pending_action;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/proxy/config/records.config.default.in
----------------------------------------------------------------------
diff --git a/proxy/config/records.config.default.in b/proxy/config/records.config.default.in
index ea74ade..9e396f0 100644
--- a/proxy/config/records.config.default.in
+++ b/proxy/config/records.config.default.in
@@ -393,7 +393,7 @@ CONFIG proxy.config.dns.validate_query_name INT 0
    # note that in order to increase hostdb.size, hostdb.storage_size should
    # also be increase. These are best guesses, you will have to monitor this.
 CONFIG proxy.config.hostdb.size INT 120000
-CONFIG proxy.config.hostdb.storage_size INT 32M
+CONFIG proxy.config.hostdb.storage_size INT 200M
    # ttl modes:
    #   0 = obey
    #   1 = ignore

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 36f5608..3b0b151 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -418,6 +418,7 @@ HttpSM::init()
   t_state.cache_info.config.cache_vary_default_other = t_state.http_config_param->cache_vary_default_other;
 
   t_state.init();
+  t_state.srv_lookup = HttpConfig::m_master.srv_enabled;
   // 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.
@@ -1975,51 +1976,34 @@ HttpSM::process_srv_info(HostDBInfo * r)
 {
   DebugSM("dns_srv", "beginning process_srv_info");
 
-  SRVHosts s(r);                /* handled by conversion constructor */
-  char new_host[MAXDNAME];
-
   /* we didnt get any SRV records, continue w normal lookup */
-  if (!r->srv_count) {
+  if (!r || !r->is_srv || !r->round_robin) {
+    t_state.dns_info.srv_hostname[0] = '\0';
+    t_state.dns_info.srv_lookup_success = false;
+    t_state.srv_lookup = false;
     DebugSM("dns_srv", "No SRV records were available, continuing to lookup %s", t_state.dns_info.lookup_name);
-    ink_strlcpy(new_host, t_state.dns_info.lookup_name, sizeof(new_host));
-    goto lookup;
-  }
-
-  s.getWeightedHost(&new_host[0]);
-
-  if (*new_host == '\0') {
-    DebugSM("dns_srv", "Weighted host returned was NULL or blank!, using %s as origin", t_state.dns_info.lookup_name);
-    ink_strlcpy(new_host, t_state.dns_info.lookup_name, sizeof(new_host));
-  } else {
-    DebugSM("dns_srv", "Weighted host now: %s", new_host);
-  }
-
-  DebugSM("dns_srv", "ending process_srv_info SRV stuff; moving on to lookup origin host");
-
-lookup:
-  DebugSM("http_seq", "[HttpSM::process_srv_info] Doing DNS Lookup based on SRV %s", new_host);
-
-  int server_port = t_state.current.server ? t_state.current.server->port : t_state.server_info.port;
-
-  HTTP_SM_SET_DEFAULT_HANDLER(&HttpSM::state_hostdb_lookup);
-
-  HostDBProcessor::Options opt;
-  if (t_state.api_txn_dns_timeout_value != -1) {
-    opt.timeout = t_state.api_txn_dns_timeout_value;
-    DebugSM("http_timeout", "beginning DNS lookup. allowing %d mseconds for DNS", t_state.api_txn_dns_timeout_value);
-  }
-  opt.flags = (t_state.cache_info.directives.does_client_permit_dns_storing) ? HostDBProcessor::HOSTDB_DO_NOT_FORCE_DNS : HostDBProcessor::HOSTDB_FORCE_DNS_RELOAD;
-  opt.port = server_port;
-  opt.host_res_style = ua_session->host_res_style;
-
-  Action *dns_lookup_action_handle = hostDBProcessor.getbyname_imm(this, (process_hostdb_info_pfn) & HttpSM::process_hostdb_info, &new_host[0], 0, opt);
-
-  if (dns_lookup_action_handle != ACTION_RESULT_DONE) {
-    ink_assert(!pending_action);
-    pending_action = dns_lookup_action_handle;
-    historical_action = pending_action;
   } else {
-    call_transact_and_set_next_state(NULL);
+    HostDBRoundRobin *rr = r->rr();
+    HostDBInfo *srv = NULL;
+    if (rr) {
+      srv = rr->select_best_srv(t_state.dns_info.srv_hostname, &mutex.m_ptr->thread_holding->generator,
+          ink_cluster_time(), (int) t_state.txn_conf->down_server_timeout);
+    }
+    if (!srv) {
+      t_state.dns_info.srv_lookup_success = false;
+      t_state.dns_info.srv_hostname[0] = '\0';
+      t_state.srv_lookup = false;
+      DebugSM("dns_srv", "SRV records empty for %s", t_state.dns_info.lookup_name);
+    } else {
+      ink_debug_assert(r->md5_high == srv->md5_high && r->md5_low == srv->md5_low &&
+          r->md5_low_low == srv->md5_low_low);
+      t_state.dns_info.srv_lookup_success = true;
+      t_state.dns_info.srv_port = srv->data.srv.srv_port;
+      t_state.dns_info.srv_app = srv->app;
+      //t_state.dns_info.single_srv = (rr->good == 1);
+      ink_debug_assert(srv->data.srv.key == makeHostHash(t_state.dns_info.srv_hostname));
+      DebugSM("dns_srv", "select SRV records %s", t_state.dns_info.srv_hostname);
+    }
   }
   return;
 }
@@ -2028,22 +2012,39 @@ void
 HttpSM::process_hostdb_info(HostDBInfo * r)
 {
   if (r && !r->failed()) {
-    HostDBInfo *rr = NULL;
+    ink_time_t now = ink_cluster_time();
+    HostDBInfo *ret = NULL;
     t_state.dns_info.lookup_success = true;
-
     if (r->round_robin) {
       // Since the time elapsed between current time and client_request_time
       // may be very large, we cannot use client_request_time to approximate
       // current time when calling select_best_http().
-      rr = r->rr()->select_best_http(&t_state.client_info.addr.sa, ink_cluster_time(), (int) t_state.txn_conf->down_server_timeout);
+      HostDBRoundRobin *rr = r->rr();
+      ret = rr->select_best_http(&t_state.client_info.addr.sa, now, (int) t_state.txn_conf->down_server_timeout);
       t_state.dns_info.round_robin = true;
+
+      // set the srv target`s last_failure
+      if (t_state.dns_info.srv_lookup_success) {
+        uint32_t last_failure = 0xFFFFFFFF;
+        for (int i = 0; i < rr->n && last_failure != 0; ++i) {
+          if (last_failure > rr->info[i].app.http_data.last_failure)
+            last_failure = rr->info[i].app.http_data.last_failure;
+        }
+
+        if (last_failure != 0 && (uint32_t) (now - t_state.txn_conf->down_server_timeout) < last_failure) {
+          HostDBApplicationInfo app;
+          app.allotment.application1 = 0;
+          app.allotment.application2 = 0;
+          app.http_data.last_failure = last_failure;
+          hostDBProcessor.setby_srv(t_state.dns_info.lookup_name, 0, t_state.dns_info.srv_hostname, &app);
+        }
+      }
     } else {
-      rr = r;
+      ret = r;
       t_state.dns_info.round_robin = false;
     }
-    if (rr) {
-//                  m_s.host_db_info = m_updated_host_db_info = *rr;
-      t_state.host_db_info = *rr;
+    if (ret) {
+      t_state.host_db_info = *ret;
       ink_release_assert(!t_state.host_db_info.reverse_dns);
       ink_release_assert(ats_is_ip(t_state.host_db_info.ip()));
     }
@@ -2088,6 +2089,35 @@ HttpSM::state_hostdb_lookup(int event, void *data)
     process_hostdb_info((HostDBInfo *) data);
     call_transact_and_set_next_state(NULL);
     break;
+    case EVENT_SRV_LOOKUP:
+    {
+      pending_action = NULL;
+      process_srv_info((HostDBInfo *) data);
+
+      char *host_name = t_state.dns_info.srv_lookup_success ? t_state.dns_info.srv_hostname : t_state.dns_info.lookup_name;
+      HostDBProcessor::Options opt;
+      opt.port = t_state.dns_info.srv_lookup_success ? t_state.dns_info.srv_port : t_state.server_info.port;
+      opt.flags = (t_state.cache_info.directives.does_client_permit_dns_storing)
+            ? HostDBProcessor::HOSTDB_DO_NOT_FORCE_DNS
+            : HostDBProcessor::HOSTDB_FORCE_DNS_RELOAD
+          ;
+      opt.timeout = (t_state.api_txn_dns_timeout_value != -1) ? t_state.api_txn_dns_timeout_value : 0;
+      opt.host_res_style = ua_session->host_res_style;
+
+      Action *dns_lookup_action_handle = hostDBProcessor.getbyname_imm(this,
+                                                                 (process_hostdb_info_pfn) & HttpSM::
+                                                                 process_hostdb_info,
+                                                                 host_name, 0,
+                                                                 opt);
+      if (dns_lookup_action_handle != ACTION_RESULT_DONE) {
+        ink_assert(!pending_action);
+        pending_action = dns_lookup_action_handle;
+        historical_action = pending_action;
+      } else {
+        call_transact_and_set_next_state(NULL);
+      }
+    }
+    break;
   case EVENT_HOST_DB_IP_REMOVED:
     ink_assert(!"Unexpected event from HostDB");
     break;
@@ -3767,16 +3797,15 @@ HttpSM::do_hostdb_lookup()
   ink_assert(pending_action == NULL);
 
   milestones.dns_lookup_begin = ink_get_hrtime();
-  bool use_srv_records = HttpConfig::m_master.srv_enabled;
+  bool use_srv_records = t_state.srv_lookup;
 
   if (use_srv_records) {
-    char* d = t_state.dns_info.srv_hostname;
+    char d[MAXDNAME];
 
     memcpy(d, "_http._tcp.", 11); // don't copy '\0'
     ink_strlcpy(d + 11, t_state.server_info.name, sizeof(d) - 11 ); // all in the name of performance!
 
     DebugSM("dns_srv", "Beginning lookup of SRV records for origin %s", d);
-    HTTP_SM_SET_DEFAULT_HANDLER(&HttpSM::state_srv_lookup);
 
     HostDBProcessor::Options opt;
     if (t_state.api_txn_dns_timeout_value != -1)
@@ -3788,6 +3817,28 @@ HttpSM::do_hostdb_lookup()
       ink_assert(!pending_action);
       pending_action = srv_lookup_action_handle;
       historical_action = pending_action;
+    } else {
+      char *host_name = t_state.dns_info.srv_lookup_success ? t_state.dns_info.srv_hostname : t_state.dns_info.lookup_name;
+      opt.port = t_state.dns_info.srv_lookup_success ? t_state.dns_info.srv_port : t_state.server_info.port;
+      opt.flags = (t_state.cache_info.directives.does_client_permit_dns_storing)
+            ? HostDBProcessor::HOSTDB_DO_NOT_FORCE_DNS
+            : HostDBProcessor::HOSTDB_FORCE_DNS_RELOAD
+          ;
+      opt.timeout = (t_state.api_txn_dns_timeout_value != -1) ? t_state.api_txn_dns_timeout_value : 0;
+      opt.host_res_style = ua_session->host_res_style;
+
+      Action *dns_lookup_action_handle = hostDBProcessor.getbyname_imm(this,
+                                                                 (process_hostdb_info_pfn) & HttpSM::
+                                                                 process_hostdb_info,
+                                                                 host_name, 0,
+                                                                 opt);
+      if (dns_lookup_action_handle != ACTION_RESULT_DONE) {
+        ink_assert(!pending_action);
+        pending_action = dns_lookup_action_handle;
+        historical_action = pending_action;
+      } else {
+        call_transact_and_set_next_state(NULL);
+      }
     }
     return;
   } else {                      /* we arent using SRV stuff... */
@@ -3891,6 +3942,14 @@ HttpSM::do_hostdb_update_if_necessary()
             sm_id,
             ats_ip_nptop(&t_state.current.server->addr.sa, addrbuf, sizeof(addrbuf)));
     }
+
+    if (t_state.dns_info.srv_lookup_success && t_state.dns_info.srv_app.http_data.last_failure != 0) {
+      t_state.dns_info.srv_app.http_data.last_failure = 0;
+      hostDBProcessor.setby_srv(t_state.dns_info.lookup_name, 0, t_state.dns_info.srv_hostname, &t_state.dns_info.srv_app);
+      DebugSM("http", "[%" PRId64 "] hostdb update marking SRV: %s as up",
+                  sm_id,
+                  t_state.dns_info.srv_hostname);
+    }
   }
 
   if (issue_update) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/proxy/http/HttpTransact.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index 931fe80..3e466f0 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -1204,6 +1204,17 @@ HttpTransact::HandleRequest(State* s)
     return;
   }
 
+  // if ip in url or cop test page, not do srv lookup.
+  if (s->srv_lookup) {
+    if (s->cop_test_page)
+      s->srv_lookup = false;
+    else {
+      IpEndpoint addr;
+      ats_ip_pton(s->server_info.name, &addr);
+      s->srv_lookup = !ats_is_ip(&addr);
+    }
+  }
+
   // if the request is a trace or options request, decrement the
   // max-forwards value. if the incoming max-forwards value was 0,
   // then we have to return a response to the client with the
@@ -3517,7 +3528,7 @@ HttpTransact::handle_response_from_server(State* s)
       // If this is a round robin DNS entry & we're tried configured
       //    number of times, we should try another node
 
-      bool use_srv_records = HttpConfig::m_master.srv_enabled;
+      //bool use_srv_records = HttpConfig::m_master.srv_enabled;
 
       if (DNSLookupInfo::OS_ADDR_TRY_CLIENT == s->dns_info.os_addr_style) {
         // attempt was based on client supplied server address. Try again
@@ -3531,10 +3542,7 @@ HttpTransact::handle_response_from_server(State* s)
         // families - that is locked in by the client source address.
         s->state_machine->ua_session->host_res_style = ats_host_res_match(&s->current.server->addr.sa);
         TRANSACT_RETURN(HttpTransact::DNS_LOOKUP, OSDNSLookup);
-      } else if (use_srv_records) {
-        delete_srv_entry(s, max_connect_retries);
-        return;
-      } else if (s->server_info.dns_round_robin &&
+      } else if ((s->dns_info.srv_lookup_success || s->server_info.dns_round_robin) &&
                  (s->txn_conf->connect_attempts_rr_retries > 0) &&
                  (s->current.attempts % s->txn_conf->connect_attempts_rr_retries == 0)) {
         delete_server_rr_entry(s, max_connect_retries);
@@ -3565,129 +3573,7 @@ HttpTransact::handle_response_from_server(State* s)
   return;
 }
 
-void
-HttpTransact::delete_srv_entry(State* s, int max_retries)
-{
-  /* we are using SRV lookups and this host failed -- lets remove it from the HostDB */
-  HostDBMD5 md5;
-  EThread *thread = this_ethread();
-  //ProxyMutex *mutex = thread->mutex;
-  md5.host_name = s->dns_info.srv_hostname;
-  if (!md5.host_name) {
-    TRANSACT_RETURN(OS_RR_MARK_DOWN, ReDNSRoundRobin);
-  }
-  md5.host_len = strlen(md5.host_name);
-  md5.db_mark = HOSTDB_MARK_SRV;
-  md5.refresh();
-
-  ProxyMutex *bucket_mutex = hostDB.lock_for_bucket((int) (fold_md5(md5.hash) % hostDB.buckets));
-
-  s->current.attempts++;
-
-  DebugTxn("http_trans", "[delete_srv_entry] attempts now: %d, max: %d", s->current.attempts, max_retries);
-  DebugTxn("dns_srv", "[delete_srv_entry] attempts now: %d, max: %d", s->current.attempts, max_retries);
 
-  MUTEX_TRY_LOCK(lock, bucket_mutex, thread);
-  if (lock) {
-//    IpEndpoint ip;
-    HostDBInfo *r = probe(bucket_mutex, md5, false);
-    if (r) {
-      if (r->is_srv) {
-        DebugTxn("dns_srv", "Marking SRV records for %s [Origin: %s] as bad", md5.host_name, s->dns_info.lookup_name);
-
-        uint64_t folded_md5 = fold_md5(md5.hash);
-
-        HostDBInfo *new_r = NULL;
-
-        DebugTxn("dns_srv", "[HttpTransact::delete_srv_entry] Adding relevent entries back into HostDB");
-
-        SRVHosts srv_hosts(r);  /* conversion constructor for SRVHosts() */
-        r->set_deleted();       //delete the original HostDB
-        hostDB.delete_block(r); //delete the original HostDB
-
-        new_r = hostDB.insert_block(folded_md5, NULL, 0);       //create new entry
-        new_r->md5_high = md5.hash[1];
-
-        SortableQueue<SRV> *q = srv_hosts.getHosts();        //get the Queue of SRV entries
-        SRV *srv_entry = NULL;
-
-        Queue<SRV> still_ok_hosts;
-
-        new_r->srv_count = 0;
-        while ((srv_entry = q->dequeue())) {    // ok to dequeue since this is the last time we are using this.
-          if (strcmp(srv_entry->getHost(), s->dns_info.lookup_name) != 0) {
-            still_ok_hosts.enqueue(srv_entry);
-            new_r->srv_count++;
-          } else {
-            SRVAllocator.free(srv_entry);
-          }
-        }
-
-        q = NULL;
-
-        /* no hosts DON'T match -- max out retries and return */
-        if (still_ok_hosts.empty()) {
-          DebugTxn("dns_srv", "No more SRV hosts to try that don't contain a host we just tried -- giving up");
-          s->current.attempts = max_retries;
-          TRANSACT_RETURN(OS_RR_MARK_DOWN, ReDNSRoundRobin);
-        }
-
-        /*
-           assert: at this point, we have (inside still_ok_hosts) those SRV records that were NOT pointing to the
-           same hostname as the one that just failed; lets reenqueue those into the HostDB and perform another "lookup"
-           which [hopefully] will find these records inside the HostDB and use them.
-         */
-
-        new_r->ip_timeout_interval = 45;        /* good for 45 seconds, then lets re-validate? */
-        new_r->ip_timestamp = hostdb_current_interval;
-        ats_ip_invalidate(new_r->ip());
-
-        /* these go into the RR area */
-        int n = new_r->srv_count;
-
-        if (n < 1) {
-          new_r->round_robin = 0;
-        } else {
-          new_r->round_robin = 1;
-          int sz = HostDBRoundRobin::size(n, true);
-          HostDBRoundRobin *rr_data = (HostDBRoundRobin *) hostDB.alloc(&new_r->app.rr.offset, sz);
-          DebugTxn("hostdb", "allocating %d bytes for %d RR at %p %d", sz, n, rr_data, new_r->app.rr.offset);
-          int i = 0;
-          while ((srv_entry = still_ok_hosts.dequeue())) {
-            DebugTxn("dns_srv", "Re-adding %s to HostDB [as a RR] after %s failed", srv_entry->getHost(), s->dns_info.lookup_name);
-            ats_ip_invalidate(rr_data->info[i].ip());
-            rr_data->info[i].round_robin = 0;
-            rr_data->info[i].reverse_dns = 0;
-
-            rr_data->info[i].srv_weight = srv_entry->getWeight();
-            rr_data->info[i].srv_priority = srv_entry->getPriority();
-            rr_data->info[i].srv_port = srv_entry->getPort();
-
-            ink_strlcpy(rr_data->rr_srv_hosts[i], srv_entry->getHost(), sizeof(rr_data->rr_srv_hosts[i]));
-            rr_data->info[i].is_srv = true;
-
-            rr_data->info[i].md5_high = new_r->md5_high;
-            rr_data->info[i].md5_low = new_r->md5_low;
-            rr_data->info[i].md5_low_low = new_r->md5_low_low;
-            rr_data->info[i].full = 1;
-            SRVAllocator.free(srv_entry);
-            i++;
-          }
-          rr_data->good = rr_data->n = n;
-          rr_data->current = 0;
-        }
-
-      } else {
-        DebugTxn("dns_srv", "Trying to delete a bad SRV for %s and something was wonky", md5.host_name);
-      }
-    } else {
-      DebugTxn("dns_srv", "No SRV data to remove. Ruh Roh Shaggy. Maxing out connection attempts...");
-      s->current.attempts = max_retries;
-    }
-  }
-
-  TRANSACT_RETURN(OS_RR_MARK_DOWN, ReDNSRoundRobin);
-}
 
 ///////////////////////////////////////////////////////////////////////////////
 // Name       : delete_server_rr_entry

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/457b231b/proxy/http/HttpTransact.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.h b/proxy/http/HttpTransact.h
index 2c3a9bd..a92b1ab 100644
--- a/proxy/http/HttpTransact.h
+++ b/proxy/http/HttpTransact.h
@@ -770,12 +770,18 @@ public:
     char srv_hostname[MAXDNAME];
     LookingUp_t looking_up;
     bool round_robin;
+    bool srv_lookup_success;
+    short srv_port;
+    HostDBApplicationInfo srv_app;
 
     _DNSLookupInfo()
     : attempts(0), os_addr_style(OS_ADDR_TRY_DEFAULT),
-        lookup_success(false), lookup_name(NULL), looking_up(UNDEFINED_LOOKUP), round_robin(false)
+        lookup_success(false), lookup_name(NULL), looking_up(UNDEFINED_LOOKUP), round_robin(false),
+        srv_lookup_success(false), srv_port(0)
     {
-      memset(&srv_hostname, 0, sizeof(srv_hostname));
+      srv_hostname[0] = '\0';
+      srv_app.allotment.application1 = 0;
+      srv_app.allotment.application2 = 0;
     }
   } DNSLookupInfo;
 
@@ -913,7 +919,8 @@ public:
 
     // for negative caching
     bool negative_caching;
-
+    // for srv_lookup
+    bool srv_lookup;
     // for authenticated content caching
     CacheAuth_t www_auth_content;
 
@@ -1038,6 +1045,7 @@ public:
         first_stats(),
         current_stats(NULL),
         negative_caching(false),
+        srv_lookup(false),
         www_auth_content(CACHE_AUTH_NONE),
         client_connection_enabled(true),
         acl_filtering_performed(false),
@@ -1235,7 +1243,6 @@ public:
   static void handle_response_from_parent(State* s);
   static void handle_response_from_server(State* s);
   static void delete_server_rr_entry(State* s, int max_retries);
-  static void delete_srv_entry(State* s, int max_retries);
   static void retry_server_connection_not_open(State* s, ServerState_t conn_state, int max_retries);
   static void handle_server_connection_not_open(State* s);
   static void handle_forward_server_connection_open(State* s);