]> git.saurik.com Git - redis.git/blobdiff - src/bio.c
32bit build fixed, broken by a previous commit fixing build on Solaris
[redis.git] / src / bio.c
index 5ff5b1c853f0ce5b75fcaa8fae57c801885bd734..eaac8e40d55663caf3fbacf4212a01247ba26972 100644 (file)
--- a/src/bio.c
+++ b/src/bio.c
  * ------
  *
  * The design is trivial, we have a structure representing a job to perform
- * and a single thread performing all the I/O operations in the queue.
+ * and a different thread and job queue for every job type.
+ * Every thread wait for new jobs in its queue, and process every job
+ * sequentially.
+ *
+ * Jobs of the same type are guaranteed to be processed from the least
+ * recently inserted to the most recently inserted (older jobs processed
+ * first).
+ *
  * Currently there is no way for the creator of the job to be notified about
  * the completion of the operation, this will only be added when/if needed.
  */
 #include "redis.h"
 #include "bio.h"
 
-static pthread_mutex_t bio_mutex;
-static pthread_cond_t bio_condvar;
-list *bio_jobs;
+static pthread_mutex_t bio_mutex[REDIS_BIO_NUM_OPS];
+static pthread_cond_t bio_condvar[REDIS_BIO_NUM_OPS];
+static list *bio_jobs[REDIS_BIO_NUM_OPS];
+/* The following array is used to hold the number of pending jobs for every
+ * OP type. This allows us to export the bioPendingJobsOfType() API that is
+ * useful when the main thread wants to perform some operation that may involve
+ * objects shared with the background thread. The main thread will just wait
+ * that there are no longer jobs of this type to be executed before performing
+ * the sensible operation. This data is also useful for reporting. */
+static unsigned long long bio_pending[REDIS_BIO_NUM_OPS];
 
 /* This structure represents a background Job. It is only used locally to this
  * file as the API deos not expose the internals at all. */
 struct bio_job {
-    int type;       /* Job type, for instance BIO_JOB_CLOSE */
-    void *data;     /* Job specific arguments pointer. */
+    time_t time; /* Time at which the job was created. */
+    /* Job specific arguments pointers. If we need to pass more than three
+     * arguments we can just pass a pointer to a structure or alike. */
+    void *arg1, *arg2, *arg3;
 };
 
 void *bioProcessBackgroundJobs(void *arg);
@@ -46,10 +62,15 @@ void bioInit(void) {
     pthread_attr_t attr;
     pthread_t thread;
     size_t stacksize;
-
-    pthread_mutex_init(&bio_mutex,NULL);
-    pthread_cond_init(&bio_condvar,NULL);
-    bio_jobs = listCreate();
+    int j;
+
+    /* Initialization of state vars and objects */
+    for (j = 0; j < REDIS_BIO_NUM_OPS; j++) {
+        pthread_mutex_init(&bio_mutex[j],NULL);
+        pthread_cond_init(&bio_condvar[j],NULL);
+        bio_jobs[j] = listCreate();
+        bio_pending[j] = 0;
+    }
 
     /* Set the stack size as by default it may be small in some system */
     pthread_attr_init(&attr);
@@ -58,50 +79,58 @@ void bioInit(void) {
     while (stacksize < REDIS_THREAD_STACK_SIZE) stacksize *= 2;
     pthread_attr_setstacksize(&attr, stacksize);
 
-    /* Ready to spawn our thread */
-    if (pthread_create(&thread,&attr,bioProcessBackgroundJobs,NULL) != 0) {
-        redisLog(REDIS_WARNING,"Fatal: Can't initialize Background Jobs.");
-        exit(1);
+    /* Ready to spawn our threads. We use the single argument the thread
+     * function accepts in order to pass the job ID the thread is
+     * responsible of. */
+    for (j = 0; j < REDIS_BIO_NUM_OPS; j++) {
+        void *arg = (void*)(unsigned long) j;
+        if (pthread_create(&thread,&attr,bioProcessBackgroundJobs,arg) != 0) {
+            redisLog(REDIS_WARNING,"Fatal: Can't initialize Background Jobs.");
+            exit(1);
+        }
     }
 }
 
-void bioCreateBackgroundJob(int type, void *data) {
+void bioCreateBackgroundJob(int type, void *arg1, void *arg2, void *arg3) {
     struct bio_job *job = zmalloc(sizeof(*job));
 
-    job->type = type;
-    job->data = data;
-    pthread_mutex_lock(&bio_mutex);
-    listAddNodeTail(bio_jobs,job);
-    pthread_cond_signal(&bio_condvar);
-    pthread_mutex_unlock(&bio_mutex);
+    job->time = time(NULL);
+    job->arg1 = arg1;
+    job->arg2 = arg2;
+    job->arg3 = arg3;
+    pthread_mutex_lock(&bio_mutex[type]);
+    listAddNodeTail(bio_jobs[type],job);
+    bio_pending[type]++;
+    pthread_cond_signal(&bio_condvar[type]);
+    pthread_mutex_unlock(&bio_mutex[type]);
 }
 
 void *bioProcessBackgroundJobs(void *arg) {
     struct bio_job *job;
-    REDIS_NOTUSED(arg);
+    unsigned long type = (unsigned long) arg;
 
     pthread_detach(pthread_self());
-    pthread_mutex_lock(&bio_mutex);
+    pthread_mutex_lock(&bio_mutex[type]);
     while(1) {
         listNode *ln;
 
         /* The loop always starts with the lock hold. */
-        if (listLength(bio_jobs) == 0) {
-            pthread_cond_wait(&bio_condvar,&bio_mutex);
+        if (listLength(bio_jobs[type]) == 0) {
+            pthread_cond_wait(&bio_condvar[type],&bio_mutex[type]);
             continue;
         }
         /* Pop the job from the queue. */
-        ln = listFirst(bio_jobs);
+        ln = listFirst(bio_jobs[type]);
         job = ln->value;
-        listDelNode(bio_jobs,ln);
         /* It is now possible to unlock the background system as we know have
          * a stand alone job structure to process.*/
-        pthread_mutex_unlock(&bio_mutex);
+        pthread_mutex_unlock(&bio_mutex[type]);
 
         /* Process the job accordingly to its type. */
-        if (job->type == REDIS_BIO_CLOSE_FILE) {
-            printf("Closing file...\n");
-            close((long)job->data);
+        if (type == REDIS_BIO_CLOSE_FILE) {
+            close((long)job->arg1);
+        } else if (type == REDIS_BIO_AOF_FSYNC) {
+            aof_fsync((long)job->arg1);
         } else {
             redisPanic("Wrong job type in bioProcessBackgroundJobs().");
         }
@@ -109,6 +138,71 @@ void *bioProcessBackgroundJobs(void *arg) {
 
         /* Lock again before reiterating the loop, if there are no longer
          * jobs to process we'll block again in pthread_cond_wait(). */
-        pthread_mutex_lock(&bio_mutex);
+        pthread_mutex_lock(&bio_mutex[type]);
+        listDelNode(bio_jobs[type],ln);
+        bio_pending[type]--;
     }
 }
+
+/* Return the number of pending jobs of the specified type. */
+unsigned long long bioPendingJobsOfType(int type) {
+    unsigned long long val;
+    pthread_mutex_lock(&bio_mutex[type]);
+    val = bio_pending[type];
+    pthread_mutex_unlock(&bio_mutex[type]);
+    return val;
+}
+
+#if 0 /* We don't use the following code for now, and bioWaitPendingJobsLE
+         probably needs a rewrite using conditional variables instead of the
+         current implementation. */
+         
+
+/* Wait until the number of pending jobs of the specified type are
+ * less or equal to the specified number.
+ *
+ * This function may block for long time, it should only be used to perform
+ * the following tasks:
+ *
+ * 1) To avoid that the main thread is pushing jobs of a given time so fast
+ *    that the background thread can't process them at the same speed.
+ *    So before creating a new job of a given type the main thread should
+ *    call something like: bioWaitPendingJobsLE(job_type,10000);
+ * 2) In order to perform special operations that make it necessary to be sure
+ *    no one is touching shared resourced in the background.
+ */
+void bioWaitPendingJobsLE(int type, unsigned long long num) {
+    unsigned long long iteration = 0;
+
+    /* We poll the jobs queue aggressively to start, and gradually relax
+     * the polling speed if it is going to take too much time. */
+    while(1) {
+        iteration++;
+        if (iteration > 1000 && iteration <= 10000) {
+            usleep(100);
+        } else if (iteration > 10000) {
+            usleep(1000);
+        }
+        if (bioPendingJobsOfType(type) <= num) break;
+    }
+}
+
+/* Return the older job of the specified type. */
+time_t bioOlderJobOfType(int type) {
+    time_t time;
+    listNode *ln;
+    struct bio_job *job;
+
+    pthread_mutex_lock(&bio_mutex[type]);
+    ln = listFirst(bio_jobs[type]);
+    if (ln == NULL) {
+        pthread_mutex_unlock(&bio_mutex[type]);
+        return 0;
+    }
+    job = ln->value;
+    time = job->time;
+    pthread_mutex_unlock(&bio_mutex[type]);
+    return time;
+}
+
+#endif