You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2011/06/05 11:08:03 UTC

svn commit: r1132253 [4/5] - in /incubator/mesos/trunk: src/ src/detector/ src/exec/ src/master/ src/messaging/ src/sched/ src/slave/ src/tests/ third_party/libprocess/ third_party/libprocess/third_party/ry-http-parser-1c3624a/

Modified: incubator/mesos/trunk/third_party/libprocess/process.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/process.hpp?rev=1132253&r1=1132252&r2=1132253&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/process.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/process.hpp Sun Jun  5 09:08:02 2011
@@ -1,5 +1,5 @@
-#ifndef PROCESS_HPP
-#define PROCESS_HPP
+#ifndef __PROCESS_HPP__
+#define __PROCESS_HPP__
 
 #include <assert.h>
 #include <stdint.h>
@@ -15,26 +15,21 @@
 #include "pid.hpp"
 
 
-typedef uint16_t MSGID;
-
-
-const MSGID PROCESS_ERROR = 1;
-const MSGID PROCESS_TIMEOUT = 2;
-const MSGID PROCESS_EXIT = 3;
-const MSGID PROCESS_TERMINATE = 4;
-const MSGID PROCESS_DISPATCH = 5;
-const MSGID PROCESS_MSGID = PROCESS_DISPATCH + 1;
+const std::string ERROR = "error";
+const std::string TIMEOUT = "timeout";
+const std::string EXIT = "exit";
+const std::string TERMINATE = "terminate";
+const std::string DISPATCH = "dispatch"; // TODO(benh): Make this internal.
 
 
 class Process;
 
 
-struct msg
-{
+struct Message {
+  std::string name;
   PID from;
   PID to;
-  MSGID id;
-  uint32_t len;
+  std::string body;
 };
 
 
@@ -48,7 +43,7 @@ public:
 
 class Filter {
 public:
-  virtual bool filter(msg *) = 0;
+  virtual bool filter(Message *) = 0;
 };
 
 
@@ -131,7 +126,8 @@ struct Result<void>;
 
 class Process {
 public:
-  Process();
+  Process(const std::string& id = "");
+
   virtual ~Process();
 
   /* Returns pid of process; valid even before calling spawn. */
@@ -142,40 +138,40 @@ protected:
   virtual void operator() ();
 
   /* Returns the sender's PID of the last dequeued (current) message. */
-  PID from() const { return current != NULL ? current->from : PID(); }
+  PID from() const;
 
-  /* Returns the id of the last dequeued (current) message. */
-  MSGID msgid() const { return current != NULL ? current->id : PROCESS_ERROR; }
+  /* Returns the name of the last dequeued (current) message. */
+  std::string name() const;
 
   /* Returns pointer and length of body of last dequeued (current) message. */
-  virtual const char * body(size_t *length) const;
+  const char * body(size_t *length) const;
 
   /* Put a message at front of queue (will not reschedule process). */
-  virtual void inject(const PID &from, MSGID id, const char *data = NULL, size_t length = 0);
+  void inject(const PID &from, const std::string &name, const char *data = NULL, size_t length = 0);
 
   /* Sends a message with data to PID. */
-  virtual void send(const PID &to, MSGID id, const char *data = NULL, size_t length = 0);
+  void send(const PID &to, const std::string &name, const char *data = NULL, size_t length = 0);
 
   /* Blocks for message at most specified seconds (0 implies forever). */
-  virtual MSGID receive(double secs = 0);
+  std::string receive(double secs = 0);
 
   /*  Processes dispatch messages. */
-  virtual MSGID serve(double secs = 0, bool forever = true);
+  std::string serve(double secs = 0, bool forever = true);
 
   /* Blocks at least specified seconds (may block longer). */
-  virtual void pause(double secs);
+  void pause(double secs);
 
   /* Links with the specified PID. */
-  virtual PID link(const PID &pid);
+  PID link(const PID &pid);
 
   /* IO events for awaiting. */
   enum { RDONLY = 01, WRONLY = 02, RDWR = 03 };
 
   /* Wait until operation is ready for file descriptor (or message received if not ignored). */
-  virtual bool await(int fd, int op, double secs = 0, bool ignore = true);
+  bool await(int fd, int op, double secs = 0, bool ignore = true);
 
   /* Returns true if operation on file descriptor is ready. */
-  virtual bool ready(int fd, int op);
+  bool ready(int fd, int op);
 
   /* Returns sub-second elapsed time (according to this process). */
   double elapsed();
@@ -214,11 +210,11 @@ public:
    * Sends a message with data without a return address.
    *
    * @param to receiver
-   * @param id message id
+   * @param name message name
    * @param data data to send (gets copied)
    * @param length length of data
    */
-  static void post(const PID &to, MSGID id, const char *data = NULL, size_t length = 0);
+  static void post(const PID &to, const std::string &name, const char *data = NULL, size_t length = 0);
 
   /**
    * Dispatches a void method on a process.
@@ -506,10 +502,10 @@ private:
   int refs;
 
   /* Queue of received messages. */
-  std::deque<msg *> msgs;
+  std::deque<Message *> messages;
 
   /* Current message. */
-  msg *current;
+  Message *current;
 
   /* Current "blocking" generation. */
   int generation;
@@ -526,10 +522,10 @@ private:
   void unlock() { pthread_mutex_unlock(&m); }
 
   /* Enqueues the specified message. */
-  void enqueue(msg *msg);
+  void enqueue(Message *message);
 
   /* Dequeues a message or returns NULL. */
-  msg * dequeue();
+  Message * dequeue();
 
   /* Dispatches the delegator to the specified process. */
   static void dispatcher(Process *, std::tr1::function<void (void)> *delegator);
@@ -844,7 +840,7 @@ Future<T> & Future<T>::operator = (const
       if (*t != NULL)
         delete *t;
       assert(trigger != NULL);
-      Process::post(trigger->self(), PROCESS_MSGID);
+      Process::post(trigger->self(), "");
       Process::wait(trigger->self());
       delete trigger;
     }
@@ -869,7 +865,7 @@ Future<T>::~Future()
     if (*t != NULL)
       delete *t;
     assert(trigger != NULL);
-    Process::post(trigger->self(), PROCESS_MSGID);
+    Process::post(trigger->self(), "");
     Process::wait(trigger->self());
     delete trigger;
   }
@@ -881,7 +877,7 @@ void Future<T>::set(const T &t_)
 {
   assert(t != NULL && *t == NULL);
   *t = new T(t_);
-  Process::post(trigger->self(), PROCESS_MSGID);
+  Process::post(trigger->self(), "");
 }
 
 
@@ -1065,4 +1061,4 @@ T Result<T>::get() const
 }
 
 
-#endif /* PROCESS_HPP */
+#endif // __PROCESS_HPP__

Added: incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/.gitignore
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/.gitignore?rev=1132253&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/.gitignore (added)
+++ incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/.gitignore Sun Jun  5 09:08:02 2011
@@ -0,0 +1,4 @@
+tags
+*.o
+test
+test_g

Added: incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/CONTRIBUTIONS
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/CONTRIBUTIONS?rev=1132253&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/CONTRIBUTIONS (added)
+++ incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/CONTRIBUTIONS Sun Jun  5 09:08:02 2011
@@ -0,0 +1,4 @@
+Contributors must agree to the Contributor License Agreement before patches
+can be accepted.
+
+http://spreadsheets2.google.com/viewform?hl=en&formkey=dDJXOGUwbzlYaWM4cHN1MERwQS1CSnc6MQ

Added: incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/LICENSE-MIT
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/LICENSE-MIT?rev=1132253&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/LICENSE-MIT (added)
+++ incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/LICENSE-MIT Sun Jun  5 09:08:02 2011
@@ -0,0 +1,19 @@
+Copyright 2009,2010 Ryan Dahl <ry...@tinyclouds.org>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to
+deal in the Software without restriction, including without limitation the
+rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+sell copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+IN THE SOFTWARE. 

Added: incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/README.md
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/README.md?rev=1132253&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/README.md (added)
+++ incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/README.md Sun Jun  5 09:08:02 2011
@@ -0,0 +1,171 @@
+HTTP Parser
+===========
+
+This is a parser for HTTP messages written in C. It parses both requests and
+responses. The parser is designed to be used in performance HTTP
+applications. It does not make any syscalls nor allocations, it does not
+buffer data, it can be interrupted at anytime. Depending on your
+architecture, it only requires about 40 bytes of data per message
+stream (in a web server that is per connection).
+
+Features:
+
+  * No dependencies
+  * Handles persistent streams (keep-alive).
+  * Decodes chunked encoding.
+  * Upgrade support
+  * Defends against buffer overflow attacks.
+
+The parser extracts the following information from HTTP messages:
+
+  * Header fields and values
+  * Content-Length
+  * Request method
+  * Response status code
+  * Transfer-Encoding
+  * HTTP version
+  * Request path, query string, fragment
+  * Message body
+
+
+Usage
+-----
+
+One `http_parser` object is used per TCP connection. Initialize the struct
+using `http_parser_init()` and set the callbacks. That might look something
+like this for a request parser:
+
+    http_parser_settings settings;
+    settings.on_path = my_path_callback;
+    settings.on_header_field = my_header_field_callback;
+    /* ... */
+
+    http_parser *parser = malloc(sizeof(http_parser));
+    http_parser_init(parser, HTTP_REQUEST);
+    parser->data = my_socket;
+
+When data is received on the socket execute the parser and check for errors.
+
+    size_t len = 80*1024, nparsed;
+    char buf[len];
+    ssize_t recved;
+
+    recved = recv(fd, buf, len, 0);
+
+    if (recved < 0) {
+      /* Handle error. */
+    }
+
+    /* Start up / continue the parser.
+     * Note we pass recved==0 to signal that EOF has been recieved.
+     */
+    nparsed = http_parser_execute(parser, &settings, buf, recved);
+
+    if (parser->upgrade) {
+      /* handle new protocol */
+    } else if (nparsed != recved) {
+      /* Handle error. Usually just close the connection. */
+    }
+
+HTTP needs to know where the end of the stream is. For example, sometimes
+servers send responses without Content-Length and expect the client to
+consume input (for the body) until EOF. To tell http_parser about EOF, give
+`0` as the forth parameter to `http_parser_execute()`. Callbacks and errors
+can still be encountered during an EOF, so one must still be prepared
+to receive them.
+
+Scalar valued message information such as `status_code`, `method`, and the
+HTTP version are stored in the parser structure. This data is only
+temporally stored in `http_parser` and gets reset on each new message. If
+this information is needed later, copy it out of the structure during the
+`headers_complete` callback.
+
+The parser decodes the transfer-encoding for both requests and responses
+transparently. That is, a chunked encoding is decoded before being sent to
+the on_body callback.
+
+
+The Special Problem of Upgrade
+------------------------------
+
+HTTP supports upgrading the connection to a different protocol. An
+increasingly common example of this is the Web Socket protocol which sends
+a request like
+
+        GET /demo HTTP/1.1
+        Upgrade: WebSocket
+        Connection: Upgrade
+        Host: example.com
+        Origin: http://example.com
+        WebSocket-Protocol: sample
+
+followed by non-HTTP data.
+
+(See http://tools.ietf.org/html/draft-hixie-thewebsocketprotocol-75 for more
+information the Web Socket protocol.)
+
+To support this, the parser will treat this as a normal HTTP message without a
+body. Issuing both on_headers_complete and on_message_complete callbacks. However
+http_parser_execute() will stop parsing at the end of the headers and return.
+
+The user is expected to check if `parser->upgrade` has been set to 1 after
+`http_parser_execute()` returns. Non-HTTP data begins at the buffer supplied
+offset by the return value of `http_parser_execute()`.
+
+
+Callbacks
+---------
+
+During the `http_parser_execute()` call, the callbacks set in
+`http_parser_settings` will be executed. The parser maintains state and
+never looks behind, so buffering the data is not necessary. If you need to
+save certain data for later usage, you can do that from the callbacks.
+
+There are two types of callbacks:
+
+* notification `typedef int (*http_cb) (http_parser*);`
+    Callbacks: on_message_begin, on_headers_complete, on_message_complete.
+* data `typedef int (*http_data_cb) (http_parser*, const char *at, size_t length);`
+    Callbacks: (requests only) on_path, on_query_string, on_uri, on_fragment,
+               (common) on_header_field, on_header_value, on_body;
+
+Callbacks must return 0 on success. Returning a non-zero value indicates
+error to the parser, making it exit immediately.
+
+In case you parse HTTP message in chunks (i.e. `read()` request line
+from socket, parse, read half headers, parse, etc) your data callbacks
+may be called more than once. Http-parser guarantees that data pointer is only
+valid for the lifetime of callback. You can also `read()` into a heap allocated
+buffer to avoid copying memory around if this fits your application.
+
+Reading headers may be a tricky task if you read/parse headers partially.
+Basically, you need to remember whether last header callback was field or value
+and apply following logic:
+
+    (on_header_field and on_header_value shortened to on_h_*)
+     ------------------------ ------------ --------------------------------------------
+    | State (prev. callback) | Callback   | Description/action                         |
+     ------------------------ ------------ --------------------------------------------
+    | nothing (first call)   | on_h_field | Allocate new buffer and copy callback data |
+    |                        |            | into it                                    |
+     ------------------------ ------------ --------------------------------------------
+    | value                  | on_h_field | New header started.                        |
+    |                        |            | Copy current name,value buffers to headers |
+    |                        |            | list and allocate new buffer for new name  |
+     ------------------------ ------------ --------------------------------------------
+    | field                  | on_h_field | Previous name continues. Reallocate name   |
+    |                        |            | buffer and append callback data to it      |
+     ------------------------ ------------ --------------------------------------------
+    | field                  | on_h_value | Value for current header started. Allocate |
+    |                        |            | new buffer and copy callback data to it    |
+     ------------------------ ------------ --------------------------------------------
+    | value                  | on_h_value | Value continues. Reallocate value buffer   |
+    |                        |            | and append callback data to it             |
+     ------------------------ ------------ --------------------------------------------
+
+
+See examples of reading in headers:
+
+* [partial example](http://gist.github.com/155877) in C
+* [from http-parser tests](http://github.com/ry/http-parser/blob/37a0ff8928fb0d83cec0d0d8909c5a4abcd221af/test.c#L403) in C
+* [from Node library](http://github.com/ry/node/blob/842eaf446d2fdcb33b296c67c911c32a0dabc747/src/http.js#L284) in Javascript

Added: incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.c
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.c?rev=1132253&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.c (added)
+++ incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.c Sun Jun  5 09:08:02 2011
@@ -0,0 +1,1612 @@
+/* Copyright 2009,2010 Ryan Dahl <ry...@tinyclouds.org>
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to
+ * deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+ * sell copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in
+ * all copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+ * FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+#include <http_parser.h>
+#include <assert.h>
+#include <stddef.h>
+
+
+#ifndef MIN
+# define MIN(a,b) ((a) < (b) ? (a) : (b))
+#endif
+
+
+#define CALLBACK2(FOR)                                               \
+do {                                                                 \
+  if (settings->on_##FOR) {                                          \
+    if (0 != settings->on_##FOR(parser)) return (p - data);          \
+  }                                                                  \
+} while (0)
+
+
+#define MARK(FOR)                                                    \
+do {                                                                 \
+  FOR##_mark = p;                                                    \
+} while (0)
+
+#define CALLBACK_NOCLEAR(FOR)                                        \
+do {                                                                 \
+  if (FOR##_mark) {                                                  \
+    if (settings->on_##FOR) {                                        \
+      if (0 != settings->on_##FOR(parser,                            \
+                                 FOR##_mark,                         \
+                                 p - FOR##_mark))                    \
+      {                                                              \
+        return (p - data);                                           \
+      }                                                              \
+    }                                                                \
+  }                                                                  \
+} while (0)
+
+
+#define CALLBACK(FOR)                                                \
+do {                                                                 \
+  CALLBACK_NOCLEAR(FOR);                                             \
+  FOR##_mark = NULL;                                                 \
+} while (0)
+
+
+#define PROXY_CONNECTION "proxy-connection"
+#define CONNECTION "connection"
+#define CONTENT_LENGTH "content-length"
+#define TRANSFER_ENCODING "transfer-encoding"
+#define UPGRADE "upgrade"
+#define CHUNKED "chunked"
+#define KEEP_ALIVE "keep-alive"
+#define CLOSE "close"
+
+
+static const char *method_strings[] =
+  { "DELETE"
+  , "GET"
+  , "HEAD"
+  , "POST"
+  , "PUT"
+  , "CONNECT"
+  , "OPTIONS"
+  , "TRACE"
+  , "COPY"
+  , "LOCK"
+  , "MKCOL"
+  , "MOVE"
+  , "PROPFIND"
+  , "PROPPATCH"
+  , "UNLOCK"
+  , "REPORT"
+  , "MKACTIVITY"
+  , "CHECKOUT"
+  , "MERGE"
+  , "M-SEARCH"
+  , "NOTIFY"
+  , "SUBSCRIBE"
+  , "UNSUBSCRIBE"
+  };
+
+
+/* Tokens as defined by rfc 2616. Also lowercases them.
+ *        token       = 1*<any CHAR except CTLs or separators>
+ *     separators     = "(" | ")" | "<" | ">" | "@"
+ *                    | "," | ";" | ":" | "\" | <">
+ *                    | "/" | "[" | "]" | "?" | "="
+ *                    | "{" | "}" | SP | HT
+ */
+static const char tokens[256] = {
+/*   0 nul    1 soh    2 stx    3 etx    4 eot    5 enq    6 ack    7 bel  */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*   8 bs     9 ht    10 nl    11 vt    12 np    13 cr    14 so    15 si   */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*  16 dle   17 dc1   18 dc2   19 dc3   20 dc4   21 nak   22 syn   23 etb */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*  24 can   25 em    26 sub   27 esc   28 fs    29 gs    30 rs    31 us  */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*  32 sp    33  !    34  "    35  #    36  $    37  %    38  &    39  '  */
+       ' ',      '!',     '"',     '#',     '$',     '%',     '&',    '\'',
+/*  40  (    41  )    42  *    43  +    44  ,    45  -    46  .    47  /  */
+        0,       0,      '*',     '+',      0,      '-',     '.',     '/',
+/*  48  0    49  1    50  2    51  3    52  4    53  5    54  6    55  7  */
+       '0',     '1',     '2',     '3',     '4',     '5',     '6',     '7',
+/*  56  8    57  9    58  :    59  ;    60  <    61  =    62  >    63  ?  */
+       '8',     '9',      0,       0,       0,       0,       0,       0,
+/*  64  @    65  A    66  B    67  C    68  D    69  E    70  F    71  G  */
+        0,      'a',     'b',     'c',     'd',     'e',     'f',     'g',
+/*  72  H    73  I    74  J    75  K    76  L    77  M    78  N    79  O  */
+       'h',     'i',     'j',     'k',     'l',     'm',     'n',     'o',
+/*  80  P    81  Q    82  R    83  S    84  T    85  U    86  V    87  W  */
+       'p',     'q',     'r',     's',     't',     'u',     'v',     'w',
+/*  88  X    89  Y    90  Z    91  [    92  \    93  ]    94  ^    95  _  */
+       'x',     'y',     'z',      0,       0,       0,      '^',     '_',
+/*  96  `    97  a    98  b    99  c   100  d   101  e   102  f   103  g  */
+       '`',     'a',     'b',     'c',     'd',     'e',     'f',     'g',
+/* 104  h   105  i   106  j   107  k   108  l   109  m   110  n   111  o  */
+       'h',     'i',     'j',     'k',     'l',     'm',     'n',     'o',
+/* 112  p   113  q   114  r   115  s   116  t   117  u   118  v   119  w  */
+       'p',     'q',     'r',     's',     't',     'u',     'v',     'w',
+/* 120  x   121  y   122  z   123  {   124  |   125  }   126  ~   127 del */
+       'x',     'y',     'z',      0,      '|',     '}',     '~',       0 };
+
+
+static const int8_t unhex[256] =
+  {-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  ,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  ,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  , 0, 1, 2, 3, 4, 5, 6, 7, 8, 9,-1,-1,-1,-1,-1,-1
+  ,-1,10,11,12,13,14,15,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  ,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  ,-1,10,11,12,13,14,15,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  ,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1
+  };
+
+
+static const uint8_t normal_url_char[256] = {
+/*   0 nul    1 soh    2 stx    3 etx    4 eot    5 enq    6 ack    7 bel  */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*   8 bs     9 ht    10 nl    11 vt    12 np    13 cr    14 so    15 si   */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*  16 dle   17 dc1   18 dc2   19 dc3   20 dc4   21 nak   22 syn   23 etb */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*  24 can   25 em    26 sub   27 esc   28 fs    29 gs    30 rs    31 us  */
+        0,       0,       0,       0,       0,       0,       0,       0,
+/*  32 sp    33  !    34  "    35  #    36  $    37  %    38  &    39  '  */
+        0,       1,       1,       0,       1,       1,       1,       1,
+/*  40  (    41  )    42  *    43  +    44  ,    45  -    46  .    47  /  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/*  48  0    49  1    50  2    51  3    52  4    53  5    54  6    55  7  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/*  56  8    57  9    58  :    59  ;    60  <    61  =    62  >    63  ?  */
+        1,       1,       1,       1,       1,       1,       1,       0,
+/*  64  @    65  A    66  B    67  C    68  D    69  E    70  F    71  G  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/*  72  H    73  I    74  J    75  K    76  L    77  M    78  N    79  O  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/*  80  P    81  Q    82  R    83  S    84  T    85  U    86  V    87  W  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/*  88  X    89  Y    90  Z    91  [    92  \    93  ]    94  ^    95  _  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/*  96  `    97  a    98  b    99  c   100  d   101  e   102  f   103  g  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/* 104  h   105  i   106  j   107  k   108  l   109  m   110  n   111  o  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/* 112  p   113  q   114  r   115  s   116  t   117  u   118  v   119  w  */
+        1,       1,       1,       1,       1,       1,       1,       1,
+/* 120  x   121  y   122  z   123  {   124  |   125  }   126  ~   127 del */
+        1,       1,       1,       1,       1,       1,       1,       0 };
+
+
+enum state
+  { s_dead = 1 /* important that this is > 0 */
+
+  , s_start_req_or_res
+  , s_res_or_resp_H
+  , s_start_res
+  , s_res_H
+  , s_res_HT
+  , s_res_HTT
+  , s_res_HTTP
+  , s_res_first_http_major
+  , s_res_http_major
+  , s_res_first_http_minor
+  , s_res_http_minor
+  , s_res_first_status_code
+  , s_res_status_code
+  , s_res_status
+  , s_res_line_almost_done
+
+  , s_start_req
+
+  , s_req_method
+  , s_req_spaces_before_url
+  , s_req_schema
+  , s_req_schema_slash
+  , s_req_schema_slash_slash
+  , s_req_host
+  , s_req_port
+  , s_req_path
+  , s_req_query_string_start
+  , s_req_query_string
+  , s_req_fragment_start
+  , s_req_fragment
+  , s_req_http_start
+  , s_req_http_H
+  , s_req_http_HT
+  , s_req_http_HTT
+  , s_req_http_HTTP
+  , s_req_first_http_major
+  , s_req_http_major
+  , s_req_first_http_minor
+  , s_req_http_minor
+  , s_req_line_almost_done
+
+  , s_header_field_start
+  , s_header_field
+  , s_header_value_start
+  , s_header_value
+
+  , s_header_almost_done
+
+  , s_headers_almost_done
+  /* Important: 's_headers_almost_done' must be the last 'header' state. All
+   * states beyond this must be 'body' states. It is used for overflow
+   * checking. See the PARSING_HEADER() macro.
+   */
+  , s_chunk_size_start
+  , s_chunk_size
+  , s_chunk_size_almost_done
+  , s_chunk_parameters
+  , s_chunk_data
+  , s_chunk_data_almost_done
+  , s_chunk_data_done
+
+  , s_body_identity
+  , s_body_identity_eof
+  };
+
+
+#define PARSING_HEADER(state) (state <= s_headers_almost_done && 0 == (parser->flags & F_TRAILING))
+
+
+enum header_states
+  { h_general = 0
+  , h_C
+  , h_CO
+  , h_CON
+
+  , h_matching_connection
+  , h_matching_proxy_connection
+  , h_matching_content_length
+  , h_matching_transfer_encoding
+  , h_matching_upgrade
+
+  , h_connection
+  , h_content_length
+  , h_transfer_encoding
+  , h_upgrade
+
+  , h_matching_transfer_encoding_chunked
+  , h_matching_connection_keep_alive
+  , h_matching_connection_close
+
+  , h_transfer_encoding_chunked
+  , h_connection_keep_alive
+  , h_connection_close
+  };
+
+
+enum flags
+  { F_CHUNKED               = 1 << 0
+  , F_CONNECTION_KEEP_ALIVE = 1 << 1
+  , F_CONNECTION_CLOSE      = 1 << 2
+  , F_TRAILING              = 1 << 3
+  , F_UPGRADE               = 1 << 4
+  , F_SKIPBODY              = 1 << 5
+  };
+
+
+#define CR '\r'
+#define LF '\n'
+#define LOWER(c) (unsigned char)(c | 0x20)
+#define TOKEN(c) tokens[(unsigned char)c]
+
+
+#define start_state (parser->type == HTTP_REQUEST ? s_start_req : s_start_res)
+
+
+#if HTTP_PARSER_STRICT
+# define STRICT_CHECK(cond) if (cond) goto error
+# define NEW_MESSAGE() (http_should_keep_alive(parser) ? start_state : s_dead)
+#else
+# define STRICT_CHECK(cond)
+# define NEW_MESSAGE() start_state
+#endif
+
+
+size_t http_parser_execute (http_parser *parser,
+                            const http_parser_settings *settings,
+                            const char *data,
+                            size_t len)
+{
+  char c, ch;
+  const char *p = data, *pe;
+  int64_t to_read;
+
+  enum state state = (enum state) parser->state;
+  enum header_states header_state = (enum header_states) parser->header_state;
+  uint64_t index = parser->index;
+  uint64_t nread = parser->nread;
+
+  if (len == 0) {
+    switch (state) {
+      case s_body_identity_eof:
+        CALLBACK2(message_complete);
+        return 0;
+
+      case s_dead:
+      case s_start_req_or_res:
+      case s_start_res:
+      case s_start_req:
+        return 0;
+
+      default:
+        return 1; // error
+    }
+  }
+
+  /* technically we could combine all of these (except for url_mark) into one
+     variable, saving stack space, but it seems more clear to have them
+     separated. */
+  const char *header_field_mark = 0;
+  const char *header_value_mark = 0;
+  const char *fragment_mark = 0;
+  const char *query_string_mark = 0;
+  const char *path_mark = 0;
+  const char *url_mark = 0;
+
+  if (state == s_header_field)
+    header_field_mark = data;
+  if (state == s_header_value)
+    header_value_mark = data;
+  if (state == s_req_fragment)
+    fragment_mark = data;
+  if (state == s_req_query_string)
+    query_string_mark = data;
+  if (state == s_req_path)
+    path_mark = data;
+  if (state == s_req_path || state == s_req_schema || state == s_req_schema_slash
+      || state == s_req_schema_slash_slash || state == s_req_port
+      || state == s_req_query_string_start || state == s_req_query_string
+      || state == s_req_host
+      || state == s_req_fragment_start || state == s_req_fragment)
+    url_mark = data;
+
+  for (p=data, pe=data+len; p != pe; p++) {
+    ch = *p;
+
+    if (PARSING_HEADER(state)) {
+      ++nread;
+      /* Buffer overflow attack */
+      if (nread > HTTP_MAX_HEADER_SIZE) goto error;
+    }
+
+    switch (state) {
+
+      case s_dead:
+        /* this state is used after a 'Connection: close' message
+         * the parser will error out if it reads another message
+         */
+        goto error;
+
+      case s_start_req_or_res:
+      {
+        if (ch == CR || ch == LF)
+          break;
+        parser->flags = 0;
+        parser->content_length = -1;
+
+        CALLBACK2(message_begin);
+
+        if (ch == 'H')
+          state = s_res_or_resp_H;
+        else {
+          parser->type = HTTP_REQUEST;
+          goto start_req_method_assign;
+        }
+        break;
+      }
+
+      case s_res_or_resp_H:
+        if (ch == 'T') {
+          parser->type = HTTP_RESPONSE;
+          state = s_res_HT;
+        } else {
+          if (ch != 'E') goto error;
+          parser->type = HTTP_REQUEST;
+          parser->method = HTTP_HEAD;
+          index = 2;
+          state = s_req_method;
+        }
+        break;
+
+      case s_start_res:
+      {
+        parser->flags = 0;
+        parser->content_length = -1;
+
+        CALLBACK2(message_begin);
+
+        switch (ch) {
+          case 'H':
+            state = s_res_H;
+            break;
+
+          case CR:
+          case LF:
+            break;
+
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_res_H:
+        STRICT_CHECK(ch != 'T');
+        state = s_res_HT;
+        break;
+
+      case s_res_HT:
+        STRICT_CHECK(ch != 'T');
+        state = s_res_HTT;
+        break;
+
+      case s_res_HTT:
+        STRICT_CHECK(ch != 'P');
+        state = s_res_HTTP;
+        break;
+
+      case s_res_HTTP:
+        STRICT_CHECK(ch != '/');
+        state = s_res_first_http_major;
+        break;
+
+      case s_res_first_http_major:
+        if (ch < '1' || ch > '9') goto error;
+        parser->http_major = ch - '0';
+        state = s_res_http_major;
+        break;
+
+      /* major HTTP version or dot */
+      case s_res_http_major:
+      {
+        if (ch == '.') {
+          state = s_res_first_http_minor;
+          break;
+        }
+
+        if (ch < '0' || ch > '9') goto error;
+
+        parser->http_major *= 10;
+        parser->http_major += ch - '0';
+
+        if (parser->http_major > 999) goto error;
+        break;
+      }
+
+      /* first digit of minor HTTP version */
+      case s_res_first_http_minor:
+        if (ch < '0' || ch > '9') goto error;
+        parser->http_minor = ch - '0';
+        state = s_res_http_minor;
+        break;
+
+      /* minor HTTP version or end of request line */
+      case s_res_http_minor:
+      {
+        if (ch == ' ') {
+          state = s_res_first_status_code;
+          break;
+        }
+
+        if (ch < '0' || ch > '9') goto error;
+
+        parser->http_minor *= 10;
+        parser->http_minor += ch - '0';
+
+        if (parser->http_minor > 999) goto error;
+        break;
+      }
+
+      case s_res_first_status_code:
+      {
+        if (ch < '0' || ch > '9') {
+          if (ch == ' ') {
+            break;
+          }
+          goto error;
+        }
+        parser->status_code = ch - '0';
+        state = s_res_status_code;
+        break;
+      }
+
+      case s_res_status_code:
+      {
+        if (ch < '0' || ch > '9') {
+          switch (ch) {
+            case ' ':
+              state = s_res_status;
+              break;
+            case CR:
+              state = s_res_line_almost_done;
+              break;
+            case LF:
+              state = s_header_field_start;
+              break;
+            default:
+              goto error;
+          }
+          break;
+        }
+
+        parser->status_code *= 10;
+        parser->status_code += ch - '0';
+
+        if (parser->status_code > 999) goto error;
+        break;
+      }
+
+      case s_res_status:
+        /* the human readable status. e.g. "NOT FOUND"
+         * we are not humans so just ignore this */
+        if (ch == CR) {
+          state = s_res_line_almost_done;
+          break;
+        }
+
+        if (ch == LF) {
+          state = s_header_field_start;
+          break;
+        }
+        break;
+
+      case s_res_line_almost_done:
+        STRICT_CHECK(ch != LF);
+        state = s_header_field_start;
+        break;
+
+      case s_start_req:
+      {
+        if (ch == CR || ch == LF)
+          break;
+        parser->flags = 0;
+        parser->content_length = -1;
+
+        CALLBACK2(message_begin);
+
+        if (ch < 'A' || 'Z' < ch) goto error;
+
+      start_req_method_assign:
+        parser->method = (enum http_method) 0;
+        index = 1;
+        switch (ch) {
+          case 'C': parser->method = HTTP_CONNECT; /* or COPY, CHECKOUT */ break;
+          case 'D': parser->method = HTTP_DELETE; break;
+          case 'G': parser->method = HTTP_GET; break;
+          case 'H': parser->method = HTTP_HEAD; break;
+          case 'L': parser->method = HTTP_LOCK; break;
+          case 'M': parser->method = HTTP_MKCOL; /* or MOVE, MKACTIVITY, MERGE, M-SEARCH */ break;
+          case 'N': parser->method = HTTP_NOTIFY; break;
+          case 'O': parser->method = HTTP_OPTIONS; break;
+          case 'P': parser->method = HTTP_POST; /* or PROPFIND or PROPPATCH or PUT */ break;
+          case 'R': parser->method = HTTP_REPORT; break;
+          case 'S': parser->method = HTTP_SUBSCRIBE; break;
+          case 'T': parser->method = HTTP_TRACE; break;
+          case 'U': parser->method = HTTP_UNLOCK; /* or UNSUBSCRIBE */ break;
+          default: goto error;
+        }
+        state = s_req_method;
+        break;
+      }
+
+      case s_req_method:
+      {
+        if (ch == '\0')
+          goto error;
+
+        const char *matcher = method_strings[parser->method];
+        if (ch == ' ' && matcher[index] == '\0') {
+          state = s_req_spaces_before_url;
+        } else if (ch == matcher[index]) {
+          ; /* nada */
+        } else if (parser->method == HTTP_CONNECT) {
+          if (index == 1 && ch == 'H') {
+            parser->method = HTTP_CHECKOUT;
+          } else if (index == 2  && ch == 'P') {
+            parser->method = HTTP_COPY;
+          }
+        } else if (parser->method == HTTP_MKCOL) {
+          if (index == 1 && ch == 'O') {
+            parser->method = HTTP_MOVE;
+          } else if (index == 1 && ch == 'E') {
+            parser->method = HTTP_MERGE;
+          } else if (index == 1 && ch == '-') {
+            parser->method = HTTP_MSEARCH;
+          } else if (index == 2 && ch == 'A') {
+            parser->method = HTTP_MKACTIVITY;
+          }
+        } else if (index == 1 && parser->method == HTTP_POST && ch == 'R') {
+          parser->method = HTTP_PROPFIND; /* or HTTP_PROPPATCH */
+        } else if (index == 1 && parser->method == HTTP_POST && ch == 'U') {
+          parser->method = HTTP_PUT;
+        } else if (index == 2 && parser->method == HTTP_UNLOCK && ch == 'S') {
+          parser->method = HTTP_UNSUBSCRIBE;
+        } else if (index == 4 && parser->method == HTTP_PROPFIND && ch == 'P') {
+          parser->method = HTTP_PROPPATCH;
+        } else {
+          goto error;
+        }
+
+        ++index;
+        break;
+      }
+      case s_req_spaces_before_url:
+      {
+        if (ch == ' ') break;
+
+        if (ch == '/' || ch == '*') {
+          MARK(url);
+          MARK(path);
+          state = s_req_path;
+          break;
+        }
+
+        c = LOWER(ch);
+
+        if (c >= 'a' && c <= 'z') {
+          MARK(url);
+          state = s_req_schema;
+          break;
+        }
+
+        goto error;
+      }
+
+      case s_req_schema:
+      {
+        c = LOWER(ch);
+
+        if (c >= 'a' && c <= 'z') break;
+
+        if (ch == ':') {
+          state = s_req_schema_slash;
+          break;
+        } else if (ch == '.') {
+          state = s_req_host;
+          break;
+        } else if ('0' <= ch && ch <= '9') {
+          state = s_req_host;
+          break;
+        }
+
+        goto error;
+      }
+
+      case s_req_schema_slash:
+        STRICT_CHECK(ch != '/');
+        state = s_req_schema_slash_slash;
+        break;
+
+      case s_req_schema_slash_slash:
+        STRICT_CHECK(ch != '/');
+        state = s_req_host;
+        break;
+
+      case s_req_host:
+      {
+        c = LOWER(ch);
+        if (c >= 'a' && c <= 'z') break;
+        if ((ch >= '0' && ch <= '9') || ch == '.' || ch == '-') break;
+        switch (ch) {
+          case ':':
+            state = s_req_port;
+            break;
+          case '/':
+            MARK(path);
+            state = s_req_path;
+            break;
+          case ' ':
+            /* The request line looks like:
+             *   "GET http://foo.bar.com HTTP/1.1"
+             * That is, there is no path.
+             */
+            CALLBACK(url);
+            state = s_req_http_start;
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_port:
+      {
+        if (ch >= '0' && ch <= '9') break;
+        switch (ch) {
+          case '/':
+            MARK(path);
+            state = s_req_path;
+            break;
+          case ' ':
+            /* The request line looks like:
+             *   "GET http://foo.bar.com:1234 HTTP/1.1"
+             * That is, there is no path.
+             */
+            CALLBACK(url);
+            state = s_req_http_start;
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_path:
+      {
+        if (normal_url_char[(unsigned char)ch]) break;
+
+        switch (ch) {
+          case ' ':
+            CALLBACK(url);
+            CALLBACK(path);
+            state = s_req_http_start;
+            break;
+          case CR:
+            CALLBACK(url);
+            CALLBACK(path);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_req_line_almost_done;
+            break;
+          case LF:
+            CALLBACK(url);
+            CALLBACK(path);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_header_field_start;
+            break;
+          case '?':
+            CALLBACK(path);
+            state = s_req_query_string_start;
+            break;
+          case '#':
+            CALLBACK(path);
+            state = s_req_fragment_start;
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_query_string_start:
+      {
+        if (normal_url_char[(unsigned char)ch]) {
+          MARK(query_string);
+          state = s_req_query_string;
+          break;
+        }
+
+        switch (ch) {
+          case '?':
+            break; /* XXX ignore extra '?' ... is this right? */
+          case ' ':
+            CALLBACK(url);
+            state = s_req_http_start;
+            break;
+          case CR:
+            CALLBACK(url);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_req_line_almost_done;
+            break;
+          case LF:
+            CALLBACK(url);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_header_field_start;
+            break;
+          case '#':
+            state = s_req_fragment_start;
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_query_string:
+      {
+        if (normal_url_char[(unsigned char)ch]) break;
+
+        switch (ch) {
+          case '?':
+            /* allow extra '?' in query string */
+            break;
+          case ' ':
+            CALLBACK(url);
+            CALLBACK(query_string);
+            state = s_req_http_start;
+            break;
+          case CR:
+            CALLBACK(url);
+            CALLBACK(query_string);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_req_line_almost_done;
+            break;
+          case LF:
+            CALLBACK(url);
+            CALLBACK(query_string);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_header_field_start;
+            break;
+          case '#':
+            CALLBACK(query_string);
+            state = s_req_fragment_start;
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_fragment_start:
+      {
+        if (normal_url_char[(unsigned char)ch]) {
+          MARK(fragment);
+          state = s_req_fragment;
+          break;
+        }
+
+        switch (ch) {
+          case ' ':
+            CALLBACK(url);
+            state = s_req_http_start;
+            break;
+          case CR:
+            CALLBACK(url);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_req_line_almost_done;
+            break;
+          case LF:
+            CALLBACK(url);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_header_field_start;
+            break;
+          case '?':
+            MARK(fragment);
+            state = s_req_fragment;
+            break;
+          case '#':
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_fragment:
+      {
+        if (normal_url_char[(unsigned char)ch]) break;
+
+        switch (ch) {
+          case ' ':
+            CALLBACK(url);
+            CALLBACK(fragment);
+            state = s_req_http_start;
+            break;
+          case CR:
+            CALLBACK(url);
+            CALLBACK(fragment);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_req_line_almost_done;
+            break;
+          case LF:
+            CALLBACK(url);
+            CALLBACK(fragment);
+            parser->http_major = 0;
+            parser->http_minor = 9;
+            state = s_header_field_start;
+            break;
+          case '?':
+          case '#':
+            break;
+          default:
+            goto error;
+        }
+        break;
+      }
+
+      case s_req_http_start:
+        switch (ch) {
+          case 'H':
+            state = s_req_http_H;
+            break;
+          case ' ':
+            break;
+          default:
+            goto error;
+        }
+        break;
+
+      case s_req_http_H:
+        STRICT_CHECK(ch != 'T');
+        state = s_req_http_HT;
+        break;
+
+      case s_req_http_HT:
+        STRICT_CHECK(ch != 'T');
+        state = s_req_http_HTT;
+        break;
+
+      case s_req_http_HTT:
+        STRICT_CHECK(ch != 'P');
+        state = s_req_http_HTTP;
+        break;
+
+      case s_req_http_HTTP:
+        STRICT_CHECK(ch != '/');
+        state = s_req_first_http_major;
+        break;
+
+      /* first digit of major HTTP version */
+      case s_req_first_http_major:
+        if (ch < '1' || ch > '9') goto error;
+        parser->http_major = ch - '0';
+        state = s_req_http_major;
+        break;
+
+      /* major HTTP version or dot */
+      case s_req_http_major:
+      {
+        if (ch == '.') {
+          state = s_req_first_http_minor;
+          break;
+        }
+
+        if (ch < '0' || ch > '9') goto error;
+
+        parser->http_major *= 10;
+        parser->http_major += ch - '0';
+
+        if (parser->http_major > 999) goto error;
+        break;
+      }
+
+      /* first digit of minor HTTP version */
+      case s_req_first_http_minor:
+        if (ch < '0' || ch > '9') goto error;
+        parser->http_minor = ch - '0';
+        state = s_req_http_minor;
+        break;
+
+      /* minor HTTP version or end of request line */
+      case s_req_http_minor:
+      {
+        if (ch == CR) {
+          state = s_req_line_almost_done;
+          break;
+        }
+
+        if (ch == LF) {
+          state = s_header_field_start;
+          break;
+        }
+
+        /* XXX allow spaces after digit? */
+
+        if (ch < '0' || ch > '9') goto error;
+
+        parser->http_minor *= 10;
+        parser->http_minor += ch - '0';
+
+        if (parser->http_minor > 999) goto error;
+        break;
+      }
+
+      /* end of request line */
+      case s_req_line_almost_done:
+      {
+        if (ch != LF) goto error;
+        state = s_header_field_start;
+        break;
+      }
+
+      case s_header_field_start:
+      {
+        if (ch == CR) {
+          state = s_headers_almost_done;
+          break;
+        }
+
+        if (ch == LF) {
+          /* they might be just sending \n instead of \r\n so this would be
+           * the second \n to denote the end of headers*/
+          state = s_headers_almost_done;
+          goto headers_almost_done;
+        }
+
+        c = TOKEN(ch);
+
+        if (!c) goto error;
+
+        MARK(header_field);
+
+        index = 0;
+        state = s_header_field;
+
+        switch (c) {
+          case 'c':
+            header_state = h_C;
+            break;
+
+          case 'p':
+            header_state = h_matching_proxy_connection;
+            break;
+
+          case 't':
+            header_state = h_matching_transfer_encoding;
+            break;
+
+          case 'u':
+            header_state = h_matching_upgrade;
+            break;
+
+          default:
+            header_state = h_general;
+            break;
+        }
+        break;
+      }
+
+      case s_header_field:
+      {
+        c = TOKEN(ch);
+
+        if (c) {
+          switch (header_state) {
+            case h_general:
+              break;
+
+            case h_C:
+              index++;
+              header_state = (c == 'o' ? h_CO : h_general);
+              break;
+
+            case h_CO:
+              index++;
+              header_state = (c == 'n' ? h_CON : h_general);
+              break;
+
+            case h_CON:
+              index++;
+              switch (c) {
+                case 'n':
+                  header_state = h_matching_connection;
+                  break;
+                case 't':
+                  header_state = h_matching_content_length;
+                  break;
+                default:
+                  header_state = h_general;
+                  break;
+              }
+              break;
+
+            /* connection */
+
+            case h_matching_connection:
+              index++;
+              if (index > sizeof(CONNECTION)-1
+                  || c != CONNECTION[index]) {
+                header_state = h_general;
+              } else if (index == sizeof(CONNECTION)-2) {
+                header_state = h_connection;
+              }
+              break;
+
+            /* proxy-connection */
+
+            case h_matching_proxy_connection:
+              index++;
+              if (index > sizeof(PROXY_CONNECTION)-1
+                  || c != PROXY_CONNECTION[index]) {
+                header_state = h_general;
+              } else if (index == sizeof(PROXY_CONNECTION)-2) {
+                header_state = h_connection;
+              }
+              break;
+
+            /* content-length */
+
+            case h_matching_content_length:
+              index++;
+              if (index > sizeof(CONTENT_LENGTH)-1
+                  || c != CONTENT_LENGTH[index]) {
+                header_state = h_general;
+              } else if (index == sizeof(CONTENT_LENGTH)-2) {
+                header_state = h_content_length;
+              }
+              break;
+
+            /* transfer-encoding */
+
+            case h_matching_transfer_encoding:
+              index++;
+              if (index > sizeof(TRANSFER_ENCODING)-1
+                  || c != TRANSFER_ENCODING[index]) {
+                header_state = h_general;
+              } else if (index == sizeof(TRANSFER_ENCODING)-2) {
+                header_state = h_transfer_encoding;
+              }
+              break;
+
+            /* upgrade */
+
+            case h_matching_upgrade:
+              index++;
+              if (index > sizeof(UPGRADE)-1
+                  || c != UPGRADE[index]) {
+                header_state = h_general;
+              } else if (index == sizeof(UPGRADE)-2) {
+                header_state = h_upgrade;
+              }
+              break;
+
+            case h_connection:
+            case h_content_length:
+            case h_transfer_encoding:
+            case h_upgrade:
+              if (ch != ' ') header_state = h_general;
+              break;
+
+            default:
+              assert(0 && "Unknown header_state");
+              break;
+          }
+          break;
+        }
+
+        if (ch == ':') {
+          CALLBACK(header_field);
+          state = s_header_value_start;
+          break;
+        }
+
+        if (ch == CR) {
+          state = s_header_almost_done;
+          CALLBACK(header_field);
+          break;
+        }
+
+        if (ch == LF) {
+          CALLBACK(header_field);
+          state = s_header_field_start;
+          break;
+        }
+
+        goto error;
+      }
+
+      case s_header_value_start:
+      {
+        if (ch == ' ') break;
+
+        MARK(header_value);
+
+        state = s_header_value;
+        index = 0;
+
+        c = LOWER(ch);
+
+        if (ch == CR) {
+          CALLBACK(header_value);
+          header_state = h_general;
+          state = s_header_almost_done;
+          break;
+        }
+
+        if (ch == LF) {
+          CALLBACK(header_value);
+          state = s_header_field_start;
+          break;
+        }
+
+        switch (header_state) {
+          case h_upgrade:
+            parser->flags |= F_UPGRADE;
+            header_state = h_general;
+            break;
+
+          case h_transfer_encoding:
+            /* looking for 'Transfer-Encoding: chunked' */
+            if ('c' == c) {
+              header_state = h_matching_transfer_encoding_chunked;
+            } else {
+              header_state = h_general;
+            }
+            break;
+
+          case h_content_length:
+            if (ch < '0' || ch > '9') goto error;
+            parser->content_length = ch - '0';
+            break;
+
+          case h_connection:
+            /* looking for 'Connection: keep-alive' */
+            if (c == 'k') {
+              header_state = h_matching_connection_keep_alive;
+            /* looking for 'Connection: close' */
+            } else if (c == 'c') {
+              header_state = h_matching_connection_close;
+            } else {
+              header_state = h_general;
+            }
+            break;
+
+          default:
+            header_state = h_general;
+            break;
+        }
+        break;
+      }
+
+      case s_header_value:
+      {
+        c = LOWER(ch);
+
+        if (ch == CR) {
+          CALLBACK(header_value);
+          state = s_header_almost_done;
+          break;
+        }
+
+        if (ch == LF) {
+          CALLBACK(header_value);
+          goto header_almost_done;
+        }
+
+        switch (header_state) {
+          case h_general:
+            break;
+
+          case h_connection:
+          case h_transfer_encoding:
+            assert(0 && "Shouldn't get here.");
+            break;
+
+          case h_content_length:
+            if (ch == ' ') break;
+            if (ch < '0' || ch > '9') goto error;
+            parser->content_length *= 10;
+            parser->content_length += ch - '0';
+            break;
+
+          /* Transfer-Encoding: chunked */
+          case h_matching_transfer_encoding_chunked:
+            index++;
+            if (index > sizeof(CHUNKED)-1
+                || c != CHUNKED[index]) {
+              header_state = h_general;
+            } else if (index == sizeof(CHUNKED)-2) {
+              header_state = h_transfer_encoding_chunked;
+            }
+            break;
+
+          /* looking for 'Connection: keep-alive' */
+          case h_matching_connection_keep_alive:
+            index++;
+            if (index > sizeof(KEEP_ALIVE)-1
+                || c != KEEP_ALIVE[index]) {
+              header_state = h_general;
+            } else if (index == sizeof(KEEP_ALIVE)-2) {
+              header_state = h_connection_keep_alive;
+            }
+            break;
+
+          /* looking for 'Connection: close' */
+          case h_matching_connection_close:
+            index++;
+            if (index > sizeof(CLOSE)-1 || c != CLOSE[index]) {
+              header_state = h_general;
+            } else if (index == sizeof(CLOSE)-2) {
+              header_state = h_connection_close;
+            }
+            break;
+
+          case h_transfer_encoding_chunked:
+          case h_connection_keep_alive:
+          case h_connection_close:
+            if (ch != ' ') header_state = h_general;
+            break;
+
+          default:
+            state = s_header_value;
+            header_state = h_general;
+            break;
+        }
+        break;
+      }
+
+      case s_header_almost_done:
+      header_almost_done:
+      {
+        STRICT_CHECK(ch != LF);
+
+        state = s_header_field_start;
+
+        switch (header_state) {
+          case h_connection_keep_alive:
+            parser->flags |= F_CONNECTION_KEEP_ALIVE;
+            break;
+          case h_connection_close:
+            parser->flags |= F_CONNECTION_CLOSE;
+            break;
+          case h_transfer_encoding_chunked:
+            parser->flags |= F_CHUNKED;
+            break;
+          default:
+            break;
+        }
+        break;
+      }
+
+      case s_headers_almost_done:
+      headers_almost_done:
+      {
+        STRICT_CHECK(ch != LF);
+
+        if (parser->flags & F_TRAILING) {
+          /* End of a chunked request */
+          CALLBACK2(message_complete);
+          state = NEW_MESSAGE();
+          break;
+        }
+
+        nread = 0;
+
+        if (parser->flags & F_UPGRADE || parser->method == HTTP_CONNECT) {
+          parser->upgrade = 1;
+        }
+
+        /* Here we call the headers_complete callback. This is somewhat
+         * different than other callbacks because if the user returns 1, we
+         * will interpret that as saying that this message has no body. This
+         * is needed for the annoying case of recieving a response to a HEAD
+         * request.
+         */
+        if (settings->on_headers_complete) {
+          switch (settings->on_headers_complete(parser)) {
+            case 0:
+              break;
+
+            case 1:
+              parser->flags |= F_SKIPBODY;
+              break;
+
+            default:
+              parser->state = state;
+              return p - data; /* Error */
+          }
+        }
+
+        /* Exit, the rest of the connect is in a different protocol. */
+        if (parser->upgrade) {
+          CALLBACK2(message_complete);
+          return (p - data);
+        }
+
+        if (parser->flags & F_SKIPBODY) {
+          CALLBACK2(message_complete);
+          state = NEW_MESSAGE();
+        } else if (parser->flags & F_CHUNKED) {
+          /* chunked encoding - ignore Content-Length header */
+          state = s_chunk_size_start;
+        } else {
+          if (parser->content_length == 0) {
+            /* Content-Length header given but zero: Content-Length: 0\r\n */
+            CALLBACK2(message_complete);
+            state = NEW_MESSAGE();
+          } else if (parser->content_length > 0) {
+            /* Content-Length header given and non-zero */
+            state = s_body_identity;
+          } else {
+            if (parser->type == HTTP_REQUEST || http_should_keep_alive(parser)) {
+              /* Assume content-length 0 - read the next */
+              CALLBACK2(message_complete);
+              state = NEW_MESSAGE();
+            } else {
+              /* Read body until EOF */
+              state = s_body_identity_eof;
+            }
+          }
+        }
+
+        break;
+      }
+
+      case s_body_identity:
+        to_read = MIN(pe - p, (int64_t)parser->content_length);
+        if (to_read > 0) {
+          if (settings->on_body) settings->on_body(parser, p, to_read);
+          p += to_read - 1;
+          parser->content_length -= to_read;
+          if (parser->content_length == 0) {
+            CALLBACK2(message_complete);
+            state = NEW_MESSAGE();
+          }
+        }
+        break;
+
+      /* read until EOF */
+      case s_body_identity_eof:
+        to_read = pe - p;
+        if (to_read > 0) {
+          if (settings->on_body) settings->on_body(parser, p, to_read);
+          p += to_read - 1;
+        }
+        break;
+
+      case s_chunk_size_start:
+      {
+        assert(parser->flags & F_CHUNKED);
+
+        c = unhex[(unsigned char)ch];
+        if (c == -1) goto error;
+        parser->content_length = c;
+        state = s_chunk_size;
+        break;
+      }
+
+      case s_chunk_size:
+      {
+        assert(parser->flags & F_CHUNKED);
+
+        if (ch == CR) {
+          state = s_chunk_size_almost_done;
+          break;
+        }
+
+        c = unhex[(unsigned char)ch];
+
+        if (c == -1) {
+          if (ch == ';' || ch == ' ') {
+            state = s_chunk_parameters;
+            break;
+          }
+          goto error;
+        }
+
+        parser->content_length *= 16;
+        parser->content_length += c;
+        break;
+      }
+
+      case s_chunk_parameters:
+      {
+        assert(parser->flags & F_CHUNKED);
+        /* just ignore this shit. TODO check for overflow */
+        if (ch == CR) {
+          state = s_chunk_size_almost_done;
+          break;
+        }
+        break;
+      }
+
+      case s_chunk_size_almost_done:
+      {
+        assert(parser->flags & F_CHUNKED);
+        STRICT_CHECK(ch != LF);
+
+        if (parser->content_length == 0) {
+          parser->flags |= F_TRAILING;
+          state = s_header_field_start;
+        } else {
+          state = s_chunk_data;
+        }
+        break;
+      }
+
+      case s_chunk_data:
+      {
+        assert(parser->flags & F_CHUNKED);
+
+        to_read = MIN(pe - p, (int64_t)(parser->content_length));
+
+        if (to_read > 0) {
+          if (settings->on_body) settings->on_body(parser, p, to_read);
+          p += to_read - 1;
+        }
+
+        if (to_read == parser->content_length) {
+          state = s_chunk_data_almost_done;
+        }
+
+        parser->content_length -= to_read;
+        break;
+      }
+
+      case s_chunk_data_almost_done:
+        assert(parser->flags & F_CHUNKED);
+        STRICT_CHECK(ch != CR);
+        state = s_chunk_data_done;
+        break;
+
+      case s_chunk_data_done:
+        assert(parser->flags & F_CHUNKED);
+        STRICT_CHECK(ch != LF);
+        state = s_chunk_size_start;
+        break;
+
+      default:
+        assert(0 && "unhandled state");
+        goto error;
+    }
+  }
+
+  CALLBACK_NOCLEAR(header_field);
+  CALLBACK_NOCLEAR(header_value);
+  CALLBACK_NOCLEAR(fragment);
+  CALLBACK_NOCLEAR(query_string);
+  CALLBACK_NOCLEAR(path);
+  CALLBACK_NOCLEAR(url);
+
+  parser->state = state;
+  parser->header_state = header_state;
+  parser->index = index;
+  parser->nread = nread;
+
+  return len;
+
+error:
+  parser->state = s_dead;
+  return (p - data);
+}
+
+
+int
+http_should_keep_alive (http_parser *parser)
+{
+  if (parser->http_major > 0 && parser->http_minor > 0) {
+    /* HTTP/1.1 */
+    if (parser->flags & F_CONNECTION_CLOSE) {
+      return 0;
+    } else {
+      return 1;
+    }
+  } else {
+    /* HTTP/1.0 or earlier */
+    if (parser->flags & F_CONNECTION_KEEP_ALIVE) {
+      return 1;
+    } else {
+      return 0;
+    }
+  }
+}
+
+
+const char * http_method_str (enum http_method m)
+{
+  return method_strings[m];
+}
+
+
+void
+http_parser_init (http_parser *parser, enum http_parser_type t)
+{
+  parser->type = t;
+  parser->state = (t == HTTP_REQUEST ? s_start_req : (t == HTTP_RESPONSE ? s_start_res : s_start_req_or_res));
+  parser->nread = 0;
+  parser->upgrade = 0;
+  parser->flags = 0;
+  parser->method = 0;
+}

Added: incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.h
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.h?rev=1132253&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.h (added)
+++ incubator/mesos/trunk/third_party/libprocess/third_party/ry-http-parser-1c3624a/http_parser.h Sun Jun  5 09:08:02 2011
@@ -0,0 +1,181 @@
+/* Copyright 2009,2010 Ryan Dahl <ry...@tinyclouds.org>
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to
+ * deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+ * sell copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in
+ * all copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+ * FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+#ifndef http_parser_h
+#define http_parser_h
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+
+#include <sys/types.h>
+#if defined(_WIN32) && !defined(__MINGW32__)
+typedef __int8 int8_t;
+typedef unsigned __int8 uint8_t;
+typedef __int16 int16_t;
+typedef unsigned __int16 uint16_t;
+typedef __int32 int32_t;
+typedef unsigned __int32 uint32_t;
+typedef __int64 int64_t;
+typedef unsigned __int64 uint64_t;
+
+typedef unsigned int size_t;
+typedef int ssize_t;
+#else
+#include <stdint.h>
+#endif
+
+/* Compile with -DHTTP_PARSER_STRICT=0 to make less checks, but run
+ * faster
+ */
+#ifndef HTTP_PARSER_STRICT
+# define HTTP_PARSER_STRICT 1
+#else
+# define HTTP_PARSER_STRICT 0
+#endif
+
+
+/* Maximium header size allowed */
+#define HTTP_MAX_HEADER_SIZE (80*1024)
+
+
+typedef struct http_parser http_parser;
+typedef struct http_parser_settings http_parser_settings;
+
+
+/* Callbacks should return non-zero to indicate an error. The parser will
+ * then halt execution.
+ *
+ * The one exception is on_headers_complete. In a HTTP_RESPONSE parser
+ * returning '1' from on_headers_complete will tell the parser that it
+ * should not expect a body. This is used when receiving a response to a
+ * HEAD request which may contain 'Content-Length' or 'Transfer-Encoding:
+ * chunked' headers that indicate the presence of a body.
+ *
+ * http_data_cb does not return data chunks. It will be call arbitrarally
+ * many times for each string. E.G. you might get 10 callbacks for "on_path"
+ * each providing just a few characters more data.
+ */
+typedef int (*http_data_cb) (http_parser*, const char *at, size_t length);
+typedef int (*http_cb) (http_parser*);
+
+
+/* Request Methods */
+enum http_method
+  { HTTP_DELETE    = 0
+  , HTTP_GET
+  , HTTP_HEAD
+  , HTTP_POST
+  , HTTP_PUT
+  /* pathological */
+  , HTTP_CONNECT
+  , HTTP_OPTIONS
+  , HTTP_TRACE
+  /* webdav */
+  , HTTP_COPY
+  , HTTP_LOCK
+  , HTTP_MKCOL
+  , HTTP_MOVE
+  , HTTP_PROPFIND
+  , HTTP_PROPPATCH
+  , HTTP_UNLOCK
+  /* subversion */
+  , HTTP_REPORT
+  , HTTP_MKACTIVITY
+  , HTTP_CHECKOUT
+  , HTTP_MERGE
+  /* upnp */
+  , HTTP_MSEARCH
+  , HTTP_NOTIFY
+  , HTTP_SUBSCRIBE
+  , HTTP_UNSUBSCRIBE
+  };
+
+
+enum http_parser_type { HTTP_REQUEST, HTTP_RESPONSE, HTTP_BOTH };
+
+
+struct http_parser {
+  /** PRIVATE **/
+  unsigned char type : 2;
+  unsigned char flags : 6;
+  unsigned char state;
+  unsigned char header_state;
+  unsigned char index;
+
+  uint32_t nread;
+  int64_t content_length;
+
+  /** READ-ONLY **/
+  unsigned short http_major;
+  unsigned short http_minor;
+  unsigned short status_code; /* responses only */
+  unsigned char method;    /* requests only */
+
+  /* 1 = Upgrade header was present and the parser has exited because of that.
+   * 0 = No upgrade header present.
+   * Should be checked when http_parser_execute() returns in addition to
+   * error checking.
+   */
+  char upgrade;
+
+  /** PUBLIC **/
+  void *data; /* A pointer to get hook to the "connection" or "socket" object */
+};
+
+
+struct http_parser_settings {
+  http_cb      on_message_begin;
+  http_data_cb on_path;
+  http_data_cb on_query_string;
+  http_data_cb on_url;
+  http_data_cb on_fragment;
+  http_data_cb on_header_field;
+  http_data_cb on_header_value;
+  http_cb      on_headers_complete;
+  http_data_cb on_body;
+  http_cb      on_message_complete;
+};
+
+
+void http_parser_init(http_parser *parser, enum http_parser_type type);
+
+
+size_t http_parser_execute(http_parser *parser,
+                           const http_parser_settings *settings,
+                           const char *data,
+                           size_t len);
+
+
+/* If http_should_keep_alive() in the on_headers_complete or
+ * on_message_complete callback returns true, then this will be should be
+ * the last message on the connection.
+ * If you are the server, respond with the "Connection: close" header.
+ * If you are the client, close the connection.
+ */
+int http_should_keep_alive(http_parser *parser);
+
+/* Returns a string version of the HTTP method. */
+const char *http_method_str(enum http_method);
+
+#ifdef __cplusplus
+}
+#endif
+#endif