CentralDB.cpp 55 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622
  1. /*
  2. * Copyright (c)2019 ZeroTier, Inc.
  3. *
  4. * Use of this software is governed by the Business Source License included
  5. * in the LICENSE.TXT file in the project's root directory.
  6. *
  7. * Change Date: 2026-01-01
  8. *
  9. * On the date above, in accordance with the Business Source License, use
  10. * of this software will be governed by version 2.0 of the Apache License.
  11. */
  12. /****/
  13. #include "CentralDB.hpp"
  14. #ifdef ZT_CONTROLLER_USE_LIBPQ
  15. #include "../node/Constants.hpp"
  16. #include "../node/SHA512.hpp"
  17. #include "../version.h"
  18. #include "CtlUtil.hpp"
  19. #include "EmbeddedNetworkController.hpp"
  20. #include "PubSubListener.hpp"
  21. #include "Redis.hpp"
  22. #include "RedisListener.hpp"
  23. #include "opentelemetry/trace/provider.h"
  24. #include <chrono>
  25. #include <climits>
  26. #include <iomanip>
  27. #include <libpq-fe.h>
  28. #include <optional>
  29. #include <pqxx/pqxx>
  30. #include <rustybits.h>
  31. #include <sstream>
  32. // #define REDIS_TRACE 1
  33. using json = nlohmann::json;
  34. namespace {
  35. static const int DB_MINIMUM_VERSION = 38;
  36. } // anonymous namespace
  37. using namespace ZeroTier;
  38. using Attrs = std::vector<std::pair<std::string, std::string> >;
  39. using Item = std::pair<std::string, Attrs>;
  40. using ItemStream = std::vector<Item>;
  41. CentralDB::CentralDB(const Identity& myId, const char* path, int listenPort, CentralDB::ListenerMode mode, ControllerConfig* cc)
  42. : DB()
  43. , _listenerMode(mode)
  44. , _controllerConfig(cc)
  45. , _pool()
  46. , _myId(myId)
  47. , _myAddress(myId.address())
  48. , _ready(0)
  49. , _connected(1)
  50. , _run(1)
  51. , _waitNoticePrinted(false)
  52. , _listenPort(listenPort)
  53. , _rc(cc->redisConfig)
  54. , _redis(NULL)
  55. , _cluster(NULL)
  56. , _redisMemberStatus(false)
  57. , _smee(NULL)
  58. {
  59. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  60. auto tracer = provider->GetTracer("CentralDB");
  61. auto span = tracer->StartSpan("CentralDB::CentralDB");
  62. auto scope = tracer->WithActiveSpan(span);
  63. rustybits::init_async_runtime();
  64. char myAddress[64];
  65. _myAddressStr = myId.address().toString(myAddress);
  66. _connString = std::string(path);
  67. auto f = std::make_shared<PostgresConnFactory>(_connString);
  68. _pool = std::make_shared<ConnectionPool<PostgresConnection> >(15, 5, std::static_pointer_cast<ConnectionFactory>(f));
  69. memset(_ssoPsk, 0, sizeof(_ssoPsk));
  70. char* const ssoPskHex = getenv("ZT_SSO_PSK");
  71. #ifdef ZT_TRACE
  72. fprintf(stderr, "ZT_SSO_PSK: %s\n", ssoPskHex);
  73. #endif
  74. if (ssoPskHex) {
  75. // SECURITY: note that ssoPskHex will always be null-terminated if libc actually
  76. // returns something non-NULL. If the hex encodes something shorter than 48 bytes,
  77. // it will be padded at the end with zeroes. If longer, it'll be truncated.
  78. Utils::unhex(ssoPskHex, _ssoPsk, sizeof(_ssoPsk));
  79. }
  80. const char* redisMemberStatus = getenv("ZT_REDIS_MEMBER_STATUS");
  81. if (redisMemberStatus && (strcmp(redisMemberStatus, "true") == 0)) {
  82. _redisMemberStatus = true;
  83. fprintf(stderr, "Using redis for member status\n");
  84. }
  85. auto c = _pool->borrow();
  86. pqxx::work txn { *c->c };
  87. pqxx::row r { txn.exec1("SELECT version FROM ztc_database") };
  88. int dbVersion = r[0].as<int>();
  89. txn.commit();
  90. if (dbVersion < DB_MINIMUM_VERSION) {
  91. fprintf(stderr, "Central database schema version too low. This controller version requires a minimum schema version of %d. Please upgrade your Central instance", DB_MINIMUM_VERSION);
  92. exit(1);
  93. }
  94. _pool->unborrow(c);
  95. if (mode == LISTENER_MODE_REDIS && _rc != NULL) {
  96. auto innerspan = tracer->StartSpan("CentralDB::CentralDB::configureRedis");
  97. auto innerscope = tracer->WithActiveSpan(innerspan);
  98. sw::redis::ConnectionOptions opts;
  99. sw::redis::ConnectionPoolOptions poolOpts;
  100. opts.host = _rc->hostname;
  101. opts.port = _rc->port;
  102. opts.password = _rc->password;
  103. opts.db = 0;
  104. opts.keep_alive = true;
  105. opts.connect_timeout = std::chrono::seconds(3);
  106. poolOpts.size = 25;
  107. poolOpts.wait_timeout = std::chrono::seconds(5);
  108. poolOpts.connection_lifetime = std::chrono::minutes(3);
  109. poolOpts.connection_idle_time = std::chrono::minutes(1);
  110. if (_rc->clusterMode) {
  111. innerspan->SetAttribute("cluster_mode", "true");
  112. fprintf(stderr, "Using Redis in Cluster Mode\n");
  113. _cluster = std::make_shared<sw::redis::RedisCluster>(opts, poolOpts);
  114. }
  115. else {
  116. innerspan->SetAttribute("cluster_mode", "false");
  117. fprintf(stderr, "Using Redis in Standalone Mode\n");
  118. _redis = std::make_shared<sw::redis::Redis>(opts, poolOpts);
  119. }
  120. }
  121. _readyLock.lock();
  122. fprintf(stderr, "[%s] NOTICE: %.10llx controller PostgreSQL waiting for initial data download..." ZT_EOL_S, ::_timestr(), (unsigned long long)_myAddress.toInt());
  123. _waitNoticePrinted = true;
  124. initializeNetworks();
  125. initializeMembers();
  126. _heartbeatThread = std::thread(&CentralDB::heartbeat, this);
  127. switch (mode) {
  128. case LISTENER_MODE_REDIS:
  129. if (_rc != NULL) {
  130. if (_rc->clusterMode) {
  131. _membersDbWatcher = std::make_shared<RedisMemberListener>(_myAddressStr, _cluster, this);
  132. _networksDbWatcher = std::make_shared<RedisNetworkListener>(_myAddressStr, _cluster, this);
  133. }
  134. else {
  135. _membersDbWatcher = std::make_shared<RedisMemberListener>(_myAddressStr, _redis, this);
  136. _networksDbWatcher = std::make_shared<RedisNetworkListener>(_myAddressStr, _redis, this);
  137. }
  138. }
  139. else {
  140. throw std::runtime_error("CentralDB: Redis listener mode selected but no Redis configuration provided");
  141. }
  142. case LISTENER_MODE_PUBSUB:
  143. if (cc->pubSubConfig != NULL) {
  144. _membersDbWatcher = std::make_shared<PubSubMemberListener>(_myAddressStr, cc->pubSubConfig->listen_timeout, this);
  145. _networksDbWatcher = std::make_shared<PubSubNetworkListener>(_myAddressStr, cc->pubSubConfig->listen_timeout, this);
  146. }
  147. else {
  148. throw std::runtime_error("CentralDB: PubSub listener mode selected but no PubSub configuration provided");
  149. }
  150. break;
  151. case LISTENER_MODE_PGSQL:
  152. default:
  153. _membersDbWatcher = std::make_shared<PostgresMemberListener>(this, _pool, "member_" + _myAddressStr, 5);
  154. _networksDbWatcher = std::make_shared<PostgresNetworkListener>(this, _pool, "network_" + _myAddressStr, 5);
  155. break;
  156. }
  157. for (int i = 0; i < ZT_CENTRAL_CONTROLLER_COMMIT_THREADS; ++i) {
  158. _commitThread[i] = std::thread(&CentralDB::commitThread, this);
  159. }
  160. _onlineNotificationThread = std::thread(&CentralDB::onlineNotificationThread, this);
  161. configureSmee();
  162. }
  163. CentralDB::~CentralDB()
  164. {
  165. if (_smee != NULL) {
  166. rustybits::smee_client_delete(_smee);
  167. _smee = NULL;
  168. }
  169. rustybits::shutdown_async_runtime();
  170. _run = 0;
  171. std::this_thread::sleep_for(std::chrono::milliseconds(100));
  172. _heartbeatThread.join();
  173. _commitQueue.stop();
  174. for (int i = 0; i < ZT_CENTRAL_CONTROLLER_COMMIT_THREADS; ++i) {
  175. _commitThread[i].join();
  176. }
  177. _onlineNotificationThread.join();
  178. }
  179. void CentralDB::configureSmee()
  180. {
  181. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  182. auto tracer = provider->GetTracer("CentralDB");
  183. auto span = tracer->StartSpan("CentralDB::configureSmee");
  184. auto scope = tracer->WithActiveSpan(span);
  185. const char* TEMPORAL_SCHEME = "ZT_TEMPORAL_SCHEME";
  186. const char* TEMPORAL_HOST = "ZT_TEMPORAL_HOST";
  187. const char* TEMPORAL_PORT = "ZT_TEMPORAL_PORT";
  188. const char* TEMPORAL_NAMESPACE = "ZT_TEMPORAL_NAMESPACE";
  189. const char* SMEE_TASK_QUEUE = "ZT_SMEE_TASK_QUEUE";
  190. const char* scheme = getenv(TEMPORAL_SCHEME);
  191. if (scheme == NULL) {
  192. scheme = "http";
  193. }
  194. const char* host = getenv(TEMPORAL_HOST);
  195. const char* port = getenv(TEMPORAL_PORT);
  196. const char* ns = getenv(TEMPORAL_NAMESPACE);
  197. const char* task_queue = getenv(SMEE_TASK_QUEUE);
  198. if (scheme != NULL && host != NULL && port != NULL && ns != NULL && task_queue != NULL) {
  199. fprintf(stderr, "creating smee client\n");
  200. std::string hostPort = std::string(scheme) + std::string("://") + std::string(host) + std::string(":") + std::string(port);
  201. this->_smee = rustybits::smee_client_new(hostPort.c_str(), ns, task_queue);
  202. }
  203. else {
  204. fprintf(stderr, "Smee client not configured\n");
  205. }
  206. }
  207. bool CentralDB::waitForReady()
  208. {
  209. while (_ready < 2) {
  210. _readyLock.lock();
  211. _readyLock.unlock();
  212. }
  213. return true;
  214. }
  215. bool CentralDB::isReady()
  216. {
  217. return ((_ready == 2) && (_connected));
  218. }
  219. bool CentralDB::save(nlohmann::json& record, bool notifyListeners)
  220. {
  221. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  222. auto tracer = provider->GetTracer("CentralDB");
  223. auto span = tracer->StartSpan("CentralDB::save");
  224. auto scope = tracer->WithActiveSpan(span);
  225. bool modified = false;
  226. try {
  227. if (! record.is_object()) {
  228. fprintf(stderr, "record is not an object?!?\n");
  229. return false;
  230. }
  231. const std::string objtype = record["objtype"];
  232. if (objtype == "network") {
  233. // fprintf(stderr, "network save\n");
  234. const uint64_t nwid = OSUtils::jsonIntHex(record["id"], 0ULL);
  235. if (nwid) {
  236. nlohmann::json old;
  237. get(nwid, old);
  238. if ((! old.is_object()) || (! _compareRecords(old, record))) {
  239. record["revision"] = OSUtils::jsonInt(record["revision"], 0ULL) + 1ULL;
  240. _commitQueue.post(std::pair<nlohmann::json, bool>(record, notifyListeners));
  241. modified = true;
  242. }
  243. }
  244. }
  245. else if (objtype == "member") {
  246. std::string networkId = record["nwid"];
  247. std::string memberId = record["id"];
  248. const uint64_t nwid = OSUtils::jsonIntHex(record["nwid"], 0ULL);
  249. const uint64_t id = OSUtils::jsonIntHex(record["id"], 0ULL);
  250. // fprintf(stderr, "member save %s-%s\n", networkId.c_str(), memberId.c_str());
  251. if ((id) && (nwid)) {
  252. nlohmann::json network, old;
  253. get(nwid, network, id, old);
  254. if ((! old.is_object()) || (! _compareRecords(old, record))) {
  255. // fprintf(stderr, "commit queue post\n");
  256. record["revision"] = OSUtils::jsonInt(record["revision"], 0ULL) + 1ULL;
  257. _commitQueue.post(std::pair<nlohmann::json, bool>(record, notifyListeners));
  258. modified = true;
  259. }
  260. else {
  261. // fprintf(stderr, "no change\n");
  262. }
  263. }
  264. }
  265. else {
  266. fprintf(stderr, "uhh waaat\n");
  267. }
  268. }
  269. catch (std::exception& e) {
  270. fprintf(stderr, "Error on PostgreSQL::save: %s\n", e.what());
  271. }
  272. catch (...) {
  273. fprintf(stderr, "Unknown error on PostgreSQL::save\n");
  274. }
  275. return modified;
  276. }
  277. void CentralDB::eraseNetwork(const uint64_t networkId)
  278. {
  279. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  280. auto tracer = provider->GetTracer("CentralDB");
  281. auto span = tracer->StartSpan("CentralDB::eraseNetwork");
  282. auto scope = tracer->WithActiveSpan(span);
  283. char networkIdStr[17];
  284. span->SetAttribute("network_id", Utils::hex(networkId, networkIdStr));
  285. fprintf(stderr, "PostgreSQL::eraseNetwork\n");
  286. char tmp2[24];
  287. waitForReady();
  288. Utils::hex(networkId, tmp2);
  289. std::pair<nlohmann::json, bool> tmp;
  290. tmp.first["id"] = tmp2;
  291. tmp.first["objtype"] = "_delete_network";
  292. tmp.second = true;
  293. _commitQueue.post(tmp);
  294. nlohmann::json nullJson;
  295. _networkChanged(tmp.first, nullJson, true);
  296. }
  297. void CentralDB::eraseMember(const uint64_t networkId, const uint64_t memberId)
  298. {
  299. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  300. auto tracer = provider->GetTracer("CentralDB");
  301. auto span = tracer->StartSpan("CentralDB::eraseMember");
  302. auto scope = tracer->WithActiveSpan(span);
  303. char networkIdStr[17];
  304. char memberIdStr[11];
  305. span->SetAttribute("network_id", Utils::hex(networkId, networkIdStr));
  306. span->SetAttribute("member_id", Utils::hex10(memberId, memberIdStr));
  307. fprintf(stderr, "PostgreSQL::eraseMember\n");
  308. char tmp2[24];
  309. waitForReady();
  310. std::pair<nlohmann::json, bool> tmp, nw;
  311. Utils::hex(networkId, tmp2);
  312. tmp.first["nwid"] = tmp2;
  313. Utils::hex(memberId, tmp2);
  314. tmp.first["id"] = tmp2;
  315. tmp.first["objtype"] = "_delete_member";
  316. tmp.second = true;
  317. _commitQueue.post(tmp);
  318. nlohmann::json nullJson;
  319. _memberChanged(tmp.first, nullJson, true);
  320. }
  321. void CentralDB::nodeIsOnline(const uint64_t networkId, const uint64_t memberId, const InetAddress& physicalAddress, const char* osArch)
  322. {
  323. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  324. auto tracer = provider->GetTracer("CentralDB");
  325. auto span = tracer->StartSpan("CentralDB::nodeIsOnline");
  326. auto scope = tracer->WithActiveSpan(span);
  327. char networkIdStr[17];
  328. char memberIdStr[11];
  329. char ipStr[INET6_ADDRSTRLEN];
  330. span->SetAttribute("network_id", Utils::hex(networkId, networkIdStr));
  331. span->SetAttribute("member_id", Utils::hex10(memberId, memberIdStr));
  332. span->SetAttribute("physical_address", physicalAddress.toString(ipStr));
  333. span->SetAttribute("os_arch", osArch);
  334. std::lock_guard<std::mutex> l(_lastOnline_l);
  335. NodeOnlineRecord& i = _lastOnline[std::pair<uint64_t, uint64_t>(networkId, memberId)];
  336. i.lastSeen = OSUtils::now();
  337. if (physicalAddress) {
  338. i.physicalAddress = physicalAddress;
  339. }
  340. i.osArch = std::string(osArch);
  341. }
  342. void CentralDB::nodeIsOnline(const uint64_t networkId, const uint64_t memberId, const InetAddress& physicalAddress)
  343. {
  344. this->nodeIsOnline(networkId, memberId, physicalAddress, "unknown/unknown");
  345. }
  346. AuthInfo CentralDB::getSSOAuthInfo(const nlohmann::json& member, const std::string& redirectURL)
  347. {
  348. if (_controllerConfig->ssoEnabled) {
  349. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  350. auto tracer = provider->GetTracer("CentralDB");
  351. auto span = tracer->StartSpan("CentralDB::getSSOAuthInfo");
  352. auto scope = tracer->WithActiveSpan(span);
  353. Metrics::db_get_sso_info++;
  354. // NONCE is just a random character string. no semantic meaning
  355. // state = HMAC SHA384 of Nonce based on shared sso key
  356. //
  357. // need nonce timeout in database? make sure it's used within X time
  358. // X is 5 minutes for now. Make configurable later?
  359. //
  360. // how do we tell when a nonce is used? if auth_expiration_time is set
  361. std::string networkId = member["nwid"];
  362. std::string memberId = member["id"];
  363. char authenticationURL[4096] = { 0 };
  364. AuthInfo info;
  365. info.enabled = true;
  366. // if (memberId == "a10dccea52" && networkId == "8056c2e21c24673d") {
  367. // fprintf(stderr, "invalid authinfo for grant's machine\n");
  368. // info.version=1;
  369. // return info;
  370. // }
  371. // fprintf(stderr, "PostgreSQL::updateMemberOnLoad: %s-%s\n", networkId.c_str(), memberId.c_str());
  372. std::shared_ptr<PostgresConnection> c;
  373. try {
  374. c = _pool->borrow();
  375. pqxx::work w(*c->c);
  376. char nonceBytes[16] = { 0 };
  377. std::string nonce = "";
  378. // check if the member exists first.
  379. pqxx::row count = w.exec_params1("SELECT count(id) FROM ztc_member WHERE id = $1 AND network_id = $2 AND deleted = false", memberId, networkId);
  380. if (count[0].as<int>() == 1) {
  381. // get active nonce, if exists.
  382. pqxx::result r = w.exec_params(
  383. "SELECT nonce FROM ztc_sso_expiry "
  384. "WHERE network_id = $1 AND member_id = $2 "
  385. "AND ((NOW() AT TIME ZONE 'UTC') <= authentication_expiry_time) AND ((NOW() AT TIME ZONE 'UTC') <= nonce_expiration)",
  386. networkId,
  387. memberId);
  388. if (r.size() == 0) {
  389. // no active nonce.
  390. // find an unused nonce, if one exists.
  391. pqxx::result r = w.exec_params(
  392. "SELECT nonce FROM ztc_sso_expiry "
  393. "WHERE network_id = $1 AND member_id = $2 "
  394. "AND authentication_expiry_time IS NULL AND ((NOW() AT TIME ZONE 'UTC') <= nonce_expiration)",
  395. networkId,
  396. memberId);
  397. if (r.size() == 1) {
  398. // we have an existing nonce. Use it
  399. nonce = r.at(0)[0].as<std::string>();
  400. Utils::unhex(nonce.c_str(), nonceBytes, sizeof(nonceBytes));
  401. }
  402. else if (r.empty()) {
  403. // create a nonce
  404. Utils::getSecureRandom(nonceBytes, 16);
  405. char nonceBuf[64] = { 0 };
  406. Utils::hex(nonceBytes, sizeof(nonceBytes), nonceBuf);
  407. nonce = std::string(nonceBuf);
  408. pqxx::result ir = w.exec_params0(
  409. "INSERT INTO ztc_sso_expiry "
  410. "(nonce, nonce_expiration, network_id, member_id) VALUES "
  411. "($1, TO_TIMESTAMP($2::double precision/1000), $3, $4)",
  412. nonce,
  413. OSUtils::now() + 300000,
  414. networkId,
  415. memberId);
  416. w.commit();
  417. }
  418. else {
  419. // > 1 ?!? Thats an error!
  420. fprintf(stderr, "> 1 unused nonce!\n");
  421. exit(6);
  422. }
  423. }
  424. else if (r.size() == 1) {
  425. nonce = r.at(0)[0].as<std::string>();
  426. Utils::unhex(nonce.c_str(), nonceBytes, sizeof(nonceBytes));
  427. }
  428. else {
  429. // more than 1 nonce in use? Uhhh...
  430. fprintf(stderr, "> 1 nonce in use for network member?!?\n");
  431. exit(7);
  432. }
  433. r = w.exec_params(
  434. "SELECT oc.client_id, oc.authorization_endpoint, oc.issuer, oc.provider, oc.sso_impl_version "
  435. "FROM ztc_network AS n "
  436. "INNER JOIN ztc_org o "
  437. " ON o.owner_id = n.owner_id "
  438. "LEFT OUTER JOIN ztc_network_oidc_config noc "
  439. " ON noc.network_id = n.id "
  440. "LEFT OUTER JOIN ztc_oidc_config oc "
  441. " ON noc.client_id = oc.client_id AND oc.org_id = o.org_id "
  442. "WHERE n.id = $1 AND n.sso_enabled = true",
  443. networkId);
  444. std::string client_id = "";
  445. std::string authorization_endpoint = "";
  446. std::string issuer = "";
  447. std::string provider = "";
  448. uint64_t sso_version = 0;
  449. if (r.size() == 1) {
  450. client_id = r.at(0)[0].as<std::optional<std::string> >().value_or("");
  451. authorization_endpoint = r.at(0)[1].as<std::optional<std::string> >().value_or("");
  452. issuer = r.at(0)[2].as<std::optional<std::string> >().value_or("");
  453. provider = r.at(0)[3].as<std::optional<std::string> >().value_or("");
  454. sso_version = r.at(0)[4].as<std::optional<uint64_t> >().value_or(1);
  455. }
  456. else if (r.size() > 1) {
  457. fprintf(stderr, "ERROR: More than one auth endpoint for an organization?!?!? NetworkID: %s\n", networkId.c_str());
  458. }
  459. else {
  460. fprintf(stderr, "No client or auth endpoint?!?\n");
  461. }
  462. info.version = sso_version;
  463. // no catch all else because we don't actually care if no records exist here. just continue as normal.
  464. if ((! client_id.empty()) && (! authorization_endpoint.empty())) {
  465. uint8_t state[48];
  466. HMACSHA384(_ssoPsk, nonceBytes, sizeof(nonceBytes), state);
  467. char state_hex[256];
  468. Utils::hex(state, 48, state_hex);
  469. if (info.version == 0) {
  470. char url[2048] = { 0 };
  471. OSUtils::ztsnprintf(
  472. url,
  473. sizeof(authenticationURL),
  474. "%s?response_type=id_token&response_mode=form_post&scope=openid+email+profile&redirect_uri=%s&nonce=%s&state=%s&client_id=%s",
  475. authorization_endpoint.c_str(),
  476. url_encode(redirectURL).c_str(),
  477. nonce.c_str(),
  478. state_hex,
  479. client_id.c_str());
  480. info.authenticationURL = std::string(url);
  481. }
  482. else if (info.version == 1) {
  483. info.ssoClientID = client_id;
  484. info.issuerURL = issuer;
  485. info.ssoProvider = provider;
  486. info.ssoNonce = nonce;
  487. info.ssoState = std::string(state_hex) + "_" + networkId;
  488. info.centralAuthURL = redirectURL;
  489. #ifdef ZT_DEBUG
  490. fprintf(
  491. stderr,
  492. "ssoClientID: %s\nissuerURL: %s\nssoNonce: %s\nssoState: %s\ncentralAuthURL: %s\nprovider: %s\n",
  493. info.ssoClientID.c_str(),
  494. info.issuerURL.c_str(),
  495. info.ssoNonce.c_str(),
  496. info.ssoState.c_str(),
  497. info.centralAuthURL.c_str(),
  498. provider.c_str());
  499. #endif
  500. }
  501. }
  502. else {
  503. fprintf(stderr, "client_id: %s\nauthorization_endpoint: %s\n", client_id.c_str(), authorization_endpoint.c_str());
  504. }
  505. }
  506. _pool->unborrow(c);
  507. }
  508. catch (std::exception& e) {
  509. span->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  510. fprintf(stderr, "ERROR: Error updating member on load for network %s: %s\n", networkId.c_str(), e.what());
  511. }
  512. return info; // std::string(authenticationURL);
  513. }
  514. return AuthInfo();
  515. }
  516. void CentralDB::initializeNetworks()
  517. {
  518. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  519. auto tracer = provider->GetTracer("CentralDB");
  520. auto span = tracer->StartSpan("CentralDB::initializeNetworks");
  521. auto scope = tracer->WithActiveSpan(span);
  522. fprintf(stderr, "Initializing networks...\n");
  523. try {
  524. char qbuf[2048];
  525. sprintf(
  526. qbuf,
  527. "SELECT id, name, configuration , (EXTRACT(EPOCH FROM creation_time AT TIME ZONE 'UTC')*1000)::bigint, "
  528. "(EXTRACT(EPOCH FROM last_modified AT TIME ZONE 'UTC')*1000)::bigint, revision "
  529. "FROM networks_ctl WHERE controller_id = '%s'",
  530. _myAddressStr.c_str());
  531. auto c = _pool->borrow();
  532. pqxx::work w(*c->c);
  533. fprintf(stderr, "Load networks from psql...\n");
  534. auto stream = pqxx::stream_from::query(w, qbuf);
  535. std::tuple<
  536. std::string // network ID
  537. ,
  538. std::optional<std::string> // name
  539. ,
  540. std::string // configuration
  541. ,
  542. std::optional<uint64_t> // creation_time
  543. ,
  544. std::optional<uint64_t> // last_modified
  545. ,
  546. std::optional<uint64_t> // revision
  547. >
  548. row;
  549. uint64_t count = 0;
  550. uint64_t total = 0;
  551. while (stream >> row) {
  552. auto start = std::chrono::high_resolution_clock::now();
  553. json empty;
  554. json config;
  555. initNetwork(config);
  556. std::string nwid = std::get<0>(row);
  557. std::string name = std::get<1>(row).value_or("");
  558. json cfgtmp = json::parse(std::get<2>(row));
  559. std::optional<uint64_t> created_at = std::get<3>(row);
  560. std::optional<uint64_t> last_modified = std::get<4>(row);
  561. std::optional<uint64_t> revision = std::get<5>(row);
  562. config["id"] = nwid;
  563. config["name"] = name;
  564. config["creationTime"] = created_at.value_or(0);
  565. config["lastModified"] = last_modified.value_or(0);
  566. config["revision"] = revision.value_or(0);
  567. config["capabilities"] = cfgtmp["capabilities"].is_array() ? cfgtmp["capabilities"] : json::array();
  568. config["enableBroadcast"] = cfgtmp["enableBroadcast"].is_boolean() ? cfgtmp["enableBroadcast"].get<bool>() : false;
  569. config["mtu"] = cfgtmp["mtu"].is_number() ? cfgtmp["mtu"].get<int32_t>() : 2800;
  570. config["multicastLimit"] = cfgtmp["multicastLimit"].is_number() ? cfgtmp["multicastLimit"].get<int32_t>() : 64;
  571. config["private"] = cfgtmp["private"].is_boolean() ? cfgtmp["private"].get<bool>() : true;
  572. config["remoteTraceLevel"] = cfgtmp["remoteTraceLevel"].is_number() ? cfgtmp["remoteTraceLevel"].get<int32_t>() : 0;
  573. config["remoteTraceTarget"] = cfgtmp["remoteTraceTarget"].is_string() ? cfgtmp["remoteTraceTarget"].get<std::string>() : "";
  574. config["revision"] = revision.value_or(0);
  575. config["rules"] = cfgtmp["rules"].is_array() ? cfgtmp["rules"] : json::array();
  576. config["tags"] = cfgtmp["tags"].is_array() ? cfgtmp["tags"] : json::array();
  577. if (cfgtmp["v4AssignMode"].is_object()) {
  578. config["v4AssignMode"] = cfgtmp["v4AssignMode"];
  579. }
  580. else {
  581. config["v4AssignMode"] = json::object();
  582. config["v4AssignMode"]["zt"] = true;
  583. }
  584. if (cfgtmp["v6AssignMode"].is_object()) {
  585. config["v6AssignMode"] = cfgtmp["v6AssignMode"];
  586. }
  587. else {
  588. config["v6AssignMode"] = json::object();
  589. config["v6AssignMode"]["zt"] = true;
  590. config["v6AssignMode"]["6plane"] = true;
  591. config["v6AssignMode"]["rfc4193"] = false;
  592. }
  593. config["ssoEnabled"] = cfgtmp["ssoEnabled"].is_boolean() ? cfgtmp["ssoEnabled"].get<bool>() : false;
  594. config["objtype"] = "network";
  595. config["routes"] = cfgtmp["routes"].is_array() ? cfgtmp["routes"] : json::array();
  596. config["clientId"] = cfgtmp["clientId"].is_string() ? cfgtmp["clientId"].get<std::string>() : "";
  597. config["authorizationEndpoint"] = cfgtmp["authorizationEndpoint"].is_string() ? cfgtmp["authorizationEndpoint"].get<std::string>() : nullptr;
  598. config["provider"] = cfgtmp["ssoProvider"].is_string() ? cfgtmp["ssoProvider"].get<std::string>() : "";
  599. if (! cfgtmp["dns"].is_object()) {
  600. cfgtmp["dns"] = json::object();
  601. cfgtmp["dns"]["domain"] = "";
  602. cfgtmp["dns"]["servers"] = json::array();
  603. }
  604. else {
  605. config["dns"] = cfgtmp["dns"];
  606. }
  607. config["ipAssignmentPools"] = cfgtmp["ipAssignmentPools"].is_array() ? cfgtmp["ipAssignmentPools"] : json::array();
  608. Metrics::network_count++;
  609. _networkChanged(empty, config, false);
  610. auto end = std::chrono::high_resolution_clock::now();
  611. auto dur = std::chrono::duration_cast<std::chrono::microseconds>(end - start);
  612. ;
  613. total += dur.count();
  614. ++count;
  615. if (count > 0 && count % 10000 == 0) {
  616. fprintf(stderr, "Averaging %lu us per network\n", (total / count));
  617. }
  618. }
  619. w.commit();
  620. _pool->unborrow(c);
  621. fprintf(stderr, "done.\n");
  622. if (++this->_ready == 2) {
  623. if (_waitNoticePrinted) {
  624. fprintf(stderr, "[%s] NOTICE: %.10llx controller PostgreSQL data download complete." ZT_EOL_S, _timestr(), (unsigned long long)_myAddress.toInt());
  625. }
  626. _readyLock.unlock();
  627. }
  628. fprintf(stderr, "network init done\n");
  629. }
  630. catch (std::exception& e) {
  631. fprintf(stderr, "ERROR: Error initializing networks: %s\n", e.what());
  632. span->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  633. std::this_thread::sleep_for(std::chrono::milliseconds(5000));
  634. exit(-1);
  635. }
  636. }
  637. void CentralDB::initializeMembers()
  638. {
  639. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  640. auto tracer = provider->GetTracer("CentralDB");
  641. auto span = tracer->StartSpan("CentralDB::initializeMembers");
  642. auto scope = tracer->WithActiveSpan(span);
  643. std::string memberId;
  644. std::string networkId;
  645. try {
  646. std::unordered_map<std::string, std::string> networkMembers;
  647. fprintf(stderr, "Initializing Members...\n");
  648. std::string setKeyBase = "network-nodes-all:{" + _myAddressStr + "}:";
  649. if (_redisMemberStatus) {
  650. fprintf(stderr, "Initialize Redis for members...\n");
  651. std::unique_lock<std::shared_mutex> l(_networks_l);
  652. std::unordered_set<std::string> deletes;
  653. for (auto it : _networks) {
  654. uint64_t nwid_i = it.first;
  655. char nwidTmp[64] = { 0 };
  656. OSUtils::ztsnprintf(nwidTmp, sizeof(nwidTmp), "%.16llx", nwid_i);
  657. std::string nwid(nwidTmp);
  658. std::string key = setKeyBase + nwid;
  659. deletes.insert(key);
  660. }
  661. if (! deletes.empty()) {
  662. try {
  663. if (_rc->clusterMode) {
  664. auto tx = _cluster->transaction(_myAddressStr, true, false);
  665. for (std::string k : deletes) {
  666. tx.del(k);
  667. }
  668. tx.exec();
  669. }
  670. else {
  671. auto tx = _redis->transaction(true, false);
  672. for (std::string k : deletes) {
  673. tx.del(k);
  674. }
  675. tx.exec();
  676. }
  677. }
  678. catch (sw::redis::Error& e) {
  679. // ignore
  680. }
  681. }
  682. }
  683. char qbuf[2048];
  684. sprintf(
  685. qbuf,
  686. "SELECT nm.device_id, nm.network_id, nm.authorized, nm.active_bridge, nm.ip_assignments, nm.no_auto_assign_ips, "
  687. "nm.sso_exempt, (EXTRACT(EPOCH FROM nm.authentication_expiry_time AT TIME ZONE 'UTC')*1000)::bigint, "
  688. "(EXTRACT(EPOCH FROM nm.creation_time AT TIME ZONE 'UTC')*1000)::bigint, nm.identity, "
  689. "(EXTRACT(EPOCH FROM nm.last_authorized_time AT TIME ZONE 'UTC')*1000)::bigint, "
  690. "(EXTRACT(EPOCH FROM nm.last_deauthorized_time AT TIME ZONE 'UTC')*1000)::bigint, "
  691. "nm.remote_trace_level, nm.remote_trace_target, nm.revision, nm.capabilities, nm.tags "
  692. "FROM network_memberships_ctl nm "
  693. "INNER JOIN networks_ctl n "
  694. " ON nm.network_id = n.id "
  695. "WHERE n.controller_id = '%s'",
  696. _myAddressStr.c_str());
  697. auto c = _pool->borrow();
  698. pqxx::work w(*c->c);
  699. fprintf(stderr, "Load members from psql...\n");
  700. auto stream = pqxx::stream_from::query(w, qbuf);
  701. std::tuple<
  702. std::string // device ID
  703. ,
  704. std::string // network ID
  705. ,
  706. bool // authorized
  707. ,
  708. std::optional<bool> // active_bridge
  709. ,
  710. std::optional<std::string> // ip_assignments
  711. ,
  712. std::optional<bool> // no_auto_assign_ips
  713. ,
  714. std::optional<bool> // sso_exempt
  715. ,
  716. std::optional<uint64_t> // authentication_expiry_time
  717. ,
  718. std::optional<uint64_t> // creation_time
  719. ,
  720. std::optional<std::string> // identity
  721. ,
  722. std::optional<uint64_t> // last_authorized_time
  723. ,
  724. std::optional<uint64_t> // last_deauthorized_time
  725. ,
  726. std::optional<int32_t> // remote_trace_level
  727. ,
  728. std::optional<std::string> // remote_trace_target
  729. ,
  730. std::optional<uint64_t> // revision
  731. ,
  732. std::optional<std::string> // capabilities
  733. ,
  734. std::optional<std::string> // tags
  735. >
  736. row;
  737. auto tmp = std::chrono::high_resolution_clock::now();
  738. uint64_t count = 0;
  739. uint64_t total = 0;
  740. while (stream >> row) {
  741. auto start = std::chrono::high_resolution_clock::now();
  742. json empty;
  743. json config;
  744. initMember(config);
  745. memberId = std::get<0>(row);
  746. networkId = std::get<1>(row);
  747. bool authorized = std::get<2>(row);
  748. std::optional<bool> active_bridge = std::get<3>(row);
  749. std::string ip_assignments = std::get<4>(row).value_or("");
  750. std::optional<bool> no_auto_assign_ips = std::get<5>(row);
  751. std::optional<bool> sso_exempt = std::get<6>(row);
  752. std::optional<uint64_t> authentication_expiry_time = std::get<7>(row);
  753. std::optional<uint64_t> creation_time = std::get<8>(row);
  754. std::optional<std::string> identity = std::get<9>(row);
  755. std::optional<uint64_t> last_authorized_time = std::get<10>(row);
  756. std::optional<uint64_t> last_deauthorized_time = std::get<11>(row);
  757. std::optional<int32_t> remote_trace_level = std::get<12>(row);
  758. std::optional<std::string> remote_trace_target = std::get<13>(row);
  759. std::optional<uint64_t> revision = std::get<14>(row);
  760. std::optional<std::string> capabilities = std::get<15>(row);
  761. std::optional<std::string> tags = std::get<16>(row);
  762. networkMembers.insert(std::pair<std::string, std::string>(setKeyBase + networkId, memberId));
  763. config["objtype"] = "member";
  764. config["id"] = memberId;
  765. config["address"] = identity.value_or("");
  766. config["nwid"] = networkId;
  767. config["authorized"] = authorized;
  768. config["activeBridge"] = active_bridge.value_or(false);
  769. config["ipAssignments"] = json::array();
  770. if (ip_assignments != "{}") {
  771. std::string tmp = ip_assignments.substr(1, ip_assignments.length() - 2);
  772. std::vector<std::string> addrs = split(tmp, ',');
  773. for (auto it = addrs.begin(); it != addrs.end(); ++it) {
  774. config["ipAssignments"].push_back(*it);
  775. }
  776. }
  777. config["capabilities"] = json::parse(capabilities.value_or("[]"));
  778. config["creationTime"] = creation_time.value_or(0);
  779. config["lastAuthorizedTime"] = last_authorized_time.value_or(0);
  780. config["lastDeauthorizedTime"] = last_deauthorized_time.value_or(0);
  781. config["noAutoAssignIPs"] = no_auto_assign_ips.value_or(false);
  782. config["remoteTraceLevel"] = remote_trace_level.value_or(0);
  783. config["remoteTraceTarget"] = remote_trace_target.value_or(nullptr);
  784. config["revision"] = revision.value_or(0);
  785. config["ssoExempt"] = sso_exempt.value_or(false);
  786. config["authenticationExpiryTime"] = authentication_expiry_time.value_or(0);
  787. config["tags"] = json::parse(tags.value_or("[]"));
  788. config["ipAssignments"] = json::array();
  789. Metrics::member_count++;
  790. _memberChanged(empty, config, false);
  791. memberId = "";
  792. networkId = "";
  793. auto end = std::chrono::high_resolution_clock::now();
  794. auto dur = std::chrono::duration_cast<std::chrono::microseconds>(end - start);
  795. total += dur.count();
  796. ++count;
  797. if (count > 0 && count % 10000 == 0) {
  798. fprintf(stderr, "Averaging %llu us per member\n", (total / count));
  799. }
  800. }
  801. if (count > 0) {
  802. fprintf(stderr, "Took %llu us per member to load\n", (total / count));
  803. }
  804. stream.complete();
  805. w.commit();
  806. _pool->unborrow(c);
  807. fprintf(stderr, "done.\n");
  808. if (_listenerMode == LISTENER_MODE_REDIS)
  809. if (! networkMembers.empty()) {
  810. if (_redisMemberStatus) {
  811. fprintf(stderr, "Load member data into redis...\n");
  812. if (_rc->clusterMode) {
  813. auto tx = _cluster->transaction(_myAddressStr, true, false);
  814. uint64_t count = 0;
  815. for (auto it : networkMembers) {
  816. tx.sadd(it.first, it.second);
  817. if (++count % 30000 == 0) {
  818. tx.exec();
  819. tx = _cluster->transaction(_myAddressStr, true, false);
  820. }
  821. }
  822. tx.exec();
  823. }
  824. else {
  825. auto tx = _redis->transaction(true, false);
  826. uint64_t count = 0;
  827. for (auto it : networkMembers) {
  828. tx.sadd(it.first, it.second);
  829. if (++count % 30000 == 0) {
  830. tx.exec();
  831. tx = _redis->transaction(true, false);
  832. }
  833. }
  834. tx.exec();
  835. }
  836. fprintf(stderr, "done.\n");
  837. }
  838. }
  839. fprintf(stderr, "Done loading members...\n");
  840. if (++this->_ready == 2) {
  841. if (_waitNoticePrinted) {
  842. fprintf(stderr, "[%s] NOTICE: %.10llx controller PostgreSQL data download complete." ZT_EOL_S, _timestr(), (unsigned long long)_myAddress.toInt());
  843. }
  844. _readyLock.unlock();
  845. }
  846. }
  847. catch (sw::redis::Error& e) {
  848. span->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  849. fprintf(stderr, "ERROR: Error initializing members (redis): %s\n", e.what());
  850. exit(-1);
  851. }
  852. catch (std::exception& e) {
  853. span->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  854. fprintf(stderr, "ERROR: Error initializing member: %s-%s %s\n", networkId.c_str(), memberId.c_str(), e.what());
  855. exit(-1);
  856. }
  857. }
  858. void CentralDB::heartbeat()
  859. {
  860. char publicId[1024];
  861. char hostnameTmp[1024];
  862. _myId.toString(false, publicId);
  863. if (gethostname(hostnameTmp, sizeof(hostnameTmp)) != 0) {
  864. hostnameTmp[0] = (char)0;
  865. }
  866. else {
  867. for (int i = 0; i < (int)sizeof(hostnameTmp); ++i) {
  868. if ((hostnameTmp[i] == '.') || (hostnameTmp[i] == 0)) {
  869. hostnameTmp[i] = (char)0;
  870. break;
  871. }
  872. }
  873. }
  874. const char* controllerId = _myAddressStr.c_str();
  875. const char* publicIdentity = publicId;
  876. const char* hostname = hostnameTmp;
  877. while (_run == 1) {
  878. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  879. auto tracer = provider->GetTracer("CentralDB");
  880. auto span = tracer->StartSpan("CentralDB::heartbeat");
  881. auto scope = tracer->WithActiveSpan(span);
  882. // fprintf(stderr, "%s: heartbeat\n", controllerId);
  883. auto c = _pool->borrow();
  884. int64_t ts = OSUtils::now();
  885. if (c->c) {
  886. std::string major = std::to_string(ZEROTIER_ONE_VERSION_MAJOR);
  887. std::string minor = std::to_string(ZEROTIER_ONE_VERSION_MINOR);
  888. std::string rev = std::to_string(ZEROTIER_ONE_VERSION_REVISION);
  889. std::string version = major + "." + minor + "." + rev;
  890. std::string versionStr = "v" + version;
  891. try {
  892. pqxx::work w { *c->c };
  893. w.exec_params0(
  894. "INSERT INTO controllers_ctl (id, hostname, last_heartbeat, public_identity, version) VALUES "
  895. "($1, $2, TO_TIMESTAMP($3::double precision/1000), $4, $5) "
  896. "ON CONFLICT (id) DO UPDATE SET hostname = EXCLUDED.hostname, last_heartbeat = EXCLUDED.last_heartbeat, "
  897. "public_identity = EXCLUDED.public_identity, version = EXCLUDED.version",
  898. controllerId,
  899. hostname,
  900. ts,
  901. publicIdentity,
  902. versionStr);
  903. w.commit();
  904. }
  905. catch (std::exception& e) {
  906. fprintf(stderr, "%s: Heartbeat update failed: %s\n", controllerId, e.what());
  907. span->End();
  908. std::this_thread::sleep_for(std::chrono::milliseconds(1000));
  909. continue;
  910. }
  911. }
  912. _pool->unborrow(c);
  913. try {
  914. if (_listenerMode == LISTENER_MODE_REDIS && _redisMemberStatus) {
  915. if (_rc->clusterMode) {
  916. _cluster->zadd("controllers", "controllerId", ts);
  917. }
  918. else {
  919. _redis->zadd("controllers", "controllerId", ts);
  920. }
  921. }
  922. }
  923. catch (sw::redis::Error& e) {
  924. fprintf(stderr, "ERROR: Redis error in heartbeat thread: %s\n", e.what());
  925. }
  926. span->End();
  927. std::this_thread::sleep_for(std::chrono::milliseconds(1000));
  928. }
  929. fprintf(stderr, "Exited heartbeat thread\n");
  930. }
  931. void CentralDB::commitThread()
  932. {
  933. fprintf(stderr, "%s: commitThread start\n", _myAddressStr.c_str());
  934. std::pair<nlohmann::json, bool> qitem;
  935. while (_commitQueue.get(qitem) & (_run == 1)) {
  936. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  937. auto tracer = provider->GetTracer("CentralDB");
  938. auto span = tracer->StartSpan("CentralDB::commitThread");
  939. auto scope = tracer->WithActiveSpan(span);
  940. // fprintf(stderr, "commitThread tick\n");
  941. if (! qitem.first.is_object()) {
  942. fprintf(stderr, "not an object\n");
  943. continue;
  944. }
  945. std::shared_ptr<PostgresConnection> c;
  946. try {
  947. c = _pool->borrow();
  948. }
  949. catch (std::exception& e) {
  950. fprintf(stderr, "ERROR: %s\n", e.what());
  951. continue;
  952. }
  953. if (! c) {
  954. fprintf(stderr, "Error getting database connection\n");
  955. continue;
  956. }
  957. Metrics::pgsql_commit_ticks++;
  958. try {
  959. nlohmann::json& config = (qitem.first);
  960. const std::string objtype = config["objtype"];
  961. if (objtype == "member") {
  962. auto mspan = tracer->StartSpan("CentralDB::commitThread::member");
  963. auto mscope = tracer->WithActiveSpan(mspan);
  964. // fprintf(stderr, "%s: commitThread: member\n", _myAddressStr.c_str());
  965. std::string memberId;
  966. std::string networkId;
  967. try {
  968. pqxx::work w(*c->c);
  969. memberId = config["id"];
  970. networkId = config["nwid"];
  971. std::string target = "NULL";
  972. if (! config["remoteTraceTarget"].is_null()) {
  973. target = config["remoteTraceTarget"];
  974. }
  975. pqxx::row nwrow = w.exec_params1("SELECT COUNT(id) FROM ztc_network WHERE id = $1", networkId);
  976. int nwcount = nwrow[0].as<int>();
  977. if (nwcount != 1) {
  978. fprintf(stderr, "network %s does not exist. skipping member upsert\n", networkId.c_str());
  979. w.abort();
  980. _pool->unborrow(c);
  981. continue;
  982. }
  983. pqxx::row mrow = w.exec_params1("SELECT COUNT(id) FROM ztc_member WHERE id = $1 AND network_id = $2", memberId, networkId);
  984. int membercount = mrow[0].as<int>();
  985. bool isNewMember = (membercount == 0);
  986. pqxx::result res = w.exec_params0(
  987. "INSERT INTO network_memberships_ctl (device_id, network_id, authorized, active_bridge, ip_assignments, "
  988. "no_auto_assign_ips, sso_exempt, authentication_expiry_time, capabilities, creation_time, "
  989. "identity, last_authorized_time, last_deauthorized_time, "
  990. "remote_trace_level, remote_trace_target, revision, tags, version_major, version_minor, "
  991. "version_revision, version_protocol) "
  992. "VALUES ($1, $2, $3, $4, $5, $6, $7, TO_TIMESTAMP($8::double precision/1000), $9, "
  993. "TO_TIMESTAMP($10::double precision/1000), $11, TO_TIMESTAMP($12::double precision/1000), "
  994. "TO_TIMESTAMP($13::double precision/1000), $14, $15, $16, $17, $18, $19, $20, $21) "
  995. "ON CONFLICT (device_id, network_id) DO UPDATE SET "
  996. "authorized = EXCLUDED.authorized, active_bridge = EXCLUDED.active_bridge, "
  997. "ip_assignments = EXCLUDED.ip_assignments, no_auto_assign_ips = EXCLUDED.no_auto_assign_ips, "
  998. "sso_exempt = EXCLUDED.sso_exempt, authentication_expiry_time = EXCLUDED.authentication_expiry_time, "
  999. "capabilities = EXCLUDED.capabilities, creation_time = EXCLUDED.creation_time, "
  1000. "identity = EXCLUDED.identity, last_authorized_time = EXCLUDED.last_authorized_time, "
  1001. "last_deauthorized_time = EXCLUDED.last_deauthorized_time, "
  1002. "remote_trace_level = EXCLUDED.remote_trace_level, remote_trace_target = EXCLUDED.remote_trace_target, "
  1003. "revision = EXCLUDED.revision, tags = EXCLUDED.tags, version_major = EXCLUDED.version_major, "
  1004. "version_minor = EXCLUDED.version_minor, version_revision = EXCLUDED.version_revision, "
  1005. "version_protocol = EXCLUDED.version_protocol",
  1006. memberId,
  1007. networkId,
  1008. (bool)config["authorized"],
  1009. (bool)config["activeBridge"],
  1010. config["ipAssignments"].get<std::vector<std::string> >(),
  1011. (bool)config["noAutoAssignIps"],
  1012. (bool)config["ssoExempt"],
  1013. (uint64_t)config["authenticationExpiryTime"],
  1014. OSUtils::jsonDump(config["capabilities"], -1),
  1015. (uint64_t)config["creationTime"],
  1016. OSUtils::jsonString(config["identity"], ""),
  1017. (uint64_t)config["lastAuthorizedTime"],
  1018. (uint64_t)config["lastDeauthorizedTime"],
  1019. (int)config["remoteTraceLevel"],
  1020. target,
  1021. (uint64_t)config["revision"],
  1022. OSUtils::jsonDump(config["tags"], -1),
  1023. (int)config["vMajor"],
  1024. (int)config["vMinor"],
  1025. (int)config["vRev"],
  1026. (int)config["vProto"]);
  1027. w.commit();
  1028. if (! isNewMember) {
  1029. pqxx::result res = w.exec_params0("DELETE FROM ztc_member_ip_assignment WHERE member_id = $1 AND network_id = $2", memberId, networkId);
  1030. }
  1031. if (_smee != NULL && isNewMember) {
  1032. // TODO: Smee Notifications for New Members
  1033. // pqxx::row row = w.exec_params1(
  1034. // "SELECT "
  1035. // " count(h.hook_id) "
  1036. // "FROM "
  1037. // " ztc_hook h "
  1038. // " INNER JOIN ztc_org o ON o.org_id = h.org_id "
  1039. // " INNER JOIN ztc_network n ON n.owner_id = o.owner_id "
  1040. // " WHERE "
  1041. // "n.id = $1 ",
  1042. // networkId);
  1043. // int64_t hookCount = row[0].as<int64_t>();
  1044. // if (hookCount > 0) {
  1045. // notifyNewMember(networkId, memberId);
  1046. // }
  1047. }
  1048. const uint64_t nwidInt = OSUtils::jsonIntHex(config["nwid"], 0ULL);
  1049. const uint64_t memberidInt = OSUtils::jsonIntHex(config["id"], 0ULL);
  1050. if (nwidInt && memberidInt) {
  1051. nlohmann::json nwOrig;
  1052. nlohmann::json memOrig;
  1053. nlohmann::json memNew(config);
  1054. get(nwidInt, nwOrig, memberidInt, memOrig);
  1055. _memberChanged(memOrig, memNew, qitem.second);
  1056. }
  1057. else {
  1058. fprintf(stderr, "%s: Can't notify of change. Error parsing nwid or memberid: %llu-%llu\n", _myAddressStr.c_str(), (unsigned long long)nwidInt, (unsigned long long)memberidInt);
  1059. }
  1060. }
  1061. catch (std::exception& e) {
  1062. fprintf(stderr, "%s ERROR: Error updating member %s-%s: %s\n", _myAddressStr.c_str(), networkId.c_str(), memberId.c_str(), e.what());
  1063. mspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1064. }
  1065. }
  1066. else if (objtype == "network") {
  1067. auto nspan = tracer->StartSpan("CentralDB::commitThread::network");
  1068. auto nscope = tracer->WithActiveSpan(nspan);
  1069. try {
  1070. // fprintf(stderr, "%s: commitThread: network\n", _myAddressStr.c_str());
  1071. pqxx::work w(*c->c);
  1072. std::string id = config["id"];
  1073. // network must already exist
  1074. pqxx::result res = w.exec_params0(
  1075. "INSERT INTO networks_ctl (id, name, configuration, controller_id, revision) "
  1076. "VALUES ($1, $2, $3, $4, $5) "
  1077. "ON CONFLICT (id) DO UPDATE SET "
  1078. "name = EXCLUDED.name, configuration = EXCLUDED.configuration, revision = EXCLUDED.revision+1",
  1079. id,
  1080. OSUtils::jsonString(config["name"], ""),
  1081. OSUtils::jsonDump(config, -1),
  1082. _myAddressStr,
  1083. ((uint64_t)config["revision"]));
  1084. w.commit();
  1085. res = w.exec_params0("DELETE FROM ztc_network_assignment_pool WHERE network_id = $1", 0);
  1086. auto pool = config["ipAssignmentPools"];
  1087. bool err = false;
  1088. for (auto i = pool.begin(); i != pool.end(); ++i) {
  1089. std::string start = (*i)["ipRangeStart"];
  1090. std::string end = (*i)["ipRangeEnd"];
  1091. res = w.exec_params0(
  1092. "INSERT INTO ztc_network_assignment_pool (network_id, ip_range_start, ip_range_end) "
  1093. "VALUES ($1, $2, $3)",
  1094. id,
  1095. start,
  1096. end);
  1097. }
  1098. const uint64_t nwidInt = OSUtils::jsonIntHex(config["nwid"], 0ULL);
  1099. if (nwidInt) {
  1100. nlohmann::json nwOrig;
  1101. nlohmann::json nwNew(config);
  1102. get(nwidInt, nwOrig);
  1103. _networkChanged(nwOrig, nwNew, qitem.second);
  1104. }
  1105. else {
  1106. fprintf(stderr, "%s: Can't notify network changed: %llu\n", _myAddressStr.c_str(), (unsigned long long)nwidInt);
  1107. }
  1108. }
  1109. catch (std::exception& e) {
  1110. nspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1111. fprintf(stderr, "%s ERROR: Error updating network: %s\n", _myAddressStr.c_str(), e.what());
  1112. }
  1113. if (_listenerMode == LISTENER_MODE_REDIS && _redisMemberStatus) {
  1114. try {
  1115. std::string id = config["id"];
  1116. std::string controllerId = _myAddressStr.c_str();
  1117. std::string key = "networks:{" + controllerId + "}";
  1118. if (_rc->clusterMode) {
  1119. _cluster->sadd(key, id);
  1120. }
  1121. else {
  1122. _redis->sadd(key, id);
  1123. }
  1124. }
  1125. catch (sw::redis::Error& e) {
  1126. nspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1127. fprintf(stderr, "ERROR: Error adding network to Redis: %s\n", e.what());
  1128. }
  1129. }
  1130. }
  1131. else if (objtype == "_delete_network") {
  1132. auto dspan = tracer->StartSpan("CentralDB::commitThread::_delete_network");
  1133. auto dscope = tracer->WithActiveSpan(dspan);
  1134. // fprintf(stderr, "%s: commitThread: delete network\n", _myAddressStr.c_str());
  1135. try {
  1136. pqxx::work w(*c->c);
  1137. std::string networkId = config["id"];
  1138. fprintf(stderr, "Deleting network %s\n", networkId.c_str());
  1139. w.exec_params0("DELETE FROM network_memberships_ctl WHERE network_id = $1", networkId);
  1140. w.exec_params0("DELETE FROM networks_ctl WHERE id = $1", networkId);
  1141. w.commit();
  1142. uint64_t nwidInt = OSUtils::jsonIntHex(config["nwid"], 0ULL);
  1143. json oldConfig;
  1144. get(nwidInt, oldConfig);
  1145. json empty;
  1146. _networkChanged(oldConfig, empty, qitem.second);
  1147. }
  1148. catch (std::exception& e) {
  1149. dspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1150. fprintf(stderr, "%s ERROR: Error deleting network: %s\n", _myAddressStr.c_str(), e.what());
  1151. }
  1152. if (_listenerMode == LISTENER_MODE_REDIS && _redisMemberStatus) {
  1153. try {
  1154. std::string id = config["id"];
  1155. std::string controllerId = _myAddressStr.c_str();
  1156. std::string key = "networks:{" + controllerId + "}";
  1157. if (_rc->clusterMode) {
  1158. _cluster->srem(key, id);
  1159. _cluster->del("network-nodes-online:{" + controllerId + "}:" + id);
  1160. }
  1161. else {
  1162. _redis->srem(key, id);
  1163. _redis->del("network-nodes-online:{" + controllerId + "}:" + id);
  1164. }
  1165. }
  1166. catch (sw::redis::Error& e) {
  1167. dspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1168. fprintf(stderr, "ERROR: Error adding network to Redis: %s\n", e.what());
  1169. }
  1170. }
  1171. }
  1172. else if (objtype == "_delete_member") {
  1173. auto mspan = tracer->StartSpan("CentralDB::commitThread::_delete_member");
  1174. auto mscope = tracer->WithActiveSpan(mspan);
  1175. // fprintf(stderr, "%s commitThread: delete member\n", _myAddressStr.c_str());
  1176. try {
  1177. pqxx::work w(*c->c);
  1178. std::string memberId = config["id"];
  1179. std::string networkId = config["nwid"];
  1180. pqxx::result res = w.exec_params0("DELETE FROM network_memberships_ctl WHERE device_id = $1 AND network_id = $2", memberId, networkId);
  1181. w.commit();
  1182. uint64_t nwidInt = OSUtils::jsonIntHex(config["nwid"], 0ULL);
  1183. uint64_t memberidInt = OSUtils::jsonIntHex(config["id"], 0ULL);
  1184. nlohmann::json networkConfig;
  1185. nlohmann::json oldConfig;
  1186. get(nwidInt, networkConfig, memberidInt, oldConfig);
  1187. json empty;
  1188. _memberChanged(oldConfig, empty, qitem.second);
  1189. }
  1190. catch (std::exception& e) {
  1191. mspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1192. fprintf(stderr, "%s ERROR: Error deleting member: %s\n", _myAddressStr.c_str(), e.what());
  1193. }
  1194. if (_listenerMode == LISTENER_MODE_REDIS && _redisMemberStatus) {
  1195. try {
  1196. std::string memberId = config["id"];
  1197. std::string networkId = config["nwid"];
  1198. std::string controllerId = _myAddressStr.c_str();
  1199. std::string key = "network-nodes-all:{" + controllerId + "}:" + networkId;
  1200. if (_rc->clusterMode) {
  1201. _cluster->srem(key, memberId);
  1202. _cluster->del("member:{" + controllerId + "}:" + networkId + ":" + memberId);
  1203. }
  1204. else {
  1205. _redis->srem(key, memberId);
  1206. _redis->del("member:{" + controllerId + "}:" + networkId + ":" + memberId);
  1207. }
  1208. }
  1209. catch (sw::redis::Error& e) {
  1210. mspan->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1211. fprintf(stderr, "ERROR: Error deleting member from Redis: %s\n", e.what());
  1212. }
  1213. }
  1214. }
  1215. else {
  1216. fprintf(stderr, "%s ERROR: unknown objtype\n", _myAddressStr.c_str());
  1217. }
  1218. }
  1219. catch (std::exception& e) {
  1220. span->SetStatus(opentelemetry::trace::StatusCode::kError, e.what());
  1221. fprintf(stderr, "%s ERROR: Error getting objtype: %s\n", _myAddressStr.c_str(), e.what());
  1222. }
  1223. _pool->unborrow(c);
  1224. c.reset();
  1225. }
  1226. fprintf(stderr, "%s commitThread finished\n", _myAddressStr.c_str());
  1227. }
  1228. void CentralDB::notifyNewMember(const std::string& networkID, const std::string& memberID)
  1229. {
  1230. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  1231. auto tracer = provider->GetTracer("CentralDB");
  1232. auto span = tracer->StartSpan("CentralDB::notifyNewMember");
  1233. auto scope = tracer->WithActiveSpan(span);
  1234. rustybits::smee_client_notify_network_joined(_smee, networkID.c_str(), memberID.c_str());
  1235. }
  1236. void CentralDB::onlineNotificationThread()
  1237. {
  1238. waitForReady();
  1239. if (_redisMemberStatus) {
  1240. onlineNotification_Redis();
  1241. }
  1242. else {
  1243. onlineNotification_Postgres();
  1244. }
  1245. }
  1246. /**
  1247. * ONLY UNCOMMENT FOR TEMPORARY DB MAINTENANCE
  1248. *
  1249. * This define temporarily turns off writing to the member status table
  1250. * so it can be reindexed when the indexes get too large.
  1251. */
  1252. // #define DISABLE_MEMBER_STATUS 1
  1253. void CentralDB::onlineNotification_Postgres()
  1254. {
  1255. _connected = 1;
  1256. nlohmann::json jtmp1, jtmp2;
  1257. while (_run == 1) {
  1258. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  1259. auto tracer = provider->GetTracer("CentralDB");
  1260. auto span = tracer->StartSpan("CentralDB::onlineNotification_Postgres");
  1261. auto scope = tracer->WithActiveSpan(span);
  1262. auto c = _pool->borrow();
  1263. auto c2 = _pool->borrow();
  1264. try {
  1265. fprintf(stderr, "%s onlineNotification_Postgres\n", _myAddressStr.c_str());
  1266. std::unordered_map<std::pair<uint64_t, uint64_t>, NodeOnlineRecord, _PairHasher> lastOnline;
  1267. {
  1268. std::lock_guard<std::mutex> l(_lastOnline_l);
  1269. lastOnline.swap(_lastOnline);
  1270. }
  1271. #ifndef DISABLE_MEMBER_STATUS
  1272. pqxx::work w(*c->c);
  1273. pqxx::work w2(*c2->c);
  1274. fprintf(stderr, "online notification tick\n");
  1275. bool firstRun = true;
  1276. bool memberAdded = false;
  1277. int updateCount = 0;
  1278. pqxx::pipeline pipe(w);
  1279. for (auto i = lastOnline.begin(); i != lastOnline.end(); ++i) {
  1280. updateCount += 1;
  1281. uint64_t nwid_i = i->first.first;
  1282. char nwidTmp[64];
  1283. char memTmp[64];
  1284. char ipTmp[64];
  1285. OSUtils::ztsnprintf(nwidTmp, sizeof(nwidTmp), "%.16llx", nwid_i);
  1286. OSUtils::ztsnprintf(memTmp, sizeof(memTmp), "%.10llx", i->first.second);
  1287. if (! get(nwid_i, jtmp1, i->first.second, jtmp2)) {
  1288. continue; // skip non existent networks/members
  1289. }
  1290. std::string networkId(nwidTmp);
  1291. std::string memberId(memTmp);
  1292. try {
  1293. pqxx::row r = w2.exec_params1("SELECT id, network_id FROM ztc_member WHERE network_id = $1 AND id = $2", networkId, memberId);
  1294. }
  1295. catch (pqxx::unexpected_rows& e) {
  1296. continue;
  1297. }
  1298. int64_t ts = i->second.lastSeen;
  1299. std::string ipAddr = i->second.physicalAddress.toIpString(ipTmp);
  1300. std::string timestamp = std::to_string(ts);
  1301. std::string osArch = i->second.osArch;
  1302. std::stringstream memberUpdate;
  1303. memberUpdate << "INSERT INTO ztc_member_status (network_id, member_id, address, last_updated) VALUES "
  1304. << "('" << networkId << "', '" << memberId << "', ";
  1305. if (ipAddr.empty()) {
  1306. memberUpdate << "NULL, ";
  1307. }
  1308. else {
  1309. memberUpdate << "'" << ipAddr << "', ";
  1310. }
  1311. memberUpdate << "TO_TIMESTAMP(" << timestamp << "::double precision/1000)) "
  1312. << " ON CONFLICT (network_id, member_id) DO UPDATE SET address = EXCLUDED.address, last_updated = EXCLUDED.last_updated";
  1313. pipe.insert(memberUpdate.str());
  1314. Metrics::pgsql_node_checkin++;
  1315. }
  1316. while (! pipe.empty()) {
  1317. pipe.retrieve();
  1318. }
  1319. pipe.complete();
  1320. w.commit();
  1321. fprintf(stderr, "%s: Updated online status of %d members\n", _myAddressStr.c_str(), updateCount);
  1322. #endif
  1323. }
  1324. catch (std::exception& e) {
  1325. fprintf(stderr, "%s: error in onlinenotification thread: %s\n", _myAddressStr.c_str(), e.what());
  1326. }
  1327. _pool->unborrow(c2);
  1328. _pool->unborrow(c);
  1329. ConnectionPoolStats stats = _pool->get_stats();
  1330. fprintf(stderr, "%s pool stats: in use size: %llu, available size: %llu, total: %llu\n", _myAddressStr.c_str(), stats.borrowed_size, stats.pool_size, (stats.borrowed_size + stats.pool_size));
  1331. span->End();
  1332. std::this_thread::sleep_for(std::chrono::seconds(10));
  1333. }
  1334. fprintf(stderr, "%s: Fell out of run loop in onlineNotificationThread\n", _myAddressStr.c_str());
  1335. if (_run == 1) {
  1336. fprintf(stderr, "ERROR: %s onlineNotificationThread should still be running! Exiting Controller.\n", _myAddressStr.c_str());
  1337. exit(6);
  1338. }
  1339. }
  1340. void CentralDB::onlineNotification_Redis()
  1341. {
  1342. _connected = 1;
  1343. char buf[11] = { 0 };
  1344. std::string controllerId = std::string(_myAddress.toString(buf));
  1345. while (_run == 1) {
  1346. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  1347. auto tracer = provider->GetTracer("CentralDB");
  1348. auto span = tracer->StartSpan("CentralDB::onlineNotification_Redis");
  1349. auto scope = tracer->WithActiveSpan(span);
  1350. fprintf(stderr, "onlineNotification tick\n");
  1351. auto start = std::chrono::high_resolution_clock::now();
  1352. uint64_t count = 0;
  1353. std::unordered_map<std::pair<uint64_t, uint64_t>, NodeOnlineRecord, _PairHasher> lastOnline;
  1354. {
  1355. std::lock_guard<std::mutex> l(_lastOnline_l);
  1356. lastOnline.swap(_lastOnline);
  1357. }
  1358. try {
  1359. if (! lastOnline.empty()) {
  1360. if (_rc->clusterMode) {
  1361. auto tx = _cluster->transaction(controllerId, true, false);
  1362. count = _doRedisUpdate(tx, controllerId, lastOnline);
  1363. }
  1364. else {
  1365. auto tx = _redis->transaction(true, false);
  1366. count = _doRedisUpdate(tx, controllerId, lastOnline);
  1367. }
  1368. }
  1369. }
  1370. catch (sw::redis::Error& e) {
  1371. fprintf(stderr, "Error in online notification thread (redis): %s\n", e.what());
  1372. }
  1373. auto end = std::chrono::high_resolution_clock::now();
  1374. auto dur = std::chrono::duration_cast<std::chrono::milliseconds>(end - start);
  1375. auto total = dur.count();
  1376. fprintf(stderr, "onlineNotification ran in %llu ms\n", total);
  1377. span->End();
  1378. std::this_thread::sleep_for(std::chrono::seconds(5));
  1379. }
  1380. }
  1381. uint64_t CentralDB::_doRedisUpdate(sw::redis::Transaction& tx, std::string& controllerId, std::unordered_map<std::pair<uint64_t, uint64_t>, NodeOnlineRecord, _PairHasher>& lastOnline)
  1382. {
  1383. auto provider = opentelemetry::trace::Provider::GetTracerProvider();
  1384. auto tracer = provider->GetTracer("CentralDB");
  1385. auto span = tracer->StartSpan("CentralDB::_doRedisUpdate");
  1386. auto scope = tracer->WithActiveSpan(span);
  1387. nlohmann::json jtmp1, jtmp2;
  1388. uint64_t count = 0;
  1389. for (auto i = lastOnline.begin(); i != lastOnline.end(); ++i) {
  1390. uint64_t nwid_i = i->first.first;
  1391. uint64_t memberid_i = i->first.second;
  1392. char nwidTmp[64];
  1393. char memTmp[64];
  1394. char ipTmp[64];
  1395. OSUtils::ztsnprintf(nwidTmp, sizeof(nwidTmp), "%.16llx", nwid_i);
  1396. OSUtils::ztsnprintf(memTmp, sizeof(memTmp), "%.10llx", memberid_i);
  1397. if (! get(nwid_i, jtmp1, memberid_i, jtmp2)) {
  1398. continue; // skip non existent members/networks
  1399. }
  1400. std::string networkId(nwidTmp);
  1401. std::string memberId(memTmp);
  1402. int64_t ts = i->second.lastSeen;
  1403. std::string ipAddr = i->second.physicalAddress.toIpString(ipTmp);
  1404. std::string timestamp = std::to_string(ts);
  1405. std::string osArch = i->second.osArch;
  1406. std::unordered_map<std::string, std::string> record = { { "id", memberId }, { "address", ipAddr }, { "last_updated", std::to_string(ts) } };
  1407. tx.zadd("nodes-online:{" + controllerId + "}", memberId, ts)
  1408. .zadd("nodes-online2:{" + controllerId + "}", networkId + "-" + memberId, ts)
  1409. .zadd("network-nodes-online:{" + controllerId + "}:" + networkId, memberId, ts)
  1410. .zadd("active-networks:{" + controllerId + "}", networkId, ts)
  1411. .sadd("network-nodes-all:{" + controllerId + "}:" + networkId, memberId)
  1412. .hmset("member:{" + controllerId + "}:" + networkId + ":" + memberId, record.begin(), record.end());
  1413. ++count;
  1414. Metrics::redis_node_checkin++;
  1415. }
  1416. // expire records from all-nodes and network-nodes member list
  1417. uint64_t expireOld = OSUtils::now() - 300000;
  1418. tx.zremrangebyscore("nodes-online:{" + controllerId + "}", sw::redis::RightBoundedInterval<double>(expireOld, sw::redis::BoundType::LEFT_OPEN));
  1419. tx.zremrangebyscore("nodes-online2:{" + controllerId + "}", sw::redis::RightBoundedInterval<double>(expireOld, sw::redis::BoundType::LEFT_OPEN));
  1420. tx.zremrangebyscore("active-networks:{" + controllerId + "}", sw::redis::RightBoundedInterval<double>(expireOld, sw::redis::BoundType::LEFT_OPEN));
  1421. {
  1422. std::shared_lock<std::shared_mutex> l(_networks_l);
  1423. for (const auto& it : _networks) {
  1424. uint64_t nwid_i = it.first;
  1425. char nwidTmp[64];
  1426. OSUtils::ztsnprintf(nwidTmp, sizeof(nwidTmp), "%.16llx", nwid_i);
  1427. tx.zremrangebyscore("network-nodes-online:{" + controllerId + "}:" + nwidTmp, sw::redis::RightBoundedInterval<double>(expireOld, sw::redis::BoundType::LEFT_OPEN));
  1428. }
  1429. }
  1430. tx.exec();
  1431. fprintf(stderr, "%s: Updated online status of %d members\n", _myAddressStr.c_str(), count);
  1432. return count;
  1433. }
  1434. #endif // ZT_CONTROLLER_USE_LIBPQ