diff --git a/bdb/phys_rep_lsn.c b/bdb/phys_rep_lsn.c index 10302ab158..290a5565bf 100644 --- a/bdb/phys_rep_lsn.c +++ b/bdb/phys_rep_lsn.c @@ -250,9 +250,13 @@ uint32_t get_next_offset(DB_ENV *dbenv, LOG_INFO log_info) return log_info.offset + log_info.size + dbenv->get_log_header_size(dbenv); } -int apply_log(DB_ENV *dbenv, unsigned int file, unsigned int offset, int64_t rectype, void *blob, int blob_len) +int apply_log(bdb_state_type *bdb_state, unsigned int file, unsigned int offset, int64_t rectype, void *blob, + int blob_len) { - return dbenv->apply_log(dbenv, file, offset, rectype, blob, blob_len); + BDB_READLOCK("apply_log"); + int rc = bdb_state->dbenv->apply_log(bdb_state->dbenv, file, offset, rectype, blob, blob_len); + BDB_RELLOCK(); + return rc; } int truncate_log_lock(bdb_state_type *bdb_state, unsigned int file, unsigned int offset, uint32_t flags) @@ -506,7 +510,8 @@ LOG_INFO find_match_lsn(void *in_bdb_state, cdb2_hndl_tp *repl_db, LOG_INFO star physrep_logmsg(LOGMSG_FATAL, "Require elect-highest-committed-gen on source- exiting\n"); exit(1); } - physrep_logmsg(LOGMSG_ERROR, "Require elect-highest-committed-gen source\n"); + physrep_logmsg(LOGMSG_ERROR, "Require elect-highest-committed-gen source {%d:%d}\n", info.file, + info.offset); } else { info.gen = *gen; } @@ -564,7 +569,7 @@ int physrep_bdb_wait_for_seqnum(bdb_state_type *bdb_state, DB_LSN *lsn, void *da return 0; } - seqnum_type seqnum; + seqnum_type seqnum = {0}; seqnum.lsn.file = lsn->file; seqnum.lsn.offset = lsn->offset; // seqnum.issue_time = ? diff --git a/bdb/phys_rep_lsn.h b/bdb/phys_rep_lsn.h index 94bd7a6624..36dbbdbf21 100644 --- a/bdb/phys_rep_lsn.h +++ b/bdb/phys_rep_lsn.h @@ -35,8 +35,8 @@ int physrep_list_ignored_tables(void); LOG_INFO get_last_lsn(struct bdb_state_tag *); LOG_INFO get_first_lsn(struct bdb_state_tag *); uint32_t get_next_offset(struct __db_env *, LOG_INFO log_info); -int apply_log(struct __db_env *, unsigned int file, unsigned int offset, - int64_t rectype, void *blob, int blob_len); +int apply_log(struct bdb_state_tag *, unsigned int file, unsigned int offset, int64_t rectype, void *blob, + int blob_len); int truncate_log_lock(struct bdb_state_tag *, unsigned int file, unsigned int offset, uint32_t flags); int find_log_timestamp(struct bdb_state_tag *, time_t time, unsigned int *file, diff --git a/berkdb/rep/rep_record.c b/berkdb/rep/rep_record.c index 286df2fd8f..5060d25975 100644 --- a/berkdb/rep/rep_record.c +++ b/berkdb/rep/rep_record.c @@ -8408,9 +8408,8 @@ __rep_verify_match(dbenv, rp, savetime, online) MUTEX_UNLOCK(dbenv, db_rep->rep_mutexp); } - /* Masters must run full recovery */ int i_am_master = F_ISSET(rep, REP_F_MASTER); - if (gbl_rep_skip_recovery && !i_am_master && log_compare(&dbenv->prev_commit_lsn, &rp->lsn) <= 0) { + if (gbl_rep_skip_recovery && !i_am_master && dbenv->prev_commit_lsn.file > 0 && log_compare(&dbenv->prev_commit_lsn, &rp->lsn) <= 0) { DB_TXNREGION *region; region = ((DB_TXNMGR *)dbenv->tx_handle)->reginfo.primary; dbenv->wrlock_recovery_lock(dbenv, __func__, __LINE__); @@ -8467,7 +8466,7 @@ __rep_verify_match(dbenv, rp, savetime, online) /* Recovery cleanup */ if (dbenv->rep_recovery_cleanup) - dbenv->rep_recovery_cleanup(dbenv, &trunclsn, i_am_master /* 0 */); + dbenv->rep_recovery_cleanup(dbenv, &trunclsn, i_am_master); dbenv->unlock_recovery_lock(dbenv, __func__, __LINE__); @@ -8485,6 +8484,7 @@ __rep_verify_match(dbenv, rp, savetime, online) logmsg(LOGMSG_WARN, "skip-recovery cannot skip, prev-commit=[%d:%d] trunc-lsn=[%d:%d]\n", dbenv->prev_commit_lsn.file, dbenv->prev_commit_lsn.offset, rp->lsn.file, rp->lsn.offset); } + ZERO_LSN(dbenv->prev_commit_lsn); if ((ret = __rep_dorecovery(dbenv, &rp->lsn, &trunclsn, online, &undid_schema_change)) != 0) { Pthread_mutex_unlock(&apply_lk); diff --git a/db/db_metrics.c b/db/db_metrics.c index cf9399831c..354ff6cef1 100644 --- a/db/db_metrics.c +++ b/db/db_metrics.c @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -107,6 +108,13 @@ struct comdb2_metrics_store { int64_t auth_denied; double watchdog_time; + int64_t page_reads; + int64_t page_writes; + int64_t fsyncs; + int64_t mempgets; + int64_t page_bytes_read; + int64_t page_bytes_written; + /* Legacy request metrics */ int64_t fastsql_execute_inline_params; int64_t fastsql_set_isolation_level; @@ -139,151 +147,125 @@ static struct comdb2_metrics_store stats; Please keep'em sorted. */ comdb2_metric gbl_metrics[] = { - {"cache_hits", "Buffer pool hits", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.cache_hits, NULL}, - {"cache_misses", "Buffer pool misses", - (int64_t)STATISTIC_COLLECTION_TYPE_CUMULATIVE, (int64_t)STATISTIC_INTEGER, + {"cache_hits", "Buffer pool hits", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.cache_hits, + NULL}, + {"cache_misses", "Buffer pool misses", (int64_t)STATISTIC_COLLECTION_TYPE_CUMULATIVE, (int64_t)STATISTIC_INTEGER, &stats.cache_misses, NULL}, - {"cache_hit_rate", "Buffer pool request hit rate", STATISTIC_DOUBLE, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.cache_hit_rate, NULL}, - {"commits", "Number of commits", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.commits, NULL}, - {"concurrent_sql", "Concurrent SQL queries", STATISTIC_DOUBLE, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.concurrent_sql, NULL}, - {"concurrent_connections", "Number of concurrent connections ", - STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, + {"cache_hit_rate", "Buffer pool request hit rate", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.cache_hit_rate, NULL}, + {"commits", "Number of commits", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.commits, NULL}, + {"concurrent_sql", "Concurrent SQL queries", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.concurrent_sql, NULL}, + {"concurrent_connections", "Number of concurrent connections ", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, &stats.concurrent_connections, NULL}, - {"connections", "Total connections", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.connections, NULL}, - {"connection_timeouts", "Timed out connection attempts", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.connection_timeouts, NULL}, + {"connections", "Total connections", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.connections, + NULL}, + {"connection_timeouts", "Timed out connection attempts", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.connection_timeouts, NULL}, {"connection_to_sql_ratio", "Ratio of total number of connections to sql " "(and nonsql) request counts", - STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.connection_to_sql_ratio, NULL}, - {"cpu_percent", "Database CPU time over last 5 seconds", STATISTIC_DOUBLE, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.cpu_percent, NULL}, - {"current_connections", "Number of current connections", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.current_connections, NULL}, - {"deadlocks", "Number of deadlocks", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.deadlocks, NULL}, - {"locks_aborted", "Number of locks aborted", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.locks_aborted, NULL}, - {"diskspace", "Disk space used (bytes)", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.diskspace, NULL}, - {"fstraps", "Number of socket requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fstraps, NULL}, - {"ismaster", "Is this machine the current master", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.ismaster, NULL}, - {"lockrequests", "Total lock requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.lockrequests, NULL}, - {"lockwaits", "Number of lock waits", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.lockwaits, NULL}, - {"lockwait_time", "Time spent in lock waits (us)", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.lock_wait_time_us, NULL}, - {"memory_ulimit", "Virtual address space ulimit", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.memory_ulimit, NULL}, - {"memory_usage", "Address space size", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.memory_usage, NULL}, - {"preads", "Number of pread()'s", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.preads, NULL}, - {"pwrites", "Number of pwrite()'s", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.pwrites, NULL}, - {"queue_depth", "Request queue depth", STATISTIC_DOUBLE, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.queue_depth, NULL}, - {"retries", "Number of retries", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.retries, NULL}, - {"service_time", "Service time", STATISTIC_DOUBLE, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.service_time, NULL}, - {"sql_cost", "Number of sql steps executed (cost)", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.sql_cost, NULL}, - {"sql_count", "Number of sql queries executed", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.sql_count, NULL}, + STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, &stats.connection_to_sql_ratio, NULL}, + {"cpu_percent", "Database CPU time over last 5 seconds", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.cpu_percent, NULL}, + {"current_connections", "Number of current connections", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.current_connections, NULL}, + {"deadlocks", "Number of deadlocks", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.deadlocks, + NULL}, + {"locks_aborted", "Number of locks aborted", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.locks_aborted, NULL}, + {"diskspace", "Disk space used (bytes)", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.diskspace, + NULL}, + {"fstraps", "Number of socket requests", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fstraps, + NULL}, + {"ismaster", "Is this machine the current master", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.ismaster, NULL}, + {"lockrequests", "Total lock requests", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.lockrequests, NULL}, + {"lockwaits", "Number of lock waits", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.lockwaits, + NULL}, + {"lockwait_time", "Time spent in lock waits (us)", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.lock_wait_time_us, NULL}, + {"memory_ulimit", "Virtual address space ulimit", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.memory_ulimit, NULL}, + {"memory_usage", "Address space size", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.memory_usage, + NULL}, + {"preads", "Number of pread()'s", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.preads, NULL}, + {"pwrites", "Number of pwrite()'s", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.pwrites, NULL}, + {"queue_depth", "Request queue depth", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, &stats.queue_depth, + NULL}, + {"retries", "Number of retries", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.retries, NULL}, + {"service_time", "Service time", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, &stats.service_time, NULL}, + {"sql_cost", "Number of sql steps executed (cost)", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.sql_cost, NULL}, + {"sql_count", "Number of sql queries executed", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.sql_count, NULL}, {"sql_ssl_count", "Number of sql queries executed, via SSL", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.sql_ssl_count, NULL}, - {"start_time", "Server start time", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.start_time, NULL}, - {"threads", "Number of threads", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.threads, NULL}, - {"num_sc_done", "Number of schema changes done", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.num_sc_done, NULL}, - {"checkpoint_ms", "Time taken for last checkpoint", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.last_checkpoint_ms}, - {"checkpoint_total_ms", "Total time taken for checkpoints", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + {"start_time", "Server start time", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.start_time, NULL}, + {"threads", "Number of threads", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.threads, NULL}, + {"num_sc_done", "Number of schema changes done", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.num_sc_done, NULL}, + {"checkpoint_ms", "Time taken for last checkpoint", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.last_checkpoint_ms}, + {"checkpoint_total_ms", "Total time taken for checkpoints", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.total_checkpoint_ms}, - {"checkpoint_count", "Total number of checkpoints taken", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.checkpoint_count}, - {"rcache_hits", "Count of root-page cache hits", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.rcache_hits}, - {"rcache_misses", "Count of root-page cache misses", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.rcache_misses}, - {"last_election_ms", "Time taken to resolve last election", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + {"checkpoint_count", "Total number of checkpoints taken", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.checkpoint_count}, + {"rcache_hits", "Count of root-page cache hits", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.rcache_hits}, + {"rcache_misses", "Count of root-page cache misses", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.rcache_misses}, + {"last_election_ms", "Time taken to resolve last election", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.last_election_ms, NULL}, - {"total_election_ms", "Total time taken to resolve elections", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.total_election_ms, NULL}, - {"election_count", "Total number of elections", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.election_count, NULL}, - {"last_election_time", "Wallclock time last election completed", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.last_election_time, NULL}, - {"udp_sent", "Number of udp packets sent", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.udp_sent, NULL}, - {"udp_failed_send", "Number of failed udp sends", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.udp_failed_send, NULL}, - {"udp_received", "Number of udp receives", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.udp_received, NULL}, - {"active_transactions", "Number of active transactions", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.active_transactions, NULL}, - {"max_active_transactions", "Maximum number of active transactions", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.maxactive_transactions, NULL}, - {"total_commits", "Number of transaction commits", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.total_commits, NULL}, - {"total_aborts", "Number of transaction aborts", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.total_aborts, NULL}, - {"sql_queue_time", "Average ms spent waiting in sql queue", - STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, &stats.sql_queue_time, + {"total_election_ms", "Total time taken to resolve elections", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.total_election_ms, NULL}, + {"election_count", "Total number of elections", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.election_count, NULL}, + {"last_election_time", "Wallclock time last election completed", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.last_election_time, NULL}, + {"udp_sent", "Number of udp packets sent", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.udp_sent, NULL}, - {"sql_queue_timeouts", "Number of sql items timed-out waiting on queue", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.sql_queue_timeouts, NULL}, - {"handle_buf_queue_time", "Average ms spent waiting in handle-buf queue", - STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.handle_buf_queue_time, NULL}, - {"denied_appsocks", "Number of denied appsock connections", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + {"udp_failed_send", "Number of failed udp sends", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.udp_failed_send, NULL}, + {"udp_received", "Number of udp receives", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.udp_received, NULL}, + {"active_transactions", "Number of active transactions", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.active_transactions, NULL}, + {"max_active_transactions", "Maximum number of active transactions", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.maxactive_transactions, NULL}, + {"total_commits", "Number of transaction commits", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.total_commits, NULL}, + {"total_aborts", "Number of transaction aborts", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.total_aborts, NULL}, + {"sql_queue_time", "Average ms spent waiting in sql queue", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.sql_queue_time, NULL}, + {"sql_queue_timeouts", "Number of sql items timed-out waiting on queue", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.sql_queue_timeouts, NULL}, + {"handle_buf_queue_time", "Average ms spent waiting in handle-buf queue", STATISTIC_DOUBLE, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.handle_buf_queue_time, NULL}, + {"denied_appsocks", "Number of denied appsock connections", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.denied_appsock_connections, NULL}, - {"locks", "Number of currently held locks", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.locks, NULL}, - {"temptable_created", "Number of temporary tables created", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + {"locks", "Number of currently held locks", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.locks, + NULL}, + {"temptable_created", "Number of temporary tables created", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.temptable_created, NULL}, - {"temptable_create_requests", "Number of create temporary table requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.temptable_create_reqs, NULL}, - {"temptable_spills", - "Number of temporary tables that had to be spilled to disk-backed tables", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.temptable_spills, NULL}, - {"net_drops", - "Number of packets that didn't fit on network queue and were dropped", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.net_drops, + {"temptable_create_requests", "Number of create temporary table requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.temptable_create_reqs, NULL}, + {"temptable_spills", "Number of temporary tables that had to be spilled to disk-backed tables", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.temptable_spills, NULL}, + {"net_drops", "Number of packets that didn't fit on network queue and were dropped", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.net_drops, NULL}, + {"net_queue_size", "Size of largest outgoing net queue", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.net_queue_size, NULL}, + {"rep_deadlocks", "Replication deadlocks", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.rep_deadlocks, NULL}, + {"rw_evictions", "Dirty page evictions", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.rw_evicts, NULL}, - {"net_queue_size", "Size of largest outgoing net queue", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.net_queue_size, NULL}, - {"rep_deadlocks", "Replication deadlocks", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.rep_deadlocks, NULL}, - {"rw_evictions", "Dirty page evictions", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.rw_evicts, NULL}, - {"standing_queue_time", "How long the database has had a standing queue", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.standing_queue_time, NULL}, - {"nonsql", "Number of non-sql requests (eg: tagged)", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.nonsql, NULL}, + {"standing_queue_time", "How long the database has had a standing queue", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.standing_queue_time, NULL}, + {"nonsql", "Number of non-sql requests (eg: tagged)", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.nonsql, NULL}, #if 0 {"minimum_truncation_file", "Minimum truncation file", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.minimum_truncation_file, NULL}, @@ -294,103 +276,78 @@ comdb2_metric gbl_metrics[] = { STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.minimum_truncation_timestamp, NULL}, #endif - {"reprepares", "Number of times statements are reprepared by sqlitex", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.reprepares, - NULL}, - {"verify_replays", "Number of replays on verify errors", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.vreplays, NULL}, - {"nsslfullhandshakes", "Number of SSL full handshakes", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.nsslfullhandshakes, NULL}, - {"nsslpartialhandshakes", "Number of SSL partial handshakes", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.nsslpartialhandshakes, NULL}, - {"weighted_queue_depth", "Weighted queue depth", STATISTIC_DOUBLE, - STATISTIC_COLLECTION_TYPE_LATEST, &stats.weighted_queue_depth, NULL}, - {"weighted_standing_queue_time", - "How long the database has had a weighted standing queue", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, - &stats.weighted_standing_queue_time, NULL}, - {"auth_allowed", "Number of successful authentication requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.auth_allowed, - NULL}, - {"auth_denied", "Number of failed authentication requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.auth_denied, - NULL}, + {"reprepares", "Number of times statements are reprepared by sqlitex", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.reprepares, NULL}, + {"verify_replays", "Number of replays on verify errors", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.vreplays, NULL}, + {"nsslfullhandshakes", "Number of SSL full handshakes", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.nsslfullhandshakes, NULL}, + {"nsslpartialhandshakes", "Number of SSL partial handshakes", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.nsslpartialhandshakes, NULL}, + {"weighted_queue_depth", "Weighted queue depth", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.weighted_queue_depth, NULL}, + {"weighted_standing_queue_time", "How long the database has had a weighted standing queue", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.weighted_standing_queue_time, NULL}, + {"auth_allowed", "Number of successful authentication requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_LATEST, &stats.auth_allowed, NULL}, + {"auth_denied", "Number of failed authentication requests", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.auth_denied, NULL}, {"watchdog_time", "Number of seconds for a successful watchdog test run", STATISTIC_DOUBLE, STATISTIC_COLLECTION_TYPE_LATEST, &stats.watchdog_time, NULL}, - - {"fastsql_execute_inline_params", "Number of fastsql 'execute' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_execute_inline_params, NULL}, - {"fastsql_set_isolation_level", - "Number of fastsql 'set isolation level' requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_isolation_level, - NULL}, - {"fastsql_set_timeout", "Number of fastsql 'set timeout' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_timeout, NULL}, - {"fastsql_set_info", "Number of fastsql 'set info' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_info, NULL}, - {"fastsql_execute_inline_params_tz", - "Number of fastsql 'execute with timezone' requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_execute_inline_params_tz, NULL}, - {"fastsql_set_heartbeat", "Number of fastsql 'set heartbeat' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_heartbeat, NULL}, - {"fastsql_pragma", "Number of fastsql pragma requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_pragma, NULL}, - {"fastsql_reset", "Number of fastsql reset requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_reset, NULL}, - {"fastsql_execute_replaceable_params", - "Number of fastsql 'execute with replacable parameters' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_execute_replaceable_params, NULL}, - {"fastsql_set_sql_debug", "Number of fastsql 'set sql debug' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_sql_debug, NULL}, - {"fastsql_grab_dbglog", "Number of fastsql 'grab dbglog' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_grab_dbglog, NULL}, - {"fastsql_set_user", "Number of fastsql 'set user' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_user, NULL}, - {"fastsql_set_password", "Number of fastsql 'set password' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_password, NULL}, - {"fastsql_set_endian", "Number of fastsql 'set endian' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_endian, NULL}, + {"page_reads", "Total page reads", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.page_reads, NULL}, + {"page_writes", "Total page writes", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.page_writes, NULL}, + {"fsyncs", "Fsync count", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.fsyncs, NULL}, + {"mempgets", "Mempget count", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, &stats.mempgets, NULL}, + {"page_bytes_read", "Total page bytes read", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.page_bytes_read, NULL}, + {"page_bytes_written", "Total page bytes written", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_LATEST, + &stats.page_bytes_written, NULL}, + {"fastsql_execute_inline_params", "Number of fastsql 'execute' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_execute_inline_params, NULL}, + {"fastsql_set_isolation_level", "Number of fastsql 'set isolation level' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_isolation_level, NULL}, + {"fastsql_set_timeout", "Number of fastsql 'set timeout' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_timeout, NULL}, + {"fastsql_set_info", "Number of fastsql 'set info' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_info, NULL}, + {"fastsql_execute_inline_params_tz", "Number of fastsql 'execute with timezone' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_execute_inline_params_tz, NULL}, + {"fastsql_set_heartbeat", "Number of fastsql 'set heartbeat' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_heartbeat, NULL}, + {"fastsql_pragma", "Number of fastsql pragma requests", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.fastsql_pragma, NULL}, + {"fastsql_reset", "Number of fastsql reset requests", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + &stats.fastsql_reset, NULL}, + {"fastsql_execute_replaceable_params", "Number of fastsql 'execute with replacable parameters' requests", + STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_execute_replaceable_params, NULL}, + {"fastsql_set_sql_debug", "Number of fastsql 'set sql debug' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_sql_debug, NULL}, + {"fastsql_grab_dbglog", "Number of fastsql 'grab dbglog' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_grab_dbglog, NULL}, + {"fastsql_set_user", "Number of fastsql 'set user' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_user, NULL}, + {"fastsql_set_password", "Number of fastsql 'set password' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_password, NULL}, + {"fastsql_set_endian", "Number of fastsql 'set endian' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_endian, NULL}, {"fastsql_execute_replaceable_params_tz", "Number of fastsql 'execute with replacable parameters & timezone' " "requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_execute_replaceable_params_tz, NULL}, - {"fastsql_get_effects", "Number of fastsql 'get effects' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_get_effects, NULL}, - {"fastsql_set_planner_effort", - "Number of fastsql 'set planner effort' requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_planner_effort, - NULL}, - {"fastsql_set_remote_access", - "Number of fastsql 'set remote access' requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_remote_access, - NULL}, - {"fastsql_osql_max_trans", "Number of fastsql 'osql max trans' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_osql_max_trans, NULL}, - {"fastsql_set_datetime_precision", - "Number of fastsql 'set datetime precision' requests", STATISTIC_INTEGER, - STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_set_datetime_precision, NULL}, - {"fastsql_sslconn", "Number of fastsql 'sslconn' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, + STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_execute_replaceable_params_tz, NULL}, + {"fastsql_get_effects", "Number of fastsql 'get effects' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_get_effects, NULL}, + {"fastsql_set_planner_effort", "Number of fastsql 'set planner effort' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_planner_effort, NULL}, + {"fastsql_set_remote_access", "Number of fastsql 'set remote access' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_remote_access, NULL}, + {"fastsql_osql_max_trans", "Number of fastsql 'osql max trans' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_osql_max_trans, NULL}, + {"fastsql_set_datetime_precision", "Number of fastsql 'set datetime precision' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_set_datetime_precision, NULL}, + {"fastsql_sslconn", "Number of fastsql 'sslconn' requests", STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_sslconn, NULL}, - {"fastsql_execute_stop", "Number of fastsql 'execute stop' requests", - STATISTIC_INTEGER, STATISTIC_COLLECTION_TYPE_CUMULATIVE, - &stats.fastsql_execute_stop, NULL}, + {"fastsql_execute_stop", "Number of fastsql 'execute stop' requests", STATISTIC_INTEGER, + STATISTIC_COLLECTION_TYPE_CUMULATIVE, &stats.fastsql_execute_stop, NULL}, }; const char *metric_collection_type_string(comdb2_collection_type t) { @@ -656,6 +613,17 @@ int refresh_metrics(void) time_metric_average(thedb->handle_buf_queue_time); stats.concurrent_connections = time_metric_average(thedb->connections); stats.watchdog_time = time_metric_average(thedb->watchdog_time); + struct global_stats gstats = {0}; + + global_request_stats(&gstats); + + stats.page_reads = gstats.page_reads; + stats.page_writes = gstats.page_writes; + stats.fsyncs = gstats.fsyncs; + stats.mempgets = gstats.mempgets; + stats.page_bytes_read = gstats.page_bytes_read; + stats.page_bytes_written = gstats.page_bytes_written; + int master = bdb_whoismaster((bdb_state_type *)thedb->bdb_env) == gbl_myhostname ? 1 : 0; diff --git a/db/db_tunables.c b/db/db_tunables.c index acf2092f40..32562eac52 100644 --- a/db/db_tunables.c +++ b/db/db_tunables.c @@ -453,7 +453,7 @@ extern int gbl_physrep_keepalive_freq_sec; extern int gbl_physrep_max_candidates; extern int gbl_physrep_max_pending_replicants; extern int gbl_physrep_reconnect_penalty; -extern int gbl_physrep_register_interval; +extern int gbl_physrep_reconnect_interval; extern int gbl_physrep_shuffle_host_list; extern int gbl_physrep_ignore_queues; extern int gbl_physrep_max_rollback; diff --git a/db/db_tunables.h b/db/db_tunables.h index 3159f9db64..a5d3243ea5 100644 --- a/db/db_tunables.h +++ b/db/db_tunables.h @@ -1871,10 +1871,8 @@ REGISTER_TUNABLE("match_on_ckp", NULL, NULL, NULL, NULL); /* physical replication */ -REGISTER_TUNABLE("blocking_physrep", - "Physical replicant blocks on select. (Default: false)", - TUNABLE_BOOLEAN, &gbl_blocking_physrep, 0, NULL, NULL, NULL, - NULL); +REGISTER_TUNABLE("blocking_physrep", "Physical replicant blocks on select. (Default: off)", TUNABLE_BOOLEAN, + &gbl_blocking_physrep, 0, NULL, NULL, NULL, NULL); REGISTER_TUNABLE("tranlog_default_timeout", "Default timeout for tranlog queries. (Default: 30)", TUNABLE_INTEGER, &gbl_tranlog_default_timeout, 0, NULL, NULL, NULL, NULL); REGISTER_TUNABLE("tranlog_incoherent_timeout", "Timeout in seconds for incoherent tranlog. (Default: 10)", @@ -1931,8 +1929,8 @@ REGISTER_TUNABLE("physrep_reconnect_penalty", "Physrep wait seconds before retry to the same node. (Default: 5)", TUNABLE_INTEGER, &gbl_physrep_reconnect_penalty, 0, NULL, NULL, NULL, NULL); -REGISTER_TUNABLE("physrep_register_interval", "Interval for physical replicant re-registration. (Default: 600)", - TUNABLE_INTEGER, &gbl_physrep_register_interval, 0, NULL, NULL, NULL, NULL); +REGISTER_TUNABLE("physrep_reconnect_interval", "Reconnect interval for physical replicants (Default: 600)", + TUNABLE_INTEGER, &gbl_physrep_reconnect_interval, 0, NULL, NULL, NULL, NULL); REGISTER_TUNABLE("physrep_shuffle_host_list", "Shuffle the host list returned by register_replicant() " "before connecting to the hosts. (Default: OFF)", diff --git a/db/phys_rep.c b/db/phys_rep.c index 69ed44f68e..d19ebbdfdd 100644 --- a/db/phys_rep.c +++ b/db/phys_rep.c @@ -54,7 +54,7 @@ typedef struct DB_Connection { } while (0) int gbl_physrep_debug = 0; -int gbl_physrep_register_interval = 600; // force re-registration every 10 mins +int gbl_physrep_reconnect_interval = 600; // force re-registration every 10 mins int gbl_physrep_reconnect_penalty = 0; int gbl_blocking_physrep = 0; int gbl_physrep_fanout = 8; @@ -102,6 +102,7 @@ static size_t repl_dbs_sz; reverse_conn_handle_tp *rev_conn_hndl = NULL; static int last_register; +static int repl_db_connect_time; static int add_replicant_host(char *hostname, char *dbname); static void dump_replicant_hosts(void); @@ -191,6 +192,8 @@ static void set_repl_db_connected(char *dbname, char *host) gbl_physrep_repl_name = dbname; gbl_physrep_repl_host = host; repl_db_connected = 1; + repl_db_connect_time = comdb2_time_epoch(); + physrep_logmsg(LOGMSG_USER, "Physical replicant is now replicating from %s@%s\n", dbname, host); } static void set_repl_db_disconnected() @@ -635,8 +638,7 @@ static LOG_INFO handle_record(cdb2_hndl_tp *repl_db, LOG_INFO prev_info) if (stop_physrep_worker == 0) { /* check if we need to call new file flag */ if (prev_info.file < file) { - rc = apply_log(thedb->bdb_env->dbenv, prev_info.file, - get_next_offset(thedb->bdb_env->dbenv, prev_info), + rc = apply_log(thedb->bdb_env, prev_info.file, get_next_offset(thedb->bdb_env->dbenv, prev_info), REP_NEWFILE, NULL, 0); if (rc != 0) { physrep_logmsg(LOGMSG_FATAL, "%s:%d: Something went wrong with applying the logs (rc: %d)\n", @@ -645,8 +647,7 @@ static LOG_INFO handle_record(cdb2_hndl_tp *repl_db, LOG_INFO prev_info) } } - rc = apply_log(thedb->bdb_env->dbenv, file, offset, REP_LOG, blob, - blob_len); + rc = apply_log(thedb->bdb_env, file, offset, REP_LOG, blob, blob_len); if (is_commit((u_int32_t)*rectype)) { if (gbl_physrep_debug) { @@ -1271,7 +1272,7 @@ static void *physrep_worker(void *args) } if (repl_db_connected && - (force_registration() || (((now = time(NULL)) - last_register) > gbl_physrep_register_interval))) { + (force_registration() || (((now = time(NULL)) - repl_db_connect_time) > gbl_physrep_reconnect_interval))) { close_repl_connection(repl_db_cnct, repl_db, __func__, __LINE__); if (gbl_physrep_debug) { physrep_logmsg(LOGMSG_USER, "%s:%d: Forcing re-registration\n", __func__, __LINE__); @@ -1394,9 +1395,6 @@ static void *physrep_worker(void *args) cdb2_close(repl_metadb); } - physrep_logmsg(LOGMSG_USER, "Physical replicant is now replicating from %s@%s\n", - repl_db_cnct->dbname, repl_db_cnct->hostname); - if (do_truncate && repl_db) { info = get_last_lsn(thedb->bdb_env); prev_info = handle_truncation(repl_db, info); @@ -1481,6 +1479,12 @@ static void *physrep_worker(void *args) goto repl_loop; } + /* Check reconnect */ + if (force_registration() || ((now = time(NULL)) - repl_db_connect_time) > gbl_physrep_reconnect_interval) { + close_repl_connection(repl_db_cnct, repl_db, __func__, __LINE__); + goto repl_loop; + } + int update_regck = gbl_physrep_update_registry_interval; if (repl_db_connected && update_regck > 0 && (comdb2_time_epoch() - last_update_registry) > update_regck) { update_registry_periodic(repl_db_cnct->dbname, repl_db_cnct->hostname); diff --git a/db/request_stats.c b/db/request_stats.c index 97339f583c..15d6cb462e 100644 --- a/db/request_stats.c +++ b/db/request_stats.c @@ -21,6 +21,7 @@ #include #include #include +#include #include "request_stats.h" @@ -34,6 +35,8 @@ extern void __berkdb_register_fsync_callback(void (*callback)(int fd)); static void user_request_done(void *st) { free(st); } +static struct global_stats global = {0}; + void user_request_begin(enum request_type type, int flags) { struct per_request_stats *st; @@ -67,6 +70,8 @@ void user_request_fsync_callback(int fd) if (st) { st->nfsyncs++; } + + ATOMIC_ADD64(global.fsyncs, 1); } void user_request_read_callback(int bytes) @@ -81,6 +86,9 @@ void user_request_read_callback(int bytes) st->nreads++; st->readbytes += bytes; } + + ATOMIC_ADD64(global.page_reads, 1); + ATOMIC_ADD64(global.page_bytes_read, bytes); } void user_request_write_callback(int bytes) @@ -95,6 +103,9 @@ void user_request_write_callback(int bytes) st->nwrites++; st->writebytes += bytes; } + + ATOMIC_ADD64(global.page_writes, 1); + ATOMIC_ADD64(global.page_bytes_written, bytes); } void user_request_memp_callback(void) @@ -107,6 +118,18 @@ void user_request_memp_callback(void) st = pthread_getspecific(key); if (st) st->mempgets++; + + ATOMIC_ADD64(global.mempgets, 1); +} + +void global_request_stats(struct global_stats *stats) +{ + stats->page_reads = ATOMIC_LOAD64(global.page_reads); + stats->page_writes = ATOMIC_LOAD64(global.page_writes); + stats->fsyncs = ATOMIC_LOAD64(global.fsyncs); + stats->mempgets = ATOMIC_LOAD64(global.mempgets); + stats->page_bytes_read = ATOMIC_LOAD64(global.page_bytes_read); + stats->page_bytes_written = ATOMIC_LOAD64(global.page_bytes_written); } void user_request_init(void) diff --git a/db/request_stats.h b/db/request_stats.h index 185f975a8b..e66ee9d114 100644 --- a/db/request_stats.h +++ b/db/request_stats.h @@ -55,6 +55,15 @@ struct per_request_stats { long long writebytes; }; +struct global_stats { + int64_t page_reads; + int64_t page_writes; + int64_t fsyncs; + int64_t mempgets; + int64_t page_bytes_read; + int64_t page_bytes_written; +}; + void user_request_begin(enum request_type type, int flags); struct per_request_stats *user_request_get_stats(void); void user_request_end(void); @@ -62,6 +71,7 @@ void user_request_read_callback(int); void user_request_write_callback(int); void user_request_memp_callback(void); void user_request_init(void); +void global_request_stats(struct global_stats *stats); void user_request_on(void); void user_request_off(void); diff --git a/sqlite/ext/comdb2/tranlog.c b/sqlite/ext/comdb2/tranlog.c index 5742c3a50b..ca11a766fe 100644 --- a/sqlite/ext/comdb2/tranlog.c +++ b/sqlite/ext/comdb2/tranlog.c @@ -588,6 +588,8 @@ static int tranlogColumn( if (pCur->data.data) LOGCOPY_32(&rectype, pCur->data.data); + normalize_rectype(&rectype); + if (rectype == DB___txn_regop_gen){ generation = get_generation_from_regop_gen_record(pCur->data.data); } diff --git a/tests/phys_rep_perf.test/Makefile b/tests/phys_rep_perf.test/Makefile new file mode 100644 index 0000000000..6abe9fb16a --- /dev/null +++ b/tests/phys_rep_perf.test/Makefile @@ -0,0 +1,10 @@ +ifeq ($(TESTSROOTDIR),) + include ../testcase.mk +else + include $(TESTSROOTDIR)/testcase.mk +endif +export CHECK_DB_AT_FINISH=0 +ifeq ($(TEST_TIMEOUT),) + export TEST_TIMEOUT=15m +endif + diff --git a/tests/phys_rep_perf.test/README b/tests/phys_rep_perf.test/README new file mode 100644 index 0000000000..c87e394f21 --- /dev/null +++ b/tests/phys_rep_perf.test/README @@ -0,0 +1 @@ +This test compares the i/o and memory utilization (and in the future, possibly other things) against a normal database against a physrep-database, and a reverse-connection physrep-database. diff --git a/tests/phys_rep_perf.test/runit b/tests/phys_rep_perf.test/runit new file mode 100755 index 0000000000..5952e085c3 --- /dev/null +++ b/tests/phys_rep_perf.test/runit @@ -0,0 +1,551 @@ +#!/usr/bin/env bash +bash -n "$0" | exit 1 + +set -x + +source ${TESTSROOTDIR}/tools/runit_common.sh +source ${TESTSROOTDIR}/tools/cluster_utils.sh + +export comdb2ar=${COMDB2AR_EXE} +export FIGLET=$(which figlet) +export META_DBNAME=metadb +export PHYSREP=physrep +export PIDLIST="" +export FIRSTNODE="" +export FIGLET=$(which figlet) + +# Stats +declare -A master_base +declare -A master_stats + +declare -A physrep_base +declare -A physrep_stats + +declare -A master_diff +declare -A physrep_diff + +function announce +{ + typeset text=$1 + echo "$text" + [[ -z "$FIGLET" ]] && figlet -f banner $text +} + +function testcase_preamble +{ + typeset testcase=$1 + announce "Running $testcase" +} + +function testcase_finish +{ + typeset testcase=$1 + announce "Finished $testcase" +} + +function cleanup_internal() +{ + local METADIR=${DBDIR}/$META_DBNAME + typeset sig=$1 + echo "Killing pids $PIDLIST" + kill -$sig $PIDLIST + + if [[ -n "$CLUSTER" ]]; then + for node in $CLUSTER; do + pid=$(ssh $node "cat ${METADIR}/${META_DBNAME}.pid") + ssh $node "kill -$sig $$pid" < /dev/null + done + + pid=$(ssh $FIRSTNODE "cat ${DBDIR}/${PHYSREP}/${PHYSREP}.pid") + ssh $FIRSTNODE "kill -$sig $$pid" < /dev/null + else + node=$(hostname) + kill -$sig $(cat ${METADIR}/${META_DBNAME}.pid) + kill -$sig $(cat ${DBDIR}/${PHYSREP}/${PHYSREP}.pid) + fi +} + +function cleanup() +{ + cleanup_internal 9 +} + +function cleanupabort() +{ + cleanup_internal 15 +} + +function cleanfailexit() +{ + cleanup + failexit $@ +} + +function gather_metric +{ + local dbname=$1 + local host=$2 + local metric=$3 + x=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${host} ${dbname} "select value from comdb2_metrics where name='${metric}'") + echo "$x" +} + +function compare_metric +{ + local metric=$1 + local threshold=$2 + local master=$3 + local physrep=$4 + + # We 'pass' if the physrep is less than threshold% above the master + local upper=$(( master + ( (master * threshold) / 100 ) )) + if [[ $physrep -gt $upper ]]; then + echo "$metric: FAILED, more than $threshold% above master" + cleanfailexit + else + echo "$metric: PASSED master: $master physrep: $physrep within $threshold%" + fi +} + +function print_master_diff +{ + echo "Master page reads: ${master_diff["page_reads"]}" + echo "Master page writes: ${master_diff["page_writes"]}" + echo "Master fsyncs: ${master_diff["fsyncs"]}" + echo "Master mempgets: ${master_diff["mempgets"]}" + echo "Master page bytes read: ${master_diff["page_bytes_read"]}" + echo "Master page bytes written: ${master_diff["page_bytes_written"]}" +} + +function print_physrep_diff +{ + echo "Physrep page reads: ${physrep_diff["page_reads"]}" + echo "Physrep page writes: ${physrep_diff["page_writes"]}" + echo "Physrep fsyncs: ${physrep_diff["fsyncs"]}" + echo "Physrep mempgets: ${physrep_diff["mempgets"]}" + echo "Physrep page bytes read: ${physrep_diff["page_bytes_read"]}" + echo "Physrep page bytes written: ${physrep_diff["page_bytes_written"]}" +} + +function compare_metrics +{ + typeset mode=$1 + compare_metric "$mode Page reads" 20 ${master_diff["page_reads"]} ${physrep_diff["page_reads"]} + compare_metric "$mode Page writes" 20 ${master_diff["page_writes"]} ${physrep_diff["page_writes"]} + compare_metric "$mode Fsyncs" 40 ${master_diff["fsyncs"]} ${physrep_diff["fsyncs"]} + compare_metric "$mode Mempgets" 20 ${master_diff["mempgets"]} ${physrep_diff["mempgets"]} + compare_metric "$mode Page bytes read" 20 ${master_diff["page_bytes_read"]} ${physrep_diff["page_bytes_read"]} + compare_metric "$mode Page bytes written" 20 ${master_diff["page_bytes_written"]} ${physrep_diff["page_bytes_written"]} +} + +function calc_master_diff +{ + local a + local b + + a=${master_stats["page_reads"]} + b=${master_base["page_reads"]} + master_diff["page_reads"]=$(( a - b )) + + a=${master_stats["page_writes"]} + b=${master_base["page_writes"]} + master_diff["page_writes"]=$(( a - b )) + + a=${master_stats["fsyncs"]} + b=${master_base["fsyncs"]} + master_diff["fsyncs"]=$(( a - b )) + + a=${master_stats["mempgets"]} + b=${master_base["mempgets"]} + master_diff["mempgets"]=$(( a - b )) + + a=${master_stats["page_bytes_read"]} + b=${master_base["page_bytes_read"]} + master_diff["page_bytes_read"]=$(( a - b )) + + a=${master_stats["page_bytes_written"]} + b=${master_base["page_bytes_written"]} + master_diff["page_bytes_written"]=$(( a - b )) + +} + +function calc_physrep_diff +{ + local a + local b + + a=${physrep_stats["page_reads"]} + b=${physrep_base["page_reads"]} + physrep_diff["page_reads"]=$(( a - b )) + + a=${physrep_stats["page_writes"]} + b=${physrep_base["page_writes"]} + physrep_diff["page_writes"]=$(( a - b )) + + a=${physrep_stats["fsyncs"]} + b=${physrep_base["fsyncs"]} + physrep_diff["fsyncs"]=$(( a - b )) + + a=${physrep_stats["mempgets"]} + b=${physrep_base["mempgets"]} + physrep_diff["mempgets"]=$(( a - b )) + + a=${physrep_stats["page_bytes_read"]} + b=${physrep_base["page_bytes_read"]} + physrep_diff["page_bytes_read"]=$(( a - b )) + + a=${physrep_stats["page_bytes_written"]} + b=${physrep_base["page_bytes_written"]} + physrep_diff["page_bytes_written"]=$(( a - b )) +} + +function gather_physrep_base +{ + physrep_base["page_reads"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_reads")" + physrep_base["page_writes"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_writes")" + physrep_base["fsyncs"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "fsyncs")" + physrep_base["mempgets"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "mempgets")" + physrep_base["page_bytes_read"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_bytes_read")" + physrep_base["page_bytes_written"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_bytes_written")" +} + +function gather_physrep_stats +{ + physrep_stats["page_reads"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_reads")" + physrep_stats["page_writes"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_writes")" + physrep_stats["fsyncs"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "fsyncs")" + physrep_stats["mempgets"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "mempgets")" + physrep_stats["page_bytes_read"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_bytes_read")" + physrep_stats["page_bytes_written"]="$(gather_metric ${PHYSREP} ${FIRSTNODE} "page_bytes_written")" +} + +function gather_master_base +{ + master=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs ${DBNAME} default "select host from comdb2_cluster where is_master='Y'") + master_base["page_reads"]=$(gather_metric ${DBNAME} ${master} "page_reads") + master_base["page_writes"]=$(gather_metric ${DBNAME} ${master} "page_writes") + master_base["fsyncs"]=$(gather_metric ${DBNAME} ${master} "fsyncs") + master_base["mempgets"]=$(gather_metric ${DBNAME} ${master} "mempgets") + master_base["page_bytes_read"]=$(gather_metric ${DBNAME} ${master} "page_bytes_read") + master_base["page_bytes_written"]=$(gather_metric ${DBNAME} ${master} "page_bytes_written") +} + +function gather_master_stats +{ + master=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs ${DBNAME} default "select host from comdb2_cluster where is_master='Y'") + master_stats["page_reads"]="$(gather_metric ${DBNAME} ${master} "page_reads")" + master_stats["page_writes"]="$(gather_metric ${DBNAME} ${master} "page_writes")" + master_stats["fsyncs"]="$(gather_metric ${DBNAME} ${master} "fsyncs")" + master_stats["mempgets"]="$(gather_metric ${DBNAME} ${master} "mempgets")" + master_stats["page_bytes_read"]="$(gather_metric ${DBNAME} ${master} "page_bytes_read")" + master_stats["page_bytes_written"]="$(gather_metric ${DBNAME} ${master} "page_bytes_written")" +} + +function create_test_table +{ + $CDB2SQL_EXE $CDB2_OPTIONS $DBNAME default "CREATE TABLE IF NOT EXISTS t1(a INT)" + count=0 + while [[ "$count" == "0" ]]; do + count=$($CDB2SQL_EXE $CDB2_OPTIONS --tabs $PHYSREP --host $FIRSTNODE "select count(*) from comdb2_tables where tablename='t1'") + sleep 1 + done +} + +function create_physrep_tables() +{ + local host=$1 + $CDB2SQL_EXE $CDB2_OPTIONS --host $host ${META_DBNAME} "CREATE TABLE comdb2_physreps(dbname CStriNG(60), host CSTRING(120), file INT, offset INT, firstfile INT, last_keepalive DATETIME, state CSTRING(60), UNIQUE(dbname,host))" + if [[ $? -ne 0 ]]; then + cleanfailexit "Failed to create table comdb2_physreps" + fi + $CDB2SQL_EXE ${CDB2_OPTIONS} --host $host $META_DBNAME "CREATE TABLE comdb2_physrep_connections(dbname CSTRING(60), host CSTRING(120), source_dbname CSTRING(60), source_host CSTRING(120), UNIQUE (dbname, host, source_dbname, source_host), FOREIGN KEY(dbname, host) REFERENCES comdb2_physreps(dbname, host) ON DELETE CASCADE)" + if [[ $? -ne 0 ]]; then + cleanfailexit "Failed to create table comdb2_physrep_connections" + fi + + $CDB2SQL_EXE ${CDB2_OPTIONS} --host $host $META_DBNAME "CREATE TABLE comdb2_physrep_sources(dbname CSTRING(60), host CSTRING(120), source_dbname CSTRING(60), source_host CSTRING(120), UNIQUE (dbname, host, source_dbname, source_host))" + + if [[ $? -ne 0 ]]; then + cleanfailexit "Failed to create table comdb2_physrep_sources" + fi + echo "Physrep tables created" +} + +function fix_lrl_and_restart_source_nodes +{ + if [[ -z "$CLUSTER" ]]; then + node=$(hostname) + echo "physrep_metadb ${META_DBNAME} $FIRSTNODE" >> ${DBDIR}/${DBNAME}.lrl + echo "physrep_debug 1" >> ${DBDIR}/${DBNAME}.lrl + kill_restart_node $node 10 & + sleep 10 + + out="" + while [[ "$out" != "1" ]]; do + out=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${node} ${DBNAME} 'select 1' 2>/dev/null) + sleep 1 + done + + else + for node in $CLUSTER; do + ssh ${node} "echo physrep_metadb ${META_DBNAME} $FIRSTNODE >> ${DBDIR}/${DBNAME}.lrl" < /dev/null + ssh ${node} "echo physrep_debug 1 >> ${DBDIR}/${DBNAME}.lrl" < /dev/null + echo "physrep_metadb ${META_DBNAME} $FIRSTNODE" >> ${DBDIR}/${DBNAME}.lrl + echo "physrep_debug 1" >> ${DBDIR}/${DBNAME}.lrl + kill_restart_node $node 10 & + done + sleep 10 + for node in $CLUSTER ; do + out="" + while [[ "$out" != "1" ]]; do + out=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${node} ${DBNAME} 'select 1' 2>/dev/null) + sleep 1 + done + done + fi +} + +function setup_physrep_metadb() +{ + local METADIR=${DBDIR}/$META_DBNAME + + mkdir -p $METADIR + if [[ -z "$CLUSTER" ]]; then + node=$(hostname) + logfile=$TESTDIR/logs/${META_DBNAME}.${node}.log + cat <> $METADIR/${META_DBNAME}.lrl +name ${META_DBNAME} +dir ${METADIR} +physrep_register_interval 10 +revsql_debug 1 +physrep_debug 1 +tranlog_incoherent_timeout 10 +forbid_remote_admin 0 +logmsg level debug +ctrace_dbdir 1 +allow_lua_print 1 +physrep_update_registry_interval 1 +physrep_keepalive_v2 1 +END + + $COMDB2_EXE ${META_DBNAME} --create --lrl ${METADIR}/${META_DBNAME}.lrl --pidfile ${METADIR}/${META_DBNAME}.pid >> $logfile 2>&1 + $COMDB2_EXE ${META_DBNAME} --lrl ${METADIR}/${META_DBNAME}.lrl --pidfile ${METADIR}/${META_DBNAME}.pid >> $logfile 2>&1 & + echo "$META_DBNAME: waiting until ready" + while [[ "$out" != "1" ]]; do + out=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${node} ${META_DBNAME} 'select 1' 2>/dev/null) + sleep 1 + done + PIDLIST="${PIDLIST} $(cat ${METADIR}/${META_DBNAME}.pid)" + create_physrep_tables ${node} + else + FIRSTNODE="" + for node in $CLUSTER; do + logfile=$TESTDIR/logs/${META_DBNAME}.${node}.log + ssh ${node} "mkdir -p $METADIR" < /dev/null + + if [[ -z "$FIRSTNODE" ]]; then + FIRSTNODE=$node + cat <> $METADIR/${META_DBNAME}.lrl +name ${META_DBNAME} +dir ${METADIR} +cluster nodes ${CLUSTER} +physrep_fanout_override $DBNAME 3 +tranlog_incoherent_timeout 10 +physrep_register_interval 10 +revsql_debug 1 +physrep_debug 1 +forbid_remote_admin 0 +logmsg level debug +ctrace_dbdir 1 +allow_lua_print 1 +physrep_update_registry_interval 1 +physrep_keepalive_v2 1 +END + scp $METADIR/${META_DBNAME}.lrl ${node}:$METADIR/${META_DBNAME}.lrl + ssh ${node} "$COMDB2_EXE ${META_DBNAME} --create --lrl ${METADIR}/${META_DBNAME}.lrl --pidfile ${METADIR}/${META_DBNAME}.pid >> $logfile 2>&1" < /dev/null + else + ssh $node "${COPYCOMDB2_EXE} -x ${COMDB2_EXE} $FIRSTNODE:${METADIR}/${META_DBNAME}.lrl $METADIR/" >> $logfile 2>&1 < /dev/null + if [[ ! $? -eq 0 ]]; then + cleanfailexit "copycomdb2 failed for $node" + fi + fi + done + + for node in $CLUSTER; do + logfile=$TESTDIR/logs/${META_DBNAME}.${node}.log + ssh ${node} "$COMDB2_EXE ${META_DBNAME} --lrl ${METADIR}/${META_DBNAME}.lrl --pidfile ${METADIR}/${META_DBNAME}.pid" >> $logfile 2>&1 < /dev/null & + PIDLIST="${PIDLIST} $!" + done + + for node in $CLUSTER; do + out="" + while [[ "$out" != "1" ]]; do + out=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${node} ${META_DBNAME} 'select 1' 2>/dev/null) + echo "$META_DBNAME: waiting until ready" + sleep 1 + done + done + + create_physrep_tables $FIRSTNODE + fi + echo "Physrep replication metadb cluster/node started!" +} + +function setup_standalone_physrep() +{ + local PHYSDIR=${DBDIR}/$PHYSREP + if [[ -z "$CLUSTER" ]]; then + node=$(hostname) + logfile=$TESTDIR/logs/${PHYSREP}.log + ${COPYCOMDB2_EXE} -x ${COMDB2_EXE} -H ${PHYSREP} -y @${FIRSTNODE} ${FIRSTNODE}:${DBDIR}/${DBNAME}.lrl $PHYSDIR $PHYSDIR >> $logfile 2>&1 + if [[ $? -ne 0 ]]; then + cleanfailexit "failed creating physrep for $node" + fi + $COMDB2_EXE ${PHYSREP} --lrl $PHYSDIR/${PHYSREP}.lrl --pidfile $PHYSDIR/${PHYSREP}.pid >> $logfile 2>&1 & + out="" + while [[ "$out" != "1" ]]; do + out=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${node} ${PHYSREP} 'select 1' 2>/dev/null) + sleep 1 + done + else + node=$FIRSTNODE + logfile=$TESTDIR/logs/${PHYSREP}.$FIRSTNODE.log + ssh ${node} "${COPYCOMDB2_EXE} -x ${COMDB2_EXE} -H ${PHYSREP} -y @${FIRSTNODE} ${FIRSTNODE}:${DBDIR}/${DBNAME}.lrl $PHYSDIR $PHYSDIR" >> $logfile 2>&1 < /dev/null + if [[ $? -ne 0 ]]; then + cleanfailexit "failed creating physrep for $node" + fi + + ssh ${node} "$COMDB2_EXE ${PHYSREP} --lrl $PHYSDIR/${PHYSREP}.lrl --pidfile $PHYSDIR/${PHYSREP}.pid" >> $logfile 2>&1 < /dev/null & + PIDLIST="${PIDLIST} $!" + out="" + while [[ "$out" != "1" ]]; do + out=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host ${node} ${PHYSREP} 'select 1' 2>/dev/null) + sleep 1 + done + + fi +} + +function setup() +{ + setup_physrep_metadb + fix_lrl_and_restart_source_nodes + setup_standalone_physrep + create_test_table +} + +function announce +{ + typeset text=$1 + echo "$text" + [[ -z "$FIGLET" ]] && figlet -f banner $text +} + +function testcase_finish +{ + typeset testcase=$1 + announce "Finished $testcase" +} + +function testcase_preamble +{ + typeset testcase=$1 + announce "Running $testcase" +} + +function block_physrep_count() +{ + local target=$1 + local count=0 + while [[ "$count" -ne "$target" ]]; do + count=$($CDB2SQL_EXE ${CDB2_OPTIONS} --tabs --host $FIRSTNODE $PHYSREP "select count(*) from t1") + sleep 1 + done +} + +function load_rows() +{ + local rows=$1 + local iters=$(( rows / 1000 )) + local i=0 + + while [[ $i -lt $iters ]]; do + $CDB2SQL_EXE $CDB2_OPTIONS $DBNAME default "insert into t1 select * from generate_series(1, 1000)" + i=$(( i + 1 )) + done + block_physrep_count $rows +} + +function count_revconn() +{ + typeset count=0 + count=$($CDB2SQL_EXE -tabs ${CDB2_OPTIONS} ${PHYSREP} --host $FIRSTNODE "exec procedure sys.cmd.send('stat thr')" | egrep "reversesql" | wc -l) + echo $count +} + +function make_physrep_revconn() +{ + if [[ -z "$CLUSTER" ]]; then + node=$(hostname) + $CDB2SQL_EXE ${CDB2_OPTIONS} $META_DBNAME --host $FIRSTNODE "insert into comdb2_physrep_sources values ('${PHYSREP}', '${FIRSTNODE}', '${DBNAME}', '${node}')" + else + for node in $CLUSTER; do + $CDB2SQL_EXE ${CDB2_OPTIONS} $META_DBNAME --host $FIRSTNODE "insert into comdb2_physrep_sources values('${PHYSREP}', '${FIRSTNODE}', '${DBNAME}', '${node}')" + done + fi + local count=0 + + while [[ "$count" -eq "0" ]]; do + count=$(count_revconn) + sleep 1 + done +} + +function load_rows_1000000() +{ + gather_master_base + gather_physrep_base + load_rows 1000000 + gather_master_stats + gather_physrep_stats + calc_master_diff + calc_physrep_diff + print_master_diff + print_physrep_diff + compare_metrics normal + $CDB2SQL_EXE ${CDB2_OPTIONS} $DBNAME default "truncate t1" +} + +function load_rows_1000000_revconn() +{ + make_physrep_revconn + gather_master_base + gather_physrep_base + load_rows 1000000 + gather_master_stats + gather_physrep_stats + calc_master_diff + calc_physrep_diff + print_master_diff + print_physrep_diff + compare_metrics revconn + $CDB2SQL_EXE ${CDB2_OPTIONS} $DBNAME default "truncate t1" +} + +function run_test() +{ + testcase="load_rows_1000000" + testcase_preable $testcase + load_rows_1000000 + testcase_finish $testcase + + testcase="load_rows_1000000_revconn" + testcase_preable $testcase + load_rows_1000000_revconn + testcase_finish $testcase +} + +setup +run_test +cleanup +echo "Success!" diff --git a/tests/phys_rep_tiered.test/lrl.options b/tests/phys_rep_tiered.test/lrl.options index 2e3b01d54a..32c3c90170 100644 --- a/tests/phys_rep_tiered.test/lrl.options +++ b/tests/phys_rep_tiered.test/lrl.options @@ -10,3 +10,4 @@ ctrace_dbdir 1 allow_lua_print 1 reverse_hosts_v2 1 physrep_keepalive_v2 1 +#rep_skip_recovery 1 diff --git a/tests/phys_rep_tiered.test/runit b/tests/phys_rep_tiered.test/runit index 4f9381f16e..20fe17af09 100755 --- a/tests/phys_rep_tiered.test/runit +++ b/tests/phys_rep_tiered.test/runit @@ -1204,6 +1204,35 @@ function test_overlap fi } +# Simple test to verify that skip-recovery ran against physical replicants +# Disabled for now- this does not work yet .. +function verify_skip_recovery +{ + for node in $CLUSTER ; do + logFile=$TESTDIR/logs/${REPL_DBNAME_PREFIX}_${node}.log + x=$(egrep "skip-recovery truncate" $logFile | wc -l) + if [[ "$x" == "0" ]]; then + cleanFailExit "skip-recovery not found in $logFile" + fi + logFile=$TESTDIR/logs/${REPL_CLUS_DBNAME}.${node}.log + x=$(egrep "skip-recovery truncate" $logFile | wc -l) + if [[ "$x" == "0" ]]; then + cleanFailExit "skip-recovery not found in $logFile" + fi + done +} + +# Verify the reverse-connection / register_interval fix +# When broken, the first REPL node would re-register 200+ times +function verify_revconn_fix +{ + logFile=$TESTDIR/logs/${REPL_DBNAME_PREFIX}_${firstNode}.log + x=$(egrep "Reverse connected" $logFile | wc -l) + if [[ "$x" -gt 80 ]]; then + cleanFailExit "Reverse connection loop in $logFile" + fi +} + # Use message traps to verify overlap logic function match_overlap { @@ -1228,6 +1257,19 @@ function match_overlap return 0 } +# Verify that we are showing generation in comdb2-transaction-logs +function verify_generation +{ + x=$(cdb2sql --tabs ${CDB2_OPTIONS} $DBNAME default "select count(*) from comdb2_transaction_logs where generation is not null") + if [[ $? != 0 ]]; then + cleanFailExit "Failed reading comdb2-transactions-logs" + fi + + if [[ $x == 0 ]]; then + cleanFailExit "Generation not shown in comdb2_transaction_logs" + fi +} + # Call register-replicant directly- look at the list of machines returned .. # Update the metadb info on the returned nodes to put them out of range, and # verify that they are not returned the next time @@ -1414,15 +1456,20 @@ function testcase_preamble function run_tests { + testcase="verify_generation" + testcase_preamble $testcase + verify_generation + testcase_finish $testcase + testcase="phys_rep_nomatch" testcase_preamble $testcase phys_rep_nomatch ${REPL_META_DBNAME} ${REPL_META_HOST} - testcase_finsh $testcase + testcase_finish $testcase testcase="tranlog_timeout" testcase_preamble $testcase tranlog_timeout ${REPL_META_DBNAME} ${REPL_META_HOST} - testcase_finsh $testcase + testcase_finish $testcase testcase="revconn_latency $lastNode" testcase_preamble $testcase @@ -1475,6 +1522,17 @@ function run_tests testcase_preamble $testcase match_overlap testcase_finish $testcase + + # TODO: skip-recovery seems to cause corruption for physreps- disable for now + #testcase="verify_skip_recovery" + #testcase_preamble $testcase + #verify_skip_recovery + #testcase_finish $testcase + + testcase="verify_revconn_fix" + testcase_preamble $testcase + verify_revconn_fix + testcase_finish $testcase } run_tests diff --git a/tests/tunables.test/t00_all_tunables.expected b/tests/tunables.test/t00_all_tunables.expected index 0869cc96af..996d488082 100644 --- a/tests/tunables.test/t00_all_tunables.expected +++ b/tests/tunables.test/t00_all_tunables.expected @@ -70,7 +70,7 @@ (name='blobmem_sz_thresh_kb', description='Sets the threshold (in KB) above which blobs are allocated by the blob allocator. (Default: 0)', type='INTEGER', value='-1', read_only='Y') (name='blobstripe', description='', type='BOOLEAN', value='ON', read_only='Y') (name='blocking_latches', description='Block on latch rather than deadlock', type='BOOLEAN', value='OFF', read_only='N') -(name='blocking_physrep', description='Physical replicant blocks on select. (Default: false)', type='BOOLEAN', value='OFF', read_only='N') +(name='blocking_physrep', description='Physical replicant blocks on select. (Default: off)', type='BOOLEAN', value='OFF', read_only='N') (name='broadcast_check_rmtpol', description='Check rmtpol before sending triggers', type='BOOLEAN', value='ON', read_only='N') (name='broken_max_rec_sz', description='', type='INTEGER', value='0', read_only='Y') (name='broken_num_parser', description='', type='BOOLEAN', value='OFF', read_only='Y') @@ -714,8 +714,8 @@ (name='physrep_max_rollback', description='Maximum logs physrep can rollback. (Default: 0)', type='INTEGER', value='0', read_only='N') (name='physrep_metadb_host', description='List of physical replication metadb cluster hosts.', type='STRING', value=NULL, read_only='Y') (name='physrep_metadb_name', description='Physical replication metadb cluster name.', type='STRING', value=NULL, read_only='Y') +(name='physrep_reconnect_interval', description='Reconnect interval for physical replicants (Default: 600)', type='INTEGER', value='600', read_only='N') (name='physrep_reconnect_penalty', description='Physrep wait seconds before retry to the same node. (Default: 5)', type='INTEGER', value='0', read_only='N') -(name='physrep_register_interval', description='Interval for physical replicant re-registration. (Default: 600)', type='INTEGER', value='600', read_only='N') (name='physrep_repl_host', description='Current physrep host.', type='STRING', value=NULL, read_only='Y') (name='physrep_repl_name', description='Current physrep parent.', type='STRING', value=NULL, read_only='Y') (name='physrep_revconn_check_interval', description='Physrep recheck revconn interval. (Default: 60)', type='INTEGER', value='60', read_only='N')