You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ph...@apache.org on 2019/03/05 01:38:31 UTC

[nifi-minifi-cpp] branch master updated: MINIFICPP-720 - Add UT containers

This is an automated email from the ASF dual-hosted git repository.

phrocker pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git


The following commit(s) were added to refs/heads/master by this push:
     new 9da533c  MINIFICPP-720 - Add UT containers
9da533c is described below

commit 9da533cd6fb4cbff663515c49c8a62284036d275
Author: Arpad Boda <ab...@hortonworks.com>
AuthorDate: Thu Jan 24 12:14:25 2019 +0100

    MINIFICPP-720 - Add UT containers
    
    MINIFICPP-658 - Port Raw Site to Site to C
    
    Add C logging library, add logging to C RawSite2Site implementation
    
    MINIFICPP-658 - Addressed review comments
    
    This closes #482.
    
    Signed-off-by: Marc Parisi <ph...@apache.org>
---
 CMakeLists.txt                                   |    1 +
 LICENSE                                          |    4 +-
 libminifi/include/sitetosite/RawSocketProtocol.h |    3 -
 nanofi/CMakeLists.txt                            |    3 +-
 nanofi/examples/CMakeLists.txt                   |    4 +
 nanofi/examples/transmit_flow.c                  |    2 +
 nanofi/examples/transmit_payload.c               |   67 ++
 nanofi/include/core/cstream.h                    |   73 ++
 nanofi/include/core/cstructs.h                   |    2 +
 nanofi/include/core/cuuid.h                      |    8 +
 nanofi/include/core/cxxstructs.h                 |    5 +
 nanofi/include/core/log.h                        |   76 ++
 nanofi/include/sitetosite/CPeer.h                |  130 +++
 nanofi/include/sitetosite/CRawSocketProtocol.h   |  216 ++++
 nanofi/include/sitetosite/CSiteToSite.h          |  423 ++++++++
 nanofi/src/core/cstream.cpp                      |  164 +++
 nanofi/src/core/{cuuid.cpp => cuuid.c}           |    0
 nanofi/{include/core/cuuid.h => src/core/log.c}  |   17 +-
 nanofi/src/sitetosite/CPeer.c                    |   65 ++
 nanofi/src/sitetosite/CRawSocketProtocol.c       | 1059 +++++++++++++++++++
 nanofi/tests/CSite2SiteTests.cpp                 |  223 ++++
 thirdparty/ut/CMakeLists.txt                     |   20 +
 thirdparty/ut/utarray.h                          |  238 +++++
 thirdparty/ut/uthash.h                           | 1227 ++++++++++++++++++++++
 thirdparty/ut/utlist.h                           | 1073 +++++++++++++++++++
 thirdparty/ut/utringbuffer.h                     |  108 ++
 thirdparty/ut/utstack.h                          |   88 ++
 thirdparty/ut/utstring.h                         |  398 +++++++
 28 files changed, 5676 insertions(+), 21 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index a540c63..8af12f7 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -286,6 +286,7 @@ else()
 	include_directories("thirdparty/uuid/include/posix")
 endif()
 add_subdirectory("thirdparty/uuid")
+include_directories(thirdparty/ut)
 set(UUID_FOUND "YES" CACHE STRING "" FORCE)
 set(UUID_LIBRARIES "uuid" CACHE STRING "" FORCE)
 
diff --git a/LICENSE b/LICENSE
index 9f5b0bd..78e086e 100644
--- a/LICENSE
+++ b/LICENSE
@@ -1379,8 +1379,8 @@ The above copyright notice and this permission notice shall be included in all c
 
 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.
 
-This product bundles "utlist.h" from uthash:
-Copyright (c) 2007-2010, Troy D. Hanson   http://uthash.sourceforge.net
+This product bundles uthash:
+Copyright (c) 2005-2018, Troy D. Hanson http://troydhanson.github.com/uthash/
 All rights reserved.
 
 Redistribution and use in source and binary forms, with or without
diff --git a/libminifi/include/sitetosite/RawSocketProtocol.h b/libminifi/include/sitetosite/RawSocketProtocol.h
index 0c7feb5..395b597 100644
--- a/libminifi/include/sitetosite/RawSocketProtocol.h
+++ b/libminifi/include/sitetosite/RawSocketProtocol.h
@@ -178,9 +178,6 @@ class RawSiteToSiteClient : public sitetosite::SiteToSiteClient {
   virtual bool handShake();
 
  private:
-
-  // Mutex for protection
-  std::mutex mutex_;
   // Logger
   std::shared_ptr<logging::Logger> logger_;
   // Batch Count
diff --git a/nanofi/CMakeLists.txt b/nanofi/CMakeLists.txt
index 34e0942..53e2c92 100644
--- a/nanofi/CMakeLists.txt
+++ b/nanofi/CMakeLists.txt
@@ -25,6 +25,7 @@ ENDIF(POLICY CMP0048)
 
 include_directories(include)
 include_directories(../libminifi/include ../thirdparty/spdlog-20170710/include)
+include_directories(../thirdparty/ut)
 
 if(WIN32)
 include_directories(../libminifi/opsys/win)
@@ -32,7 +33,7 @@ else()
 include_directories(../libminifi/opsys/posix)
 endif()
 
-file(GLOB NANOFI_SOURCES  "src/api/*.cpp" "src/core/*.cpp" "src/cxx/*.cpp")
+file(GLOB NANOFI_SOURCES  "src/api/*.cpp" "src/core/*.c*" "src/cxx/*.cpp" "src/sitetosite/*.c*")
 
 file(GLOB NANOFI_EXAMPLES_SOURCES  "examples/*.c" )
 
diff --git a/nanofi/examples/CMakeLists.txt b/nanofi/examples/CMakeLists.txt
index 7cad4e4..18bf761 100644
--- a/nanofi/examples/CMakeLists.txt
+++ b/nanofi/examples/CMakeLists.txt
@@ -71,6 +71,10 @@ add_executable(transmit_flow transmit_flow.c)
 
 target_link_libraries(transmit_flow nanofi ${CMAKE_THREAD_LIBS_INIT}  ${LINK_FLAGS} minifi-http-curl ${LINK_END_FLAGS})
 
+add_executable(transmit_payload transmit_payload.c)
+
+target_link_libraries(transmit_payload nanofi ${CMAKE_THREAD_LIBS_INIT}  ${LINK_FLAGS} minifi-http-curl ${LINK_END_FLAGS})
+
 add_executable(monitor_directory monitor_directory.c)
 
 target_link_libraries(monitor_directory nanofi ${CMAKE_THREAD_LIBS_INIT}  ${LINK_FLAGS} minifi-http-curl ${LINK_END_FLAGS})
diff --git a/nanofi/examples/transmit_flow.c b/nanofi/examples/transmit_flow.c
index 66bedde..68bda55 100644
--- a/nanofi/examples/transmit_flow.c
+++ b/nanofi/examples/transmit_flow.c
@@ -102,3 +102,5 @@ int main(int argc, char **argv) {
 
   free_instance(instance);
 }
+
+
diff --git a/nanofi/examples/transmit_payload.c b/nanofi/examples/transmit_payload.c
new file mode 100644
index 0000000..3338b53
--- /dev/null
+++ b/nanofi/examples/transmit_payload.c
@@ -0,0 +1,67 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "api/nanofi.h"
+#include "core/log.h"
+#include "sitetosite/CPeer.h"
+#include "sitetosite/CRawSocketProtocol.h"
+
+int main(int argc, char **argv) {
+
+  if (argc < 4) {
+    printf("Error: must run ./transmit_payload <host> <port> <nifi port>\n");
+    exit(1);
+  }
+
+  set_log_level(info);
+
+  char *host = argv[1];
+  char *port_str = argv[2];
+  char *nifi_port_str = argv[3];
+
+  uint16_t port_num = atoi(port_str);
+
+  struct CRawSiteToSiteClient * client = createClient(host, port_num, nifi_port_str);
+
+  const char * payload = "Test MiNiFi payload";
+
+  attribute attribute1;
+
+  attribute1.key = "some_key";
+  const char * attr_value = "some value";
+  attribute1.value = (void *)attr_value;
+  attribute1.value_size = strlen(attr_value);
+
+  attribute_set as;
+  as.size = 1;
+  as.attributes = &attribute1;
+
+  if(transmitPayload(client, payload, &as) == 0){
+    printf("Packet successfully sent\n");
+  } else {
+    printf("Failed to send packet\n");
+  }
+
+  destroyClient(client);
+
+  return 0;
+}
\ No newline at end of file
diff --git a/nanofi/include/core/cstream.h b/nanofi/include/core/cstream.h
new file mode 100644
index 0000000..5d2c833
--- /dev/null
+++ b/nanofi/include/core/cstream.h
@@ -0,0 +1,73 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#ifndef NIFI_MINIFI_CPP_CSTREAM_H
+#define NIFI_MINIFI_CPP_CSTREAM_H
+
+#include "cstructs.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+enum Bool {True = 1, False = 0};
+
+int write_uint64_t(uint64_t value, cstream * stream);
+
+int write_uint32_t(uint32_t value, cstream * stream);
+
+int write_uint16_t(uint16_t value, cstream * stream);
+
+int write_uint8_t(uint8_t value, cstream * stream);
+
+int write_char(char value, cstream * stream);
+
+int write_buffer(const uint8_t *value, int len, cstream * stream);
+
+int writeUTF(const char * cstr, uint64_t len, enum Bool widen, cstream * stream);
+
+int read_char(char *value, cstream * stream);
+
+int read_uint8_t(uint8_t *value, cstream * stream);
+
+int read_uint16_t(uint16_t *value, cstream * stream);
+
+int read_uint32_t(uint32_t *value, cstream * stream);
+
+int read_uint64_t(uint64_t *value, cstream * stream);
+
+int read_buffer(uint8_t *value, int len, cstream * stream);
+
+int readUTFLen(uint32_t * utflen, cstream * stream);
+
+int readUTF(char * buf, uint64_t buflen, cstream * stream);
+
+void close_stream(cstream * stream);
+
+int open_stream(cstream * stream);
+
+cstream * create_socket(const char * host, uint16_t portnum);
+
+void free_socket(cstream * stream);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif //NIFI_MINIFI_CPP_CSTREAM_H
diff --git a/nanofi/include/core/cstructs.h b/nanofi/include/core/cstructs.h
index 9a78544..c137cf8 100644
--- a/nanofi/include/core/cstructs.h
+++ b/nanofi/include/core/cstructs.h
@@ -133,4 +133,6 @@ typedef struct file_buffer {
   uint64_t file_len;
 } file_buffer;
 
+typedef struct cstream cstream;
+
 #endif /* LIBMINIFI_SRC_CAPI_CSTRUCTS_H_ */
diff --git a/nanofi/include/core/cuuid.h b/nanofi/include/core/cuuid.h
index d4f6dea..ea26810 100644
--- a/nanofi/include/core/cuuid.h
+++ b/nanofi/include/core/cuuid.h
@@ -21,6 +21,10 @@
 
 #include "uuid/uuid.h"
 
+#ifdef __cplusplus
+extern "C" {
+#endif
+
 #define CUUID_TIME_IMPL 0
 #define CUUID_RANDOM_IMPL 1
 #define CUUID_DEFAULT_IMPL 2
@@ -31,4 +35,8 @@ typedef struct CIDGenerator {
 
 void generate_uuid(const CIDGenerator * generator, char * out);
 
+#ifdef __cplusplus
+}
+#endif
+
 #endif //NIFI_MINIFI_CPP_CUUID_H
diff --git a/nanofi/include/core/cxxstructs.h b/nanofi/include/core/cxxstructs.h
index d32b720..b0b90eb 100644
--- a/nanofi/include/core/cxxstructs.h
+++ b/nanofi/include/core/cxxstructs.h
@@ -21,6 +21,7 @@
 
 #include "cstructs.h"
 #include "cxx/Plan.h"
+#include "io/DataStream.h"
 
 struct flow : public ExecutionPlan {
   using ExecutionPlan::ExecutionPlan;
@@ -42,4 +43,8 @@ struct processor_context : public core::ProcessContext {
   using core::ProcessContext::ProcessContext;
 };
 
+struct cstream {
+  org::apache::nifi::minifi::io::BaseStream * impl;
+};
+
 #endif //NIFI_MINIFI_CPP_CXXSTRUCTS_H
diff --git a/nanofi/include/core/log.h b/nanofi/include/core/log.h
new file mode 100644
index 0000000..a1749d2
--- /dev/null
+++ b/nanofi/include/core/log.h
@@ -0,0 +1,76 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef NIFI_MINIFI_CPP_LOG_H
+#define NIFI_MINIFI_CPP_LOG_H
+
+#include <stdio.h>
+#include <stdlib.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+typedef enum
+{
+  off = 0,
+  critical = 1,
+  err = 2,
+  warn = 3,
+  info = 4,
+  debug = 5,
+  trace = 6,
+} log_level;
+
+extern volatile int global_log_level;
+
+static const char *log_level_str[trace+1] = { "OFF", "CRITICAL", "ERROR", "WARN", "INFO", "DEBUG", "TRACE" };
+
+#if __STDC_VERSION__ >= 199901L //C99 compiler support for __func__
+#if defined(__GNUC__)
+#define logc(level, format, ...) \
+  if (level <= global_log_level && level > off) do { \
+    fprintf(stderr, "%s:%u: [%s] %s: " format "\n", __FILE__, __LINE__, __func__, log_level_str[level], ##__VA_ARGS__); \
+  } while (0)
+#else  // no __GNUC__
+#define logc(level, format, ...) \
+  if (level <= global_log_level && level > off) do { \
+    fprintf(stderr, "%s:%u: [%s] %s: " format "\n", __FILE__, __LINE__, __func__, log_level_str[level],  __VA_ARGS__); \
+  } while (0)
+#endif //__GNUC__
+#else // no C99
+#define logc(level, ...) \
+  if (level <= global_log_level && level > off) do { \
+    fprintf(stderr, "%s:%d: %s:", __FILE__, __LINE__, log_level_str[level]); \
+            fprintf(stderr, __VA_ARGS__); \
+            fprintf(stderr, "\n"); \
+  } while (0)
+#endif //C99 compiler support
+
+static void set_log_level(log_level lvl) {
+  if(lvl >= off && lvl <= trace) {
+    global_log_level = lvl;
+    logc(info, "Log level was set to %s", log_level_str[lvl]);
+  }
+}
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif //NIFI_MINIFI_CPP_LOG_H
diff --git a/nanofi/include/sitetosite/CPeer.h b/nanofi/include/sitetosite/CPeer.h
new file mode 100644
index 0000000..47bdb63
--- /dev/null
+++ b/nanofi/include/sitetosite/CPeer.h
@@ -0,0 +1,130 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef LIBMINIFI_INCLUDE_CSITETOSITE_CPEER_H_
+#define LIBMINIFI_INCLUDE_CSITETOSITE_CPEER_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <errno.h>
+#include <uuid/uuid.h>
+#include "core/cstream.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+static const char MAGIC_BYTES[] = { 'N', 'i', 'F', 'i' };
+
+struct SiteToSiteCPeer;
+
+// open connection to the peer
+int openPeer(struct SiteToSiteCPeer * peer);
+
+// close connection to the peer
+void closePeer(struct SiteToSiteCPeer * peer);
+
+// Site2SitePeer Class
+struct SiteToSiteCPeer {
+
+  cstream * _stream;
+
+  // URL
+  char * _url;
+
+  char * _host;
+
+  uint16_t _port;
+
+  enum Bool _owns_resource;
+};
+
+static const char * getURL(const struct SiteToSiteCPeer * peer) {
+  return peer->_url;
+}
+
+static void setHostName(struct SiteToSiteCPeer * peer, const char * hostname) {
+  if(peer->_host) {
+    free(peer->_host);
+  }
+  if(peer->_url) {
+    free(peer->_url);
+  }
+  if(hostname == NULL || strlen(hostname) == 0) {
+    peer->_host = NULL;
+    peer->_url = NULL;
+    return;
+  }
+  size_t host_len = strlen(hostname);
+  peer->_host = (char*)malloc(host_len + 1); // +1 for trailing zero
+  peer->_url = (char*)malloc(host_len + 14); // +1 for trailing zero, 1 for ':', at most 5 for port, 7 for "nifi://" suffix
+  memset(peer->_url, 0, host_len + 14); // make sure to have zero padding no matter the length of the port
+  strncpy(peer->_host, hostname, host_len);
+  strncpy(peer->_url, "nifi://", 7);
+  strncpy(peer->_url + 7, hostname, host_len);
+  peer->_host[host_len] = '\0';
+  peer->_url[host_len + 7] = ':';
+  if(peer->_port != 0) {
+    snprintf(peer->_url + host_len + 8, 5, "%d", peer->_port);
+  }
+  return;
+}
+
+static void setPort(struct SiteToSiteCPeer * peer, uint16_t port) {
+  peer->_port = port;
+  if(peer->_url != NULL) {
+    int i;
+    for(i = strlen(peer->_url) -1; i >= 0; --i) { // look for the last ':' in the string
+      if(peer->_url[i] == ':'){
+        memset(peer->_url + i + 1, 0, 6); // zero the port area  - the new port can be shorter
+        snprintf(peer->_url + i + 1, 6, "%d", peer->_port);
+        break;
+      }
+    }
+  }
+}
+
+static void initPeer(struct SiteToSiteCPeer * peer, cstream * injected_socket, const char * host, uint16_t port, const char * ifc) {
+  peer->_stream = injected_socket;
+  //peer->local_network_interface_= std::move(io::NetworkInterface(ifc, nullptr));
+  peer->_host = NULL;
+  peer->_url = NULL;
+  peer->_port = 0;
+  setHostName(peer, host);
+  setPort(peer, port);
+
+  if(peer->_stream == NULL) {
+    peer->_owns_resource = True;
+  }
+}
+
+static void freePeer(struct SiteToSiteCPeer * peer) {
+  closePeer(peer);
+  setHostName(peer, NULL);
+
+  if(peer->_owns_resource == True && peer->_stream != NULL) {
+    free_socket(peer->_stream);
+    peer->_stream = NULL;
+  }
+}
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* LIBMINIFI_INCLUDE_CSITETOSITE_CPEER_H_ */
diff --git a/nanofi/include/sitetosite/CRawSocketProtocol.h b/nanofi/include/sitetosite/CRawSocketProtocol.h
new file mode 100644
index 0000000..f99b8bd
--- /dev/null
+++ b/nanofi/include/sitetosite/CRawSocketProtocol.h
@@ -0,0 +1,216 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __CSITE2SITE_CLIENT_PROTOCOL_H__
+#define __CSITE2SITE_CLIENT_PROTOCOL_H__
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <ctype.h>
+#include <uuid/uuid.h>
+
+#include "api/nanofi.h"
+#include "CSiteToSite.h"
+#include "CPeer.h"
+
+#include "uthash.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#define DESCRIPTION_BUFFER_SIZE 2048
+
+struct CRawSiteToSiteClient;
+
+int readResponse(struct CRawSiteToSiteClient* client, RespondCode *code);
+
+int writeResponse(struct CRawSiteToSiteClient* client, RespondCode code, const char * message);
+
+int readRequestType(struct CRawSiteToSiteClient* client, RequestType *type);
+
+int writeRequestType(struct CRawSiteToSiteClient* client, RequestType type);
+
+void tearDown(struct CRawSiteToSiteClient* client);
+
+int initiateResourceNegotiation(struct CRawSiteToSiteClient* client);
+
+int initiateCodecResourceNegotiation(struct CRawSiteToSiteClient* client);
+
+int negotiateCodec(struct CRawSiteToSiteClient* client);
+
+int establish(struct CRawSiteToSiteClient* client);
+
+void addTransaction(struct CRawSiteToSiteClient * client, CTransaction * transaction);
+
+CTransaction * findTransaction(const struct CRawSiteToSiteClient * client, const char * id);
+
+void deleteTransaction(struct CRawSiteToSiteClient * client, const char * id);
+
+void clearTransactions(struct CRawSiteToSiteClient * client);
+
+int handShake(struct CRawSiteToSiteClient * client);
+
+int bootstrap(struct CRawSiteToSiteClient * client);
+
+int complete(struct CRawSiteToSiteClient * client, const char * transactionID);
+
+int confirm(struct CRawSiteToSiteClient * client, const char * transactionID);
+
+int transmitPayload(struct CRawSiteToSiteClient * client, const char * payload, const attribute_set * attributes);
+
+int16_t sendPacket(struct CRawSiteToSiteClient * client, const char * transactionID, CDataPacket *packet, flow_file_record * ff);
+
+CTransaction* createTransaction(struct CRawSiteToSiteClient * client, TransferDirection direction);
+
+static const char * getResourceName(const struct CRawSiteToSiteClient * c) {
+  return "SocketFlowFileProtocol";
+}
+
+static const char * getCodecResourceName(const struct CRawSiteToSiteClient * c) {
+  return "StandardFlowFileCodec";
+}
+
+static RespondCodeContext *getRespondCodeContext(RespondCode code) {
+  unsigned int i;
+  for ( i = 0; i < sizeof(respondCodeContext) / sizeof(RespondCodeContext); i++) {
+    if (respondCodeContext[i].code == code) {
+      return &respondCodeContext[i];
+    }
+  }
+  return NULL;
+}
+
+// RawSiteToSiteClient Class
+struct CRawSiteToSiteClient {
+  // Batch Count
+  uint64_t _batch_count;
+  // Batch Size
+  uint64_t _batch_size;
+  // Batch Duration in msec
+  uint64_t _batch_duration;
+  // Timeout in msec
+  uint64_t _timeout;
+
+  // commsIdentifier
+  char _commsIdentifier[37];
+
+  // Peer State
+  PeerState _peer_state;
+
+  // portIDStr
+  char _port_id_str[37];
+
+  char _description_buffer[DESCRIPTION_BUFFER_SIZE]; //should be big enough
+
+  // Peer Connection
+  struct SiteToSiteCPeer* _peer;
+
+  // Indicatates if _peer is owned by the client
+  enum Bool _owns_resource;
+
+
+  CTransaction * _known_transactions;
+
+  // BATCH_SEND_NANOS
+  uint64_t _batchSendNanos;
+
+  /***
+   * versioning
+   */
+  uint32_t _supportedVersion[5];
+  uint32_t _currentVersion;
+  int _currentVersionIndex;
+  uint32_t _supportedCodecVersion[1];
+  uint32_t _currentCodecVersion;
+  int _currentCodecVersionIndex;
+};
+
+static const char * getPortId(const struct CRawSiteToSiteClient * client) {
+  return client->_port_id_str;
+}
+
+static void setPortId(struct CRawSiteToSiteClient * client, const char * id) {
+  strncpy(client->_port_id_str, id, 37);
+  client->_port_id_str[36] = '\0';
+  int i;
+  for(i = 0; i < 37; i++){
+    client->_port_id_str[i] = tolower(client->_port_id_str[i]);
+  }
+}
+
+static void setBatchSize(struct CRawSiteToSiteClient *client, uint64_t size) {
+  client->_batch_size = size;
+}
+
+static void setBatchCount(struct CRawSiteToSiteClient *client, uint64_t count) {
+  client->_batch_count = count;
+}
+
+static void setBatchDuration(struct CRawSiteToSiteClient *client, uint64_t duration) {
+  client->_batch_duration = duration;
+}
+
+static uint64_t getTimeOut(const struct CRawSiteToSiteClient *client) {
+  return client->_timeout;
+}
+
+static void initRawClient(struct CRawSiteToSiteClient *client, struct SiteToSiteCPeer * peer) {
+  client->_owns_resource = False;
+  client->_peer = peer;
+  client->_peer_state = IDLE;
+  client->_batch_size = 0;
+  client->_batch_count = 0;
+  client->_batch_duration = 0;
+  client->_batchSendNanos = 5000000000;  // 5 seconds
+  client->_timeout = 30000;  // 30 seconds
+  client->_supportedVersion[0] = 5;
+  client->_supportedVersion[1] = 4;
+  client->_supportedVersion[2] = 3;
+  client->_supportedVersion[3] = 2;
+  client->_supportedVersion[4] = 1;
+  client->_currentVersion = client->_supportedVersion[0];
+  client->_currentVersionIndex = 0;
+  client->_supportedCodecVersion[0] = 1;
+  client->_currentCodecVersion = client->_supportedCodecVersion[0];
+  client->_currentCodecVersionIndex = 0;
+  client->_known_transactions = NULL;
+  memset(client->_description_buffer, 0, DESCRIPTION_BUFFER_SIZE);
+}
+
+static struct CRawSiteToSiteClient* createClient(const char * host, uint16_t port, const char * nifi_port) {
+  struct SiteToSiteCPeer * peer = (struct SiteToSiteCPeer *)malloc(sizeof(struct SiteToSiteCPeer));
+  initPeer(peer, NULL, host, port, "");
+  struct CRawSiteToSiteClient* client = (struct CRawSiteToSiteClient*)malloc(sizeof(struct CRawSiteToSiteClient));
+  initRawClient(client, peer);
+  client->_owns_resource = True;
+  setPortId(client, nifi_port);
+  return client;
+}
+
+static void destroyClient(struct CRawSiteToSiteClient * client){
+  tearDown(client);
+  if(client->_owns_resource == True) {
+    freePeer(client->_peer);
+  }
+}
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif
diff --git a/nanofi/include/sitetosite/CSiteToSite.h b/nanofi/include/sitetosite/CSiteToSite.h
new file mode 100644
index 0000000..8728cef
--- /dev/null
+++ b/nanofi/include/sitetosite/CSiteToSite.h
@@ -0,0 +1,423 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CSITETOSITE_CSITETOSITE_H_
+#define LIBMINIFI_INCLUDE_CORE_CSITETOSITE_CSITETOSITE_H_
+
+#include <zlib.h>
+#include <string.h>
+#include "uthash.h"
+#include "CPeer.h"
+#include "core/cstream.h"
+#include "core/cuuid.h"
+
+#ifdef WIN32
+#include <winsock2.h>
+#else
+#include <arpa/inet.h>
+#endif
+
+#if defined(__GNUC__) || defined(__GNUG__)
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wunused-variable"
+#endif
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#define htonll_r(x) ((((uint64_t)htonl(x)) << 32) + htonl((x) >> 32))
+
+// Resource Negotiated Status Code
+#define RESOURCE_OK 20
+#define DIFFERENT_RESOURCE_VERSION 21
+#define NEGOTIATED_ABORT 255
+// ! Max attributes
+#define MAX_NUM_ATTRIBUTES 25000
+
+// Respond Code Sequence Pattern
+static const uint8_t CODE_SEQUENCE_VALUE_1 = (uint8_t) 'R';
+static const uint8_t CODE_SEQUENCE_VALUE_2 = (uint8_t) 'C';
+
+/**
+ * Enumeration of Properties that can be used for the Site-to-Site Socket
+ * Protocol.
+ */
+typedef enum {
+  /**
+   * Boolean value indicating whether or not the contents of a FlowFile should
+   * be GZipped when transferred.
+   */
+  GZIP,
+  /**
+   * The unique identifier of the port to communicate with
+   */
+  PORT_IDENTIFIER,
+  /**
+   * Indicates the number of milliseconds after the request was made that the
+   * client will wait for a response. If no response has been received by the
+   * time this value expires, the server can move on without attempting to
+   * service the request because the client will have already disconnected.
+   */
+  REQUEST_EXPIRATION_MILLIS,
+  /**
+   * The preferred number of FlowFiles that the server should send to the
+   * client when pulling data. This property was introduced in version 5 of
+   * the protocol.
+   */
+  BATCH_COUNT,
+  /**
+   * The preferred number of bytes that the server should send to the client
+   * when pulling data. This property was introduced in version 5 of the
+   * protocol.
+   */
+  BATCH_SIZE,
+  /**
+   * The preferred amount of time that the server should send data to the
+   * client when pulling data. This property was introduced in version 5 of
+   * the protocol. Value is in milliseconds.
+   */
+  BATCH_DURATION,
+  MAX_HANDSHAKE_PROPERTY
+} HandshakeProperty;
+
+typedef enum {
+  RAW,
+  HTTP
+} CLIENT_TYPE;
+
+/**
+ * An enumeration for specifying the direction in which data should be
+ * transferred between a client and a remote NiFi instance.
+ */
+typedef enum {
+  /**
+   * * The client is to send data to the remote instance.
+   * */
+  SEND,
+  /**
+   * * The client is to receive data from the remote instance.
+   * */
+  RECEIVE
+} TransferDirection;
+
+// Peer State
+typedef enum {
+  /**
+   * * IDLE
+   * */
+  IDLE = 0,
+  /**
+   * * Socket Established
+   * */
+  ESTABLISHED,
+  /**
+   * * HandShake Done
+   * */
+  HANDSHAKED,
+  /**
+   * * After CodeDec Completion
+   * */
+  READY
+} PeerState;
+
+// Transaction State
+typedef enum {
+  /**
+   * * Transaction has been started but no data has been sent or received.
+   * */
+  TRANSACTION_STARTED,
+  /**
+   * * Transaction has been started and data has been sent or received.
+   * */
+  DATA_EXCHANGED,
+  /**
+   * * Data that has been transferred has been confirmed via its CRC.
+   * * Transaction is ready to be completed.
+   * */
+  TRANSACTION_CONFIRMED,
+  /**
+   * * Transaction has been successfully completed.
+   * */
+  TRANSACTION_COMPLETED,
+  /**
+   * * The Transaction has been canceled.
+   * */
+  TRANSACTION_CANCELED,
+
+  /**
+   * * Transaction has been successfully closed.
+   * */
+  TRANSACTION_CLOSED,
+  /**
+   * * The Transaction ended in an error.
+   * */
+  TRANSACTION_ERROR
+} TransactionState;
+
+// Request Type
+typedef enum {
+  NEGOTIATE_FLOWFILE_CODEC = 0,
+  REQUEST_PEER_LIST,
+  SEND_FLOWFILES,
+  RECEIVE_FLOWFILES,
+  SHUTDOWN,
+  MAX_REQUEST_TYPE
+} RequestType;
+
+
+// Respond Code
+typedef enum {
+  RESERVED = 0,
+  // ResponseCode, so that we can indicate a 0 followed by some other bytes
+
+  // handshaking properties
+  PROPERTIES_OK = 1,
+  UNKNOWN_PROPERTY_NAME = 230,
+  ILLEGAL_PROPERTY_VALUE = 231,
+  MISSING_PROPERTY = 232,
+  // transaction indicators
+  CONTINUE_TRANSACTION = 10,
+  FINISH_TRANSACTION = 11,
+  CONFIRM_TRANSACTION = 12,// "Explanation" of this code is the checksum
+  TRANSACTION_FINISHED = 13,
+  TRANSACTION_FINISHED_BUT_DESTINATION_FULL = 14,
+  CANCEL_TRANSACTION = 15,
+  BAD_CHECKSUM = 19,
+  // data availability indicators
+  MORE_DATA = 20,
+  NO_MORE_DATA = 21,
+  // port state indicators
+  UNKNOWN_PORT = 200,
+  PORT_NOT_IN_VALID_STATE = 201,
+  PORTS_DESTINATION_FULL = 202,
+  // authorization
+  UNAUTHORIZED = 240,
+  // error indicators
+  ABORT = 250,
+  UNRECOGNIZED_RESPONSE_CODE = 254,
+  END_OF_STREAM = 255
+}RespondCode;
+
+// Respond Code Class
+typedef struct {
+  RespondCode code;
+  const char *description; int hasDescription;
+} RespondCodeContext;
+
+
+
+// Request Type Str
+static const char *RequestTypeStr[MAX_REQUEST_TYPE] = { "NEGOTIATE_FLOWFILE_CODEC", "REQUEST_PEER_LIST", "SEND_FLOWFILES", "RECEIVE_FLOWFILES", "SHUTDOWN" };
+static RespondCodeContext respondCodeContext[21] = { //NOLINT
+  { RESERVED, "Reserved for Future Use", 0 },  //NOLINT
+  { PROPERTIES_OK, "Properties OK", 0 },  //NOLINT
+  { UNKNOWN_PROPERTY_NAME, "Unknown Property Name", 1 },  //NOLINT
+  { ILLEGAL_PROPERTY_VALUE, "Illegal Property Value", 1 },  //NOLINT
+  { MISSING_PROPERTY, "Missing Property", 1 },  //NOLINT
+  { CONTINUE_TRANSACTION, "Continue Transaction", 0 },  //NOLINT
+  { FINISH_TRANSACTION, "Finish Transaction", 0 },  //NOLINT
+  { CONFIRM_TRANSACTION, "Confirm Transaction", 1 },  //NOLINT
+  { TRANSACTION_FINISHED, "Transaction Finished", 0 },  //NOLINT
+  { TRANSACTION_FINISHED_BUT_DESTINATION_FULL, "Transaction Finished But Destination is Full", 0 },  //NOLINT
+  { CANCEL_TRANSACTION, "Cancel Transaction", 1 },  //NOLINT
+  { BAD_CHECKSUM, "Bad Checksum", 0 },  //NOLINT
+  { MORE_DATA, "More Data Exists", 0 },  //NOLINT
+  { NO_MORE_DATA, "No More Data Exists", 0 },  //NOLINT
+  { UNKNOWN_PORT, "Unknown Port", 0 },  //NOLINT
+  { PORT_NOT_IN_VALID_STATE, "Port Not in a Valid State", 1 },  //NOLINT
+  { PORTS_DESTINATION_FULL, "Port's Destination is Full", 0 },  //NOLINT
+  { UNAUTHORIZED, "User Not Authorized", 1 },  //NOLINT
+  { ABORT, "Abort", 1 },  //NOLINT
+  { UNRECOGNIZED_RESPONSE_CODE, "Unrecognized Response Code", 0 },  //NOLINT
+  { END_OF_STREAM, "End of Stream", 0 }
+};
+
+
+
+// Transaction Class
+typedef struct {
+
+  // Number of current transfers
+  int current_transfers_;
+  // number of total seen transfers
+  int total_transfers_;
+
+  // Number of content bytes
+  uint64_t _bytes;
+
+  // Transaction State
+  TransactionState _state;
+
+  // Whether received data is available
+  int _dataAvailable;
+
+  //org::apache::nifi::minifi::io::BaseStream* _stream;
+  cstream * _stream;
+
+  uint64_t _crc;
+
+  char _uuid_str[37];
+
+  TransferDirection _direction;
+
+  UT_hash_handle hh;
+} CTransaction;
+
+static void InitTransaction(CTransaction * transaction, TransferDirection direction, cstream * stream) {
+  transaction->_stream = stream;
+  transaction->_state = TRANSACTION_STARTED;
+  transaction->_direction = direction;
+  transaction->_dataAvailable = 0;
+  transaction->current_transfers_ = 0;
+  transaction->total_transfers_ = 0;
+  transaction->_bytes = 0;
+  transaction->_crc = 0;
+
+  CIDGenerator gen;
+  gen.implementation_ = CUUID_DEFAULT_IMPL;
+  generate_uuid(&gen, transaction->_uuid_str);
+  transaction->_uuid_str[36]='\0';
+}
+
+static TransferDirection getDirection(const CTransaction * transaction) {
+  return transaction->_direction;
+}
+
+static TransactionState getState(const CTransaction * transaction) {
+  return transaction->_state;
+}
+
+static int isDataAvailable(const CTransaction * transaction) {
+  return transaction->_dataAvailable;
+}
+
+static void setDataAvailable(CTransaction * transaction, int available) {
+  transaction->_dataAvailable = available;
+}
+
+static uint64_t getCRC(const CTransaction * transaction) {
+  return transaction->_crc;
+}
+
+static const char * getUUIDStr(const CTransaction * transation) {
+  return transation->_uuid_str;
+}
+
+static void updateCRC(CTransaction * transaction, const uint8_t *buffer, uint32_t length) {
+  transaction->_crc = crc32(transaction->_crc, buffer, length);
+}
+
+static int writeData(CTransaction * transaction, const uint8_t *value, int size) {
+  int ret = write_buffer(value, size, transaction->_stream);
+  transaction->_crc = crc32(transaction->_crc, value, size);
+  return ret;
+}
+
+static int readData(CTransaction * transaction, uint8_t *buf, int buflen) {
+  //int ret = transaction->_stream->read(buf, buflen);
+  int ret = read_buffer(buf, buflen, transaction->_stream);
+  transaction->_crc = crc32(transaction->_crc, buf, ret);
+  return ret;
+}
+
+static int is_little_endian() {
+  static unsigned int x = 1;
+  static char *c = (char*) &x;
+  return  (*c == 1) ? 1 : 0;
+}
+
+static int write_uint64t(CTransaction * transaction, uint64_t base_value) {
+  const uint64_t value = is_little_endian() == 1 ? htonll_r(base_value) : base_value;
+  const uint8_t * buf = (uint8_t*)(&value);
+
+  return writeData(transaction, buf, sizeof(uint64_t));
+}
+
+static int write_uint32t(CTransaction * transaction, uint32_t base_value) {
+  const uint32_t value = is_little_endian() == 1 ? htonl(base_value) : base_value;
+  const uint8_t * buf = (uint8_t*)(&value);
+
+  return writeData(transaction, buf, sizeof(uint32_t));
+}
+
+static int write_uint16t(CTransaction * transaction, uint16_t base_value) {
+  const uint16_t value = is_little_endian() == 1 ? htons(base_value) : base_value;
+  const uint8_t *buf = (uint8_t *) (&value);
+
+  return writeData(transaction, buf, sizeof(uint16_t));
+}
+
+static int write_UTF_len(CTransaction * transaction, const char * str, size_t len, enum Bool widen) {
+  if (len > 65535) {
+    return -1;
+  }
+
+  int ret;
+  if (!widen) {
+    uint16_t shortlen = len;
+    ret = write_uint16t(transaction, shortlen);
+  } else {
+    ret = write_uint32t(transaction, len);
+  }
+
+  if(len == 0 || ret < 0) {
+    return ret;
+  }
+
+  const uint8_t *underlyingPtr = (const uint8_t *)str;
+
+  if (!widen) {
+    uint16_t short_length = len;
+    ret = writeData(transaction, underlyingPtr, short_length);
+  } else {
+    ret = writeData(transaction, underlyingPtr, len);
+  }
+  return ret;
+}
+
+static int write_UTF(CTransaction * transaction, const char * str, enum Bool widen) {
+  //return transaction->_stream->writeUTF(str, widen);
+  return write_UTF_len(transaction, str, strlen(str), widen);
+}
+
+/**
+* Represents a piece of data that is to be sent to or that was received from a
+* NiFi instance.
+*/
+typedef struct {
+  const attribute_set * _attributes;
+  CTransaction* transaction_;
+  const char * payload_;
+} CDataPacket;
+
+static void initPacket(CDataPacket * packet, CTransaction* transaction, const attribute_set * attributes, const char * payload) {
+  packet->payload_ = payload;
+  packet->transaction_ = transaction;
+  packet->_attributes = attributes;
+}
+
+
+#if defined(__GNUC__) || defined(__GNUG__)
+#pragma GCC diagnostic pop
+#endif
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CSITETOSITE_CSITETOSITE_H_ */
diff --git a/nanofi/src/core/cstream.cpp b/nanofi/src/core/cstream.cpp
new file mode 100644
index 0000000..a463437
--- /dev/null
+++ b/nanofi/src/core/cstream.cpp
@@ -0,0 +1,164 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "api/nanofi.h"
+#include "core/cstructs.h"
+#include "core/cxxstructs.h"
+#include "core/cstream.h"
+#include "io/BaseStream.h"
+#include "io/DataStream.h"
+#include "io/ClientSocket.h"
+#include "cxx/Instance.h"
+
+int write_uint64_t(uint64_t value, cstream * stream) {
+  return stream->impl->Serializable::write(value, stream->impl);
+}
+int write_uint32_t(uint32_t value, cstream * stream) {
+  return stream->impl->Serializable::write(value, stream->impl);
+}
+int write_uint16_t(uint16_t value, cstream * stream) {
+  return stream->impl->Serializable::write(value, stream->impl);
+}
+int write_uint8_t(uint8_t value, cstream * stream) {
+  return stream->impl->Serializable::write(value, stream->impl);
+}
+int write_char(char value, cstream * stream) {
+  return stream->impl->Serializable::write(value, stream->impl);
+}
+int write_buffer(const uint8_t *value, int len, cstream * stream) {
+  int ret_val = stream->impl->Serializable::write(const_cast<uint8_t *>(value), len, stream->impl);
+  return ret_val;
+}
+
+int writeUTF(const char * cstr, uint64_t len, Bool widen, cstream * stream) {
+  std::string str(cstr, len);
+  return stream->impl->Serializable::writeUTF(str, stream->impl, widen == True);
+}
+
+int read_char(char *value, cstream * stream) {
+  char val;
+  int ret = stream->impl->Serializable::read(val, stream->impl);
+  if(ret == sizeof(char)) {
+    *value = val;
+  }
+  return ret;
+}
+int read_uint8_t(uint8_t *value, cstream * stream) {
+  uint8_t val;
+  int ret = stream->impl->Serializable::read(val, stream->impl);
+  if(ret == sizeof(uint8_t)) {
+    *value = val;
+  }
+  return ret;
+}
+int read_uint16_t(uint16_t *value, cstream * stream) {
+  uint16_t val;
+  int ret = stream->impl->Serializable::read(val, stream->impl);
+  if(ret == sizeof(uint16_t)) {
+    *value = val;
+  }
+  return ret;
+}
+int read_uint32_t(uint32_t *value, cstream * stream) {
+  uint32_t val;
+  int ret = stream->impl->Serializable::read(val, stream->impl);
+  if(ret == sizeof(uint32_t)) {
+    *value = val;
+  }
+  return ret;
+}
+int read_uint64_t(uint64_t *value, cstream * stream) {
+  uint64_t val;
+  int ret = stream->impl->Serializable::read(val, stream->impl);
+  if(ret == sizeof(uint64_t)) {
+    *value = val;
+  }
+  return ret;
+}
+
+int read_buffer(uint8_t *value, int len, cstream * stream) {
+  return stream->impl->Serializable::read(value, len, stream->impl);
+}
+
+int readUTFLen(uint32_t * utflen, cstream * stream) {
+  int ret = 1;
+  uint16_t shortLength = 0;
+  ret = read_uint16_t(&shortLength, stream);
+  if (ret > 0) {
+    *utflen = shortLength;
+  }
+  return ret;
+}
+
+int readUTF(char * buf, uint64_t buflen, cstream * stream) {
+  return stream->impl->readData((uint8_t*)buf, buflen);
+}
+
+void close_stream(cstream * stream) {
+  if(stream != NULL && stream->impl != NULL) {
+    stream->impl->closeStream();
+  }
+}
+
+int open_stream(cstream * stream) {
+  if(stream != NULL && stream->impl != NULL) {
+    return stream->impl->initialize();
+  }
+  return -1;
+}
+
+cstream * create_socket(const char * host, uint16_t portnum) {
+  nifi_port nport;
+
+  char random_port[6] = "65443";
+
+  nport.port_id = random_port;
+
+  nifi_instance *instance = create_instance(host, &nport);
+
+  auto minifi_instance_ref = static_cast<minifi::Instance*>(instance->instance_ptr);
+
+  auto stream_factory_ = minifi::io::StreamFactory::getInstance(minifi_instance_ref->getConfiguration());
+
+  cstream * stream = (cstream*)malloc(sizeof(cstream));
+
+  auto socket = stream_factory_->createSocket(host, portnum);
+
+  free_instance(instance);
+
+  if(socket) {
+    if(socket->initialize() == 0) {
+      stream->impl = socket.release();
+      return stream;
+    }
+  }
+  return NULL;
+}
+
+void free_socket(cstream * stream) {
+  if(stream != NULL) {
+    if(stream->impl != NULL) {
+      auto socket = static_cast<minifi::io::Socket*>(stream->impl);
+      if(socket) {
+        socket->closeStream();
+        delete socket; //This is ugly, but only sockets get deleted this way
+      }
+    }
+    free(stream);
+  }
+}
diff --git a/nanofi/src/core/cuuid.cpp b/nanofi/src/core/cuuid.c
similarity index 100%
rename from nanofi/src/core/cuuid.cpp
rename to nanofi/src/core/cuuid.c
diff --git a/nanofi/include/core/cuuid.h b/nanofi/src/core/log.c
similarity index 70%
copy from nanofi/include/core/cuuid.h
copy to nanofi/src/core/log.c
index d4f6dea..9b795c3 100644
--- a/nanofi/include/core/cuuid.h
+++ b/nanofi/src/core/log.c
@@ -16,19 +16,6 @@
  * limitations under the License.
  */
 
-#ifndef NIFI_MINIFI_CPP_CUUID_H
-#define NIFI_MINIFI_CPP_CUUID_H
+#include "core/log.h"
 
-#include "uuid/uuid.h"
-
-#define CUUID_TIME_IMPL 0
-#define CUUID_RANDOM_IMPL 1
-#define CUUID_DEFAULT_IMPL 2
-
-typedef struct CIDGenerator {
-  int implementation_;
-} CIDGenerator;
-
-void generate_uuid(const CIDGenerator * generator, char * out);
-
-#endif //NIFI_MINIFI_CPP_CUUID_H
+volatile int global_log_level = 0;
\ No newline at end of file
diff --git a/nanofi/src/sitetosite/CPeer.c b/nanofi/src/sitetosite/CPeer.c
new file mode 100644
index 0000000..fb1f417
--- /dev/null
+++ b/nanofi/src/sitetosite/CPeer.c
@@ -0,0 +1,65 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <stdio.h>
+#include <string.h>
+#include "sitetosite/CPeer.h"
+#include "core/log.h"
+
+
+int openPeer(struct SiteToSiteCPeer * peer) {
+  if (peer->_host == NULL || strlen(peer->_host) == 0) {
+    logc(err, "%s", "no valid host is specified");
+    return -1;
+  }
+
+  //In case there was no socket injected, let's create it
+  if(peer->_stream == NULL && peer->_owns_resource == True) {
+    peer->_stream = create_socket(peer->_host, peer->_port);
+    if(peer->_stream == NULL) {
+      logc(err, "%s", "failed to open socket");
+      return -1;
+    }
+  }
+
+  /**
+   * We may override the interface provided to us within the socket in this step; however, this is a
+   * known configuration path, and thus we will allow the RPG configuration to override anything provided to us
+   * previously by the socket preference.
+   */
+
+  // TODO: support provided interface
+
+  if(open_stream(peer->_stream) != 0) {
+    logc(err, "%s", "failed to open stream");
+    return -1;
+  }
+
+  uint16_t data_size = sizeof MAGIC_BYTES;
+
+  if(write_buffer((uint8_t*)MAGIC_BYTES, data_size, peer->_stream) != data_size) {
+    logc(err, "%s", "failed to write buffer");
+    return -1;
+  }
+
+  logc(info, "%s", "successfully openned peer");
+  return 0;
+}
+
+void closePeer(struct SiteToSiteCPeer * peer) {
+  close_stream(peer->_stream);
+}
diff --git a/nanofi/src/sitetosite/CRawSocketProtocol.c b/nanofi/src/sitetosite/CRawSocketProtocol.c
new file mode 100644
index 0000000..e89a36c
--- /dev/null
+++ b/nanofi/src/sitetosite/CRawSocketProtocol.c
@@ -0,0 +1,1059 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+
+#include "uthash.h"
+#include "sitetosite/CRawSocketProtocol.h"
+#include "sitetosite/CPeer.h"
+
+#include "core/cstream.h"
+
+#include "api/nanofi.h"
+#include "core/log.h"
+
+static const char *HandShakePropertyStr[MAX_HANDSHAKE_PROPERTY] = {
+/**
+ * Boolean value indicating whether or not the contents of a FlowFile should
+ * be GZipped when transferred.
+ */
+"GZIP",
+/**
+ * The unique identifier of the port to communicate with
+ */
+"PORT_IDENTIFIER",
+/**
+ * Indicates the number of milliseconds after the request was made that the
+ * client will wait for a response. If no response has been received by the
+ * time this value expires, the server can move on without attempting to
+ * service the request because the client will have already disconnected.
+ */
+"REQUEST_EXPIRATION_MILLIS",
+/**
+ * The preferred number of FlowFiles that the server should send to the
+ * client when pulling data. This property was introduced in version 5 of
+ * the protocol.
+ */
+"BATCH_COUNT",
+/**
+ * The preferred number of bytes that the server should send to the client
+ * when pulling data. This property was introduced in version 5 of the
+ * protocol.
+ */
+"BATCH_SIZE",
+/**
+ * The preferred amount of time that the server should send data to the
+ * client when pulling data. This property was introduced in version 5 of
+ * the protocol. Value is in milliseconds.
+ */
+"BATCH_DURATION" };
+
+typedef struct {
+  const char * name;
+  char value[40];
+  UT_hash_handle hh;
+} PropertyValue;
+
+int handShake(struct CRawSiteToSiteClient * client) {
+  if (client->_peer_state != ESTABLISHED) {
+    //client->logger_->log_error("Site2Site peer state is not established while handshake");
+    return -1;
+  }
+  //client->logger_->log_debug("Site2Site Protocol Perform hand shake with destination port %s", client->_port_id_str);
+
+  CIDGenerator gen;
+  gen.implementation_ = CUUID_DEFAULT_IMPL;
+  generate_uuid(&gen, client->_commsIdentifier);
+  client->_commsIdentifier[36]='\0';
+
+  int ret = writeUTF(client->_commsIdentifier, strlen(client->_commsIdentifier), False, client->_peer->_stream);
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  uint32_t prop_size;
+  PropertyValue *current, *tmp, * properties = NULL;
+
+  current = (PropertyValue *)malloc(sizeof(PropertyValue));
+
+  current->name = HandShakePropertyStr[GZIP];
+  strncpy(current->value, "false", strlen("false") +1);
+
+  HASH_ADD_KEYPTR(hh, properties, current->name, strlen(current->name), current);
+
+  current = (PropertyValue *)malloc(sizeof(PropertyValue));
+
+  current->name = HandShakePropertyStr[PORT_IDENTIFIER];
+  strncpy(current->value, client->_port_id_str, strlen(client->_port_id_str) +1);
+
+  HASH_ADD_KEYPTR(hh, properties, current->name, strlen(current->name), current);
+
+  current = (PropertyValue *)malloc(sizeof(PropertyValue));
+
+  current->name = HandShakePropertyStr[REQUEST_EXPIRATION_MILLIS];
+  sprintf(current->value, "%llu", client->_timeout);
+
+  HASH_ADD_KEYPTR(hh, properties, current->name, strlen(current->name), current);
+
+  prop_size = 3;
+
+  if (client->_currentVersion >= 5) {
+    if (client->_batch_count > 0) {
+      current = (PropertyValue *)malloc(sizeof(PropertyValue));
+
+      current->name = HandShakePropertyStr[BATCH_COUNT];
+      sprintf(current->value, "%llu", client->_batch_count);
+
+      HASH_ADD_KEYPTR(hh, properties, current->name, strlen(current->name), current);
+
+      prop_size++;
+    }
+    if (client->_batch_size > 0) {
+      current = (PropertyValue *)malloc(sizeof(PropertyValue));
+
+      current->name = HandShakePropertyStr[BATCH_SIZE];
+      sprintf(current->value, "%llu", client->_batch_size);
+
+      HASH_ADD_KEYPTR(hh, properties, current->name, strlen(current->name), current);
+
+      prop_size++;
+    }
+    if (client->_batch_duration > 0) {
+      current = (PropertyValue *)malloc(sizeof(PropertyValue));
+
+      current->name = HandShakePropertyStr[BATCH_DURATION];
+      sprintf(current->value, "%llu", client->_batch_duration);
+
+      HASH_ADD_KEYPTR(hh, properties, current->name, strlen(current->name), current);
+
+      prop_size++;
+    }
+  }
+
+  int ret_val = 0;
+
+  if (client->_currentVersion >= 3) {
+
+    //ret = client->_peer->writeUTF(client->_peer->getURL());
+    const char * urlstr = getURL(client->_peer);
+    ret = writeUTF(urlstr, strlen(urlstr), False, client->_peer->_stream);
+    if (ret <= 0) {
+      ret_val = -1;
+    }
+  }
+
+  if(ret_val == 0) {
+    ret = write_uint32_t(prop_size, client->_peer->_stream);
+  }
+  if (ret <= 0) {
+    ret_val = -1;
+  }
+
+  HASH_ITER(hh, properties, current, tmp) {
+    if(ret_val == 0 && writeUTF(current->name, strlen(current->name), False, client->_peer->_stream) <= 0) {
+      ret_val = -1;
+    }
+    if(ret_val == 0 && writeUTF(current->value, strlen(current->value), False, client->_peer->_stream) <= 0) {
+      ret_val = -1;
+    }
+    logc(debug, "Site2Site Protocol Send handshake properties %s %s", current->name, current->value);
+    HASH_DEL(properties, current);
+    free(current);
+  }
+
+  if(ret_val < 0) {
+    logc(err, "%s", "Failed to transfer handshake properties");
+    return -1;
+  }
+
+  RespondCode code;
+
+  ret = readResponse(client, &code);
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  RespondCodeContext *resCode = getRespondCodeContext(code);
+
+  if(resCode == NULL) {
+    logc(err, "Received invalid respond code: %d", code);
+    return -1;
+  }
+
+  if (resCode->hasDescription) {
+    uint32_t utflen;
+    ret = readUTFLen(&utflen, client->_peer->_stream);
+    if (ret <= 0)
+      return -1;
+
+    memset(client->_description_buffer, 0, utflen+1);
+    ret = readUTF(client->_description_buffer, utflen, client->_peer->_stream);
+    if (ret <= 0)
+      return -1;
+  }
+
+  const char * error = "";
+
+  switch (code) {
+    case PROPERTIES_OK:
+      logc(debug, "%s", "Site2Site HandShake Completed");
+      client->_peer_state = HANDSHAKED;
+      return 0;
+    case PORT_NOT_IN_VALID_STATE:
+      error = "in invalid state";
+      break;
+    case UNKNOWN_PORT:
+      error = "an unknown port";
+      break;
+    case PORTS_DESTINATION_FULL:
+      error = "full";
+      break;
+    // Unknown error
+    default:
+      logc(err, "HandShake Failed because of unknown respond code %d", code);
+      return -1;
+  }
+
+  // All known error cases handled here
+  logc(err, "Site2Site HandShake Failed because destination port, %s, is %s", client->_port_id_str, error);
+  return -2;
+}
+
+
+/*bool CRawSiteToSiteClient::getPeerList(std::vector<CPeerStatus> &peers) {
+  if (establish(this) == 0 && handShake()) {
+    int status = writeRequestType(this, REQUEST_PEER_LIST);
+
+    if (status <= 0) {
+      tearDown(this);
+      return false;
+    }
+
+    uint32_t number;
+    status = _peer->read(number);
+
+    if (status <= 0) {
+      tearDown(this);
+      return false;
+    }
+
+    for (uint32_t i = 0; i < number; i++) {
+      std::string host;
+      status = _peer->readUTF(host);
+      if (status <= 0) {
+        tearDown(this);
+        return false;
+      }
+      uint32_t port;
+      status = _peer->read(port);
+      if (status <= 0) {
+        tearDown(this);
+        return false;
+      }
+      uint8_t secure;
+      status = _peer->read(secure);
+      if (status <= 0) {
+        tearDown(this);
+        return false;
+      }
+      uint32_t count;
+      status = _peer->read(count);
+      if (status <= 0) {
+        tearDown(this);
+        return false;
+      }
+      CPeerStatus status(std::make_shared<CPeer>(port_id_, host, port, secure), count, true);
+      peers.push_back(std::move(status));
+      logging::LOG_TRACE(logger_) << "Site2Site Peer host " << host << " port " << port << " Secure " << secure;
+    }
+
+    tearDown(this);
+    return true;
+  } else {
+    tearDown(this);
+    return false;
+  }
+}*/
+
+int bootstrap(struct CRawSiteToSiteClient * client) {
+  if (client->_peer_state == READY)
+    return 0;
+
+  tearDown(client);
+
+  if (establish(client) ==0 && handShake(client) == 0 && negotiateCodec(client) == 0) {
+    logc(debug, "%s", "Site to Site ready for data transaction");
+    return 0;
+  } else {
+    tearDown(client);
+    return -1;
+  }
+}
+
+CTransaction* createTransaction(struct CRawSiteToSiteClient * client, TransferDirection direction) {
+  int ret;
+  int dataAvailable = 0;
+  CTransaction* transaction = NULL;
+
+  if (client->_peer_state != READY) {
+    bootstrap(client);
+  }
+
+  if (client->_peer_state != READY) {
+    return transaction;
+  }
+
+  if (direction == RECEIVE) {
+    ret = writeRequestType(client, RECEIVE_FLOWFILES);
+
+    if (ret <= 0) {
+      return transaction;
+    }
+
+    RespondCode code;
+
+    ret = readResponse(client, &code);
+
+    if (ret <= 0) {
+      return transaction;
+    }
+
+    RespondCodeContext *resCode = getRespondCodeContext(code);
+
+    if(resCode == NULL) {
+      logc(err, "Received invalid respond code: %d", code);
+      return NULL;
+    }
+
+    if (resCode->hasDescription) {
+      uint32_t utflen;
+      ret = readUTFLen(&utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return transaction;
+      memset(client->_description_buffer, 0, utflen+1);
+      ret = readUTF(client->_description_buffer, utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return transaction;
+    }
+    
+    switch (code) {
+      case MORE_DATA:
+        dataAvailable = 1;
+        logc(trace, "%s", "Site2Site peer indicates that data is available");
+        break;
+      case NO_MORE_DATA:
+        dataAvailable = 0;
+        logc(trace, "%s", "Site2Site peer indicates that no data is available");
+        break;
+      default:
+        logc(warn, "Site2Site got unexpected response %d when asking for data", code);
+        return NULL;
+    }
+    transaction = (CTransaction*)malloc(1* sizeof(CTransaction));
+    InitTransaction(transaction, direction, client->_peer->_stream);
+    addTransaction(client, transaction);
+    setDataAvailable(transaction, dataAvailable);
+    logc(trace, "Site2Site create transaction %s", getUUIDStr(transaction));
+    return transaction;
+  } else {
+    ret = writeRequestType(client, SEND_FLOWFILES);
+
+    if (ret <= 0) {
+      return NULL;
+    } else {
+      transaction = (CTransaction*)malloc(1* sizeof(CTransaction));
+      InitTransaction(transaction, direction, client->_peer->_stream);
+      addTransaction(client, transaction);
+      logc(trace, "Site2Site create transaction %s", getUUIDStr(transaction));
+      return transaction;
+    }
+  }
+}
+
+int transmitPayload(struct CRawSiteToSiteClient * client, const char * payload, const attribute_set * attributes) {
+  CTransaction* transaction = NULL;
+
+  if (payload == NULL && attributes == NULL) {
+    return -1;
+  }
+
+  if (client->_peer_state != READY) {
+    if (bootstrap(client) != 0) {
+      return -1;
+    }
+  }
+
+  if (client->_peer_state != READY) {
+    tearDown(client);
+  }
+
+  // Create the transaction
+  const char * transactionID;
+  transaction = createTransaction(client, SEND);
+
+  if (transaction == NULL) {
+    tearDown(client);
+    return -1;
+  }
+
+  transactionID = getUUIDStr(transaction);
+
+  CDataPacket packet;
+
+  initPacket(&packet, transaction, attributes, payload);
+
+  int16_t resp = sendPacket(client, transactionID, &packet, NULL);
+  if (resp != 0) {
+    deleteTransaction(client, transactionID);
+    tearDown(client);
+    return resp;
+  }
+  logc(info, "Site2Site transaction %s sent bytes length %lu", transactionID, strlen(payload));
+
+
+  int ret = confirm(client, transactionID);
+
+  if(ret == 0) {
+    ret = complete(client, transactionID);
+  }
+
+  deleteTransaction(client, transactionID);
+
+  if (ret != 0) {
+    tearDown(client);
+  }
+
+  return ret;
+}
+
+// Complete the transaction
+int complete(struct CRawSiteToSiteClient * client, const char * transactionID) {
+  if (client->_peer_state != READY) {
+    bootstrap(client);
+  }
+
+  if (client->_peer_state != READY) {
+    return -1;
+  }
+
+  CTransaction* transaction = findTransaction(client, transactionID);
+
+  if (!transaction) {
+    return -1;
+  }
+
+  if (transaction->total_transfers_ > 0 && getState(transaction) != TRANSACTION_CONFIRMED) {
+    return -1;
+  }
+  if (getDirection(transaction) == RECEIVE) {
+    if (transaction->current_transfers_ == 0) {
+      transaction->_state = TRANSACTION_COMPLETED;
+      return 0;
+    } else {
+      logc(debug, "Site2Site transaction %s send finished", transactionID);
+      if(writeResponse(client, TRANSACTION_FINISHED, "Finished") <= 0) {
+        return -1;
+      } else {
+        transaction->_state = TRANSACTION_COMPLETED;
+        return 0;
+      }
+    }
+  } else {
+    RespondCode code;
+
+    if (readResponse(client, &code) <= 0) {
+      return -1;
+    }
+
+    RespondCodeContext *resCode = getRespondCodeContext(code);
+
+    if(resCode == NULL) {
+      logc(err, "Received invalid respond code: %d", code);
+      return -1;
+    }
+
+    if (resCode->hasDescription) {
+      uint32_t utflen;
+      int ret = readUTFLen(&utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return -1;
+      memset(client->_description_buffer, 0, utflen+1);
+      ret = readUTF(client->_description_buffer, utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return -1;
+    }
+
+    if (code == TRANSACTION_FINISHED) {
+      logc(debug, "Site2Site transaction %s peer finished transaction", transactionID);
+      transaction->_state = TRANSACTION_COMPLETED;
+      return 0;
+    } else {
+      logc(warn, "Site2Site transaction %s peer unknown respond code %d", transactionID, code);
+      return -1;
+    }
+  }
+}
+
+int confirm(struct CRawSiteToSiteClient * client, const char * transactionID) {
+
+  if (client->_peer_state != READY) {
+    bootstrap(client);
+  }
+
+  if (client->_peer_state != READY) {
+    return -1;
+  }
+
+  CTransaction* transaction = findTransaction(client, transactionID);
+
+  if (!transaction) {
+    return -1;
+  }
+
+  if (getState(transaction) == TRANSACTION_STARTED && isDataAvailable(transaction) == 0 && getDirection(transaction) == RECEIVE) {
+    transaction->_state = TRANSACTION_CONFIRMED;
+    return 0;
+  }
+
+  if (getState(transaction) != DATA_EXCHANGED)
+    return -1;
+
+  if (getDirection(transaction) == RECEIVE) {
+    if (isDataAvailable(transaction) != 0)
+      return -1;
+
+    // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+    // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+    // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+    // session and then when we send the response back to the peer, the peer may have timed out and may not
+    // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+    // Critical Section involved in this transaction so that rather than the Critical Section being the
+    // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+    int64_t crcValue = getCRC(transaction);
+    char crc[40];
+    sprintf(crc, "%lld", crcValue);
+
+    logc(debug, "Site2Site Send confirm with CRC %lld to transaction %s", crcValue, transactionID);
+    if (writeResponse(client, CONFIRM_TRANSACTION, crc) <= 0) {
+      return -1;
+    }
+
+    RespondCode code;
+    if (readResponse(client, &code) <= 0) {
+      return -1;
+    }
+
+    RespondCodeContext *resCode = getRespondCodeContext(code);
+
+    if(resCode == NULL) {
+      logc(err, "Received invalid respond code: %d", code);
+      return -1;
+    }
+
+    if (resCode->hasDescription) {
+      uint32_t utflen;
+      int ret = readUTFLen(&utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return -1;
+      memset(client->_description_buffer, 0, utflen+1);
+      ret = readUTF(client->_description_buffer, utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return -1;
+    }
+
+    if (code == CONFIRM_TRANSACTION) {
+      logc(debug, "Site2Site transaction %s peer confirm transaction", transactionID);
+      transaction->_state = TRANSACTION_CONFIRMED;
+      return 0;
+    } else if (code == BAD_CHECKSUM) {
+      logc(debug, "Site2Site transaction %s peer indicate bad checksum", transactionID);
+      return -1;
+    } else {
+      logc(debug, "Site2Site transaction %s peer unknown response code %d", transactionID, code);
+      return -1;
+    }
+  } else {
+    logc(debug, "Site2Site Send FINISH TRANSACTION for transaction %s", transactionID);
+    if (writeResponse(client, FINISH_TRANSACTION, "FINISH_TRANSACTION") <= 0) {
+      return -1;
+    }
+
+    RespondCode code;
+    if(readResponse(client, &code) <= 0) {
+      return -1;
+    }
+
+    RespondCodeContext *resCode = getRespondCodeContext(code);
+
+    if(resCode == NULL) {
+      logc(err, "Received invalid respond code: %d", code);
+      return -1;
+    }
+
+    if (resCode->hasDescription) {
+      uint32_t utflen;
+      int ret = readUTFLen(&utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return -1;
+      memset(client->_description_buffer, 0, utflen+1);
+      ret = readUTF(client->_description_buffer, utflen, client->_peer->_stream);
+      if (ret <= 0)
+        return -1;
+    }
+
+    // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+    if (code == CONFIRM_TRANSACTION) {
+      logc(debug, "Site2Site transaction %s peer confirm transaction with CRC %s", transactionID, client->_description_buffer);
+
+      if (client->_currentVersion > 3) {
+        int64_t crcValue = getCRC(transaction);
+        char crc[40];
+        memset(crc, 0, 40);
+        sprintf(crc, "%lld", crcValue);
+
+        if (strcmp(client->_description_buffer, crc) == 0) {
+          logc(debug, "Site2Site transaction %s CRC matched", transactionID);
+          if(writeResponse(client, CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION") <= 0) {
+            return -1;
+          }
+          transaction->_state = TRANSACTION_CONFIRMED;
+          return 0;
+        } else {
+          logc(warn, "Site2Site transaction %s CRC not matched %s", transactionID, crc);
+          writeResponse(client, BAD_CHECKSUM, "BAD_CHECKSUM");
+          return -1;
+        }
+      }
+      if (writeResponse(client, CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION") <= 0) {
+        return -1;
+      }
+      transaction->_state = TRANSACTION_CONFIRMED;
+      return 0;
+    } else {
+      logc(debug, "Site2Site transaction %s peer unknown respond code %d", transactionID, code);
+      return -1;
+    }
+  }
+}
+
+  int16_t sendPacket(struct CRawSiteToSiteClient * client, const char * transactionID, CDataPacket *packet, flow_file_record * ff) {
+
+    if (client->_peer_state != READY) {
+      bootstrap(client);
+    }
+
+    if (client->_peer_state != READY) {
+      return -1;
+    }
+    CTransaction* transaction = findTransaction(client, transactionID);
+
+    if (!transaction) {
+      return -1;
+    }
+
+    if (getState(transaction) != TRANSACTION_STARTED && getState(transaction) != DATA_EXCHANGED) {
+      logc(warn, "Site2Site transaction %s is not at started or exchanged state", transactionID);
+      return -1;
+    }
+
+    if (getDirection(transaction) != SEND) {
+      logc(warn, "Site2Site transaction %s direction is wrong", transactionID);
+      return -1;
+    }
+
+    int ret;
+
+    if (transaction->current_transfers_ > 0) {
+      ret = writeResponse(client, CONTINUE_TRANSACTION, "CONTINUE_TRANSACTION");
+      if (ret <= 0) {
+        return -1;
+      }
+    }
+    // start to read the packet
+    uint32_t numAttributes = packet->_attributes->size;
+    ret = write_uint32t(transaction, numAttributes);
+    if (ret != 4) {
+      return -1;
+    }
+
+    int i;
+    for (i = 0; i < packet->_attributes->size; ++i) {
+      const char *key = packet->_attributes->attributes[i].key;
+
+      ret = write_UTF(transaction, key, True);
+
+      if (ret <= 0) {
+        return -1;
+      }
+
+      const char *value = (const char *) packet->_attributes->attributes[i].value;
+
+      ret = write_UTF_len(transaction, value, packet->_attributes->attributes[i].value_size, True);
+      if (ret <= 0) {
+        return -1;
+      }
+    }
+
+    uint64_t len = 0;
+
+    uint64_t content_size = 0;
+
+    if(ff != NULL) {
+      content_size = ff->size;
+
+      uint8_t * content_buf = NULL;
+
+      if(content_size > 0 && ff->crp != NULL) {
+        content_buf = (uint8_t*)malloc(content_size*sizeof(uint8_t));
+        if(get_content(ff, content_buf, content_size) <= 0) {
+          return -2;
+        }
+        ret = write_uint64t(transaction, len);
+        if (ret != 8) {
+          logc(debug, "ret != 8");
+          return -1;
+        }
+        writeData(transaction, content_buf, len);
+      }
+
+    } else if (strlen(packet->payload_) > 0) {
+      len = strlen(packet->payload_);
+
+      ret = write_uint64t(transaction, len);
+      if (ret != 8) {
+        return -1;
+      }
+
+      ret = writeData(transaction, (uint8_t *)(packet->payload_), len);
+      if (ret != (int64_t)len) {
+        logc(debug, "ret != len");
+        return -1;
+      }
+    }
+
+    transaction->current_transfers_++;
+    transaction->total_transfers_++;
+    transaction->_state = DATA_EXCHANGED;
+    transaction->_bytes += len;
+
+    logc(info, "Site to Site transaction %s sent flow %d flow records, with total size %llu", transactionID,
+        transaction->total_transfers_, transaction->_bytes);
+
+    return 0;
+  }
+
+int readResponse(struct CRawSiteToSiteClient* client, RespondCode *code) {
+  uint8_t firstByte;
+
+  int ret = read_uint8_t(&firstByte, client->_peer->_stream);
+
+  if (ret <= 0 || firstByte != CODE_SEQUENCE_VALUE_1)
+    return -1;
+
+  uint8_t secondByte;
+
+  ret = read_uint8_t(&secondByte, client->_peer->_stream);
+
+  if (ret <= 0 || secondByte != CODE_SEQUENCE_VALUE_2)
+    return -1;
+
+  uint8_t thirdByte;
+
+  ret = read_uint8_t(&thirdByte, client->_peer->_stream);
+
+  if (ret <= 0)
+    return ret;
+
+  *code = (RespondCode) thirdByte;
+
+  RespondCodeContext *resCode = getRespondCodeContext(*code);
+
+  if (resCode == NULL) {
+    logc(err, "Received invalid response code: %u", thirdByte);
+    return -1;
+  }
+  return 3;
+}
+
+
+int writeResponse(struct CRawSiteToSiteClient* client, RespondCode code, const char * message) {
+  RespondCodeContext *resCode = getRespondCodeContext(code);
+
+  if (resCode == NULL) {
+    logc(err, "Received invalid respond code: %d", code);
+    // Not a valid respond code
+    return -1;
+  }
+
+  uint8_t codeSeq[3];
+  codeSeq[0] = CODE_SEQUENCE_VALUE_1;
+  codeSeq[1] = CODE_SEQUENCE_VALUE_2;
+  codeSeq[2] = (uint8_t) code;
+
+  int ret = write_buffer(codeSeq, 3, client->_peer->_stream);
+
+  if (ret != 3)
+    return -1;
+
+  if (resCode->hasDescription) {
+    ret = writeUTF(message, strlen(message), False, client->_peer->_stream);
+    if (ret > 0) {
+      return (3 + ret);
+    } else {
+      return ret;
+    }
+  } else {
+    return 3;
+  }
+}
+
+int writeRequestType(struct CRawSiteToSiteClient* client, RequestType type) {
+  if (type >= MAX_REQUEST_TYPE)
+    return -1;
+
+  const char * typestr = RequestTypeStr[type];
+
+  return writeUTF(typestr, strlen(typestr), False, client->_peer->_stream);
+}
+
+int readRequestType(struct CRawSiteToSiteClient* client, RequestType *type) {
+  char requestTypeStr[128];
+
+  uint32_t utflen;
+
+  int ret = readUTFLen(&utflen, client->_peer->_stream);
+
+  if (ret <= 0)
+    return ret;
+
+  memset(requestTypeStr, 0, 128);
+  ret = readUTF(requestTypeStr, utflen, client->_peer->_stream);
+
+  if (ret <= 0)
+    return ret;
+
+  requestTypeStr[utflen] = '\0';
+
+  int i;
+  for (i = NEGOTIATE_FLOWFILE_CODEC; i <= SHUTDOWN; i++) {
+    if (strcmp(RequestTypeStr[i], requestTypeStr) == 0) {
+      *type = (RequestType) i;
+      return ret;
+    }
+  }
+
+  return -1;
+}
+
+void tearDown(struct CRawSiteToSiteClient* client) {
+  if (client->_peer_state >= ESTABLISHED) {
+    // need to write shutdown request
+    writeRequestType(client, SHUTDOWN);
+  }
+
+  clearTransactions(client);
+  closePeer(client->_peer);
+  client->_peer_state = IDLE;
+}
+
+int initiateResourceNegotiation(struct CRawSiteToSiteClient* client) {
+  // Negotiate the version
+  if (client->_peer_state != IDLE) {
+    return -1;
+  }
+
+  int ret = writeUTF(getResourceName(client), strlen(getResourceName(client)), False, client->_peer->_stream);
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  ret = write_uint32_t(client->_currentVersion, client->_peer->_stream);
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  uint8_t statusCode;
+
+  ret = read_uint8_t(&statusCode, client->_peer->_stream);
+
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  uint32_t serverVersion;
+
+  switch (statusCode) {
+    case RESOURCE_OK:
+      logc(info, "Resource negotiation completed successfully. Using version: %u", client->_currentVersion);
+      return 0;
+    case DIFFERENT_RESOURCE_VERSION:
+      ret = read_uint32_t(&serverVersion, client->_peer->_stream);
+      if (ret <= 0) {
+        return -1;
+      }
+
+      unsigned int i;
+      for (i = (client->_currentVersionIndex + 1); i < sizeof(client->_supportedVersion) / sizeof(uint32_t); i++) {
+        if (serverVersion >= client->_supportedVersion[i]) {
+          client->_currentVersion = client->_supportedVersion[i];
+          client->_currentVersionIndex = i;
+          return initiateResourceNegotiation(client);
+        }
+      }
+      logc(err, "Server version %u not supported", serverVersion);
+      return -2;
+    case NEGOTIATED_ABORT:
+      logc(err, "%s", "Server aborted negotiation");
+      return -2;
+    default:
+      logc(err, "Received invalid status code: %u", statusCode);
+      return -1;
+  }
+}
+
+int initiateCodecResourceNegotiation(struct CRawSiteToSiteClient* client) {
+  // Negotiate the version
+  if (client->_peer_state != HANDSHAKED) {
+    return -1;
+  }
+
+  const char * coderresource = getCodecResourceName(client);
+
+  int ret = writeUTF(coderresource, strlen(coderresource), False, client->_peer->_stream);
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  ret = write_uint32_t(client->_currentCodecVersion, client->_peer->_stream);
+
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  uint8_t statusCode;
+  ret = read_uint8_t(&statusCode, client->_peer->_stream);
+
+  if (ret <= 0) {
+    return -1;
+  }
+
+  uint32_t serverVersion;
+  switch (statusCode) {
+    case RESOURCE_OK:
+      logc(info, "Resource codec negotiation completed successfully. Using version: %u", client->_currentCodecVersion);
+      return 0;
+    case DIFFERENT_RESOURCE_VERSION:
+      ret = read_uint32_t(&serverVersion, client->_peer->_stream);
+      if (ret <= 0) {
+        return -1;
+      }
+
+      unsigned int i;
+      for (i = (client->_currentCodecVersionIndex + 1);
+           i < sizeof(client->_supportedCodecVersion) / sizeof(uint32_t); i++) {
+        if (serverVersion >= client->_supportedCodecVersion[i]) {
+          client->_currentCodecVersion = client->_supportedCodecVersion[i];
+          client->_currentCodecVersionIndex = i;
+          return initiateCodecResourceNegotiation(client);
+        }
+      }
+      logc(err, "Server codec version %u not supported", serverVersion);
+      return -1;
+    case NEGOTIATED_ABORT:
+      logc(err, "%s", "Server aborted codec negotiation");
+      return -2;
+    default:
+      logc(err, "Received invalid status code: %u", statusCode);
+      return -1;
+  }
+}
+
+int negotiateCodec(struct CRawSiteToSiteClient* client) {
+  if (client->_peer_state != HANDSHAKED) {
+    return -1;
+  }
+  int status = writeRequestType(client, NEGOTIATE_FLOWFILE_CODEC);
+
+  if (status <= 0) {
+    return -1;
+  }
+
+  if (initiateCodecResourceNegotiation(client) != 0) {
+    return -2;
+  }
+
+  client->_peer_state = READY;
+  return 0;
+}
+
+int establish(struct CRawSiteToSiteClient* client) {
+  if (client->_peer_state != IDLE) {
+    return -1;
+  }
+
+  if(openPeer(client->_peer) != 0) {
+    return -1;
+  }
+
+  // Negotiate the version
+  if(initiateResourceNegotiation(client) != 0) {
+    return -1;
+  }
+
+  client->_peer_state = ESTABLISHED;
+
+  return 0;
+}
+
+void addTransaction(struct CRawSiteToSiteClient * client, CTransaction * transaction) {
+  HASH_ADD_STR(client->_known_transactions, _uuid_str, transaction);
+}
+
+CTransaction * findTransaction(const struct CRawSiteToSiteClient * client, const char * id) {
+  CTransaction * transaction = NULL;
+  HASH_FIND_STR(client->_known_transactions, id, transaction);
+  return transaction;
+}
+
+void deleteTransaction(struct CRawSiteToSiteClient * client, const char * id) {
+  CTransaction * transaction = findTransaction(client, id);
+  if(transaction) {
+    HASH_DEL(client->_known_transactions, transaction);
+    free(transaction);
+  }
+}
+
+void clearTransactions(struct CRawSiteToSiteClient * client) {
+  if(client->_known_transactions == NULL) {
+    return;
+  }
+
+  CTransaction *transaction, *tmp = NULL;
+
+  HASH_ITER(hh, client->_known_transactions, transaction, tmp) {
+    HASH_DEL(client->_known_transactions, transaction);
+    free(transaction);
+  }
+}
diff --git a/nanofi/tests/CSite2SiteTests.cpp b/nanofi/tests/CSite2SiteTests.cpp
new file mode 100644
index 0000000..4b37de2
--- /dev/null
+++ b/nanofi/tests/CSite2SiteTests.cpp
@@ -0,0 +1,223 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdlib.h>
+
+#include <uuid/uuid.h>
+#include <algorithm>
+#include <string>
+#include <memory>
+#include <utility>
+#include <map>
+#include "io/BaseStream.h"
+#include "TestBase.h"
+#include "unit/SiteToSiteHelper.h"
+#include "sitetosite/CPeer.h"
+#include "sitetosite/CRawSocketProtocol.h"
+#include "sitetosite/CSiteToSite.h"
+#include <algorithm>
+#include <core/cxxstructs.h>
+
+#define FMT_DEFAULT fmt_lower
+
+TEST_CASE("TestSetPortId", "[S2S1]") {
+  auto stream_ptr = std::unique_ptr<minifi::io::BaseStream>(new org::apache::nifi::minifi::io::BaseStream());
+
+  cstream cstrm;
+  cstrm.impl = stream_ptr.get();
+
+  SiteToSiteCPeer peer;
+  initPeer(&peer, &cstrm, "fake_host", 65433, "");
+  CRawSiteToSiteClient * protocol = (CRawSiteToSiteClient*)malloc(sizeof(CRawSiteToSiteClient));
+
+  initRawClient(protocol, &peer);
+
+  std::string uuid_str = "c56a4180-65aa-42ec-a945-5fd21dec0538";
+
+  setPortId(protocol, uuid_str.c_str());
+
+  REQUIRE(uuid_str == std::string(getPortId(protocol)));
+
+  tearDown(protocol);
+
+  freePeer(&peer);
+
+  free(protocol);
+}
+
+TEST_CASE("TestSetPortIdUppercase", "[S2S2]") {
+  auto stream_ptr = std::unique_ptr<minifi::io::BaseStream>(new org::apache::nifi::minifi::io::BaseStream());
+
+  cstream cstrm;
+  cstrm.impl = stream_ptr.get();
+
+  SiteToSiteCPeer peer;
+  initPeer(&peer, &cstrm, "fake_host", 65433, "");
+  CRawSiteToSiteClient * protocol = (CRawSiteToSiteClient*)malloc(sizeof(CRawSiteToSiteClient));
+
+  initRawClient(protocol, &peer);
+
+  std::string uuid_str = "C56A4180-65AA-42EC-A945-5FD21DEC0538";
+
+  //setPortId(protocol, uuid_str.c_str());
+
+  //REQUIRE(uuid_str != getPortId(protocol));
+
+  std::transform(uuid_str.begin(), uuid_str.end(), uuid_str.begin(), ::tolower);
+
+  //REQUIRE(uuid_str == std::string(getPortId(protocol)));
+
+  tearDown(protocol);
+
+  freePeer(&peer);
+
+  free(protocol);
+}
+
+void sunny_path_bootstrap(SiteToSiteResponder *collector) {
+  char a = 0x14;  // RESOURCE_OK
+  std::string resp_code;
+  resp_code.insert(resp_code.begin(), a);
+  collector->push_response(resp_code);
+
+  // Handshake respond code
+  resp_code = "R";
+  collector->push_response(resp_code);
+  resp_code = "C";
+  collector->push_response(resp_code);
+  char b = 0x1;
+  resp_code = b;
+  collector->push_response(resp_code);
+
+  // Codec Negotiation
+  resp_code = a;
+  collector->push_response(resp_code);
+}
+
+TEST_CASE("TestSiteToSiteVerifySend", "[S2S3]") {
+
+  SiteToSiteResponder *collector = new SiteToSiteResponder();
+  sunny_path_bootstrap(collector);
+
+  auto stream_ptr = std::unique_ptr<minifi::io::BaseStream>(new org::apache::nifi::minifi::io::BaseStream(collector));
+
+  cstream cstrm;
+  cstrm.impl = stream_ptr.get();
+
+  SiteToSiteCPeer peer;
+  initPeer(&peer, &cstrm, "fake_host", 65433, "");
+
+  CRawSiteToSiteClient * protocol = (CRawSiteToSiteClient*)malloc(sizeof(CRawSiteToSiteClient));
+
+  initRawClient(protocol, &peer);
+
+  std::string uuid_str = "C56A4180-65AA-42EC-A945-5FD21DEC0538";
+
+  setPortId(protocol, uuid_str.c_str());
+
+  REQUIRE(0 == bootstrap(protocol));
+
+  REQUIRE(collector->get_next_client_response() == "NiFi");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "SocketFlowFileProtocol");
+  collector->get_next_client_response();
+  collector->get_next_client_response();
+  collector->get_next_client_response();
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "nifi://fake_host:65433");
+  collector->get_next_client_response();
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "GZIP");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "false");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "PORT_IDENTIFIER");
+  collector->get_next_client_response();
+
+  std::string temp_val = collector->get_next_client_response();
+  std::transform(temp_val.begin(), temp_val.end(), temp_val.begin(), ::tolower);
+
+  REQUIRE(temp_val == "c56a4180-65aa-42ec-a945-5fd21dec0538");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "REQUEST_EXPIRATION_MILLIS");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "30000");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "NEGOTIATE_FLOWFILE_CODEC");
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "StandardFlowFileCodec");
+  collector->get_next_client_response();  // codec version
+
+  // start to send the stuff
+  // Create the transaction
+  const char * transactionID;
+  const char * payload = "Test MiNiFi payload";
+  CTransaction* transaction;
+  transaction = createTransaction(protocol, SEND);
+  REQUIRE(transaction != NULL);
+  transactionID = getUUIDStr(transaction);
+  collector->get_next_client_response();
+  REQUIRE(collector->get_next_client_response() == "SEND_FLOWFILES");
+  attribute_set as;
+  as.size = 0;
+  as.attributes = NULL;
+  CDataPacket packet;
+  initPacket(&packet, transaction, &as, payload);
+  REQUIRE(sendPacket(protocol, transactionID, &packet, nullptr) == 0);
+  collector->get_next_client_response();
+  collector->get_next_client_response();
+
+  std::string rx_payload = collector->get_next_client_response();
+  REQUIRE(payload == rx_payload);
+
+  freePeer(&peer);
+
+  free(protocol);
+}
+
+TEST_CASE("TestSiteToSiteVerifyNegotiationFail", "[S2S4]") {
+  SiteToSiteResponder *collector = new SiteToSiteResponder();
+
+  char a = 0xFF;
+  std::string resp_code;
+  resp_code.insert(resp_code.begin(), a);
+  collector->push_response(resp_code);
+  collector->push_response(resp_code);
+
+  auto stream_ptr = std::unique_ptr<minifi::io::BaseStream>(new org::apache::nifi::minifi::io::BaseStream(collector));
+
+  cstream cstrm;
+  cstrm.impl = stream_ptr.get();
+
+  SiteToSiteCPeer peer;
+  initPeer(&peer, &cstrm, "fake_host", 65433, "");
+
+  CRawSiteToSiteClient * protocol = (CRawSiteToSiteClient*)malloc(sizeof(CRawSiteToSiteClient));
+
+  initRawClient(protocol, &peer);
+
+  std::string uuid_str = "C56A4180-65AA-42EC-A945-5FD21DEC0538";
+
+  setPortId(protocol, uuid_str.c_str());
+
+  REQUIRE(-1 == bootstrap(protocol));
+
+  freePeer(&peer);
+
+  free(protocol);
+}
diff --git a/thirdparty/ut/CMakeLists.txt b/thirdparty/ut/CMakeLists.txt
new file mode 100644
index 0000000..e245a5a
--- /dev/null
+++ b/thirdparty/ut/CMakeLists.txt
@@ -0,0 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+#
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+include_directories(".")
\ No newline at end of file
diff --git a/thirdparty/ut/utarray.h b/thirdparty/ut/utarray.h
new file mode 100755
index 0000000..6ed0dce
--- /dev/null
+++ b/thirdparty/ut/utarray.h
@@ -0,0 +1,238 @@
+/*
+Copyright (c) 2008-2018, Troy D. Hanson   http://troydhanson.github.com/uthash/
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/
+
+/* a dynamic array implementation using macros
+ */
+#ifndef UTARRAY_H
+#define UTARRAY_H
+
+#define UTARRAY_VERSION 2.1.0
+
+#include <stddef.h>  /* size_t */
+#include <string.h>  /* memset, etc */
+#include <stdlib.h>  /* exit */
+
+#ifdef __GNUC__
+#define UTARRAY_UNUSED __attribute__((__unused__))
+#else
+#define UTARRAY_UNUSED
+#endif
+
+#ifndef oom
+#define oom() exit(-1)
+#endif
+
+typedef void (ctor_f)(void *dst, const void *src);
+typedef void (dtor_f)(void *elt);
+typedef void (init_f)(void *elt);
+typedef struct {
+    size_t sz;
+    init_f *init;
+    ctor_f *copy;
+    dtor_f *dtor;
+} UT_icd;
+
+typedef struct {
+    unsigned i,n;/* i: index of next available slot, n: num slots */
+    UT_icd icd;  /* initializer, copy and destructor functions */
+    char *d;     /* n slots of size icd->sz*/
+} UT_array;
+
+#define utarray_init(a,_icd) do {                                             \
+  memset(a,0,sizeof(UT_array));                                               \
+  (a)->icd = *(_icd);                                                         \
+} while(0)
+
+#define utarray_done(a) do {                                                  \
+  if ((a)->n) {                                                               \
+    if ((a)->icd.dtor) {                                                      \
+      unsigned _ut_i;                                                         \
+      for(_ut_i=0; _ut_i < (a)->i; _ut_i++) {                                 \
+        (a)->icd.dtor(utarray_eltptr(a,_ut_i));                               \
+      }                                                                       \
+    }                                                                         \
+    free((a)->d);                                                             \
+  }                                                                           \
+  (a)->n=0;                                                                   \
+} while(0)
+
+#define utarray_new(a,_icd) do {                                              \
+  (a) = (UT_array*)malloc(sizeof(UT_array));                                  \
+  if ((a) == NULL) oom();                                                     \
+  utarray_init(a,_icd);                                                       \
+} while(0)
+
+#define utarray_free(a) do {                                                  \
+  utarray_done(a);                                                            \
+  free(a);                                                                    \
+} while(0)
+
+#define utarray_reserve(a,by) do {                                            \
+  if (((a)->i+(by)) > (a)->n) {                                               \
+    char *utarray_tmp;                                                        \
+    while (((a)->i+(by)) > (a)->n) { (a)->n = ((a)->n ? (2*(a)->n) : 8); }    \
+    utarray_tmp=(char*)realloc((a)->d, (a)->n*(a)->icd.sz);                   \
+    if (utarray_tmp == NULL) oom();                                           \
+    (a)->d=utarray_tmp;                                                       \
+  }                                                                           \
+} while(0)
+
+#define utarray_push_back(a,p) do {                                           \
+  utarray_reserve(a,1);                                                       \
+  if ((a)->icd.copy) { (a)->icd.copy( _utarray_eltptr(a,(a)->i++), p); }      \
+  else { memcpy(_utarray_eltptr(a,(a)->i++), p, (a)->icd.sz); };              \
+} while(0)
+
+#define utarray_pop_back(a) do {                                              \
+  if ((a)->icd.dtor) { (a)->icd.dtor( _utarray_eltptr(a,--((a)->i))); }       \
+  else { (a)->i--; }                                                          \
+} while(0)
+
+#define utarray_extend_back(a) do {                                           \
+  utarray_reserve(a,1);                                                       \
+  if ((a)->icd.init) { (a)->icd.init(_utarray_eltptr(a,(a)->i)); }            \
+  else { memset(_utarray_eltptr(a,(a)->i),0,(a)->icd.sz); }                   \
+  (a)->i++;                                                                   \
+} while(0)
+
+#define utarray_len(a) ((a)->i)
+
+#define utarray_eltptr(a,j) (((j) < (a)->i) ? _utarray_eltptr(a,j) : NULL)
+#define _utarray_eltptr(a,j) ((a)->d + ((a)->icd.sz * (j)))
+
+#define utarray_insert(a,p,j) do {                                            \
+  if ((j) > (a)->i) utarray_resize(a,j);                                      \
+  utarray_reserve(a,1);                                                       \
+  if ((j) < (a)->i) {                                                         \
+    memmove( _utarray_eltptr(a,(j)+1), _utarray_eltptr(a,j),                  \
+             ((a)->i - (j))*((a)->icd.sz));                                   \
+  }                                                                           \
+  if ((a)->icd.copy) { (a)->icd.copy( _utarray_eltptr(a,j), p); }             \
+  else { memcpy(_utarray_eltptr(a,j), p, (a)->icd.sz); };                     \
+  (a)->i++;                                                                   \
+} while(0)
+
+#define utarray_inserta(a,w,j) do {                                           \
+  if (utarray_len(w) == 0) break;                                             \
+  if ((j) > (a)->i) utarray_resize(a,j);                                      \
+  utarray_reserve(a,utarray_len(w));                                          \
+  if ((j) < (a)->i) {                                                         \
+    memmove(_utarray_eltptr(a,(j)+utarray_len(w)),                            \
+            _utarray_eltptr(a,j),                                             \
+            ((a)->i - (j))*((a)->icd.sz));                                    \
+  }                                                                           \
+  if ((a)->icd.copy) {                                                        \
+    unsigned _ut_i;                                                           \
+    for(_ut_i=0;_ut_i<(w)->i;_ut_i++) {                                       \
+      (a)->icd.copy(_utarray_eltptr(a, (j) + _ut_i), _utarray_eltptr(w, _ut_i)); \
+    }                                                                         \
+  } else {                                                                    \
+    memcpy(_utarray_eltptr(a,j), _utarray_eltptr(w,0),                        \
+           utarray_len(w)*((a)->icd.sz));                                     \
+  }                                                                           \
+  (a)->i += utarray_len(w);                                                   \
+} while(0)
+
+#define utarray_resize(dst,num) do {                                          \
+  unsigned _ut_i;                                                             \
+  if ((dst)->i > (unsigned)(num)) {                                           \
+    if ((dst)->icd.dtor) {                                                    \
+      for (_ut_i = (num); _ut_i < (dst)->i; ++_ut_i) {                        \
+        (dst)->icd.dtor(_utarray_eltptr(dst, _ut_i));                         \
+      }                                                                       \
+    }                                                                         \
+  } else if ((dst)->i < (unsigned)(num)) {                                    \
+    utarray_reserve(dst, (num) - (dst)->i);                                   \
+    if ((dst)->icd.init) {                                                    \
+      for (_ut_i = (dst)->i; _ut_i < (unsigned)(num); ++_ut_i) {              \
+        (dst)->icd.init(_utarray_eltptr(dst, _ut_i));                         \
+      }                                                                       \
+    } else {                                                                  \
+      memset(_utarray_eltptr(dst, (dst)->i), 0, (dst)->icd.sz*((num) - (dst)->i)); \
+    }                                                                         \
+  }                                                                           \
+  (dst)->i = (num);                                                           \
+} while(0)
+
+#define utarray_concat(dst,src) do {                                          \
+  utarray_inserta(dst, src, utarray_len(dst));                                \
+} while(0)
+
+#define utarray_erase(a,pos,len) do {                                         \
+  if ((a)->icd.dtor) {                                                        \
+    unsigned _ut_i;                                                           \
+    for (_ut_i = 0; _ut_i < (len); _ut_i++) {                                 \
+      (a)->icd.dtor(utarray_eltptr(a, (pos) + _ut_i));                        \
+    }                                                                         \
+  }                                                                           \
+  if ((a)->i > ((pos) + (len))) {                                             \
+    memmove(_utarray_eltptr(a, pos), _utarray_eltptr(a, (pos) + (len)),       \
+            ((a)->i - ((pos) + (len))) * (a)->icd.sz);                        \
+  }                                                                           \
+  (a)->i -= (len);                                                            \
+} while(0)
+
+#define utarray_renew(a,u) do {                                               \
+  if (a) utarray_clear(a);                                                    \
+  else utarray_new(a, u);                                                     \
+} while(0)
+
+#define utarray_clear(a) do {                                                 \
+  if ((a)->i > 0) {                                                           \
+    if ((a)->icd.dtor) {                                                      \
+      unsigned _ut_i;                                                         \
+      for(_ut_i=0; _ut_i < (a)->i; _ut_i++) {                                 \
+        (a)->icd.dtor(_utarray_eltptr(a, _ut_i));                             \
+      }                                                                       \
+    }                                                                         \
+    (a)->i = 0;                                                               \
+  }                                                                           \
+} while(0)
+
+#define utarray_sort(a,cmp) do {                                              \
+  qsort((a)->d, (a)->i, (a)->icd.sz, cmp);                                    \
+} while(0)
+
+#define utarray_find(a,v,cmp) bsearch((v),(a)->d,(a)->i,(a)->icd.sz,cmp)
+
+#define utarray_front(a) (((a)->i) ? (_utarray_eltptr(a,0)) : NULL)
+#define utarray_next(a,e) (((e)==NULL) ? utarray_front(a) : ((((a)->i) > (utarray_eltidx(a,e)+1)) ? _utarray_eltptr(a,utarray_eltidx(a,e)+1) : NULL))
+#define utarray_prev(a,e) (((e)==NULL) ? utarray_back(a) : ((utarray_eltidx(a,e) > 0) ? _utarray_eltptr(a,utarray_eltidx(a,e)-1) : NULL))
+#define utarray_back(a) (((a)->i) ? (_utarray_eltptr(a,(a)->i-1)) : NULL)
+#define utarray_eltidx(a,e) (((char*)(e) >= (a)->d) ? (((char*)(e) - (a)->d)/(a)->icd.sz) : -1)
+
+/* last we pre-define a few icd for common utarrays of ints and strings */
+static void utarray_str_cpy(void *dst, const void *src) {
+  char **_src = (char**)src, **_dst = (char**)dst;
+  *_dst = (*_src == NULL) ? NULL : strdup(*_src);
+}
+static void utarray_str_dtor(void *elt) {
+  char **eltc = (char**)elt;
+  if (*eltc != NULL) free(*eltc);
+}
+static const UT_icd ut_str_icd UTARRAY_UNUSED = {sizeof(char*),NULL,utarray_str_cpy,utarray_str_dtor};
+static const UT_icd ut_int_icd UTARRAY_UNUSED = {sizeof(int),NULL,NULL,NULL};
+static const UT_icd ut_ptr_icd UTARRAY_UNUSED = {sizeof(void*),NULL,NULL,NULL};
+
+
+#endif /* UTARRAY_H */
diff --git a/thirdparty/ut/uthash.h b/thirdparty/ut/uthash.h
new file mode 100755
index 0000000..76bdca6
--- /dev/null
+++ b/thirdparty/ut/uthash.h
@@ -0,0 +1,1227 @@
+/*
+Copyright (c) 2003-2018, Troy D. Hanson     http://troydhanson.github.com/uthash/
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/
+
+#ifndef UTHASH_H
+#define UTHASH_H
+
+#define UTHASH_VERSION 2.1.0
+
+#include <string.h>   /* memcmp, memset, strlen */
+#include <stddef.h>   /* ptrdiff_t */
+#include <stdlib.h>   /* exit */
+
+/* These macros use decltype or the earlier __typeof GNU extension.
+   As decltype is only available in newer compilers (VS2010 or gcc 4.3+
+   when compiling c++ source) this code uses whatever method is needed
+   or, for VS2008 where neither is available, uses casting workarounds. */
+#if !defined(DECLTYPE) && !defined(NO_DECLTYPE)
+#if defined(_MSC_VER)   /* MS compiler */
+#if _MSC_VER >= 1600 && defined(__cplusplus)  /* VS2010 or newer in C++ mode */
+#define DECLTYPE(x) (decltype(x))
+#else                   /* VS2008 or older (or VS2010 in C mode) */
+#define NO_DECLTYPE
+#endif
+#elif defined(__BORLANDC__) || defined(__ICCARM__) || defined(__LCC__) || defined(__WATCOMC__)
+#define NO_DECLTYPE
+#else                   /* GNU, Sun and other compilers */
+#define DECLTYPE(x) (__typeof(x))
+#endif
+#endif
+
+#ifdef NO_DECLTYPE
+#define DECLTYPE(x)
+#define DECLTYPE_ASSIGN(dst,src)                                                 \
+do {                                                                             \
+  char **_da_dst = (char**)(&(dst));                                             \
+  *_da_dst = (char*)(src);                                                       \
+} while (0)
+#else
+#define DECLTYPE_ASSIGN(dst,src)                                                 \
+do {                                                                             \
+  (dst) = DECLTYPE(dst)(src);                                                    \
+} while (0)
+#endif
+
+/* a number of the hash function use uint32_t which isn't defined on Pre VS2010 */
+#if defined(_WIN32)
+#if defined(_MSC_VER) && _MSC_VER >= 1600
+#include <stdint.h>
+#elif defined(__WATCOMC__) || defined(__MINGW32__) || defined(__CYGWIN__)
+#include <stdint.h>
+#else
+typedef unsigned int uint32_t;
+typedef unsigned char uint8_t;
+#endif
+#elif defined(__GNUC__) && !defined(__VXWORKS__)
+#include <stdint.h>
+#else
+typedef unsigned int uint32_t;
+typedef unsigned char uint8_t;
+#endif
+
+#ifndef uthash_malloc
+#define uthash_malloc(sz) malloc(sz)      /* malloc fcn                      */
+#endif
+#ifndef uthash_free
+#define uthash_free(ptr,sz) free(ptr)     /* free fcn                        */
+#endif
+#ifndef uthash_bzero
+#define uthash_bzero(a,n) memset(a,'\0',n)
+#endif
+#ifndef uthash_strlen
+#define uthash_strlen(s) strlen(s)
+#endif
+
+#ifdef uthash_memcmp
+/* This warning will not catch programs that define uthash_memcmp AFTER including uthash.h. */
+#warning "uthash_memcmp is deprecated; please use HASH_KEYCMP instead"
+#else
+#define uthash_memcmp(a,b,n) memcmp(a,b,n)
+#endif
+
+#ifndef HASH_KEYCMP
+#define HASH_KEYCMP(a,b,n) uthash_memcmp(a,b,n)
+#endif
+
+#ifndef uthash_noexpand_fyi
+#define uthash_noexpand_fyi(tbl)          /* can be defined to log noexpand  */
+#endif
+#ifndef uthash_expand_fyi
+#define uthash_expand_fyi(tbl)            /* can be defined to log expands   */
+#endif
+
+#ifndef HASH_NONFATAL_OOM
+#define HASH_NONFATAL_OOM 0
+#endif
+
+#if HASH_NONFATAL_OOM
+/* malloc failures can be recovered from */
+
+#ifndef uthash_nonfatal_oom
+#define uthash_nonfatal_oom(obj) do {} while (0)    /* non-fatal OOM error */
+#endif
+
+#define HASH_RECORD_OOM(oomed) do { (oomed) = 1; } while (0)
+#define IF_HASH_NONFATAL_OOM(x) x
+
+#else
+/* malloc failures result in lost memory, hash tables are unusable */
+
+#ifndef uthash_fatal
+#define uthash_fatal(msg) exit(-1)        /* fatal OOM error */
+#endif
+
+#define HASH_RECORD_OOM(oomed) uthash_fatal("out of memory")
+#define IF_HASH_NONFATAL_OOM(x)
+
+#endif
+
+/* initial number of buckets */
+#define HASH_INITIAL_NUM_BUCKETS 32U     /* initial number of buckets        */
+#define HASH_INITIAL_NUM_BUCKETS_LOG2 5U /* lg2 of initial number of buckets */
+#define HASH_BKT_CAPACITY_THRESH 10U     /* expand when bucket count reaches */
+
+/* calculate the element whose hash handle address is hhp */
+#define ELMT_FROM_HH(tbl,hhp) ((void*)(((char*)(hhp)) - ((tbl)->hho)))
+/* calculate the hash handle from element address elp */
+#define HH_FROM_ELMT(tbl,elp) ((UT_hash_handle *)(((char*)(elp)) + ((tbl)->hho)))
+
+#define HASH_ROLLBACK_BKT(hh, head, itemptrhh)                                   \
+do {                                                                             \
+  struct UT_hash_handle *_hd_hh_item = (itemptrhh);                              \
+  unsigned _hd_bkt;                                                              \
+  HASH_TO_BKT(_hd_hh_item->hashv, (head)->hh.tbl->num_buckets, _hd_bkt);         \
+  (head)->hh.tbl->buckets[_hd_bkt].count++;                                      \
+  _hd_hh_item->hh_next = NULL;                                                   \
+  _hd_hh_item->hh_prev = NULL;                                                   \
+} while (0)
+
+#define HASH_VALUE(keyptr,keylen,hashv)                                          \
+do {                                                                             \
+  HASH_FCN(keyptr, keylen, hashv);                                               \
+} while (0)
+
+#define HASH_FIND_BYHASHVALUE(hh,head,keyptr,keylen,hashval,out)                 \
+do {                                                                             \
+  (out) = NULL;                                                                  \
+  if (head) {                                                                    \
+    unsigned _hf_bkt;                                                            \
+    HASH_TO_BKT(hashval, (head)->hh.tbl->num_buckets, _hf_bkt);                  \
+    if (HASH_BLOOM_TEST((head)->hh.tbl, hashval) != 0) {                         \
+      HASH_FIND_IN_BKT((head)->hh.tbl, hh, (head)->hh.tbl->buckets[ _hf_bkt ], keyptr, keylen, hashval, out); \
+    }                                                                            \
+  }                                                                              \
+} while (0)
+
+#define HASH_FIND(hh,head,keyptr,keylen,out)                                     \
+do {                                                                             \
+  unsigned _hf_hashv;                                                            \
+  HASH_VALUE(keyptr, keylen, _hf_hashv);                                         \
+  HASH_FIND_BYHASHVALUE(hh, head, keyptr, keylen, _hf_hashv, out);               \
+} while (0)
+
+#ifdef HASH_BLOOM
+#define HASH_BLOOM_BITLEN (1UL << HASH_BLOOM)
+#define HASH_BLOOM_BYTELEN (HASH_BLOOM_BITLEN/8UL) + (((HASH_BLOOM_BITLEN%8UL)!=0UL) ? 1UL : 0UL)
+#define HASH_BLOOM_MAKE(tbl,oomed)                                               \
+do {                                                                             \
+  (tbl)->bloom_nbits = HASH_BLOOM;                                               \
+  (tbl)->bloom_bv = (uint8_t*)uthash_malloc(HASH_BLOOM_BYTELEN);                 \
+  if (!(tbl)->bloom_bv) {                                                        \
+    HASH_RECORD_OOM(oomed);                                                      \
+  } else {                                                                       \
+    uthash_bzero((tbl)->bloom_bv, HASH_BLOOM_BYTELEN);                           \
+    (tbl)->bloom_sig = HASH_BLOOM_SIGNATURE;                                     \
+  }                                                                              \
+} while (0)
+
+#define HASH_BLOOM_FREE(tbl)                                                     \
+do {                                                                             \
+  uthash_free((tbl)->bloom_bv, HASH_BLOOM_BYTELEN);                              \
+} while (0)
+
+#define HASH_BLOOM_BITSET(bv,idx) (bv[(idx)/8U] |= (1U << ((idx)%8U)))
+#define HASH_BLOOM_BITTEST(bv,idx) (bv[(idx)/8U] & (1U << ((idx)%8U)))
+
+#define HASH_BLOOM_ADD(tbl,hashv)                                                \
+  HASH_BLOOM_BITSET((tbl)->bloom_bv, ((hashv) & (uint32_t)((1UL << (tbl)->bloom_nbits) - 1U)))
+
+#define HASH_BLOOM_TEST(tbl,hashv)                                               \
+  HASH_BLOOM_BITTEST((tbl)->bloom_bv, ((hashv) & (uint32_t)((1UL << (tbl)->bloom_nbits) - 1U)))
+
+#else
+#define HASH_BLOOM_MAKE(tbl,oomed)
+#define HASH_BLOOM_FREE(tbl)
+#define HASH_BLOOM_ADD(tbl,hashv)
+#define HASH_BLOOM_TEST(tbl,hashv) (1)
+#define HASH_BLOOM_BYTELEN 0U
+#endif
+
+#define HASH_MAKE_TABLE(hh,head,oomed)                                           \
+do {                                                                             \
+  (head)->hh.tbl = (UT_hash_table*)uthash_malloc(sizeof(UT_hash_table));         \
+  if (!(head)->hh.tbl) {                                                         \
+    HASH_RECORD_OOM(oomed);                                                      \
+  } else {                                                                       \
+    uthash_bzero((head)->hh.tbl, sizeof(UT_hash_table));                         \
+    (head)->hh.tbl->tail = &((head)->hh);                                        \
+    (head)->hh.tbl->num_buckets = HASH_INITIAL_NUM_BUCKETS;                      \
+    (head)->hh.tbl->log2_num_buckets = HASH_INITIAL_NUM_BUCKETS_LOG2;            \
+    (head)->hh.tbl->hho = (char*)(&(head)->hh) - (char*)(head);                  \
+    (head)->hh.tbl->buckets = (UT_hash_bucket*)uthash_malloc(                    \
+        HASH_INITIAL_NUM_BUCKETS * sizeof(struct UT_hash_bucket));               \
+    (head)->hh.tbl->signature = HASH_SIGNATURE;                                  \
+    if (!(head)->hh.tbl->buckets) {                                              \
+      HASH_RECORD_OOM(oomed);                                                    \
+      uthash_free((head)->hh.tbl, sizeof(UT_hash_table));                        \
+    } else {                                                                     \
+      uthash_bzero((head)->hh.tbl->buckets,                                      \
+          HASH_INITIAL_NUM_BUCKETS * sizeof(struct UT_hash_bucket));             \
+      HASH_BLOOM_MAKE((head)->hh.tbl, oomed);                                    \
+      IF_HASH_NONFATAL_OOM(                                                      \
+        if (oomed) {                                                             \
+          uthash_free((head)->hh.tbl->buckets,                                   \
+              HASH_INITIAL_NUM_BUCKETS*sizeof(struct UT_hash_bucket));           \
+          uthash_free((head)->hh.tbl, sizeof(UT_hash_table));                    \
+        }                                                                        \
+      )                                                                          \
+    }                                                                            \
+  }                                                                              \
+} while (0)
+
+#define HASH_REPLACE_BYHASHVALUE_INORDER(hh,head,fieldname,keylen_in,hashval,add,replaced,cmpfcn) \
+do {                                                                             \
+  (replaced) = NULL;                                                             \
+  HASH_FIND_BYHASHVALUE(hh, head, &((add)->fieldname), keylen_in, hashval, replaced); \
+  if (replaced) {                                                                \
+    HASH_DELETE(hh, head, replaced);                                             \
+  }                                                                              \
+  HASH_ADD_KEYPTR_BYHASHVALUE_INORDER(hh, head, &((add)->fieldname), keylen_in, hashval, add, cmpfcn); \
+} while (0)
+
+#define HASH_REPLACE_BYHASHVALUE(hh,head,fieldname,keylen_in,hashval,add,replaced) \
+do {                                                                             \
+  (replaced) = NULL;                                                             \
+  HASH_FIND_BYHASHVALUE(hh, head, &((add)->fieldname), keylen_in, hashval, replaced); \
+  if (replaced) {                                                                \
+    HASH_DELETE(hh, head, replaced);                                             \
+  }                                                                              \
+  HASH_ADD_KEYPTR_BYHASHVALUE(hh, head, &((add)->fieldname), keylen_in, hashval, add); \
+} while (0)
+
+#define HASH_REPLACE(hh,head,fieldname,keylen_in,add,replaced)                   \
+do {                                                                             \
+  unsigned _hr_hashv;                                                            \
+  HASH_VALUE(&((add)->fieldname), keylen_in, _hr_hashv);                         \
+  HASH_REPLACE_BYHASHVALUE(hh, head, fieldname, keylen_in, _hr_hashv, add, replaced); \
+} while (0)
+
+#define HASH_REPLACE_INORDER(hh,head,fieldname,keylen_in,add,replaced,cmpfcn)    \
+do {                                                                             \
+  unsigned _hr_hashv;                                                            \
+  HASH_VALUE(&((add)->fieldname), keylen_in, _hr_hashv);                         \
+  HASH_REPLACE_BYHASHVALUE_INORDER(hh, head, fieldname, keylen_in, _hr_hashv, add, replaced, cmpfcn); \
+} while (0)
+
+#define HASH_APPEND_LIST(hh, head, add)                                          \
+do {                                                                             \
+  (add)->hh.next = NULL;                                                         \
+  (add)->hh.prev = ELMT_FROM_HH((head)->hh.tbl, (head)->hh.tbl->tail);           \
+  (head)->hh.tbl->tail->next = (add);                                            \
+  (head)->hh.tbl->tail = &((add)->hh);                                           \
+} while (0)
+
+#define HASH_AKBI_INNER_LOOP(hh,head,add,cmpfcn)                                 \
+do {                                                                             \
+  do {                                                                           \
+    if (cmpfcn(DECLTYPE(head)(_hs_iter), add) > 0) {                             \
+      break;                                                                     \
+    }                                                                            \
+  } while ((_hs_iter = HH_FROM_ELMT((head)->hh.tbl, _hs_iter)->next));           \
+} while (0)
+
+#ifdef NO_DECLTYPE
+#undef HASH_AKBI_INNER_LOOP
+#define HASH_AKBI_INNER_LOOP(hh,head,add,cmpfcn)                                 \
+do {                                                                             \
+  char *_hs_saved_head = (char*)(head);                                          \
+  do {                                                                           \
+    DECLTYPE_ASSIGN(head, _hs_iter);                                             \
+    if (cmpfcn(head, add) > 0) {                                                 \
+      DECLTYPE_ASSIGN(head, _hs_saved_head);                                     \
+      break;                                                                     \
+    }                                                                            \
+    DECLTYPE_ASSIGN(head, _hs_saved_head);                                       \
+  } while ((_hs_iter = HH_FROM_ELMT((head)->hh.tbl, _hs_iter)->next));           \
+} while (0)
+#endif
+
+#if HASH_NONFATAL_OOM
+
+#define HASH_ADD_TO_TABLE(hh,head,keyptr,keylen_in,hashval,add,oomed)            \
+do {                                                                             \
+  if (!(oomed)) {                                                                \
+    unsigned _ha_bkt;                                                            \
+    (head)->hh.tbl->num_items++;                                                 \
+    HASH_TO_BKT(hashval, (head)->hh.tbl->num_buckets, _ha_bkt);                  \
+    HASH_ADD_TO_BKT((head)->hh.tbl->buckets[_ha_bkt], hh, &(add)->hh, oomed);    \
+    if (oomed) {                                                                 \
+      HASH_ROLLBACK_BKT(hh, head, &(add)->hh);                                   \
+      HASH_DELETE_HH(hh, head, &(add)->hh);                                      \
+      (add)->hh.tbl = NULL;                                                      \
+      uthash_nonfatal_oom(add);                                                  \
+    } else {                                                                     \
+      HASH_BLOOM_ADD((head)->hh.tbl, hashval);                                   \
+      HASH_EMIT_KEY(hh, head, keyptr, keylen_in);                                \
+    }                                                                            \
+  } else {                                                                       \
+    (add)->hh.tbl = NULL;                                                        \
+    uthash_nonfatal_oom(add);                                                    \
+  }                                                                              \
+} while (0)
+
+#else
+
+#define HASH_ADD_TO_TABLE(hh,head,keyptr,keylen_in,hashval,add,oomed)            \
+do {                                                                             \
+  unsigned _ha_bkt;                                                              \
+  (head)->hh.tbl->num_items++;                                                   \
+  HASH_TO_BKT(hashval, (head)->hh.tbl->num_buckets, _ha_bkt);                    \
+  HASH_ADD_TO_BKT((head)->hh.tbl->buckets[_ha_bkt], hh, &(add)->hh, oomed);      \
+  HASH_BLOOM_ADD((head)->hh.tbl, hashval);                                       \
+  HASH_EMIT_KEY(hh, head, keyptr, keylen_in);                                    \
+} while (0)
+
+#endif
+
+
+#define HASH_ADD_KEYPTR_BYHASHVALUE_INORDER(hh,head,keyptr,keylen_in,hashval,add,cmpfcn) \
+do {                                                                             \
+  IF_HASH_NONFATAL_OOM( int _ha_oomed = 0; )                                     \
+  (add)->hh.hashv = (hashval);                                                   \
+  (add)->hh.key = (char*) (keyptr);                                              \
+  (add)->hh.keylen = (unsigned) (keylen_in);                                     \
+  if (!(head)) {                                                                 \
+    (add)->hh.next = NULL;                                                       \
+    (add)->hh.prev = NULL;                                                       \
+    HASH_MAKE_TABLE(hh, add, _ha_oomed);                                         \
+    IF_HASH_NONFATAL_OOM( if (!_ha_oomed) { )                                    \
+      (head) = (add);                                                            \
+    IF_HASH_NONFATAL_OOM( } )                                                    \
+  } else {                                                                       \
+    void *_hs_iter = (head);                                                     \
+    (add)->hh.tbl = (head)->hh.tbl;                                              \
+    HASH_AKBI_INNER_LOOP(hh, head, add, cmpfcn);                                 \
+    if (_hs_iter) {                                                              \
+      (add)->hh.next = _hs_iter;                                                 \
+      if (((add)->hh.prev = HH_FROM_ELMT((head)->hh.tbl, _hs_iter)->prev)) {     \
+        HH_FROM_ELMT((head)->hh.tbl, (add)->hh.prev)->next = (add);              \
+      } else {                                                                   \
+        (head) = (add);                                                          \
+      }                                                                          \
+      HH_FROM_ELMT((head)->hh.tbl, _hs_iter)->prev = (add);                      \
+    } else {                                                                     \
+      HASH_APPEND_LIST(hh, head, add);                                           \
+    }                                                                            \
+  }                                                                              \
+  HASH_ADD_TO_TABLE(hh, head, keyptr, keylen_in, hashval, add, _ha_oomed);       \
+  HASH_FSCK(hh, head, "HASH_ADD_KEYPTR_BYHASHVALUE_INORDER");                    \
+} while (0)
+
+#define HASH_ADD_KEYPTR_INORDER(hh,head,keyptr,keylen_in,add,cmpfcn)             \
+do {                                                                             \
+  unsigned _hs_hashv;                                                            \
+  HASH_VALUE(keyptr, keylen_in, _hs_hashv);                                      \
+  HASH_ADD_KEYPTR_BYHASHVALUE_INORDER(hh, head, keyptr, keylen_in, _hs_hashv, add, cmpfcn); \
+} while (0)
+
+#define HASH_ADD_BYHASHVALUE_INORDER(hh,head,fieldname,keylen_in,hashval,add,cmpfcn) \
+  HASH_ADD_KEYPTR_BYHASHVALUE_INORDER(hh, head, &((add)->fieldname), keylen_in, hashval, add, cmpfcn)
+
+#define HASH_ADD_INORDER(hh,head,fieldname,keylen_in,add,cmpfcn)                 \
+  HASH_ADD_KEYPTR_INORDER(hh, head, &((add)->fieldname), keylen_in, add, cmpfcn)
+
+#define HASH_ADD_KEYPTR_BYHASHVALUE(hh,head,keyptr,keylen_in,hashval,add)        \
+do {                                                                             \
+  IF_HASH_NONFATAL_OOM( int _ha_oomed = 0; )                                     \
+  (add)->hh.hashv = (hashval);                                                   \
+  (add)->hh.key = (char*) (keyptr);                                              \
+  (add)->hh.keylen = (unsigned) (keylen_in);                                     \
+  if (!(head)) {                                                                 \
+    (add)->hh.next = NULL;                                                       \
+    (add)->hh.prev = NULL;                                                       \
+    HASH_MAKE_TABLE(hh, add, _ha_oomed);                                         \
+    IF_HASH_NONFATAL_OOM( if (!_ha_oomed) { )                                    \
+      (head) = (add);                                                            \
+    IF_HASH_NONFATAL_OOM( } )                                                    \
+  } else {                                                                       \
+    (add)->hh.tbl = (head)->hh.tbl;                                              \
+    HASH_APPEND_LIST(hh, head, add);                                             \
+  }                                                                              \
+  HASH_ADD_TO_TABLE(hh, head, keyptr, keylen_in, hashval, add, _ha_oomed);       \
+  HASH_FSCK(hh, head, "HASH_ADD_KEYPTR_BYHASHVALUE");                            \
+} while (0)
+
+#define HASH_ADD_KEYPTR(hh,head,keyptr,keylen_in,add)                            \
+do {                                                                             \
+  unsigned _ha_hashv;                                                            \
+  HASH_VALUE(keyptr, keylen_in, _ha_hashv);                                      \
+  HASH_ADD_KEYPTR_BYHASHVALUE(hh, head, keyptr, keylen_in, _ha_hashv, add);      \
+} while (0)
+
+#define HASH_ADD_BYHASHVALUE(hh,head,fieldname,keylen_in,hashval,add)            \
+  HASH_ADD_KEYPTR_BYHASHVALUE(hh, head, &((add)->fieldname), keylen_in, hashval, add)
+
+#define HASH_ADD(hh,head,fieldname,keylen_in,add)                                \
+  HASH_ADD_KEYPTR(hh, head, &((add)->fieldname), keylen_in, add)
+
+#define HASH_TO_BKT(hashv,num_bkts,bkt)                                          \
+do {                                                                             \
+  bkt = ((hashv) & ((num_bkts) - 1U));                                           \
+} while (0)
+
+/* delete "delptr" from the hash table.
+ * "the usual" patch-up process for the app-order doubly-linked-list.
+ * The use of _hd_hh_del below deserves special explanation.
+ * These used to be expressed using (delptr) but that led to a bug
+ * if someone used the same symbol for the head and deletee, like
+ *  HASH_DELETE(hh,users,users);
+ * We want that to work, but by changing the head (users) below
+ * we were forfeiting our ability to further refer to the deletee (users)
+ * in the patch-up process. Solution: use scratch space to
+ * copy the deletee pointer, then the latter references are via that
+ * scratch pointer rather than through the repointed (users) symbol.
+ */
+#define HASH_DELETE(hh,head,delptr)                                              \
+    HASH_DELETE_HH(hh, head, &(delptr)->hh)
+
+#define HASH_DELETE_HH(hh,head,delptrhh)                                         \
+do {                                                                             \
+  struct UT_hash_handle *_hd_hh_del = (delptrhh);                                \
+  if ((_hd_hh_del->prev == NULL) && (_hd_hh_del->next == NULL)) {                \
+    HASH_BLOOM_FREE((head)->hh.tbl);                                             \
+    uthash_free((head)->hh.tbl->buckets,                                         \
+                (head)->hh.tbl->num_buckets * sizeof(struct UT_hash_bucket));    \
+    uthash_free((head)->hh.tbl, sizeof(UT_hash_table));                          \
+    (head) = NULL;                                                               \
+  } else {                                                                       \
+    unsigned _hd_bkt;                                                            \
+    if (_hd_hh_del == (head)->hh.tbl->tail) {                                    \
+      (head)->hh.tbl->tail = HH_FROM_ELMT((head)->hh.tbl, _hd_hh_del->prev);     \
+    }                                                                            \
+    if (_hd_hh_del->prev != NULL) {                                              \
+      HH_FROM_ELMT((head)->hh.tbl, _hd_hh_del->prev)->next = _hd_hh_del->next;   \
+    } else {                                                                     \
+      DECLTYPE_ASSIGN(head, _hd_hh_del->next);                                   \
+    }                                                                            \
+    if (_hd_hh_del->next != NULL) {                                              \
+      HH_FROM_ELMT((head)->hh.tbl, _hd_hh_del->next)->prev = _hd_hh_del->prev;   \
+    }                                                                            \
+    HASH_TO_BKT(_hd_hh_del->hashv, (head)->hh.tbl->num_buckets, _hd_bkt);        \
+    HASH_DEL_IN_BKT((head)->hh.tbl->buckets[_hd_bkt], _hd_hh_del);               \
+    (head)->hh.tbl->num_items--;                                                 \
+  }                                                                              \
+  HASH_FSCK(hh, head, "HASH_DELETE_HH");                                         \
+} while (0)
+
+/* convenience forms of HASH_FIND/HASH_ADD/HASH_DEL */
+#define HASH_FIND_STR(head,findstr,out)                                          \
+do {                                                                             \
+    unsigned _uthash_hfstr_keylen = (unsigned)uthash_strlen(findstr);            \
+    HASH_FIND(hh, head, findstr, _uthash_hfstr_keylen, out);                     \
+} while (0)
+#define HASH_ADD_STR(head,strfield,add)                                          \
+do {                                                                             \
+    unsigned _uthash_hastr_keylen = (unsigned)uthash_strlen((add)->strfield);    \
+    HASH_ADD(hh, head, strfield[0], _uthash_hastr_keylen, add);                  \
+} while (0)
+#define HASH_REPLACE_STR(head,strfield,add,replaced)                             \
+do {                                                                             \
+    unsigned _uthash_hrstr_keylen = (unsigned)uthash_strlen((add)->strfield);    \
+    HASH_REPLACE(hh, head, strfield[0], _uthash_hrstr_keylen, add, replaced);    \
+} while (0)
+#define HASH_FIND_INT(head,findint,out)                                          \
+    HASH_FIND(hh,head,findint,sizeof(int),out)
+#define HASH_ADD_INT(head,intfield,add)                                          \
+    HASH_ADD(hh,head,intfield,sizeof(int),add)
+#define HASH_REPLACE_INT(head,intfield,add,replaced)                             \
+    HASH_REPLACE(hh,head,intfield,sizeof(int),add,replaced)
+#define HASH_FIND_PTR(head,findptr,out)                                          \
+    HASH_FIND(hh,head,findptr,sizeof(void *),out)
+#define HASH_ADD_PTR(head,ptrfield,add)                                          \
+    HASH_ADD(hh,head,ptrfield,sizeof(void *),add)
+#define HASH_REPLACE_PTR(head,ptrfield,add,replaced)                             \
+    HASH_REPLACE(hh,head,ptrfield,sizeof(void *),add,replaced)
+#define HASH_DEL(head,delptr)                                                    \
+    HASH_DELETE(hh,head,delptr)
+
+/* HASH_FSCK checks hash integrity on every add/delete when HASH_DEBUG is defined.
+ * This is for uthash developer only; it compiles away if HASH_DEBUG isn't defined.
+ */
+#ifdef HASH_DEBUG
+#define HASH_OOPS(...) do { fprintf(stderr,__VA_ARGS__); exit(-1); } while (0)
+#define HASH_FSCK(hh,head,where)                                                 \
+do {                                                                             \
+  struct UT_hash_handle *_thh;                                                   \
+  if (head) {                                                                    \
+    unsigned _bkt_i;                                                             \
+    unsigned _count = 0;                                                         \
+    char *_prev;                                                                 \
+    for (_bkt_i = 0; _bkt_i < (head)->hh.tbl->num_buckets; ++_bkt_i) {           \
+      unsigned _bkt_count = 0;                                                   \
+      _thh = (head)->hh.tbl->buckets[_bkt_i].hh_head;                            \
+      _prev = NULL;                                                              \
+      while (_thh) {                                                             \
+        if (_prev != (char*)(_thh->hh_prev)) {                                   \
+          HASH_OOPS("%s: invalid hh_prev %p, actual %p\n",                       \
+              (where), (void*)_thh->hh_prev, (void*)_prev);                      \
+        }                                                                        \
+        _bkt_count++;                                                            \
+        _prev = (char*)(_thh);                                                   \
+        _thh = _thh->hh_next;                                                    \
+      }                                                                          \
+      _count += _bkt_count;                                                      \
+      if ((head)->hh.tbl->buckets[_bkt_i].count !=  _bkt_count) {                \
+        HASH_OOPS("%s: invalid bucket count %u, actual %u\n",                    \
+            (where), (head)->hh.tbl->buckets[_bkt_i].count, _bkt_count);         \
+      }                                                                          \
+    }                                                                            \
+    if (_count != (head)->hh.tbl->num_items) {                                   \
+      HASH_OOPS("%s: invalid hh item count %u, actual %u\n",                     \
+          (where), (head)->hh.tbl->num_items, _count);                           \
+    }                                                                            \
+    _count = 0;                                                                  \
+    _prev = NULL;                                                                \
+    _thh =  &(head)->hh;                                                         \
+    while (_thh) {                                                               \
+      _count++;                                                                  \
+      if (_prev != (char*)_thh->prev) {                                          \
+        HASH_OOPS("%s: invalid prev %p, actual %p\n",                            \
+            (where), (void*)_thh->prev, (void*)_prev);                           \
+      }                                                                          \
+      _prev = (char*)ELMT_FROM_HH((head)->hh.tbl, _thh);                         \
+      _thh = (_thh->next ? HH_FROM_ELMT((head)->hh.tbl, _thh->next) : NULL);     \
+    }                                                                            \
+    if (_count != (head)->hh.tbl->num_items) {                                   \
+      HASH_OOPS("%s: invalid app item count %u, actual %u\n",                    \
+          (where), (head)->hh.tbl->num_items, _count);                           \
+    }                                                                            \
+  }                                                                              \
+} while (0)
+#else
+#define HASH_FSCK(hh,head,where)
+#endif
+
+/* When compiled with -DHASH_EMIT_KEYS, length-prefixed keys are emitted to
+ * the descriptor to which this macro is defined for tuning the hash function.
+ * The app can #include <unistd.h> to get the prototype for write(2). */
+#ifdef HASH_EMIT_KEYS
+#define HASH_EMIT_KEY(hh,head,keyptr,fieldlen)                                   \
+do {                                                                             \
+  unsigned _klen = fieldlen;                                                     \
+  write(HASH_EMIT_KEYS, &_klen, sizeof(_klen));                                  \
+  write(HASH_EMIT_KEYS, keyptr, (unsigned long)fieldlen);                        \
+} while (0)
+#else
+#define HASH_EMIT_KEY(hh,head,keyptr,fieldlen)
+#endif
+
+/* default to Jenkin's hash unless overridden e.g. DHASH_FUNCTION=HASH_SAX */
+#ifdef HASH_FUNCTION
+#define HASH_FCN HASH_FUNCTION
+#else
+#define HASH_FCN HASH_JEN
+#endif
+
+/* The Bernstein hash function, used in Perl prior to v5.6. Note (x<<5+x)=x*33. */
+#define HASH_BER(key,keylen,hashv)                                               \
+do {                                                                             \
+  unsigned _hb_keylen = (unsigned)keylen;                                        \
+  const unsigned char *_hb_key = (const unsigned char*)(key);                    \
+  (hashv) = 0;                                                                   \
+  while (_hb_keylen-- != 0U) {                                                   \
+    (hashv) = (((hashv) << 5) + (hashv)) + *_hb_key++;                           \
+  }                                                                              \
+} while (0)
+
+
+/* SAX/FNV/OAT/JEN hash functions are macro variants of those listed at
+ * http://eternallyconfuzzled.com/tuts/algorithms/jsw_tut_hashing.aspx */
+#define HASH_SAX(key,keylen,hashv)                                               \
+do {                                                                             \
+  unsigned _sx_i;                                                                \
+  const unsigned char *_hs_key = (const unsigned char*)(key);                    \
+  hashv = 0;                                                                     \
+  for (_sx_i=0; _sx_i < keylen; _sx_i++) {                                       \
+    hashv ^= (hashv << 5) + (hashv >> 2) + _hs_key[_sx_i];                       \
+  }                                                                              \
+} while (0)
+/* FNV-1a variation */
+#define HASH_FNV(key,keylen,hashv)                                               \
+do {                                                                             \
+  unsigned _fn_i;                                                                \
+  const unsigned char *_hf_key = (const unsigned char*)(key);                    \
+  (hashv) = 2166136261U;                                                         \
+  for (_fn_i=0; _fn_i < keylen; _fn_i++) {                                       \
+    hashv = hashv ^ _hf_key[_fn_i];                                              \
+    hashv = hashv * 16777619U;                                                   \
+  }                                                                              \
+} while (0)
+
+#define HASH_OAT(key,keylen,hashv)                                               \
+do {                                                                             \
+  unsigned _ho_i;                                                                \
+  const unsigned char *_ho_key=(const unsigned char*)(key);                      \
+  hashv = 0;                                                                     \
+  for(_ho_i=0; _ho_i < keylen; _ho_i++) {                                        \
+      hashv += _ho_key[_ho_i];                                                   \
+      hashv += (hashv << 10);                                                    \
+      hashv ^= (hashv >> 6);                                                     \
+  }                                                                              \
+  hashv += (hashv << 3);                                                         \
+  hashv ^= (hashv >> 11);                                                        \
+  hashv += (hashv << 15);                                                        \
+} while (0)
+
+#define HASH_JEN_MIX(a,b,c)                                                      \
+do {                                                                             \
+  a -= b; a -= c; a ^= ( c >> 13 );                                              \
+  b -= c; b -= a; b ^= ( a << 8 );                                               \
+  c -= a; c -= b; c ^= ( b >> 13 );                                              \
+  a -= b; a -= c; a ^= ( c >> 12 );                                              \
+  b -= c; b -= a; b ^= ( a << 16 );                                              \
+  c -= a; c -= b; c ^= ( b >> 5 );                                               \
+  a -= b; a -= c; a ^= ( c >> 3 );                                               \
+  b -= c; b -= a; b ^= ( a << 10 );                                              \
+  c -= a; c -= b; c ^= ( b >> 15 );                                              \
+} while (0)
+
+#define HASH_JEN(key,keylen,hashv)                                               \
+do {                                                                             \
+  unsigned _hj_i,_hj_j,_hj_k;                                                    \
+  unsigned const char *_hj_key=(unsigned const char*)(key);                      \
+  hashv = 0xfeedbeefu;                                                           \
+  _hj_i = _hj_j = 0x9e3779b9u;                                                   \
+  _hj_k = (unsigned)(keylen);                                                    \
+  while (_hj_k >= 12U) {                                                         \
+    _hj_i +=    (_hj_key[0] + ( (unsigned)_hj_key[1] << 8 )                      \
+        + ( (unsigned)_hj_key[2] << 16 )                                         \
+        + ( (unsigned)_hj_key[3] << 24 ) );                                      \
+    _hj_j +=    (_hj_key[4] + ( (unsigned)_hj_key[5] << 8 )                      \
+        + ( (unsigned)_hj_key[6] << 16 )                                         \
+        + ( (unsigned)_hj_key[7] << 24 ) );                                      \
+    hashv += (_hj_key[8] + ( (unsigned)_hj_key[9] << 8 )                         \
+        + ( (unsigned)_hj_key[10] << 16 )                                        \
+        + ( (unsigned)_hj_key[11] << 24 ) );                                     \
+                                                                                 \
+     HASH_JEN_MIX(_hj_i, _hj_j, hashv);                                          \
+                                                                                 \
+     _hj_key += 12;                                                              \
+     _hj_k -= 12U;                                                               \
+  }                                                                              \
+  hashv += (unsigned)(keylen);                                                   \
+  switch ( _hj_k ) {                                                             \
+    case 11: hashv += ( (unsigned)_hj_key[10] << 24 ); /* FALLTHROUGH */         \
+    case 10: hashv += ( (unsigned)_hj_key[9] << 16 );  /* FALLTHROUGH */         \
+    case 9:  hashv += ( (unsigned)_hj_key[8] << 8 );   /* FALLTHROUGH */         \
+    case 8:  _hj_j += ( (unsigned)_hj_key[7] << 24 );  /* FALLTHROUGH */         \
+    case 7:  _hj_j += ( (unsigned)_hj_key[6] << 16 );  /* FALLTHROUGH */         \
+    case 6:  _hj_j += ( (unsigned)_hj_key[5] << 8 );   /* FALLTHROUGH */         \
+    case 5:  _hj_j += _hj_key[4];                      /* FALLTHROUGH */         \
+    case 4:  _hj_i += ( (unsigned)_hj_key[3] << 24 );  /* FALLTHROUGH */         \
+    case 3:  _hj_i += ( (unsigned)_hj_key[2] << 16 );  /* FALLTHROUGH */         \
+    case 2:  _hj_i += ( (unsigned)_hj_key[1] << 8 );   /* FALLTHROUGH */         \
+    case 1:  _hj_i += _hj_key[0];                                                \
+  }                                                                              \
+  HASH_JEN_MIX(_hj_i, _hj_j, hashv);                                             \
+} while (0)
+
+/* The Paul Hsieh hash function */
+#undef get16bits
+#if (defined(__GNUC__) && defined(__i386__)) || defined(__WATCOMC__)             \
+  || defined(_MSC_VER) || defined (__BORLANDC__) || defined (__TURBOC__)
+#define get16bits(d) (*((const uint16_t *) (d)))
+#endif
+
+#if !defined (get16bits)
+#define get16bits(d) ((((uint32_t)(((const uint8_t *)(d))[1])) << 8)             \
+                       +(uint32_t)(((const uint8_t *)(d))[0]) )
+#endif
+#define HASH_SFH(key,keylen,hashv)                                               \
+do {                                                                             \
+  unsigned const char *_sfh_key=(unsigned const char*)(key);                     \
+  uint32_t _sfh_tmp, _sfh_len = (uint32_t)keylen;                                \
+                                                                                 \
+  unsigned _sfh_rem = _sfh_len & 3U;                                             \
+  _sfh_len >>= 2;                                                                \
+  hashv = 0xcafebabeu;                                                           \
+                                                                                 \
+  /* Main loop */                                                                \
+  for (;_sfh_len > 0U; _sfh_len--) {                                             \
+    hashv    += get16bits (_sfh_key);                                            \
+    _sfh_tmp  = ((uint32_t)(get16bits (_sfh_key+2)) << 11) ^ hashv;              \
+    hashv     = (hashv << 16) ^ _sfh_tmp;                                        \
+    _sfh_key += 2U*sizeof (uint16_t);                                            \
+    hashv    += hashv >> 11;                                                     \
+  }                                                                              \
+                                                                                 \
+  /* Handle end cases */                                                         \
+  switch (_sfh_rem) {                                                            \
+    case 3: hashv += get16bits (_sfh_key);                                       \
+            hashv ^= hashv << 16;                                                \
+            hashv ^= (uint32_t)(_sfh_key[sizeof (uint16_t)]) << 18;              \
+            hashv += hashv >> 11;                                                \
+            break;                                                               \
+    case 2: hashv += get16bits (_sfh_key);                                       \
+            hashv ^= hashv << 11;                                                \
+            hashv += hashv >> 17;                                                \
+            break;                                                               \
+    case 1: hashv += *_sfh_key;                                                  \
+            hashv ^= hashv << 10;                                                \
+            hashv += hashv >> 1;                                                 \
+  }                                                                              \
+                                                                                 \
+  /* Force "avalanching" of final 127 bits */                                    \
+  hashv ^= hashv << 3;                                                           \
+  hashv += hashv >> 5;                                                           \
+  hashv ^= hashv << 4;                                                           \
+  hashv += hashv >> 17;                                                          \
+  hashv ^= hashv << 25;                                                          \
+  hashv += hashv >> 6;                                                           \
+} while (0)
+
+#ifdef HASH_USING_NO_STRICT_ALIASING
+/* The MurmurHash exploits some CPU's (x86,x86_64) tolerance for unaligned reads.
+ * For other types of CPU's (e.g. Sparc) an unaligned read causes a bus error.
+ * MurmurHash uses the faster approach only on CPU's where we know it's safe.
+ *
+ * Note the preprocessor built-in defines can be emitted using:
+ *
+ *   gcc -m64 -dM -E - < /dev/null                  (on gcc)
+ *   cc -## a.c (where a.c is a simple test file)   (Sun Studio)
+ */
+#if (defined(__i386__) || defined(__x86_64__)  || defined(_M_IX86))
+#define MUR_GETBLOCK(p,i) p[i]
+#else /* non intel */
+#define MUR_PLUS0_ALIGNED(p) (((unsigned long)p & 3UL) == 0UL)
+#define MUR_PLUS1_ALIGNED(p) (((unsigned long)p & 3UL) == 1UL)
+#define MUR_PLUS2_ALIGNED(p) (((unsigned long)p & 3UL) == 2UL)
+#define MUR_PLUS3_ALIGNED(p) (((unsigned long)p & 3UL) == 3UL)
+#define WP(p) ((uint32_t*)((unsigned long)(p) & ~3UL))
+#if (defined(__BIG_ENDIAN__) || defined(SPARC) || defined(__ppc__) || defined(__ppc64__))
+#define MUR_THREE_ONE(p) ((((*WP(p))&0x00ffffff) << 8) | (((*(WP(p)+1))&0xff000000) >> 24))
+#define MUR_TWO_TWO(p)   ((((*WP(p))&0x0000ffff) <<16) | (((*(WP(p)+1))&0xffff0000) >> 16))
+#define MUR_ONE_THREE(p) ((((*WP(p))&0x000000ff) <<24) | (((*(WP(p)+1))&0xffffff00) >>  8))
+#else /* assume little endian non-intel */
+#define MUR_THREE_ONE(p) ((((*WP(p))&0xffffff00) >> 8) | (((*(WP(p)+1))&0x000000ff) << 24))
+#define MUR_TWO_TWO(p)   ((((*WP(p))&0xffff0000) >>16) | (((*(WP(p)+1))&0x0000ffff) << 16))
+#define MUR_ONE_THREE(p) ((((*WP(p))&0xff000000) >>24) | (((*(WP(p)+1))&0x00ffffff) <<  8))
+#endif
+#define MUR_GETBLOCK(p,i) (MUR_PLUS0_ALIGNED(p) ? ((p)[i]) :           \
+                            (MUR_PLUS1_ALIGNED(p) ? MUR_THREE_ONE(p) : \
+                             (MUR_PLUS2_ALIGNED(p) ? MUR_TWO_TWO(p) :  \
+                                                      MUR_ONE_THREE(p))))
+#endif
+#define MUR_ROTL32(x,r) (((x) << (r)) | ((x) >> (32 - (r))))
+#define MUR_FMIX(_h) \
+do {                 \
+  _h ^= _h >> 16;    \
+  _h *= 0x85ebca6bu; \
+  _h ^= _h >> 13;    \
+  _h *= 0xc2b2ae35u; \
+  _h ^= _h >> 16;    \
+} while (0)
+
+#define HASH_MUR(key,keylen,hashv)                                     \
+do {                                                                   \
+  const uint8_t *_mur_data = (const uint8_t*)(key);                    \
+  const int _mur_nblocks = (int)(keylen) / 4;                          \
+  uint32_t _mur_h1 = 0xf88D5353u;                                      \
+  uint32_t _mur_c1 = 0xcc9e2d51u;                                      \
+  uint32_t _mur_c2 = 0x1b873593u;                                      \
+  uint32_t _mur_k1 = 0;                                                \
+  const uint8_t *_mur_tail;                                            \
+  const uint32_t *_mur_blocks = (const uint32_t*)(_mur_data+(_mur_nblocks*4)); \
+  int _mur_i;                                                          \
+  for (_mur_i = -_mur_nblocks; _mur_i != 0; _mur_i++) {                \
+    _mur_k1 = MUR_GETBLOCK(_mur_blocks,_mur_i);                        \
+    _mur_k1 *= _mur_c1;                                                \
+    _mur_k1 = MUR_ROTL32(_mur_k1,15);                                  \
+    _mur_k1 *= _mur_c2;                                                \
+                                                                       \
+    _mur_h1 ^= _mur_k1;                                                \
+    _mur_h1 = MUR_ROTL32(_mur_h1,13);                                  \
+    _mur_h1 = (_mur_h1*5U) + 0xe6546b64u;                              \
+  }                                                                    \
+  _mur_tail = (const uint8_t*)(_mur_data + (_mur_nblocks*4));          \
+  _mur_k1=0;                                                           \
+  switch ((keylen) & 3U) {                                             \
+    case 0: break;                                                     \
+    case 3: _mur_k1 ^= (uint32_t)_mur_tail[2] << 16; /* FALLTHROUGH */ \
+    case 2: _mur_k1 ^= (uint32_t)_mur_tail[1] << 8;  /* FALLTHROUGH */ \
+    case 1: _mur_k1 ^= (uint32_t)_mur_tail[0];                         \
+    _mur_k1 *= _mur_c1;                                                \
+    _mur_k1 = MUR_ROTL32(_mur_k1,15);                                  \
+    _mur_k1 *= _mur_c2;                                                \
+    _mur_h1 ^= _mur_k1;                                                \
+  }                                                                    \
+  _mur_h1 ^= (uint32_t)(keylen);                                       \
+  MUR_FMIX(_mur_h1);                                                   \
+  hashv = _mur_h1;                                                     \
+} while (0)
+#endif  /* HASH_USING_NO_STRICT_ALIASING */
+
+/* iterate over items in a known bucket to find desired item */
+#define HASH_FIND_IN_BKT(tbl,hh,head,keyptr,keylen_in,hashval,out)               \
+do {                                                                             \
+  if ((head).hh_head != NULL) {                                                  \
+    DECLTYPE_ASSIGN(out, ELMT_FROM_HH(tbl, (head).hh_head));                     \
+  } else {                                                                       \
+    (out) = NULL;                                                                \
+  }                                                                              \
+  while ((out) != NULL) {                                                        \
+    if ((out)->hh.hashv == (hashval) && (out)->hh.keylen == (keylen_in)) {       \
+      if (HASH_KEYCMP((out)->hh.key, keyptr, keylen_in) == 0) {              \
+        break;                                                                   \
+      }                                                                          \
+    }                                                                            \
+    if ((out)->hh.hh_next != NULL) {                                             \
+      DECLTYPE_ASSIGN(out, ELMT_FROM_HH(tbl, (out)->hh.hh_next));                \
+    } else {                                                                     \
+      (out) = NULL;                                                              \
+    }                                                                            \
+  }                                                                              \
+} while (0)
+
+/* add an item to a bucket  */
+#define HASH_ADD_TO_BKT(head,hh,addhh,oomed)                                     \
+do {                                                                             \
+  UT_hash_bucket *_ha_head = &(head);                                            \
+  _ha_head->count++;                                                             \
+  (addhh)->hh_next = _ha_head->hh_head;                                          \
+  (addhh)->hh_prev = NULL;                                                       \
+  if (_ha_head->hh_head != NULL) {                                               \
+    _ha_head->hh_head->hh_prev = (addhh);                                        \
+  }                                                                              \
+  _ha_head->hh_head = (addhh);                                                   \
+  if ((_ha_head->count >= ((_ha_head->expand_mult + 1U) * HASH_BKT_CAPACITY_THRESH)) \
+      && !(addhh)->tbl->noexpand) {                                              \
+    HASH_EXPAND_BUCKETS(addhh,(addhh)->tbl, oomed);                              \
+    IF_HASH_NONFATAL_OOM(                                                        \
+      if (oomed) {                                                               \
+        HASH_DEL_IN_BKT(head,addhh);                                             \
+      }                                                                          \
+    )                                                                            \
+  }                                                                              \
+} while (0)
+
+/* remove an item from a given bucket */
+#define HASH_DEL_IN_BKT(head,delhh)                                              \
+do {                                                                             \
+  UT_hash_bucket *_hd_head = &(head);                                            \
+  _hd_head->count--;                                                             \
+  if (_hd_head->hh_head == (delhh)) {                                            \
+    _hd_head->hh_head = (delhh)->hh_next;                                        \
+  }                                                                              \
+  if ((delhh)->hh_prev) {                                                        \
+    (delhh)->hh_prev->hh_next = (delhh)->hh_next;                                \
+  }                                                                              \
+  if ((delhh)->hh_next) {                                                        \
+    (delhh)->hh_next->hh_prev = (delhh)->hh_prev;                                \
+  }                                                                              \
+} while (0)
+
+/* Bucket expansion has the effect of doubling the number of buckets
+ * and redistributing the items into the new buckets. Ideally the
+ * items will distribute more or less evenly into the new buckets
+ * (the extent to which this is true is a measure of the quality of
+ * the hash function as it applies to the key domain).
+ *
+ * With the items distributed into more buckets, the chain length
+ * (item count) in each bucket is reduced. Thus by expanding buckets
+ * the hash keeps a bound on the chain length. This bounded chain
+ * length is the essence of how a hash provides constant time lookup.
+ *
+ * The calculation of tbl->ideal_chain_maxlen below deserves some
+ * explanation. First, keep in mind that we're calculating the ideal
+ * maximum chain length based on the *new* (doubled) bucket count.
+ * In fractions this is just n/b (n=number of items,b=new num buckets).
+ * Since the ideal chain length is an integer, we want to calculate
+ * ceil(n/b). We don't depend on floating point arithmetic in this
+ * hash, so to calculate ceil(n/b) with integers we could write
+ *
+ *      ceil(n/b) = (n/b) + ((n%b)?1:0)
+ *
+ * and in fact a previous version of this hash did just that.
+ * But now we have improved things a bit by recognizing that b is
+ * always a power of two. We keep its base 2 log handy (call it lb),
+ * so now we can write this with a bit shift and logical AND:
+ *
+ *      ceil(n/b) = (n>>lb) + ( (n & (b-1)) ? 1:0)
+ *
+ */
+#define HASH_EXPAND_BUCKETS(hh,tbl,oomed)                                        \
+do {                                                                             \
+  unsigned _he_bkt;                                                              \
+  unsigned _he_bkt_i;                                                            \
+  struct UT_hash_handle *_he_thh, *_he_hh_nxt;                                   \
+  UT_hash_bucket *_he_new_buckets, *_he_newbkt;                                  \
+  _he_new_buckets = (UT_hash_bucket*)uthash_malloc(                              \
+           2UL * (tbl)->num_buckets * sizeof(struct UT_hash_bucket));            \
+  if (!_he_new_buckets) {                                                        \
+    HASH_RECORD_OOM(oomed);                                                      \
+  } else {                                                                       \
+    uthash_bzero(_he_new_buckets,                                                \
+        2UL * (tbl)->num_buckets * sizeof(struct UT_hash_bucket));               \
+    (tbl)->ideal_chain_maxlen =                                                  \
+       ((tbl)->num_items >> ((tbl)->log2_num_buckets+1U)) +                      \
+       ((((tbl)->num_items & (((tbl)->num_buckets*2U)-1U)) != 0U) ? 1U : 0U);    \
+    (tbl)->nonideal_items = 0;                                                   \
+    for (_he_bkt_i = 0; _he_bkt_i < (tbl)->num_buckets; _he_bkt_i++) {           \
+      _he_thh = (tbl)->buckets[ _he_bkt_i ].hh_head;                             \
+      while (_he_thh != NULL) {                                                  \
+        _he_hh_nxt = _he_thh->hh_next;                                           \
+        HASH_TO_BKT(_he_thh->hashv, (tbl)->num_buckets * 2U, _he_bkt);           \
+        _he_newbkt = &(_he_new_buckets[_he_bkt]);                                \
+        if (++(_he_newbkt->count) > (tbl)->ideal_chain_maxlen) {                 \
+          (tbl)->nonideal_items++;                                               \
+          if (_he_newbkt->count > _he_newbkt->expand_mult * (tbl)->ideal_chain_maxlen) { \
+            _he_newbkt->expand_mult++;                                           \
+          }                                                                      \
+        }                                                                        \
+        _he_thh->hh_prev = NULL;                                                 \
+        _he_thh->hh_next = _he_newbkt->hh_head;                                  \
+        if (_he_newbkt->hh_head != NULL) {                                       \
+          _he_newbkt->hh_head->hh_prev = _he_thh;                                \
+        }                                                                        \
+        _he_newbkt->hh_head = _he_thh;                                           \
+        _he_thh = _he_hh_nxt;                                                    \
+      }                                                                          \
+    }                                                                            \
+    uthash_free((tbl)->buckets, (tbl)->num_buckets * sizeof(struct UT_hash_bucket)); \
+    (tbl)->num_buckets *= 2U;                                                    \
+    (tbl)->log2_num_buckets++;                                                   \
+    (tbl)->buckets = _he_new_buckets;                                            \
+    (tbl)->ineff_expands = ((tbl)->nonideal_items > ((tbl)->num_items >> 1)) ?   \
+        ((tbl)->ineff_expands+1U) : 0U;                                          \
+    if ((tbl)->ineff_expands > 1U) {                                             \
+      (tbl)->noexpand = 1;                                                       \
+      uthash_noexpand_fyi(tbl);                                                  \
+    }                                                                            \
+    uthash_expand_fyi(tbl);                                                      \
+  }                                                                              \
+} while (0)
+
+
+/* This is an adaptation of Simon Tatham's O(n log(n)) mergesort */
+/* Note that HASH_SORT assumes the hash handle name to be hh.
+ * HASH_SRT was added to allow the hash handle name to be passed in. */
+#define HASH_SORT(head,cmpfcn) HASH_SRT(hh,head,cmpfcn)
+#define HASH_SRT(hh,head,cmpfcn)                                                 \
+do {                                                                             \
+  unsigned _hs_i;                                                                \
+  unsigned _hs_looping,_hs_nmerges,_hs_insize,_hs_psize,_hs_qsize;               \
+  struct UT_hash_handle *_hs_p, *_hs_q, *_hs_e, *_hs_list, *_hs_tail;            \
+  if (head != NULL) {                                                            \
+    _hs_insize = 1;                                                              \
+    _hs_looping = 1;                                                             \
+    _hs_list = &((head)->hh);                                                    \
+    while (_hs_looping != 0U) {                                                  \
+      _hs_p = _hs_list;                                                          \
+      _hs_list = NULL;                                                           \
+      _hs_tail = NULL;                                                           \
+      _hs_nmerges = 0;                                                           \
+      while (_hs_p != NULL) {                                                    \
+        _hs_nmerges++;                                                           \
+        _hs_q = _hs_p;                                                           \
+        _hs_psize = 0;                                                           \
+        for (_hs_i = 0; _hs_i < _hs_insize; ++_hs_i) {                           \
+          _hs_psize++;                                                           \
+          _hs_q = ((_hs_q->next != NULL) ?                                       \
+            HH_FROM_ELMT((head)->hh.tbl, _hs_q->next) : NULL);                   \
+          if (_hs_q == NULL) {                                                   \
+            break;                                                               \
+          }                                                                      \
+        }                                                                        \
+        _hs_qsize = _hs_insize;                                                  \
+        while ((_hs_psize != 0U) || ((_hs_qsize != 0U) && (_hs_q != NULL))) {    \
+          if (_hs_psize == 0U) {                                                 \
+            _hs_e = _hs_q;                                                       \
+            _hs_q = ((_hs_q->next != NULL) ?                                     \
+              HH_FROM_ELMT((head)->hh.tbl, _hs_q->next) : NULL);                 \
+            _hs_qsize--;                                                         \
+          } else if ((_hs_qsize == 0U) || (_hs_q == NULL)) {                     \
+            _hs_e = _hs_p;                                                       \
+            if (_hs_p != NULL) {                                                 \
+              _hs_p = ((_hs_p->next != NULL) ?                                   \
+                HH_FROM_ELMT((head)->hh.tbl, _hs_p->next) : NULL);               \
+            }                                                                    \
+            _hs_psize--;                                                         \
+          } else if ((cmpfcn(                                                    \
+                DECLTYPE(head)(ELMT_FROM_HH((head)->hh.tbl, _hs_p)),             \
+                DECLTYPE(head)(ELMT_FROM_HH((head)->hh.tbl, _hs_q))              \
+                )) <= 0) {                                                       \
+            _hs_e = _hs_p;                                                       \
+            if (_hs_p != NULL) {                                                 \
+              _hs_p = ((_hs_p->next != NULL) ?                                   \
+                HH_FROM_ELMT((head)->hh.tbl, _hs_p->next) : NULL);               \
+            }                                                                    \
+            _hs_psize--;                                                         \
+          } else {                                                               \
+            _hs_e = _hs_q;                                                       \
+            _hs_q = ((_hs_q->next != NULL) ?                                     \
+              HH_FROM_ELMT((head)->hh.tbl, _hs_q->next) : NULL);                 \
+            _hs_qsize--;                                                         \
+          }                                                                      \
+          if ( _hs_tail != NULL ) {                                              \
+            _hs_tail->next = ((_hs_e != NULL) ?                                  \
+              ELMT_FROM_HH((head)->hh.tbl, _hs_e) : NULL);                       \
+          } else {                                                               \
+            _hs_list = _hs_e;                                                    \
+          }                                                                      \
+          if (_hs_e != NULL) {                                                   \
+            _hs_e->prev = ((_hs_tail != NULL) ?                                  \
+              ELMT_FROM_HH((head)->hh.tbl, _hs_tail) : NULL);                    \
+          }                                                                      \
+          _hs_tail = _hs_e;                                                      \
+        }                                                                        \
+        _hs_p = _hs_q;                                                           \
+      }                                                                          \
+      if (_hs_tail != NULL) {                                                    \
+        _hs_tail->next = NULL;                                                   \
+      }                                                                          \
+      if (_hs_nmerges <= 1U) {                                                   \
+        _hs_looping = 0;                                                         \
+        (head)->hh.tbl->tail = _hs_tail;                                         \
+        DECLTYPE_ASSIGN(head, ELMT_FROM_HH((head)->hh.tbl, _hs_list));           \
+      }                                                                          \
+      _hs_insize *= 2U;                                                          \
+    }                                                                            \
+    HASH_FSCK(hh, head, "HASH_SRT");                                             \
+  }                                                                              \
+} while (0)
+
+/* This function selects items from one hash into another hash.
+ * The end result is that the selected items have dual presence
+ * in both hashes. There is no copy of the items made; rather
+ * they are added into the new hash through a secondary hash
+ * hash handle that must be present in the structure. */
+#define HASH_SELECT(hh_dst, dst, hh_src, src, cond)                              \
+do {                                                                             \
+  unsigned _src_bkt, _dst_bkt;                                                   \
+  void *_last_elt = NULL, *_elt;                                                 \
+  UT_hash_handle *_src_hh, *_dst_hh, *_last_elt_hh=NULL;                         \
+  ptrdiff_t _dst_hho = ((char*)(&(dst)->hh_dst) - (char*)(dst));                 \
+  if ((src) != NULL) {                                                           \
+    for (_src_bkt=0; _src_bkt < (src)->hh_src.tbl->num_buckets; _src_bkt++) {    \
+      for (_src_hh = (src)->hh_src.tbl->buckets[_src_bkt].hh_head;               \
+        _src_hh != NULL;                                                         \
+        _src_hh = _src_hh->hh_next) {                                            \
+        _elt = ELMT_FROM_HH((src)->hh_src.tbl, _src_hh);                         \
+        if (cond(_elt)) {                                                        \
+          IF_HASH_NONFATAL_OOM( int _hs_oomed = 0; )                             \
+          _dst_hh = (UT_hash_handle*)(((char*)_elt) + _dst_hho);                 \
+          _dst_hh->key = _src_hh->key;                                           \
+          _dst_hh->keylen = _src_hh->keylen;                                     \
+          _dst_hh->hashv = _src_hh->hashv;                                       \
+          _dst_hh->prev = _last_elt;                                             \
+          _dst_hh->next = NULL;                                                  \
+          if (_last_elt_hh != NULL) {                                            \
+            _last_elt_hh->next = _elt;                                           \
+          }                                                                      \
+          if ((dst) == NULL) {                                                   \
+            DECLTYPE_ASSIGN(dst, _elt);                                          \
+            HASH_MAKE_TABLE(hh_dst, dst, _hs_oomed);                             \
+            IF_HASH_NONFATAL_OOM(                                                \
+              if (_hs_oomed) {                                                   \
+                uthash_nonfatal_oom(_elt);                                       \
+                (dst) = NULL;                                                    \
+                continue;                                                        \
+              }                                                                  \
+            )                                                                    \
+          } else {                                                               \
+            _dst_hh->tbl = (dst)->hh_dst.tbl;                                    \
+          }                                                                      \
+          HASH_TO_BKT(_dst_hh->hashv, _dst_hh->tbl->num_buckets, _dst_bkt);      \
+          HASH_ADD_TO_BKT(_dst_hh->tbl->buckets[_dst_bkt], hh_dst, _dst_hh, _hs_oomed); \
+          (dst)->hh_dst.tbl->num_items++;                                        \
+          IF_HASH_NONFATAL_OOM(                                                  \
+            if (_hs_oomed) {                                                     \
+              HASH_ROLLBACK_BKT(hh_dst, dst, _dst_hh);                           \
+              HASH_DELETE_HH(hh_dst, dst, _dst_hh);                              \
+              _dst_hh->tbl = NULL;                                               \
+              uthash_nonfatal_oom(_elt);                                         \
+              continue;                                                          \
+            }                                                                    \
+          )                                                                      \
+          HASH_BLOOM_ADD(_dst_hh->tbl, _dst_hh->hashv);                          \
+          _last_elt = _elt;                                                      \
+          _last_elt_hh = _dst_hh;                                                \
+        }                                                                        \
+      }                                                                          \
+    }                                                                            \
+  }                                                                              \
+  HASH_FSCK(hh_dst, dst, "HASH_SELECT");                                         \
+} while (0)
+
+#define HASH_CLEAR(hh,head)                                                      \
+do {                                                                             \
+  if ((head) != NULL) {                                                          \
+    HASH_BLOOM_FREE((head)->hh.tbl);                                             \
+    uthash_free((head)->hh.tbl->buckets,                                         \
+                (head)->hh.tbl->num_buckets*sizeof(struct UT_hash_bucket));      \
+    uthash_free((head)->hh.tbl, sizeof(UT_hash_table));                          \
+    (head) = NULL;                                                               \
+  }                                                                              \
+} while (0)
+
+#define HASH_OVERHEAD(hh,head)                                                   \
+ (((head) != NULL) ? (                                                           \
+ (size_t)(((head)->hh.tbl->num_items   * sizeof(UT_hash_handle))   +             \
+          ((head)->hh.tbl->num_buckets * sizeof(UT_hash_bucket))   +             \
+           sizeof(UT_hash_table)                                   +             \
+           (HASH_BLOOM_BYTELEN))) : 0U)
+
+#ifdef NO_DECLTYPE
+#define HASH_ITER(hh,head,el,tmp)                                                \
+for(((el)=(head)), ((*(char**)(&(tmp)))=(char*)((head!=NULL)?(head)->hh.next:NULL)); \
+  (el) != NULL; ((el)=(tmp)), ((*(char**)(&(tmp)))=(char*)((tmp!=NULL)?(tmp)->hh.next:NULL)))
+#else
+#define HASH_ITER(hh,head,el,tmp)                                                \
+for(((el)=(head)), ((tmp)=DECLTYPE(el)((head!=NULL)?(head)->hh.next:NULL));      \
+  (el) != NULL; ((el)=(tmp)), ((tmp)=DECLTYPE(el)((tmp!=NULL)?(tmp)->hh.next:NULL)))
+#endif
+
+/* obtain a count of items in the hash */
+#define HASH_COUNT(head) HASH_CNT(hh,head)
+#define HASH_CNT(hh,head) ((head != NULL)?((head)->hh.tbl->num_items):0U)
+
+typedef struct UT_hash_bucket {
+   struct UT_hash_handle *hh_head;
+   unsigned count;
+
+   /* expand_mult is normally set to 0. In this situation, the max chain length
+    * threshold is enforced at its default value, HASH_BKT_CAPACITY_THRESH. (If
+    * the bucket's chain exceeds this length, bucket expansion is triggered).
+    * However, setting expand_mult to a non-zero value delays bucket expansion
+    * (that would be triggered by additions to this particular bucket)
+    * until its chain length reaches a *multiple* of HASH_BKT_CAPACITY_THRESH.
+    * (The multiplier is simply expand_mult+1). The whole idea of this
+    * multiplier is to reduce bucket expansions, since they are expensive, in
+    * situations where we know that a particular bucket tends to be overused.
+    * It is better to let its chain length grow to a longer yet-still-bounded
+    * value, than to do an O(n) bucket expansion too often.
+    */
+   unsigned expand_mult;
+
+} UT_hash_bucket;
+
+/* random signature used only to find hash tables in external analysis */
+#define HASH_SIGNATURE 0xa0111fe1u
+#define HASH_BLOOM_SIGNATURE 0xb12220f2u
+
+typedef struct UT_hash_table {
+   UT_hash_bucket *buckets;
+   unsigned num_buckets, log2_num_buckets;
+   unsigned num_items;
+   struct UT_hash_handle *tail; /* tail hh in app order, for fast append    */
+   ptrdiff_t hho; /* hash handle offset (byte pos of hash handle in element */
+
+   /* in an ideal situation (all buckets used equally), no bucket would have
+    * more than ceil(#items/#buckets) items. that's the ideal chain length. */
+   unsigned ideal_chain_maxlen;
+
+   /* nonideal_items is the number of items in the hash whose chain position
+    * exceeds the ideal chain maxlen. these items pay the penalty for an uneven
+    * hash distribution; reaching them in a chain traversal takes >ideal steps */
+   unsigned nonideal_items;
+
+   /* ineffective expands occur when a bucket doubling was performed, but
+    * afterward, more than half the items in the hash had nonideal chain
+    * positions. If this happens on two consecutive expansions we inhibit any
+    * further expansion, as it's not helping; this happens when the hash
+    * function isn't a good fit for the key domain. When expansion is inhibited
+    * the hash will still work, albeit no longer in constant time. */
+   unsigned ineff_expands, noexpand;
+
+   uint32_t signature; /* used only to find hash tables in external analysis */
+#ifdef HASH_BLOOM
+   uint32_t bloom_sig; /* used only to test bloom exists in external analysis */
+   uint8_t *bloom_bv;
+   uint8_t bloom_nbits;
+#endif
+
+} UT_hash_table;
+
+typedef struct UT_hash_handle {
+   struct UT_hash_table *tbl;
+   void *prev;                       /* prev element in app order      */
+   void *next;                       /* next element in app order      */
+   struct UT_hash_handle *hh_prev;   /* previous hh in bucket order    */
+   struct UT_hash_handle *hh_next;   /* next hh in bucket order        */
+   void *key;                        /* ptr to enclosing struct's key  */
+   unsigned keylen;                  /* enclosing struct's key len     */
+   unsigned hashv;                   /* result of hash-fcn(key)        */
+} UT_hash_handle;
+
+#endif /* UTHASH_H */
diff --git a/thirdparty/ut/utlist.h b/thirdparty/ut/utlist.h
new file mode 100755
index 0000000..5bb1ac9
--- /dev/null
+++ b/thirdparty/ut/utlist.h
@@ -0,0 +1,1073 @@
+/*
+Copyright (c) 2007-2018, Troy D. Hanson   http://troydhanson.github.com/uthash/
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/
+
+#ifndef UTLIST_H
+#define UTLIST_H
+
+#define UTLIST_VERSION 2.1.0
+
+#include <assert.h>
+
+/*
+ * This file contains macros to manipulate singly and doubly-linked lists.
+ *
+ * 1. LL_ macros:  singly-linked lists.
+ * 2. DL_ macros:  doubly-linked lists.
+ * 3. CDL_ macros: circular doubly-linked lists.
+ *
+ * To use singly-linked lists, your structure must have a "next" pointer.
+ * To use doubly-linked lists, your structure must "prev" and "next" pointers.
+ * Either way, the pointer to the head of the list must be initialized to NULL.
+ *
+ * ----------------.EXAMPLE -------------------------
+ * struct item {
+ *      int id;
+ *      struct item *prev, *next;
+ * }
+ *
+ * struct item *list = NULL:
+ *
+ * int main() {
+ *      struct item *item;
+ *      ... allocate and populate item ...
+ *      DL_APPEND(list, item);
+ * }
+ * --------------------------------------------------
+ *
+ * For doubly-linked lists, the append and delete macros are O(1)
+ * For singly-linked lists, append and delete are O(n) but prepend is O(1)
+ * The sort macro is O(n log(n)) for all types of single/double/circular lists.
+ */
+
+/* These macros use decltype or the earlier __typeof GNU extension.
+   As decltype is only available in newer compilers (VS2010 or gcc 4.3+
+   when compiling c++ source) this code uses whatever method is needed
+   or, for VS2008 where neither is available, uses casting workarounds. */
+#if !defined(LDECLTYPE) && !defined(NO_DECLTYPE)
+#if defined(_MSC_VER)   /* MS compiler */
+#if _MSC_VER >= 1600 && defined(__cplusplus)  /* VS2010 or newer in C++ mode */
+#define LDECLTYPE(x) decltype(x)
+#else                   /* VS2008 or older (or VS2010 in C mode) */
+#define NO_DECLTYPE
+#endif
+#elif defined(__BORLANDC__) || defined(__ICCARM__) || defined(__LCC__) || defined(__WATCOMC__)
+#define NO_DECLTYPE
+#else                   /* GNU, Sun and other compilers */
+#define LDECLTYPE(x) __typeof(x)
+#endif
+#endif
+
+/* for VS2008 we use some workarounds to get around the lack of decltype,
+ * namely, we always reassign our tmp variable to the list head if we need
+ * to dereference its prev/next pointers, and save/restore the real head.*/
+#ifdef NO_DECLTYPE
+#define IF_NO_DECLTYPE(x) x
+#define LDECLTYPE(x) char*
+#define UTLIST_SV(elt,list) _tmp = (char*)(list); {char **_alias = (char**)&(list); *_alias = (elt); }
+#define UTLIST_NEXT(elt,list,next) ((char*)((list)->next))
+#define UTLIST_NEXTASGN(elt,list,to,next) { char **_alias = (char**)&((list)->next); *_alias=(char*)(to); }
+/* #define UTLIST_PREV(elt,list,prev) ((char*)((list)->prev)) */
+#define UTLIST_PREVASGN(elt,list,to,prev) { char **_alias = (char**)&((list)->prev); *_alias=(char*)(to); }
+#define UTLIST_RS(list) { char **_alias = (char**)&(list); *_alias=_tmp; }
+#define UTLIST_CASTASGN(a,b) { char **_alias = (char**)&(a); *_alias=(char*)(b); }
+#else
+#define IF_NO_DECLTYPE(x)
+#define UTLIST_SV(elt,list)
+#define UTLIST_NEXT(elt,list,next) ((elt)->next)
+#define UTLIST_NEXTASGN(elt,list,to,next) ((elt)->next)=(to)
+/* #define UTLIST_PREV(elt,list,prev) ((elt)->prev) */
+#define UTLIST_PREVASGN(elt,list,to,prev) ((elt)->prev)=(to)
+#define UTLIST_RS(list)
+#define UTLIST_CASTASGN(a,b) (a)=(b)
+#endif
+
+/******************************************************************************
+ * The sort macro is an adaptation of Simon Tatham's O(n log(n)) mergesort    *
+ * Unwieldy variable names used here to avoid shadowing passed-in variables.  *
+ *****************************************************************************/
+#define LL_SORT(list, cmp)                                                                     \
+    LL_SORT2(list, cmp, next)
+
+#define LL_SORT2(list, cmp, next)                                                              \
+do {                                                                                           \
+  LDECLTYPE(list) _ls_p;                                                                       \
+  LDECLTYPE(list) _ls_q;                                                                       \
+  LDECLTYPE(list) _ls_e;                                                                       \
+  LDECLTYPE(list) _ls_tail;                                                                    \
+  IF_NO_DECLTYPE(LDECLTYPE(list) _tmp;)                                                        \
+  int _ls_insize, _ls_nmerges, _ls_psize, _ls_qsize, _ls_i, _ls_looping;                       \
+  if (list) {                                                                                  \
+    _ls_insize = 1;                                                                            \
+    _ls_looping = 1;                                                                           \
+    while (_ls_looping) {                                                                      \
+      UTLIST_CASTASGN(_ls_p,list);                                                             \
+      (list) = NULL;                                                                           \
+      _ls_tail = NULL;                                                                         \
+      _ls_nmerges = 0;                                                                         \
+      while (_ls_p) {                                                                          \
+        _ls_nmerges++;                                                                         \
+        _ls_q = _ls_p;                                                                         \
+        _ls_psize = 0;                                                                         \
+        for (_ls_i = 0; _ls_i < _ls_insize; _ls_i++) {                                         \
+          _ls_psize++;                                                                         \
+          UTLIST_SV(_ls_q,list); _ls_q = UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list);        \
+          if (!_ls_q) break;                                                                   \
+        }                                                                                      \
+        _ls_qsize = _ls_insize;                                                                \
+        while (_ls_psize > 0 || (_ls_qsize > 0 && _ls_q)) {                                    \
+          if (_ls_psize == 0) {                                                                \
+            _ls_e = _ls_q; UTLIST_SV(_ls_q,list); _ls_q =                                      \
+              UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list); _ls_qsize--;                      \
+          } else if (_ls_qsize == 0 || !_ls_q) {                                               \
+            _ls_e = _ls_p; UTLIST_SV(_ls_p,list); _ls_p =                                      \
+              UTLIST_NEXT(_ls_p,list,next); UTLIST_RS(list); _ls_psize--;                      \
+          } else if (cmp(_ls_p,_ls_q) <= 0) {                                                  \
+            _ls_e = _ls_p; UTLIST_SV(_ls_p,list); _ls_p =                                      \
+              UTLIST_NEXT(_ls_p,list,next); UTLIST_RS(list); _ls_psize--;                      \
+          } else {                                                                             \
+            _ls_e = _ls_q; UTLIST_SV(_ls_q,list); _ls_q =                                      \
+              UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list); _ls_qsize--;                      \
+          }                                                                                    \
+          if (_ls_tail) {                                                                      \
+            UTLIST_SV(_ls_tail,list); UTLIST_NEXTASGN(_ls_tail,list,_ls_e,next); UTLIST_RS(list); \
+          } else {                                                                             \
+            UTLIST_CASTASGN(list,_ls_e);                                                       \
+          }                                                                                    \
+          _ls_tail = _ls_e;                                                                    \
+        }                                                                                      \
+        _ls_p = _ls_q;                                                                         \
+      }                                                                                        \
+      if (_ls_tail) {                                                                          \
+        UTLIST_SV(_ls_tail,list); UTLIST_NEXTASGN(_ls_tail,list,NULL,next); UTLIST_RS(list);   \
+      }                                                                                        \
+      if (_ls_nmerges <= 1) {                                                                  \
+        _ls_looping=0;                                                                         \
+      }                                                                                        \
+      _ls_insize *= 2;                                                                         \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+
+#define DL_SORT(list, cmp)                                                                     \
+    DL_SORT2(list, cmp, prev, next)
+
+#define DL_SORT2(list, cmp, prev, next)                                                        \
+do {                                                                                           \
+  LDECLTYPE(list) _ls_p;                                                                       \
+  LDECLTYPE(list) _ls_q;                                                                       \
+  LDECLTYPE(list) _ls_e;                                                                       \
+  LDECLTYPE(list) _ls_tail;                                                                    \
+  IF_NO_DECLTYPE(LDECLTYPE(list) _tmp;)                                                        \
+  int _ls_insize, _ls_nmerges, _ls_psize, _ls_qsize, _ls_i, _ls_looping;                       \
+  if (list) {                                                                                  \
+    _ls_insize = 1;                                                                            \
+    _ls_looping = 1;                                                                           \
+    while (_ls_looping) {                                                                      \
+      UTLIST_CASTASGN(_ls_p,list);                                                             \
+      (list) = NULL;                                                                           \
+      _ls_tail = NULL;                                                                         \
+      _ls_nmerges = 0;                                                                         \
+      while (_ls_p) {                                                                          \
+        _ls_nmerges++;                                                                         \
+        _ls_q = _ls_p;                                                                         \
+        _ls_psize = 0;                                                                         \
+        for (_ls_i = 0; _ls_i < _ls_insize; _ls_i++) {                                         \
+          _ls_psize++;                                                                         \
+          UTLIST_SV(_ls_q,list); _ls_q = UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list);        \
+          if (!_ls_q) break;                                                                   \
+        }                                                                                      \
+        _ls_qsize = _ls_insize;                                                                \
+        while ((_ls_psize > 0) || ((_ls_qsize > 0) && _ls_q)) {                                \
+          if (_ls_psize == 0) {                                                                \
+            _ls_e = _ls_q; UTLIST_SV(_ls_q,list); _ls_q =                                      \
+              UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list); _ls_qsize--;                      \
+          } else if ((_ls_qsize == 0) || (!_ls_q)) {                                           \
+            _ls_e = _ls_p; UTLIST_SV(_ls_p,list); _ls_p =                                      \
+              UTLIST_NEXT(_ls_p,list,next); UTLIST_RS(list); _ls_psize--;                      \
+          } else if (cmp(_ls_p,_ls_q) <= 0) {                                                  \
+            _ls_e = _ls_p; UTLIST_SV(_ls_p,list); _ls_p =                                      \
+              UTLIST_NEXT(_ls_p,list,next); UTLIST_RS(list); _ls_psize--;                      \
+          } else {                                                                             \
+            _ls_e = _ls_q; UTLIST_SV(_ls_q,list); _ls_q =                                      \
+              UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list); _ls_qsize--;                      \
+          }                                                                                    \
+          if (_ls_tail) {                                                                      \
+            UTLIST_SV(_ls_tail,list); UTLIST_NEXTASGN(_ls_tail,list,_ls_e,next); UTLIST_RS(list); \
+          } else {                                                                             \
+            UTLIST_CASTASGN(list,_ls_e);                                                       \
+          }                                                                                    \
+          UTLIST_SV(_ls_e,list); UTLIST_PREVASGN(_ls_e,list,_ls_tail,prev); UTLIST_RS(list);   \
+          _ls_tail = _ls_e;                                                                    \
+        }                                                                                      \
+        _ls_p = _ls_q;                                                                         \
+      }                                                                                        \
+      UTLIST_CASTASGN((list)->prev, _ls_tail);                                                 \
+      UTLIST_SV(_ls_tail,list); UTLIST_NEXTASGN(_ls_tail,list,NULL,next); UTLIST_RS(list);     \
+      if (_ls_nmerges <= 1) {                                                                  \
+        _ls_looping=0;                                                                         \
+      }                                                                                        \
+      _ls_insize *= 2;                                                                         \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+#define CDL_SORT(list, cmp)                                                                    \
+    CDL_SORT2(list, cmp, prev, next)
+
+#define CDL_SORT2(list, cmp, prev, next)                                                       \
+do {                                                                                           \
+  LDECLTYPE(list) _ls_p;                                                                       \
+  LDECLTYPE(list) _ls_q;                                                                       \
+  LDECLTYPE(list) _ls_e;                                                                       \
+  LDECLTYPE(list) _ls_tail;                                                                    \
+  LDECLTYPE(list) _ls_oldhead;                                                                 \
+  LDECLTYPE(list) _tmp;                                                                        \
+  int _ls_insize, _ls_nmerges, _ls_psize, _ls_qsize, _ls_i, _ls_looping;                       \
+  if (list) {                                                                                  \
+    _ls_insize = 1;                                                                            \
+    _ls_looping = 1;                                                                           \
+    while (_ls_looping) {                                                                      \
+      UTLIST_CASTASGN(_ls_p,list);                                                             \
+      UTLIST_CASTASGN(_ls_oldhead,list);                                                       \
+      (list) = NULL;                                                                           \
+      _ls_tail = NULL;                                                                         \
+      _ls_nmerges = 0;                                                                         \
+      while (_ls_p) {                                                                          \
+        _ls_nmerges++;                                                                         \
+        _ls_q = _ls_p;                                                                         \
+        _ls_psize = 0;                                                                         \
+        for (_ls_i = 0; _ls_i < _ls_insize; _ls_i++) {                                         \
+          _ls_psize++;                                                                         \
+          UTLIST_SV(_ls_q,list);                                                               \
+          if (UTLIST_NEXT(_ls_q,list,next) == _ls_oldhead) {                                   \
+            _ls_q = NULL;                                                                      \
+          } else {                                                                             \
+            _ls_q = UTLIST_NEXT(_ls_q,list,next);                                              \
+          }                                                                                    \
+          UTLIST_RS(list);                                                                     \
+          if (!_ls_q) break;                                                                   \
+        }                                                                                      \
+        _ls_qsize = _ls_insize;                                                                \
+        while (_ls_psize > 0 || (_ls_qsize > 0 && _ls_q)) {                                    \
+          if (_ls_psize == 0) {                                                                \
+            _ls_e = _ls_q; UTLIST_SV(_ls_q,list); _ls_q =                                      \
+              UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list); _ls_qsize--;                      \
+            if (_ls_q == _ls_oldhead) { _ls_q = NULL; }                                        \
+          } else if (_ls_qsize == 0 || !_ls_q) {                                               \
+            _ls_e = _ls_p; UTLIST_SV(_ls_p,list); _ls_p =                                      \
+              UTLIST_NEXT(_ls_p,list,next); UTLIST_RS(list); _ls_psize--;                      \
+            if (_ls_p == _ls_oldhead) { _ls_p = NULL; }                                        \
+          } else if (cmp(_ls_p,_ls_q) <= 0) {                                                  \
+            _ls_e = _ls_p; UTLIST_SV(_ls_p,list); _ls_p =                                      \
+              UTLIST_NEXT(_ls_p,list,next); UTLIST_RS(list); _ls_psize--;                      \
+            if (_ls_p == _ls_oldhead) { _ls_p = NULL; }                                        \
+          } else {                                                                             \
+            _ls_e = _ls_q; UTLIST_SV(_ls_q,list); _ls_q =                                      \
+              UTLIST_NEXT(_ls_q,list,next); UTLIST_RS(list); _ls_qsize--;                      \
+            if (_ls_q == _ls_oldhead) { _ls_q = NULL; }                                        \
+          }                                                                                    \
+          if (_ls_tail) {                                                                      \
+            UTLIST_SV(_ls_tail,list); UTLIST_NEXTASGN(_ls_tail,list,_ls_e,next); UTLIST_RS(list); \
+          } else {                                                                             \
+            UTLIST_CASTASGN(list,_ls_e);                                                       \
+          }                                                                                    \
+          UTLIST_SV(_ls_e,list); UTLIST_PREVASGN(_ls_e,list,_ls_tail,prev); UTLIST_RS(list);   \
+          _ls_tail = _ls_e;                                                                    \
+        }                                                                                      \
+        _ls_p = _ls_q;                                                                         \
+      }                                                                                        \
+      UTLIST_CASTASGN((list)->prev,_ls_tail);                                                  \
+      UTLIST_CASTASGN(_tmp,list);                                                              \
+      UTLIST_SV(_ls_tail,list); UTLIST_NEXTASGN(_ls_tail,list,_tmp,next); UTLIST_RS(list);     \
+      if (_ls_nmerges <= 1) {                                                                  \
+        _ls_looping=0;                                                                         \
+      }                                                                                        \
+      _ls_insize *= 2;                                                                         \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+/******************************************************************************
+ * singly linked list macros (non-circular)                                   *
+ *****************************************************************************/
+#define LL_PREPEND(head,add)                                                                   \
+    LL_PREPEND2(head,add,next)
+
+#define LL_PREPEND2(head,add,next)                                                             \
+do {                                                                                           \
+  (add)->next = (head);                                                                        \
+  (head) = (add);                                                                              \
+} while (0)
+
+#define LL_CONCAT(head1,head2)                                                                 \
+    LL_CONCAT2(head1,head2,next)
+
+#define LL_CONCAT2(head1,head2,next)                                                           \
+do {                                                                                           \
+  LDECLTYPE(head1) _tmp;                                                                       \
+  if (head1) {                                                                                 \
+    _tmp = (head1);                                                                            \
+    while (_tmp->next) { _tmp = _tmp->next; }                                                  \
+    _tmp->next=(head2);                                                                        \
+  } else {                                                                                     \
+    (head1)=(head2);                                                                           \
+  }                                                                                            \
+} while (0)
+
+#define LL_APPEND(head,add)                                                                    \
+    LL_APPEND2(head,add,next)
+
+#define LL_APPEND2(head,add,next)                                                              \
+do {                                                                                           \
+  LDECLTYPE(head) _tmp;                                                                        \
+  (add)->next=NULL;                                                                            \
+  if (head) {                                                                                  \
+    _tmp = (head);                                                                             \
+    while (_tmp->next) { _tmp = _tmp->next; }                                                  \
+    _tmp->next=(add);                                                                          \
+  } else {                                                                                     \
+    (head)=(add);                                                                              \
+  }                                                                                            \
+} while (0)
+
+#define LL_INSERT_INORDER(head,add,cmp)                                                        \
+    LL_INSERT_INORDER2(head,add,cmp,next)
+
+#define LL_INSERT_INORDER2(head,add,cmp,next)                                                  \
+do {                                                                                           \
+  LDECLTYPE(head) _tmp;                                                                        \
+  if (head) {                                                                                  \
+    LL_LOWER_BOUND2(head, _tmp, add, cmp, next);                                               \
+    LL_APPEND_ELEM2(head, _tmp, add, next);                                                    \
+  } else {                                                                                     \
+    (head) = (add);                                                                            \
+    (head)->next = NULL;                                                                       \
+  }                                                                                            \
+} while (0)
+
+#define LL_LOWER_BOUND(head,elt,like,cmp)                                                      \
+    LL_LOWER_BOUND2(head,elt,like,cmp,next)
+
+#define LL_LOWER_BOUND2(head,elt,like,cmp,next)                                                \
+  do {                                                                                         \
+    if ((head) == NULL || (cmp(head, like)) >= 0) {                                            \
+      (elt) = NULL;                                                                            \
+    } else {                                                                                   \
+      for ((elt) = (head); (elt)->next != NULL; (elt) = (elt)->next) {                         \
+        if (cmp((elt)->next, like) >= 0) {                                                     \
+          break;                                                                               \
+        }                                                                                      \
+      }                                                                                        \
+    }                                                                                          \
+  } while (0)
+
+#define LL_DELETE(head,del)                                                                    \
+    LL_DELETE2(head,del,next)
+
+#define LL_DELETE2(head,del,next)                                                              \
+do {                                                                                           \
+  LDECLTYPE(head) _tmp;                                                                        \
+  if ((head) == (del)) {                                                                       \
+    (head)=(head)->next;                                                                       \
+  } else {                                                                                     \
+    _tmp = (head);                                                                             \
+    while (_tmp->next && (_tmp->next != (del))) {                                              \
+      _tmp = _tmp->next;                                                                       \
+    }                                                                                          \
+    if (_tmp->next) {                                                                          \
+      _tmp->next = (del)->next;                                                                \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+#define LL_COUNT(head,el,counter)                                                              \
+    LL_COUNT2(head,el,counter,next)                                                            \
+
+#define LL_COUNT2(head,el,counter,next)                                                        \
+do {                                                                                           \
+  (counter) = 0;                                                                               \
+  LL_FOREACH2(head,el,next) { ++(counter); }                                                   \
+} while (0)
+
+#define LL_FOREACH(head,el)                                                                    \
+    LL_FOREACH2(head,el,next)
+
+#define LL_FOREACH2(head,el,next)                                                              \
+    for ((el) = (head); el; (el) = (el)->next)
+
+#define LL_FOREACH_SAFE(head,el,tmp)                                                           \
+    LL_FOREACH_SAFE2(head,el,tmp,next)
+
+#define LL_FOREACH_SAFE2(head,el,tmp,next)                                                     \
+  for ((el) = (head); (el) && ((tmp) = (el)->next, 1); (el) = (tmp))
+
+#define LL_SEARCH_SCALAR(head,out,field,val)                                                   \
+    LL_SEARCH_SCALAR2(head,out,field,val,next)
+
+#define LL_SEARCH_SCALAR2(head,out,field,val,next)                                             \
+do {                                                                                           \
+    LL_FOREACH2(head,out,next) {                                                               \
+      if ((out)->field == (val)) break;                                                        \
+    }                                                                                          \
+} while (0)
+
+#define LL_SEARCH(head,out,elt,cmp)                                                            \
+    LL_SEARCH2(head,out,elt,cmp,next)
+
+#define LL_SEARCH2(head,out,elt,cmp,next)                                                      \
+do {                                                                                           \
+    LL_FOREACH2(head,out,next) {                                                               \
+      if ((cmp(out,elt))==0) break;                                                            \
+    }                                                                                          \
+} while (0)
+
+#define LL_REPLACE_ELEM2(head, el, add, next)                                                  \
+do {                                                                                           \
+ LDECLTYPE(head) _tmp;                                                                         \
+ assert((head) != NULL);                                                                       \
+ assert((el) != NULL);                                                                         \
+ assert((add) != NULL);                                                                        \
+ (add)->next = (el)->next;                                                                     \
+ if ((head) == (el)) {                                                                         \
+  (head) = (add);                                                                              \
+ } else {                                                                                      \
+  _tmp = (head);                                                                               \
+  while (_tmp->next && (_tmp->next != (el))) {                                                 \
+   _tmp = _tmp->next;                                                                          \
+  }                                                                                            \
+  if (_tmp->next) {                                                                            \
+    _tmp->next = (add);                                                                        \
+  }                                                                                            \
+ }                                                                                             \
+} while (0)
+
+#define LL_REPLACE_ELEM(head, el, add)                                                         \
+    LL_REPLACE_ELEM2(head, el, add, next)
+
+#define LL_PREPEND_ELEM2(head, el, add, next)                                                  \
+do {                                                                                           \
+ if (el) {                                                                                     \
+  LDECLTYPE(head) _tmp;                                                                        \
+  assert((head) != NULL);                                                                      \
+  assert((add) != NULL);                                                                       \
+  (add)->next = (el);                                                                          \
+  if ((head) == (el)) {                                                                        \
+   (head) = (add);                                                                             \
+  } else {                                                                                     \
+   _tmp = (head);                                                                              \
+   while (_tmp->next && (_tmp->next != (el))) {                                                \
+    _tmp = _tmp->next;                                                                         \
+   }                                                                                           \
+   if (_tmp->next) {                                                                           \
+     _tmp->next = (add);                                                                       \
+   }                                                                                           \
+  }                                                                                            \
+ } else {                                                                                      \
+  LL_APPEND2(head, add, next);                                                                 \
+ }                                                                                             \
+} while (0)                                                                                    \
+
+#define LL_PREPEND_ELEM(head, el, add)                                                         \
+    LL_PREPEND_ELEM2(head, el, add, next)
+
+#define LL_APPEND_ELEM2(head, el, add, next)                                                   \
+do {                                                                                           \
+ if (el) {                                                                                     \
+  assert((head) != NULL);                                                                      \
+  assert((add) != NULL);                                                                       \
+  (add)->next = (el)->next;                                                                    \
+  (el)->next = (add);                                                                          \
+ } else {                                                                                      \
+  LL_PREPEND2(head, add, next);                                                                \
+ }                                                                                             \
+} while (0)                                                                                    \
+
+#define LL_APPEND_ELEM(head, el, add)                                                          \
+    LL_APPEND_ELEM2(head, el, add, next)
+
+#ifdef NO_DECLTYPE
+/* Here are VS2008 / NO_DECLTYPE replacements for a few functions */
+
+#undef LL_CONCAT2
+#define LL_CONCAT2(head1,head2,next)                                                           \
+do {                                                                                           \
+  char *_tmp;                                                                                  \
+  if (head1) {                                                                                 \
+    _tmp = (char*)(head1);                                                                     \
+    while ((head1)->next) { (head1) = (head1)->next; }                                         \
+    (head1)->next = (head2);                                                                   \
+    UTLIST_RS(head1);                                                                          \
+  } else {                                                                                     \
+    (head1)=(head2);                                                                           \
+  }                                                                                            \
+} while (0)
+
+#undef LL_APPEND2
+#define LL_APPEND2(head,add,next)                                                              \
+do {                                                                                           \
+  if (head) {                                                                                  \
+    (add)->next = head;     /* use add->next as a temp variable */                             \
+    while ((add)->next->next) { (add)->next = (add)->next->next; }                             \
+    (add)->next->next=(add);                                                                   \
+  } else {                                                                                     \
+    (head)=(add);                                                                              \
+  }                                                                                            \
+  (add)->next=NULL;                                                                            \
+} while (0)
+
+#undef LL_INSERT_INORDER2
+#define LL_INSERT_INORDER2(head,add,cmp,next)                                                  \
+do {                                                                                           \
+  if ((head) == NULL || (cmp(head, add)) >= 0) {                                               \
+    (add)->next = (head);                                                                      \
+    (head) = (add);                                                                            \
+  } else {                                                                                     \
+    char *_tmp = (char*)(head);                                                                \
+    while ((head)->next != NULL && (cmp((head)->next, add)) < 0) {                             \
+      (head) = (head)->next;                                                                   \
+    }                                                                                          \
+    (add)->next = (head)->next;                                                                \
+    (head)->next = (add);                                                                      \
+    UTLIST_RS(head);                                                                           \
+  }                                                                                            \
+} while (0)
+
+#undef LL_DELETE2
+#define LL_DELETE2(head,del,next)                                                              \
+do {                                                                                           \
+  if ((head) == (del)) {                                                                       \
+    (head)=(head)->next;                                                                       \
+  } else {                                                                                     \
+    char *_tmp = (char*)(head);                                                                \
+    while ((head)->next && ((head)->next != (del))) {                                          \
+      (head) = (head)->next;                                                                   \
+    }                                                                                          \
+    if ((head)->next) {                                                                        \
+      (head)->next = ((del)->next);                                                            \
+    }                                                                                          \
+    UTLIST_RS(head);                                                                           \
+  }                                                                                            \
+} while (0)
+
+#undef LL_REPLACE_ELEM2
+#define LL_REPLACE_ELEM2(head, el, add, next)                                                  \
+do {                                                                                           \
+  assert((head) != NULL);                                                                      \
+  assert((el) != NULL);                                                                        \
+  assert((add) != NULL);                                                                       \
+  if ((head) == (el)) {                                                                        \
+    (head) = (add);                                                                            \
+  } else {                                                                                     \
+    (add)->next = head;                                                                        \
+    while ((add)->next->next && ((add)->next->next != (el))) {                                 \
+      (add)->next = (add)->next->next;                                                         \
+    }                                                                                          \
+    if ((add)->next->next) {                                                                   \
+      (add)->next->next = (add);                                                               \
+    }                                                                                          \
+  }                                                                                            \
+  (add)->next = (el)->next;                                                                    \
+} while (0)
+
+#undef LL_PREPEND_ELEM2
+#define LL_PREPEND_ELEM2(head, el, add, next)                                                  \
+do {                                                                                           \
+  if (el) {                                                                                    \
+    assert((head) != NULL);                                                                    \
+    assert((add) != NULL);                                                                     \
+    if ((head) == (el)) {                                                                      \
+      (head) = (add);                                                                          \
+    } else {                                                                                   \
+      (add)->next = (head);                                                                    \
+      while ((add)->next->next && ((add)->next->next != (el))) {                               \
+        (add)->next = (add)->next->next;                                                       \
+      }                                                                                        \
+      if ((add)->next->next) {                                                                 \
+        (add)->next->next = (add);                                                             \
+      }                                                                                        \
+    }                                                                                          \
+    (add)->next = (el);                                                                        \
+  } else {                                                                                     \
+    LL_APPEND2(head, add, next);                                                               \
+  }                                                                                            \
+} while (0)                                                                                    \
+
+#endif /* NO_DECLTYPE */
+
+/******************************************************************************
+ * doubly linked list macros (non-circular)                                   *
+ *****************************************************************************/
+#define DL_PREPEND(head,add)                                                                   \
+    DL_PREPEND2(head,add,prev,next)
+
+#define DL_PREPEND2(head,add,prev,next)                                                        \
+do {                                                                                           \
+ (add)->next = (head);                                                                         \
+ if (head) {                                                                                   \
+   (add)->prev = (head)->prev;                                                                 \
+   (head)->prev = (add);                                                                       \
+ } else {                                                                                      \
+   (add)->prev = (add);                                                                        \
+ }                                                                                             \
+ (head) = (add);                                                                               \
+} while (0)
+
+#define DL_APPEND(head,add)                                                                    \
+    DL_APPEND2(head,add,prev,next)
+
+#define DL_APPEND2(head,add,prev,next)                                                         \
+do {                                                                                           \
+  if (head) {                                                                                  \
+      (add)->prev = (head)->prev;                                                              \
+      (head)->prev->next = (add);                                                              \
+      (head)->prev = (add);                                                                    \
+      (add)->next = NULL;                                                                      \
+  } else {                                                                                     \
+      (head)=(add);                                                                            \
+      (head)->prev = (head);                                                                   \
+      (head)->next = NULL;                                                                     \
+  }                                                                                            \
+} while (0)
+
+#define DL_INSERT_INORDER(head,add,cmp)                                                        \
+    DL_INSERT_INORDER2(head,add,cmp,prev,next)
+
+#define DL_INSERT_INORDER2(head,add,cmp,prev,next)                                             \
+do {                                                                                           \
+  LDECLTYPE(head) _tmp;                                                                        \
+  if (head) {                                                                                  \
+    DL_LOWER_BOUND2(head, _tmp, add, cmp, next);                                               \
+    DL_APPEND_ELEM2(head, _tmp, add, prev, next);                                              \
+  } else {                                                                                     \
+    (head) = (add);                                                                            \
+    (head)->prev = (head);                                                                     \
+    (head)->next = NULL;                                                                       \
+  }                                                                                            \
+} while (0)
+
+#define DL_LOWER_BOUND(head,elt,like,cmp)                                                      \
+    DL_LOWER_BOUND2(head,elt,like,cmp,next)
+
+#define DL_LOWER_BOUND2(head,elt,like,cmp,next)                                                \
+do {                                                                                           \
+  if ((head) == NULL || (cmp(head, like)) >= 0) {                                              \
+    (elt) = NULL;                                                                              \
+  } else {                                                                                     \
+    for ((elt) = (head); (elt)->next != NULL; (elt) = (elt)->next) {                           \
+      if ((cmp((elt)->next, like)) >= 0) {                                                     \
+        break;                                                                                 \
+      }                                                                                        \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+#define DL_CONCAT(head1,head2)                                                                 \
+    DL_CONCAT2(head1,head2,prev,next)
+
+#define DL_CONCAT2(head1,head2,prev,next)                                                      \
+do {                                                                                           \
+  LDECLTYPE(head1) _tmp;                                                                       \
+  if (head2) {                                                                                 \
+    if (head1) {                                                                               \
+        UTLIST_CASTASGN(_tmp, (head2)->prev);                                                  \
+        (head2)->prev = (head1)->prev;                                                         \
+        (head1)->prev->next = (head2);                                                         \
+        UTLIST_CASTASGN((head1)->prev, _tmp);                                                  \
+    } else {                                                                                   \
+        (head1)=(head2);                                                                       \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+#define DL_DELETE(head,del)                                                                    \
+    DL_DELETE2(head,del,prev,next)
+
+#define DL_DELETE2(head,del,prev,next)                                                         \
+do {                                                                                           \
+  assert((head) != NULL);                                                                      \
+  assert((del)->prev != NULL);                                                                 \
+  if ((del)->prev == (del)) {                                                                  \
+      (head)=NULL;                                                                             \
+  } else if ((del)==(head)) {                                                                  \
+      (del)->next->prev = (del)->prev;                                                         \
+      (head) = (del)->next;                                                                    \
+  } else {                                                                                     \
+      (del)->prev->next = (del)->next;                                                         \
+      if ((del)->next) {                                                                       \
+          (del)->next->prev = (del)->prev;                                                     \
+      } else {                                                                                 \
+          (head)->prev = (del)->prev;                                                          \
+      }                                                                                        \
+  }                                                                                            \
+} while (0)
+
+#define DL_COUNT(head,el,counter)                                                              \
+    DL_COUNT2(head,el,counter,next)                                                            \
+
+#define DL_COUNT2(head,el,counter,next)                                                        \
+do {                                                                                           \
+  (counter) = 0;                                                                               \
+  DL_FOREACH2(head,el,next) { ++(counter); }                                                   \
+} while (0)
+
+#define DL_FOREACH(head,el)                                                                    \
+    DL_FOREACH2(head,el,next)
+
+#define DL_FOREACH2(head,el,next)                                                              \
+    for ((el) = (head); el; (el) = (el)->next)
+
+/* this version is safe for deleting the elements during iteration */
+#define DL_FOREACH_SAFE(head,el,tmp)                                                           \
+    DL_FOREACH_SAFE2(head,el,tmp,next)
+
+#define DL_FOREACH_SAFE2(head,el,tmp,next)                                                     \
+  for ((el) = (head); (el) && ((tmp) = (el)->next, 1); (el) = (tmp))
+
+/* these are identical to their singly-linked list counterparts */
+#define DL_SEARCH_SCALAR LL_SEARCH_SCALAR
+#define DL_SEARCH LL_SEARCH
+#define DL_SEARCH_SCALAR2 LL_SEARCH_SCALAR2
+#define DL_SEARCH2 LL_SEARCH2
+
+#define DL_REPLACE_ELEM2(head, el, add, prev, next)                                            \
+do {                                                                                           \
+ assert((head) != NULL);                                                                       \
+ assert((el) != NULL);                                                                         \
+ assert((add) != NULL);                                                                        \
+ if ((head) == (el)) {                                                                         \
+  (head) = (add);                                                                              \
+  (add)->next = (el)->next;                                                                    \
+  if ((el)->next == NULL) {                                                                    \
+   (add)->prev = (add);                                                                        \
+  } else {                                                                                     \
+   (add)->prev = (el)->prev;                                                                   \
+   (add)->next->prev = (add);                                                                  \
+  }                                                                                            \
+ } else {                                                                                      \
+  (add)->next = (el)->next;                                                                    \
+  (add)->prev = (el)->prev;                                                                    \
+  (add)->prev->next = (add);                                                                   \
+  if ((el)->next == NULL) {                                                                    \
+   (head)->prev = (add);                                                                       \
+  } else {                                                                                     \
+   (add)->next->prev = (add);                                                                  \
+  }                                                                                            \
+ }                                                                                             \
+} while (0)
+
+#define DL_REPLACE_ELEM(head, el, add)                                                         \
+    DL_REPLACE_ELEM2(head, el, add, prev, next)
+
+#define DL_PREPEND_ELEM2(head, el, add, prev, next)                                            \
+do {                                                                                           \
+ if (el) {                                                                                     \
+  assert((head) != NULL);                                                                      \
+  assert((add) != NULL);                                                                       \
+  (add)->next = (el);                                                                          \
+  (add)->prev = (el)->prev;                                                                    \
+  (el)->prev = (add);                                                                          \
+  if ((head) == (el)) {                                                                        \
+   (head) = (add);                                                                             \
+  } else {                                                                                     \
+   (add)->prev->next = (add);                                                                  \
+  }                                                                                            \
+ } else {                                                                                      \
+  DL_APPEND2(head, add, prev, next);                                                           \
+ }                                                                                             \
+} while (0)                                                                                    \
+
+#define DL_PREPEND_ELEM(head, el, add)                                                         \
+    DL_PREPEND_ELEM2(head, el, add, prev, next)
+
+#define DL_APPEND_ELEM2(head, el, add, prev, next)                                             \
+do {                                                                                           \
+ if (el) {                                                                                     \
+  assert((head) != NULL);                                                                      \
+  assert((add) != NULL);                                                                       \
+  (add)->next = (el)->next;                                                                    \
+  (add)->prev = (el);                                                                          \
+  (el)->next = (add);                                                                          \
+  if ((add)->next) {                                                                           \
+   (add)->next->prev = (add);                                                                  \
+  } else {                                                                                     \
+   (head)->prev = (add);                                                                       \
+  }                                                                                            \
+ } else {                                                                                      \
+  DL_PREPEND2(head, add, prev, next);                                                          \
+ }                                                                                             \
+} while (0)                                                                                    \
+
+#define DL_APPEND_ELEM(head, el, add)                                                          \
+   DL_APPEND_ELEM2(head, el, add, prev, next)
+
+#ifdef NO_DECLTYPE
+/* Here are VS2008 / NO_DECLTYPE replacements for a few functions */
+
+#undef DL_INSERT_INORDER2
+#define DL_INSERT_INORDER2(head,add,cmp,prev,next)                                             \
+do {                                                                                           \
+  if ((head) == NULL) {                                                                        \
+    (add)->prev = (add);                                                                       \
+    (add)->next = NULL;                                                                        \
+    (head) = (add);                                                                            \
+  } else if ((cmp(head, add)) >= 0) {                                                          \
+    (add)->prev = (head)->prev;                                                                \
+    (add)->next = (head);                                                                      \
+    (head)->prev = (add);                                                                      \
+    (head) = (add);                                                                            \
+  } else {                                                                                     \
+    char *_tmp = (char*)(head);                                                                \
+    while ((head)->next && (cmp((head)->next, add)) < 0) {                                     \
+      (head) = (head)->next;                                                                   \
+    }                                                                                          \
+    (add)->prev = (head);                                                                      \
+    (add)->next = (head)->next;                                                                \
+    (head)->next = (add);                                                                      \
+    UTLIST_RS(head);                                                                           \
+    if ((add)->next) {                                                                         \
+      (add)->next->prev = (add);                                                               \
+    } else {                                                                                   \
+      (head)->prev = (add);                                                                    \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+#endif /* NO_DECLTYPE */
+
+/******************************************************************************
+ * circular doubly linked list macros                                         *
+ *****************************************************************************/
+#define CDL_APPEND(head,add)                                                                   \
+    CDL_APPEND2(head,add,prev,next)
+
+#define CDL_APPEND2(head,add,prev,next)                                                        \
+do {                                                                                           \
+ if (head) {                                                                                   \
+   (add)->prev = (head)->prev;                                                                 \
+   (add)->next = (head);                                                                       \
+   (head)->prev = (add);                                                                       \
+   (add)->prev->next = (add);                                                                  \
+ } else {                                                                                      \
+   (add)->prev = (add);                                                                        \
+   (add)->next = (add);                                                                        \
+   (head) = (add);                                                                             \
+ }                                                                                             \
+} while (0)
+
+#define CDL_PREPEND(head,add)                                                                  \
+    CDL_PREPEND2(head,add,prev,next)
+
+#define CDL_PREPEND2(head,add,prev,next)                                                       \
+do {                                                                                           \
+ if (head) {                                                                                   \
+   (add)->prev = (head)->prev;                                                                 \
+   (add)->next = (head);                                                                       \
+   (head)->prev = (add);                                                                       \
+   (add)->prev->next = (add);                                                                  \
+ } else {                                                                                      \
+   (add)->prev = (add);                                                                        \
+   (add)->next = (add);                                                                        \
+ }                                                                                             \
+ (head) = (add);                                                                               \
+} while (0)
+
+#define CDL_INSERT_INORDER(head,add,cmp)                                                       \
+    CDL_INSERT_INORDER2(head,add,cmp,prev,next)
+
+#define CDL_INSERT_INORDER2(head,add,cmp,prev,next)                                            \
+do {                                                                                           \
+  LDECLTYPE(head) _tmp;                                                                        \
+  if (head) {                                                                                  \
+    CDL_LOWER_BOUND2(head, _tmp, add, cmp, next);                                              \
+    CDL_APPEND_ELEM2(head, _tmp, add, prev, next);                                             \
+  } else {                                                                                     \
+    (head) = (add);                                                                            \
+    (head)->next = (head);                                                                     \
+    (head)->prev = (head);                                                                     \
+  }                                                                                            \
+} while (0)
+
+#define CDL_LOWER_BOUND(head,elt,like,cmp)                                                     \
+    CDL_LOWER_BOUND2(head,elt,like,cmp,next)
+
+#define CDL_LOWER_BOUND2(head,elt,like,cmp,next)                                               \
+do {                                                                                           \
+  if ((head) == NULL || (cmp(head, like)) >= 0) {                                              \
+    (elt) = NULL;                                                                              \
+  } else {                                                                                     \
+    for ((elt) = (head); (elt)->next != (head); (elt) = (elt)->next) {                         \
+      if ((cmp((elt)->next, like)) >= 0) {                                                     \
+        break;                                                                                 \
+      }                                                                                        \
+    }                                                                                          \
+  }                                                                                            \
+} while (0)
+
+#define CDL_DELETE(head,del)                                                                   \
+    CDL_DELETE2(head,del,prev,next)
+
+#define CDL_DELETE2(head,del,prev,next)                                                        \
+do {                                                                                           \
+  if (((head)==(del)) && ((head)->next == (head))) {                                           \
+      (head) = NULL;                                                                           \
+  } else {                                                                                     \
+     (del)->next->prev = (del)->prev;                                                          \
+     (del)->prev->next = (del)->next;                                                          \
+     if ((del) == (head)) (head)=(del)->next;                                                  \
+  }                                                                                            \
+} while (0)
+
+#define CDL_COUNT(head,el,counter)                                                             \
+    CDL_COUNT2(head,el,counter,next)                                                           \
+
+#define CDL_COUNT2(head, el, counter,next)                                                     \
+do {                                                                                           \
+  (counter) = 0;                                                                               \
+  CDL_FOREACH2(head,el,next) { ++(counter); }                                                  \
+} while (0)
+
+#define CDL_FOREACH(head,el)                                                                   \
+    CDL_FOREACH2(head,el,next)
+
+#define CDL_FOREACH2(head,el,next)                                                             \
+    for ((el)=(head);el;(el)=(((el)->next==(head)) ? NULL : (el)->next))
+
+#define CDL_FOREACH_SAFE(head,el,tmp1,tmp2)                                                    \
+    CDL_FOREACH_SAFE2(head,el,tmp1,tmp2,prev,next)
+
+#define CDL_FOREACH_SAFE2(head,el,tmp1,tmp2,prev,next)                                         \
+  for ((el) = (head), (tmp1) = (head) ? (head)->prev : NULL;                                   \
+       (el) && ((tmp2) = (el)->next, 1);                                                       \
+       (el) = ((el) == (tmp1) ? NULL : (tmp2)))
+
+#define CDL_SEARCH_SCALAR(head,out,field,val)                                                  \
+    CDL_SEARCH_SCALAR2(head,out,field,val,next)
+
+#define CDL_SEARCH_SCALAR2(head,out,field,val,next)                                            \
+do {                                                                                           \
+    CDL_FOREACH2(head,out,next) {                                                              \
+      if ((out)->field == (val)) break;                                                        \
+    }                                                                                          \
+} while (0)
+
+#define CDL_SEARCH(head,out,elt,cmp)                                                           \
+    CDL_SEARCH2(head,out,elt,cmp,next)
+
+#define CDL_SEARCH2(head,out,elt,cmp,next)                                                     \
+do {                                                                                           \
+    CDL_FOREACH2(head,out,next) {                                                              \
+      if ((cmp(out,elt))==0) break;                                                            \
+    }                                                                                          \
+} while (0)
+
+#define CDL_REPLACE_ELEM2(head, el, add, prev, next)                                           \
+do {                                                                                           \
+ assert((head) != NULL);                                                                       \
+ assert((el) != NULL);                                                                         \
+ assert((add) != NULL);                                                                        \
+ if ((el)->next == (el)) {                                                                     \
+  (add)->next = (add);                                                                         \
+  (add)->prev = (add);                                                                         \
+  (head) = (add);                                                                              \
+ } else {                                                                                      \
+  (add)->next = (el)->next;                                                                    \
+  (add)->prev = (el)->prev;                                                                    \
+  (add)->next->prev = (add);                                                                   \
+  (add)->prev->next = (add);                                                                   \
+  if ((head) == (el)) {                                                                        \
+   (head) = (add);                                                                             \
+  }                                                                                            \
+ }                                                                                             \
+} while (0)
+
+#define CDL_REPLACE_ELEM(head, el, add)                                                        \
+    CDL_REPLACE_ELEM2(head, el, add, prev, next)
+
+#define CDL_PREPEND_ELEM2(head, el, add, prev, next)                                           \
+do {                                                                                           \
+  if (el) {                                                                                    \
+    assert((head) != NULL);                                                                    \
+    assert((add) != NULL);                                                                     \
+    (add)->next = (el);                                                                        \
+    (add)->prev = (el)->prev;                                                                  \
+    (el)->prev = (add);                                                                        \
+    (add)->prev->next = (add);                                                                 \
+    if ((head) == (el)) {                                                                      \
+      (head) = (add);                                                                          \
+    }                                                                                          \
+  } else {                                                                                     \
+    CDL_APPEND2(head, add, prev, next);                                                        \
+  }                                                                                            \
+} while (0)
+
+#define CDL_PREPEND_ELEM(head, el, add)                                                        \
+    CDL_PREPEND_ELEM2(head, el, add, prev, next)
+
+#define CDL_APPEND_ELEM2(head, el, add, prev, next)                                            \
+do {                                                                                           \
+ if (el) {                                                                                     \
+  assert((head) != NULL);                                                                      \
+  assert((add) != NULL);                                                                       \
+  (add)->next = (el)->next;                                                                    \
+  (add)->prev = (el);                                                                          \
+  (el)->next = (add);                                                                          \
+  (add)->next->prev = (add);                                                                   \
+ } else {                                                                                      \
+  CDL_PREPEND2(head, add, prev, next);                                                         \
+ }                                                                                             \
+} while (0)
+
+#define CDL_APPEND_ELEM(head, el, add)                                                         \
+    CDL_APPEND_ELEM2(head, el, add, prev, next)
+
+#ifdef NO_DECLTYPE
+/* Here are VS2008 / NO_DECLTYPE replacements for a few functions */
+
+#undef CDL_INSERT_INORDER2
+#define CDL_INSERT_INORDER2(head,add,cmp,prev,next)                                            \
+do {                                                                                           \
+  if ((head) == NULL) {                                                                        \
+    (add)->prev = (add);                                                                       \
+    (add)->next = (add);                                                                       \
+    (head) = (add);                                                                            \
+  } else if ((cmp(head, add)) >= 0) {                                                          \
+    (add)->prev = (head)->prev;                                                                \
+    (add)->next = (head);                                                                      \
+    (add)->prev->next = (add);                                                                 \
+    (head)->prev = (add);                                                                      \
+    (head) = (add);                                                                            \
+  } else {                                                                                     \
+    char *_tmp = (char*)(head);                                                                \
+    while ((char*)(head)->next != _tmp && (cmp((head)->next, add)) < 0) {                      \
+      (head) = (head)->next;                                                                   \
+    }                                                                                          \
+    (add)->prev = (head);                                                                      \
+    (add)->next = (head)->next;                                                                \
+    (add)->next->prev = (add);                                                                 \
+    (head)->next = (add);                                                                      \
+    UTLIST_RS(head);                                                                           \
+  }                                                                                            \
+} while (0)
+#endif /* NO_DECLTYPE */
+
+#endif /* UTLIST_H */
diff --git a/thirdparty/ut/utringbuffer.h b/thirdparty/ut/utringbuffer.h
new file mode 100755
index 0000000..ce2890e
--- /dev/null
+++ b/thirdparty/ut/utringbuffer.h
@@ -0,0 +1,108 @@
+/*
+Copyright (c) 2015-2018, Troy D. Hanson   http://troydhanson.github.com/uthash/
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/
+
+/* a ring-buffer implementation using macros
+ */
+#ifndef UTRINGBUFFER_H
+#define UTRINGBUFFER_H
+
+#define UTRINGBUFFER_VERSION 2.1.0
+
+#include <stdlib.h>
+#include <string.h>
+#include "utarray.h"  // for "UT_icd"
+
+typedef struct {
+    unsigned i;       /* index of next available slot; wraps at n */
+    unsigned n;       /* capacity */
+    unsigned char f;  /* full */
+    UT_icd icd;       /* initializer, copy and destructor functions */
+    char *d;          /* n slots of size icd->sz */
+} UT_ringbuffer;
+
+#define utringbuffer_init(a, _n, _icd) do {                               \
+  memset(a, 0, sizeof(UT_ringbuffer));                                    \
+  (a)->icd = *(_icd);                                                     \
+  (a)->n = (_n);                                                          \
+  if ((a)->n) { (a)->d = (char*)malloc((a)->n * (_icd)->sz); }            \
+} while(0)
+
+#define utringbuffer_clear(a) do {                                        \
+  if ((a)->icd.dtor) {                                                    \
+    if ((a)->f) {                                                         \
+      unsigned _ut_i;                                                     \
+      for (_ut_i = 0; _ut_i < (a)->n; ++_ut_i) {                          \
+        (a)->icd.dtor(utringbuffer_eltptr(a, _ut_i));                     \
+      }                                                                   \
+    } else {                                                              \
+      unsigned _ut_i;                                                     \
+      for (_ut_i = 0; _ut_i < (a)->i; ++_ut_i) {                          \
+        (a)->icd.dtor(utringbuffer_eltptr(a, _ut_i));                     \
+      }                                                                   \
+    }                                                                     \
+  }                                                                       \
+  (a)->i = 0;                                                             \
+  (a)->f = 0;                                                             \
+} while(0)
+
+#define utringbuffer_done(a) do {                                         \
+  utringbuffer_clear(a);                                                  \
+  free((a)->d); (a)->d = NULL;                                            \
+  (a)->n = 0;                                                             \
+} while(0)
+
+#define utringbuffer_new(a,n,_icd) do {                                   \
+  a = (UT_ringbuffer*)malloc(sizeof(UT_ringbuffer));                      \
+  utringbuffer_init(a, n, _icd);                                          \
+} while(0)
+
+#define utringbuffer_free(a) do {                                         \
+  utringbuffer_done(a);                                                   \
+  free(a);                                                                \
+} while(0)
+
+#define utringbuffer_push_back(a,p) do {                                                \
+  if ((a)->icd.dtor && (a)->f) { (a)->icd.dtor(_utringbuffer_internalptr(a,(a)->i)); }  \
+  if ((a)->icd.copy) { (a)->icd.copy( _utringbuffer_internalptr(a,(a)->i), p); }        \
+  else { memcpy(_utringbuffer_internalptr(a,(a)->i), p, (a)->icd.sz); };                \
+  if (++(a)->i == (a)->n) { (a)->i = 0; (a)->f = 1; }                                   \
+} while(0)
+
+#define utringbuffer_len(a) ((a)->f ? (a)->n : (a)->i)
+#define utringbuffer_empty(a) ((a)->i == 0 && !(a)->f)
+#define utringbuffer_full(a) ((a)->f != 0)
+
+#define _utringbuffer_real_idx(a,j) ((a)->f ? ((j) + (a)->i) % (a)->n : (j))
+#define _utringbuffer_internalptr(a,j) ((void*)((a)->d + ((a)->icd.sz * (j))))
+#define utringbuffer_eltptr(a,j) ((0 <= (j) && (j) < utringbuffer_len(a)) ? _utringbuffer_internalptr(a,_utringbuffer_real_idx(a,j)) : NULL)
+
+#define _utringbuffer_fake_idx(a,j) ((a)->f ? ((j) + (a)->n - (a)->i) % (a)->n : (j))
+#define _utringbuffer_internalidx(a,e) (((char*)(e) >= (a)->d) ? (((char*)(e) - (a)->d)/(a)->icd.sz) : -1)
+#define utringbuffer_eltidx(a,e) _utringbuffer_fake_idx(a, _utringbuffer_internalidx(a,e))
+
+#define utringbuffer_front(a) utringbuffer_eltptr(a,0)
+#define utringbuffer_next(a,e) ((e)==NULL ? utringbuffer_front(a) : utringbuffer_eltptr(a, utringbuffer_eltidx(a,e)+1))
+#define utringbuffer_prev(a,e) ((e)==NULL ? utringbuffer_back(a) : utringbuffer_eltptr(a, utringbuffer_eltidx(a,e)-1))
+#define utringbuffer_back(a) (utringbuffer_empty(a) ? NULL : utringbuffer_eltptr(a, utringbuffer_len(a) - 1))
+
+#endif /* UTRINGBUFFER_H */
diff --git a/thirdparty/ut/utstack.h b/thirdparty/ut/utstack.h
new file mode 100755
index 0000000..3b0c1a0
--- /dev/null
+++ b/thirdparty/ut/utstack.h
@@ -0,0 +1,88 @@
+/*
+Copyright (c) 2018-2018, Troy D. Hanson   http://troydhanson.github.com/uthash/
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/
+
+#ifndef UTSTACK_H
+#define UTSTACK_H
+
+#define UTSTACK_VERSION 2.1.0
+
+/*
+ * This file contains macros to manipulate a singly-linked list as a stack.
+ *
+ * To use utstack, your structure must have a "next" pointer.
+ *
+ * ----------------.EXAMPLE -------------------------
+ * struct item {
+ *      int id;
+ *      struct item *next;
+ * }
+ *
+ * struct item *stack = NULL:
+ *
+ * int main() {
+ *      int count;
+ *      struct item *tmp;
+ *      struct item *item = malloc(sizeof *item);
+ *      item->id = 42;
+ *      STACK_COUNT(stack, tmp, count); assert(count == 0);
+ *      STACK_PUSH(stack, item);
+ *      STACK_COUNT(stack, tmp, count); assert(count == 1);
+ *      STACK_POP(stack, item);
+ *      free(item);
+ *      STACK_COUNT(stack, tmp, count); assert(count == 0);
+ * }
+ * --------------------------------------------------
+ */
+
+#define STACK_TOP(head) (head)
+
+#define STACK_EMPTY(head) (!(head))
+
+#define STACK_PUSH(head,add)                                         \
+    STACK_PUSH2(head,add,next)
+
+#define STACK_PUSH2(head,add,next)                                   \
+do {                                                                 \
+  (add)->next = (head);                                              \
+  (head) = (add);                                                    \
+} while (0)
+
+#define STACK_POP(head,result)                                       \
+    STACK_POP2(head,result,next)
+
+#define STACK_POP2(head,result,next)                                 \
+do {                                                                 \
+  (result) = (head);                                                 \
+  (head) = (head)->next;                                             \
+} while (0)
+
+#define STACK_COUNT(head,el,counter)                                 \
+    STACK_COUNT2(head,el,counter,next)                               \
+
+#define STACK_COUNT2(head,el,counter,next)                           \
+do {                                                                 \
+  (counter) = 0;                                                     \
+  for ((el) = (head); el; (el) = (el)->next) { ++(counter); }        \
+} while (0)
+
+#endif /* UTSTACK_H */
diff --git a/thirdparty/ut/utstring.h b/thirdparty/ut/utstring.h
new file mode 100755
index 0000000..ca25c90
--- /dev/null
+++ b/thirdparty/ut/utstring.h
@@ -0,0 +1,398 @@
+/*
+Copyright (c) 2008-2018, Troy D. Hanson   http://troydhanson.github.com/uthash/
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/
+
+/* a dynamic string implementation using macros
+ */
+#ifndef UTSTRING_H
+#define UTSTRING_H
+
+#define UTSTRING_VERSION 2.1.0
+
+#include <stdlib.h>
+#include <string.h>
+#include <stdio.h>
+#include <stdarg.h>
+
+#ifdef __GNUC__
+#define UTSTRING_UNUSED __attribute__((__unused__))
+#else
+#define UTSTRING_UNUSED
+#endif
+
+#ifndef oom
+#define oom() exit(-1)
+#endif
+
+typedef struct {
+    char *d;  /* pointer to allocated buffer */
+    size_t n; /* allocated capacity */
+    size_t i; /* index of first unused byte */
+} UT_string;
+
+#define utstring_reserve(s,amt)                            \
+do {                                                       \
+  if (((s)->n - (s)->i) < (size_t)(amt)) {                 \
+    char *utstring_tmp = (char*)realloc(                   \
+      (s)->d, (s)->n + (amt));                             \
+    if (utstring_tmp == NULL) oom();                       \
+    (s)->d = utstring_tmp;                                 \
+    (s)->n += (amt);                                       \
+  }                                                        \
+} while(0)
+
+#define utstring_init(s)                                   \
+do {                                                       \
+  (s)->n = 0; (s)->i = 0; (s)->d = NULL;                   \
+  utstring_reserve(s,100);                                 \
+  (s)->d[0] = '\0';                                        \
+} while(0)
+
+#define utstring_done(s)                                   \
+do {                                                       \
+  if ((s)->d != NULL) free((s)->d);                        \
+  (s)->n = 0;                                              \
+} while(0)
+
+#define utstring_free(s)                                   \
+do {                                                       \
+  utstring_done(s);                                        \
+  free(s);                                                 \
+} while(0)
+
+#define utstring_new(s)                                    \
+do {                                                       \
+   (s) = (UT_string*)malloc(sizeof(UT_string));            \
+   if (!(s)) oom();                                        \
+   utstring_init(s);                                       \
+} while(0)
+
+#define utstring_renew(s)                                  \
+do {                                                       \
+   if (s) {                                                \
+     utstring_clear(s);                                    \
+   } else {                                                \
+     utstring_new(s);                                      \
+   }                                                       \
+} while(0)
+
+#define utstring_clear(s)                                  \
+do {                                                       \
+  (s)->i = 0;                                              \
+  (s)->d[0] = '\0';                                        \
+} while(0)
+
+#define utstring_bincpy(s,b,l)                             \
+do {                                                       \
+  utstring_reserve((s),(l)+1);                             \
+  if (l) memcpy(&(s)->d[(s)->i], b, l);                    \
+  (s)->i += (l);                                           \
+  (s)->d[(s)->i]='\0';                                     \
+} while(0)
+
+#define utstring_concat(dst,src)                                 \
+do {                                                             \
+  utstring_reserve((dst),((src)->i)+1);                          \
+  if ((src)->i) memcpy(&(dst)->d[(dst)->i], (src)->d, (src)->i); \
+  (dst)->i += (src)->i;                                          \
+  (dst)->d[(dst)->i]='\0';                                       \
+} while(0)
+
+#define utstring_len(s) ((s)->i)
+
+#define utstring_body(s) ((s)->d)
+
+UTSTRING_UNUSED static void utstring_printf_va(UT_string *s, const char *fmt, va_list ap) {
+   int n;
+   va_list cp;
+   for (;;) {
+#ifdef _WIN32
+      cp = ap;
+#else
+      va_copy(cp, ap);
+#endif
+      n = vsnprintf (&s->d[s->i], s->n-s->i, fmt, cp);
+      va_end(cp);
+
+      if ((n > -1) && ((size_t) n < (s->n-s->i))) {
+        s->i += n;
+        return;
+      }
+
+      /* Else try again with more space. */
+      if (n > -1) utstring_reserve(s,n+1); /* exact */
+      else utstring_reserve(s,(s->n)*2);   /* 2x */
+   }
+}
+#ifdef __GNUC__
+/* support printf format checking (2=the format string, 3=start of varargs) */
+static void utstring_printf(UT_string *s, const char *fmt, ...)
+  __attribute__ (( format( printf, 2, 3) ));
+#endif
+UTSTRING_UNUSED static void utstring_printf(UT_string *s, const char *fmt, ...) {
+   va_list ap;
+   va_start(ap,fmt);
+   utstring_printf_va(s,fmt,ap);
+   va_end(ap);
+}
+
+/*******************************************************************************
+ * begin substring search functions                                            *
+ ******************************************************************************/
+/* Build KMP table from left to right. */
+UTSTRING_UNUSED static void _utstring_BuildTable(
+    const char *P_Needle,
+    size_t P_NeedleLen,
+    long *P_KMP_Table)
+{
+    long i, j;
+
+    i = 0;
+    j = i - 1;
+    P_KMP_Table[i] = j;
+    while (i < (long) P_NeedleLen)
+    {
+        while ( (j > -1) && (P_Needle[i] != P_Needle[j]) )
+        {
+           j = P_KMP_Table[j];
+        }
+        i++;
+        j++;
+        if (i < (long) P_NeedleLen)
+        {
+            if (P_Needle[i] == P_Needle[j])
+            {
+                P_KMP_Table[i] = P_KMP_Table[j];
+            }
+            else
+            {
+                P_KMP_Table[i] = j;
+            }
+        }
+        else
+        {
+            P_KMP_Table[i] = j;
+        }
+    }
+
+    return;
+}
+
+
+/* Build KMP table from right to left. */
+UTSTRING_UNUSED static void _utstring_BuildTableR(
+    const char *P_Needle,
+    size_t P_NeedleLen,
+    long *P_KMP_Table)
+{
+    long i, j;
+
+    i = P_NeedleLen - 1;
+    j = i + 1;
+    P_KMP_Table[i + 1] = j;
+    while (i >= 0)
+    {
+        while ( (j < (long) P_NeedleLen) && (P_Needle[i] != P_Needle[j]) )
+        {
+           j = P_KMP_Table[j + 1];
+        }
+        i--;
+        j--;
+        if (i >= 0)
+        {
+            if (P_Needle[i] == P_Needle[j])
+            {
+                P_KMP_Table[i + 1] = P_KMP_Table[j + 1];
+            }
+            else
+            {
+                P_KMP_Table[i + 1] = j;
+            }
+        }
+        else
+        {
+            P_KMP_Table[i + 1] = j;
+        }
+    }
+
+    return;
+}
+
+
+/* Search data from left to right. ( Multiple search mode. ) */
+UTSTRING_UNUSED static long _utstring_find(
+    const char *P_Haystack,
+    size_t P_HaystackLen,
+    const char *P_Needle,
+    size_t P_NeedleLen,
+    long *P_KMP_Table)
+{
+    long i, j;
+    long V_FindPosition = -1;
+
+    /* Search from left to right. */
+    i = j = 0;
+    while ( (j < (int)P_HaystackLen) && (((P_HaystackLen - j) + i) >= P_NeedleLen) )
+    {
+        while ( (i > -1) && (P_Needle[i] != P_Haystack[j]) )
+        {
+            i = P_KMP_Table[i];
+        }
+        i++;
+        j++;
+        if (i >= (int)P_NeedleLen)
+        {
+            /* Found. */
+            V_FindPosition = j - i;
+            break;
+        }
+    }
+
+    return V_FindPosition;
+}
+
+
+/* Search data from right to left. ( Multiple search mode. ) */
+UTSTRING_UNUSED static long _utstring_findR(
+    const char *P_Haystack,
+    size_t P_HaystackLen,
+    const char *P_Needle,
+    size_t P_NeedleLen,
+    long *P_KMP_Table)
+{
+    long i, j;
+    long V_FindPosition = -1;
+
+    /* Search from right to left. */
+    j = (P_HaystackLen - 1);
+    i = (P_NeedleLen - 1);
+    while ( (j >= 0) && (j >= i) )
+    {
+        while ( (i < (int)P_NeedleLen) && (P_Needle[i] != P_Haystack[j]) )
+        {
+            i = P_KMP_Table[i + 1];
+        }
+        i--;
+        j--;
+        if (i < 0)
+        {
+            /* Found. */
+            V_FindPosition = j + 1;
+            break;
+        }
+    }
+
+    return V_FindPosition;
+}
+
+
+/* Search data from left to right. ( One time search mode. ) */
+UTSTRING_UNUSED static long utstring_find(
+    UT_string *s,
+    long P_StartPosition,   /* Start from 0. -1 means last position. */
+    const char *P_Needle,
+    size_t P_NeedleLen)
+{
+    long V_StartPosition;
+    long V_HaystackLen;
+    long *V_KMP_Table;
+    long V_FindPosition = -1;
+
+    if (P_StartPosition < 0)
+    {
+        V_StartPosition = s->i + P_StartPosition;
+    }
+    else
+    {
+        V_StartPosition = P_StartPosition;
+    }
+    V_HaystackLen = s->i - V_StartPosition;
+    if ( (V_HaystackLen >= (long) P_NeedleLen) && (P_NeedleLen > 0) )
+    {
+        V_KMP_Table = (long *)malloc(sizeof(long) * (P_NeedleLen + 1));
+        if (V_KMP_Table != NULL)
+        {
+            _utstring_BuildTable(P_Needle, P_NeedleLen, V_KMP_Table);
+
+            V_FindPosition = _utstring_find(s->d + V_StartPosition,
+                                            V_HaystackLen,
+                                            P_Needle,
+                                            P_NeedleLen,
+                                            V_KMP_Table);
+            if (V_FindPosition >= 0)
+            {
+                V_FindPosition += V_StartPosition;
+            }
+
+            free(V_KMP_Table);
+        }
+    }
+
+    return V_FindPosition;
+}
+
+
+/* Search data from right to left. ( One time search mode. ) */
+UTSTRING_UNUSED static long utstring_findR(
+    UT_string *s,
+    long P_StartPosition,   /* Start from 0. -1 means last position. */
+    const char *P_Needle,
+    size_t P_NeedleLen)
+{
+    long V_StartPosition;
+    long V_HaystackLen;
+    long *V_KMP_Table;
+    long V_FindPosition = -1;
+
+    if (P_StartPosition < 0)
+    {
+        V_StartPosition = s->i + P_StartPosition;
+    }
+    else
+    {
+        V_StartPosition = P_StartPosition;
+    }
+    V_HaystackLen = V_StartPosition + 1;
+    if ( (V_HaystackLen >= (long) P_NeedleLen) && (P_NeedleLen > 0) )
+    {
+        V_KMP_Table = (long *)malloc(sizeof(long) * (P_NeedleLen + 1));
+        if (V_KMP_Table != NULL)
+        {
+            _utstring_BuildTableR(P_Needle, P_NeedleLen, V_KMP_Table);
+
+            V_FindPosition = _utstring_findR(s->d,
+                                             V_HaystackLen,
+                                             P_Needle,
+                                             P_NeedleLen,
+                                             V_KMP_Table);
+
+            free(V_KMP_Table);
+        }
+    }
+
+    return V_FindPosition;
+}
+/*******************************************************************************
+ * end substring search functions                                              *
+ ******************************************************************************/
+
+#endif /* UTSTRING_H */