You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by am...@apache.org on 2011/07/28 17:41:52 UTC

svn commit: r1151896 - in /trafficserver/traffic/trunk: iocore/net/ lib/ts/ mgmt/preparse/ proxy/ proxy/http/ proxy/http/remap/

Author: amc
Date: Thu Jul 28 15:41:49 2011
New Revision: 1151896

URL: http://svn.apache.org/viewvc?rev=1151896&view=rev
Log:
Remove IPRange class, replace with faster, IPv6 compatible IpMap.

Added:
    trafficserver/traffic/trunk/lib/ts/IpMapConf.cc
    trafficserver/traffic/trunk/lib/ts/IpMapConf.h
Modified:
    trafficserver/traffic/trunk/iocore/net/P_Socks.h
    trafficserver/traffic/trunk/iocore/net/Socks.cc
    trafficserver/traffic/trunk/iocore/net/UnixNetProcessor.cc
    trafficserver/traffic/trunk/lib/ts/IpMap.cc
    trafficserver/traffic/trunk/lib/ts/IpMap.h
    trafficserver/traffic/trunk/lib/ts/Makefile.am
    trafficserver/traffic/trunk/lib/ts/MatcherUtils.cc
    trafficserver/traffic/trunk/lib/ts/MatcherUtils.h
    trafficserver/traffic/trunk/lib/ts/ink_inet.cc
    trafficserver/traffic/trunk/lib/ts/ink_inet.h
    trafficserver/traffic/trunk/mgmt/preparse/Makefile.am
    trafficserver/traffic/trunk/proxy/ControlBase.cc
    trafficserver/traffic/trunk/proxy/ControlMatcher.cc
    trafficserver/traffic/trunk/proxy/IPAllow.cc
    trafficserver/traffic/trunk/proxy/IPAllow.h
    trafficserver/traffic/trunk/proxy/InkAPI.cc
    trafficserver/traffic/trunk/proxy/Prefetch.cc
    trafficserver/traffic/trunk/proxy/Prefetch.h
    trafficserver/traffic/trunk/proxy/http/HttpAccept.h
    trafficserver/traffic/trunk/proxy/http/remap/AclFiltering.h
    trafficserver/traffic/trunk/proxy/http/remap/UrlRewrite.cc

Modified: trafficserver/traffic/trunk/iocore/net/P_Socks.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/net/P_Socks.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/net/P_Socks.h (original)
+++ trafficserver/traffic/trunk/iocore/net/P_Socks.h Thu Jul 28 15:41:49 2011
@@ -28,12 +28,11 @@
 
 #ifdef SOCKS_WITH_TS
 #include "ParentSelection.h"
-#include "IPRange.h"
+#include <ts/IpMap.h>
 #endif
 
 enum
 {
-
   //types of events for Socks auth handlers
   SOCKS_AUTH_OPEN,
   SOCKS_AUTH_WRITE_COMPLETE,
@@ -59,7 +58,7 @@ struct socks_conf_struct
   unsigned short http_port;
 
 #ifdef SOCKS_WITH_TS
-  IPRange ip_range;
+  IpMap ip_map;
 #endif
 
 #ifndef SOCKS_WITH_TS

Modified: trafficserver/traffic/trunk/iocore/net/Socks.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/net/Socks.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/net/Socks.cc (original)
+++ trafficserver/traffic/trunk/iocore/net/Socks.cc Thu Jul 28 15:41:49 2011
@@ -32,6 +32,7 @@
 
 #include "P_Net.h"
 #include "I_Layout.h"
+#include <ts/IpMapConf.h>
 
 socks_conf_struct *g_socks_conf_stuff = 0;
 
@@ -495,7 +496,12 @@ loadSocksConfiguration(socks_conf_struct
     goto error;
   }
 #ifdef SOCKS_WITH_TS
-  tmp = socks_conf_stuff->ip_range.read_table_from_file(socks_config_fd, "no_socks");
+  tmp = Load_IpMap_From_File(
+    &socks_conf_stuff->ip_map,
+    socks_config_fd,
+    "no_socks"
+  );
+//  tmp = socks_conf_stuff->ip_range.read_table_from_file(socks_config_fd, "no_socks");
 
   if (tmp) {
     Error("SOCKS Config: Error while reading ip_range: %s.", tmp);

Modified: trafficserver/traffic/trunk/iocore/net/UnixNetProcessor.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/net/UnixNetProcessor.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/net/UnixNetProcessor.cc (original)
+++ trafficserver/traffic/trunk/iocore/net/UnixNetProcessor.cc Thu Jul 28 15:41:49 2011
@@ -250,7 +250,7 @@ UnixNetProcessor::connect_re_internal(Co
                            * we need to connect using socks server even
                            * if this ip is in no_socks list.
                            */
-                          !socks_conf_stuff->ip_range.match(ip))
+                          !socks_conf_stuff->ip_map.contains(ip))
 #endif
     );
   SocksEntry *socksEntry = NULL;

Modified: trafficserver/traffic/trunk/lib/ts/IpMap.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/IpMap.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/IpMap.cc (original)
+++ trafficserver/traffic/trunk/lib/ts/IpMap.cc Thu Jul 28 15:41:49 2011
@@ -744,7 +744,7 @@ IpMapBase<N>::print() {
 }
 
 //----------------------------------------------------------------------------
-typedef Interval<uint32_t, uint32_t> Ip4Span;
+typedef Interval<in_addr_t, in_addr_t> Ip4Span;
 
 /** Node for IPv4 map.
     We store the address in host order in the @a _min and @a _max
@@ -1028,7 +1028,7 @@ class Ip6Map : public IpMapBase<Ip6Node>
 //----------------------------------------------------------------------------
 namespace {
   ///< @return The network order IPv4 address in @a target.
-  inline uint32_t const& ip4_addr(sockaddr const* target) {
+  inline in_addr_t const& ip4_addr(sockaddr const* target) {
     return ink_inet_ip4_cast(target)->sin_addr.s_addr;
   }
 }
@@ -1058,6 +1058,11 @@ IpMap::contains(sockaddr const* target, 
   return zret;
 }
 
+bool
+IpMap::contains(in_addr_t target, void** ptr) {
+  return _m4->contains(ntohl(target));
+}
+
 IpMap&
 IpMap::mark(
   sockaddr const* min,
@@ -1075,7 +1080,7 @@ IpMap::mark(
 }
 
 IpMap&
-IpMap::mark(uint32_t min, uint32_t max, void* data) {
+IpMap::mark(in_addr_t min, in_addr_t max, void* data) {
   this->force4()->mark(min, max, data);
   return *this;
 }

Modified: trafficserver/traffic/trunk/lib/ts/IpMap.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/IpMap.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/IpMap.h (original)
+++ trafficserver/traffic/trunk/lib/ts/IpMap.h Thu Jul 28 15:41:49 2011
@@ -346,8 +346,39 @@ public:
       @return This object.
   */
   self& mark(
-    uint32_t min, ///< Minimum address (network order).
-    uint32_t max, ///< Maximum address (network order).
+    in_addr_t min, ///< Minimum address (network order).
+    in_addr_t max, ///< Maximum address (network order).
+    void* data = 0 ///< Client data.
+  );
+
+  /** Mark an IPv4 address @a addr with @a data.
+      This is equivalent to calling @c mark(addr, addr, data).
+      @note Convenience overload for IPv4 addresses.
+      @return This object.
+  */
+  self& mark(
+    in_addr_t addr, ///< Address (network order).
+    void* data = 0 ///< Client data.
+  );
+
+  /** Mark a range.
+      All addresses in the range [ @a min , @a max ] are marked with @a data.
+      @note Convenience overload for IPv6 addresses.
+      @return This object.
+  */
+  self& mark(
+    sockaddr_in6 const* min, ///< Minimum address (network order).
+    sockaddr_in6 const* max, ///< Maximum address (network order).
+    void* data = 0 ///< Client data.
+  );
+
+  /** Mark an IPv6 address @a addr with @a data.
+      This is equivalent to calling @c mark(addr, addr, data).
+      @note Convenience overload for IPv6 addresses.
+      @return This object.
+  */
+  self& mark(
+    sockaddr_in6 const* addr, ///< Address (network order).
     void* data = 0 ///< Client data.
   );
 
@@ -370,7 +401,20 @@ public:
       is set to the client data for the address.
   */
   bool contains(
-    sockaddr const* target, ///< Search target value.
+    sockaddr const* target, ///< Search target (network order).
+    void **ptr = 0 ///< Client data return.
+  );
+
+  /** Test for membership.
+
+      @note Covenience overload for IPv4.
+
+      @return @c true if the address is in the map, @c false if not.
+      If the address is in the map and @a ptr is not @c NULL, @c *ptr
+      is set to the client data for the address.
+  */
+  bool contains(
+    in_addr_t target, ///< Search target (network order).
     void **ptr = 0 ///< Client data return.
   );
 
@@ -398,6 +442,18 @@ protected:
   
 };
 
+inline IpMap& IpMap::mark(in_addr_t addr, void* data) {
+  return this->mark(addr, addr, data);
+}
+
+inline IpMap& IpMap::mark(sockaddr_in6 const* addr, void* data) {
+  return this->mark(ink_inet_sa_cast(addr), ink_inet_sa_cast(addr), data);
+}
+
+inline IpMap& IpMap::mark(sockaddr_in6 const* min, sockaddr_in6 const* max, void* data) {
+  return this->mark(ink_inet_sa_cast(min), ink_inet_sa_cast(max), data);
+}
+
 inline IpMap::iterator
 IpMap::end() {
   return iterator(this, 0);

Added: trafficserver/traffic/trunk/lib/ts/IpMapConf.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/IpMapConf.cc?rev=1151896&view=auto
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/IpMapConf.cc (added)
+++ trafficserver/traffic/trunk/lib/ts/IpMapConf.cc Thu Jul 28 15:41:49 2011
@@ -0,0 +1,161 @@
+/** @file
+
+  Loading @c IpMap from a configuration file.
+
+  @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.
+ */
+
+// Copied from IPRange.cc for backwards compatibility.
+
+# include <ts/IpMap.h>
+# include <ts/IpMapConf.h>
+# include <ts/ink_memory.h>
+
+static size_t const ERR_STRING_LEN = 256;
+static size_t const MAX_LINE_SIZE = 2048;
+
+// Returns 0 if successful, 1 if failed
+int
+read_addr(
+  char *line, ///< Input text (source line).
+  int n, ///< Amount of data in @a line.
+  int *i, ///< [in,out] Offset in line.
+  sockaddr* addr, ///< [out] Destination for address.
+  char* err ///< Buffer for error string (must be ERR_STRING_LEN big).
+) {
+  int k;
+  char dst[INET6_ADDRSTRLEN];
+  char* src = line + *i;
+
+  for ( k = 0
+      ; k < INET6_ADDRSTRLEN
+        && *i < n
+        && (isxdigit(*src) || '.' == *src || ':' == *src)
+      ; ++k, ++*i, ++src
+  ) {
+    dst[k] = *src;
+  }
+
+  if (k == sizeof(dst)) {
+    snprintf(err, ERR_STRING_LEN, "IP address too long");
+    return EINVAL;
+  }
+
+  dst[k] = '\0';
+  if (0 != ink_inet_pton(dst, addr)) {
+    snprintf(err, ERR_STRING_LEN,
+      "IP address '%s' improperly formatted", dst
+    );
+    return EINVAL;
+  }
+  return 0;
+}
+
+char *
+Load_IpMap_From_File(IpMap* map, int fd, const char *key_str) {
+  char* zret = 0;
+  FILE* f = fdopen(dup(fd), "r"); // dup so we don't close the original fd.
+  if (f) zret = Load_IpMap_From_File(map, f, key_str);
+  else {
+    zret = (char *) ink_malloc(ERR_STRING_LEN);
+    snprintf(zret, ERR_STRING_LEN, "Unable to reopen file descriptor as stream %d:%s", errno, strerror(errno));
+  }
+  return zret;
+}
+
+// Skip space in line, returning true if more data is available
+// (not end of line).
+static inline bool skip_space(
+  char* line, ///< Source line.
+  int n, ///< Line length.
+  int& offset ///< Current offset
+) {
+  while (offset < n && isspace(line[offset])) ++offset;
+  return offset < n;
+}
+
+// Returns 0 if successful, error string otherwise
+char *
+Load_IpMap_From_File(IpMap* map, FILE* f, const char *key_str)
+{
+  int i, n, line_no;
+  int key_len = strlen(key_str);
+  sockaddr_in6 laddr, raddr;
+  char line[MAX_LINE_SIZE];
+  char err_buff[ERR_STRING_LEN];
+
+  // First hardcode 127.0.0.1 into the table
+  map->mark(INADDR_LOOPBACK);
+
+  line_no = 0;
+  while (fgets(line, MAX_LINE_SIZE, f)) {
+    ++line_no;
+    n = strlen(line);
+    // Find first white space which terminates the line key.
+    for ( i = 0 ; i < n && ! isspace(line[i]); ++i )
+      ;
+    if (i != key_len || 0 != strncmp(line, key_str, key_len))
+      continue;
+    // Now look for IP address
+    while (true) {
+      if (!skip_space(line, n, i)) break;
+
+      if (0 != read_addr(line, n,  &i, ink_inet_sa_cast(&laddr), err_buff)) {
+        char *error_str = (char *) ink_malloc(ERR_STRING_LEN);
+        snprintf(error_str, ERR_STRING_LEN, "Invalid input configuration (%s) at line %d offset %d - '%s'", err_buff, line_no, i, line);
+        return error_str;
+      }
+
+      if (!skip_space(line, n, i) || line[i] == ',') {
+        // You have read an IP address. Enter it in the table
+        map->mark(&laddr);
+        if (i == n)
+          break;
+        else
+          ++i;
+      } else if (line[i] == '-') {
+        // What you have just read is the start of the range,
+        // Now, read the end of the IP range
+        ++i;
+        if (!skip_space(line, n, i)) {
+          char *error_str = (char *) ink_malloc(ERR_STRING_LEN);
+          snprintf(error_str, ERR_STRING_LEN, "Invalid input (unterminated range) at line %d offset %d - '%s'", line_no, i, line);
+          return error_str;
+        } else if (0 != read_addr(line, n, &i, ink_inet_sa_cast(&raddr), err_buff)) {
+          char *error_str = (char *) ink_malloc(ERR_STRING_LEN);
+          snprintf(error_str, ERR_STRING_LEN, "Invalid input (%s) at line %d offset %d - '%s'", err_buff, line_no, i, line);
+          return error_str;
+        }
+        map->mark(ink_inet_sa_cast(&laddr), ink_inet_sa_cast(&raddr));
+        if (!skip_space(line, n, i)) break;
+        if (line[i] != ',') {
+          char *error_str = (char *) ink_malloc(ERR_STRING_LEN);
+          snprintf(error_str, ERR_STRING_LEN, "Invalid input (expecting comma) at line %d offset %d - '%s'", line_no, i, line);
+          return error_str;
+        }
+        ++i;
+      } else {
+        char *error_str = (char *) ink_malloc(ERR_STRING_LEN);
+        snprintf(error_str, ERR_STRING_LEN, "Invalid input (expecting dash or comma) at line %d offset %d", line_no, i);
+        return error_str;
+      }
+    }
+  }
+  return 0;
+}

Added: trafficserver/traffic/trunk/lib/ts/IpMapConf.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/IpMapConf.h?rev=1151896&view=auto
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/IpMapConf.h (added)
+++ trafficserver/traffic/trunk/lib/ts/IpMapConf.h Thu Jul 28 15:41:49 2011
@@ -0,0 +1,31 @@
+/** @file
+
+  Loading @c IpMap from a configuration file.
+
+  @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.
+ */
+
+// Copied from IPRange.cc for backwards compatibility.
+
+class IpMap; // declare in name only.
+
+// Returns 0 if successful, error string otherwise
+char* Load_IpMap_From_File(IpMap* map, int fd, char const* key_str);
+// Returns 0 if successful, error string otherwise
+char* Load_IpMap_From_File(IpMap* map, FILE* f, char const* key_str);

Modified: trafficserver/traffic/trunk/lib/ts/Makefile.am
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/Makefile.am?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/Makefile.am (original)
+++ trafficserver/traffic/trunk/lib/ts/Makefile.am Thu Jul 28 15:41:49 2011
@@ -127,7 +127,7 @@ libtsutil_la_SOURCES = \
   lockfile.cc \
   I_Layout.h \
   IntrusiveDList.h \
-  IpMap.h IpMap.cc \
+  IpMap.h IpMap.cc IpMapConf.h IpMapConf.cc \
   Layout.cc \
   MatcherUtils.cc \
   MatcherUtils.h \

Modified: trafficserver/traffic/trunk/lib/ts/MatcherUtils.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/MatcherUtils.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/MatcherUtils.cc (original)
+++ trafficserver/traffic/trunk/lib/ts/MatcherUtils.cc Thu Jul 28 15:41:49 2011
@@ -139,8 +139,23 @@ unescapifyStr(char *buffer)
   return (write - buffer);
 }
 
-//   char* ExtractIpRange(char* match_str, ip_addr_t* addr1,
-//                         ip_addr_t* addr2)
+char const*
+ExtractIpRange(char* match_str, in_addr_t* min, in_addr_t* max) {
+  sockaddr_in6 min6, max6;
+  char const* zret = ExtractIpRange(match_str, ink_inet_sa_cast(&min6), ink_inet_sa_cast(&max6));
+  if (0 == zret) { // success
+    if (ink_inet_is_ip4(&min6) && ink_inet_is_ip4(&max6)) {
+      if (min) *min = ntohl(ink_inet_ip4_addr_cast(&min6));
+      if (max) *max = ntohl(ink_inet_ip4_addr_cast(&max6));
+    } else {
+      zret = "The addresses were not IPv4 addresses.";
+    }
+  }
+  return zret;
+}
+
+//   char* ExtractIpRange(char* match_str, sockaddr* addr1,
+//                         sockaddr* addr2)
 //
 //   Attempts to extract either an Ip Address or an IP Range
 //     from match_str.  The range should be two addresses
@@ -154,19 +169,15 @@ unescapifyStr(char *buffer)
 //     that describes the reason for the error.
 //
 const char *
-ExtractIpRange(char *match_str, ip_addr_t *addr1, ip_addr_t *addr2)
+ExtractIpRange(char *match_str, sockaddr* addr1, sockaddr* addr2)
 {
   Tokenizer rangeTok("-/");
-  bool mask = false;
+  bool mask = strchr(match_str, '/') != NULL;
   int mask_bits;
   int mask_val;
   int numToks;
-  ip_addr_t addr1_local;
-  ip_addr_t addr2_local;
+  sockaddr_in6 la1, la2;
 
-  if (strchr(match_str, '/') != NULL) {
-    mask = true;
-  }
   // Extract the IP addresses from match data
   numToks = rangeTok.Initialize(match_str, SHARE_TOKS);
 
@@ -176,15 +187,17 @@ ExtractIpRange(char *match_str, ip_addr_
     return "malformed IP range";
   }
 
-  addr1_local = htonl(inet_addr(rangeTok[0]));
-
-  if (addr1_local == (ip_addr_t) - 1 && strcmp(rangeTok[0], "255.255.255.255") != 0) {
-    return "malformed ip address";
+  if (0 != ink_inet_pton(rangeTok[0], &la1)) {
+    return "malformed IP address";
   }
+
   // Handle a IP range
   if (numToks == 2) {
 
-    if (mask == true) {
+    if (mask) {
+      if (!ink_inet_is_ip4(&la1)) {
+        return "Masks supported only for IPv4";
+      }
       // coverity[secure_coding]
       if (sscanf(rangeTok[1], "%d", &mask_bits) != 1) {
         return "bad mask specification";
@@ -197,28 +210,28 @@ ExtractIpRange(char *match_str, ip_addr_
       if (mask_bits == 32) {
         mask_val = 0;
       } else {
-        mask_val = 0xffffffff >> mask_bits;
+        mask_val = htonl(0xffffffff >> mask_bits);
       }
-
-      addr2_local = addr1_local | mask_val;
-      addr1_local = addr1_local & (mask_val ^ 0xffffffff);
+      in_addr_t a = ink_inet_ip4_addr_cast(&la1);
+      ink_inet_ip4_set(&la2, a | mask_val);
+      ink_inet_ip4_set(&la1, a & (mask_val ^ 0xffffffff));
 
     } else {
-      addr2_local = htonl(inet_addr(rangeTok[1]));
-      if (addr2_local == (ip_addr_t) - 1 && strcmp(rangeTok[1], "255.255.255.255") != 0) {
+      if (0 != ink_inet_pton(rangeTok[1], &la2)) {
         return "malformed ip address at range end";
       }
     }
 
-    if (addr1_local > addr2_local) {
+    if (1 == ink_inet_cmp(&la1, &la2)) {
       return "range start greater than range end";
     }
+
+    ink_inet_copy(addr2, ink_inet_sa_cast(&la2));
   } else {
-    addr2_local = addr1_local;
+    ink_inet_copy(addr2, ink_inet_sa_cast(&la1));
   }
 
-  *addr1 = addr1_local;
-  *addr2 = addr2_local;
+  ink_inet_copy(addr1, ink_inet_sa_cast(&la1));
   return NULL;
 }
 

Modified: trafficserver/traffic/trunk/lib/ts/MatcherUtils.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/MatcherUtils.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/MatcherUtils.h (original)
+++ trafficserver/traffic/trunk/lib/ts/MatcherUtils.h Thu Jul 28 15:41:49 2011
@@ -38,8 +38,23 @@ char *readIntoBuffer(char *file_path, co
 
 int unescapifyStr(char *buffer);
 
-typedef unsigned long ip_addr_t;
-const char *ExtractIpRange(char *match_str, ip_addr_t * addr1, ip_addr_t * addr2);
+/** Extract an IP range.
+    @a min and @a max should be at least the size of @c sockaddr_in6 to hold
+    an IP address.
+*/
+char const* ExtractIpRange(
+  char *match_str,
+  sockaddr* min,
+  sockaddr* max
+);
+
+/// Convenience overload for IPv4.
+char const* ExtractIpRange(
+  char *match_str,
+  in_addr_t * addr1, ///< [in,out] Returned address in host order.
+  in_addr_t * addr2 ///< [in,out] Returned address in host order.
+);
+
 char *tokLine(char *buf, char **last);
 
 const char *processDurationString(char *str, int *seconds);

Modified: trafficserver/traffic/trunk/lib/ts/ink_inet.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/ink_inet.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/ink_inet.cc (original)
+++ trafficserver/traffic/trunk/lib/ts/ink_inet.cc Thu Jul 28 15:41:49 2011
@@ -194,3 +194,23 @@ uint16_t ink_inet_port(const struct sock
 
   return port;
 }
+
+int ink_inet_pton(char const* text, sockaddr* addr) {
+  int zret = -1;
+  addrinfo hints; // [out]
+  addrinfo *ai; // [in]
+
+  memset(&hints, 0, sizeof(hints));
+  hints.ai_family = PF_UNSPEC;
+  hints.ai_flags = AI_NUMERICHOST|AI_PASSIVE;
+  if (0 == (zret = getaddrinfo(text, 0, &hints, &ai))) {
+    if (addr) {
+      if (ink_inet_copy(addr, ai->ai_addr))
+        zret = 0;
+    } else if (ink_inet_is_ip(addr)) {
+      zret = 0;
+    }
+    freeaddrinfo(ai);
+  }
+  return zret;
+}

Modified: trafficserver/traffic/trunk/lib/ts/ink_inet.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/lib/ts/ink_inet.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/lib/ts/ink_inet.h (original)
+++ trafficserver/traffic/trunk/lib/ts/ink_inet.h Thu Jul 28 15:41:49 2011
@@ -94,41 +94,6 @@ uint16_t ink_inet_port(const struct sock
 /// Size in bytes of an IPv6 address.
 size_t const INK_IP6_SIZE = sizeof(in6_addr);
 
-/** Write a null terminated string for @a addr to @a dst.
-    A buffer of size INET6_ADDRSTRLEN suffices, including a terminating nul.
- */
-char const* ink_inet_ntop(
-  const sockaddr *addr, ///< Address.
-  char *dst, ///< Output buffer.
-  size_t size ///< Length of buffer.
-);
-
-static size_t const INET6_ADDRPORTSTRLEN = INET6_ADDRSTRLEN + 6;
-
-/** Write a null terminated string for @a addr to @a dst with port.
-    A buffer of size INET6_ADDRPORTSTRLEN suffices, including a terminating nul.
- */
-char const* ink_inet_nptop(
-  const sockaddr *addr, ///< Address.
-  char *dst, ///< Output buffer.
-  size_t size ///< Length of buffer.
-);
-
-/** Convert @a text to an IP address and write it to @a addr.
-
-    @a text is expected to be an explicit address, not a hostname.  No
-    hostname resolution is done.
-
-    @note This uses @c getaddrinfo internally and so involves memory
-    allocation.
-
-    @return 0 on success, non-zero on failure.
-*/
-int ink_inet_pton(
-  char const* text, ///< [in] text.
-  sockaddr* addr ///< [out] address
-);
-
 /// Reset an address to invalid.
 /// @note Useful for marking a member as not yet set.
 inline void ink_inet_invalidate(sockaddr* addr) {
@@ -145,6 +110,12 @@ inline bool ink_inet_is_ip(sockaddr cons
 inline bool ink_inet_is_ip4(sockaddr const* addr) {
   return AF_INET == addr->sa_family;
 }
+/// Test for IPv4 protocol.
+/// @note Convenience overload.
+/// @return @c true if the address is IPv4, @c false otherwise.
+inline bool ink_inet_is_ip4(sockaddr_in6 const* addr) {
+  return AF_INET == addr->sin6_family;
+}
 /// Test for IPv6 protocol.
 /// @return @c true if the address is IPv6, @c false otherwise.
 inline bool ink_inet_is_ip6(sockaddr const* addr) {
@@ -267,6 +238,7 @@ inline in_addr_t& ink_inet_ip4_addr_cast
     : (dummy = 0)
     ;
 }
+
 /** Access the IPv4 address.
 
     If this is not an IPv4 address a zero valued address is returned.
@@ -282,6 +254,33 @@ inline in_addr_t const& ink_inet_ip4_add
     : static_cast<in_addr_t const&>(dummy = 0)
     ;
 }
+
+/** Access the IPv4 address.
+
+    If this is not an IPv4 address a zero valued address is returned.
+    @note This is direct access to the address so it will be in
+    network order.
+    @note Convenience overload.
+
+    @return A reference to the IPv4 address in @a addr.
+*/
+inline in_addr_t& ink_inet_ip4_addr_cast(sockaddr_in6* addr) {
+  return ink_inet_ip4_addr_cast(ink_inet_sa_cast(addr));
+}
+
+/** Access the IPv4 address.
+
+    If this is not an IPv4 address a zero valued address is returned.
+    @note This is direct access to the address so it will be in
+    network order.
+    @note Convenience overload.
+
+    @return A reference to the IPv4 address in @a addr.
+*/
+inline in_addr_t const& ink_inet_ip4_addr_cast(sockaddr_in6 const* addr) {
+  return ink_inet_ip4_addr_cast(ink_inet_sa_cast(addr));
+}
+
 /** Access the IPv6 address.
 
     If this is not an IPv6 address a zero valued address is returned.
@@ -294,7 +293,6 @@ inline in6_addr& ink_inet_ip6_addr_cast(
   return ink_inet_ip6_cast(addr)->sin6_addr;
 }
   
-
 /// @name Address operators
 //@{
 
@@ -379,6 +377,14 @@ inline int ink_inet_cmp(
   return zret;
 }
 
+/** Compare two addresses.
+    @note Convenience overload.
+    @see ink_inet_cmp(sockaddr const* lhs, sockaddr const* rhs)
+*/
+inline int ink_inet_cmp(sockaddr_in6 const* lhs, sockaddr_in6 const* rhs) {
+  return ink_inet_cmp(ink_inet_sa_cast(lhs), ink_inet_sa_cast(rhs));
+}
+
 //@}
 
 /// Get IP TCP/UDP port.
@@ -437,6 +443,59 @@ inline void ink_inet_ip4_set(
   ink_inet_ip4_set(ink_inet_ip4_cast(dst), ip4, port);
 }
 
+/** Write a null terminated string for @a addr to @a dst.
+    A buffer of size INET6_ADDRSTRLEN suffices, including a terminating nul.
+ */
+char const* ink_inet_ntop(
+  const sockaddr *addr, ///< Address.
+  char *dst, ///< Output buffer.
+  size_t size ///< Length of buffer.
+);
+
+static size_t const INET6_ADDRPORTSTRLEN = INET6_ADDRSTRLEN + 6;
+
+/** Write a null terminated string for @a addr to @a dst with port.
+    A buffer of size INET6_ADDRPORTSTRLEN suffices, including a terminating nul.
+ */
+char const* ink_inet_nptop(
+  const sockaddr *addr, ///< Address.
+  char *dst, ///< Output buffer.
+  size_t size ///< Length of buffer.
+);
+
+/** Convert @a text to an IP address and write it to @a addr.
+
+    @a text is expected to be an explicit address, not a hostname.  No
+    hostname resolution is done.
+
+    @note This uses @c getaddrinfo internally and so involves memory
+    allocation.
+
+    @return 0 on success, non-zero on failure.
+*/
+int ink_inet_pton(
+  char const* text, ///< [in] text.
+  sockaddr* addr ///< [out] address
+);
+
+/** Convert @a text to an IP address and write it to @a addr.
+
+    @a text is expected to be an explicit address, not a hostname.  No
+    hostname resolution is done.
+
+    @note This uses @c getaddrinfo internally and so involves memory
+    allocation.
+    @note Convenience overload.
+
+    @return 0 on success, non-zero on failure.
+*/
+inline int ink_inet_pton(
+  char const* text, ///< [in] text.
+  sockaddr_in6* addr ///< [out] address
+) {
+  return ink_inet_pton(text, ink_inet_sa_cast(addr));
+}
+
 /** Storage for an IP address.
     In some cases we want to store just the address and not the
     ancillary information (such as port, or flow data) in

Modified: trafficserver/traffic/trunk/mgmt/preparse/Makefile.am
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/mgmt/preparse/Makefile.am?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/mgmt/preparse/Makefile.am (original)
+++ trafficserver/traffic/trunk/mgmt/preparse/Makefile.am Thu Jul 28 15:41:49 2011
@@ -24,8 +24,6 @@ DEFS += $(MGMT_DEFS)
 noinst_LIBRARIES = libpreparse.a
 
 libpreparse_a_SOURCES = \
-  IPRange.cc \
-  IPRange.h \
   RemapReadConfig.cc \
   SocksParser.cc \
   StoreReadConfig.cc

Modified: trafficserver/traffic/trunk/proxy/ControlBase.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/ControlBase.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/ControlBase.cc (original)
+++ trafficserver/traffic/trunk/proxy/ControlBase.cc Thu Jul 28 15:41:49 2011
@@ -250,8 +250,8 @@ IPortMod::make(char* value, char const *
 // ----------
 struct SrcIPMod : public ControlBase::Modifier {
   // Stored in host order because that's how they are compared.
-  ip_addr_t start_addr; ///< Start address in HOST order.
-  ip_addr_t end_addr; ///< End address in HOST order.
+  in_addr_t start_addr; ///< Start address in HOST order.
+  in_addr_t end_addr; ///< End address in HOST order.
 
   static char const * const NAME;
 

Modified: trafficserver/traffic/trunk/proxy/ControlMatcher.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/ControlMatcher.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/ControlMatcher.cc (original)
+++ trafficserver/traffic/trunk/proxy/ControlMatcher.cc Thu Jul 28 15:41:49 2011
@@ -487,7 +487,7 @@ template<class Data, class Result> char 
   const char *errPtr;
   char *errBuf;
   char *match_data;
-  ip_addr_t addr1, addr2;
+  in_addr_t addr1, addr2;
 
   // Make sure space has been allocated
   ink_assert(num_el >= 0);

Modified: trafficserver/traffic/trunk/proxy/IPAllow.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/IPAllow.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/IPAllow.cc (original)
+++ trafficserver/traffic/trunk/proxy/IPAllow.cc Thu Jul 28 15:41:49 2011
@@ -180,8 +180,8 @@ IpAllow::BuildTable()
   char errBuf[1024];
   char *file_buf = NULL;
   int line_num = 0;
-  ip_addr_t addr1 = 0;
-  ip_addr_t addr2 = 0;
+  in_addr_t addr1 = 0;
+  in_addr_t addr2 = 0;
   matcher_line line_info;
   bool alarmAlready = false;
 

Modified: trafficserver/traffic/trunk/proxy/IPAllow.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/IPAllow.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/IPAllow.h (original)
+++ trafficserver/traffic/trunk/proxy/IPAllow.h Thu Jul 28 15:41:49 2011
@@ -42,14 +42,13 @@ void reloadIPAllow();
 //    a reconfig event happens that the old table gets thrown
 //    away
 //
-#define IP_ALLOW_TIMEOUT            (HRTIME_HOUR*1)
+static uint64_t const IP_ALLOW_TIMEOUT = HRTIME_HOUR;
 
 // INKqa05845
-#define IP_ALLOW 1
-#define IP_DENY -1
-class IpAllowRecord
-{
-public:
+static int const IP_ALLOW = 1;
+static int const IP_DENY = -1;
+
+struct IpAllowRecord {
   int access;
   int line_num;
 };

Modified: trafficserver/traffic/trunk/proxy/InkAPI.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/InkAPI.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/InkAPI.cc (original)
+++ trafficserver/traffic/trunk/proxy/InkAPI.cc Thu Jul 28 15:41:49 2011
@@ -6870,7 +6870,7 @@ char *
 TSMatcherExtractIPRange(char *match_str, uint32_t *addr1, uint32_t *addr2)
 {
   sdk_assert(sdk_sanity_check_null_ptr((void*)match_str) == TS_SUCCESS);
-  return (char*)ExtractIpRange(match_str, (ip_addr_t *) addr1, (ip_addr_t *) addr2);
+  return (char*)ExtractIpRange(match_str, addr1, addr2);
 }
 
 TSMatcherLine

Modified: trafficserver/traffic/trunk/proxy/Prefetch.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/Prefetch.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/Prefetch.cc (original)
+++ trafficserver/traffic/trunk/proxy/Prefetch.cc Thu Jul 28 15:41:49 2011
@@ -27,6 +27,7 @@
 #include "HdrUtils.h"
 #include "HttpCompat.h"
 #include "I_Layout.h"
+#include <ts/IpMapConf.h>
 
 #ifdef PREFETCH
 
@@ -609,7 +610,7 @@ check_n_attach_prefetch_transform(HttpSM
             "since recursion depth(%d) is greater than max allowed (%d)", rec_depth, prefetch_config.max_recursion);
       return;
     }
-  } else if (!prefetch_config.ip_range.match(client_ip)) {
+  } else if (!prefetch_config.ip_map.contains(client_ip)) {
     Debug("PrefetchParser", "client (%u.%u.%u.%u) does not match any of the "
           "prefetch_children mentioned in configuration\n", IPSTRARGS(client_ip));
     return;
@@ -1875,7 +1876,7 @@ PrefetchConfiguration::readConfiguration
   }
 
   char *temp_str;
-  if ((temp_str = ip_range.read_table_from_file(fd, "prefetch_children", FALSE)) != 0) {
+  if ((temp_str = Load_IpMap_From_File(&ip_map, fd, "prefetch_children")) != 0) {
     Error("PrefetchProcessor: Error in reading ip_range from %s: %.256s\n", conf_path, temp_str);
     xfree(temp_str);
     goto Lerror;

Modified: trafficserver/traffic/trunk/proxy/Prefetch.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/Prefetch.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/Prefetch.h (original)
+++ trafficserver/traffic/trunk/proxy/Prefetch.h Thu Jul 28 15:41:49 2011
@@ -24,7 +24,7 @@
 #ifndef _PREFETCH_H_
 #define _PREFETCH_H_
 
-#include "IPRange.h"
+#include <ts/IpMap.h>
 #include "TransformInternal.h"
 
 #ifdef PREFETCH
@@ -54,7 +54,7 @@ extern BlasterUrlList *multicastUrlBlast
 struct PrefetchConfiguration
 {
   int prefetch_enabled;
-  IPRange ip_range;
+  IpMap ip_map;
   struct html_tag *html_tags_table;
   struct html_tag *html_attrs_table;
 

Modified: trafficserver/traffic/trunk/proxy/http/HttpAccept.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/http/HttpAccept.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/http/HttpAccept.h (original)
+++ trafficserver/traffic/trunk/proxy/http/HttpAccept.h Thu Jul 28 15:41:49 2011
@@ -28,8 +28,6 @@
 #include "P_EventSystem.h"
 #include "HttpConfig.h"
 #include "HTTP.h"
-#include "IPRange.h"
-
 
 /**
    The continuation mutex is NULL to allow parellel accepts in NT. The

Modified: trafficserver/traffic/trunk/proxy/http/remap/AclFiltering.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/http/remap/AclFiltering.h?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/http/remap/AclFiltering.h (original)
+++ trafficserver/traffic/trunk/proxy/http/remap/AclFiltering.h Thu Jul 28 15:41:49 2011
@@ -36,8 +36,8 @@
 
 typedef struct src_ip_info_t
 {
-  unsigned long start;          // IPv4 address value stores start of a range (host byte order)
-  unsigned long end;            // IPv4 address value stores end of a range (host byte order)
+  in_addr_t start;          // IPv4 address value stores start of a range (host byte order)
+  in_addr_t end;            // IPv4 address value stores end of a range (host byte order)
   bool invert;                  // Should we "invert" the meaning of this IP range ("not in range")
 
   void reset(void)

Modified: trafficserver/traffic/trunk/proxy/http/remap/UrlRewrite.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/http/remap/UrlRewrite.cc?rev=1151896&r1=1151895&r2=1151896&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/http/remap/UrlRewrite.cc (original)
+++ trafficserver/traffic/trunk/proxy/http/remap/UrlRewrite.cc Thu Jul 28 15:41:49 2011
@@ -313,7 +313,7 @@ validate_filter_args(acl_filter_rule ** 
         ipi->invert = true;
       strncpy(tmpbuf, argptr, sizeof(tmpbuf) - 1);
       tmpbuf[sizeof(tmpbuf) - 1] = 0; // important! use copy of argument
-      if (ExtractIpRange(tmpbuf, (unsigned long*) &ipi->start, &ipi->end) != NULL) {
+      if (ExtractIpRange(tmpbuf, &ipi->start, &ipi->end) != NULL) {
         Debug("url_rewrite", "[validate_filter_args] Unable to parse IP value in %s", argv[i]);
         snprintf(errStrBuf, errStrBufSize, "Unable to parse IP value in %s", argv[i]);
         errStrBuf[errStrBufSize - 1] = 0;