Browse Source

Multithreading in network controller. Threads are only started if controller is used.

Adam Ierymenko 8 years ago
parent
commit
e26bee45fb
3 changed files with 832 additions and 692 deletions
  1. 741 692
      controller/EmbeddedNetworkController.cpp
  2. 27 0
      controller/EmbeddedNetworkController.hpp
  3. 64 0
      osdep/BlockingQueue.hpp

+ 741 - 692
controller/EmbeddedNetworkController.cpp

@@ -459,6 +459,7 @@ static bool _parseRule(json &r,ZT_VirtualNetworkRule &rule)
 }
 
 EmbeddedNetworkController::EmbeddedNetworkController(Node *node,const char *dbPath) :
+	_threadsStarted(false),
 	_db(dbPath),
 	_node(node)
 {
@@ -468,6 +469,13 @@ EmbeddedNetworkController::EmbeddedNetworkController(Node *node,const char *dbPa
 
 EmbeddedNetworkController::~EmbeddedNetworkController()
 {
+	Mutex::Lock _l(_threads_m);
+	if (_threadsStarted) {
+		for(int i=0;i<(ZT_EMBEDDEDNETWORKCONTROLLER_BACKGROUND_THREAD_COUNT*2);++i)
+			_queue.post((_RQEntry *)0);
+		for(int i=0;i<ZT_EMBEDDEDNETWORKCONTROLLER_BACKGROUND_THREAD_COUNT;++i)
+			Thread::join(_threads[i]);
+	}
 }
 
 void EmbeddedNetworkController::init(const Identity &signingId,Sender *sender)
@@ -486,658 +494,200 @@ void EmbeddedNetworkController::request(
 	if (((!_signingId)||(!_signingId.hasPrivate()))||(_signingId.address().toInt() != (nwid >> 24))||(!_sender))
 		return;
 
-	const uint64_t now = OSUtils::now();
-
-	if (requestPacketId) {
-		Mutex::Lock _l(_lastRequestTime_m);
-		uint64_t &lrt = _lastRequestTime[std::pair<uint64_t,uint64_t>(identity.address().toInt(),nwid)];
-		if ((now - lrt) <= ZT_NETCONF_MIN_REQUEST_PERIOD)
-			return;
-		lrt = now;
-	}
-
-	char nwids[24];
-	Utils::snprintf(nwids,sizeof(nwids),"%.16llx",nwid);
-	json network;
-	json member;
 	{
-		Mutex::Lock _l(_db_m);
-		network = _db.get("network",nwids,0);
-		member = _db.get("network",nwids,"member",identity.address().toString(),0);
+		Mutex::Lock _l(_threads_m);
+		if (!_threadsStarted) {
+			for(int i=0;i<ZT_EMBEDDEDNETWORKCONTROLLER_BACKGROUND_THREAD_COUNT;++i)
+				_threads[i] = Thread::start(this);
+		}
+		_threadsStarted = true;
 	}
 
-	if (!network.size()) {
-		_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_OBJECT_NOT_FOUND);
-		return;
-	}
+	_RQEntry *qe = new _RQEntry;
+	qe->nwid = nwid;
+	qe->requestPacketId = requestPacketId;
+	qe->fromAddr = fromAddr;
+	qe->identity = identity;
+	qe->metaData = metaData;
+	_queue.post(qe);
+}
 
-	json origMember(member); // for detecting modification later
-	_initMember(member);
+unsigned int EmbeddedNetworkController::handleControlPlaneHttpGET(
+	const std::vector<std::string> &path,
+	const std::map<std::string,std::string> &urlArgs,
+	const std::map<std::string,std::string> &headers,
+	const std::string &body,
+	std::string &responseBody,
+	std::string &responseContentType)
+{
+	if ((path.size() > 0)&&(path[0] == "network")) {
 
-	{
-		std::string haveIdStr(_jS(member["identity"],""));
-		if (haveIdStr.length() > 0) {
-			// If we already know this member's identity perform a full compare. This prevents
-			// a "collision" from being able to auth onto our network in place of an already
-			// known member.
-			try {
-				if (Identity(haveIdStr.c_str()) != identity) {
-					_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_ACCESS_DENIED);
-					return;
-				}
-			} catch ( ... ) {
-				_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_ACCESS_DENIED);
-				return;
+		if ((path.size() >= 2)&&(path[1].length() == 16)) {
+			const uint64_t nwid = Utils::hexStrToU64(path[1].c_str());
+			char nwids[24];
+			Utils::snprintf(nwids,sizeof(nwids),"%.16llx",(unsigned long long)nwid);
+
+			json network;
+			{
+				Mutex::Lock _l(_db_m);
+				network = _db.get("network",nwids,0);
 			}
-		} else {
-			// If we do not yet know this member's identity, learn it.
-			member["identity"] = identity.toString(false);
-		}
-	}
+			if (!network.size())
+				return 404;
 
-	// These are always the same, but make sure they are set
-	member["id"] = identity.address().toString();
-	member["address"] = member["id"];
-	member["nwid"] = nwids;
+			if (path.size() >= 3) {
 
-	// Determine whether and how member is authorized
-	const char *authorizedBy = (const char *)0;
-	if (_jB(member["authorized"],false)) {
-		authorizedBy = "memberIsAuthorized";
-	} else if (!_jB(network["private"],true)) {
-		authorizedBy = "networkIsPublic";
-		if (!member.count("authorized")) {
-			member["authorized"] = true;
-			json ah;
-			ah["a"] = true;
-			ah["by"] = authorizedBy;
-			ah["ts"] = now;
-			ah["ct"] = json();
-			ah["c"] = json();
-			member["authHistory"].push_back(ah);
-			member["lastModified"] = now;
-			json &revj = member["revision"];
-			member["revision"] = (revj.is_number() ? ((uint64_t)revj + 1ULL) : 1ULL);
-		}
-	} else {
-		char presentedAuth[512];
-		if (metaData.get(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_AUTH,presentedAuth,sizeof(presentedAuth)) > 0) {
-			presentedAuth[511] = (char)0; // sanity check
+				if (path[2] == "member") {
 
-			// Check for bearer token presented by member
-			if ((strlen(presentedAuth) > 6)&&(!strncmp(presentedAuth,"token:",6))) {
-				const char *const presentedToken = presentedAuth + 6;
+					if (path.size() >= 4) {
+						const uint64_t address = Utils::hexStrToU64(path[3].c_str());
 
-				json &authTokens = network["authTokens"];
-				if (authTokens.is_array()) {
-					for(unsigned long i=0;i<authTokens.size();++i) {
-						json &token = authTokens[i];
-						if (token.is_object()) {
-							const uint64_t expires = _jI(token["expires"],0ULL);
-							const uint64_t maxUses = _jI(token["maxUsesPerMember"],0ULL);
-							std::string tstr = _jS(token["token"],"");
+						json member;
+						{
+							Mutex::Lock _l(_db_m);
+							member = _db.get("network",nwids,"member",Address(address).toString(),0);
+						}
+						if (!member.size())
+							return 404;
 
-							if (((expires == 0ULL)||(expires > now))&&(tstr == presentedToken)) {
-								bool usable = (maxUses == 0);
-								if (!usable) {
-									uint64_t useCount = 0;
-									json &ahist = member["authHistory"];
-									if (ahist.is_array()) {
-										for(unsigned long j=0;j<ahist.size();++j) {
-											json &ah = ahist[j];
-											if ((_jS(ah["ct"],"") == "token")&&(_jS(ah["c"],"") == tstr)&&(_jB(ah["a"],false)))
-												++useCount;
-										}
-									}
-									usable = (useCount < maxUses);
-								}
-								if (usable) {
-									authorizedBy = "token";
-									member["authorized"] = true;
-									json ah;
-									ah["a"] = true;
-									ah["by"] = authorizedBy;
-									ah["ts"] = now;
-									ah["ct"] = "token";
-									ah["c"] = tstr;
-									member["authHistory"].push_back(ah);
-									member["lastModified"] = now;
-									json &revj = member["revision"];
-									member["revision"] = (revj.is_number() ? ((uint64_t)revj + 1ULL) : 1ULL);
-								}
+						_addMemberNonPersistedFields(member,OSUtils::now());
+						responseBody = member.dump(2);
+						responseContentType = "application/json";
+
+						return 200;
+					} else {
+
+						Mutex::Lock _l(_db_m);
+
+						responseBody = "{";
+						std::string pfx(std::string("network/") + nwids + "member/");
+						_db.filter(pfx,120000,[&responseBody](const std::string &n,const json &member) {
+							if (member.size() > 0) {
+								responseBody.append((responseBody.length() == 1) ? "\"" : ",\"");
+								responseBody.append(_jS(member["id"],""));
+								responseBody.append("\":");
+								responseBody.append(_jS(member["revision"],"0"));
 							}
-						}
+							return true; // never delete
+						});
+						responseBody.push_back('}');
+						responseContentType = "application/json";
+
+						return 200;
 					}
-				}
-			}
-		}
-	}
 
-	// Log this request
-	if (requestPacketId) { // only log if this is a request, not for generated pushes
-		json rlEntry = json::object();
-		rlEntry["ts"] = now;
-		rlEntry["authorized"] = (authorizedBy) ? true : false;
-		rlEntry["authorizedBy"] = (authorizedBy) ? authorizedBy : "";
-		rlEntry["clientMajorVersion"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_NODE_MAJOR_VERSION,0);
-		rlEntry["clientMinorVersion"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_NODE_MINOR_VERSION,0);
-		rlEntry["clientRevision"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_NODE_REVISION,0);
-		rlEntry["clientProtocolVersion"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_PROTOCOL_VERSION,0);
-		if (fromAddr)
-			rlEntry["fromAddr"] = fromAddr.toString();
+				} else if ((path[2] == "test")&&(path.size() >= 4)) {
 
-		json recentLog = json::array();
-		recentLog.push_back(rlEntry);
-		json &oldLog = member["recentLog"];
-		if (oldLog.is_array()) {
-			for(unsigned long i=0;i<oldLog.size();++i) {
-				recentLog.push_back(oldLog[i]);
-				if (recentLog.size() >= ZT_NETCONF_DB_MEMBER_HISTORY_LENGTH)
-					break;
-			}
-		}
-		member["recentLog"] = recentLog;
+					Mutex::Lock _l(_circuitTests_m);
+					std::map< uint64_t,_CircuitTestEntry >::iterator cte(_circuitTests.find(Utils::hexStrToU64(path[3].c_str())));
+					if ((cte != _circuitTests.end())&&(cte->second.test)) {
 
-		// Also only do this on real requests
-		member["lastRequestMetaData"] = metaData.data();
-	}
+						responseBody = "[";
+						responseBody.append(cte->second.jsonResults);
+						responseBody.push_back(']');
+						responseContentType = "application/json";
 
-	// If they are not authorized, STOP!
-	if (!authorizedBy) {
-		if (origMember != member) {
-			member["lastModified"] = now;
-			Mutex::Lock _l(_db_m);
-			_db.put("network",nwids,"member",identity.address().toString(),member);
-		}
-		_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_ACCESS_DENIED);
-		return;
-	}
+						return 200;
 
-	// -------------------------------------------------------------------------
-	// If we made it this far, they are authorized.
-	// -------------------------------------------------------------------------
+					} // else 404
 
-	NetworkConfig nc;
-	_NetworkMemberInfo nmi;
-	_getNetworkMemberInfo(now,nwid,nmi);
+				} // else 404
 
-	// Compute credential TTL. This is the "moving window" for COM agreement and
-	// the global TTL for Capability and Tag objects. (The same value is used
-	// for both.) This is computed by reference to the last time we deauthorized
-	// a member, since within the time period since this event any temporal
-	// differences are not particularly relevant.
-	uint64_t credentialtmd = ZT_NETWORKCONFIG_DEFAULT_CREDENTIAL_TIME_MIN_MAX_DELTA;
-	if (now > nmi.mostRecentDeauthTime)
-		credentialtmd += (now - nmi.mostRecentDeauthTime);
-	if (credentialtmd > ZT_NETWORKCONFIG_DEFAULT_CREDENTIAL_TIME_MAX_MAX_DELTA)
-		credentialtmd = ZT_NETWORKCONFIG_DEFAULT_CREDENTIAL_TIME_MAX_MAX_DELTA;
+			} else {
 
-	nc.networkId = nwid;
-	nc.type = _jB(network["private"],true) ? ZT_NETWORK_TYPE_PRIVATE : ZT_NETWORK_TYPE_PUBLIC;
-	nc.timestamp = now;
-	nc.credentialTimeMaxDelta = credentialtmd;
-	nc.revision = _jI(network["revision"],0ULL);
-	nc.issuedTo = identity.address();
-	if (_jB(network["enableBroadcast"],true)) nc.flags |= ZT_NETWORKCONFIG_FLAG_ENABLE_BROADCAST;
-	if (_jB(network["allowPassiveBridging"],false)) nc.flags |= ZT_NETWORKCONFIG_FLAG_ALLOW_PASSIVE_BRIDGING;
-	Utils::scopy(nc.name,sizeof(nc.name),_jS(network["name"],"").c_str());
-	nc.multicastLimit = (unsigned int)_jI(network["multicastLimit"],32ULL);
+				const uint64_t now = OSUtils::now();
+				_NetworkMemberInfo nmi;
+				_getNetworkMemberInfo(now,nwid,nmi);
+				_addNetworkNonPersistedFields(network,now,nmi);
+				responseBody = network.dump(2);
+				responseContentType = "application/json";
+				return 200;
 
-	for(std::set<Address>::const_iterator ab(nmi.activeBridges.begin());ab!=nmi.activeBridges.end();++ab) {
-		nc.addSpecialist(*ab,ZT_NETWORKCONFIG_SPECIALIST_TYPE_ACTIVE_BRIDGE);
-	}
+			}
+		} else if (path.size() == 1) {
 
-	json &v4AssignMode = network["v4AssignMode"];
-	json &v6AssignMode = network["v6AssignMode"];
-	json &ipAssignmentPools = network["ipAssignmentPools"];
-	json &routes = network["routes"];
-	json &rules = network["rules"];
-	json &capabilities = network["capabilities"];
-	json &memberCapabilities = member["capabilities"];
-	json &memberTags = member["tags"];
-
-	if (metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_RULES_ENGINE_REV,0) <= 0) {
-		// Old versions with no rules engine support get an allow everything rule.
-		// Since rules are enforced bidirectionally, newer versions *will* still
-		// enforce rules on the inbound side.
-		nc.ruleCount = 1;
-		nc.rules[0].t = ZT_NETWORK_RULE_ACTION_ACCEPT;
-	} else {
-		if (rules.is_array()) {
-			for(unsigned long i=0;i<rules.size();++i) {
-				if (nc.ruleCount >= ZT_MAX_NETWORK_RULES)
-					break;
-				if (_parseRule(rules[i],nc.rules[nc.ruleCount]))
-					++nc.ruleCount;
-			}
-		}
-
-		if ((memberCapabilities.is_array())&&(memberCapabilities.size() > 0)&&(capabilities.is_array())) {
-			std::map< uint64_t,json * > capsById;
-			for(unsigned long i=0;i<capabilities.size();++i) {
-				json &cap = capabilities[i];
-				if (cap.is_object())
-					capsById[_jI(cap["id"],0ULL) & 0xffffffffULL] = &cap;
+			std::set<std::string> networkIds;
+			{
+				Mutex::Lock _l(_db_m);
+				_db.filter("network/",120000,[&networkIds](const std::string &n,const json &obj) {
+					if (n.length() == (16 + 8))
+						networkIds.insert(n.substr(8));
+					return true; // do not delete
+				});
 			}
 
-			for(unsigned long i=0;i<memberCapabilities.size();++i) {
-				const uint64_t capId = _jI(memberCapabilities[i],0ULL) & 0xffffffffULL;
-				json *cap = capsById[capId];
-				if ((cap->is_object())&&(cap->size() > 0)) {
-					ZT_VirtualNetworkRule capr[ZT_MAX_CAPABILITY_RULES];
-					unsigned int caprc = 0;
-					json &caprj = (*cap)["rules"];
-					if ((caprj.is_array())&&(caprj.size() > 0)) {
-						for(unsigned long j=0;j<caprj.size();++j) {
-							if (caprc >= ZT_MAX_CAPABILITY_RULES)
-								break;
-							if (_parseRule(caprj[j],capr[caprc]))
-								++caprc;
-						}
-					}
-					nc.capabilities[nc.capabilityCount] = Capability((uint32_t)capId,nwid,now,1,capr,caprc);
-					if (nc.capabilities[nc.capabilityCount].sign(_signingId,identity.address()))
-						++nc.capabilityCount;
-					if (nc.capabilityCount >= ZT_MAX_NETWORK_CAPABILITIES)
-						break;
-				}
+			responseBody.push_back('[');
+			for(std::set<std::string>::iterator i(networkIds.begin());i!=networkIds.end();++i) {
+				responseBody.append((responseBody.length() == 1) ? "\"" : ",\"");
+				responseBody.append(*i);
+				responseBody.append("\"");
 			}
-		}
+			responseBody.push_back(']');
+			responseContentType = "application/json";
+			return 200;
 
-		if (memberTags.is_array()) {
-			std::map< uint32_t,uint32_t > tagsById;
-			for(unsigned long i=0;i<memberTags.size();++i) {
-				json &t = memberTags[i];
-				if ((t.is_array())&&(t.size() == 2))
-					tagsById[(uint32_t)(_jI(t[0],0ULL) & 0xffffffffULL)] = (uint32_t)(_jI(t[1],0ULL) & 0xffffffffULL);
-			}
-			for(std::map< uint32_t,uint32_t >::const_iterator t(tagsById.begin());t!=tagsById.end();++t) {
-				if (nc.tagCount >= ZT_MAX_NETWORK_TAGS)
-					break;
-				nc.tags[nc.tagCount] = Tag(nwid,now,identity.address(),t->first,t->second);
-				if (nc.tags[nc.tagCount].sign(_signingId))
-					++nc.tagCount;
-			}
-		}
-	}
+		} // else 404
 
-	if (routes.is_array()) {
-		for(unsigned long i=0;i<routes.size();++i) {
-			if (nc.routeCount >= ZT_MAX_NETWORK_ROUTES)
-				break;
-			json &route = routes[i];
-			json &target = route["target"];
-			json &via = route["via"];
-			if (target.is_string()) {
-				const InetAddress t(target.get<std::string>());
-				InetAddress v;
-				if (via.is_string()) v.fromString(via.get<std::string>());
-				if ((t.ss_family == AF_INET)||(t.ss_family == AF_INET6)) {
-					ZT_VirtualNetworkRoute *r = &(nc.routes[nc.routeCount]);
-					*(reinterpret_cast<InetAddress *>(&(r->target))) = t;
-					if (v.ss_family == t.ss_family)
-						*(reinterpret_cast<InetAddress *>(&(r->via))) = v;
-					++nc.routeCount;
-				}
-			}
-		}
-	}
+	} else {
 
-	const bool noAutoAssignIps = _jB(member["noAutoAssignIps"],false);
+		char tmp[4096];
+		Utils::snprintf(tmp,sizeof(tmp),"{\n\t\"controller\": true,\n\t\"apiVersion\": %d,\n\t\"clock\": %llu\n}\n",ZT_NETCONF_CONTROLLER_API_VERSION,(unsigned long long)OSUtils::now());
+		responseBody = tmp;
+		responseContentType = "application/json";
+		return 200;
 
-	if ((v6AssignMode.is_object())&&(!noAutoAssignIps)) {
-		if ((_jB(v6AssignMode["rfc4193"],false))&&(nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES)) {
-			nc.staticIps[nc.staticIpCount++] = InetAddress::makeIpv6rfc4193(nwid,identity.address().toInt());
-			nc.flags |= ZT_NETWORKCONFIG_FLAG_ENABLE_IPV6_NDP_EMULATION;
-		}
-		if ((_jB(v6AssignMode["6plane"],false))&&(nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES)) {
-			nc.staticIps[nc.staticIpCount++] = InetAddress::makeIpv66plane(nwid,identity.address().toInt());
-			nc.flags |= ZT_NETWORKCONFIG_FLAG_ENABLE_IPV6_NDP_EMULATION;
-		}
 	}
 
-	bool haveManagedIpv4AutoAssignment = false;
-	bool haveManagedIpv6AutoAssignment = false; // "special" NDP-emulated address types do not count
-	json ipAssignments = member["ipAssignments"]; // we want to make a copy
-	if (ipAssignments.is_array()) {
-		for(unsigned long i=0;i<ipAssignments.size();++i) {
-			if (!ipAssignments[i].is_string())
-				continue;
-			std::string ips = ipAssignments[i];
-			InetAddress ip(ips);
+	return 404;
+}
 
-			// IP assignments are only pushed if there is a corresponding local route. We also now get the netmask bits from
-			// this route, ignoring the netmask bits field of the assigned IP itself. Using that was worthless and a source
-			// of user error / poor UX.
-			int routedNetmaskBits = 0;
-			for(unsigned int rk=0;rk<nc.routeCount;++rk) {
-				if ( (!nc.routes[rk].via.ss_family) && (reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->containsAddress(ip)) )
-					routedNetmaskBits = reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->netmaskBits();
-			}
+unsigned int EmbeddedNetworkController::handleControlPlaneHttpPOST(
+	const std::vector<std::string> &path,
+	const std::map<std::string,std::string> &urlArgs,
+	const std::map<std::string,std::string> &headers,
+	const std::string &body,
+	std::string &responseBody,
+	std::string &responseContentType)
+{
+	if (path.empty())
+		return 404;
 
-			if (routedNetmaskBits > 0) {
-				if (nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES) {
-					ip.setPort(routedNetmaskBits);
-					nc.staticIps[nc.staticIpCount++] = ip;
-				}
-				if (ip.ss_family == AF_INET)
-					haveManagedIpv4AutoAssignment = true;
-				else if (ip.ss_family == AF_INET6)
-					haveManagedIpv6AutoAssignment = true;
-			}
+	json b;
+	try {
+		b = json::parse(body);
+		if (!b.is_object()) {
+			responseBody = "{ \"message\": \"body is not a JSON object\" }";
+			responseContentType = "application/json";
+			return 400;
 		}
-	} else {
-		ipAssignments = json::array();
+	} catch ( ... ) {
+		responseBody = "{ \"message\": \"body JSON is invalid\" }";
+		responseContentType = "application/json";
+		return 400;
 	}
+	const uint64_t now = OSUtils::now();
 
-	if ( (ipAssignmentPools.is_array()) && ((v6AssignMode.is_object())&&(_jB(v6AssignMode["zt"],false))) && (!haveManagedIpv6AutoAssignment) && (!noAutoAssignIps) ) {
-		for(unsigned long p=0;((p<ipAssignmentPools.size())&&(!haveManagedIpv6AutoAssignment));++p) {
-			json &pool = ipAssignmentPools[p];
-			if (pool.is_object()) {
-				InetAddress ipRangeStart(_jS(pool["ipRangeStart"],""));
-				InetAddress ipRangeEnd(_jS(pool["ipRangeEnd"],""));
-				if ( (ipRangeStart.ss_family == AF_INET6) && (ipRangeEnd.ss_family == AF_INET6) ) {
-					uint64_t s[2],e[2],x[2],xx[2];
-					memcpy(s,ipRangeStart.rawIpData(),16);
-					memcpy(e,ipRangeEnd.rawIpData(),16);
-					s[0] = Utils::ntoh(s[0]);
-					s[1] = Utils::ntoh(s[1]);
-					e[0] = Utils::ntoh(e[0]);
-					e[1] = Utils::ntoh(e[1]);
-					x[0] = s[0];
-					x[1] = s[1];
+	if (path[0] == "network") {
 
-					for(unsigned int trialCount=0;trialCount<1000;++trialCount) {
-						if ((trialCount == 0)&&(e[1] > s[1])&&((e[1] - s[1]) >= 0xffffffffffULL)) {
-							// First see if we can just cram a ZeroTier ID into the higher 64 bits. If so do that.
-							xx[0] = Utils::hton(x[0]);
-							xx[1] = Utils::hton(x[1] + identity.address().toInt());
-						} else {
-							// Otherwise pick random addresses -- this technically doesn't explore the whole range if the lower 64 bit range is >= 1 but that won't matter since that would be huge anyway
-							Utils::getSecureRandom((void *)xx,16);
-							if ((e[0] > s[0]))
-								xx[0] %= (e[0] - s[0]);
-							else xx[0] = 0;
-							if ((e[1] > s[1]))
-								xx[1] %= (e[1] - s[1]);
-							else xx[1] = 0;
-							xx[0] = Utils::hton(x[0] + xx[0]);
-							xx[1] = Utils::hton(x[1] + xx[1]);
-						}
+		if ((path.size() >= 2)&&(path[1].length() == 16)) {
+			uint64_t nwid = Utils::hexStrToU64(path[1].c_str());
+			char nwids[24];
+			Utils::snprintf(nwids,sizeof(nwids),"%.16llx",(unsigned long long)nwid);
 
-						InetAddress ip6((const void *)xx,16,0);
+			if (path.size() >= 3) {
 
-						// Check if this IP is within a local-to-Ethernet routed network
-						int routedNetmaskBits = 0;
-						for(unsigned int rk=0;rk<nc.routeCount;++rk) {
-							if ( (!nc.routes[rk].via.ss_family) && (nc.routes[rk].target.ss_family == AF_INET6) && (reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->containsAddress(ip6)) )
-								routedNetmaskBits = reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->netmaskBits();
-						}
+				if ((path.size() == 4)&&(path[2] == "member")&&(path[3].length() == 10)) {
+					uint64_t address = Utils::hexStrToU64(path[3].c_str());
+					char addrs[24];
+					Utils::snprintf(addrs,sizeof(addrs),"%.10llx",(unsigned long long)address);
 
-						// If it's routed, then try to claim and assign it and if successful end loop
-						if ((routedNetmaskBits > 0)&&(!nmi.allocatedIps.count(ip6))) {
-							ipAssignments.push_back(ip6.toIpString());
-							member["ipAssignments"] = ipAssignments;
-							ip6.setPort((unsigned int)routedNetmaskBits);
-							if (nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES)
-								nc.staticIps[nc.staticIpCount++] = ip6;
-							haveManagedIpv6AutoAssignment = true;
-							break;
-						}
+					json member;
+					{
+						Mutex::Lock _l(_db_m);
+						member = _db.get("network",nwids,"member",Address(address).toString(),0);
 					}
-				}
-			}
-		}
-	}
-
-	if ( (ipAssignmentPools.is_array()) && ((v4AssignMode.is_object())&&(_jB(v4AssignMode["zt"],false))) && (!haveManagedIpv4AutoAssignment) && (!noAutoAssignIps) ) {
-		for(unsigned long p=0;((p<ipAssignmentPools.size())&&(!haveManagedIpv4AutoAssignment));++p) {
-			json &pool = ipAssignmentPools[p];
-			if (pool.is_object()) {
-				InetAddress ipRangeStartIA(_jS(pool["ipRangeStart"],""));
-				InetAddress ipRangeEndIA(_jS(pool["ipRangeEnd"],""));
-				if ( (ipRangeStartIA.ss_family == AF_INET) && (ipRangeEndIA.ss_family == AF_INET) ) {
-					uint32_t ipRangeStart = Utils::ntoh((uint32_t)(reinterpret_cast<struct sockaddr_in *>(&ipRangeStartIA)->sin_addr.s_addr));
-					uint32_t ipRangeEnd = Utils::ntoh((uint32_t)(reinterpret_cast<struct sockaddr_in *>(&ipRangeEndIA)->sin_addr.s_addr));
-					if ((ipRangeEnd < ipRangeStart)||(ipRangeStart == 0))
-						continue;
-					uint32_t ipRangeLen = ipRangeEnd - ipRangeStart;
-
-					// Start with the LSB of the member's address
-					uint32_t ipTrialCounter = (uint32_t)(identity.address().toInt() & 0xffffffff);
-
-					for(uint32_t k=ipRangeStart,trialCount=0;((k<=ipRangeEnd)&&(trialCount < 1000));++k,++trialCount) {
-						uint32_t ip = (ipRangeLen > 0) ? (ipRangeStart + (ipTrialCounter % ipRangeLen)) : ipRangeStart;
-						++ipTrialCounter;
-						if ((ip & 0x000000ff) == 0x000000ff)
-							continue; // don't allow addresses that end in .255
-
-						// Check if this IP is within a local-to-Ethernet routed network
-						int routedNetmaskBits = -1;
-						for(unsigned int rk=0;rk<nc.routeCount;++rk) {
-							if (nc.routes[rk].target.ss_family == AF_INET) {
-								uint32_t targetIp = Utils::ntoh((uint32_t)(reinterpret_cast<const struct sockaddr_in *>(&(nc.routes[rk].target))->sin_addr.s_addr));
-								int targetBits = Utils::ntoh((uint16_t)(reinterpret_cast<const struct sockaddr_in *>(&(nc.routes[rk].target))->sin_port));
-								if ((ip & (0xffffffff << (32 - targetBits))) == targetIp) {
-									routedNetmaskBits = targetBits;
-									break;
-								}
-							}
-						}
-
-						// If it's routed, then try to claim and assign it and if successful end loop
-						const InetAddress ip4(Utils::hton(ip),0);
-						if ((routedNetmaskBits > 0)&&(!nmi.allocatedIps.count(ip4))) {
-							ipAssignments.push_back(ip4.toIpString());
-							member["ipAssignments"] = ipAssignments;
-							if (nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES) {
-								struct sockaddr_in *const v4ip = reinterpret_cast<struct sockaddr_in *>(&(nc.staticIps[nc.staticIpCount++]));
-								v4ip->sin_family = AF_INET;
-								v4ip->sin_port = Utils::hton((uint16_t)routedNetmaskBits);
-								v4ip->sin_addr.s_addr = Utils::hton(ip);
-							}
-							haveManagedIpv4AutoAssignment = true;
-							break;
-						}
-					}
-				}
-			}
-		}
-	}
-
-	CertificateOfMembership com(now,credentialtmd,nwid,identity.address());
-	if (com.sign(_signingId)) {
-		nc.com = com;
-	} else {
-		_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_INTERNAL_SERVER_ERROR);
-		return;
-	}
-
-	if (member != origMember) {
-		member["lastModified"] = now;
-		Mutex::Lock _l(_db_m);
-		_db.put("network",nwids,"member",identity.address().toString(),member);
-	}
-
-	_sender->ncSendConfig(nwid,requestPacketId,identity.address(),nc,metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_VERSION,0) < 6);
-}
-
-unsigned int EmbeddedNetworkController::handleControlPlaneHttpGET(
-	const std::vector<std::string> &path,
-	const std::map<std::string,std::string> &urlArgs,
-	const std::map<std::string,std::string> &headers,
-	const std::string &body,
-	std::string &responseBody,
-	std::string &responseContentType)
-{
-	if ((path.size() > 0)&&(path[0] == "network")) {
-
-		if ((path.size() >= 2)&&(path[1].length() == 16)) {
-			const uint64_t nwid = Utils::hexStrToU64(path[1].c_str());
-			char nwids[24];
-			Utils::snprintf(nwids,sizeof(nwids),"%.16llx",(unsigned long long)nwid);
-
-			json network;
-			{
-				Mutex::Lock _l(_db_m);
-				network = _db.get("network",nwids,0);
-			}
-			if (!network.size())
-				return 404;
-
-			if (path.size() >= 3) {
-
-				if (path[2] == "member") {
-
-					if (path.size() >= 4) {
-						const uint64_t address = Utils::hexStrToU64(path[3].c_str());
-
-						json member;
-						{
-							Mutex::Lock _l(_db_m);
-							member = _db.get("network",nwids,"member",Address(address).toString(),0);
-						}
-						if (!member.size())
-							return 404;
-
-						_addMemberNonPersistedFields(member,OSUtils::now());
-						responseBody = member.dump(2);
-						responseContentType = "application/json";
-
-						return 200;
-					} else {
-
-						Mutex::Lock _l(_db_m);
-
-						responseBody = "{";
-						std::string pfx(std::string("network/") + nwids + "member/");
-						_db.filter(pfx,120000,[&responseBody](const std::string &n,const json &member) {
-							if (member.size() > 0) {
-								responseBody.append((responseBody.length() == 1) ? "\"" : ",\"");
-								responseBody.append(_jS(member["id"],""));
-								responseBody.append("\":");
-								responseBody.append(_jS(member["revision"],"0"));
-							}
-							return true; // never delete
-						});
-						responseBody.push_back('}');
-						responseContentType = "application/json";
-
-						return 200;
-					}
-
-				} else if ((path[2] == "test")&&(path.size() >= 4)) {
-
-					Mutex::Lock _l(_circuitTests_m);
-					std::map< uint64_t,_CircuitTestEntry >::iterator cte(_circuitTests.find(Utils::hexStrToU64(path[3].c_str())));
-					if ((cte != _circuitTests.end())&&(cte->second.test)) {
-
-						responseBody = "[";
-						responseBody.append(cte->second.jsonResults);
-						responseBody.push_back(']');
-						responseContentType = "application/json";
-
-						return 200;
-
-					} // else 404
-
-				} // else 404
-
-			} else {
-
-				const uint64_t now = OSUtils::now();
-				_NetworkMemberInfo nmi;
-				_getNetworkMemberInfo(now,nwid,nmi);
-				_addNetworkNonPersistedFields(network,now,nmi);
-				responseBody = network.dump(2);
-				responseContentType = "application/json";
-				return 200;
-
-			}
-		} else if (path.size() == 1) {
-
-			std::set<std::string> networkIds;
-			{
-				Mutex::Lock _l(_db_m);
-				_db.filter("network/",120000,[&networkIds](const std::string &n,const json &obj) {
-					if (n.length() == (16 + 8))
-						networkIds.insert(n.substr(8));
-					return true; // do not delete
-				});
-			}
-
-			responseBody.push_back('[');
-			for(std::set<std::string>::iterator i(networkIds.begin());i!=networkIds.end();++i) {
-				responseBody.append((responseBody.length() == 1) ? "\"" : ",\"");
-				responseBody.append(*i);
-				responseBody.append("\"");
-			}
-			responseBody.push_back(']');
-			responseContentType = "application/json";
-			return 200;
-
-		} // else 404
-
-	} else {
-
-		char tmp[4096];
-		Utils::snprintf(tmp,sizeof(tmp),"{\n\t\"controller\": true,\n\t\"apiVersion\": %d,\n\t\"clock\": %llu\n}\n",ZT_NETCONF_CONTROLLER_API_VERSION,(unsigned long long)OSUtils::now());
-		responseBody = tmp;
-		responseContentType = "application/json";
-		return 200;
-
-	}
-
-	return 404;
-}
-
-unsigned int EmbeddedNetworkController::handleControlPlaneHttpPOST(
-	const std::vector<std::string> &path,
-	const std::map<std::string,std::string> &urlArgs,
-	const std::map<std::string,std::string> &headers,
-	const std::string &body,
-	std::string &responseBody,
-	std::string &responseContentType)
-{
-	if (path.empty())
-		return 404;
-
-	json b;
-	try {
-		b = json::parse(body);
-		if (!b.is_object()) {
-			responseBody = "{ \"message\": \"body is not a JSON object\" }";
-			responseContentType = "application/json";
-			return 400;
-		}
-	} catch ( ... ) {
-		responseBody = "{ \"message\": \"body JSON is invalid\" }";
-		responseContentType = "application/json";
-		return 400;
-	}
-	const uint64_t now = OSUtils::now();
-
-	if (path[0] == "network") {
-
-		if ((path.size() >= 2)&&(path[1].length() == 16)) {
-			uint64_t nwid = Utils::hexStrToU64(path[1].c_str());
-			char nwids[24];
-			Utils::snprintf(nwids,sizeof(nwids),"%.16llx",(unsigned long long)nwid);
-
-			if (path.size() >= 3) {
-
-				if ((path.size() == 4)&&(path[2] == "member")&&(path[3].length() == 10)) {
-					uint64_t address = Utils::hexStrToU64(path[3].c_str());
-					char addrs[24];
-					Utils::snprintf(addrs,sizeof(addrs),"%.10llx",(unsigned long long)address);
-
-					json member;
-					{
-						Mutex::Lock _l(_db_m);
-						member = _db.get("network",nwids,"member",Address(address).toString(),0);
-					}
-					json origMember(member); // for detecting changes
-					_initMember(member);
+					json origMember(member); // for detecting changes
+					_initMember(member);
 
 					try {
 						if (b.count("activeBridge")) member["activeBridge"] = _jB(b["activeBridge"],false);
@@ -1546,107 +1096,606 @@ unsigned int EmbeddedNetworkController::handleControlPlaneHttpDELETE(
 				Mutex::Lock _l(_db_m);
 				network = _db.get("network",nwids,0);
 			}
-			if (!network.size())
-				return 404;
+			if (!network.size())
+				return 404;
+
+			if (path.size() >= 3) {
+				if ((path.size() == 4)&&(path[2] == "member")&&(path[3].length() == 10)) {
+					const uint64_t address = Utils::hexStrToU64(path[3].c_str());
+
+					Mutex::Lock _l(_db_m);
+
+					json member = _db.get("network",nwids,"member",Address(address).toString(),0);
+					_db.erase("network",nwids,"member",Address(address).toString());
+
+					if (!member.size())
+						return 404;
+					responseBody = member.dump(2);
+					responseContentType = "application/json";
+					return 200;
+				}
+			} else {
+				Mutex::Lock _l(_db_m);
+
+				std::string pfx("network/"); pfx.append(nwids);
+				_db.filter(pfx,120000,[](const std::string &n,const json &obj) {
+					return false; // delete
+				});
+
+				Mutex::Lock _l2(_nmiCache_m);
+				_nmiCache.erase(nwid);
+
+				responseBody = network.dump(2);
+				responseContentType = "application/json";
+				return 200;
+			}
+		} // else 404
+
+	} // else 404
+
+	return 404;
+}
+
+void EmbeddedNetworkController::threadMain()
+	throw()
+{
+	for(;;) {
+		_RQEntry *const qe = _queue.get();
+		if (!qe) break; // enqueue a NULL to terminate threads
+		try {
+			_request(qe->nwid,qe->fromAddr,qe->requestPacketId,qe->identity,qe->metaData);
+		} catch ( ... ) {}
+		delete qe;
+	}
+}
+
+void EmbeddedNetworkController::_circuitTestCallback(ZT_Node *node,ZT_CircuitTest *test,const ZT_CircuitTestReport *report)
+{
+	char tmp[65535];
+	EmbeddedNetworkController *const self = reinterpret_cast<EmbeddedNetworkController *>(test->ptr);
+
+	if (!test)
+		return;
+	if (!report)
+		return;
+
+	Mutex::Lock _l(self->_circuitTests_m);
+	std::map< uint64_t,_CircuitTestEntry >::iterator cte(self->_circuitTests.find(test->testId));
+
+	if (cte == self->_circuitTests.end()) { // sanity check: a circuit test we didn't launch?
+		self->_node->circuitTestEnd(test);
+		::free((void *)test);
+		return;
+	}
+
+	Utils::snprintf(tmp,sizeof(tmp),
+		"%s{\n"
+		"\t\"timestamp\": %llu," ZT_EOL_S
+		"\t\"testId\": \"%.16llx\"," ZT_EOL_S
+		"\t\"upstream\": \"%.10llx\"," ZT_EOL_S
+		"\t\"current\": \"%.10llx\"," ZT_EOL_S
+		"\t\"receivedTimestamp\": %llu," ZT_EOL_S
+		"\t\"sourcePacketId\": \"%.16llx\"," ZT_EOL_S
+		"\t\"flags\": %llu," ZT_EOL_S
+		"\t\"sourcePacketHopCount\": %u," ZT_EOL_S
+		"\t\"errorCode\": %u," ZT_EOL_S
+		"\t\"vendor\": %d," ZT_EOL_S
+		"\t\"protocolVersion\": %u," ZT_EOL_S
+		"\t\"majorVersion\": %u," ZT_EOL_S
+		"\t\"minorVersion\": %u," ZT_EOL_S
+		"\t\"revision\": %u," ZT_EOL_S
+		"\t\"platform\": %d," ZT_EOL_S
+		"\t\"architecture\": %d," ZT_EOL_S
+		"\t\"receivedOnLocalAddress\": \"%s\"," ZT_EOL_S
+		"\t\"receivedFromRemoteAddress\": \"%s\"" ZT_EOL_S
+		"}",
+		((cte->second.jsonResults.length() > 0) ? ",\n" : ""),
+		(unsigned long long)report->timestamp,
+		(unsigned long long)test->testId,
+		(unsigned long long)report->upstream,
+		(unsigned long long)report->current,
+		(unsigned long long)OSUtils::now(),
+		(unsigned long long)report->sourcePacketId,
+		(unsigned long long)report->flags,
+		report->sourcePacketHopCount,
+		report->errorCode,
+		(int)report->vendor,
+		report->protocolVersion,
+		report->majorVersion,
+		report->minorVersion,
+		report->revision,
+		(int)report->platform,
+		(int)report->architecture,
+		reinterpret_cast<const InetAddress *>(&(report->receivedOnLocalAddress))->toString().c_str(),
+		reinterpret_cast<const InetAddress *>(&(report->receivedFromRemoteAddress))->toString().c_str());
+
+	cte->second.jsonResults.append(tmp);
+}
+
+void EmbeddedNetworkController::_request(
+	uint64_t nwid,
+	const InetAddress &fromAddr,
+	uint64_t requestPacketId,
+	const Identity &identity,
+	const Dictionary<ZT_NETWORKCONFIG_METADATA_DICT_CAPACITY> &metaData)
+{
+	if (((!_signingId)||(!_signingId.hasPrivate()))||(_signingId.address().toInt() != (nwid >> 24))||(!_sender))
+		return;
+
+	const uint64_t now = OSUtils::now();
+
+	if (requestPacketId) {
+		Mutex::Lock _l(_lastRequestTime_m);
+		uint64_t &lrt = _lastRequestTime[std::pair<uint64_t,uint64_t>(identity.address().toInt(),nwid)];
+		if ((now - lrt) <= ZT_NETCONF_MIN_REQUEST_PERIOD)
+			return;
+		lrt = now;
+	}
+
+	char nwids[24];
+	Utils::snprintf(nwids,sizeof(nwids),"%.16llx",nwid);
+	json network;
+	json member;
+	{
+		Mutex::Lock _l(_db_m);
+		network = _db.get("network",nwids,0);
+		member = _db.get("network",nwids,"member",identity.address().toString(),0);
+	}
+
+	if (!network.size()) {
+		_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_OBJECT_NOT_FOUND);
+		return;
+	}
+
+	json origMember(member); // for detecting modification later
+	_initMember(member);
+
+	{
+		std::string haveIdStr(_jS(member["identity"],""));
+		if (haveIdStr.length() > 0) {
+			// If we already know this member's identity perform a full compare. This prevents
+			// a "collision" from being able to auth onto our network in place of an already
+			// known member.
+			try {
+				if (Identity(haveIdStr.c_str()) != identity) {
+					_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_ACCESS_DENIED);
+					return;
+				}
+			} catch ( ... ) {
+				_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_ACCESS_DENIED);
+				return;
+			}
+		} else {
+			// If we do not yet know this member's identity, learn it.
+			member["identity"] = identity.toString(false);
+		}
+	}
+
+	// These are always the same, but make sure they are set
+	member["id"] = identity.address().toString();
+	member["address"] = member["id"];
+	member["nwid"] = nwids;
+
+	// Determine whether and how member is authorized
+	const char *authorizedBy = (const char *)0;
+	if (_jB(member["authorized"],false)) {
+		authorizedBy = "memberIsAuthorized";
+	} else if (!_jB(network["private"],true)) {
+		authorizedBy = "networkIsPublic";
+		if (!member.count("authorized")) {
+			member["authorized"] = true;
+			json ah;
+			ah["a"] = true;
+			ah["by"] = authorizedBy;
+			ah["ts"] = now;
+			ah["ct"] = json();
+			ah["c"] = json();
+			member["authHistory"].push_back(ah);
+			member["lastModified"] = now;
+			json &revj = member["revision"];
+			member["revision"] = (revj.is_number() ? ((uint64_t)revj + 1ULL) : 1ULL);
+		}
+	} else {
+		char presentedAuth[512];
+		if (metaData.get(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_AUTH,presentedAuth,sizeof(presentedAuth)) > 0) {
+			presentedAuth[511] = (char)0; // sanity check
+
+			// Check for bearer token presented by member
+			if ((strlen(presentedAuth) > 6)&&(!strncmp(presentedAuth,"token:",6))) {
+				const char *const presentedToken = presentedAuth + 6;
+
+				json &authTokens = network["authTokens"];
+				if (authTokens.is_array()) {
+					for(unsigned long i=0;i<authTokens.size();++i) {
+						json &token = authTokens[i];
+						if (token.is_object()) {
+							const uint64_t expires = _jI(token["expires"],0ULL);
+							const uint64_t maxUses = _jI(token["maxUsesPerMember"],0ULL);
+							std::string tstr = _jS(token["token"],"");
+
+							if (((expires == 0ULL)||(expires > now))&&(tstr == presentedToken)) {
+								bool usable = (maxUses == 0);
+								if (!usable) {
+									uint64_t useCount = 0;
+									json &ahist = member["authHistory"];
+									if (ahist.is_array()) {
+										for(unsigned long j=0;j<ahist.size();++j) {
+											json &ah = ahist[j];
+											if ((_jS(ah["ct"],"") == "token")&&(_jS(ah["c"],"") == tstr)&&(_jB(ah["a"],false)))
+												++useCount;
+										}
+									}
+									usable = (useCount < maxUses);
+								}
+								if (usable) {
+									authorizedBy = "token";
+									member["authorized"] = true;
+									json ah;
+									ah["a"] = true;
+									ah["by"] = authorizedBy;
+									ah["ts"] = now;
+									ah["ct"] = "token";
+									ah["c"] = tstr;
+									member["authHistory"].push_back(ah);
+									member["lastModified"] = now;
+									json &revj = member["revision"];
+									member["revision"] = (revj.is_number() ? ((uint64_t)revj + 1ULL) : 1ULL);
+								}
+							}
+						}
+					}
+				}
+			}
+		}
+	}
+
+	// Log this request
+	if (requestPacketId) { // only log if this is a request, not for generated pushes
+		json rlEntry = json::object();
+		rlEntry["ts"] = now;
+		rlEntry["authorized"] = (authorizedBy) ? true : false;
+		rlEntry["authorizedBy"] = (authorizedBy) ? authorizedBy : "";
+		rlEntry["clientMajorVersion"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_NODE_MAJOR_VERSION,0);
+		rlEntry["clientMinorVersion"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_NODE_MINOR_VERSION,0);
+		rlEntry["clientRevision"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_NODE_REVISION,0);
+		rlEntry["clientProtocolVersion"] = metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_PROTOCOL_VERSION,0);
+		if (fromAddr)
+			rlEntry["fromAddr"] = fromAddr.toString();
+
+		json recentLog = json::array();
+		recentLog.push_back(rlEntry);
+		json &oldLog = member["recentLog"];
+		if (oldLog.is_array()) {
+			for(unsigned long i=0;i<oldLog.size();++i) {
+				recentLog.push_back(oldLog[i]);
+				if (recentLog.size() >= ZT_NETCONF_DB_MEMBER_HISTORY_LENGTH)
+					break;
+			}
+		}
+		member["recentLog"] = recentLog;
+
+		// Also only do this on real requests
+		member["lastRequestMetaData"] = metaData.data();
+	}
+
+	// If they are not authorized, STOP!
+	if (!authorizedBy) {
+		if (origMember != member) {
+			member["lastModified"] = now;
+			Mutex::Lock _l(_db_m);
+			_db.put("network",nwids,"member",identity.address().toString(),member);
+		}
+		_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_ACCESS_DENIED);
+		return;
+	}
+
+	// -------------------------------------------------------------------------
+	// If we made it this far, they are authorized.
+	// -------------------------------------------------------------------------
+
+	NetworkConfig nc;
+	_NetworkMemberInfo nmi;
+	_getNetworkMemberInfo(now,nwid,nmi);
+
+	// Compute credential TTL. This is the "moving window" for COM agreement and
+	// the global TTL for Capability and Tag objects. (The same value is used
+	// for both.) This is computed by reference to the last time we deauthorized
+	// a member, since within the time period since this event any temporal
+	// differences are not particularly relevant.
+	uint64_t credentialtmd = ZT_NETWORKCONFIG_DEFAULT_CREDENTIAL_TIME_MIN_MAX_DELTA;
+	if (now > nmi.mostRecentDeauthTime)
+		credentialtmd += (now - nmi.mostRecentDeauthTime);
+	if (credentialtmd > ZT_NETWORKCONFIG_DEFAULT_CREDENTIAL_TIME_MAX_MAX_DELTA)
+		credentialtmd = ZT_NETWORKCONFIG_DEFAULT_CREDENTIAL_TIME_MAX_MAX_DELTA;
+
+	nc.networkId = nwid;
+	nc.type = _jB(network["private"],true) ? ZT_NETWORK_TYPE_PRIVATE : ZT_NETWORK_TYPE_PUBLIC;
+	nc.timestamp = now;
+	nc.credentialTimeMaxDelta = credentialtmd;
+	nc.revision = _jI(network["revision"],0ULL);
+	nc.issuedTo = identity.address();
+	if (_jB(network["enableBroadcast"],true)) nc.flags |= ZT_NETWORKCONFIG_FLAG_ENABLE_BROADCAST;
+	if (_jB(network["allowPassiveBridging"],false)) nc.flags |= ZT_NETWORKCONFIG_FLAG_ALLOW_PASSIVE_BRIDGING;
+	Utils::scopy(nc.name,sizeof(nc.name),_jS(network["name"],"").c_str());
+	nc.multicastLimit = (unsigned int)_jI(network["multicastLimit"],32ULL);
+
+	for(std::set<Address>::const_iterator ab(nmi.activeBridges.begin());ab!=nmi.activeBridges.end();++ab) {
+		nc.addSpecialist(*ab,ZT_NETWORKCONFIG_SPECIALIST_TYPE_ACTIVE_BRIDGE);
+	}
+
+	json &v4AssignMode = network["v4AssignMode"];
+	json &v6AssignMode = network["v6AssignMode"];
+	json &ipAssignmentPools = network["ipAssignmentPools"];
+	json &routes = network["routes"];
+	json &rules = network["rules"];
+	json &capabilities = network["capabilities"];
+	json &memberCapabilities = member["capabilities"];
+	json &memberTags = member["tags"];
+
+	if (metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_RULES_ENGINE_REV,0) <= 0) {
+		// Old versions with no rules engine support get an allow everything rule.
+		// Since rules are enforced bidirectionally, newer versions *will* still
+		// enforce rules on the inbound side.
+		nc.ruleCount = 1;
+		nc.rules[0].t = ZT_NETWORK_RULE_ACTION_ACCEPT;
+	} else {
+		if (rules.is_array()) {
+			for(unsigned long i=0;i<rules.size();++i) {
+				if (nc.ruleCount >= ZT_MAX_NETWORK_RULES)
+					break;
+				if (_parseRule(rules[i],nc.rules[nc.ruleCount]))
+					++nc.ruleCount;
+			}
+		}
+
+		if ((memberCapabilities.is_array())&&(memberCapabilities.size() > 0)&&(capabilities.is_array())) {
+			std::map< uint64_t,json * > capsById;
+			for(unsigned long i=0;i<capabilities.size();++i) {
+				json &cap = capabilities[i];
+				if (cap.is_object())
+					capsById[_jI(cap["id"],0ULL) & 0xffffffffULL] = &cap;
+			}
+
+			for(unsigned long i=0;i<memberCapabilities.size();++i) {
+				const uint64_t capId = _jI(memberCapabilities[i],0ULL) & 0xffffffffULL;
+				json *cap = capsById[capId];
+				if ((cap->is_object())&&(cap->size() > 0)) {
+					ZT_VirtualNetworkRule capr[ZT_MAX_CAPABILITY_RULES];
+					unsigned int caprc = 0;
+					json &caprj = (*cap)["rules"];
+					if ((caprj.is_array())&&(caprj.size() > 0)) {
+						for(unsigned long j=0;j<caprj.size();++j) {
+							if (caprc >= ZT_MAX_CAPABILITY_RULES)
+								break;
+							if (_parseRule(caprj[j],capr[caprc]))
+								++caprc;
+						}
+					}
+					nc.capabilities[nc.capabilityCount] = Capability((uint32_t)capId,nwid,now,1,capr,caprc);
+					if (nc.capabilities[nc.capabilityCount].sign(_signingId,identity.address()))
+						++nc.capabilityCount;
+					if (nc.capabilityCount >= ZT_MAX_NETWORK_CAPABILITIES)
+						break;
+				}
+			}
+		}
+
+		if (memberTags.is_array()) {
+			std::map< uint32_t,uint32_t > tagsById;
+			for(unsigned long i=0;i<memberTags.size();++i) {
+				json &t = memberTags[i];
+				if ((t.is_array())&&(t.size() == 2))
+					tagsById[(uint32_t)(_jI(t[0],0ULL) & 0xffffffffULL)] = (uint32_t)(_jI(t[1],0ULL) & 0xffffffffULL);
+			}
+			for(std::map< uint32_t,uint32_t >::const_iterator t(tagsById.begin());t!=tagsById.end();++t) {
+				if (nc.tagCount >= ZT_MAX_NETWORK_TAGS)
+					break;
+				nc.tags[nc.tagCount] = Tag(nwid,now,identity.address(),t->first,t->second);
+				if (nc.tags[nc.tagCount].sign(_signingId))
+					++nc.tagCount;
+			}
+		}
+	}
+
+	if (routes.is_array()) {
+		for(unsigned long i=0;i<routes.size();++i) {
+			if (nc.routeCount >= ZT_MAX_NETWORK_ROUTES)
+				break;
+			json &route = routes[i];
+			json &target = route["target"];
+			json &via = route["via"];
+			if (target.is_string()) {
+				const InetAddress t(target.get<std::string>());
+				InetAddress v;
+				if (via.is_string()) v.fromString(via.get<std::string>());
+				if ((t.ss_family == AF_INET)||(t.ss_family == AF_INET6)) {
+					ZT_VirtualNetworkRoute *r = &(nc.routes[nc.routeCount]);
+					*(reinterpret_cast<InetAddress *>(&(r->target))) = t;
+					if (v.ss_family == t.ss_family)
+						*(reinterpret_cast<InetAddress *>(&(r->via))) = v;
+					++nc.routeCount;
+				}
+			}
+		}
+	}
 
-			if (path.size() >= 3) {
-				if ((path.size() == 4)&&(path[2] == "member")&&(path[3].length() == 10)) {
-					const uint64_t address = Utils::hexStrToU64(path[3].c_str());
+	const bool noAutoAssignIps = _jB(member["noAutoAssignIps"],false);
 
-					Mutex::Lock _l(_db_m);
+	if ((v6AssignMode.is_object())&&(!noAutoAssignIps)) {
+		if ((_jB(v6AssignMode["rfc4193"],false))&&(nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES)) {
+			nc.staticIps[nc.staticIpCount++] = InetAddress::makeIpv6rfc4193(nwid,identity.address().toInt());
+			nc.flags |= ZT_NETWORKCONFIG_FLAG_ENABLE_IPV6_NDP_EMULATION;
+		}
+		if ((_jB(v6AssignMode["6plane"],false))&&(nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES)) {
+			nc.staticIps[nc.staticIpCount++] = InetAddress::makeIpv66plane(nwid,identity.address().toInt());
+			nc.flags |= ZT_NETWORKCONFIG_FLAG_ENABLE_IPV6_NDP_EMULATION;
+		}
+	}
 
-					json member = _db.get("network",nwids,"member",Address(address).toString(),0);
-					_db.erase("network",nwids,"member",Address(address).toString());
+	bool haveManagedIpv4AutoAssignment = false;
+	bool haveManagedIpv6AutoAssignment = false; // "special" NDP-emulated address types do not count
+	json ipAssignments = member["ipAssignments"]; // we want to make a copy
+	if (ipAssignments.is_array()) {
+		for(unsigned long i=0;i<ipAssignments.size();++i) {
+			if (!ipAssignments[i].is_string())
+				continue;
+			std::string ips = ipAssignments[i];
+			InetAddress ip(ips);
 
-					if (!member.size())
-						return 404;
-					responseBody = member.dump(2);
-					responseContentType = "application/json";
-					return 200;
+			// IP assignments are only pushed if there is a corresponding local route. We also now get the netmask bits from
+			// this route, ignoring the netmask bits field of the assigned IP itself. Using that was worthless and a source
+			// of user error / poor UX.
+			int routedNetmaskBits = 0;
+			for(unsigned int rk=0;rk<nc.routeCount;++rk) {
+				if ( (!nc.routes[rk].via.ss_family) && (reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->containsAddress(ip)) )
+					routedNetmaskBits = reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->netmaskBits();
+			}
+
+			if (routedNetmaskBits > 0) {
+				if (nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES) {
+					ip.setPort(routedNetmaskBits);
+					nc.staticIps[nc.staticIpCount++] = ip;
 				}
-			} else {
-				Mutex::Lock _l(_db_m);
+				if (ip.ss_family == AF_INET)
+					haveManagedIpv4AutoAssignment = true;
+				else if (ip.ss_family == AF_INET6)
+					haveManagedIpv6AutoAssignment = true;
+			}
+		}
+	} else {
+		ipAssignments = json::array();
+	}
 
-				std::string pfx("network/"); pfx.append(nwids);
-				_db.filter(pfx,120000,[](const std::string &n,const json &obj) {
-					return false; // delete
-				});
+	if ( (ipAssignmentPools.is_array()) && ((v6AssignMode.is_object())&&(_jB(v6AssignMode["zt"],false))) && (!haveManagedIpv6AutoAssignment) && (!noAutoAssignIps) ) {
+		for(unsigned long p=0;((p<ipAssignmentPools.size())&&(!haveManagedIpv6AutoAssignment));++p) {
+			json &pool = ipAssignmentPools[p];
+			if (pool.is_object()) {
+				InetAddress ipRangeStart(_jS(pool["ipRangeStart"],""));
+				InetAddress ipRangeEnd(_jS(pool["ipRangeEnd"],""));
+				if ( (ipRangeStart.ss_family == AF_INET6) && (ipRangeEnd.ss_family == AF_INET6) ) {
+					uint64_t s[2],e[2],x[2],xx[2];
+					memcpy(s,ipRangeStart.rawIpData(),16);
+					memcpy(e,ipRangeEnd.rawIpData(),16);
+					s[0] = Utils::ntoh(s[0]);
+					s[1] = Utils::ntoh(s[1]);
+					e[0] = Utils::ntoh(e[0]);
+					e[1] = Utils::ntoh(e[1]);
+					x[0] = s[0];
+					x[1] = s[1];
 
-				Mutex::Lock _l2(_nmiCache_m);
-				_nmiCache.erase(nwid);
+					for(unsigned int trialCount=0;trialCount<1000;++trialCount) {
+						if ((trialCount == 0)&&(e[1] > s[1])&&((e[1] - s[1]) >= 0xffffffffffULL)) {
+							// First see if we can just cram a ZeroTier ID into the higher 64 bits. If so do that.
+							xx[0] = Utils::hton(x[0]);
+							xx[1] = Utils::hton(x[1] + identity.address().toInt());
+						} else {
+							// Otherwise pick random addresses -- this technically doesn't explore the whole range if the lower 64 bit range is >= 1 but that won't matter since that would be huge anyway
+							Utils::getSecureRandom((void *)xx,16);
+							if ((e[0] > s[0]))
+								xx[0] %= (e[0] - s[0]);
+							else xx[0] = 0;
+							if ((e[1] > s[1]))
+								xx[1] %= (e[1] - s[1]);
+							else xx[1] = 0;
+							xx[0] = Utils::hton(x[0] + xx[0]);
+							xx[1] = Utils::hton(x[1] + xx[1]);
+						}
 
-				responseBody = network.dump(2);
-				responseContentType = "application/json";
-				return 200;
+						InetAddress ip6((const void *)xx,16,0);
+
+						// Check if this IP is within a local-to-Ethernet routed network
+						int routedNetmaskBits = 0;
+						for(unsigned int rk=0;rk<nc.routeCount;++rk) {
+							if ( (!nc.routes[rk].via.ss_family) && (nc.routes[rk].target.ss_family == AF_INET6) && (reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->containsAddress(ip6)) )
+								routedNetmaskBits = reinterpret_cast<const InetAddress *>(&(nc.routes[rk].target))->netmaskBits();
+						}
+
+						// If it's routed, then try to claim and assign it and if successful end loop
+						if ((routedNetmaskBits > 0)&&(!nmi.allocatedIps.count(ip6))) {
+							ipAssignments.push_back(ip6.toIpString());
+							member["ipAssignments"] = ipAssignments;
+							ip6.setPort((unsigned int)routedNetmaskBits);
+							if (nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES)
+								nc.staticIps[nc.staticIpCount++] = ip6;
+							haveManagedIpv6AutoAssignment = true;
+							break;
+						}
+					}
+				}
 			}
-		} // else 404
+		}
+	}
 
-	} // else 404
+	if ( (ipAssignmentPools.is_array()) && ((v4AssignMode.is_object())&&(_jB(v4AssignMode["zt"],false))) && (!haveManagedIpv4AutoAssignment) && (!noAutoAssignIps) ) {
+		for(unsigned long p=0;((p<ipAssignmentPools.size())&&(!haveManagedIpv4AutoAssignment));++p) {
+			json &pool = ipAssignmentPools[p];
+			if (pool.is_object()) {
+				InetAddress ipRangeStartIA(_jS(pool["ipRangeStart"],""));
+				InetAddress ipRangeEndIA(_jS(pool["ipRangeEnd"],""));
+				if ( (ipRangeStartIA.ss_family == AF_INET) && (ipRangeEndIA.ss_family == AF_INET) ) {
+					uint32_t ipRangeStart = Utils::ntoh((uint32_t)(reinterpret_cast<struct sockaddr_in *>(&ipRangeStartIA)->sin_addr.s_addr));
+					uint32_t ipRangeEnd = Utils::ntoh((uint32_t)(reinterpret_cast<struct sockaddr_in *>(&ipRangeEndIA)->sin_addr.s_addr));
+					if ((ipRangeEnd < ipRangeStart)||(ipRangeStart == 0))
+						continue;
+					uint32_t ipRangeLen = ipRangeEnd - ipRangeStart;
 
-	return 404;
-}
+					// Start with the LSB of the member's address
+					uint32_t ipTrialCounter = (uint32_t)(identity.address().toInt() & 0xffffffff);
 
-void EmbeddedNetworkController::_circuitTestCallback(ZT_Node *node,ZT_CircuitTest *test,const ZT_CircuitTestReport *report)
-{
-	char tmp[65535];
-	EmbeddedNetworkController *const self = reinterpret_cast<EmbeddedNetworkController *>(test->ptr);
+					for(uint32_t k=ipRangeStart,trialCount=0;((k<=ipRangeEnd)&&(trialCount < 1000));++k,++trialCount) {
+						uint32_t ip = (ipRangeLen > 0) ? (ipRangeStart + (ipTrialCounter % ipRangeLen)) : ipRangeStart;
+						++ipTrialCounter;
+						if ((ip & 0x000000ff) == 0x000000ff)
+							continue; // don't allow addresses that end in .255
 
-	if (!test)
-		return;
-	if (!report)
-		return;
+						// Check if this IP is within a local-to-Ethernet routed network
+						int routedNetmaskBits = -1;
+						for(unsigned int rk=0;rk<nc.routeCount;++rk) {
+							if (nc.routes[rk].target.ss_family == AF_INET) {
+								uint32_t targetIp = Utils::ntoh((uint32_t)(reinterpret_cast<const struct sockaddr_in *>(&(nc.routes[rk].target))->sin_addr.s_addr));
+								int targetBits = Utils::ntoh((uint16_t)(reinterpret_cast<const struct sockaddr_in *>(&(nc.routes[rk].target))->sin_port));
+								if ((ip & (0xffffffff << (32 - targetBits))) == targetIp) {
+									routedNetmaskBits = targetBits;
+									break;
+								}
+							}
+						}
 
-	Mutex::Lock _l(self->_circuitTests_m);
-	std::map< uint64_t,_CircuitTestEntry >::iterator cte(self->_circuitTests.find(test->testId));
+						// If it's routed, then try to claim and assign it and if successful end loop
+						const InetAddress ip4(Utils::hton(ip),0);
+						if ((routedNetmaskBits > 0)&&(!nmi.allocatedIps.count(ip4))) {
+							ipAssignments.push_back(ip4.toIpString());
+							member["ipAssignments"] = ipAssignments;
+							if (nc.staticIpCount < ZT_MAX_ZT_ASSIGNED_ADDRESSES) {
+								struct sockaddr_in *const v4ip = reinterpret_cast<struct sockaddr_in *>(&(nc.staticIps[nc.staticIpCount++]));
+								v4ip->sin_family = AF_INET;
+								v4ip->sin_port = Utils::hton((uint16_t)routedNetmaskBits);
+								v4ip->sin_addr.s_addr = Utils::hton(ip);
+							}
+							haveManagedIpv4AutoAssignment = true;
+							break;
+						}
+					}
+				}
+			}
+		}
+	}
 
-	if (cte == self->_circuitTests.end()) { // sanity check: a circuit test we didn't launch?
-		self->_node->circuitTestEnd(test);
-		::free((void *)test);
+	CertificateOfMembership com(now,credentialtmd,nwid,identity.address());
+	if (com.sign(_signingId)) {
+		nc.com = com;
+	} else {
+		_sender->ncSendError(nwid,requestPacketId,identity.address(),NetworkController::NC_ERROR_INTERNAL_SERVER_ERROR);
 		return;
 	}
 
-	Utils::snprintf(tmp,sizeof(tmp),
-		"%s{\n"
-		"\t\"timestamp\": %llu," ZT_EOL_S
-		"\t\"testId\": \"%.16llx\"," ZT_EOL_S
-		"\t\"upstream\": \"%.10llx\"," ZT_EOL_S
-		"\t\"current\": \"%.10llx\"," ZT_EOL_S
-		"\t\"receivedTimestamp\": %llu," ZT_EOL_S
-		"\t\"sourcePacketId\": \"%.16llx\"," ZT_EOL_S
-		"\t\"flags\": %llu," ZT_EOL_S
-		"\t\"sourcePacketHopCount\": %u," ZT_EOL_S
-		"\t\"errorCode\": %u," ZT_EOL_S
-		"\t\"vendor\": %d," ZT_EOL_S
-		"\t\"protocolVersion\": %u," ZT_EOL_S
-		"\t\"majorVersion\": %u," ZT_EOL_S
-		"\t\"minorVersion\": %u," ZT_EOL_S
-		"\t\"revision\": %u," ZT_EOL_S
-		"\t\"platform\": %d," ZT_EOL_S
-		"\t\"architecture\": %d," ZT_EOL_S
-		"\t\"receivedOnLocalAddress\": \"%s\"," ZT_EOL_S
-		"\t\"receivedFromRemoteAddress\": \"%s\"" ZT_EOL_S
-		"}",
-		((cte->second.jsonResults.length() > 0) ? ",\n" : ""),
-		(unsigned long long)report->timestamp,
-		(unsigned long long)test->testId,
-		(unsigned long long)report->upstream,
-		(unsigned long long)report->current,
-		(unsigned long long)OSUtils::now(),
-		(unsigned long long)report->sourcePacketId,
-		(unsigned long long)report->flags,
-		report->sourcePacketHopCount,
-		report->errorCode,
-		(int)report->vendor,
-		report->protocolVersion,
-		report->majorVersion,
-		report->minorVersion,
-		report->revision,
-		(int)report->platform,
-		(int)report->architecture,
-		reinterpret_cast<const InetAddress *>(&(report->receivedOnLocalAddress))->toString().c_str(),
-		reinterpret_cast<const InetAddress *>(&(report->receivedFromRemoteAddress))->toString().c_str());
+	if (member != origMember) {
+		member["lastModified"] = now;
+		Mutex::Lock _l(_db_m);
+		_db.put("network",nwids,"member",identity.address().toString(),member);
+	}
 
-	cte->second.jsonResults.append(tmp);
+	_sender->ncSendConfig(nwid,requestPacketId,identity.address(),nc,metaData.getUI(ZT_NETWORKCONFIG_REQUEST_METADATA_KEY_VERSION,0) < 6);
 }
 
 void EmbeddedNetworkController::_getNetworkMemberInfo(uint64_t now,uint64_t nwid,_NetworkMemberInfo &nmi)

+ 27 - 0
controller/EmbeddedNetworkController.hpp

@@ -37,11 +37,15 @@
 
 #include "../osdep/OSUtils.hpp"
 #include "../osdep/Thread.hpp"
+#include "../osdep/BlockingQueue.hpp"
 
 #include "../ext/json/json.hpp"
 
 #include "JSONDB.hpp"
 
+// Number of background threads to start -- not actually started until needed
+#define ZT_EMBEDDEDNETWORKCONTROLLER_BACKGROUND_THREAD_COUNT 2
+
 namespace ZeroTier {
 
 class Node;
@@ -83,8 +87,31 @@ public:
 		std::string &responseBody,
 		std::string &responseContentType);
 
+	void threadMain()
+		throw();
+
 private:
 	static void _circuitTestCallback(ZT_Node *node,ZT_CircuitTest *test,const ZT_CircuitTestReport *report);
+	void _request(
+		uint64_t nwid,
+		const InetAddress &fromAddr,
+		uint64_t requestPacketId,
+		const Identity &identity,
+		const Dictionary<ZT_NETWORKCONFIG_METADATA_DICT_CAPACITY> &metaData);
+
+	struct _RQEntry
+	{
+		uint64_t nwid;
+		uint64_t requestPacketId;
+		InetAddress fromAddr;
+		Identity identity;
+		Dictionary<ZT_NETWORKCONFIG_METADATA_DICT_CAPACITY> metaData;
+	};
+	BlockingQueue<_RQEntry *> _queue;
+
+	Thread _threads[ZT_EMBEDDEDNETWORKCONTROLLER_BACKGROUND_THREAD_COUNT];
+	bool _threadsStarted;
+	Mutex _threads_m;
 
 	// Gathers a bunch of statistics about members of a network, IP assignments, etc. that we need in various places
 	// This does lock _networkMemberCache_m

+ 64 - 0
osdep/BlockingQueue.hpp

@@ -0,0 +1,64 @@
+/*
+ * ZeroTier One - Network Virtualization Everywhere
+ * Copyright (C) 2011-2016  ZeroTier, Inc.  https://www.zerotier.com/
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation, either version 3 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program.  If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#ifndef ZT_BLOCKINGQUEUE_HPP
+#define ZT_BLOCKINGQUEUE_HPP
+
+#include <queue>
+#include <mutex>
+#include <condition_variable>
+
+namespace ZeroTier {
+
+/**
+ * Simple C++11 thread-safe queue
+ *
+ * Do not use in node/ since we have not gone C++11 there yet.
+ */
+template <class T>
+class BlockingQueue
+{
+public:
+	BlockingQueue(void) {}
+
+	inline void post(T t)
+	{
+		std::lock_guard<std::mutex> lock(m);
+		q.push(t);
+		c.notify_one();
+	}
+
+	inline T get(void)
+	{
+		std::unique_lock<std::mutex> lock(m);
+		while(q.empty())
+			c.wait(lock);
+		T val = q.front();
+		q.pop();
+		return val;
+	}
+
+private:
+	std::queue<T> q;
+	mutable std::mutex m;
+	std::condition_variable c;
+};
+
+} // namespace ZeroTier
+
+#endif