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 2010/02/16 21:54:22 UTC

svn commit: r910687 - in /incubator/trafficserver/traffic/trunk/proxy: ./ http2/ http2/remap/

Author: zwoop
Date: Tue Feb 16 20:54:21 2010
New Revision: 910687

URL: http://svn.apache.org/viewvc?rev=910687&view=rev
Log:
TS-87: Implement trie lookup for resolving remap rules.
       Author: Manjesh Nilange
       Review: Anirban Kundu / Leif

Added:
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/Trie.h
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.cc
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.h
Modified:
    incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.cc
    incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.h
    incubator/trafficserver/traffic/trunk/proxy/http2/Makefile.am
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/StringHash.cc
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.cc
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.h
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.cc
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.h
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.cc
    incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.h

Modified: incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.cc?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.cc (original)
+++ incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.cc Tue Feb 16 20:54:21 2010
@@ -43,7 +43,6 @@
 #include "RemapProcessor.h"
 #include "UrlRewrite.h"
 #include "UrlMapping.h"
-#include "UmsHelper.h"
 #include "StringHash.h"
 #include "ink_unused.h"      /* MAGIC_EDITING_TAG */
 

Modified: incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.h?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.h (original)
+++ incubator/trafficserver/traffic/trunk/proxy/ReverseProxy.h Tue Feb 16 20:54:21 2010
@@ -40,7 +40,6 @@
 #include "HttpTransact.h"
 #include "RemapPluginInfo.h"
 #include "UrlRewrite.h"
-#include "UmsHelper.h"
 #include "UrlMapping.h"
 
 #define EMPTY_PORT_MAPPING (ink32)~0

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/Makefile.am?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/Makefile.am (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/Makefile.am Tue Feb 16 20:54:21 2010
@@ -90,9 +90,11 @@
   remap/RemapProcessor.h \
   remap/StringHash.cc \
   remap/StringHash.h \
-  remap/UmsHelper.cc \
-  remap/UmsHelper.h \
   remap/UrlMapping.cc \
   remap/UrlMapping.h \
   remap/UrlRewrite.cc \
-  remap/UrlRewrite.h
+  remap/UrlRewrite.h \
+  remap/Trie.h \
+  remap/UrlMappingPathIndex.h \
+  remap/UrlMappingPathIndex.cc
+

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/StringHash.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/StringHash.cc?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/StringHash.cc (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/StringHash.cc Tue Feb 16 20:54:21 2010
@@ -24,7 +24,6 @@
 #include "StringHash.h"
 #include "Main.h"
 #include "Error.h"
-#include "UmsHelper.h"
 
 
 // ===============================================================================

Added: incubator/trafficserver/traffic/trunk/proxy/http2/remap/Trie.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/Trie.h?rev=910687&view=auto
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/Trie.h (added)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/Trie.h Tue Feb 16 20:54:21 2010
@@ -0,0 +1,220 @@
+/** @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.
+*/
+#ifndef _TRIE_H
+#define _TRIE_H
+
+#include <list>
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+template<typename T>
+class Trie
+{
+public:
+  Trie() { m_root.Clear(); }
+
+  // will return false for duplicates; key should be NULL-terminated
+  // if key_len is defaulted to -1
+  bool Insert(const char *key, const T &value, int rank, int key_len = -1);
+
+  // will return false if not found; else value_ptr will point to found value
+  bool Search(const char *key, T *&value_ptr, int key_len = -1);
+
+  void Clear();
+
+  typedef std::list<T*> ValuePointerList;
+  const ValuePointerList &GetValues() const { return m_value_list; }
+
+  virtual ~Trie() { Clear(); };
+  
+
+private:
+  static const int N_NODE_CHILDREN = 256;
+
+  struct Node 
+  {
+    T value;
+    bool occupied;
+    int rank;
+    Node *children[N_NODE_CHILDREN];
+    void Clear() {
+      occupied = false;
+      rank = 0;
+      bzero(children, sizeof(Node *) * N_NODE_CHILDREN);
+    };
+    void Print(const char *debug_tag) const;
+  };
+
+  Node m_root;
+  ValuePointerList m_value_list;
+  
+  void _CheckArgs(const char *key, int &key_len) const;
+  void _Clear(Node *node);
+
+  // make copy-constructor and assignment operator private
+  // till we properly implement them
+  Trie(const Trie<T> &rhs) { };
+  Trie &operator =(const Trie<T> &rhs) { return *this; }
+};
+
+template<typename T>
+void
+Trie<T>::_CheckArgs(const char *key, int &key_len) const
+{
+  if (!key) {
+    key_len = 0;
+  } 
+  else if (key_len == -1) {
+    key_len = strlen(key);
+  }
+}
+
+template<typename T> 
+bool 
+Trie<T>::Insert(const char *key, const T &value, int rank, int key_len /* = -1 */) 
+{
+  _CheckArgs(key, key_len);
+
+  bool retval = false;
+  Node *next_node;
+  Node *curr_node = &m_root;
+  int i = 0;
+
+  while (true) {
+    if (is_debug_tag_set("Trie::Insert")) {
+      Debug("Trie::Insert", "Visiting Node...");
+      curr_node->Print("Trie::Insert");
+    }
+    if (i == key_len) {
+      break;
+    }
+    next_node = curr_node->children[key[i]];
+    if (!next_node) {
+      while (i < key_len) {
+        Debug("Trie::Insert", "Creating child node for char %c (%d)", key[i], key[i]);
+        curr_node->children[key[i]] = static_cast<Node*>(ink_malloc(sizeof(Node)));
+        curr_node = curr_node->children[key[i]];
+        curr_node->Clear();
+        ++i;
+      }
+      break;
+    }
+    curr_node = next_node;
+    ++i;
+  }
+
+  if (curr_node->occupied) {
+    Error("Cannot insert duplicate!");
+  }
+  else {
+    curr_node->occupied = true;
+    curr_node->value = value;
+    curr_node->rank = rank;
+    m_value_list.push_back(&(curr_node->value));
+    retval = true;
+    Debug("Trie::Insert", "inserted new element!");
+  }
+  return retval;
+}
+  
+template<typename T>
+bool
+Trie<T>::Search(const char *key, T *&value_ptr, int key_len /* = -1 */)
+{
+  _CheckArgs(key, key_len);
+  
+  Node *found_node = 0;
+  Node *curr_node = &m_root;
+  int i = 0;
+
+  while (curr_node) {
+    if (is_debug_tag_set("Trie::Search")) {
+      Debug("Trie::Search", "Visiting node...");
+      curr_node->Print("Trie::Search");
+    }
+    if (curr_node->occupied) {
+      if (!found_node) {
+        found_node = curr_node;
+      }
+      else {
+        if (curr_node->rank < found_node->rank) {
+          found_node = curr_node;
+        }
+      }
+    }
+    if (i == key_len) {
+      break;
+    }
+    curr_node = curr_node->children[key[i]];
+    ++i;
+  }
+
+  if (found_node) {
+    Debug("Trie::Search", "Returning element with rank %d", found_node->rank);
+    value_ptr = &(found_node->value);
+    return true;
+  }
+  return false;
+}
+
+template<typename T>
+void
+Trie<T>::_Clear(Node *node) 
+{
+  for (int i = 0; i < N_NODE_CHILDREN; ++i) {
+    if (node->children[i]) {
+      _Clear(node->children[i]);
+      ink_free(node->children[i]);
+    }
+  }
+}
+
+template<typename T>
+void
+Trie<T>::Clear()
+{
+  _Clear(&m_root);
+  m_root.Clear();
+  m_value_list.clear();
+}
+
+template<typename T>
+void
+Trie<T>::Node::Print(const char *debug_tag) const
+{
+  if (occupied) {
+    Debug(debug_tag, "Node is occupied");
+    Debug(debug_tag, "Node has rank %d", rank);
+  } else {
+    Debug(debug_tag, "Node is not occupied");
+  }
+  for (int i = 0; i < N_NODE_CHILDREN; ++i) {
+    if (children[i]) {
+      Debug(debug_tag, "Node has child for char %c", static_cast<char>(i));
+    }
+  }
+}
+
+#endif // _TRIE_H

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.cc?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.cc (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.cc Tue Feb 16 20:54:21 2010
@@ -1,193 +0,0 @@
-/** @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.
- */
-
-#include "UmsHelper.h"
-#include "StringHash.h"
-
-ums_helper::ums_helper():empty_list(NULL), unique_list(NULL), hash_table(NULL), min_path_size(0), max_path_size(0), map_cnt(0),
-tag_present(false)
-{
-}
-
-void
-ums_helper::delete_hash_table(void)
-{
-  if (hash_table) {
-    delete hash_table;
-    hash_table = NULL;
-  }
-}
-
-ums_helper::~ums_helper()
-{
-  delete_hash_table();
-}
-
-StringHash *
-ums_helper::init_hash_table(int _map_cnt)
-{
-  int hash_tbl_size;
-  delete_hash_table();
-  if (_map_cnt < 0)
-    _map_cnt = map_cnt;
-  if (unlikely((hash_tbl_size = (_map_cnt << 5)) > STRINGHASH_MAX_TBL_SIZE)) {
-    hash_tbl_size = STRINGHASH_MAX_TBL_SIZE;
-  }
-  hash_table = NEW(new StringHash(hash_tbl_size));
-  return hash_table;
-}
-
-int
-ums_helper::load_hash_table(url_mapping * list)
-{
-  url_mapping *ul, *ut;
-  StringHashEntry *he;
-  const char *from_path;
-  int from_path_len, load_cnt = 0;
-
-  if (likely(hash_table)) {
-    for (ul = list; ul; ul = ul->next_schema) {
-      ul->next_hash = NULL;
-      if ((from_path = ul->fromURL.path_get(&from_path_len)) != NULL) {
-        if (from_path_len > min_path_size)
-          from_path_len = min_path_size;
-        if (likely((he = hash_table->find_or_add(ul, from_path, from_path_len)) != NULL) && he->ptr != (void *) ul) {
-          for (ut = (url_mapping *) he->ptr; ut->next_hash;)
-            ut = ut->next_hash;
-          ut->next_hash = ul;
-        }
-      }
-      load_cnt++;
-    }
-  }
-  return load_cnt;
-}
-
-url_mapping *
-ums_helper::lookup_best_empty(const char *request_host, int request_port, char *tag)
-{
-  url_mapping *ht_entry = empty_list;
-
-  if (unlikely(tag_present)) {
-    for (; ht_entry; ht_entry = ht_entry->next_schema) {
-      bool tags_match = (ht_entry->tag && (!tag || strcmp(tag, ht_entry->tag))) ? false : true;
-      if (tags_match && (*request_host == '\0' || request_port == ht_entry->fromURL.port_get()))
-        break;
-    }
-  } else {
-    for (; ht_entry; ht_entry = ht_entry->next_schema) {
-      if (*request_host == '\0' || request_port == ht_entry->fromURL.port_get())
-        break;
-    }
-  }
-  return ht_entry;
-}
-
-url_mapping *
-ums_helper::lookup_best_notempty(url_mapping * ht_entry, const char *request_host, int request_port,
-                                 const char *request_path, int request_path_len, char *tag)
-{
-  bool tags_match;
-  const char *from_path;
-  int from_path_len, tmp;
-  StringHashEntry *he;
-
-  if (unique_list)              // most complicated but very rare case - unique_list != 0
-  {
-    for (; ht_entry; ht_entry = ht_entry->next_schema) {
-      tags_match = (ht_entry->tag && (!tag || strcmp(tag, ht_entry->tag))) ? false : true;
-      if (tags_match && (!request_host[0] || request_port == ht_entry->fromURL.port_get())) {
-        from_path = ht_entry->fromURL.path_get(&from_path_len);
-        if (ht_entry->unique) {
-          if (from_path && from_path_len == request_path_len && !memcmp(from_path, request_path, from_path_len))
-            break;
-        } else if (!from_path || (request_path_len >= from_path_len && !memcmp(from_path, request_path, from_path_len)))
-          break;
-      }
-    }
-    return ht_entry;
-  }
-  // unique_list is empty
-  if (empty_list)               // unique_list == 0 && empty_list != 0
-  {
-    for (; ht_entry; ht_entry = ht_entry->next_schema) {
-      tags_match = (ht_entry->tag && (!tag || strcmp(tag, ht_entry->tag))) ? false : true;
-      if (tags_match && (!request_host[0] || request_port == ht_entry->fromURL.port_get())) {
-        from_path = ht_entry->fromURL.path_get(&from_path_len);
-        if (!from_path || (request_path_len >= from_path_len && !memcmp(from_path, request_path, from_path_len)))
-          break;
-      }
-    }
-    return ht_entry;
-  }
-  // lh->unique_list == 0 && lh->empty_list == 0
-  if (likely((tmp = request_path_len) >= min_path_size)) {
-    if (hash_table)             // the best possible case from a performance point of view - we can use sorted hash
-    {
-      int lookup_size = (tmp > min_path_size) ? min_path_size : tmp;
-      if ((he = hash_table->find_or_add(0, request_path, lookup_size)) != NULL) {
-        if (tag_present) {
-          for (ht_entry = (url_mapping *) he->ptr; ht_entry; ht_entry = ht_entry->next_hash) {
-            tags_match = (ht_entry->tag && (!tag || strcmp(tag, ht_entry->tag))) ? false : true;
-            if (tags_match && (!request_host[0] || request_port == ht_entry->fromURL.port_get())) {
-              from_path = ht_entry->fromURL.path_get(&from_path_len);
-              if (request_path_len >= from_path_len && !memcmp(from_path, request_path, from_path_len))
-                break;
-            }
-          }
-        } else {
-          for (ht_entry = (url_mapping *) he->ptr; ht_entry; ht_entry = ht_entry->next_hash) {
-            if (!request_host[0] || request_port == ht_entry->fromURL.port_get()) {
-              from_path = ht_entry->fromURL.path_get(&from_path_len);
-              if (request_path_len >= from_path_len && !memcmp(from_path, request_path, from_path_len))
-                break;
-            }
-          }
-        }
-      } else
-        ht_entry = NULL;
-    } else                      // there is no lookup table - very limited search (from one item)
-    {
-      if (tag_present) {
-        for (; ht_entry; ht_entry = ht_entry->next_schema) {
-          tags_match = (ht_entry->tag && (!tag || strcmp(tag, ht_entry->tag))) ? false : true;
-          if (tags_match && (!request_host[0] || request_port == ht_entry->fromURL.port_get())) {
-            from_path = ht_entry->fromURL.path_get(&from_path_len);
-            if (request_path_len >= from_path_len && !memcmp(from_path, request_path, from_path_len))
-              break;
-          }
-        }
-      } else {
-        for (; ht_entry; ht_entry = ht_entry->next_schema) {
-          if ((!request_host[0] || request_port == ht_entry->fromURL.port_get())) {
-            from_path = ht_entry->fromURL.path_get(&from_path_len);
-            if (request_path_len >= from_path_len && !memcmp(from_path, request_path, from_path_len))
-              break;
-          }
-        }
-      }
-    }
-    return ht_entry;
-  }
-  return NULL;
-}

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.h?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.h (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UmsHelper.h Tue Feb 16 20:54:21 2010
@@ -1,56 +0,0 @@
-/** @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.
- */
-
-#ifndef _UMS_HELPER_H_
-#define _UMS_HELPER_H_
-
-#include "UrlMapping.h"
-#include "StringHash.h"
-
-class url_mapping;
-
-/**
- * Used to store addtional information for fast search in UrlRewrite::TableLookup
-**/
-class ums_helper
-{
-public:
-  url_mapping * empty_list;
-  url_mapping *unique_list;
-  StringHash *hash_table;
-  int min_path_size;
-  int max_path_size;
-  int map_cnt;
-  bool tag_present;
-    ums_helper();
-   ~ums_helper();
-  StringHash *init_hash_table(int _map_cnt = (-1));
-  void delete_hash_table(void);
-  int load_hash_table(url_mapping * list);
-  url_mapping *lookup_best_empty(const char *request_host, int request_port, char *tag);
-  url_mapping *lookup_best_notempty(url_mapping * ht_entry,
-                                    const char *request_host, int request_port,
-                                    const char *request_path, int request_path_len, char *tag);
-};
-
-#endif

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.cc?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.cc (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.cc Tue Feb 16 20:54:21 2010
@@ -27,12 +27,10 @@
  * 
 **/
 url_mapping::url_mapping(int rank /* = 0 */)
-:next(NULL), next_root_schema(NULL), next_schema(NULL), next_empty(NULL),
-next_unique(NULL), next_hash(NULL), lookup_helper(NULL), from_path_len(0), fromURL(), toURL(),
-homePageRedirect(false), unique(false), default_redirect_url(false), optional_referer(false),
-negative_referer(false), no_negative_cache(false), wildcard_from_scheme(false), pristine_host_hdr(-1),
-chunking_enabled(-1), tag(NULL), filter_redirect_url(NULL), map_id(0), referer_list(0), redir_chunk_list(0),
-filter(NULL), _plugin_count(0), _cur_instance_count(0), _rank(rank)
+  : from_path_len(0), fromURL(), toURL(), homePageRedirect(false), unique(false), default_redirect_url(false), 
+    optional_referer(false), negative_referer(false), no_negative_cache(false), wildcard_from_scheme(false), 
+    pristine_host_hdr(-1), chunking_enabled(-1), tag(NULL), filter_redirect_url(NULL), referer_list(0), 
+    redir_chunk_list(0), filter(NULL), _plugin_count(0), _cur_instance_count(0), _rank(rank)
 { /* nop */ ;
 }
 
@@ -146,10 +144,6 @@
   redirect_tag_str *rc;
   acl_filter_rule *afr;
 
-  if (lookup_helper) {
-    delete lookup_helper;
-    lookup_helper = NULL;
-  }
   if (tag) {
     tag = (char *) xfree_null(tag);
   }

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.h?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.h (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMapping.h Tue Feb 16 20:54:21 2010
@@ -24,7 +24,6 @@
 #ifndef _URL_MAPPING_H_
 #define _URL_MAPPING_H_
 
-#include "UmsHelper.h"
 #include "AclFiltering.h"
 #include <deque>
 #include <map>
@@ -35,8 +34,6 @@
 #include "RemapPluginInfo.h"
 #include "HttpTransact.h"
 
-class ums_helper;
-
 /**
  * Used to store http referer strings (and/or regexp) 
 **/
@@ -103,13 +100,6 @@
 public:
   url_mapping(int rank = 0);
   ~url_mapping();
-  url_mapping *next;            // next in main list (primary hash by host name)
-  url_mapping *next_root_schema;        // list of different schemas (valid only for root nodes)
-  url_mapping *next_schema;     // next with the same schema
-  url_mapping *next_empty;      // next in ums_helper.empty_list
-  url_mapping *next_unique;     // next in ums_helper.unique_list
-  url_mapping *next_hash;       // next in hash table list
-  ums_helper *lookup_helper;    // primary helper (from host name hash list)
 
   bool add_plugin(remap_plugin_info *);
   remap_plugin_info *get_plugin(unsigned int);

Added: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.cc?rev=910687&view=auto
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.cc (added)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.cc Tue Feb 16 20:54:21 2010
@@ -0,0 +1,99 @@
+/** @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.
+*/
+#include "UrlMappingPathIndex.h"
+
+bool
+UrlMappingPathIndex::Insert(url_mapping *mapping)
+{
+  URLType url_type;
+  int port = (mapping->fromURL).port_get();
+  UrlMappingTrie *trie;
+  int from_path_len;
+  const char *from_path;
+
+  trie = _GetTrie(&(mapping->fromURL), url_type, port);
+  
+  if (!trie) {
+    trie = new UrlMappingTrie();
+    m_tries.insert(UrlMappingGroup::value_type(UrlMappingTrieKey(url_type, port), trie));
+    Debug("UrlMappingPathIndex::Insert", "Created new trie for url type, port combo <%d, %d>", url_type, port);
+  }
+  
+  from_path = mapping->fromURL.path_get(&from_path_len);
+  if (!trie->Insert(from_path, mapping, mapping->getRank(), from_path_len)) {
+    Error("Couldn't insert into trie!");
+    return false;
+  }
+  Debug("UrlMappingPathIndex::Insert", "Inserted new element!");
+  return true;
+}
+
+url_mapping *
+UrlMappingPathIndex::Search(URL *request_url, int request_port) const
+{
+  url_mapping **retval = 0;
+  URLType url_type;
+  UrlMappingTrie *trie;
+  int path_len;
+  const char *path;
+
+  trie = _GetTrie(request_url, url_type, request_port);
+  
+  if (!trie) {
+    Debug("UrlMappingPathIndex::Search", "No mappings exist for url type, port combo <%d, %d>",
+          url_type, request_port);
+    goto lFail;
+  }
+
+  path = request_url->path_get(&path_len);
+  if (!trie->Search(path, retval, path_len)) {
+    Debug("UrlMappingPathIndex::Search", "Couldn't find entry for url with path [%.*s]", path_len, path);
+    goto lFail;
+  }
+  return *retval;
+  
+lFail:
+  return 0;
+}
+
+void
+UrlMappingPathIndex::GetMappings(MappingList &mapping_list) const
+{
+  for (UrlMappingGroup::const_iterator group_iter = m_tries.begin(); 
+       group_iter != m_tries.end(); ++group_iter) {
+    const UrlMappingTrie::ValuePointerList &value_pointers = group_iter->second->GetValues();
+    for (UrlMappingTrie::ValuePointerList::const_iterator list_iter = value_pointers.begin();
+         list_iter != value_pointers.end(); ++list_iter) {
+      mapping_list.push_back(*(*list_iter));
+    }
+  }
+}
+
+void
+UrlMappingPathIndex::Clear()
+{
+  for (UrlMappingGroup::iterator group_iter = m_tries.begin(); group_iter != m_tries.end(); ++group_iter) {
+    delete group_iter->second;
+  }
+  m_tries.clear();
+}

Added: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.h?rev=910687&view=auto
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.h (added)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlMappingPathIndex.h Tue Feb 16 20:54:21 2010
@@ -0,0 +1,91 @@
+/** @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.
+*/
+#ifndef _URL_MAPPING_PATH_INDEX_H
+
+#define _URL_MAPPING_PATH_INDEX_H
+
+#include <list>
+#include <map>
+
+#include "URL.h"
+#include "UrlMapping.h"
+#include "Trie.h"
+
+class UrlMappingPathIndex
+{
+public:
+  UrlMappingPathIndex() { };
+
+  bool Insert(url_mapping *mapping);
+
+  url_mapping *Search(URL *request_url, int request_port) const;
+
+  typedef std::list<url_mapping *> MappingList;
+  
+  void GetMappings(MappingList &mapping_list) const;
+
+  void Clear();
+
+  virtual ~UrlMappingPathIndex() 
+  {
+    Clear();
+  }
+
+
+private:
+  typedef Trie<url_mapping *> UrlMappingTrie;
+  struct UrlMappingTrieKey 
+  {
+    URLType url_type;
+    int port;
+    UrlMappingTrieKey(URLType type, int p) : url_type(type), port(p) { };
+    bool operator <(const UrlMappingTrieKey &rhs) const 
+    {
+      if (url_type == rhs.url_type) {
+        return (port < rhs.port);
+      }
+      return (url_type < rhs.url_type);
+    };
+  };
+  
+  typedef std::map<UrlMappingTrieKey, UrlMappingTrie *> UrlMappingGroup;
+  UrlMappingGroup m_tries;
+    
+  // make copy-constructor and assignment operator private
+  // till we properly implement them
+  UrlMappingPathIndex(const UrlMappingPathIndex &rhs) { };
+  UrlMappingPathIndex &operator =(const UrlMappingPathIndex &rhs) { return *this; }
+
+  inline UrlMappingTrie *_GetTrie(URL *url, URLType &url_type, int port) const
+  {
+    url_type = static_cast<URLType>(url->type_get());
+    UrlMappingGroup::const_iterator group_iter = m_tries.find(UrlMappingTrieKey(url_type, port));
+    if (group_iter != m_tries.end()) {
+      return group_iter->second;
+    }
+    return 0;
+  };
+  
+};
+
+#endif // _URL_MAPPING_PATH_INDEX_H

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.cc?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.cc (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.cc Tue Feb 16 20:54:21 2010
@@ -32,6 +32,7 @@
 #include "MatcherUtils.h"
 #include "Tokenizer.h"
 #include "RemapAPI.h"
+#include "UrlMappingPathIndex.h"
 
 #include "ink_string.h"
 
@@ -648,16 +649,20 @@
 {
   InkHashTableEntry *ht_entry;
   InkHashTableIteratorState ht_iter;
-  url_mapping *value;
-  url_mapping *next;
+  UrlMappingPathIndex *item;
+  UrlMappingPathIndex::MappingList mappings;
 
   if (h_table != NULL) {        // Iterate over the hash tabel freeing up the all the url_mappings
     //   contained with in
     for (ht_entry = ink_hash_table_iterator_first(h_table, &ht_iter); ht_entry != NULL;) {
-      for (value = (url_mapping *) ink_hash_table_entry_value(h_table, ht_entry); value; value = next) {
-        next = value->next;
-        delete value;
+      item = (UrlMappingPathIndex *) ink_hash_table_entry_value(h_table, ht_entry);
+      item->GetMappings(mappings);
+      for (UrlMappingPathIndex::MappingList::iterator mapping_iter = mappings.begin(); 
+           mapping_iter != mappings.end(); ++mapping_iter) {
+        delete *mapping_iter;
       }
+      mappings.clear();
+      delete item;
       ht_entry = ink_hash_table_iterator_next(h_table, &ht_iter);
     }
     ink_hash_table_destroy(h_table);
@@ -699,18 +704,25 @@
 {
   InkHashTableEntry *ht_entry;
   InkHashTableIteratorState ht_iter;
-  url_mapping *value;
+  UrlMappingPathIndex *value;
   char from_url_buf[2048], to_url_buf[2048];
+  UrlMappingPathIndex::MappingList mappings;
+  url_mapping *um;
 
   for (ht_entry = ink_hash_table_iterator_first(h_table, &ht_iter); ht_entry != NULL;) {
-    for (value = (url_mapping *) ink_hash_table_entry_value(h_table, ht_entry); value; value = value->next) {
-      value->fromURL.string_get_buf(from_url_buf, (int) sizeof(from_url_buf));
-      value->toURL.string_get_buf(to_url_buf, (int) sizeof(to_url_buf));
+    value = (UrlMappingPathIndex *) ink_hash_table_entry_value(h_table, ht_entry); 
+    value->GetMappings(mappings);
+    for (UrlMappingPathIndex::MappingList::iterator mapping_iter = mappings.begin();
+         mapping_iter != mappings.end(); ++mapping_iter) {
+      um = *mapping_iter;
+      um->fromURL.string_get_buf(from_url_buf, (int) sizeof(from_url_buf));
+      um->toURL.string_get_buf(to_url_buf, (int) sizeof(to_url_buf));
       printf("\t %s %s=> %s %s <%s> [plugins %s enabled; running with %d plugins]\n", from_url_buf,
-             value->unique ? "(unique)" : "", to_url_buf,
-             value->homePageRedirect ? "(R)" : "", value->tag ? value->tag : "",
-             value->_plugin_count > 0 ? "are" : "not", value->_plugin_count);
+             um->unique ? "(unique)" : "", to_url_buf,
+             um->homePageRedirect ? "(R)" : "", um->tag ? um->tag : "",
+             um->_plugin_count > 0 ? "are" : "not", um->_plugin_count);
     }
+    mappings.clear();
     ht_entry = ink_hash_table_iterator_next(h_table, &ht_iter);
   }
 }
@@ -724,12 +736,10 @@
 UrlRewrite::_tableLookup(InkHashTable * h_table, URL * request_url,
                         int request_port, const char *request_host, int request_host_len, char *tag)
 {
-  url_mapping *ht_entry, *um;
-  ums_helper *lh;
+  UrlMappingPathIndex *ht_entry;
+  url_mapping *um = NULL;
   char host_lower_buf[1024], *request_host_lower, *xfree_buf_ptr;
-  const char *from_path, *request_path, *request_url_scheme;
-  URL *map_from;
-  int ht_result, tmp, from_path_len, request_path_len;
+  int ht_result, tmp;
 
   if (unlikely(!request_host || !request_url || request_host_len < 0))
     return NULL;
@@ -758,58 +768,9 @@
     xfree(xfree_buf_ptr);
 
   if (likely(ht_result && ht_entry)) {
-    request_path = request_url->path_get(&request_path_len);    // do it only once
-    request_url_scheme = request_url->scheme_get(&tmp);
-
-    for (um = ht_entry; um; um = um->next_root_schema) {
-      if (um->fromURL.scheme_get(&tmp) == request_url_scheme ||
-          (request_url_scheme == URL_SCHEME_HTTPS && um->wildcard_from_scheme))
-        break;
-    }
-    if (unlikely((ht_entry = um) == NULL))
-      return NULL;              /* we don't have such scheme in our list */
-
-    if (likely((lh = ht_entry->lookup_helper) != NULL)) {
-      if (unlikely(!request_path))      // extreme case - return first best match from empty list or NULL
-      {
-        return lh->lookup_best_empty(request_host, request_port, tag);
-      }
-      // request_path != NULL
-      return lh->lookup_best_notempty(ht_entry, request_host, request_port, request_path, request_path_len, tag);
-    }
-    // backup search method - old search without lookup_helper
-    // Search through the chain of remappings for this hostname
-    //   until we find one that matches.  We take the first matching
-    //   remap because the chain is stored in the order of the configuration
-    //   file and earlier entries in the file take precedence over later
-    //   entries
-    for (; ht_entry; ht_entry = ht_entry->next) { 
-      // If the incoming request has no tag but the entry does, or both
-      // have tags that do not match, then we do NOT have a match.
-      bool tags_match = (ht_entry->tag && (!tag || strcmp(tag, ht_entry->tag))) ? false : true;
-
-      map_from = &ht_entry->fromURL;
-      if (tags_match &&
-          (request_url_scheme == map_from->scheme_get(&tmp) ||
-           (request_url_scheme == URL_SCHEME_HTTPS && ht_entry->wildcard_from_scheme)) &&
-          // If the request had no determinable host do not check
-          //  the port since the map_from URL will not have a port
-          //  number other than the default port
-          (*request_host == '\0' || request_port == map_from->port_get())) {    // Port and scheme match so check the path
-        from_path = map_from->path_get(&from_path_len);
-
-        if (ht_entry->unique) {
-          if ((!from_path && !request_path) || (from_path && request_path &&
-                                                from_path_len == request_path_len &&
-                                                !strncmp(from_path, request_path, from_path_len)))
-            return ht_entry;
-        } else if (!from_path || (request_path && request_path_len >= from_path_len &&
-                                  !strncmp(from_path, request_path, from_path_len)))
-          return ht_entry;
-      }
-    }                           /* end for */
+    um = ht_entry->Search(request_url, request_port);
   }
-  return NULL;
+  return um;
 }
 
 /**
@@ -885,8 +846,10 @@
   if (!plugin_modified_host)
     request_url->host_set(toHost, toHostLen);
 
-  if (!plugin_modified_port && (requestPort != map_to->port_get()))
+  if (!plugin_modified_port && 
+      ((requestPort != map_to->port_get()) || map_to->port_get_raw())) {
     request_url->port_set(map_to->port_get_raw());
+  }
 
   // Extra byte is potentially needed for prefix path '/'.
   // Added an extra 3 so that TS wouldn't crash in the field.
@@ -2131,97 +2094,10 @@
 
   xfree(file_buf);
 
-  CreateLookupHelper(forward_mappings.hash_lookup);
-  CreateLookupHelper(reverse_mappings.hash_lookup);
-  CreateLookupHelper(permanent_redirects.hash_lookup);
-  CreateLookupHelper(temporary_redirects.hash_lookup);
-
   return 0;
 }
 
 /**
-  Create lookup helper info inside first url_mapping in hash list.
-  Since remap pattern can be different for each host we should adjust
-  lookup parameters for each hash hit list.
-
-*/
-void
-UrlRewrite::CreateLookupHelper(InkHashTable * h_table)
-{
-  InkHashTableEntry *ht_entry;
-  InkHashTableIteratorState ht_iter;
-  url_mapping *um_root, *um, *ul, **uppe, **uppu;
-  ums_helper *lh;
-  const char *from_path;
-  int from_path_len, i;
-
-  if (h_table) {
-    for (ht_entry = ink_hash_table_iterator_first(h_table, &ht_iter); ht_entry != NULL;) {
-      if (likely((um_root = (url_mapping *) ink_hash_table_entry_value(h_table, ht_entry)) != NULL)) {
-        for (um = um_root->next; um; um = um->next) {
-          from_path = um->fromURL.scheme_get(&i);
-          for (ul = um_root; ul; ul = ul->next_root_schema) {
-            if (ul->fromURL.scheme_get(&from_path_len) == from_path && i == from_path_len)
-              break;
-          }
-          if (likely(ul)) {
-            for (uppu = &ul->next_schema; *uppu; uppu = &((*uppu)->next_schema));
-          } else {
-            for (uppu = &um_root->next_root_schema; *uppu; uppu = &((*uppu)->next_root_schema));
-          }
-          *uppu = um;
-        }
-        // create lookup helper for each scheme
-        for (um = um_root; um; um = um->next_root_schema) {
-          if (unlikely(um->lookup_helper)) {
-            delete um->lookup_helper;
-            um->lookup_helper = NULL;
-          }
-          um->lookup_helper = (lh = NEW(new ums_helper()));
-          lh->min_path_size = 1024 * 256;
-
-          uppe = &lh->empty_list;
-          uppu = &lh->unique_list;
-
-          for (ul = um; ul; ul = ul->next_schema) {
-            // #1. check unique flag
-            if (ul->unique) {
-              *uppu = ul;
-              uppu = &ul->next_unique;
-            }
-            // #2. check min & max path size
-            if (NULL == (from_path = ul->fromURL.path_get(&from_path_len))) {
-              *uppe = ul;
-              uppe = &ul->next_empty;
-              from_path_len = 0;
-            }
-            if (lh->max_path_size < from_path_len)
-              lh->max_path_size = from_path_len;
-            if (lh->min_path_size > from_path_len) {
-              lh->min_path_size = from_path_len;
-            }
-            if (unlikely(lh->min_path_size > lh->max_path_size))
-              lh->min_path_size = lh->max_path_size;
-            if (ul->tag)
-              lh->tag_present = true;
-            lh->map_cnt++;
-          }
-          // create hash table only if empty and unique lists are empty
-          if (!lh->empty_list && !lh->unique_list && lh->min_path_size > 0 && lh->map_cnt > 1) {
-            if (lh->init_hash_table() && lh->load_hash_table(um) && lh->hash_table->max_hit_level > 3) {
-              lh->delete_hash_table();
-              lh->init_hash_table(lh->map_cnt << 1);
-              lh->load_hash_table(um);
-            }
-          }
-        }
-      }
-      ht_entry = ink_hash_table_iterator_next(h_table, &ht_iter);
-    }
-  }
-}
-
-/**
   Inserts arg mapping in h_table with key src_host chaining the mapping
   of existing entries bound to src_host if necessary.
 
@@ -2230,22 +2106,27 @@
 UrlRewrite::TableInsert(InkHashTable * h_table, url_mapping * mapping, char *src_host)
 {
   char src_host_tmp_buf[1];
-  url_mapping *ht_contents;
+  UrlMappingPathIndex *ht_contents;
 
   if (!src_host) {
     src_host = &src_host_tmp_buf[0];
     src_host_tmp_buf[0] = 0;
   }
   // Insert the new_mapping into hash table
-  if (ink_hash_table_lookup(h_table, src_host, (void **) &ht_contents)) {       // There is already a mapping so chain the entries
-    //   Be sure to preserve the order of the file so that we can resolve
-    //    conflicting directives
-    if (ht_contents != NULL) {
-      for (; ht_contents->next; ht_contents = ht_contents->next);
-      (ht_contents->next = mapping)->next = NULL;
+  if (ink_hash_table_lookup(h_table, src_host, (void**) &ht_contents)) {       
+    // There is already a path index for this host
+    if (ht_contents == NULL) {
+      // why should this happen?
+      Error("Found entry cannot be null!");
+      return;
     }
   } else {
-    ink_hash_table_insert(h_table, src_host, mapping);
+    ht_contents = new UrlMappingPathIndex();
+    ink_hash_table_insert(h_table, src_host, ht_contents);
+  }
+  if (!ht_contents->Insert(mapping)) {
+    Error("Could not insert new mapping");
+    // @todo - should we delete these now?
   }
 }
 

Modified: incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.h?rev=910687&r1=910686&r2=910687&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.h (original)
+++ incubator/trafficserver/traffic/trunk/proxy/http2/remap/UrlRewrite.h Tue Feb 16 20:54:21 2010
@@ -26,7 +26,6 @@
 
 #include "StringHash.h"
 #include "UrlMapping.h"
-#include "UmsHelper.h"
 
 #include <pcre.h>
 #include <list>
@@ -79,7 +78,6 @@
   UrlRewrite(char *file_var_in);
    ~UrlRewrite();
   int BuildTable();
-  void CreateLookupHelper(InkHashTable * h_table);
   bool Remap(HttpTransact::State * s, HTTPHdr * request_header, char **redirect_url, char **orig_url,
              char *tag = NULL, unsigned int filter_mask = URL_REMAP_FILTER_NONE);
   mapping_type Remap_redirect(HTTPHdr * request_header, char **redirect_url, char **orig_url, char *tag = NULL);