#50485 Ticket 50459 - c_mutex to use pthread_mutex to allow ns sharing
Closed 3 years ago by spichugi. Opened 4 years ago by firstyear.
firstyear/389-ds-base 50459-c_mutex_rewrite  into  master

file modified
+2 -2
@@ -88,7 +88,7 @@ 

       * flag and abort the operation at a convenient time.

       */

  

-     PR_EnterMonitor(pb_conn->c_mutex);

+     pthread_mutex_lock(&(pb_conn->c_mutex));

      for (o = pb_conn->c_ops; o != NULL; o = o->o_next) {

          if (o->o_msgid == id && o != pb_op)

              break;
@@ -151,7 +151,7 @@ 

                           o->o_results.r.r_search.nentries, (int64_t)o_hr_time_end.tv_sec, (int64_t)o_hr_time_end.tv_nsec);

      }

  

-     PR_ExitMonitor(pb_conn->c_mutex);

+     pthread_mutex_unlock(&(pb_conn->c_mutex));

      /*

       * Wake up the persistent searches, so they

       * can notice if they've been abandoned.

file modified
+2 -2
@@ -232,7 +232,7 @@ 

          slapi_pblock_get(pb, SLAPI_PWPOLICY, &pw_response_requested);

      }

  

-     PR_EnterMonitor(pb_conn->c_mutex);

+     pthread_mutex_lock(&(pb_conn->c_mutex));

  

      bind_credentials_clear(pb_conn, PR_FALSE, /* do not lock conn */

                             PR_FALSE /* do not clear external creds. */);
@@ -263,7 +263,7 @@ 

       * bound user can work properly

       */

      pb_conn->c_needpw = 0;

-     PR_ExitMonitor(pb_conn->c_mutex);

+     pthread_mutex_unlock(&(pb_conn->c_mutex));

  

      log_bind_access(pb, dn ? dn : "empty", method, version, saslmech, NULL);

  

file modified
+34 -33
@@ -144,18 +144,19 @@ 

      connection_cleanup(conn);

      /* free the private content, the buffer has been freed by above connection_cleanup */

      slapi_ch_free((void **)&conn->c_private);

+     pthread_mutex_destroy(&(conn->c_mutex));

      if (NULL != conn->c_sb) {

          ber_sockbuf_free(conn->c_sb);

      }

-     if (NULL != conn->c_mutex) {

-         PR_DestroyMonitor(conn->c_mutex);

-     }

      if (NULL != conn->c_pdumutex) {

          PR_DestroyLock(conn->c_pdumutex);

      }

      /* PAGED_RESULTS */

      pagedresults_cleanup_all(conn, 0);

  

+     /* Finally, flag that we are clean - basically write a 0 ...*/

+     conn->c_state = CONN_STATE_FREE;

+ 

      /*

       * WARNING: There is a memory leak here! During a shutdown, connections

       * can still have events in ns add io timeout job because of post connection
@@ -751,12 +752,12 @@ 

      int rc;

  

      if (use_lock) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

      }

      rc = conn->c_sd == SLAPD_INVALID_SOCKET && conn->c_refcnt == 0 &&

           !(conn->c_flags & CONN_FLAG_CLOSING);

      if (use_lock) {

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

  

      return rc;
@@ -1130,7 +1131,7 @@ 

      size_t buffer_data_avail;

      int conn_closed = 0;

  

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /*

       * if the socket is still valid, get the ber element

       * waiting for us on this connection. timeout is handled
@@ -1323,16 +1324,16 @@ 

      }

      op->o_tag = *tag;

  done:

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      return ret;

  }

  

  void

  connection_make_readable(Connection *conn)

  {

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      conn->c_gettingber = 0;

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      signal_listner();

  }

  
@@ -1356,7 +1357,7 @@ 

  {

      int current_count = 0;

      int delta_count = 0;

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /* get the current op count */

      current_count = conn->c_opscompleted;

      /* compare to the previous op count */
@@ -1367,7 +1368,7 @@ 

      conn->c_private->previous_op_count = current_count;

      /* update the last checked time */

      conn->c_private->previous_count_check_time = slapi_current_utc_time();

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      slapi_log_err(SLAPI_LOG_CONNS, "connection_check_activity_level", "conn %" PRIu64 " activity level = %d\n", conn->c_connid, delta_count);

  }

  
@@ -1415,7 +1416,7 @@ 

      int connection_count = 0;

      int our_rank = 0;

      int threshold_rank = 0;

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /* We can already be in turbo mode, or not */

      current_mode = current_turbo_flag;

      if (pagedresults_in_use_nolock(conn)) {
@@ -1460,7 +1461,7 @@ 

              new_mode = 1;

          }

      }

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      if (current_mode != new_mode) {

          if (current_mode) {

              slapi_log_err(SLAPI_LOG_CONNS, "connection_enter_leave_turbo", "conn %" PRIu64 " leaving turbo mode\n", conn->c_connid);
@@ -1541,7 +1542,7 @@ 

                      return;

                  }

  

-                 PR_EnterMonitor(pb_conn->c_mutex);

+                 pthread_mutex_lock(&(pb_conn->c_mutex));

                  if (pb_conn->c_anonlimits_set == 0) {

                      /*

                       * We have a new connection, set the anonymous reslimit idletimeout
@@ -1567,7 +1568,7 @@ 

                       */

                      pb_conn->c_anonlimits_set = 1;

                  }

-                 PR_ExitMonitor(pb_conn->c_mutex);

+                 pthread_mutex_unlock(&(pb_conn->c_mutex));

  

                  if (connection_call_io_layer_callbacks(pb_conn)) {

                      slapi_log_err(SLAPI_LOG_ERR, "connection_threadmain",
@@ -1584,14 +1585,14 @@ 

              */

              PR_Sleep(PR_INTERVAL_NO_WAIT);

  

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

              /* Make our own pb in turbo mode */

              connection_make_new_pb(pb, conn);

              if (connection_call_io_layer_callbacks(conn)) {

                  slapi_log_err(SLAPI_LOG_ERR, "connection_threadmain",

                                "Could not add/remove IO layers from connection\n");

              }

-             PR_ExitMonitor(conn->c_mutex);

+             pthread_mutex_unlock(&(conn->c_mutex));

              if (!config_check_referral_mode()) {

                  slapi_counter_increment(ops_initiated);

                  slapi_counter_increment(g_get_global_snmp_vars()->ops_tbl.dsInOps);
@@ -1706,9 +1707,9 @@ 

          if ((tag != LDAP_REQ_UNBIND) && !thread_turbo_flag && !replication_connection) {

              if (!more_data) {

                  conn->c_flags &= ~CONN_FLAG_MAX_THREADS;

-                 PR_EnterMonitor(conn->c_mutex);

+                 pthread_mutex_lock(&(conn->c_mutex));

                  connection_make_readable_nolock(conn);

-                 PR_ExitMonitor(conn->c_mutex);

+                 pthread_mutex_unlock(&(conn->c_mutex));

                  /* once the connection is readable, another thread may access conn,

                   * so need locking from here on */

                  signal_listner();
@@ -1720,7 +1721,7 @@ 

                   */

              } else if (!enable_nunc_stans) { /* more data in conn - just put back on work_q - bypass poll */

                  bypasspollcnt++;

-                 PR_EnterMonitor(conn->c_mutex);

+                 pthread_mutex_lock(&(conn->c_mutex));

                  /* don't do this if it would put us over the max threads per conn */

                  if (conn->c_threadnumber < maxthreads) {

                      /* for turbo, c_idlesince is set above - for !turbo and
@@ -1736,7 +1737,7 @@ 

                      /* keep count of how many times maxthreads has blocked an operation */

                      conn->c_maxthreadsblocked++;

                  }

-                 PR_ExitMonitor(conn->c_mutex);

+                 pthread_mutex_unlock(&(conn->c_mutex));

              }

          }

  
@@ -1772,14 +1773,14 @@ 

  

      done:

          if (doshutdown) {

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

              connection_remove_operation_ext(pb, conn, op);

              connection_make_readable_nolock(conn);

              conn->c_threadnumber--;

              slapi_counter_decrement(conns_in_maxthreads);

              slapi_counter_decrement(g_get_global_snmp_vars()->ops_tbl.dsConnectionsInMaxThreads);

              connection_release_nolock(conn);

-             PR_ExitMonitor(conn->c_mutex);

+             pthread_mutex_unlock(&(conn->c_mutex));

              signal_listner();

              slapi_pblock_destroy(pb);

              return;
@@ -1804,9 +1805,9 @@ 

               * continues to hold the connection

               */

              if (!thread_turbo_flag && !more_data) {

-                 PR_EnterMonitor(conn->c_mutex);

+                 pthread_mutex_lock(&(conn->c_mutex));

                  connection_release_nolock(conn); /* psearch acquires ref to conn - release this one now */

-                 PR_ExitMonitor(conn->c_mutex);

+                 pthread_mutex_unlock(&(conn->c_mutex));

              }

              /* ps_add makes a shallow copy of the pb - so we

                   * can't free it or init it here - just set operation to NULL.
@@ -1817,7 +1818,7 @@ 

          } else {

              /* delete from connection operation queue & decr refcnt */

              int conn_closed = 0;

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

              connection_remove_operation_ext(pb, conn, op);

  

              /* If we're in turbo mode, we keep our reference to the connection alive */
@@ -1869,7 +1870,7 @@ 

                      signal_listner();

                  }

              }

-             PR_ExitMonitor(conn->c_mutex);

+             pthread_mutex_unlock(&(conn->c_mutex));

          }

      } /* while (1) */

  }
@@ -2125,7 +2126,7 @@ 

      size_t dnlen;

      size_t typelen;

  

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      dnlen = conn->c_dn ? strlen(conn->c_dn) : 0;

      typelen = conn->c_authtype ? strlen(conn->c_authtype) : 0;

  
@@ -2157,14 +2158,14 @@ 

          op->o_ssf = conn->c_local_ssf;

      }

  

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

  }

  

  /* Sets the SSL SSF in the connection struct. */

  static void

  connection_set_ssl_ssf(Connection *conn)

  {

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

  

      if (conn->c_flags & CONN_FLAG_SSL) {

          SSL_SecurityStatus(conn->c_prfd, NULL, NULL, NULL, &(conn->c_ssl_ssf), NULL, NULL);
@@ -2172,7 +2173,7 @@ 

          conn->c_ssl_ssf = 0;

      }

  

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

  }

  

  static int
@@ -2223,9 +2224,9 @@ 

  void

  disconnect_server(Connection *conn, PRUint64 opconnid, int opid, PRErrorCode reason, PRInt32 error)

  {

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      disconnect_server_nomutex(conn, opconnid, opid, reason, error);

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

  }

  

  static ps_wakeup_all_fn_ptr ps_wakeup_all_fn = NULL;

file modified
+37 -20
@@ -51,6 +51,11 @@ 

          ct->c[i].c_prev = NULL;

          ct->c[i].c_ci = i;

          ct->c[i].c_fdi = SLAPD_INVALID_SOCKET_INDEX;

+         /*

+          * Technically this is a no-op due to calloc, but we should always be

+          * careful with things like this ....

+          */

+         ct->c[i].c_state = CONN_STATE_FREE;

      }

      return ct;

  }
@@ -75,10 +80,10 @@ 

  {

      int i;

      for (i = 0; i < ct->size; i++) {

-         if (ct->c[i].c_mutex) {

-             PR_EnterMonitor(ct->c[i].c_mutex);

+         if (ct->c[i].c_state != CONN_STATE_FREE) {

+             pthread_mutex_lock(&(ct->c[i].c_mutex));

              connection_abandon_operations(&ct->c[i]);

-             PR_ExitMonitor(ct->c[i].c_mutex);

+             pthread_mutex_unlock(&(ct->c[i].c_mutex));

          }

      }

  }
@@ -87,11 +92,11 @@ 

  connection_table_disconnect_all(Connection_Table *ct)

  {

      for (size_t i = 0; i < ct->size; i++) {

-         if (ct->c[i].c_mutex) {

+         if (ct->c[i].c_state != CONN_STATE_FREE) {

              Connection *c = &(ct->c[i]);

-             PR_EnterMonitor(c->c_mutex);

+             pthread_mutex_lock(&(c->c_mutex));

              disconnect_server_nomutex(c, c->c_connid, -1, SLAPD_DISCONNECT_ABORT, ECANCELED);

-             PR_ExitMonitor(c->c_mutex);

+             pthread_mutex_unlock(&(c->c_mutex));

          }

      }

  }
@@ -117,11 +122,10 @@ 

          /* Do not use slot 0, slot 0 is head of the list of active connections */

          if (index == 0) {

              continue;

-         } else if (ct->c[index].c_mutex == NULL) {

+         } else if (ct->c[index].c_state == CONN_STATE_FREE) {

              break;

-         }

- 

-         if (connection_is_free(&(ct->c[index]), 1 /*use lock */)) {

+         } else if (connection_is_free(&(ct->c[index]), 1 /*use lock */)) {

+             /* Connection must be allocated, check if it's okay */

              break;

          }

      }
@@ -132,17 +136,30 @@ 

          PR_ASSERT(c->c_next == NULL);

          PR_ASSERT(c->c_prev == NULL);

          PR_ASSERT(c->c_extension == NULL);

-         if (c->c_mutex == NULL) {

+ 

+         if (c->c_state == CONN_STATE_FREE) {

              PR_Lock(ct->table_mutex);

-             c->c_mutex = PR_NewMonitor();

+ 

+             c->c_state = CONN_STATE_INIT;

+ 

+             pthread_mutexattr_t monitor_attr = {0};

+             pthread_mutexattr_init(&monitor_attr);

+             pthread_mutexattr_settype(&monitor_attr, PTHREAD_MUTEX_RECURSIVE);

+             if (pthread_mutex_init(&(c->c_mutex), &monitor_attr) != 0) {

+                 slapi_log_err(SLAPI_LOG_ERR, "connection_table_get_connection", "pthread_mutex_init failed\n");

+                 exit(1);

+             }

+ 

Sorry to be nit picker but I still have a concern.

Currently there is no real issue because the new connection handler is a single thread.
But with NS we can imagine parallel handler, that access the connection table in parallel. Correct ?
Is that part of code robust for parallel new connection handers ?
IMHO the ct->table_mutex should be acquired for the all functions.

Setting c_state=INIT before the real allocation of mutex means that an handler can return this connection while it is not yet fully initialized.

              c->c_pdumutex = PR_NewLock();

              PR_Unlock(ct->table_mutex);

-             if (c->c_mutex == NULL || c->c_pdumutex == NULL) {

-                 c->c_mutex = NULL;

+             if (c->c_pdumutex == NULL) {

                  c->c_pdumutex = NULL;

                  slapi_log_err(SLAPI_LOG_ERR, "connection_table_get_connection", "PR_NewLock failed\n");

                  exit(1);

              }

+         } else {

+             slapi_log_err(SLAPI_LOG_ERR, "connection_table_get_connection", "Invalide connection table state - We tried to allocate to a conn NOT in state CONN_STATE_FREE - this is a complete disaster!\n");

+             exit(1);

          }

          /* Let's make sure there's no cruft left on there from the last time this connection was used. */

          /* Note: no need to lock c->c_mutex because this function is only
@@ -364,14 +381,14 @@ 

      nreadwaiters = 0;

      for (i = 0; i < (ct != NULL ? ct->size : 0); i++) {

          PR_Lock(ct->table_mutex);

-         if ((ct->c[i].c_mutex == NULL) || (ct->c[i].c_mutex == (PRMonitor *)-1)) {

+         if (ct->c[i].c_state == CONN_STATE_FREE) {

              PR_Unlock(ct->table_mutex);

              continue;

          }

          /* Can't take c_mutex if holding table_mutex; temporarily unlock */

          PR_Unlock(ct->table_mutex);

  

-         PR_EnterMonitor(ct->c[i].c_mutex);

+         pthread_mutex_lock(&(ct->c[i].c_mutex));

          if (ct->c[i].c_sd != SLAPD_INVALID_SOCKET) {

              char buf2[SLAPI_TIMESTAMP_BUFSIZE+1];

              size_t lendn = ct->c[i].c_dn ? strlen(ct->c[i].c_dn) : 6; /* "NULLDN" */
@@ -445,7 +462,7 @@ 

              attrlist_merge(&e->e_attrs, "connection", vals);

              slapi_ch_free_string(&newbuf);

          }

-         PR_ExitMonitor(ct->c[i].c_mutex);

+         pthread_mutex_unlock(&(ct->c[i].c_mutex));

      }

  

      snprintf(buf, sizeof(buf), "%d", nconns);
@@ -486,10 +503,10 @@ 

  

      for (i = 0; i < ct->size; i++) {

          Connection *c = &(ct->c[i]);

-         if (c->c_mutex) {

+         if (c->c_state) {

              /* Find the connection we are referring to */

              int j = c->c_fdi;

-             PR_EnterMonitor(c->c_mutex);

+             pthread_mutex_lock(&(c->c_mutex));

              if ((c->c_sd != SLAPD_INVALID_SOCKET) &&

                  (j >= 0) && (c->c_prfd == ct->fd[j].fd)) {

                  int r = ct->fd[j].out_flags & SLAPD_POLL_FLAGS;
@@ -498,7 +515,7 @@ 

                                    "activity on %d%s\n", i, r ? "r" : "");

                  }

              }

-             PR_ExitMonitor(c->c_mutex);

+             pthread_mutex_unlock(&(c->c_mutex));

          }

      }

  }

file modified
+16 -78
@@ -167,69 +167,6 @@ 

      ns_handle_pr_read_ready,

      ns_handle_closure

  };

- /*

-  * NSPR has different implementations for PRMonitor, depending

-  * on the availble threading model

-  * The PR_TestAndEnterMonitor is not available for pthreads

-  * so this is a implementation based on the code in

-  * prmon.c adapted to resemble the implementation in ptsynch.c

-  *

-  * The function needs access to the elements of the PRMonitor struct.

-  * Therfor the pthread variant of PRMonitor is copied here.

-  */

- typedef struct MY_PRMonitor

- {

-     const char *name;

-     pthread_mutex_t lock;

-     pthread_t owner;

-     pthread_cond_t entryCV;

-     pthread_cond_t waitCV;

-     PRInt32 refCount;

-     PRUint32 entryCount;

-     PRIntn notifyTimes;

- } MY_PRMonitor;

- 

- static PRBool

- MY_TestAndEnterMonitor(MY_PRMonitor *mon)

- {

-     pthread_t self = pthread_self();

-     PRStatus rv;

-     PRBool rc = PR_FALSE;

- 

-     PR_ASSERT(mon != NULL);

-     rv = pthread_mutex_lock(&mon->lock);

-     if (rv != 0) {

-         slapi_log_err(SLAPI_LOG_ERR, "TestAndEnterMonitor",

-                       "Failed to acquire monitor mutex, error (%d)\n", rv);

-         return rc;

-     }

-     if (mon->entryCount != 0) {

-         if (pthread_equal(mon->owner, self))

-             goto done;

-         rv = pthread_mutex_unlock(&mon->lock);

-         if (rv != 0) {

-             slapi_log_err(SLAPI_LOG_ERR, "TestAndEnterMonitor",

-                           "Failed to release monitor mutex, error (%d)\n", rv);

-         }

-         return PR_FALSE;

-     }

-     /* and now I have the monitor */

-     PR_ASSERT(mon->notifyTimes == 0);

-     PR_ASSERT((mon->owner) == 0);

-     mon->owner = self;

- 

- done:

-     mon->entryCount += 1;

-     rv = pthread_mutex_unlock(&mon->lock);

-     if (rv == PR_SUCCESS) {

-         rc = PR_TRUE;

-     } else {

-         slapi_log_err(SLAPI_LOG_ERR, "TestAndEnterMonitor",

-                       "Failed to release monitor mutex, error (%d)\n", rv);

-         rc = PR_FALSE;

-     }

-     return rc;

- }

  /* Globals which are used to store the sockets between

   * calls to daemon_pre_setuid_init() and the daemon thread

   * creation. */
@@ -1491,13 +1428,13 @@ 

      c = connection_table_get_first_active_connection(ct);

      while (c) {

          next = connection_table_get_next_active_connection(ct, c);

-         if (c->c_mutex == NULL) {

+         if (c->c_state == CONN_STATE_FREE) {

              connection_table_move_connection_out_of_active_list(ct, c);

          } else {

              /* we try to acquire the connection mutex, if it is already

               * acquired by another thread, don't wait

               */

-             if (PR_FALSE == MY_TestAndEnterMonitor((MY_PRMonitor *)c->c_mutex)) {

+             if (pthread_mutex_trylock(&(c->c_mutex)) == EBUSY) {

                  c = next;

                  continue;

              }
@@ -1538,7 +1475,7 @@ 

                      c->c_fdi = SLAPD_INVALID_SOCKET_INDEX;

                  }

              }

-             PR_ExitMonitor(c->c_mutex);

+             pthread_mutex_unlock(&(c->c_mutex));

          }

          c = next;

      }
@@ -1579,12 +1516,13 @@ 

       */

      for (c = connection_table_get_first_active_connection(ct); c != NULL;

           c = connection_table_get_next_active_connection(ct, c)) {

-         if (c->c_mutex != NULL) {

+         if (c->c_state != CONN_STATE_FREE) {

              /* this check can be done without acquiring the mutex */

-             if (c->c_gettingber)

+             if (c->c_gettingber) {

                  continue;

+             }

  

-             PR_EnterMonitor(c->c_mutex);

+             pthread_mutex_lock(&(c->c_mutex));

              if (connection_is_active_nolock(c) && c->c_gettingber == 0) {

                  PRInt16 out_flags;

                  short readready;
@@ -1634,7 +1572,7 @@ 

                                                SLAPD_DISCONNECT_IDLE_TIMEOUT, EAGAIN);

                  }

              }

-             PR_ExitMonitor(c->c_mutex);

+             pthread_mutex_unlock(&(c->c_mutex));

          }

      }

  }
@@ -1668,7 +1606,7 @@ 

      Connection *c = (Connection *)ns_job_get_data(job);

      int do_yield = 0;

  

-     PR_EnterMonitor(c->c_mutex);

+     pthread_mutex_lock(&(c->c_mutex));

      /* Assert we really have the right job state. */

      PR_ASSERT(job == c->c_job);

  
@@ -1678,7 +1616,7 @@ 

      /* Because handle closure will add a new job, we need to detach our current one. */

      c->c_job = NULL;

      do_yield = ns_handle_closure_nomutex(c);

-     PR_ExitMonitor(c->c_mutex);

+     pthread_mutex_unlock(&(c->c_mutex));

      /* Remove this task now. */

      ns_job_done(job);

      if (do_yield) {
@@ -1855,7 +1793,7 @@ 

  {

      Connection *c = (Connection *)ns_job_get_data(job);

  

-     PR_EnterMonitor(c->c_mutex);

+     pthread_mutex_lock(&(c->c_mutex));

      /* Assert we really have the right job state. */

      PR_ASSERT(job == c->c_job);

  
@@ -1921,7 +1859,7 @@ 

                        c->c_connid, c->c_sd);

      }

      /* Since we call done on the job, we need to remove it here. */

-     PR_ExitMonitor(c->c_mutex);

+     pthread_mutex_unlock(&(c->c_mutex));

      ns_job_done(job);

      return;

  }
@@ -2390,7 +2328,7 @@ 

          PR_Close(pr_acceptfd);

          return -1;

      }

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

  

      /*

       * Set the default idletimeout and the handle.  We'll update c_idletimeout
@@ -2478,7 +2416,7 @@ 

          connection_table_move_connection_on_to_active_list(the_connection_table, conn);

      }

  

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

  

      g_increment_current_conn_count();

  
@@ -2529,9 +2467,9 @@ 

       * that poll() was avoided, even at the expense of putting this new fd back

       * in nunc-stans to poll for read ready.

       */

-     PR_EnterMonitor(c->c_mutex);

+     pthread_mutex_lock(&(c->c_mutex));

      ns_connection_post_io_or_closing(c);

-     PR_ExitMonitor(c->c_mutex);

+     pthread_mutex_unlock(&(c->c_mutex));

      return;

  }

  

@@ -136,10 +136,10 @@ 

       */

      slapi_pblock_get(pb, SLAPI_CONNECTION, &pb_conn);

      if (pb_conn) {

-         PR_EnterMonitor(pb_conn->c_mutex);

+         pthread_mutex_lock(&(pb_conn->c_mutex));

          pb_conn->c_flags |= CONN_FLAG_IMPORT;

          pb_conn->c_bi_backend = be;

-         PR_ExitMonitor(pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pb_conn->c_mutex));

      }

  

      slapi_pblock_set(pb, SLAPI_EXT_OP_RET_OID, EXTOP_BULK_IMPORT_START_OID);
@@ -164,11 +164,11 @@ 

      Connection *pb_conn;

  

      slapi_pblock_get(pb, SLAPI_CONNECTION, &pb_conn);

-     PR_EnterMonitor(pb_conn->c_mutex);

+     pthread_mutex_lock(&(pb_conn->c_mutex));

      pb_conn->c_flags &= ~CONN_FLAG_IMPORT;

      be = pb_conn->c_bi_backend;

      pb_conn->c_bi_backend = NULL;

-     PR_ExitMonitor(pb_conn->c_mutex);

+     pthread_mutex_unlock(&(pb_conn->c_mutex));

  

      if ((be == NULL) || (be->be_wire_import == NULL)) {

          /* can this even happen? */

@@ -542,7 +542,7 @@ 

  {

      int rc;

  

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /* rc = connection_acquire_nolock(conn); */

      /* connection in the closing state can't be acquired */

      if (conn->c_flags & CONN_FLAG_CLOSING) {
@@ -555,7 +555,7 @@ 

          conn->c_refcnt++;

          rc = 0;

      }

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      return (rc);

  }

  
@@ -565,7 +565,7 @@ 

      int rc = 0;

      Slapi_Operation **olist = &conn->c_ops;

      Slapi_Operation **tmp;

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /* connection_remove_operation_ext(pb, conn,op); */

      for (tmp = olist; *tmp != NULL && *tmp != op; tmp = &(*tmp)->o_next)

          ; /* NULL */
@@ -594,7 +594,7 @@ 

              rc = 0;

          }

      }

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      return (rc);

  }

  

@@ -660,7 +660,7 @@ 

         * In async paged result case, the search result might be released

         * by other theads.  We need to double check it in the locked region.

         */

-             PR_EnterMonitor(pb_conn->c_mutex);

+             pthread_mutex_lock(&(pb_conn->c_mutex));

              pr_search_result = pagedresults_get_search_result(pb_conn, operation, 1 /*locked*/, pr_idx);

              if (pr_search_result) {

                  if (pagedresults_is_abandoned_or_notavailable(pb_conn, 1 /*locked*/, pr_idx)) {
@@ -668,7 +668,7 @@ 

                      /* Previous operation was abandoned and the simplepaged object is not in use. */

                      send_ldap_result(pb, 0, NULL, "Simple Paged Results Search abandoned", 0, NULL);

                      rc = LDAP_SUCCESS;

-                     PR_ExitMonitor(pb_conn->c_mutex);

+                     pthread_mutex_unlock(&(pb_conn->c_mutex));

                      goto free_and_return;

                  } else {

                      slapi_pblock_set(pb, SLAPI_SEARCH_RESULT_SET, pr_search_result);
@@ -682,7 +682,7 @@ 

                  pr_stat = PAGEDRESULTS_SEARCH_END;

                  rc = LDAP_SUCCESS;

              }

-             PR_ExitMonitor(pb_conn->c_mutex);

+             pthread_mutex_unlock(&(pb_conn->c_mutex));

              pagedresults_unlock(pb_conn, pr_idx);

  

              if ((PAGEDRESULTS_SEARCH_END == pr_stat) || (0 == pnentries)) {
@@ -803,10 +803,10 @@ 

                  /* PAGED RESULTS */

                  if (op_is_pagedresults(operation)) {

                      /* cleanup the slot */

-                     PR_EnterMonitor(pb_conn->c_mutex);

+                     pthread_mutex_lock(&(pb_conn->c_mutex));

                      pagedresults_set_search_result(pb_conn, operation, NULL, 1, pr_idx);

                      rc = pagedresults_set_current_be(pb_conn, NULL, pr_idx, 1);

-                     PR_ExitMonitor(pb_conn->c_mutex);

+                     pthread_mutex_unlock(&(pb_conn->c_mutex));

                  }

                  if (1 == flag_no_such_object) {

                      break;
@@ -845,11 +845,11 @@ 

                      slapi_pblock_get(pb, SLAPI_SEARCH_RESULT_SET, &sr);

                      if ((PAGEDRESULTS_SEARCH_END == pr_stat) || (0 == pnentries)) {

                          /* no more entries, but at least another backend */

-                         PR_EnterMonitor(pb_conn->c_mutex);

+                         pthread_mutex_lock(&(pb_conn->c_mutex));

                          pagedresults_set_search_result(pb_conn, operation, NULL, 1, pr_idx);

                          be->be_search_results_release(&sr);

                          rc = pagedresults_set_current_be(pb_conn, next_be, pr_idx, 1);

-                         PR_ExitMonitor(pb_conn->c_mutex);

+                         pthread_mutex_unlock(&(pb_conn->c_mutex));

                          pr_stat = PAGEDRESULTS_SEARCH_END; /* make sure stat is SEARCH_END */

                          if (NULL == next_be) {

                              /* no more entries && no more backends */
@@ -875,9 +875,9 @@ 

                      next_be = NULL; /* to break the loop */

                      if (operation->o_status & SLAPI_OP_STATUS_ABANDONED) {

                          /* It turned out this search was abandoned. */

-                         PR_EnterMonitor(pb_conn->c_mutex);

+                         pthread_mutex_lock(&(pb_conn->c_mutex));

                          pagedresults_free_one_msgid_nolock(pb_conn, operation->o_msgid);

-                         PR_ExitMonitor(pb_conn->c_mutex);

+                         pthread_mutex_unlock(&(pb_conn->c_mutex));

                          /* paged-results-request was abandoned; making an empty cookie. */

                          pagedresults_set_response_control(pb, 0, estimate, -1, pr_idx);

                          send_ldap_result(pb, 0, NULL, "Simple Paged Results Search abandoned", 0, NULL);

@@ -98,7 +98,7 @@ 

          return LDAP_UNWILLING_TO_PERFORM;

      }

  

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /* the ber encoding is no longer needed */

      ber_free(ber, 1);

      if (cookie.bv_len <= 0) {
@@ -206,7 +206,7 @@ 

              }

          }

      }

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

  

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_parse_control_value",

                    "<= idx %d\n", *index);
@@ -300,7 +300,7 @@ 

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_free_one",

                    "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (conn->c_pagedresults.prl_count <= 0) {

              slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_free_one",

                            "conn=%" PRIu64 " paged requests list count is %d\n",
@@ -311,7 +311,7 @@ 

              conn->c_pagedresults.prl_count--;

              rc = 0;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

  

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_free_one", "<= %d\n", rc);
@@ -363,11 +363,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_current_be", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              be = conn->c_pagedresults.prl_list[index].pr_current_be;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_current_be", "<= %p\n", be);
@@ -382,13 +382,13 @@ 

                    "pagedresults_set_current_be", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

          if (!nolock)

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              conn->c_pagedresults.prl_list[index].pr_current_be = be;

          }

          rc = 0;

          if (!nolock)

-             PR_ExitMonitor(conn->c_mutex);

+             pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_current_be", "<= %d\n", rc);
@@ -407,13 +407,13 @@ 

                    locked ? "locked" : "not locked", index);

      if (conn && (index > -1)) {

          if (!locked) {

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

          }

          if (index < conn->c_pagedresults.prl_maxlen) {

              sr = conn->c_pagedresults.prl_list[index].pr_search_result_set;

          }

          if (!locked) {

-             PR_ExitMonitor(conn->c_mutex);

+             pthread_mutex_unlock(&(conn->c_mutex));

          }

      }

      slapi_log_err(SLAPI_LOG_TRACE,
@@ -433,7 +433,7 @@ 

                    index, sr);

      if (conn && (index > -1)) {

          if (!locked)

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              PagedResults *prp = conn->c_pagedresults.prl_list + index;

              if (!(prp->pr_flags & CONN_FLAG_PAGEDRESULTS_ABANDONED) || !sr) {
@@ -443,7 +443,7 @@ 

              rc = 0;

          }

          if (!locked)

-             PR_ExitMonitor(conn->c_mutex);

+             pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_search_result", "=> %d\n", rc);
@@ -460,11 +460,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_search_result_count", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              count = conn->c_pagedresults.prl_list[index].pr_search_result_count;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_search_result_count", "<= %d\n", count);
@@ -481,11 +481,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_search_result_count", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              conn->c_pagedresults.prl_list[index].pr_search_result_count = count;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

          rc = 0;

      }

      slapi_log_err(SLAPI_LOG_TRACE,
@@ -506,11 +506,11 @@ 

                    "pagedresults_get_search_result_set_size_estimate",

                    "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              count = conn->c_pagedresults.prl_list[index].pr_search_result_set_size_estimate;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_search_result_set_size_estimate", "<= %d\n",
@@ -532,11 +532,11 @@ 

                    "pagedresults_set_search_result_set_size_estimate",

                    "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              conn->c_pagedresults.prl_list[index].pr_search_result_set_size_estimate = count;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

          rc = 0;

      }

      slapi_log_err(SLAPI_LOG_TRACE,
@@ -555,11 +555,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_with_sort", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              flags = conn->c_pagedresults.prl_list[index].pr_flags & CONN_FLAG_PAGEDRESULTS_WITH_SORT;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_with_sort", "<= %d\n", flags);
@@ -576,14 +576,14 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_with_sort", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              if (flags & OP_FLAG_SERVER_SIDE_SORTING) {

                  conn->c_pagedresults.prl_list[index].pr_flags |=

                      CONN_FLAG_PAGEDRESULTS_WITH_SORT;

              }

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

          rc = 0;

      }

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_set_with_sort", "<= %d\n", rc);
@@ -600,11 +600,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_unindexed", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              flags = conn->c_pagedresults.prl_list[index].pr_flags & CONN_FLAG_PAGEDRESULTS_UNINDEXED;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_unindexed", "<= %d\n", flags);
@@ -621,12 +621,12 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_unindexed", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              conn->c_pagedresults.prl_list[index].pr_flags |=

                  CONN_FLAG_PAGEDRESULTS_UNINDEXED;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

          rc = 0;

      }

      slapi_log_err(SLAPI_LOG_TRACE,
@@ -644,11 +644,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_sort_result_code", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              code = conn->c_pagedresults.prl_list[index].pr_sort_result_code;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_get_sort_result_code", "<= %d\n", code);
@@ -665,11 +665,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_sort_result_code", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              conn->c_pagedresults.prl_list[index].pr_sort_result_code = code;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

          rc = 0;

      }

      slapi_log_err(SLAPI_LOG_TRACE,
@@ -687,11 +687,11 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_timelimit", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              slapi_timespec_expire_at(timelimit, &(conn->c_pagedresults.prl_list[index].pr_timelimit_hr));

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

          rc = 0;

      }

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_set_timelimit", "<= %d\n", rc);
@@ -746,7 +746,7 @@ 

      }

  

      if (needlock) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

      }

      for (i = 0; conn->c_pagedresults.prl_list &&

                  i < conn->c_pagedresults.prl_maxlen;
@@ -765,7 +765,7 @@ 

      }

      conn->c_pagedresults.prl_count = 0;

      if (needlock) {

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_cleanup", "<= %d\n", rc);

      return rc;
@@ -792,7 +792,7 @@ 

      }

  

      if (needlock) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

      }

      for (i = 0; conn->c_pagedresults.prl_list &&

                  i < conn->c_pagedresults.prl_maxlen;
@@ -812,7 +812,7 @@ 

      conn->c_pagedresults.prl_maxlen = 0;

      conn->c_pagedresults.prl_count = 0;

      if (needlock) {

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE, "pagedresults_cleanup_all", "<= %d\n", rc);

      return rc;
@@ -831,7 +831,7 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_check_or_set_processing", "=>\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              ret = (conn->c_pagedresults.prl_list[index].pr_flags &

                     CONN_FLAG_PAGEDRESULTS_PROCESSING);
@@ -839,7 +839,7 @@ 

              conn->c_pagedresults.prl_list[index].pr_flags |=

                                                CONN_FLAG_PAGEDRESULTS_PROCESSING;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_check_or_set_processing", "<= %d\n", ret);
@@ -858,7 +858,7 @@ 

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_reset_processing", "=> idx=%d\n", index);

      if (conn && (index > -1)) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              ret = (conn->c_pagedresults.prl_list[index].pr_flags &

                     CONN_FLAG_PAGEDRESULTS_PROCESSING);
@@ -866,7 +866,7 @@ 

              conn->c_pagedresults.prl_list[index].pr_flags &=

                                               ~CONN_FLAG_PAGEDRESULTS_PROCESSING;

          }

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_reset_processing", "<= %d\n", ret);
@@ -977,9 +977,9 @@ 

      if (!conn || (index < 0) || (index >= conn->c_pagedresults.prl_maxlen)) {

          return;

      }

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      prp = conn->c_pagedresults.prl_list + index;

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      if (prp->pr_mutex) {

          PR_Lock(prp->pr_mutex);

      }
@@ -993,9 +993,9 @@ 

      if (!conn || (index < 0) || (index >= conn->c_pagedresults.prl_maxlen)) {

          return;

      }

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      prp = conn->c_pagedresults.prl_list + index;

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      if (prp->pr_mutex) {

          PR_Unlock(prp->pr_mutex);

      }
@@ -1010,11 +1010,11 @@ 

          return 1; /* not abandoned, but do not want to proceed paged results op. */

      }

      if (!locked) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

      }

      prp = conn->c_pagedresults.prl_list + index;

      if (!locked) {

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

      return prp->pr_flags & CONN_FLAG_PAGEDRESULTS_ABANDONED;

  }
@@ -1039,13 +1039,14 @@ 

                    "pagedresults_set_search_result_pb", "=> idx=%d, sr=%p\n", index, sr);

      if (conn && (index > -1)) {

          if (!locked)

-             PR_EnterMonitor(conn->c_mutex);

+             pthread_mutex_lock(&(conn->c_mutex));

          if (index < conn->c_pagedresults.prl_maxlen) {

              conn->c_pagedresults.prl_list[index].pr_search_result_set = sr;

              rc = 0;

          }

-         if (!locked)

-             PR_ExitMonitor(conn->c_mutex);

+         if (!locked) {

+             pthread_mutex_unlock(&(conn->c_mutex));

+         }

      }

      slapi_log_err(SLAPI_LOG_TRACE,

                    "pagedresults_set_search_result_pb", "<= %d\n", rc);

file modified
+32 -32
@@ -427,9 +427,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_DN \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(char **)value) = (NULL == pblock->pb_conn->c_dn ? NULL : slapi_ch_strdup(pblock->pb_conn->c_dn));

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_AUTHTYPE: /* deprecated */

          if (pblock->pb_conn == NULL) {
@@ -437,9 +437,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_AUTHTYPE \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          authtype = pblock->pb_conn->c_authtype;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          if (authtype == NULL) {

              (*(char **)value) = NULL;

          } else if (strcasecmp(authtype, SLAPD_AUTH_NONE) == 0) {
@@ -464,44 +464,44 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_AUTHMETHOD \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(char **)value) = pblock->pb_conn->c_authtype ? slapi_ch_strdup(pblock->pb_conn->c_authtype) : NULL;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_CLIENTNETADDR:

          if (pblock->pb_conn == NULL) {

              memset(value, 0, sizeof(PRNetAddr));

              break;

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          if (pblock->pb_conn->cin_addr == NULL) {

              memset(value, 0, sizeof(PRNetAddr));

          } else {

              (*(PRNetAddr *)value) =

                  *(pblock->pb_conn->cin_addr);

          }

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_SERVERNETADDR:

          if (pblock->pb_conn == NULL) {

              memset(value, 0, sizeof(PRNetAddr));

              break;

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          if (pblock->pb_conn->cin_destaddr == NULL) {

              memset(value, 0, sizeof(PRNetAddr));

          } else {

              (*(PRNetAddr *)value) =

                  *(pblock->pb_conn->cin_destaddr);

          }

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_CLIENTIP:

          if (pblock->pb_conn == NULL) {

              memset(value, 0, sizeof(struct in_addr));

              break;

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          if (pblock->pb_conn->cin_addr == NULL) {

              memset(value, 0, sizeof(struct in_addr));

          } else {
@@ -516,14 +516,14 @@ 

                  memset(value, 0, sizeof(struct in_addr));

              }

          }

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_SERVERIP:

          if (pblock->pb_conn == NULL) {

              memset(value, 0, sizeof(struct in_addr));

              break;

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          if (pblock->pb_conn->cin_destaddr == NULL) {

              memset(value, 0, sizeof(PRNetAddr));

          } else {
@@ -538,7 +538,7 @@ 

                  memset(value, 0, sizeof(struct in_addr));

              }

          }

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_IS_REPLICATION_SESSION:

          if (pblock->pb_conn == NULL) {
@@ -546,9 +546,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_IS_REPLICATION_SESSION \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(int *)value) = pblock->pb_conn->c_isreplication_session;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_IS_SSL_SESSION:

          if (pblock->pb_conn == NULL) {
@@ -556,9 +556,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_IS_SSL_SESSION \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(int *)value) = pblock->pb_conn->c_flags & CONN_FLAG_SSL;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_SASL_SSF:

          if (pblock->pb_conn == NULL) {
@@ -566,9 +566,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_SASL_SSF \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(int *)value) = pblock->pb_conn->c_sasl_ssf;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_SSL_SSF:

          if (pblock->pb_conn == NULL) {
@@ -576,9 +576,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_SSL_SSF \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(int *)value) = pblock->pb_conn->c_ssl_ssf;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_LOCAL_SSF:

          if (pblock->pb_conn == NULL) {
@@ -586,9 +586,9 @@ 

                            "slapi_pblock_get", "Connection is NULL and hence cannot access SLAPI_CONN_LOCAL_SSF \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          (*(int *)value) = pblock->pb_conn->c_local_ssf;

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_CERT:

          if (pblock->pb_conn == NULL) {
@@ -2566,10 +2566,10 @@ 

                            "Connection is NULL and hence cannot access SLAPI_CONN_AUTHMETHOD \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          slapi_ch_free((void **)&pblock->pb_conn->c_authtype);

          pblock->pb_conn->c_authtype = slapi_ch_strdup((char *)value);

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

      case SLAPI_CONN_IS_REPLICATION_SESSION:

          if (pblock->pb_conn == NULL) {
@@ -2578,9 +2578,9 @@ 

                            "Connection is NULL and hence cannot access SLAPI_CONN_IS_REPLICATION_SESSION \n");

              return (-1);

          }

-         PR_EnterMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_lock(&(pblock->pb_conn->c_mutex));

          pblock->pb_conn->c_isreplication_session = *((int *)value);

-         PR_ExitMonitor(pblock->pb_conn->c_mutex);

+         pthread_mutex_unlock(&(pblock->pb_conn->c_mutex));

          break;

  

      /* stuff related to config file processing */
@@ -4191,7 +4191,7 @@ 

  bind_credentials_clear(Connection *conn, PRBool lock_conn, PRBool clear_externalcreds)

  {

      if (lock_conn) {

-         PR_EnterMonitor(conn->c_mutex);

+         pthread_mutex_lock(&(conn->c_mutex));

      }

  

      if (conn->c_dn != NULL) {                   /* a non-anonymous bind has occurred */
@@ -4217,7 +4217,7 @@ 

      }

  

      if (lock_conn) {

-         PR_ExitMonitor(conn->c_mutex);

+         pthread_mutex_unlock(&(conn->c_mutex));

      }

  }

  
@@ -4380,10 +4380,10 @@ 

  void

  bind_credentials_set(Connection *conn, char *authtype, char *normdn, char *extauthtype, char *externaldn, CERTCertificate *clientcert, Slapi_Entry *bind_target_entry)

  {

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      bind_credentials_set_nolock(conn, authtype, normdn,

                                  extauthtype, externaldn, clientcert, bind_target_entry);

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

  }

  

  void

file modified
+4 -4
@@ -278,9 +278,9 @@ 

  

      /* need to acquire a reference to this connection so that it will not

         be released or cleaned up out from under us */

-     PR_EnterMonitor(pb_conn->c_mutex);

+     pthread_mutex_lock(&(pb_conn->c_mutex));

      conn_acq_flag = connection_acquire_nolock(pb_conn);

-     PR_ExitMonitor(pb_conn->c_mutex);

+     pthread_mutex_unlock(&(pb_conn->c_mutex));

  

      if (conn_acq_flag) {

          slapi_log_err(SLAPI_LOG_CONNS, "ps_send_results",
@@ -397,7 +397,7 @@ 

  

      conn = pb_conn; /* save to release later - connection_remove_operation_ext will NULL the pb_conn */

      /* Clean up the connection structure */

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

  

      slapi_log_err(SLAPI_LOG_CONNS, "ps_send_results",

                    "conn=%" PRIu64 " op=%d Releasing the connection and operation\n",
@@ -409,7 +409,7 @@ 

      if (conn_acq_flag == 0) { /* we acquired it, so release it */

          connection_release_nolock(conn);

      }

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      conn = NULL;

  

      PR_DestroyLock(ps->ps_lock);

file modified
+13 -13
@@ -781,7 +781,7 @@ 

          sasl_conn = (sasl_conn_t *)pb_conn->c_sasl_conn;

          if (sasl_conn != NULL) {

              /* sasl library mechanisms are connection dependent */

-             PR_EnterMonitor(pb_conn->c_mutex);

+             pthread_mutex_lock(&(pb_conn->c_mutex));

              if (sasl_listmech(sasl_conn,

                                NULL, /* username */

                                "", ",", "",
@@ -795,7 +795,7 @@ 

                  charray_free(others);

                  slapi_ch_free((void **)&dupstr);

              }

-             PR_ExitMonitor(pb_conn->c_mutex);

+             pthread_mutex_unlock(&(pb_conn->c_mutex));

          }

      }

  
@@ -889,13 +889,13 @@ 

          return;

      }

  

-     PR_EnterMonitor(pb_conn->c_mutex); /* BIG LOCK */

+     pthread_mutex_lock(&(pb_conn->c_mutex)); /* BIG LOCK */

      continuing = pb_conn->c_flags & CONN_FLAG_SASL_CONTINUE;

      pb_conn->c_flags &= ~CONN_FLAG_SASL_CONTINUE; /* reset flag */

  

      sasl_conn = (sasl_conn_t *)pb_conn->c_sasl_conn;

      if (sasl_conn == NULL) {

-         PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+         pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

          send_ldap_result(pb, LDAP_AUTH_METHOD_NOT_SUPPORTED, NULL,

                           "sasl library unavailable", 0, NULL);

          return;
@@ -979,7 +979,7 @@ 

          if (sasl_conn == NULL) {

              send_ldap_result(pb, LDAP_AUTH_METHOD_NOT_SUPPORTED, NULL,

                               "sasl library unavailable", 0, NULL);

-             PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+             pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

              return;

          }

      }
@@ -995,7 +995,7 @@ 

          /* retrieve the authenticated username */

          if (sasl_getprop(sasl_conn, SASL_USERNAME,

                           (const void **)&username) != SASL_OK) {

-             PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+             pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

              send_ldap_result(pb, LDAP_OPERATIONS_ERROR, NULL,

                               "could not obtain sasl username", 0, NULL);

              break;
@@ -1016,7 +1016,7 @@ 

              }

          }

          if (dn == NULL) {

-             PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+             pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

              send_ldap_result(pb, LDAP_OPERATIONS_ERROR, NULL,

                               "could not get auth dn from sasl", 0, NULL);

              break;
@@ -1058,7 +1058,7 @@ 

                                      slapi_ch_strdup(normdn),

                                      NULL, NULL, NULL, bind_target_entry);

  

-         PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+         pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

  

          if (plugin_call_plugins(pb, SLAPI_PLUGIN_PRE_BIND_FN) != 0) {

              break;
@@ -1116,9 +1116,9 @@ 

          /* see if we negotiated a security layer */

          if (*ssfp > 0) {

              /* Enable SASL I/O on the connection */

-             PR_EnterMonitor(pb_conn->c_mutex);

+             pthread_mutex_lock(&(pb_conn->c_mutex));

              connection_set_io_layer_cb(pb_conn, sasl_io_enable, NULL, NULL);

-             PR_ExitMonitor(pb_conn->c_mutex);

+             pthread_mutex_unlock(&(pb_conn->c_mutex));

          }

  

          /* send successful result */
@@ -1131,7 +1131,7 @@ 

  

      case SASL_CONTINUE: /* another step needed */

          pb_conn->c_flags |= CONN_FLAG_SASL_CONTINUE;

-         PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+         pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

  

          if (plugin_call_plugins(pb, SLAPI_PLUGIN_PRE_BIND_FN) != 0) {

              break;
@@ -1153,7 +1153,7 @@ 

  

      case SASL_NOMECH:

  

-         PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+         pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

          send_ldap_result(pb, LDAP_AUTH_METHOD_NOT_SUPPORTED, NULL,

                           "sasl mechanism not supported", 0, NULL);

          break;
@@ -1161,7 +1161,7 @@ 

      default: /* other error */

          errstr = sasl_errdetail(sasl_conn);

  

-         PR_ExitMonitor(pb_conn->c_mutex); /* BIG LOCK */

+         pthread_mutex_unlock(&(pb_conn->c_mutex)); /* BIG LOCK */

          slapi_pblock_set(pb, SLAPI_PB_RESULT_TEXT, (void *)errstr);

          send_ldap_result(pb, LDAP_INVALID_CREDENTIALS, NULL, NULL, 0, NULL);

          break;

file modified
+10 -1
@@ -1604,9 +1604,15 @@ 

  struct Conn_Private;

  typedef struct Conn_private Conn_private;

  

+ typedef enum _conn_state {

+     CONN_STATE_FREE = 0,

+     CONN_STATE_INIT = 1,

+ } conn_state;

+ 

  typedef struct conn

  {

      Sockbuf *c_sb;                   /* ber connection stuff          */

+     conn_state c_state;              /* Used in connection table and done to see what's free or not. Later we could use this for other state handlings. */

      int c_sd;                        /* the actual socket descriptor      */

      int c_ldapversion;               /* version of LDAP protocol       */

      char *c_dn;                      /* current DN bound to this conn  */
@@ -1630,7 +1636,10 @@ 

      uint64_t c_anonlimits_set;       /* default anon limits are set */

      PRInt32 c_threadnumber;          /* # threads used in this conn    */

      int c_refcnt;                    /* # ops refering to this conn    */

-     PRMonitor *c_mutex;              /* protect each conn structure; need to be re-entrant */

+     pthread_mutex_t c_mutex;         /* protect each conn structure; need to be re-entrant */

+                                      /* Note that c_mutex is a pthreadmutex to allow sharing

+                                       * into nunc-stans.

+                                       */

      PRLock *c_pdumutex;              /* only write one pdu at a time   */

      time_t c_idlesince;              /* last time of activity on conn  */

      int c_idletimeout;               /* local copy of idletimeout */

@@ -173,7 +173,7 @@ 

      /* At least we know that the request was indeed an Start TLS one. */

  

      slapi_pblock_get(pb, SLAPI_CONNECTION, &conn);

-     PR_EnterMonitor(conn->c_mutex);

+     pthread_mutex_lock(&(conn->c_mutex));

      /* cannot call slapi_send_ldap_result with mutex locked - will deadlock if ber_flush returns error */

      if (conn->c_prfd == (PRFileDesc *)NULL) {

          slapi_log_err(SLAPI_LOG_PLUGIN, "start_tls",
@@ -249,7 +249,7 @@ 

       * we send a success response back to the client. */

      ldapmsg = "Start TLS request accepted.Server willing to negotiate SSL.";

  unlock_and_return:

-     PR_ExitMonitor(conn->c_mutex);

+     pthread_mutex_unlock(&(conn->c_mutex));

      slapi_send_ldap_result(pb, ldaprc, NULL, ldapmsg, 0, NULL);

  

      return (SLAPI_PLUGIN_EXTENDED_SENT_RESULT);
@@ -317,7 +317,7 @@ 

         */

      }

  

-     PR_EnterMonitor(c->c_mutex);

+     pthread_mutex_lock(&(c->c_mutex));

  

      /* "Unimport" the socket from SSL, i.e. get rid of the upper layer of the

       * file descriptor stack, which represents SSL.
@@ -347,7 +347,7 @@ 

  

      bind_credentials_clear(c, PR_FALSE, PR_TRUE);

  

-     PR_ExitMonitor(c->c_mutex);

+     pthread_mutex_unlock(&(c->c_mutex));

  

      return (SLAPI_PLUGIN_EXTENDED_SENT_RESULT);

  }

file modified
+2 -2
@@ -75,9 +75,9 @@ 

      }

  

      /* target spec is used to decide which plugins are applicable for the operation */

-     PR_EnterMonitor(pb_conn->c_mutex);

+     pthread_mutex_lock(&(pb_conn->c_mutex));

      operation_set_target_spec_str(operation, pb_conn->c_dn);

-     PR_ExitMonitor(pb_conn->c_mutex);

+     pthread_mutex_unlock(&(pb_conn->c_mutex));

  

      /* ONREPL - plugins should be called and passed bind dn and, possibly, other data */

  

Bug Description: To allow nunc-stans to share the same lock as c_mutex
we need to change conn to use a pthread_mutex instead.

Fix Description: Change c_mutex to pthread

https://pagure.io/389-ds-base/issue/50459

Author: William Brown william@blackhats.net.au

Review by: ???

Passes basic tests, I will load/stress test this tomorrow.

Could you add the "invalid state" to the log message?

Why did you remove the pdumutex condition test? While it's unlikely it would fail, it doesn't hurt to keep the check.

Could you add the "invalid state" to the log message?

Never mind, I see there are only two possible states :-)

Anyway, some minor comments but the rest looks good, ack.

Minor point. Do you mind to set c->c_state = CONN_STATE_INIT only after pthread_mutex_init call.

@mreynolds Thanks for the detailed check, I'll fix these up now.

I think you maybe misread the diff, I left the pdumutex check inplace? I just removed the "c_mutex || pdumutex" and changed it to "pdumutex" only?

@tbordaz CONN_STATE_INIT must be the first thing we set to indicate we have started to change the structure, meaning that after this point it could be in a partial init state, requiring free. But not that it even matters, this is C, so the compiler can and will re-arrange all of these assignments based on what's fast, up until we init/lock a mutex. So really, no matter how we arrange this, the compiler will find a way to surprise you ;)

I'll fix the comment and log message now :)

1 new commit added

  • Update based on mark and thierry review
4 years ago

Sorry to be nit picker but I still have a concern.

Currently there is no real issue because the new connection handler is a single thread.
But with NS we can imagine parallel handler, that access the connection table in parallel. Correct ?
Is that part of code robust for parallel new connection handers ?
IMHO the ct->table_mutex should be acquired for the all functions.

Setting c_state=INIT before the real allocation of mutex means that an handler can return this connection while it is not yet fully initialized.

@tbordaz I think that's a concern for the NS ticket, not this one. This is just the change to pthread mutex, and I would like this merged as a single, simple unit of change. We can worry about the conntable management in the NS ticket which already has a number of these concurrency concerns. Is that reasonable for me to acknowledge your (very valid and thoughtful!) concern in the ns work ticket instead?

@firstyear, you are right. This concern existed before your patch. The patch looks very good. ACK.

Regarding the concern, I preferred to open a separated ticket (https://pagure.io/389-ds-base/issue/50489) as the NS ticket/PR (#49569 / https://pagure.io/389-ds-base/pull-request/49636#) is more focus on solving the double lock (c_mutex, job.monitor) deadlock than concurrent access to the connection table.

rebased onto d622686

4 years ago

Pull-Request has been merged by firstyear

4 years ago

It looks like the addition of the c_state change is highlighting a potential connection allocation issue under NS, patch to come shortly ...

389-ds-base is moving from Pagure to Github. This means that new issues and pull requests
will be accepted only in 389-ds-base's github repository.

This pull request has been cloned to Github as issue and is available here:
- https://github.com/389ds/389-ds-base/issues/3542

If you want to continue to work on the PR, please navigate to the github issue,
download the patch from the attachments and file a new pull request.

Thank you for understanding. We apologize for all inconvenience.

Pull-Request has been closed by spichugi

3 years ago