summaryrefslogtreecommitdiffstats
path: root/database/sqlite
diff options
context:
space:
mode:
Diffstat (limited to 'database/sqlite')
-rw-r--r--database/sqlite/sqlite_aclk.c15
-rw-r--r--database/sqlite/sqlite_aclk.h2
-rw-r--r--database/sqlite/sqlite_aclk_alert.c128
-rw-r--r--database/sqlite/sqlite_aclk_alert.h10
-rw-r--r--database/sqlite/sqlite_aclk_chart.c601
-rw-r--r--database/sqlite/sqlite_aclk_chart.h27
-rw-r--r--database/sqlite/sqlite_aclk_node.c30
-rw-r--r--database/sqlite/sqlite_functions.c167
-rw-r--r--database/sqlite/sqlite_functions.h2
-rw-r--r--database/sqlite/sqlite_health.c7
-rw-r--r--database/sqlite/sqlite_health.h2
11 files changed, 740 insertions, 251 deletions
diff --git a/database/sqlite/sqlite_aclk.c b/database/sqlite/sqlite_aclk.c
index 63196a81e..989328097 100644
--- a/database/sqlite/sqlite_aclk.c
+++ b/database/sqlite/sqlite_aclk.c
@@ -209,7 +209,7 @@ struct aclk_database_worker_config *find_inactive_wc_by_node_id(char *node_id)
void aclk_sync_exit_all()
{
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *host = localhost;
while(host) {
struct aclk_database_worker_config *wc = host->dbsync_worker;
@@ -508,11 +508,16 @@ void aclk_database_worker(void *arg)
aclk_update_retention(wc, cmd);
aclk_process_dimension_deletion(wc, cmd);
break;
-#endif
// NODE_INSTANCE DETECTION
+ case ACLK_DATABASE_ORPHAN_HOST:
+ wc->host = NULL;
+ wc->is_orphan = 1;
+ aclk_add_worker_thread(wc);
+ break;
+#endif
case ACLK_DATABASE_TIMER:
- if (unlikely(localhost && !wc->host)) {
+ if (unlikely(localhost && !wc->host && !wc->is_orphan)) {
if (claimed()) {
wc->host = rrdhost_find_by_guid(wc->host_guid, 0);
if (wc->host) {
@@ -567,7 +572,7 @@ void aclk_database_worker(void *arg)
freez(loop);
- rrd_wrlock();
+ rrd_rdlock();
if (likely(wc->host))
wc->host->dbsync_worker = NULL;
freez(wc);
@@ -815,7 +820,7 @@ void aclk_data_rotated(void)
return;
time_t next_rotation_time = now_realtime_sec()+ACLK_DATABASE_ROTATION_DELAY;
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *this_host = localhost;
while (this_host) {
struct aclk_database_worker_config *wc = this_host->dbsync_worker;
diff --git a/database/sqlite/sqlite_aclk.h b/database/sqlite/sqlite_aclk.h
index 424949740..894d93489 100644
--- a/database/sqlite/sqlite_aclk.h
+++ b/database/sqlite/sqlite_aclk.h
@@ -125,6 +125,7 @@ enum aclk_database_opcode {
ACLK_DATABASE_CHART_ACK,
ACLK_DATABASE_UPD_RETENTION,
ACLK_DATABASE_DIM_DELETION,
+ ACLK_DATABASE_ORPHAN_HOST,
#endif
ACLK_DATABASE_ALARM_HEALTH_LOG,
ACLK_DATABASE_CLEANUP,
@@ -194,6 +195,7 @@ struct aclk_database_worker_config {
int chart_pending;
int chart_reset_count;
volatile unsigned is_shutting_down;
+ volatile unsigned is_orphan;
struct aclk_database_worker_config *next;
};
diff --git a/database/sqlite/sqlite_aclk_alert.c b/database/sqlite/sqlite_aclk_alert.c
index 238b500a8..54e8be4a7 100644
--- a/database/sqlite/sqlite_aclk_alert.c
+++ b/database/sqlite/sqlite_aclk_alert.c
@@ -127,7 +127,7 @@ void aclk_push_alert_event(struct aclk_database_worker_config *wc, struct aclk_d
int rc;
if (unlikely(!wc->alert_updates)) {
- log_access("AC [%s (%s)]: Ignoring alert push event, updates have been turned off for this node.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
+ log_access("ACLK STA [%s (%s)]: Ignoring alert push event, updates have been turned off for this node.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
return;
}
@@ -135,6 +135,11 @@ void aclk_push_alert_event(struct aclk_database_worker_config *wc, struct aclk_d
if (unlikely(!claim_id))
return;
+ if (unlikely(!wc->host)) {
+ freez(claim_id);
+ return;
+ }
+
BUFFER *sql = buffer_create(1024);
if (wc->alerts_start_seq_id != 0) {
@@ -242,7 +247,9 @@ void aclk_push_alert_event(struct aclk_database_worker_config *wc, struct aclk_d
alarm_log.old_value = (calculated_number) sqlite3_column_double(res, 24);
alarm_log.updated = (sqlite3_column_int64(res, 8) & HEALTH_ENTRY_FLAG_UPDATED) ? 1 : 0;
- alarm_log.rendered_info = strdupz((char *)sqlite3_column_text(res, 18));
+ alarm_log.rendered_info = sqlite3_column_type(res, 18) == SQLITE_NULL ?
+ strdupz((char *)"") :
+ strdupz((char *)sqlite3_column_text(res, 18));
aclk_send_alarm_log_entry(&alarm_log);
@@ -267,7 +274,13 @@ void aclk_push_alert_event(struct aclk_database_worker_config *wc, struct aclk_d
db_execute(buffer_tostring(sql));
} else {
if (log_first_sequence_id)
- log_access("OG [%s (%s)]: Sent alert events, first sequence_id %"PRIu64", last sequence_id %"PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", log_first_sequence_id, log_last_sequence_id);
+ log_access(
+ "ACLK RES [%s (%s)]: ALERTS SENT from %" PRIu64 " to %" PRIu64 " batch=%" PRIu64,
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ log_first_sequence_id,
+ log_last_sequence_id,
+ wc->alerts_batch_id);
log_first_sequence_id = 0;
log_last_sequence_id = 0;
}
@@ -288,23 +301,32 @@ void aclk_send_alarm_health_log(char *node_id)
if (unlikely(!node_id))
return;
- log_access("IN [%s (N/A)]: Request to send alarm health log.", node_id);
+ char *hostname= NULL;
struct aclk_database_worker_config *wc = NULL;
struct aclk_database_cmd cmd;
memset(&cmd, 0, sizeof(cmd));
cmd.opcode = ACLK_DATABASE_ALARM_HEALTH_LOG;
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *host = find_host_by_node_id(node_id);
- if (likely(host))
+ if (likely(host)) {
wc = (struct aclk_database_worker_config *)host->dbsync_worker;
+ hostname = host->hostname;
+ }
+ else
+ hostname = get_hostname_by_node_id(node_id);
rrd_unlock();
+
+ log_access("ACLK REQ [%s (%s)]: HEALTH LOG request received", node_id, hostname ? hostname : "N/A");
+ if (unlikely(!host))
+ freez(hostname);
+
if (wc)
aclk_database_enq_cmd(wc, &cmd);
else {
if (aclk_worker_enq_cmd(node_id, &cmd))
- log_access("AC [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
+ log_access("ACLK STA [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
}
return;
}
@@ -323,7 +345,7 @@ void aclk_push_alarm_health_log(struct aclk_database_worker_config *wc, struct a
RRDHOST *host = wc->host;
if (unlikely(!host)) {
- rrd_wrlock();
+ rrd_rdlock();
host = find_host_by_node_id(wc->node_id);
rrd_unlock();
@@ -391,7 +413,7 @@ void aclk_push_alarm_health_log(struct aclk_database_worker_config *wc, struct a
wc->alert_sequence_id = last_sequence;
aclk_send_alarm_log_health(&alarm_log);
- log_access("OG [%s (%s)]: Alarm health log sent, first sequence id %"PRIu64", last sequence id %"PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", first_sequence, last_sequence);
+ log_access("ACLK RES [%s (%s)]: HEALTH LOG SENT from %"PRIu64" to %"PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", first_sequence, last_sequence);
rc = sqlite3_finalize(res);
if (unlikely(rc != SQLITE_OK))
@@ -415,7 +437,7 @@ void aclk_send_alarm_configuration(char *config_hash)
return;
}
- log_access("IN [%s (%s)]: Request to send alert config %s.", wc->node_id, wc->host ? wc->host->hostname : "N/A", config_hash);
+ log_access("ACLK REQ [%s (%s)]: Request to send alert config %s.", wc->node_id, wc->host ? wc->host->hostname : "N/A", config_hash);
struct aclk_database_cmd cmd;
memset(&cmd, 0, sizeof(cmd));
@@ -525,14 +547,14 @@ int aclk_push_alert_config_event(struct aclk_database_worker_config *wc, struct
}
if (likely(p_alarm_config.cfg_hash)) {
- log_access("OG [%s (%s)]: Sent alert config %s.", wc->node_id, wc->host ? wc->host->hostname : "N/A", config_hash);
+ log_access("ACLK RES [%s (%s)]: Sent alert config %s.", wc->node_id, wc->host ? wc->host->hostname : "N/A", config_hash);
aclk_send_provide_alarm_cfg(&p_alarm_config);
freez((char *) cmd.data_param);
freez(p_alarm_config.cfg_hash);
destroy_aclk_alarm_configuration(&alarm_config);
}
else
- log_access("AC [%s (%s)]: Alert config for %s not found.", wc->node_id, wc->host ? wc->host->hostname : "N/A", config_hash);
+ log_access("ACLK STA [%s (%s)]: Alert config for %s not found.", wc->node_id, wc->host ? wc->host->hostname : "N/A", config_hash);
bind_fail:
rc = sqlite3_finalize(res);
@@ -552,28 +574,30 @@ void aclk_start_alert_streaming(char *node_id, uint64_t batch_id, uint64_t start
if (unlikely(!node_id))
return;
- log_access("IN [%s (N/A)]: Start streaming alerts with batch_id %"PRIu64" and start_seq_id %"PRIu64".", node_id, batch_id, start_seq_id);
+ //log_access("ACLK REQ [%s (N/A)]: ALERTS STREAM from %"PRIu64" batch=%"PRIu64".", node_id, start_seq_id, batch_id);
uuid_t node_uuid;
if (uuid_parse(node_id, node_uuid))
return;
struct aclk_database_worker_config *wc = NULL;
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *host = find_host_by_node_id(node_id);
- if (likely(host))
+ rrd_unlock();
+ if (likely(host)) {
wc = (struct aclk_database_worker_config *)host->dbsync_worker ?
(struct aclk_database_worker_config *)host->dbsync_worker :
(struct aclk_database_worker_config *)find_inactive_wc_by_node_id(node_id);
- rrd_unlock();
- if (unlikely(!host->health_enabled)) {
- log_access("AC [%s (N/A)]: Ignoring request to stream alert state changes, health is disabled.", node_id);
- return;
- }
+ if (unlikely(!host->health_enabled)) {
+ log_access("ACLK STA [%s (N/A)]: Ignoring request to stream alert state changes, health is disabled.", node_id);
+ return;
+ }
+ } else
+ wc = (struct aclk_database_worker_config *)find_inactive_wc_by_node_id(node_id);
if (likely(wc)) {
- log_access("AC [%s (%s)]: Start streaming alerts enabled with batch_id %"PRIu64" and start_seq_id %"PRIu64".", node_id, wc->host ? wc->host->hostname : "N/A", batch_id, start_seq_id);
+ log_access("ACLK REQ [%s (%s)]: ALERTS STREAM from %"PRIu64" batch=%"PRIu64, node_id, wc->host ? wc->host->hostname : "N/A", start_seq_id, batch_id);
__sync_synchronize();
wc->alerts_batch_id = batch_id;
wc->alerts_start_seq_id = start_seq_id;
@@ -581,7 +605,7 @@ void aclk_start_alert_streaming(char *node_id, uint64_t batch_id, uint64_t start
__sync_synchronize();
}
else
- log_access("AC [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
+ log_access("ACLK STA [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
#else
UNUSED(node_id);
@@ -607,7 +631,7 @@ void sql_process_queue_removed_alerts_to_aclk(struct aclk_database_worker_config
db_execute(buffer_tostring(sql));
- log_access("AC [%s (%s)]: Queued removed alerts.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
+ log_access("ACLK STA [%s (%s)]: Queued removed alerts.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
buffer_free(sql);
#endif
@@ -644,7 +668,7 @@ void aclk_process_send_alarm_snapshot(char *node_id, char *claim_id, uint64_t sn
return;
struct aclk_database_worker_config *wc = NULL;
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *host = find_host_by_node_id(node_id);
if (likely(host))
wc = (struct aclk_database_worker_config *)host->dbsync_worker;
@@ -657,6 +681,8 @@ void aclk_process_send_alarm_snapshot(char *node_id, char *claim_id, uint64_t sn
wc->host ? wc->host->hostname : "N/A",
snapshot_id,
sequence_id);
+ if (wc->alerts_snapshot_id == snapshot_id)
+ return;
__sync_synchronize();
wc->alerts_snapshot_id = snapshot_id;
wc->alerts_ack_sequence_id = sequence_id;
@@ -669,7 +695,7 @@ void aclk_process_send_alarm_snapshot(char *node_id, char *claim_id, uint64_t sn
cmd.completion = NULL;
aclk_database_enq_cmd(wc, &cmd);
} else
- log_access("AC [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
+ log_access("ACLK STA [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
#else
UNUSED(node_id);
UNUSED(snapshot_id);
@@ -714,7 +740,7 @@ void health_alarm_entry2proto_nolock(struct alarm_log_entry *alarm_log, ALARM_EN
alarm_log->utc_offset = host->utc_offset;
alarm_log->timezone = strdupz((char *)host->abbrev_timezone);
alarm_log->exec_path = ae->exec ? strdupz((char *)ae->exec) : strdupz((char *)host->health_default_exec);
- alarm_log->conf_source = ae->source ? strdupz((char *)ae->source) : "";
+ alarm_log->conf_source = ae->source ? strdupz((char *)ae->source) : strdupz((char *)"");
alarm_log->command = strdupz((char *)edit_command);
@@ -738,7 +764,7 @@ void health_alarm_entry2proto_nolock(struct alarm_log_entry *alarm_log, ALARM_EN
alarm_log->old_value = (!isnan(ae->old_value)) ? (calculated_number)ae->old_value : 0;
alarm_log->updated = (ae->flags & HEALTH_ENTRY_FLAG_UPDATED) ? 1 : 0;
- alarm_log->rendered_info = strdupz(ae->info);
+ alarm_log->rendered_info = ae->info ? strdupz(ae->info) : strdupz((char *)"");
freez(edit_command);
}
@@ -770,7 +796,7 @@ void aclk_push_alert_snapshot_event(struct aclk_database_worker_config *wc, stru
UNUSED(cmd);
// we perhaps we don't need this for snapshots
if (unlikely(!wc->alert_updates)) {
- log_access("AC [%s (%s)]: Ignoring alert snapshot event, updates have been turned off for this node.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
+ log_access("ACLK STA [%s (%s)]: Ignoring alert snapshot event, updates have been turned off for this node.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
return;
}
@@ -779,11 +805,14 @@ void aclk_push_alert_snapshot_event(struct aclk_database_worker_config *wc, stru
return;
}
+ if (unlikely(!wc->alerts_snapshot_id))
+ return;
+
char *claim_id = is_agent_claimed();
if (unlikely(!claim_id))
return;
- log_access("OG [%s (%s)]: Sending alerts snapshot, snapshot_id %" PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", wc->alerts_snapshot_id);
+ log_access("ACLK REQ [%s (%s)]: Sending alerts snapshot, snapshot_id %" PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", wc->alerts_snapshot_id);
aclk_mark_alert_cloud_ack(wc->uuid_str, wc->alerts_ack_sequence_id);
@@ -906,3 +935,44 @@ void sql_aclk_alert_clean_dead_entries(RRDHOST *host)
UNUSED(host);
#endif
}
+
+int get_proto_alert_status(RRDHOST *host, struct proto_alert_status *proto_alert_status)
+{
+ int rc;
+ struct aclk_database_worker_config *wc = NULL;
+ wc = (struct aclk_database_worker_config *)host->dbsync_worker;
+ if (!wc)
+ return 1;
+
+ proto_alert_status->alert_updates = wc->alert_updates;
+ proto_alert_status->alerts_batch_id = wc->alerts_batch_id;
+
+ BUFFER *sql = buffer_create(1024);
+ sqlite3_stmt *res = NULL;
+
+ buffer_sprintf(sql, "SELECT MIN(sequence_id), MAX(sequence_id), " \
+ "(select MAX(sequence_id) from aclk_alert_%s where date_cloud_ack is not NULL), " \
+ "(select MAX(sequence_id) from aclk_alert_%s where date_submitted is not NULL) " \
+ "FROM aclk_alert_%s where date_submitted is null;", wc->uuid_str, wc->uuid_str, wc->uuid_str);
+
+ rc = sqlite3_prepare_v2(db_meta, buffer_tostring(sql), -1, &res, 0);
+ if (rc != SQLITE_OK) {
+ error_report("Failed to prepare statement to get alert log status from the database.");
+ buffer_free(sql);
+ return 1;
+ }
+
+ while (sqlite3_step(res) == SQLITE_ROW) {
+ proto_alert_status->pending_min_sequence_id = sqlite3_column_bytes(res, 0) > 0 ? (uint64_t) sqlite3_column_int64(res, 0) : 0;
+ proto_alert_status->pending_max_sequence_id = sqlite3_column_bytes(res, 1) > 0 ? (uint64_t) sqlite3_column_int64(res, 1) : 0;
+ proto_alert_status->last_acked_sequence_id = sqlite3_column_bytes(res, 2) > 0 ? (uint64_t) sqlite3_column_int64(res, 2) : 0;
+ proto_alert_status->last_submitted_sequence_id = sqlite3_column_bytes(res, 3) > 0 ? (uint64_t) sqlite3_column_int64(res, 3) : 0;
+ }
+
+ rc = sqlite3_finalize(res);
+ if (unlikely(rc != SQLITE_OK))
+ error_report("Failed to finalize statement to get alert log status from the database, rc = %d", rc);
+
+ buffer_free(sql);
+ return 0;
+}
diff --git a/database/sqlite/sqlite_aclk_alert.h b/database/sqlite/sqlite_aclk_alert.h
index 1aaaa5d23..957cb94ac 100644
--- a/database/sqlite/sqlite_aclk_alert.h
+++ b/database/sqlite/sqlite_aclk_alert.h
@@ -5,6 +5,15 @@
extern sqlite3 *db_meta;
+struct proto_alert_status {
+ int alert_updates;
+ uint64_t alerts_batch_id;
+ uint64_t last_acked_sequence_id;
+ uint64_t pending_min_sequence_id;
+ uint64_t pending_max_sequence_id;
+ uint64_t last_submitted_sequence_id;
+};
+
int aclk_add_alert_event(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd);
void aclk_push_alert_event(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd);
void aclk_send_alarm_health_log(char *node_id);
@@ -16,5 +25,6 @@ void sql_queue_removed_alerts_to_aclk(RRDHOST *host);
void sql_process_queue_removed_alerts_to_aclk(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd);
void aclk_push_alert_snapshot_event(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd);
void aclk_process_send_alarm_snapshot(char *node_id, char *claim_id, uint64_t snapshot_id, uint64_t sequence_id);
+int get_proto_alert_status(RRDHOST *host, struct proto_alert_status *proto_alert_status);
#endif //NETDATA_SQLITE_ACLK_ALERT_H
diff --git a/database/sqlite/sqlite_aclk_chart.c b/database/sqlite/sqlite_aclk_chart.c
index eea48a567..7afa1d451 100644
--- a/database/sqlite/sqlite_aclk_chart.c
+++ b/database/sqlite/sqlite_aclk_chart.c
@@ -3,12 +3,12 @@
#include "sqlite_functions.h"
#include "sqlite_aclk_chart.h"
-#ifdef ENABLE_NEW_CLOUD_PROTOCOL
+#if defined(ENABLE_ACLK) && defined(ENABLE_NEW_CLOUD_PROTOCOL)
#include "../../aclk/aclk_charts_api.h"
#include "../../aclk/aclk.h"
-static inline int sql_queue_chart_payload(struct aclk_database_worker_config *wc,
- void *data, enum aclk_database_opcode opcode)
+static inline int
+sql_queue_chart_payload(struct aclk_database_worker_config *wc, void *data, enum aclk_database_opcode opcode)
{
int rc;
if (unlikely(!wc))
@@ -29,23 +29,22 @@ static int payload_sent(char *uuid_str, uuid_t *uuid, void *payload, size_t payl
int send_status = 0;
if (unlikely(!res)) {
- BUFFER *sql = buffer_create(1024);
- buffer_sprintf(sql,"SELECT 1 FROM aclk_chart_latest_%s acl, aclk_chart_payload_%s acp "
+ char sql[ACLK_SYNC_QUERY_SIZE];
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1, "SELECT 1 FROM aclk_chart_latest_%s acl, aclk_chart_payload_%s acp "
"WHERE acl.unique_id = acp.unique_id AND acl.uuid = @uuid AND acp.payload = @payload;",
uuid_str, uuid_str);
- rc = prepare_statement(db_meta, (char *) buffer_tostring(sql), &res);
- buffer_free(sql);
+ rc = prepare_statement(db_meta, sql, &res);
if (rc != SQLITE_OK) {
error_report("Failed to prepare statement to check payload data");
return 0;
}
}
- rc = sqlite3_bind_blob(res, 1, uuid , sizeof(*uuid), SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res, 1, uuid, sizeof(*uuid), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
- rc = sqlite3_bind_blob(res, 2, payload , payload_size, SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res, 2, payload, payload_size, SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
@@ -60,24 +59,23 @@ bind_fail:
}
static int aclk_add_chart_payload(struct aclk_database_worker_config *wc, uuid_t *uuid, char *claim_id,
- ACLK_PAYLOAD_TYPE payload_type, void *payload, size_t payload_size)
+ ACLK_PAYLOAD_TYPE payload_type, void *payload, size_t payload_size, int *send_status)
{
static __thread sqlite3_stmt *res_chart = NULL;
int rc;
rc = payload_sent(wc->uuid_str, uuid, payload, payload_size);
+ if (send_status)
+ *send_status = rc;
if (rc == 1)
return 0;
if (unlikely(!res_chart)) {
- BUFFER *sql = buffer_create(1024);
-
- buffer_sprintf(sql,"INSERT INTO aclk_chart_payload_%s (unique_id, uuid, claim_id, date_created, type, payload) " \
- "VALUES (@unique_id, @uuid, @claim_id, strftime('%%s','now'), @type, @payload);", wc->uuid_str);
-
- rc = prepare_statement(db_meta, (char *) buffer_tostring(sql), &res_chart);
- buffer_free(sql);
-
+ char sql[ACLK_SYNC_QUERY_SIZE];
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1,
+ "INSERT INTO aclk_chart_payload_%s (unique_id, uuid, claim_id, date_created, type, payload) " \
+ "VALUES (@unique_id, @uuid, @claim_id, strftime('%%s','now'), @type, @payload);", wc->uuid_str);
+ rc = prepare_statement(db_meta, sql, &res_chart);
if (rc != SQLITE_OK) {
error_report("Failed to prepare statement to store chart payload data");
return 1;
@@ -91,15 +89,15 @@ static int aclk_add_chart_payload(struct aclk_database_worker_config *wc, uuid_t
if (uuid_parse(claim_id, claim_uuid))
return 1;
- rc = sqlite3_bind_blob(res_chart, 1, &unique_uuid , sizeof(unique_uuid), SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res_chart, 1, &unique_uuid, sizeof(unique_uuid), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
- rc = sqlite3_bind_blob(res_chart, 2, uuid , sizeof(*uuid), SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res_chart, 2, uuid, sizeof(*uuid), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
- rc = sqlite3_bind_blob(res_chart, 3, &claim_uuid , sizeof(claim_uuid), SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res_chart, 3, &claim_uuid, sizeof(claim_uuid), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
@@ -127,7 +125,6 @@ bind_fail:
return (rc != SQLITE_DONE);
}
-
int aclk_add_chart_event(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd)
{
int rc = 0;
@@ -162,7 +159,7 @@ int aclk_add_chart_event(struct aclk_database_worker_config *wc, struct aclk_dat
size_t size;
char *payload = generate_chart_instance_updated(&size, &chart_payload);
if (likely(payload))
- rc = aclk_add_chart_payload(wc, st->chart_uuid, claim_id, ACLK_PAYLOAD_CHART, (void *) payload, size);
+ rc = aclk_add_chart_payload(wc, st->chart_uuid, claim_id, ACLK_PAYLOAD_CHART, (void *) payload, size, NULL);
freez(payload);
chart_instance_updated_destroy(&chart_payload);
}
@@ -170,7 +167,8 @@ int aclk_add_chart_event(struct aclk_database_worker_config *wc, struct aclk_dat
}
static inline int aclk_upd_dimension_event(struct aclk_database_worker_config *wc, char *claim_id, uuid_t *dim_uuid,
- const char *dim_id, const char *dim_name, const char *chart_type_id, time_t first_time, time_t last_time)
+ const char *dim_id, const char *dim_name, const char *chart_type_id, time_t first_time, time_t last_time,
+ int *send_status)
{
int rc = 0;
size_t size;
@@ -185,6 +183,9 @@ static inline int aclk_upd_dimension_event(struct aclk_database_worker_config *w
if (!first_time)
info("Host %s (node %s) deleting dimension id=[%s] name=[%s] chart=[%s]",
wc->host_guid, wc->node_id, dim_id, dim_name, chart_type_id);
+ if (last_time)
+ info("Host %s (node %s) stopped collecting dimension id=[%s] name=[%s] chart=[%s] %ld seconds ago at %ld",
+ wc->host_guid, wc->node_id, dim_id, dim_name, chart_type_id, now_realtime_sec() - last_time, last_time);
#endif
dim_payload.node_id = wc->node_id;
@@ -196,7 +197,7 @@ static inline int aclk_upd_dimension_event(struct aclk_database_worker_config *w
dim_payload.last_timestamp.tv_sec = last_time;
char *payload = generate_chart_dimension_updated(&size, &dim_payload);
if (likely(payload))
- rc = aclk_add_chart_payload(wc, dim_uuid, claim_id, ACLK_PAYLOAD_DIMENSION, (void *)payload, size);
+ rc = aclk_add_chart_payload(wc, dim_uuid, claim_id, ACLK_PAYLOAD_DIMENSION, (void *)payload, size, send_status);
freez(payload);
return rc;
}
@@ -220,8 +221,13 @@ void aclk_process_dimension_deletion(struct aclk_database_worker_config *wc, str
if (!claim_id)
return;
- rc = sqlite3_prepare_v2(db_meta, "DELETE FROM dimension_delete where host_id = @host_id " \
- "RETURNING dimension_id, dimension_name, chart_type_id, dim_id LIMIT 10;", -1, &res, 0);
+ rc = sqlite3_prepare_v2(
+ db_meta,
+ "DELETE FROM dimension_delete where host_id = @host_id "
+ "RETURNING dimension_id, dimension_name, chart_type_id, dim_id LIMIT 10;",
+ -1,
+ &res,
+ 0);
if (rc != SQLITE_OK) {
error_report("Failed to prepare statement when trying to delete dimension deletes");
@@ -229,7 +235,7 @@ void aclk_process_dimension_deletion(struct aclk_database_worker_config *wc, str
return;
}
- rc = sqlite3_bind_blob(res, 1, &host_id , sizeof(host_id), SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res, 1, &host_id, sizeof(host_id), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
@@ -243,7 +249,8 @@ void aclk_process_dimension_deletion(struct aclk_database_worker_config *wc, str
(const char *)sqlite3_column_text(res, 1),
(const char *)sqlite3_column_text(res, 2),
0,
- 0);
+ 0,
+ NULL);
count++;
}
@@ -272,12 +279,13 @@ int aclk_add_dimension_event(struct aclk_database_worker_config *wc, struct aclk
RRDDIM *rd = cmd.data;
if (likely(claim_id)) {
+ int send_status = 0;
time_t now = now_realtime_sec();
time_t first_t = rd->state->query_ops.oldest_time(rd);
- time_t last_t = rd->state->query_ops.latest_time(rd);
+ time_t last_t = rd->state->query_ops.latest_time(rd);
- int live = ((now - last_t) < (RRDSET_MINIMUM_LIVE_COUNT * rd->update_every));
+ int live = ((now - last_t) < (RRDSET_MINIMUM_DIM_LIVE_MULTIPLIER * rd->update_every));
rc = aclk_upd_dimension_event(
wc,
@@ -287,7 +295,11 @@ int aclk_add_dimension_event(struct aclk_database_worker_config *wc, struct aclk
rd->name,
rd->rrdset->id,
first_t,
- live ? 0 : last_t);
+ live ? 0 : last_t,
+ &send_status);
+
+ if (!send_status)
+ rd->state->aclk_live_status = live;
freez(claim_id);
}
@@ -295,14 +307,16 @@ int aclk_add_dimension_event(struct aclk_database_worker_config *wc, struct aclk
return rc;
}
-
void aclk_send_chart_event(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd)
{
int rc;
wc->chart_pending = 0;
if (unlikely(!wc->chart_updates)) {
- log_access("AC [%s (%s)]: Ignoring chart push event, updates have been turned off for this node.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
+ log_access(
+ "ACLK STA [%s (%s)]: Ignoring chart push event, updates have been turned off for this node.",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A");
return;
}
@@ -320,32 +334,31 @@ void aclk_send_chart_event(struct aclk_database_worker_config *wc, struct aclk_d
uint64_t last_sequence;
time_t last_timestamp = 0;
- BUFFER *sql = buffer_create(1024);
-
- sqlite3_stmt *res = NULL;
-
- buffer_sprintf(sql, "SELECT ac.sequence_id, acp.payload, ac.date_created, ac.type, ac.uuid " \
- "FROM aclk_chart_%s ac, aclk_chart_payload_%s acp " \
- "WHERE ac.date_submitted IS NULL AND ac.unique_id = acp.unique_id AND ac.update_count > 0 " \
- "AND acp.claim_id = @claim_id ORDER BY ac.sequence_id ASC LIMIT %d;", wc->uuid_str, wc->uuid_str, limit);
+ char sql[ACLK_SYNC_QUERY_SIZE];
+ static __thread sqlite3_stmt *res = NULL;
- rc = sqlite3_prepare_v2(db_meta, buffer_tostring(sql), -1, &res, 0);
- if (rc != SQLITE_OK) {
- error_report("Failed to prepare statement when trying to send a chart update via ACLK");
- buffer_free(sql);
- freez(claim_id);
- return;
+ if (unlikely(!res)) {
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1,"SELECT ac.sequence_id, acp.payload, ac.date_created, ac.type, ac.uuid " \
+ "FROM aclk_chart_%s ac, aclk_chart_payload_%s acp " \
+ "WHERE ac.date_submitted IS NULL AND ac.unique_id = acp.unique_id AND ac.update_count > 0 " \
+ "AND acp.claim_id = @claim_id ORDER BY ac.sequence_id ASC LIMIT %d;", wc->uuid_str, wc->uuid_str, limit);
+ rc = prepare_statement(db_meta, sql, &res);
+ if (rc != SQLITE_OK) {
+ error_report("Failed to prepare statement when trying to send a chart update via ACLK");
+ freez(claim_id);
+ return;
+ }
}
- rc = sqlite3_bind_blob(res, 1, claim_uuid , sizeof(claim_uuid), SQLITE_STATIC);
+ rc = sqlite3_bind_blob(res, 1, claim_uuid, sizeof(claim_uuid), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK))
goto bind_fail;
- char **payload_list = callocz(limit+1, sizeof(char *));
- size_t *payload_list_size = callocz(limit+1, sizeof(size_t));
- size_t *payload_list_max_size = callocz(limit+1, sizeof(size_t));
- struct aclk_message_position *position_list = callocz(limit+1, sizeof(*position_list));
- int *is_dim = callocz(limit+1, sizeof(*is_dim));
+ char **payload_list = callocz(limit + 1, sizeof(char *));
+ size_t *payload_list_size = callocz(limit + 1, sizeof(size_t));
+ size_t *payload_list_max_size = callocz(limit + 1, sizeof(size_t));
+ struct aclk_message_position *position_list = callocz(limit + 1, sizeof(*position_list));
+ int *is_dim = callocz(limit + 1, sizeof(*is_dim));
int loop = cmd.param1;
@@ -388,29 +401,31 @@ void aclk_send_chart_event(struct aclk_database_worker_config *wc, struct aclk_d
error_report("Failed to reset statement when pushing chart events, rc = %d", rc);
if (likely(first_sequence)) {
- buffer_flush(sql);
db_lock();
- buffer_sprintf(sql, "UPDATE aclk_chart_%s SET status = NULL, date_submitted=strftime('%%s','now') "
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1, "UPDATE aclk_chart_%s SET status = NULL, date_submitted=strftime('%%s','now') "
"WHERE date_submitted IS NULL AND sequence_id BETWEEN %" PRIu64 " AND %" PRIu64 ";",
wc->uuid_str, first_sequence, last_sequence);
- db_execute(buffer_tostring(sql));
-
- buffer_flush(sql);
- buffer_sprintf(sql, "INSERT OR REPLACE INTO aclk_chart_latest_%s (uuid, unique_id, date_submitted) "
+ db_execute(sql);
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1, "INSERT OR REPLACE INTO aclk_chart_latest_%s (uuid, unique_id, date_submitted) "
" SELECT uuid, unique_id, date_submitted FROM aclk_chart_%s s "
" WHERE date_submitted IS NOT NULL AND sequence_id BETWEEN %" PRIu64 " AND %" PRIu64
" ;",
wc->uuid_str, wc->uuid_str, first_sequence, last_sequence);
- db_execute(buffer_tostring(sql));
+ db_execute(sql);
db_unlock();
aclk_chart_inst_and_dim_update(payload_list, payload_list_size, is_dim, position_list, wc->batch_id);
- log_access("OG [%s (%s)]: Sending charts and dimensions update, batch_id %"PRIu64", first sequence %"PRIu64", last sequence %"PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", wc->batch_id, first_sequence, last_sequence);
+ log_access(
+ "ACLK RES [%s (%s)]: CHARTS SENT from %" PRIu64 " to %" PRIu64 " batch=%" PRIu64,
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ first_sequence,
+ last_sequence,
+ wc->batch_id);
wc->chart_sequence_id = last_sequence;
wc->chart_timestamp = last_timestamp;
- }
- else
+ } else
break;
--loop;
}
@@ -419,11 +434,14 @@ void aclk_send_chart_event(struct aclk_database_worker_config *wc, struct aclk_d
time_t now = now_realtime_sec();
if (wc->rotation_after > now && wc->rotation_after < now + ACLK_DATABASE_ROTATION_DELAY)
wc->rotation_after = now + ACLK_DATABASE_ROTATION_DELAY;
- }
- else {
+ } else {
wc->chart_payload_count = sql_get_pending_count(wc);
if (!wc->chart_payload_count)
- log_access("AC [%s (%s)]: Sync of charts and dimensions done in %ld seconds.", wc->node_id, wc->host ? wc->host->hostname : "N/A", now_realtime_sec() - wc->startup_time);
+ log_access(
+ "ACLK STA [%s (%s)]: Sync of charts and dimensions done in %ld seconds.",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ now_realtime_sec() - wc->startup_time);
}
for (int i = 0; i <= limit; ++i)
@@ -436,11 +454,10 @@ void aclk_send_chart_event(struct aclk_database_worker_config *wc, struct aclk_d
freez(is_dim);
bind_fail:
- rc = sqlite3_finalize(res);
+ rc = sqlite3_reset(res);
if (unlikely(rc != SQLITE_OK))
- error_report("Failed to finalize statement when pushing chart events, rc = %d", rc);
+ error_report("Failed to reset statement when pushing chart events, rc = %d", rc);
- buffer_free(sql);
freez(claim_id);
return;
}
@@ -496,40 +513,44 @@ int aclk_send_chart_config(struct aclk_database_worker_config *wc, struct aclk_d
}
if (likely(chart_config.config_hash)) {
- log_access("OG [%s (%s)]: Sending chart config for %s.", wc->node_id, wc->host ? wc->host->hostname : "N/A", hash_id);
+ log_access(
+ "ACLK REQ [%s (%s)]: Sending chart config for %s.",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ hash_id);
aclk_chart_config_updated(&chart_config, 1);
destroy_chart_config_updated(&chart_config);
- }
- else
- log_access("AC [%s (%s)]: Chart config for %s not found.", wc->node_id, wc->host ? wc->host->hostname : "N/A", hash_id);
+ } else
+ log_access(
+ "ACLK STA [%s (%s)]: Chart config for %s not found.",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ hash_id);
- bind_fail:
- rc = sqlite3_finalize(res);
- if (unlikely(rc != SQLITE_OK))
- error_report("Failed to reset statement when pushing chart config hash, rc = %d", rc);
- fail:
- freez((char *) cmd.data_param);
- buffer_free(sql);
- return rc;
+bind_fail:
+ rc = sqlite3_finalize(res);
+ if (unlikely(rc != SQLITE_OK))
+ error_report("Failed to reset statement when pushing chart config hash, rc = %d", rc);
+fail:
+ freez((char *)cmd.data_param);
+ buffer_free(sql);
+ return rc;
}
-
void aclk_receive_chart_ack(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd)
{
int rc;
sqlite3_stmt *res = NULL;
- log_access("IN [%s (%s)]: Received ack chart sequence id %"PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", cmd.param1);
+ char sql[ACLK_SYNC_QUERY_SIZE];
- BUFFER *sql = buffer_create(1024);
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1,"UPDATE aclk_chart_%s SET date_updated=strftime('%%s','now') WHERE sequence_id <= @sequence_id "
+ "AND date_submitted IS NOT NULL AND date_updated IS NULL;", wc->uuid_str);
- buffer_sprintf(sql, "UPDATE aclk_chart_%s SET date_updated=strftime('%%s','now') WHERE sequence_id <= @sequence_id "
- "AND date_submitted IS NOT NULL AND date_updated IS NULL;", wc->uuid_str);
-
- rc = sqlite3_prepare_v2(db_meta, buffer_tostring(sql), -1, &res, 0);
+ rc = sqlite3_prepare_v2(db_meta, sql, -1, &res, 0);
if (rc != SQLITE_OK) {
- error_report("Failed to prepare statement count sequence ids in the database");
- goto prepare_fail;
+ error_report("Failed to prepare statement to ack chart sequence ids");
+ return;
}
rc = sqlite3_bind_int64(res, 1, (uint64_t) cmd.param1);
@@ -539,28 +560,34 @@ void aclk_receive_chart_ack(struct aclk_database_worker_config *wc, struct aclk_
rc = execute_insert(res);
if (rc != SQLITE_DONE)
error_report("Failed to ACK sequence id, rc = %d", rc);
+ else
+ log_access(
+ "ACLK STA [%s (%s)]: CHARTS ACKNOWLEDGED in the database upto %" PRIu64,
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ cmd.param1);
- bind_fail:
- if (unlikely(sqlite3_finalize(res) != SQLITE_OK))
- error_report("Failed to finalize statement to ACK older sequence ids, rc = %d", rc);
-
- prepare_fail:
- buffer_free(sql);
- return;
+bind_fail:
+ if (unlikely(sqlite3_finalize(res) != SQLITE_OK))
+ error_report("Failed to finalize statement to ACK older sequence ids, rc = %d", rc);
+ return;
}
void aclk_receive_chart_reset(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd)
{
BUFFER *sql = buffer_create(1024);
- buffer_sprintf(sql, "UPDATE aclk_chart_%s SET status = NULL, date_submitted = NULL WHERE sequence_id >= %"PRIu64";",
- wc->uuid_str, cmd.param1);
+ buffer_sprintf(
+ sql,
+ "UPDATE aclk_chart_%s SET status = NULL, date_submitted = NULL WHERE sequence_id >= %" PRIu64 ";",
+ wc->uuid_str,
+ cmd.param1);
db_execute(buffer_tostring(sql));
if (cmd.param1 == 1) {
- db_lock();
buffer_flush(sql);
- log_access("IN [%s (%s)]: Received chart full resync.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
+ log_access("ACLK REQ [%s (%s)]: Received chart full resync.", wc->node_id, wc->host ? wc->host->hostname : "N/A");
buffer_sprintf(sql, "DELETE FROM aclk_chart_payload_%s; DELETE FROM aclk_chart_%s; " \
"DELETE FROM aclk_chart_latest_%s;", wc->uuid_str, wc->uuid_str, wc->uuid_str);
+ db_lock();
db_execute("BEGIN TRANSACTION;");
db_execute(buffer_tostring(sql));
@@ -587,12 +614,14 @@ void aclk_receive_chart_reset(struct aclk_database_worker_config *wc, struct acl
rrdset_unlock(st);
}
rrdhost_unlock(host);
- }
- else
+ } else
error_report("ACLK synchronization thread for %s is not linked to HOST", wc->host_guid);
- }
- else {
- log_access("AC [%s (%s)]: Restarting chart sync from sequence %"PRIu64, wc->node_id, wc->host ? wc->host->hostname : "N/A", cmd.param1);
+ } else {
+ log_access(
+ "ACLK STA [%s (%s)]: Restarting chart sync from sequence %" PRIu64,
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ cmd.param1);
wc->chart_payload_count = sql_get_pending_count(wc);
sql_get_last_chart_sequence(wc);
}
@@ -601,7 +630,6 @@ void aclk_receive_chart_reset(struct aclk_database_worker_config *wc, struct acl
return;
}
-
//
// Functions called directly from ACLK threads and will queue commands
//
@@ -617,7 +645,12 @@ void aclk_get_chart_config(char **hash_id)
cmd.opcode = ACLK_DATABASE_PUSH_CHART_CONFIG;
for (int i = 0; hash_id[i]; ++i) {
// TODO: Verify that we have a valid hash_id
- log_access("IN [%s (%s)]: Request %d for chart config with hash %s received.", wc->node_id, wc->host ? wc->host->hostname : "N/A", i, hash_id[i]);
+ log_access(
+ "ACLK REQ [%s (%s)]: Request %d for chart config with hash %s received.",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ i,
+ hash_id[i]);
cmd.data_param = (void *)strdupz(hash_id[i]);
aclk_database_enq_cmd(wc, &cmd);
}
@@ -632,13 +665,13 @@ static void aclk_submit_param_command(char *node_id, enum aclk_database_opcode a
if (unlikely(!node_id))
return;
- struct aclk_database_worker_config *wc = NULL;
+ struct aclk_database_worker_config *wc = NULL;
struct aclk_database_cmd cmd;
memset(&cmd, 0, sizeof(cmd));
cmd.opcode = aclk_command;
cmd.param1 = param;
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *host = find_host_by_node_id(node_id);
if (likely(host))
wc = (struct aclk_database_worker_config *)host->dbsync_worker;
@@ -647,7 +680,7 @@ static void aclk_submit_param_command(char *node_id, enum aclk_database_opcode a
aclk_database_enq_cmd(wc, &cmd);
else {
if (aclk_worker_enq_cmd(node_id, &cmd))
- log_access("AC [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
+ log_access("ACLK STA [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
}
return;
}
@@ -657,7 +690,10 @@ void aclk_ack_chart_sequence_id(char *node_id, uint64_t last_sequence_id)
if (unlikely(!node_id))
return;
- log_access("AC [%s (N/A)]: Node reports last sequence id received %"PRIu64, node_id, last_sequence_id);
+ char *hostname = get_hostname_by_node_id(node_id);
+ log_access("ACLK REQ [%s (%s)]: CHARTS ACKNOWLEDGED upto %" PRIu64, node_id, hostname ? hostname : "N/A",
+ last_sequence_id);
+ freez(hostname);
aclk_submit_param_command(node_id, ACLK_DATABASE_CHART_ACK, last_sequence_id);
return;
}
@@ -669,15 +705,17 @@ void aclk_start_streaming(char *node_id, uint64_t sequence_id, time_t created_at
if (unlikely(!node_id))
return;
- log_access("IN [%s (N/A)]: Start streaming charts from sequence %"PRIu64" t=%ld, batch=%"PRIu64, node_id,
- sequence_id, created_at, batch_id);
+ // log_access("ACLK REQ [%s (N/A)]: CHARTS STREAM from %"PRIu64" t=%ld batch=%"PRIu64, node_id,
+ // sequence_id, created_at, batch_id);
uuid_t node_uuid;
- if (uuid_parse(node_id, node_uuid))
+ if (uuid_parse(node_id, node_uuid)) {
+ log_access("ACLK REQ [%s (N/A)]: CHARTS STREAM ignored, invalid node id", node_id);
return;
+ }
struct aclk_database_worker_config *wc = NULL;
- rrd_wrlock();
+ rrd_rdlock();
RRDHOST *host = localhost;
while(host) {
if (host->node_id && !(uuid_compare(*host->node_id, node_uuid))) {
@@ -692,10 +730,23 @@ void aclk_start_streaming(char *node_id, uint64_t sequence_id, time_t created_at
wc->batch_id = batch_id;
__sync_synchronize();
wc->batch_created = now_realtime_sec();
+ log_access(
+ "ACLK REQ [%s (%s)]: CHARTS STREAM from %" PRIu64 " t=%ld resets=%d",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ wc->chart_sequence_id,
+ wc->chart_timestamp,
+ wc->chart_reset_count);
if (sequence_id > wc->chart_sequence_id || wc->chart_reset_count > 10) {
- log_access("AC [%s (%s)]: Requesting full resync from the cloud "
- "(reset=%d, remote_seq=%"PRIu64", local_seq=%"PRIu64")"
- , wc->node_id, wc->host ? wc->host->hostname : "N/A", wc->chart_reset_count, sequence_id, wc->chart_sequence_id);
+ log_access(
+ "ACLK RES [%s (%s)]: CHARTS FULL RESYNC REQUEST "
+ "remote_seq=%" PRIu64 " local_seq=%" PRIu64 " resets=%d ",
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ sequence_id,
+ wc->chart_sequence_id,
+ wc->chart_reset_count);
+
chart_reset_t chart_reset;
chart_reset.claim_id = is_agent_claimed();
if (chart_reset.claim_id) {
@@ -710,26 +761,34 @@ void aclk_start_streaming(char *node_id, uint64_t sequence_id, time_t created_at
struct aclk_database_cmd cmd;
memset(&cmd, 0, sizeof(cmd));
// TODO: handle timestamp
- if (sequence_id < wc->chart_sequence_id || !sequence_id) { // || created_at != wc->chart_timestamp) {
- log_access("AC [%s (%s)]: Reset streaming charts from sequence %"PRIu64 \
- " t=%ld (reset count=%d)", wc->node_id, wc->host ? wc->host->hostname : "N/A", wc->chart_sequence_id,
- wc->chart_timestamp, wc->chart_reset_count);
+ if (sequence_id < wc->chart_sequence_id ||
+ !sequence_id) { // || created_at != wc->chart_timestamp) {
+ log_access(
+ "ACLK REQ [%s (%s)]: CHART RESET from %" PRIu64 " t=%ld batch=%" PRIu64,
+ wc->node_id,
+ wc->host ? wc->host->hostname : "N/A",
+ wc->chart_sequence_id,
+ wc->chart_timestamp,
+ wc->batch_id);
cmd.opcode = ACLK_DATABASE_RESET_CHART;
cmd.param1 = sequence_id + 1;
cmd.completion = NULL;
aclk_database_enq_cmd(wc, &cmd);
- }
- else {
- log_access("AC [%s (%s)]: Start streaming charts enabled -- last streamed sequence %"PRIu64 \
- " t=%ld (reset count=%d)", wc->node_id, wc->host ? wc->host->hostname : "N/A", wc->chart_sequence_id,
- wc->chart_timestamp, wc->chart_reset_count);
+ } else {
+// log_access(
+// "ACLK RES [%s (%s)]: CHARTS STREAM from %" PRIu64
+// " t=%ld resets=%d",
+// wc->node_id,
+// wc->host ? wc->host->hostname : "N/A",
+// wc->chart_sequence_id,
+// wc->chart_timestamp,
+// wc->chart_reset_count);
wc->chart_reset_count = 0;
wc->chart_updates = 1;
}
}
- }
- else
- log_access("AC [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
+ } else
+ log_access("ACLK STA [%s (N/A)]: ACLK synchronization thread is not active.", node_id);
return;
}
host = host->next;
@@ -761,7 +820,7 @@ static RRD_MEMORY_MODE sql_get_host_memory_mode(uuid_t *host_id)
}
while (sqlite3_step(res) == SQLITE_ROW) {
- memory_mode = (RRD_MEMORY_MODE) sqlite3_column_int(res, 0);
+ memory_mode = (RRD_MEMORY_MODE)sqlite3_column_int(res, 0);
}
failed:
@@ -771,11 +830,13 @@ failed:
return memory_mode;
}
-#define SELECT_HOST_DIMENSION_LIST "SELECT d.dim_id, c.update_every, c.type||'.'||c.id, d.id, d.name FROM chart c, dimension d " \
- "WHERE d.chart_id = c.chart_id AND c.host_id = @host_id ORDER BY c.update_every ASC;"
+#define SELECT_HOST_DIMENSION_LIST \
+ "SELECT d.dim_id, c.update_every, c.type||'.'||c.id, d.id, d.name FROM chart c, dimension d " \
+ "WHERE d.chart_id = c.chart_id AND c.host_id = @host_id ORDER BY c.update_every ASC;"
-#define SELECT_HOST_CHART_LIST "SELECT distinct h.host_id, c.update_every, c.type||'.'||c.id FROM chart c, host h " \
- "WHERE c.host_id = h.host_id AND c.host_id = @host_id ORDER BY c.update_every ASC;"
+#define SELECT_HOST_CHART_LIST \
+ "SELECT distinct h.host_id, c.update_every, c.type||'.'||c.id FROM chart c, host h " \
+ "WHERE c.host_id = h.host_id AND c.host_id = @host_id ORDER BY c.update_every ASC;"
void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd)
{
@@ -821,10 +882,12 @@ void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_d
goto failed;
}
- time_t start_time = LONG_MAX;
- time_t first_entry_t;
- time_t last_entry_t;
+ time_t start_time = LONG_MAX;
+ time_t first_entry_t;
+ time_t last_entry_t;
uint32_t update_every = 0;
+ uint32_t dimension_update_count = 0;
+ int send_status;
struct retention_updated rotate_data;
@@ -840,11 +903,11 @@ void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_d
rotate_data.claim_id = claim_id;
rotate_data.node_id = strdupz(wc->node_id);
- // time_t now = now_realtime_sec();
+ time_t now = now_realtime_sec();
while (sqlite3_step(res) == SQLITE_ROW) {
- if (!update_every || update_every != (uint32_t) sqlite3_column_int(res, 1)) {
+ if (!update_every || update_every != (uint32_t)sqlite3_column_int(res, 1)) {
if (update_every) {
- debug(D_ACLK_SYNC,"Update %s for %u oldest time = %ld", wc->host_guid, update_every, start_time);
+ debug(D_ACLK_SYNC, "Update %s for %u oldest time = %ld", wc->host_guid, update_every, start_time);
if (start_time == LONG_MAX)
rotate_data.interval_durations[rotate_data.interval_duration_count].retention = 0;
else
@@ -852,13 +915,14 @@ void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_d
rotate_data.rotation_timestamp.tv_sec - start_time;
rotate_data.interval_duration_count++;
}
- update_every = (uint32_t) sqlite3_column_int(res, 1);
+ update_every = (uint32_t)sqlite3_column_int(res, 1);
rotate_data.interval_durations[rotate_data.interval_duration_count].update_every = update_every;
start_time = LONG_MAX;
}
#ifdef ENABLE_DBENGINE
if (memory_mode == RRD_MEMORY_MODE_DBENGINE)
- rc = rrdeng_metric_latest_time_by_uuid((uuid_t *)sqlite3_column_blob(res, 0), &first_entry_t, &last_entry_t);
+ rc =
+ rrdeng_metric_latest_time_by_uuid((uuid_t *)sqlite3_column_blob(res, 0), &first_entry_t, &last_entry_t);
else
#endif
{
@@ -869,8 +933,7 @@ void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_d
first_entry_t = rrdset_first_entry_t(st);
last_entry_t = rrdset_last_entry_t(st);
}
- }
- else {
+ } else {
rc = 0;
first_entry_t = rotate_data.rotation_timestamp.tv_sec;
}
@@ -878,6 +941,24 @@ void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_d
if (likely(!rc && first_entry_t))
start_time = MIN(start_time, first_entry_t);
+
+ if (memory_mode == RRD_MEMORY_MODE_DBENGINE && wc->chart_updates) {
+ int live = ((now - last_entry_t) < (RRDSET_MINIMUM_DIM_LIVE_MULTIPLIER * update_every));
+ if ((!live || !first_entry_t) && (dimension_update_count < ACLK_MAX_DIMENSION_CLEANUP)) {
+ (void)aclk_upd_dimension_event(
+ wc,
+ claim_id,
+ (uuid_t *)sqlite3_column_blob(res, 0),
+ (const char *)(const char *)sqlite3_column_text(res, 3),
+ (const char *)(const char *)sqlite3_column_text(res, 4),
+ (const char *)(const char *)sqlite3_column_text(res, 2),
+ first_entry_t,
+ live ? 0 : last_entry_t,
+ &send_status);
+ if (!send_status)
+ dimension_update_count++;
+ }
+ }
}
if (update_every) {
debug(D_ACLK_SYNC, "Update %s for %u oldest time = %ld", wc->host_guid, update_every, start_time);
@@ -891,8 +972,12 @@ void aclk_update_retention(struct aclk_database_worker_config *wc, struct aclk_d
#ifdef NETDATA_INTERNAL_CHECKS
for (int i = 0; i < rotate_data.interval_duration_count; ++i)
- info("Update for host %s (node %s) for %u Retention = %u", wc->host_guid, wc->node_id,
- rotate_data.interval_durations[i].update_every, rotate_data.interval_durations[i].retention);
+ info(
+ "Update for host %s (node %s) for %u Retention = %u",
+ wc->host_guid,
+ wc->node_id,
+ rotate_data.interval_durations[i].update_every,
+ rotate_data.interval_durations[i].retention);
#endif
aclk_retention_updated(&rotate_data);
freez(rotate_data.node_id);
@@ -906,63 +991,60 @@ failed:
return;
}
-
uint32_t sql_get_pending_count(struct aclk_database_worker_config *wc)
{
- BUFFER *sql = buffer_create(1024);
- sqlite3_stmt *res = NULL;
+ char sql[ACLK_SYNC_QUERY_SIZE];
+ static __thread sqlite3_stmt *res = NULL;
- buffer_sprintf(sql,"SELECT count(1) FROM aclk_chart_%s ac WHERE ac.date_submitted IS NULL;", wc->uuid_str);
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1, "SELECT count(1) FROM aclk_chart_%s ac WHERE ac.date_submitted IS NULL;", wc->uuid_str);
int rc;
uint32_t chart_payload_count = 0;
- rc = sqlite3_prepare_v2(db_meta, buffer_tostring(sql), -1, &res, 0);
- if (rc != SQLITE_OK) {
- error_report("Failed to prepare statement to count pending messages");
- goto fail;
+ if (unlikely(!res)) {
+ rc = prepare_statement(db_meta, sql, &res);
+ if (rc != SQLITE_OK) {
+ error_report("Failed to prepare statement to count pending messages");
+ return 0;
+ }
}
while (sqlite3_step(res) == SQLITE_ROW)
chart_payload_count = (uint32_t) sqlite3_column_int(res, 0);
- rc = sqlite3_finalize(res);
+ rc = sqlite3_reset(res);
if (unlikely(rc != SQLITE_OK))
error_report("Failed to reset statement when fetching pending messages, rc = %d", rc);
-fail:
- buffer_free(sql);
return chart_payload_count;
}
void sql_get_last_chart_sequence(struct aclk_database_worker_config *wc)
{
- BUFFER *sql = buffer_create(1024);
+ char sql[ACLK_SYNC_QUERY_SIZE];
- buffer_sprintf(sql,"SELECT ac.sequence_id, ac.date_created FROM aclk_chart_%s ac " \
- "WHERE ac.date_submitted IS NOT NULL ORDER BY ac.sequence_id DESC LIMIT 1;", wc->uuid_str);
+ snprintfz(sql,ACLK_SYNC_QUERY_SIZE-1, "SELECT ac.sequence_id, ac.date_created FROM aclk_chart_%s ac " \
+ "WHERE ac.date_submitted IS NOT NULL ORDER BY ac.sequence_id DESC LIMIT 1;", wc->uuid_str);
int rc;
sqlite3_stmt *res = NULL;
- rc = sqlite3_prepare_v2(db_meta, buffer_tostring(sql), -1, &res, 0);
+ rc = sqlite3_prepare_v2(db_meta, sql, -1, &res, 0);
if (rc != SQLITE_OK) {
error_report("Failed to prepare statement to find last chart sequence id");
- goto fail;
+ return;
}
wc->chart_sequence_id = 0;
wc->chart_timestamp = 0;
while (sqlite3_step(res) == SQLITE_ROW) {
- wc->chart_sequence_id = (uint64_t) sqlite3_column_int64(res, 0);
- wc->chart_timestamp = (time_t) sqlite3_column_int64(res, 1);
+ wc->chart_sequence_id = (uint64_t)sqlite3_column_int64(res, 0);
+ wc->chart_timestamp = (time_t)sqlite3_column_int64(res, 1);
}
- debug(D_ACLK_SYNC,"Node %s reports last sequence_id=%"PRIu64, wc->node_id, wc->chart_sequence_id);
+ debug(D_ACLK_SYNC, "Node %s reports last sequence_id=%" PRIu64, wc->node_id, wc->chart_sequence_id);
rc = sqlite3_finalize(res);
if (unlikely(rc != SQLITE_OK))
error_report("Failed to reset statement when fetching chart sequence info, rc = %d", rc);
-fail:
- buffer_free(sql);
return;
}
@@ -973,6 +1055,184 @@ int queue_dimension_to_aclk(RRDDIM *rd)
return rc;
}
+void aclk_send_dimension_update(RRDDIM *rd)
+{
+ if (!aclk_use_new_cloud_arch)
+ return;
+
+ char *claim_id = is_agent_claimed();
+ if (unlikely(!claim_id))
+ return;
+
+ time_t first_entry_t = rrddim_first_entry_t(rd);
+ time_t last_entry_t = rrddim_last_entry_t(rd);
+
+ time_t now = now_realtime_sec();
+ int live = ((now - rd->last_collected_time.tv_sec) < (RRDSET_MINIMUM_DIM_LIVE_MULTIPLIER * rd->update_every));
+
+ if (!live || rd->state->aclk_live_status != live || !first_entry_t) {
+ (void)aclk_upd_dimension_event(
+ rd->rrdset->rrdhost->dbsync_worker,
+ claim_id,
+ &rd->state->metric_uuid,
+ rd->id,
+ rd->name,
+ rd->rrdset->id,
+ first_entry_t,
+ live ? 0 : last_entry_t,
+ NULL);
+
+ if (!first_entry_t)
+ debug(
+ D_ACLK_SYNC,
+ "%s: Update dimension chart=%s dim=%s live=%d (%ld, %ld)",
+ rd->rrdset->rrdhost->hostname,
+ rd->rrdset->name,
+ rd->name,
+ live,
+ first_entry_t,
+ last_entry_t);
+ else
+ debug(
+ D_ACLK_SYNC,
+ "%s: Update dimension chart=%s dim=%s live=%d (%ld, %ld) collected %ld seconds ago",
+ rd->rrdset->rrdhost->hostname,
+ rd->rrdset->name,
+ rd->name,
+ live,
+ first_entry_t,
+ last_entry_t,
+ now - last_entry_t);
+ rd->state->aclk_live_status = live;
+ }
+
+ freez(claim_id);
+ return;
+}
+
+#define SQL_SEQ_NULL(result, n) sqlite3_column_type(result, n) == SQLITE_NULL ? 0 : sqlite3_column_int64(result, n)
+
+struct aclk_chart_sync_stats *aclk_get_chart_sync_stats(RRDHOST *host)
+{
+ struct aclk_chart_sync_stats *aclk_statistics = NULL;
+
+ struct aclk_database_worker_config *wc = NULL;
+ wc = (struct aclk_database_worker_config *)host->dbsync_worker;
+ if (!wc)
+ return NULL;
+
+ aclk_statistics = callocz(1, sizeof(struct aclk_chart_sync_stats));
+
+ aclk_statistics->updates = wc->chart_updates;
+ aclk_statistics->batch_id = wc->batch_id;
+
+ char host_uuid_fixed[GUID_LEN + 1];
+
+ strncpy(host_uuid_fixed, host->machine_guid, GUID_LEN);
+ host_uuid_fixed[GUID_LEN] = 0;
+
+ host_uuid_fixed[8] = '_';
+ host_uuid_fixed[13] = '_';
+ host_uuid_fixed[18] = '_';
+ host_uuid_fixed[23] = '_';
+
+ sqlite3_stmt *res = NULL;
+ BUFFER *sql = buffer_create(1024);
+ buffer_sprintf(sql, "SELECT min(sequence_id), max(sequence_id), 0 FROM aclk_chart_%s;", host_uuid_fixed);
+ buffer_sprintf(sql, "SELECT min(sequence_id), max(sequence_id), 0 FROM aclk_chart_%s WHERE date_submitted IS NULL;", host_uuid_fixed);
+ buffer_sprintf(sql, "SELECT min(sequence_id), max(sequence_id), 0 FROM aclk_chart_%s WHERE date_submitted IS NOT NULL;", host_uuid_fixed);
+ buffer_sprintf(sql, "SELECT min(sequence_id), max(sequence_id), 0 FROM aclk_chart_%s WHERE date_updated IS NOT NULL;", host_uuid_fixed);
+ buffer_sprintf(sql, "SELECT max(date_created), max(date_submitted), max(date_updated), 0 FROM aclk_chart_%s;", host_uuid_fixed);
+
+ int rc = sqlite3_prepare_v2(db_meta, buffer_tostring(sql), -1, &res, 0);
+ if (rc != SQLITE_OK) {
+ buffer_free(sql);
+ freez(aclk_statistics);
+ return NULL;
+ }
+
+ rc = sqlite3_step(res);
+ if (rc == SQLITE_ROW) {
+ aclk_statistics->min_seqid = SQL_SEQ_NULL(res, 0);
+ aclk_statistics->max_seqid = SQL_SEQ_NULL(res, 1);
+ }
+
+ rc = sqlite3_step(res);
+ if (rc == SQLITE_ROW) {
+ aclk_statistics->min_seqid_pend = SQL_SEQ_NULL(res, 0);
+ aclk_statistics->max_seqid_pend = SQL_SEQ_NULL(res, 1);
+ }
+
+ rc = sqlite3_step(res);
+ if (rc == SQLITE_ROW) {
+ aclk_statistics->min_seqid_sent = SQL_SEQ_NULL(res, 0);
+ aclk_statistics->max_seqid_sent = SQL_SEQ_NULL(res, 1);
+ }
+
+ rc = sqlite3_step(res);
+ if (rc == SQLITE_ROW) {
+ aclk_statistics->min_seqid_ack = SQL_SEQ_NULL(res, 0);
+ aclk_statistics->max_seqid_ack = SQL_SEQ_NULL(res, 1);
+ }
+
+ rc = sqlite3_step(res);
+ if (rc == SQLITE_ROW) {
+ aclk_statistics->min_seqid_ack = SQL_SEQ_NULL(res, 0);
+ aclk_statistics->max_seqid_ack = SQL_SEQ_NULL(res, 1);
+ }
+
+ rc = sqlite3_step(res);
+ if (rc == SQLITE_ROW) {
+ aclk_statistics->max_date_created = (time_t) SQL_SEQ_NULL(res, 0);
+ aclk_statistics->max_date_submitted = (time_t) SQL_SEQ_NULL(res, 1);
+ aclk_statistics->max_date_ack = (time_t) SQL_SEQ_NULL(res, 2);
+ }
+
+ rc = sqlite3_finalize(res);
+ if (unlikely(rc != SQLITE_OK))
+ error_report("Failed to finalize statement when fetching aclk sync statistics, rc = %d", rc);
+
+ buffer_free(sql);
+ return aclk_statistics;
+}
+
+void sql_check_chart_liveness(RRDSET *st) {
+ RRDDIM *rd;
+
+ if (unlikely(st->state->is_ar_chart))
+ return;
+
+ rrdset_rdlock(st);
+ if (unlikely(!rrdset_flag_check(st, RRDSET_FLAG_ACLK))) {
+ if (likely(st->dimensions && st->counter_done && !queue_chart_to_aclk(st))) {
+ debug(D_ACLK_SYNC,"Check chart liveness [%s] submit chart definition", st->name);
+ rrdset_flag_set(st, RRDSET_FLAG_ACLK);
+ }
+ }
+ else
+ debug(D_ACLK_SYNC,"Check chart liveness [%s] chart definition already submitted", st->name);
+ time_t mark = now_realtime_sec();
+
+ debug(D_ACLK_SYNC,"Check chart liveness [%s] scanning dimensions", st->name);
+ rrddim_foreach_read(rd, st) {
+ if (!rrddim_flag_check(rd, RRDDIM_FLAG_HIDDEN)) {
+ int live = (mark - rd->last_collected_time.tv_sec) < RRDSET_MINIMUM_DIM_LIVE_MULTIPLIER * rd->update_every;
+ if (unlikely(live != rd->state->aclk_live_status)) {
+ if (likely(rrdset_flag_check(st, RRDSET_FLAG_ACLK))) {
+ if (likely(!queue_dimension_to_aclk(rd))) {
+ debug(D_ACLK_SYNC,"Dimension change [%s] on [%s] from live %d --> %d", rd->id, rd->rrdset->name, rd->state->aclk_live_status, live);
+ rd->state->aclk_live_status = live;
+ rrddim_flag_set(rd, RRDDIM_FLAG_ACLK);
+ }
+ }
+ }
+ else
+ debug(D_ACLK_SYNC,"Dimension check [%s] on [%s] liveness matches", rd->id, st->name);
+ }
+ }
+ rrdset_unlock(st);
+}
+
#endif //ENABLE_NEW_CLOUD_PROTOCOL
// ST is read locked
@@ -994,4 +1254,3 @@ int queue_chart_to_aclk(RRDSET *st)
st, ACLK_DATABASE_ADD_CHART);
#endif
}
-
diff --git a/database/sqlite/sqlite_aclk_chart.h b/database/sqlite/sqlite_aclk_chart.h
index 67d81a534..1d25de24e 100644
--- a/database/sqlite/sqlite_aclk_chart.h
+++ b/database/sqlite/sqlite_aclk_chart.h
@@ -12,10 +12,30 @@ typedef enum payload_type {
extern sqlite3 *db_meta;
-#ifndef RRDSET_MINIMUM_LIVE_COUNT
-#define RRDSET_MINIMUM_LIVE_COUNT 3
+#ifndef RRDSET_MINIMUM_DIM_LIVE_MULTIPLIER
+#define RRDSET_MINIMUM_DIM_LIVE_MULTIPLIER (3)
#endif
+#ifndef ACLK_MAX_DIMENSION_CLEANUP
+#define ACLK_MAX_DIMENSION_CLEANUP (500)
+#endif
+
+struct aclk_chart_sync_stats {
+ int updates;
+ uint64_t batch_id;
+ uint64_t min_seqid;
+ uint64_t max_seqid;
+ uint64_t min_seqid_pend;
+ uint64_t max_seqid_pend;
+ uint64_t min_seqid_sent;
+ uint64_t max_seqid_sent;
+ uint64_t min_seqid_ack;
+ uint64_t max_seqid_ack;
+ time_t max_date_created;
+ time_t max_date_submitted;
+ time_t max_date_ack;
+};
+
extern int queue_chart_to_aclk(RRDSET *st);
extern int queue_dimension_to_aclk(RRDDIM *rd);
extern void sql_create_aclk_table(RRDHOST *host, uuid_t *host_uuid, uuid_t *node_id);
@@ -34,4 +54,7 @@ void aclk_receive_chart_reset(struct aclk_database_worker_config *wc, struct acl
void aclk_receive_chart_ack(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd);
void aclk_process_dimension_deletion(struct aclk_database_worker_config *wc, struct aclk_database_cmd cmd);
uint32_t sql_get_pending_count(struct aclk_database_worker_config *wc);
+void aclk_send_dimension_update(RRDDIM *rd);
+struct aclk_chart_sync_stats *aclk_get_chart_sync_stats(RRDHOST *host);
+void sql_check_chart_liveness(RRDSET *st);
#endif //NETDATA_SQLITE_ACLK_CHART_H
diff --git a/database/sqlite/sqlite_aclk_node.c b/database/sqlite/sqlite_aclk_node.c
index 6261b9af5..97e6bebd1 100644
--- a/database/sqlite/sqlite_aclk_node.c
+++ b/database/sqlite/sqlite_aclk_node.c
@@ -17,16 +17,23 @@ void sql_build_node_info(struct aclk_database_worker_config *wc, struct aclk_dat
if (!wc->host)
return;
- rrd_wrlock();
+ rrd_rdlock();
node_info.node_id = wc->node_id;
node_info.claim_id = is_agent_claimed();
node_info.machine_guid = wc->host_guid;
node_info.child = (wc->host != localhost);
- node_info.ml_info.ml_capable = localhost->system_info->ml_capable;
- node_info.ml_info.ml_enabled = wc->host->ml_host != NULL;
+ node_info.ml_info.ml_capable = ml_capable(localhost);
+ node_info.ml_info.ml_enabled = ml_enabled(wc->host);
now_realtime_timeval(&node_info.updated_at);
RRDHOST *host = wc->host;
+ char *host_version = NULL;
+ if (host != localhost) {
+ netdata_mutex_lock(&host->receiver_lock);
+ host_version =
+ strdupz(host->receiver && host->receiver->program_version ? host->receiver->program_version : "unknown");
+ netdata_mutex_unlock(&host->receiver_lock);
+ }
node_info.data.name = host->hostname;
node_info.data.os = (char *) host->os;
@@ -35,15 +42,15 @@ void sql_build_node_info(struct aclk_database_worker_config *wc, struct aclk_dat
node_info.data.kernel_name = host->system_info->kernel_name;
node_info.data.kernel_version = host->system_info->kernel_version;
node_info.data.architecture = host->system_info->architecture;
- node_info.data.cpus = str2uint32_t(host->system_info->host_cores);
- node_info.data.cpu_frequency = host->system_info->host_cpu_freq;
- node_info.data.memory = host->system_info->host_ram_total;
- node_info.data.disk_space = host->system_info->host_disk_space;
- node_info.data.version = VERSION;
+ node_info.data.cpus = host->system_info->host_cores ? str2uint32_t(host->system_info->host_cores) : 0;
+ node_info.data.cpu_frequency = host->system_info->host_cpu_freq ? host->system_info->host_cpu_freq : "0";
+ node_info.data.memory = host->system_info->host_ram_total ? host->system_info->host_ram_total : "0";
+ node_info.data.disk_space = host->system_info->host_disk_space ? host->system_info->host_disk_space : "0";
+ node_info.data.version = host_version ? host_version : VERSION;
node_info.data.release_channel = "nightly";
node_info.data.timezone = (char *) host->abbrev_timezone;
- node_info.data.virtualization_type = host->system_info->virtualization;
- node_info.data.container_type = host->system_info->container;
+ node_info.data.virtualization_type = host->system_info->virtualization ? host->system_info->virtualization : "unknown";
+ node_info.data.container_type = host->system_info->container ? host->system_info->container : "unknown";
node_info.data.custom_info = config_get(CONFIG_SECTION_WEB, "custom dashboard_info.js", "");
node_info.data.services = NULL; // char **
node_info.data.service_count = 0;
@@ -56,11 +63,12 @@ void sql_build_node_info(struct aclk_database_worker_config *wc, struct aclk_dat
node_info.data.host_labels_head = labels->head;
aclk_update_node_info(&node_info);
- log_access("OG [%s (%s)]: Sending node info for guid [%s] (%s).", wc->node_id, wc->host->hostname, wc->host_guid, wc->host == localhost ? "parent" : "child");
+ log_access("ACLK RES [%s (%s)]: NODE INFO SENT for guid [%s] (%s)", wc->node_id, wc->host->hostname, wc->host_guid, wc->host == localhost ? "parent" : "child");
netdata_rwlock_unlock(&labels->labels_rwlock);
rrd_unlock();
freez(node_info.claim_id);
+ freez(host_version);
#else
UNUSED(wc);
#endif
diff --git a/database/sqlite/sqlite_functions.c b/database/sqlite/sqlite_functions.c
index d5afdb6ee..1e1d2a741 100644
--- a/database/sqlite/sqlite_functions.c
+++ b/database/sqlite/sqlite_functions.c
@@ -2,6 +2,8 @@
#include "sqlite_functions.h"
+#define DB_METADATA_VERSION "1"
+
const char *database_config[] = {
"PRAGMA auto_vacuum=incremental; PRAGMA synchronous=1 ; PRAGMA journal_mode=WAL; PRAGMA temp_store=MEMORY;",
"PRAGMA journal_size_limit=16777216;",
@@ -40,6 +42,9 @@ const char *database_config[] = {
"CREATE VIEW IF NOT EXISTS v_chart_hash as SELECT ch.*, chm.chart_id FROM chart_hash ch, chart_hash_map chm "
"WHERE ch.hash_id = chm.hash_id;",
+ "CREATE TRIGGER IF NOT EXISTS ins_host AFTER INSERT ON host BEGIN INSERT INTO node_instance (host_id, date_created)"
+ " SELECT new.host_id, strftime(\"%s\") WHERE new.host_id NOT IN (SELECT host_id FROM node_instance); END;",
+
"CREATE TRIGGER IF NOT EXISTS tr_v_chart_hash INSTEAD OF INSERT on v_chart_hash BEGIN "
"INSERT INTO chart_hash (hash_id, type, id, name, family, context, title, unit, plugin, "
"module, priority, chart_type, last_used) "
@@ -49,6 +54,7 @@ const char *database_config[] = {
"INSERT INTO chart_hash_map (chart_id, hash_id) values (new.chart_id, new.hash_id) "
"on conflict (chart_id, hash_id) do nothing; END; ",
+ "PRAGMA user_version="DB_METADATA_VERSION";",
NULL
};
@@ -115,7 +121,7 @@ static int store_active_uuid_object(sqlite3_stmt **res, char *statement, uuid_t
// Check if we should need to prepare the statement
if (!*res) {
- rc = sqlite3_prepare_v2(db_meta, statement, -1, res, 0);
+ rc = prepare_statement(db_meta, statement, res);
if (unlikely(rc != SQLITE_OK)) {
error_report("Failed to prepare statement to store active object, rc = %d", rc);
return rc;
@@ -136,7 +142,7 @@ static int store_active_uuid_object(sqlite3_stmt **res, char *statement, uuid_t
*/
void store_active_chart(uuid_t *chart_uuid)
{
- sqlite3_stmt *res = NULL;
+ static __thread sqlite3_stmt *res = NULL;
int rc;
if (unlikely(!db_meta)) {
@@ -152,7 +158,7 @@ void store_active_chart(uuid_t *chart_uuid)
if (rc != SQLITE_DONE)
error_report("Failed to store active chart, rc = %d", rc);
- rc = sqlite3_finalize(res);
+ rc = sqlite3_reset(res);
if (unlikely(rc != SQLITE_OK))
error_report("Failed to finalize statement in store active chart, rc = %d", rc);
return;
@@ -164,7 +170,7 @@ void store_active_chart(uuid_t *chart_uuid)
*/
void store_active_dimension(uuid_t *dimension_uuid)
{
- sqlite3_stmt *res = NULL;
+ static __thread sqlite3_stmt *res = NULL;
int rc;
if (unlikely(!db_meta)) {
@@ -180,7 +186,7 @@ void store_active_dimension(uuid_t *dimension_uuid)
if (rc != SQLITE_DONE)
error_report("Failed to store active dimension, rc = %d", rc);
- rc = sqlite3_finalize(res);
+ rc = sqlite3_reset(res);
if (unlikely(rc != SQLITE_OK))
error_report("Failed to finalize statement in store active dimension, rc = %d", rc);
return;
@@ -905,6 +911,49 @@ bind_fail:
return 1;
}
+/*
+ * Store set option for a dimension
+ */
+int sql_set_dimension_option(uuid_t *dim_uuid, char *option)
+{
+ sqlite3_stmt *res = NULL;
+ int rc;
+
+ if (unlikely(!db_meta)) {
+ if (default_rrd_memory_mode != RRD_MEMORY_MODE_DBENGINE)
+ return 0;
+ error_report("Database has not been initialized");
+ return 1;
+ }
+
+ rc = sqlite3_prepare_v2(db_meta, "UPDATE dimension SET options = @options WHERE dim_id = @dim_id", -1, &res, 0);
+ if (unlikely(rc != SQLITE_OK)) {
+ error_report("Failed to prepare statement to update dimension options");
+ return 0;
+ };
+
+ rc = sqlite3_bind_blob(res, 2, dim_uuid, sizeof(*dim_uuid), SQLITE_STATIC);
+ if (unlikely(rc != SQLITE_OK))
+ goto bind_fail;
+
+ if (!option || !strcmp(option,"unhide"))
+ rc = sqlite3_bind_null(res, 1);
+ else
+ rc = sqlite3_bind_text(res, 1, option, -1, SQLITE_STATIC);
+ if (unlikely(rc != SQLITE_OK))
+ goto bind_fail;
+
+ rc = execute_insert(res);
+ if (unlikely(rc != SQLITE_DONE))
+ error_report("Failed to update dimension option, rc = %d", rc);
+
+bind_fail:
+ rc = sqlite3_finalize(res);
+ if (unlikely(rc != SQLITE_OK))
+ error_report("Failed to finalize statement in update dimension options, rc = %d", rc);
+ return 0;
+}
+
//
// Support for archived charts
@@ -1286,7 +1335,7 @@ void add_migrated_file(char *path, uint64_t file_size)
void sql_store_chart_label(uuid_t *chart_uuid, int source_type, char *label, char *value)
{
- sqlite3_stmt *res = NULL;
+ static __thread sqlite3_stmt *res = NULL;
int rc;
if (unlikely(!db_meta)) {
@@ -1295,10 +1344,12 @@ void sql_store_chart_label(uuid_t *chart_uuid, int source_type, char *label, cha
return;
}
- rc = sqlite3_prepare_v2(db_meta, SQL_INS_CHART_LABEL, -1, &res, 0);
- if (unlikely(rc != SQLITE_OK)) {
- error_report("Failed to prepare statement store chart labels");
- return;
+ if (unlikely(!res)) {
+ rc = prepare_statement(db_meta, SQL_INS_CHART_LABEL, &res);
+ if (unlikely(rc != SQLITE_OK)) {
+ error_report("Failed to prepare statement store chart labels");
+ return;
+ }
}
rc = sqlite3_bind_blob(res, 1, chart_uuid, sizeof(*chart_uuid), SQLITE_STATIC);
@@ -1330,8 +1381,8 @@ void sql_store_chart_label(uuid_t *chart_uuid, int source_type, char *label, cha
error_report("Failed to store chart label entry, rc = %d", rc);
failed:
- if (unlikely(sqlite3_finalize(res) != SQLITE_OK))
- error_report("Failed to finalize the prepared statement when storing chart label information");
+ if (unlikely(sqlite3_reset(res) != SQLITE_OK))
+ error_report("Failed to reset the prepared statement when storing chart label information");
return;
}
@@ -1402,12 +1453,14 @@ static RRDDIM *create_rrdim_entry(RRDSET *st, char *id, char *name, uuid_t *metr
}
#endif
-#define SELECT_CHART_CONTEXT "select d.dim_id, d.id, d.name, c.id, c.type, c.name, c.update_every, c.chart_id from chart c, " \
+#define SELECT_CHART_CONTEXT "select d.dim_id, d.id, d.name, c.id, c.type, c.name, c.update_every, c.chart_id, " \
+ "c.context, CASE WHEN d.options = 'hidden' THEN 1 else 0 END from chart c, " \
"dimension d, host h " \
"where d.chart_id = c.chart_id and c.host_id = h.host_id and c.host_id = @host_id and c.context = @context " \
"order by c.chart_id asc, c.type||c.id desc;"
-#define SELECT_CHART_SINGLE "select d.dim_id, d.id, d.name, c.id, c.type, c.name, c.update_every, c.chart_id, c.context from chart c, " \
+#define SELECT_CHART_SINGLE "select d.dim_id, d.id, d.name, c.id, c.type, c.name, c.update_every, c.chart_id, " \
+ "c.context, CASE WHEN d.options = 'hidden' THEN 1 else 0 END from chart c, " \
"dimension d, host h " \
"where d.chart_id = c.chart_id and c.host_id = h.host_id and c.host_id = @host_id and c.type||'.'||c.id = @chart " \
"order by c.chart_id asc, c.type||'.'||c.id desc;"
@@ -1501,6 +1554,8 @@ void sql_build_context_param_list(struct context_param **param_list, RRDHOST *ho
st->last_entry_t = MAX(st->last_entry_t, (*param_list)->last_entry_t);
RRDDIM *rd = create_rrdim_entry(st, (char *)sqlite3_column_text(res, 1), (char *)sqlite3_column_text(res, 2), &rrdeng_uuid);
+ if (sqlite3_column_int(res, 9) == 1)
+ rrddim_flag_set(rd, RRDDIM_FLAG_HIDDEN);
rd->next = (*param_list)->rd;
(*param_list)->rd = rd;
}
@@ -1828,25 +1883,75 @@ failed:
return rc - 1;
}
-#define SQL_SELECT_HOST_BY_NODE_ID "select host_id from node_instance where node_id = @node_id;"
+#define SQL_SELECT_HOSTNAME_BY_NODE_ID "SELECT h.hostname FROM node_instance ni, " \
+"host h WHERE ni.host_id = h.host_id AND ni.node_id = @node_id;"
-int get_host_id(uuid_t *node_id, uuid_t *host_id)
+char *get_hostname_by_node_id(char *node)
{
sqlite3_stmt *res = NULL;
+ char *hostname = NULL;
int rc;
+ rrd_rdlock();
+ RRDHOST *host = find_host_by_node_id(node);
+ rrd_unlock();
+ if (host)
+ return strdupz(host->hostname);
+
if (unlikely(!db_meta)) {
if (default_rrd_memory_mode == RRD_MEMORY_MODE_DBENGINE)
error_report("Database has not been initialized");
- return 1;
+ return NULL;
+ }
+
+ uuid_t node_id;
+ if (uuid_parse(node, node_id))
+ return NULL;
+
+ rc = sqlite3_prepare_v2(db_meta, SQL_SELECT_HOSTNAME_BY_NODE_ID, -1, &res, 0);
+ if (unlikely(rc != SQLITE_OK)) {
+ error_report("Failed to prepare statement to fetch hostname by node id");
+ return NULL;
}
- rc = sqlite3_prepare_v2(db_meta, SQL_SELECT_HOST_BY_NODE_ID, -1, &res, 0);
+ rc = sqlite3_bind_blob(res, 1, &node_id, sizeof(node_id), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK)) {
- error_report("Failed to prepare statement to select node instance information for a node");
+ error_report("Failed to bind host_id parameter to select node instance information");
+ goto failed;
+ }
+
+ rc = sqlite3_step(res);
+ if (likely(rc == SQLITE_ROW))
+ hostname = strdupz((char *)sqlite3_column_text(res, 0));
+
+failed:
+ if (unlikely(sqlite3_finalize(res) != SQLITE_OK))
+ error_report("Failed to finalize the prepared statement when search for hostname by node id");
+
+ return hostname;
+}
+
+#define SQL_SELECT_HOST_BY_NODE_ID "select host_id from node_instance where node_id = @node_id;"
+
+int get_host_id(uuid_t *node_id, uuid_t *host_id)
+{
+ static __thread sqlite3_stmt *res = NULL;
+ int rc;
+
+ if (unlikely(!db_meta)) {
+ if (default_rrd_memory_mode == RRD_MEMORY_MODE_DBENGINE)
+ error_report("Database has not been initialized");
return 1;
}
+ if (unlikely(!res)) {
+ rc = prepare_statement(db_meta, SQL_SELECT_HOST_BY_NODE_ID, &res);
+ if (unlikely(rc != SQLITE_OK)) {
+ error_report("Failed to prepare statement to select node instance information for a node");
+ return 1;
+ }
+ }
+
rc = sqlite3_bind_blob(res, 1, node_id, sizeof(*node_id), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK)) {
error_report("Failed to bind host_id parameter to select node instance information");
@@ -1858,8 +1963,8 @@ int get_host_id(uuid_t *node_id, uuid_t *host_id)
uuid_copy(*host_id, *((uuid_t *) sqlite3_column_blob(res, 0)));
failed:
- if (unlikely(sqlite3_finalize(res) != SQLITE_OK))
- error_report("Failed to finalize the prepared statement when selecting node instance information");
+ if (unlikely(sqlite3_reset(res) != SQLITE_OK))
+ error_report("Failed to reset the prepared statement when selecting node instance information");
return (rc == SQLITE_ROW) ? 0 : -1;
}
@@ -1978,7 +2083,7 @@ struct node_instance_list *get_node_list(void)
node_list = callocz(row + 1, sizeof(*node_list));
int max_rows = row;
row = 0;
- rrd_wrlock();
+ rrd_rdlock();
while (sqlite3_step(res) == SQLITE_ROW) {
if (sqlite3_column_bytes(res, 0) == sizeof(uuid_t))
uuid_copy(node_list[row].node_id, *((uuid_t *)sqlite3_column_blob(res, 0)));
@@ -2011,7 +2116,7 @@ failed:
void sql_load_node_id(RRDHOST *host)
{
- sqlite3_stmt *res = NULL;
+ static __thread sqlite3_stmt *res = NULL;
int rc;
if (unlikely(!db_meta)) {
@@ -2020,11 +2125,13 @@ void sql_load_node_id(RRDHOST *host)
return;
}
- rc = sqlite3_prepare_v2(db_meta, SQL_GET_HOST_NODE_ID, -1, &res, 0);
- if (unlikely(rc != SQLITE_OK)) {
- error_report("Failed to prepare statement to fetch node id");
- return;
- };
+ if (unlikely(!res)) {
+ rc = prepare_statement(db_meta, SQL_GET_HOST_NODE_ID, &res);
+ if (unlikely(rc != SQLITE_OK)) {
+ error_report("Failed to prepare statement to fetch node id");
+ return;
+ };
+ }
rc = sqlite3_bind_blob(res, 1, &host->host_uuid, sizeof(host->host_uuid), SQLITE_STATIC);
if (unlikely(rc != SQLITE_OK)) {
@@ -2041,8 +2148,8 @@ void sql_load_node_id(RRDHOST *host)
}
failed:
- if (unlikely(sqlite3_finalize(res) != SQLITE_OK))
- error_report("Failed to finalize the prepared statement when loading node instance information");
+ if (unlikely(sqlite3_reset(res) != SQLITE_OK))
+ error_report("Failed to reset the prepared statement when loading node instance information");
return;
};
diff --git a/database/sqlite/sqlite_functions.h b/database/sqlite/sqlite_functions.h
index 3e41f6aaa..30b8dee6c 100644
--- a/database/sqlite/sqlite_functions.h
+++ b/database/sqlite/sqlite_functions.h
@@ -98,4 +98,6 @@ extern void invalidate_node_instances(uuid_t *host_id, uuid_t *claim_id);
extern struct node_instance_list *get_node_list(void);
extern void sql_load_node_id(RRDHOST *host);
extern void compute_chart_hash(RRDSET *st);
+extern int sql_set_dimension_option(uuid_t *dim_uuid, char *option);
+char *get_hostname_by_node_id(char *node_id);
#endif //NETDATA_SQLITE_FUNCTIONS_H
diff --git a/database/sqlite/sqlite_health.c b/database/sqlite/sqlite_health.c
index 27c67c3aa..8ba95628f 100644
--- a/database/sqlite/sqlite_health.c
+++ b/database/sqlite/sqlite_health.c
@@ -900,7 +900,8 @@ int sql_store_alert_config_hash(uuid_t *hash_id, struct alert_config *cfg)
#endif
int alert_hash_and_store_config(
uuid_t hash_id,
- struct alert_config *cfg)
+ struct alert_config *cfg,
+ int store_hash)
{
#if !defined DISABLE_CLOUD && defined ENABLE_HTTPS
EVP_MD_CTX *evpctx;
@@ -946,10 +947,12 @@ int alert_hash_and_store_config(
uuid_copy(hash_id, *((uuid_t *)&hash_value));
/* store everything, so it can be recreated when not in memory or just a subset ? */
- (void)sql_store_alert_config_hash( (uuid_t *)&hash_value, cfg);
+ if (store_hash)
+ (void)sql_store_alert_config_hash( (uuid_t *)&hash_value, cfg);
#else
UNUSED(hash_id);
UNUSED(cfg);
+ UNUSED(store_hash);
#endif
return 1;
diff --git a/database/sqlite/sqlite_health.h b/database/sqlite/sqlite_health.h
index 3b9460897..ef837894a 100644
--- a/database/sqlite/sqlite_health.h
+++ b/database/sqlite/sqlite_health.h
@@ -12,6 +12,6 @@ extern void sql_health_alarm_log_update(RRDHOST *host, ALARM_ENTRY *ae);
extern void sql_health_alarm_log_insert(RRDHOST *host, ALARM_ENTRY *ae);
extern void sql_health_alarm_log_save(RRDHOST *host, ALARM_ENTRY *ae);
extern void sql_health_alarm_log_cleanup(RRDHOST *host);
-extern int alert_hash_and_store_config(uuid_t hash_id, struct alert_config *cfg);
+extern int alert_hash_and_store_config(uuid_t hash_id, struct alert_config *cfg, int store_hash);
extern void sql_aclk_alert_clean_dead_entries(RRDHOST *host);
#endif //NETDATA_SQLITE_HEALTH_H