You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@celix.apache.org by GitBox <gi...@apache.org> on 2020/10/12 09:13:13 UTC

[GitHub] [celix] Oipo commented on a change in pull request #286: Feature/async svc registration

Oipo commented on a change in pull request #286:
URL: https://github.com/apache/celix/pull/286#discussion_r501061104



##########
File path: bundles/pubsub/pubsub_spi/src/pubsub_endpoint.c
##########
@@ -170,8 +170,9 @@ celix_properties_t* pubsubEndpoint_createFromPublisherTrackerInfo(bundle_context
     data.topic = topic;
     celix_bundleContext_useBundle(ctx, bundleId, &data, retrieveTopicProperties);
 
+    pubsubEndpoint_setFields(ep, fwUUID, scope, topic, PUBSUB_PUBLISHER_ENDPOINT_TYPE, NULL, NULL, NULL, data.props);
+
     if (data.props != NULL) {

Review comment:
       You can just remove this if statement, celix_properties_destroy also checks for NULL.

##########
File path: libs/framework/gtest/src/bundle_context_bundles_tests.cpp
##########
@@ -378,6 +378,90 @@ TEST_F(CelixBundleContextBundlesTests, trackBundlesTest) {
     celix_bundleContext_stopTracker(ctx, trackerId);
 };
 
+
+TEST_F(CelixBundleContextBundlesTests, trackBundlesTestAsync) {
+    struct data {
+        std::atomic<int> installedCount{0};
+        std::atomic<int> startedCount{0};
+        std::atomic<int> stoppedCount{0};
+    };
+    struct data data;
+
+    auto installed = [](void *handle, const bundle_t *bnd) {
+        auto *d = static_cast<struct data*>(handle);
+        EXPECT_TRUE(bnd != nullptr);
+        d->installedCount.fetch_add(1);

Review comment:
       Stylistic choice: `installedCount.fetch_add(1)` is semantically the same as `installedCount += 1`. It's only when you want to specifiy memory order that the former is the only option.

##########
File path: libs/framework/gtest/src/bundle_context_bundles_tests.cpp
##########
@@ -378,6 +378,90 @@ TEST_F(CelixBundleContextBundlesTests, trackBundlesTest) {
     celix_bundleContext_stopTracker(ctx, trackerId);
 };
 
+
+TEST_F(CelixBundleContextBundlesTests, trackBundlesTestAsync) {
+    struct data {
+        std::atomic<int> installedCount{0};
+        std::atomic<int> startedCount{0};
+        std::atomic<int> stoppedCount{0};
+    };
+    struct data data;
+
+    auto installed = [](void *handle, const bundle_t *bnd) {
+        auto *d = static_cast<struct data*>(handle);
+        EXPECT_TRUE(bnd != nullptr);
+        d->installedCount.fetch_add(1);
+    };
+
+    auto started = [](void *handle, const bundle_t *bnd) {
+        auto *d = static_cast<struct data*>(handle);
+        EXPECT_TRUE(bnd != nullptr);
+        d->startedCount.fetch_add(1);
+    };
+
+    auto stopped = [](void *handle, const bundle_t *bnd) {
+        auto *d = static_cast<struct data*>(handle);
+        if (bnd == nullptr) {
+            celix_logUtils_logToStdout("test", CELIX_LOG_LEVEL_ERROR, "bnd should not be null");
+        }
+        EXPECT_TRUE(bnd != nullptr);
+        d->stoppedCount.fetch_add(1);
+    };
+
+    celix_bundle_tracking_options_t opts{};
+    opts.callbackHandle = static_cast<void*>(&data);
+    opts.onInstalled = installed;
+    opts.onStarted = started;
+    opts.onStopped = stopped;
+
+    long bundleId1 = celix_bundleContext_installBundle(ctx, TEST_BND1_LOC, true);
+    celix_framework_waitForEmptyEventQueue(fw);
+    EXPECT_TRUE(bundleId1 >= 0);
+
+    /*
+     * NOTE for bundles already installed (TEST_BND1) the callbacks are called on the
+     * thread of celix_bundleContext_trackBundlesWithOptions.
+     * For Bundles installed after the celix_bundleContext_trackBundlesWithOptions function
+     * the called are called on the Celix framework event queue thread.
+     */
+    long trackerId = celix_bundleContext_trackBundlesWithOptionsAsync(ctx, &opts);
+    celix_bundleContext_waitForAsyncTracker(ctx, trackerId);
+    EXPECT_EQ(1, data.installedCount.load());

Review comment:
       Same stylistic choice here, the implicit conversion to int is the same as `.load()`.

##########
File path: libs/utils/src/utils.c
##########
@@ -150,9 +150,9 @@ int utils_compareServiceIdsAndRanking(unsigned long servId, long servRank, unsig
     if (servId == otherServId) {
         result = 0;
     } else if (servRank != otherServRank) {
-        result = servRank < otherServRank ? -1 : 1;

Review comment:
       Is this a bug or a breaking change?

##########
File path: libs/framework/include/celix_framework.h
##########
@@ -156,6 +156,12 @@ void celix_framework_waitForEmptyEventQueue(celix_framework_t *fw);
 void celix_framework_setLogCallback(celix_framework_t* fw, void* logHandle, void (*logFunction)(void* handle, celix_log_level_e level, const char* file, const char *function, int line, const char *format, va_list formatArgs));
 
 
+/**
+ * wait till all events for the bundle identified by the bndId are processed.
+ */
+void celix_framework_waitForEvents(celix_framework_t* fw, long bndId);

Review comment:
       A better name would probably be `celix_framework_waitUntilNoEventsForBnd` or something

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -382,30 +546,74 @@ typedef struct celix_service_tracking_options {
     .removeWithProperties = NULL, \
     .setWithOwner = NULL, \
     .addWithOwner = NULL, \
-    .removeWithOwner = NULL}
+    .removeWithOwner = NULL, \
+    .trackerCreatedCallbackData = NULL, \
+    .trackerCreatedCallback = NULL }
 #endif
 
 /**
  * Tracks services using the provided tracker options.
  * The tracker options are only using during this call and can safely be freed/reused after this call returns.
  *
+ * The service tracker will be created async on the Celix event loop thread. This means that the function can return
+ * before the tracker is created.
+ *
+ * @param ctx The bundle context.
+ * @param opts The pointer to the tracker options.
+ * @return the tracker id (>=0) or < 0 if unsuccessful.
+ */
+long celix_bundleContext_trackServicesWithOptionsAsync(celix_bundle_context_t *ctx, const celix_service_tracking_options_t *opts);

Review comment:
       Does `celix_service_tracking_options_t` contain a completion handler? Would be useful here as well.

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -248,15 +353,39 @@ long celix_bundleContext_findServiceWithOptions(celix_bundle_context_t *ctx, con
  */
 celix_array_list_t* celix_bundleContext_findServicesWithOptions(celix_bundle_context_t *ctx, const celix_service_filter_options_t *opts);
 
+/**
+ * track the highest ranking service with the provided serviceName.
+ * The highest ranking services will used for the callback.
+ * If a new and higher ranking services the callback with be called again with the new service.
+ * If a service is removed a the callback with be called with next highest ranking service or NULL as service.
+ *
+ * The service tracker will be created async on the Celix event loop thread. This means that the function can return
+ * before the tracker is created.
+ *
+ * @param ctx The bundle context.
+ * @param serviceName The required service name to track.
+ *                    If NULL is all service are tracked.
+ * @param callbackHandle The data pointer, which will be used in the callbacks
+ * @param set is a required callback, which will be called when a new highest ranking service is set.
+ * @return the tracker id (>=0) or < 0 if unsuccessful.
+ */
+long celix_bundleContext_trackServiceAsync(
+        celix_bundle_context_t* ctx,
+        const char* serviceName,
+        void* callbackHandle,
+        void (*set)(void* handle, void* svc)

Review comment:
       A completion handler would be useful here as well

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -137,18 +190,50 @@ typedef struct celix_service_registration_options {
     .serviceName = NULL, \
     .properties = NULL, \
     .serviceLanguage = NULL, \
-    .serviceVersion = NULL }
+    .serviceVersion = NULL, \
+    .asyncData = NULL, \
+    .asyncCallback = NULL }
 #endif
 
+/**
+ * Register a service to the Celix framework using the provided service registration options.
+ *
+ * The service will be registered async on the Celix event loop thread. This means that service registration is (probably)
+ * not yet concluded when this function returns, but is added to the event loop..
+ * Use celix_bundleContext_waitForAsyncRegistration to synchronise with the
+ * actual service registration in the framework's service registry.
+ *
+ * @param ctx The bundle context
+ * @param opts The pointer to the registration options. The options are only in the during registration call.
+ * @return The serviceId (>= 0) or -1 if the registration was unsuccessful and -2 if the registration was cancelled (@see celix_bundleContext_reserveSvcId).
+ */
+long celix_bundleContext_registerServiceWithOptionsAsync(celix_bundle_context_t *ctx, const celix_service_registration_options_t *opts);

Review comment:
       Does `celix_service_registration_options_t` contain a completion handler? Would be useful here as well.

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -40,21 +40,63 @@ extern "C" {
 #define OPTS_INIT
 #endif
 
+
+/**
+ * Register a service to the Celix framework.
+ *
+ * The service will be registered async on the Celix event loop thread. This means that service registration is (probably)
+ * not yet concluded when this function returns, but is added to the event loop.
+ * Use celix_bundleContext_waitForAsyncRegistration to synchronise with the

Review comment:
       What would probably fit the event loop idiom better is a completion handler. Though `celix_bundleContext_waitForAsyncRegistration` could still be allowed for when sleeping is fine.

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -266,13 +395,37 @@ long celix_bundleContext_trackService(
         const char* serviceName,
         void* callbackHandle,
         void (*set)(void* handle, void* svc)
+); //__attribute__((deprecated("Use celix_bundleContext_trackServiceSync instead!")));
+
+/**
+ * track services with the provided serviceName.
+ *
+ * The service tracker will be created async on the Celix event loop thread. This means that the function can return
+ * before the tracker is created.
+ *
+ * @param ctx The bundle context.
+ * @param serviceName The required service name to track
+ *                    If NULL is all service are tracked.
+ * @param callbackHandle The data pointer, which will be used in the callbacks
+ * @param add is a required callback, which will be called when a service is added and initially for the existing service.
+ * @param remove is a required callback, which will be called when a service is removed
+ * @return the tracker id (>=0) or < 0 if unsuccessful.
+ */
+long celix_bundleContext_trackServicesAsync(
+        celix_bundle_context_t* ctx,
+        const char* serviceName,
+        void* callbackHandle,
+        void (*add)(void* handle, void* svc),
+        void (*remove)(void* handle, void* svc)

Review comment:
       A completion handler would be useful here as well

##########
File path: libs/framework/include/service_registry.h
##########
@@ -134,6 +144,37 @@ bool celix_serviceRegistry_getServiceInfo(
 long celix_serviceRegistry_nextSvcId(celix_service_registry_t* registry);
 
 
+/**
+ * Unregister service for the provided service id (owned by bnd).
+ * Will print an error if the service id is invalid
+ */
+void celix_serviceRegistry_unregisterService(celix_service_registry_t* registry, celix_bundle_t* bnd, long serviceId);
+
+
+/**
+ * Create a LDAP filter for the provided filter parts.
+ * @param serviceName       The optional service name
+ * @param versionRange      The optional version range
+ * @param filter            The optional filter

Review comment:
       name typo, should be `additionalFilter`. Also probably missing is why it is additional?

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -655,23 +863,46 @@ bool celix_bundleContext_startBundle(celix_bundle_context_t *ctx, long bndId);
  */
 char* celix_bundleContext_getBundleSymbolicName(celix_bundle_context_t *ctx, long bndId);
 
+
 /**
  * track bundles
  * The add bundle callback will also be called for already installed bundles.

Review comment:
       add bundle callback?

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -655,23 +863,46 @@ bool celix_bundleContext_startBundle(celix_bundle_context_t *ctx, long bndId);
  */
 char* celix_bundleContext_getBundleSymbolicName(celix_bundle_context_t *ctx, long bndId);
 
+
 /**
  * track bundles
  * The add bundle callback will also be called for already installed bundles.
  *
+ * The bundle tracker will be created async on the Celix event loop thread. This means that the function can return
+ * before the tracker is created.
+ *
  * @param ctx               The bundle context.
  * @param callbackHandle    The data pointer, which will be used in the callbacks
  * @param add               The callback which will be called for started bundles.
  * @param remove            The callback which will be called when bundles are stopped.
  * @return                  The bundle tracker id or < 0 if unsuccessful.
  */
-long celix_bundleContext_trackBundles(
+long celix_bundleContext_trackBundlesAsync(
         celix_bundle_context_t* ctx,
         void* callbackHandle,
         void (*onStarted)(void* handle, const celix_bundle_t *bundle),
         void (*onStopped)(void *handle, const celix_bundle_t *bundle)
 );
 
+/**
+ * track bundles
+ * The add bundle callback will also be called for already installed bundles.
+ *
+ * Note: please use celix_bundleContext_trackBundlesAsync instead.
+ *
+ * @param ctx               The bundle context.
+ * @param callbackHandle    The data pointer, which will be used in the callbacks
+ * @param add               The callback which will be called for started bundles.

Review comment:
       Yeah, probably onStarted/onStopped.

##########
File path: libs/framework/src/framework.c
##########
@@ -341,7 +331,7 @@ celix_status_t framework_destroy(framework_pt framework) {
             const char *bndName = celix_bundle_getSymbolicName(bnd);
             fw_log(framework->logger, CELIX_LOG_LEVEL_FATAL, "Cannot destroy framework. The use count of bundle %s (bnd id %li) is not 0, but %u.", bndName, entry->bndId, count);
             celixThreadMutex_lock(&framework->dispatcher.mutex);
-            int nrOfRequests = celix_arrayList_size(framework->dispatcher.requests);
+            int nrOfRequests = framework->dispatcher.eventQueueSize + celix_arrayList_size(framework->dispatcher.dynamicEventQueue);

Review comment:
       Can't comment on line 369, but because it's destroying a locked mutex, thread sanitizer crashes.

##########
File path: libs/framework/src/service_registry.c
##########
@@ -1046,6 +920,137 @@ static inline void celix_waitAndDestroyServiceListener(celix_service_registry_se
     free(entry);
 }
 
+/**
+ * Create a LDAP filter for the provided filter parts.

Review comment:
       This documentation should only be available in the header file, no?

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -655,23 +863,46 @@ bool celix_bundleContext_startBundle(celix_bundle_context_t *ctx, long bndId);
  */
 char* celix_bundleContext_getBundleSymbolicName(celix_bundle_context_t *ctx, long bndId);
 
+
 /**
  * track bundles
  * The add bundle callback will also be called for already installed bundles.
  *
+ * The bundle tracker will be created async on the Celix event loop thread. This means that the function can return
+ * before the tracker is created.
+ *
  * @param ctx               The bundle context.
  * @param callbackHandle    The data pointer, which will be used in the callbacks
  * @param add               The callback which will be called for started bundles.

Review comment:
       Probably onStarted/onStopped?

##########
File path: libs/framework/src/framework.c
##########
@@ -2528,13 +2631,108 @@ bool celix_framework_startBundle(celix_framework_t *fw, long bndId) {
 }
 
 void celix_framework_waitForEmptyEventQueue(celix_framework_t *fw) {
+    assert(!celix_framework_isCurrentThreadTheEventLoop(fw));
+
     celixThreadMutex_lock(&fw->dispatcher.mutex);
-    while ((celix_arrayList_size(fw->dispatcher.requests) + fw->dispatcher.nrOfLocalRequest) != 0) {
+    while (fw->dispatcher.eventQueueSize > 0 || celix_arrayList_size(fw->dispatcher.dynamicEventQueue) > 0) {
         celixThreadCondition_wait(&fw->dispatcher.cond, &fw->dispatcher.mutex);
     }
     celixThreadMutex_unlock(&fw->dispatcher.mutex);
 }
 
+void celix_framework_waitForEvents(celix_framework_t* fw, long bndId) {
+    assert(!celix_framework_isCurrentThreadTheEventLoop(fw));
+
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    bool eventInProgress = true;
+    while (eventInProgress) {
+        eventInProgress = false;
+        for (int i = 0; i < fw->dispatcher.eventQueueSize; ++i) {
+            int index = (fw->dispatcher.eventQueueFirstEntry + i) % CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE;
+            celix_framework_event_t* e = &fw->dispatcher.eventQueue[index];
+            if (e->bndEntry != NULL && e->bndEntry->bndId == bndId) {
+                eventInProgress = true;
+                break;
+            }
+        }
+        for (int i = 0; !eventInProgress && i < celix_arrayList_size(fw->dispatcher.dynamicEventQueue); ++i) {
+            celix_framework_event_t* e = celix_arrayList_get(fw->dispatcher.dynamicEventQueue, i);
+            if (e->bndEntry != NULL && e->bndEntry->bndId == bndId) {
+                eventInProgress = true;
+                break;
+            }
+        }
+        if (eventInProgress) {
+            celixThreadCondition_timedwaitRelative(&fw->dispatcher.cond, &fw->dispatcher.mutex, 5, 0);
+        }
+    }
+    celixThreadMutex_unlock(&fw->dispatcher.mutex);
+}
+
+
 void celix_framework_setLogCallback(celix_framework_t* fw, void* logHandle, void (*logFunction)(void* handle, celix_log_level_e level, const char* file, const char *function, int line, const char *format, va_list formatArgs)) {
     celix_frameworkLogger_setLogCallback(fw->logger, logHandle, logFunction);
+}
+
+
+long celix_framework_fireGenericEvent(framework_t* fw, long eventId, long bndId, const char *eventName, void* processData, void (*processCallback)(void *data), void* doneData, void (*doneCallback)(void* doneData)) {
+    celix_framework_bundle_entry_t* bndEntry = NULL;
+    if (bndId >=0) {
+        bndEntry = fw_bundleEntry_getBundleEntryAndIncreaseUseCount(fw, bndId);
+        if (bndEntry == NULL) {
+            fw_log(fw->logger, CELIX_LOG_LEVEL_ERROR, "Cannot find bundle for id %li", bndId);
+            return -1L;
+        }
+    }
+
+    if (eventId < 0) {
+        eventId = celix_framework_nextEventId(fw);
+    }
+
+    celix_framework_event_t event;
+    memset(&event, 0, sizeof(event));
+    event.type = CELIX_GENERIC_EVENT;
+    event.bndEntry = bndEntry;
+    event.genericEventId = eventId;
+    event.genericEventName = eventName;
+    event.genericProcessData = processData;
+    event.genericProcess = processCallback;
+    event.doneData = doneData;
+    event.doneCallback = doneCallback;
+    celix_framework_addToEventQueue(fw, &event);
+
+    return eventId;
+}
+
+long celix_framework_nextEventId(framework_t *fw) {
+    return __atomic_fetch_add(&fw->nextGenericEventId, 1, __ATOMIC_SEQ_CST);;

Review comment:
       Technically, this can even be `__ATOMIC_RELAXED`, because it's a simple counter. See https://stackoverflow.com/a/48148318/1460998 for a more detailed answer.

##########
File path: libs/framework/src/framework.c
##########
@@ -2015,67 +1934,112 @@ static void fw_handleEventRequest(celix_framework_t *framework, request_t* reque
         for (int i = 0; i < celix_arrayList_size(localListeners); ++i) {
             fw_bundle_listener_pt listener = arrayList_get(localListeners, i);
 
-            bundle_event_t event;
-            memset(&event, 0, sizeof(event));
-            event.bnd = request->bndEntry->bnd;
-            event.type = request->eventType;
-            fw_invokeBundleListener(framework, listener->listener, &event, listener->bundle);
+            bundle_event_t bEvent;
+            memset(&bEvent, 0, sizeof(bEvent));
+            bEvent.bnd = event->bndEntry->bnd;
+            bEvent.type = event->bundleEvent;
+            fw_invokeBundleListener(framework, listener->listener, &bEvent, listener->bundle);
 
             fw_bundleListener_decreaseUseCount(listener);
         }
         celix_arrayList_destroy(localListeners);
-    } else  if (request->type == FRAMEWORK_EVENT_TYPE) {
+    } else if (event->type == CELIX_FRAMEWORK_EVENT_TYPE) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw event");
         celixThreadMutex_lock(&framework->frameworkListenersLock);
-        //TODO refactor use of framework listeners to use a useCount + conditition.
         for (int i = 0; i < celix_arrayList_size(framework->frameworkListeners); ++i) {
             fw_framework_listener_pt listener = celix_arrayList_get(framework->frameworkListeners, i);
-            framework_event_t event;
-            memset(&event, 0, sizeof(event));
-            event.type = request->eventType;
-            event.error = request->error;
-            event.errorCode = request->errorCode;
+            framework_event_t fEvent;
+            memset(&fEvent, 0, sizeof(fEvent));
+            fEvent.type = event->fwEvent;
+            fEvent.error = event->error;
+            fEvent.errorCode = event->errorCode;
 
-            fw_invokeFrameworkListener(framework, listener->listener, &event, listener->bundle);
+            fw_invokeFrameworkListener(framework, listener->listener, &fEvent, listener->bundle);
         }
         celixThreadMutex_unlock(&framework->frameworkListenersLock);
+    } else if (event->type == CELIX_REGISTER_SERVICE_EVENT) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw service registration event for service '%s' with svc id %li", event->serviceName, event->registerServiceId);
+        service_registration_t* reg = NULL;
+        celix_status_t status;
+        if (event->factory != NULL) {
+            status = celix_serviceRegistry_registerServiceFactory(framework->registry, event->bndEntry->bnd, event->serviceName, event->factory, event->properties, event->registerServiceId, &reg);
+        } else {
+            status = celix_serviceRegistry_registerService(framework->registry, event->bndEntry->bnd, event->serviceName, event->svc, event->properties, event->registerServiceId, &reg);
+        }
+        if (status != CELIX_SUCCESS) {
+            fw_log(framework->logger, CELIX_LOG_LEVEL_ERROR, "Could not register service async. svc name is %s, error is %s", event->serviceName, celix_strerror(status));
+        } else if (event->registerCallback != NULL) {
+            event->registerCallback(event->registerData, serviceRegistration_getServiceId(reg));
+        }
+    } else if (event->type == CELIX_UNREGISTER_SERVICE_EVENT) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw service unregister event for service id %li", event->unregisterServiceId);
+        celix_serviceRegistry_unregisterService(framework->registry, event->bndEntry->bnd, event->unregisterServiceId);
+    } else if (event->type == CELIX_GENERIC_EVENT) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling event %s", event->eventName);
+        if (event->genericProcess != NULL) {
+            event->genericProcess(event->genericProcessData);
+        }
+    }
+
+    if (event->doneCallback != NULL) {
+        event->doneCallback(event->doneData);
     }
 }
 
-static inline void fw_handleEvents(celix_framework_t* framework, celix_array_list_t* localRequests) {
+static inline celix_framework_event_t* fw_topEventFromQueue(celix_framework_t* fw) {
+    celix_framework_event_t* e = NULL;
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    if (fw->dispatcher.eventQueueSize > 0) {
+        e = &fw->dispatcher.eventQueue[fw->dispatcher.eventQueueFirstEntry];
+    } else if (celix_arrayList_size(fw->dispatcher.dynamicEventQueue) > 0) {
+        e = celix_arrayList_get(fw->dispatcher.dynamicEventQueue, 0);
+    }
+    celixThreadMutex_unlock(&fw->dispatcher.mutex);
+    return e;
+}
+
+static inline bool fw_removeTopEventFromQueue(celix_framework_t* fw) {
+    bool dynamicallyAllocated = false;
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    if (fw->dispatcher.eventQueueSize > 0) {
+        fw->dispatcher.eventQueueFirstEntry = (fw->dispatcher.eventQueueFirstEntry+1) % CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE;
+        fw->dispatcher.eventQueueSize -= 1;

Review comment:
       Hmm, I think a dynamically sized ring buffer solves this quirk of having two separate internal event queues.
   
   Basically, have one `void *data`, one `uint64_t size`, one `uint64_t beginIndex` and one `uint64_t endIndex`.  The range between begin/end signify the relevant events. If `beginIndex == endIndex` the ringbuffer is empty and when inserting, `endIndex < beginIndex && endIndex + 1 == beginIndex || endIndex + 1 == size && beginIndex == 0` signifies it's full. In the latter case, allocate a new `void*` with twice the size and memcpy everything between begin/end. Insert a new event is also simply a memcpy.

##########
File path: libs/framework/include/service_registry.h
##########
@@ -95,6 +93,17 @@ celix_status_t celix_serviceRegistry_addServiceListener(celix_service_registry_t
 
 celix_status_t celix_serviceRegistry_removeServiceListener(celix_service_registry_t *reg, celix_service_listener_t *listener);
 
+bool celix_serviceRegistry_isServiceRegistered(celix_service_registry_t* reg, long serviceId);

Review comment:
       Missing doxygen-style documentation here. One of the questions I have, for example, is what does `reserveId` do?

##########
File path: libs/framework/src/bundle_context.c
##########
@@ -80,15 +86,19 @@ celix_status_t bundleContext_destroy(bundle_context_pt context) {
 	    celixThreadMutex_lock(&context->mutex);
 
 
-	    bundleContext_cleanupBundleTrackers(context);
-	    bundleContext_cleanupServiceTrackers(context);
-        bundleContext_cleanupServiceTrackerTrackers(context);
+	    assert(hashMap_size(context->bundleTrackers) == 0);
+        hashMap_destroy(context->bundleTrackers, false, false);
+        assert(hashMap_size(context->serviceTrackers) == 0);
+        hashMap_destroy(context->serviceTrackers, false, false);
+        assert(hashMap_size(context->metaTrackers) == 0);
+        hashMap_destroy(context->metaTrackers, false, false);
+        assert(celix_arrayList_size(context->svcRegistrations) == 0);
+        celix_arrayList_destroy(context->svcRegistrations);
 
-	    //NOTE still present service registrations will be cleared during bundle stop in the
+        //NOTE still present service registrations will be cleared during bundle stop in the

Review comment:
       But the context->svcRegistrations assert only 3 lines above says otherwise?

##########
File path: libs/framework/include/celix_bundle_context.h
##########
@@ -655,23 +863,46 @@ bool celix_bundleContext_startBundle(celix_bundle_context_t *ctx, long bndId);
  */
 char* celix_bundleContext_getBundleSymbolicName(celix_bundle_context_t *ctx, long bndId);
 
+
 /**
  * track bundles
  * The add bundle callback will also be called for already installed bundles.
  *
+ * The bundle tracker will be created async on the Celix event loop thread. This means that the function can return
+ * before the tracker is created.
+ *
  * @param ctx               The bundle context.
  * @param callbackHandle    The data pointer, which will be used in the callbacks
  * @param add               The callback which will be called for started bundles.
  * @param remove            The callback which will be called when bundles are stopped.
  * @return                  The bundle tracker id or < 0 if unsuccessful.
  */
-long celix_bundleContext_trackBundles(
+long celix_bundleContext_trackBundlesAsync(
         celix_bundle_context_t* ctx,
         void* callbackHandle,
         void (*onStarted)(void* handle, const celix_bundle_t *bundle),
         void (*onStopped)(void *handle, const celix_bundle_t *bundle)
 );
 
+/**
+ * track bundles
+ * The add bundle callback will also be called for already installed bundles.

Review comment:
       add bundle callback? Do you mean onStarted callback here?

##########
File path: libs/framework/src/framework.c
##########
@@ -1922,88 +1861,68 @@ bundle_context_t* framework_getContext(const_framework_pt framework) {
     return result;
 }
 
-celix_status_t fw_fireBundleEvent(framework_pt framework, bundle_event_type_e eventType, celix_framework_bundle_entry_t* entry) {
-    celix_status_t status = CELIX_SUCCESS;
-
+void fw_fireBundleEvent(framework_pt framework, bundle_event_type_e eventType, celix_framework_bundle_entry_t* entry) {
     if (eventType == OSGI_FRAMEWORK_BUNDLE_EVENT_STOPPING || eventType == OSGI_FRAMEWORK_BUNDLE_EVENT_UNINSTALLED || eventType == OSGI_FRAMEWORK_BUNDLE_EVENT_STOPPED) {
         if (entry->bndId == framework->bundleId) {
             //NOTE for framework bundle not triggering events while framework is stopped (and as result in use)
-            return CELIX_SUCCESS;
+            return;
         }
     }
 
-    request_t* request = (request_t*) calloc(1, sizeof(*request));
-    if (!request) {
-        status = CELIX_ENOMEM;
-    } else {
-        fw_bundleEntry_increaseUseCount(entry);
+    fw_bundleEntry_increaseUseCount(entry);
 
-        request->eventType = eventType;
-        request->filter = NULL;
-        request->type = BUNDLE_EVENT_TYPE;
-        request->error = NULL;
-        request->bndEntry = entry;
+    celix_framework_event_t event;
+    memset(&event, 0, sizeof(event));
+    event.type = CELIX_BUNDLE_EVENT_TYPE;
+    event.bndEntry = entry;
+    event.bundleEvent = eventType;
+    celix_framework_addToEventQueue(framework, &event);
+}
 
-        celixThreadMutex_lock(&framework->dispatcher.mutex);
-        if (framework->dispatcher.active) {
-            //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Adding dispatcher bundle event request for bnd id %li with event type %i", entry->bndId, eventType);
-            celix_arrayList_add(framework->dispatcher.requests, request);
-            celixThreadCondition_broadcast(&framework->dispatcher.cond);
-        } else {
-            /*
-             * NOTE because stopping the framework is done through stopping the framework bundle,
-             * most bundle stopping / stopped events cannot be fired.
-             *
-             * TBD if this needs to addressed.
-             */
-            fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Cannot fire event dispatcher not active. Event is %x for bundle %s", eventType, celix_bundle_getSymbolicName(entry->bnd));
-            fw_bundleEntry_decreaseUseCount(entry);
-            free(request);
-        }
-        celixThreadMutex_unlock(&framework->dispatcher.mutex);
+void fw_fireFrameworkEvent(framework_pt framework, framework_event_type_e eventType, celix_status_t errorCode) {
+    celix_framework_event_t event;
+    memset(&event, 0, sizeof(event));
+    event.type = CELIX_FRAMEWORK_EVENT_TYPE;
+    event.fwEvent = eventType;
+    event.errorCode = errorCode;
+    event.error = "";
+    if (errorCode != CELIX_SUCCESS) {
+        event.error = celix_strerror(errorCode);
     }
 
-    framework_logIfError(framework->logger, status, NULL, "Failed to fire bundle event");
-
-    return status;
+    celix_framework_addToEventQueue(framework, &event);
 }
 
-celix_status_t fw_fireFrameworkEvent(framework_pt framework, framework_event_type_e eventType, celix_status_t errorCode) {
-    celix_status_t status = CELIX_SUCCESS;
-
-    request_t* request = calloc(1, sizeof(*request));
-    if (!request) {
-        status = CELIX_ENOMEM;
+static void celix_framework_addToEventQueue(celix_framework_t *fw, const celix_framework_event_t* event) {
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    //try to add to static queue
+    if (celix_arrayList_size(fw->dispatcher.dynamicEventQueue) > 0) { //always to dynamic queue if not empty (to ensure order)
+        celix_framework_event_t *e = malloc(sizeof(*e));
+        *e = *event; //shallow copy
+        celix_arrayList_add(fw->dispatcher.dynamicEventQueue, e);
+        if (celix_arrayList_size(fw->dispatcher.dynamicEventQueue) % 100 == 0) {
+            fw_log(fw->logger, CELIX_LOG_LEVEL_WARNING, "dynamic event queue size is %i. Is there a bundle blocking on the event loop thread?", celix_arrayList_size(fw->dispatcher.dynamicEventQueue));
+        }
+    } else if (fw->dispatcher.eventQueueSize < CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE) {
+        size_t index = (fw->dispatcher.eventQueueFirstEntry + fw->dispatcher.eventQueueSize) %
+                       CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE;
+        fw->dispatcher.eventQueue[index] = *event; //shallow copy
+        fw->dispatcher.eventQueueSize += 1;
     } else {
-        request->eventType = eventType;
-        request->filter = NULL;
-        request->type = FRAMEWORK_EVENT_TYPE;
-        request->errorCode = errorCode;
-        request->error = "";
-
-        if (errorCode != CELIX_SUCCESS) {
-            request->error = celix_strerror(errorCode);
-        }
-
-        celixThreadMutex_lock(&framework->dispatcher.mutex);
-        if (framework->dispatcher.active) {
-            //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Adding dispatcher framework event request for event type %i", eventType);
-            celix_arrayList_add(framework->dispatcher.requests, request);
-            celixThreadCondition_broadcast(&framework->dispatcher.cond);
-        } else {
-            free(request);
-        }
-        celixThreadMutex_unlock(&framework->dispatcher.mutex);
-    }
-
-    framework_logIfError(framework->logger, status, NULL, "Failed to fire framework event");
-
-    return status;
+        //static queue is full, dynamics queue is empty. Add first entry to dynamic queue
+        fw_log(fw->logger, CELIX_LOG_LEVEL_WARNING,
+               "Static event queue for celix framework is full, falling back to dynamic allocated events. Increase static event queue size, current size is %i", CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE);
+        celix_framework_event_t *e = malloc(sizeof(*e));
+        *e = *event; //shallow copy
+        celix_arrayList_add(fw->dispatcher.dynamicEventQueue, e);
+    }
+    celixThreadCondition_broadcast(&fw->dispatcher.cond);
+    celixThreadMutex_unlock(&fw->dispatcher.mutex);
 }
 
-
-static void fw_handleEventRequest(celix_framework_t *framework, request_t* request) {
-    if (request->type == BUNDLE_EVENT_TYPE) {
+static void fw_handleEventRequest(celix_framework_t *framework, celix_framework_event_t* event) {
+    if (event->type == CELIX_BUNDLE_EVENT_TYPE) {

Review comment:
       This if is best done with a switch/case: https://gcc.godbolt.org/z/oKKnEY

##########
File path: libs/framework/src/framework.c
##########
@@ -2015,67 +1934,112 @@ static void fw_handleEventRequest(celix_framework_t *framework, request_t* reque
         for (int i = 0; i < celix_arrayList_size(localListeners); ++i) {
             fw_bundle_listener_pt listener = arrayList_get(localListeners, i);
 
-            bundle_event_t event;
-            memset(&event, 0, sizeof(event));
-            event.bnd = request->bndEntry->bnd;
-            event.type = request->eventType;
-            fw_invokeBundleListener(framework, listener->listener, &event, listener->bundle);
+            bundle_event_t bEvent;
+            memset(&bEvent, 0, sizeof(bEvent));
+            bEvent.bnd = event->bndEntry->bnd;
+            bEvent.type = event->bundleEvent;
+            fw_invokeBundleListener(framework, listener->listener, &bEvent, listener->bundle);
 
             fw_bundleListener_decreaseUseCount(listener);
         }
         celix_arrayList_destroy(localListeners);
-    } else  if (request->type == FRAMEWORK_EVENT_TYPE) {
+    } else if (event->type == CELIX_FRAMEWORK_EVENT_TYPE) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw event");
         celixThreadMutex_lock(&framework->frameworkListenersLock);
-        //TODO refactor use of framework listeners to use a useCount + conditition.
         for (int i = 0; i < celix_arrayList_size(framework->frameworkListeners); ++i) {
             fw_framework_listener_pt listener = celix_arrayList_get(framework->frameworkListeners, i);
-            framework_event_t event;
-            memset(&event, 0, sizeof(event));
-            event.type = request->eventType;
-            event.error = request->error;
-            event.errorCode = request->errorCode;
+            framework_event_t fEvent;
+            memset(&fEvent, 0, sizeof(fEvent));
+            fEvent.type = event->fwEvent;
+            fEvent.error = event->error;
+            fEvent.errorCode = event->errorCode;
 
-            fw_invokeFrameworkListener(framework, listener->listener, &event, listener->bundle);
+            fw_invokeFrameworkListener(framework, listener->listener, &fEvent, listener->bundle);
         }
         celixThreadMutex_unlock(&framework->frameworkListenersLock);
+    } else if (event->type == CELIX_REGISTER_SERVICE_EVENT) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw service registration event for service '%s' with svc id %li", event->serviceName, event->registerServiceId);
+        service_registration_t* reg = NULL;
+        celix_status_t status;
+        if (event->factory != NULL) {
+            status = celix_serviceRegistry_registerServiceFactory(framework->registry, event->bndEntry->bnd, event->serviceName, event->factory, event->properties, event->registerServiceId, &reg);
+        } else {
+            status = celix_serviceRegistry_registerService(framework->registry, event->bndEntry->bnd, event->serviceName, event->svc, event->properties, event->registerServiceId, &reg);
+        }
+        if (status != CELIX_SUCCESS) {
+            fw_log(framework->logger, CELIX_LOG_LEVEL_ERROR, "Could not register service async. svc name is %s, error is %s", event->serviceName, celix_strerror(status));
+        } else if (event->registerCallback != NULL) {
+            event->registerCallback(event->registerData, serviceRegistration_getServiceId(reg));
+        }
+    } else if (event->type == CELIX_UNREGISTER_SERVICE_EVENT) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw service unregister event for service id %li", event->unregisterServiceId);
+        celix_serviceRegistry_unregisterService(framework->registry, event->bndEntry->bnd, event->unregisterServiceId);
+    } else if (event->type == CELIX_GENERIC_EVENT) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling event %s", event->eventName);
+        if (event->genericProcess != NULL) {
+            event->genericProcess(event->genericProcessData);
+        }
+    }
+
+    if (event->doneCallback != NULL) {
+        event->doneCallback(event->doneData);
     }
 }
 
-static inline void fw_handleEvents(celix_framework_t* framework, celix_array_list_t* localRequests) {
+static inline celix_framework_event_t* fw_topEventFromQueue(celix_framework_t* fw) {
+    celix_framework_event_t* e = NULL;
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    if (fw->dispatcher.eventQueueSize > 0) {
+        e = &fw->dispatcher.eventQueue[fw->dispatcher.eventQueueFirstEntry];
+    } else if (celix_arrayList_size(fw->dispatcher.dynamicEventQueue) > 0) {
+        e = celix_arrayList_get(fw->dispatcher.dynamicEventQueue, 0);
+    }
+    celixThreadMutex_unlock(&fw->dispatcher.mutex);
+    return e;
+}
+
+static inline bool fw_removeTopEventFromQueue(celix_framework_t* fw) {
+    bool dynamicallyAllocated = false;
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    if (fw->dispatcher.eventQueueSize > 0) {
+        fw->dispatcher.eventQueueFirstEntry = (fw->dispatcher.eventQueueFirstEntry+1) % CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE;
+        fw->dispatcher.eventQueueSize -= 1;

Review comment:
       Am I correct in saying that if the ring buffer is overflowed and dynamic events have been allocated, that the FIFO principle gets violated? As in, if an event gets inserted into the ring buffer (once at least one event has been handled) before the ring buffer is empty, the dynamically allocated events have to wait?

##########
File path: libs/framework/src/framework.c
##########
@@ -1922,88 +1861,68 @@ bundle_context_t* framework_getContext(const_framework_pt framework) {
     return result;
 }
 
-celix_status_t fw_fireBundleEvent(framework_pt framework, bundle_event_type_e eventType, celix_framework_bundle_entry_t* entry) {
-    celix_status_t status = CELIX_SUCCESS;
-
+void fw_fireBundleEvent(framework_pt framework, bundle_event_type_e eventType, celix_framework_bundle_entry_t* entry) {
     if (eventType == OSGI_FRAMEWORK_BUNDLE_EVENT_STOPPING || eventType == OSGI_FRAMEWORK_BUNDLE_EVENT_UNINSTALLED || eventType == OSGI_FRAMEWORK_BUNDLE_EVENT_STOPPED) {
         if (entry->bndId == framework->bundleId) {
             //NOTE for framework bundle not triggering events while framework is stopped (and as result in use)
-            return CELIX_SUCCESS;
+            return;
         }
     }
 
-    request_t* request = (request_t*) calloc(1, sizeof(*request));
-    if (!request) {
-        status = CELIX_ENOMEM;
-    } else {
-        fw_bundleEntry_increaseUseCount(entry);
+    fw_bundleEntry_increaseUseCount(entry);
 
-        request->eventType = eventType;
-        request->filter = NULL;
-        request->type = BUNDLE_EVENT_TYPE;
-        request->error = NULL;
-        request->bndEntry = entry;
+    celix_framework_event_t event;
+    memset(&event, 0, sizeof(event));
+    event.type = CELIX_BUNDLE_EVENT_TYPE;
+    event.bndEntry = entry;
+    event.bundleEvent = eventType;
+    celix_framework_addToEventQueue(framework, &event);
+}
 
-        celixThreadMutex_lock(&framework->dispatcher.mutex);
-        if (framework->dispatcher.active) {
-            //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Adding dispatcher bundle event request for bnd id %li with event type %i", entry->bndId, eventType);
-            celix_arrayList_add(framework->dispatcher.requests, request);
-            celixThreadCondition_broadcast(&framework->dispatcher.cond);
-        } else {
-            /*
-             * NOTE because stopping the framework is done through stopping the framework bundle,
-             * most bundle stopping / stopped events cannot be fired.
-             *
-             * TBD if this needs to addressed.
-             */
-            fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Cannot fire event dispatcher not active. Event is %x for bundle %s", eventType, celix_bundle_getSymbolicName(entry->bnd));
-            fw_bundleEntry_decreaseUseCount(entry);
-            free(request);
-        }
-        celixThreadMutex_unlock(&framework->dispatcher.mutex);
+void fw_fireFrameworkEvent(framework_pt framework, framework_event_type_e eventType, celix_status_t errorCode) {
+    celix_framework_event_t event;
+    memset(&event, 0, sizeof(event));
+    event.type = CELIX_FRAMEWORK_EVENT_TYPE;
+    event.fwEvent = eventType;
+    event.errorCode = errorCode;
+    event.error = "";
+    if (errorCode != CELIX_SUCCESS) {
+        event.error = celix_strerror(errorCode);
     }
 
-    framework_logIfError(framework->logger, status, NULL, "Failed to fire bundle event");
-
-    return status;
+    celix_framework_addToEventQueue(framework, &event);
 }
 
-celix_status_t fw_fireFrameworkEvent(framework_pt framework, framework_event_type_e eventType, celix_status_t errorCode) {
-    celix_status_t status = CELIX_SUCCESS;
-
-    request_t* request = calloc(1, sizeof(*request));
-    if (!request) {
-        status = CELIX_ENOMEM;
+static void celix_framework_addToEventQueue(celix_framework_t *fw, const celix_framework_event_t* event) {
+    celixThreadMutex_lock(&fw->dispatcher.mutex);
+    //try to add to static queue
+    if (celix_arrayList_size(fw->dispatcher.dynamicEventQueue) > 0) { //always to dynamic queue if not empty (to ensure order)
+        celix_framework_event_t *e = malloc(sizeof(*e));
+        *e = *event; //shallow copy
+        celix_arrayList_add(fw->dispatcher.dynamicEventQueue, e);
+        if (celix_arrayList_size(fw->dispatcher.dynamicEventQueue) % 100 == 0) {
+            fw_log(fw->logger, CELIX_LOG_LEVEL_WARNING, "dynamic event queue size is %i. Is there a bundle blocking on the event loop thread?", celix_arrayList_size(fw->dispatcher.dynamicEventQueue));
+        }
+    } else if (fw->dispatcher.eventQueueSize < CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE) {
+        size_t index = (fw->dispatcher.eventQueueFirstEntry + fw->dispatcher.eventQueueSize) %
+                       CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE;
+        fw->dispatcher.eventQueue[index] = *event; //shallow copy
+        fw->dispatcher.eventQueueSize += 1;
     } else {
-        request->eventType = eventType;
-        request->filter = NULL;
-        request->type = FRAMEWORK_EVENT_TYPE;
-        request->errorCode = errorCode;
-        request->error = "";
-
-        if (errorCode != CELIX_SUCCESS) {
-            request->error = celix_strerror(errorCode);
-        }
-
-        celixThreadMutex_lock(&framework->dispatcher.mutex);
-        if (framework->dispatcher.active) {
-            //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Adding dispatcher framework event request for event type %i", eventType);
-            celix_arrayList_add(framework->dispatcher.requests, request);
-            celixThreadCondition_broadcast(&framework->dispatcher.cond);
-        } else {
-            free(request);
-        }
-        celixThreadMutex_unlock(&framework->dispatcher.mutex);
-    }
-
-    framework_logIfError(framework->logger, status, NULL, "Failed to fire framework event");
-
-    return status;
+        //static queue is full, dynamics queue is empty. Add first entry to dynamic queue
+        fw_log(fw->logger, CELIX_LOG_LEVEL_WARNING,
+               "Static event queue for celix framework is full, falling back to dynamic allocated events. Increase static event queue size, current size is %i", CELIX_FRAMEWORK_STATIC_EVENT_QUEUE_SIZE);
+        celix_framework_event_t *e = malloc(sizeof(*e));
+        *e = *event; //shallow copy
+        celix_arrayList_add(fw->dispatcher.dynamicEventQueue, e);
+    }
+    celixThreadCondition_broadcast(&fw->dispatcher.cond);
+    celixThreadMutex_unlock(&fw->dispatcher.mutex);
 }
 
-
-static void fw_handleEventRequest(celix_framework_t *framework, request_t* request) {
-    if (request->type == BUNDLE_EVENT_TYPE) {
+static void fw_handleEventRequest(celix_framework_t *framework, celix_framework_event_t* event) {
+    if (event->type == CELIX_BUNDLE_EVENT_TYPE) {
+        //fw_log(framework->logger, CELIX_LOG_LEVEL_TRACE, "Handling fw bundle event for bundle %s", event->bndEntry->bnd->symbolicName);

Review comment:
       Is this statement still needed?

##########
File path: libs/framework/src/service_tracker.c
##########
@@ -128,345 +111,285 @@ celix_status_t serviceTracker_create(bundle_context_pt context, const char * ser
 	return status;
 }
 
-celix_status_t serviceTracker_createWithFilter(bundle_context_pt context, const char * filter, service_tracker_customizer_pt customizer, service_tracker_pt *tracker) {
-	celix_status_t status = CELIX_SUCCESS;
-	*tracker = (service_tracker_pt) calloc(1, sizeof(**tracker));
-	if (!*tracker) {
-		status = CELIX_ENOMEM;
-	} else {
-		(*tracker)->context = context;
-		(*tracker)->filter = strdup(filter);
-        (*tracker)->customizer = customizer;
-	}
+celix_status_t serviceTracker_createWithFilter(bundle_context_pt context, const char * filter, service_tracker_customizer_pt customizer, service_tracker_pt *out) {
+	service_tracker_t* tracker = calloc(1, sizeof(*tracker));
+	*out = tracker;
+    tracker->context = context;
+    tracker->filter = celix_utils_strdup(filter);
+    tracker->customizer = *customizer;
+    free(customizer);
 
-	framework_logIfError(celix_frameworkLogger_globalLogger(), status, NULL, "Cannot create service tracker [filter=%s]", filter);
+    celixThreadMutex_create(&tracker->closeSync.mutex, NULL);
+    celixThreadCondition_init(&tracker->closeSync.cond, NULL);
 
-	return status;
+    celixThreadMutex_create(&tracker->mutex, NULL);
+    celixThreadCondition_init(&tracker->cond, NULL);
+    tracker->trackedServices = celix_arrayList_create();
+    tracker->untrackingServices = celix_arrayList_create();
+
+    celixThreadMutex_create(&tracker->mutex, NULL);
+    tracker->currentHighestServiceId = -1;
+
+    tracker->listener.handle = tracker;
+    tracker->listener.serviceChanged = (void *) serviceTracker_serviceChanged;
+
+    tracker->callbackHandle = tracker->callbackHandle;
+
+    tracker->add = tracker->add;
+    tracker->addWithProperties = tracker->addWithProperties;
+    tracker->addWithOwner = tracker->addWithOwner;
+    tracker->set = tracker->set;
+    tracker->setWithProperties = tracker->setWithProperties;
+    tracker->setWithOwner = tracker->setWithOwner;
+    tracker->remove = tracker->remove;
+    tracker->removeWithProperties = tracker->removeWithProperties;
+    tracker->removeWithOwner = tracker->removeWithOwner;
+
+	return CELIX_SUCCESS;
 }
 
 celix_status_t serviceTracker_destroy(service_tracker_pt tracker) {
-    if (tracker->customizer != NULL) {
-	    serviceTrackerCustomizer_destroy(tracker->customizer);
-	}
-
     free(tracker->serviceName);
 	free(tracker->filter);
-	free(tracker);
-
+    celixThreadMutex_destroy(&tracker->closeSync.mutex);
+    celixThreadCondition_destroy(&tracker->closeSync.cond);
+    celixThreadMutex_destroy(&tracker->mutex);
+    celixThreadCondition_destroy(&tracker->cond);
+    celix_arrayList_destroy(tracker->trackedServices);
+    celix_arrayList_destroy(tracker->untrackingServices);
+    free(tracker);
 	return CELIX_SUCCESS;
 }
 
 celix_status_t serviceTracker_open(service_tracker_pt tracker) {
-    celix_service_listener_t *listener = NULL;
-    celix_service_tracker_instance_t *instance = NULL;
-    celix_status_t status = CELIX_SUCCESS;
-
-    bool addListener = false;
-
-    celixThreadRwlock_writeLock(&tracker->instanceLock);
-    if (tracker->instance == NULL) {
-        instance = calloc(1, sizeof(*instance));
-        instance->context = tracker->context;
-
-        instance->closing = false;
-        instance->activeServiceChangeCalls = 0;
-        celixThreadMutex_create(&instance->closingLock, NULL);
-        celixThreadCondition_init(&instance->activeServiceChangeCallsCond, NULL);
-
-
-        celixThreadRwlock_create(&instance->lock, NULL);
-        instance->trackedServices = celix_arrayList_create();
-
-        celixThreadMutex_create(&instance->mutex, NULL);
-        instance->currentHighestServiceId = -1;
-
-        instance->listener.handle = instance;
-        instance->listener.serviceChanged = (void *) serviceTracker_serviceChanged;
-        listener = &instance->listener;
-
-        instance->callbackHandle = tracker->callbackHandle;
-        instance->filter = strdup(tracker->filter);
-        if (tracker->customizer != NULL) {
-            memcpy(&instance->customizer, tracker->customizer, sizeof(instance->customizer));
-        }
-        instance->add = tracker->add;
-        instance->addWithProperties = tracker->addWithProperties;
-        instance->addWithOwner = tracker->addWithOwner;
-        instance->set = tracker->set;
-        instance->setWithProperties = tracker->setWithProperties;
-        instance->setWithOwner = tracker->setWithOwner;
-        instance->remove = tracker->remove;
-        instance->removeWithProperties = tracker->removeWithProperties;
-        instance->removeWithOwner = tracker->removeWithOwner;
-
-        tracker->instance = instance;
-
-        addListener = true;
-    } else {
-        //already open
-        framework_logIfError(tracker->context->framework->logger, status, NULL, "Tracker already open");
+    celixThreadMutex_lock(&tracker->mutex);
+    bool alreadyOpen = tracker->open;
+    tracker->open = true;
+    celixThreadMutex_unlock(&tracker->mutex);
 
+    if (!alreadyOpen) {
+        bundleContext_addServiceListener(tracker->context, &tracker->listener, tracker->filter);
     }
-    celixThreadRwlock_unlock(&tracker->instanceLock);
-
-	if (addListener) {
-	    bundleContext_addServiceListener(tracker->context, listener, tracker->filter);
-	}
 	return CELIX_SUCCESS;
 }
 
-celix_status_t serviceTracker_close(service_tracker_pt tracker) {
+celix_status_t serviceTracker_close(service_tracker_t* tracker) {
 	//put all tracked entries in tmp array list, so that the untrack (etc) calls are not blocked.
     //set state to close to prevent service listener events
 
-    celixThreadRwlock_writeLock(&tracker->instanceLock);
-    celix_service_tracker_instance_t *instance = tracker->instance;
-    tracker->instance = NULL;
-    if (instance != NULL) {
-        celixThreadMutex_lock(&instance->closingLock);
-        //prevent service listener events
-        instance->closing = true;
-        celixThreadMutex_unlock(&instance->closingLock);
-    }
-    celixThreadRwlock_unlock(&tracker->instanceLock);
-
-    if (instance != NULL) {
-        celixThreadRwlock_writeLock(&instance->lock);
-        unsigned int size = celix_arrayList_size(instance->trackedServices);
-        if(size > 0) {
-            celix_tracked_entry_t *trackedEntries[size];
-            for (unsigned int i = 0u; i < size; i++) {
-                trackedEntries[i] = (celix_tracked_entry_t *) arrayList_get(instance->trackedServices, i);
-            }
-            arrayList_clear(instance->trackedServices);
-            celixThreadRwlock_unlock(&instance->lock);
+    celixThreadMutex_lock(&tracker->mutex);
+    bool open = tracker->open;
+    tracker->open = false;
+    celixThreadMutex_unlock(&tracker->mutex);
 
-            //loop trough tracked entries an untrack
-            for (unsigned int i = 0u; i < size; i++) {
-                serviceTracker_untrackTracked(instance, trackedEntries[i]);
+    if (!open) {
+        return CELIX_SUCCESS;
+    }
+
+
+    //indicate that the service tracking is closing and wait for the still pending service registration events.
+    celixThreadMutex_lock(&tracker->closeSync.mutex);
+    tracker->closeSync.closing = true;
+    while (tracker->closeSync.activeCalls > 0) {
+        celixThreadCondition_wait(&tracker->closeSync.cond, &tracker->closeSync.mutex);
+    }
+    celixThreadMutex_unlock(&tracker->closeSync.mutex);
+
+    int nrOfTrackedEntries;
+    do {
+        celixThreadMutex_lock(&tracker->mutex);
+        celix_tracked_entry_t* tracked = NULL;
+        nrOfTrackedEntries = celix_arrayList_size(tracker->trackedServices);
+        if (nrOfTrackedEntries > 0) {
+            tracked = celix_arrayList_get(tracker->trackedServices, 0);
+            celix_arrayList_removeAt(tracker->trackedServices, 0);
+            celix_arrayList_add(tracker->untrackingServices, tracked);
+        }
+        celixThreadMutex_unlock(&tracker->mutex);
+
+        if (tracked != NULL) {
+            int currentSize = nrOfTrackedEntries - 1;
+            if (currentSize == 0) {
+                serviceTracker_checkAndInvokeSetService(tracker, NULL, NULL, NULL);
+            } else {
+                celix_serviceTracker_useHighestRankingService(tracker, tracked->serviceName, tracker, NULL, NULL, serviceTracker_checkAndInvokeSetService);
             }
-        } else {
-            celixThreadRwlock_unlock(&instance->lock);
-        }
 
-        celixThreadMutex_lock(&instance->closingLock);
-        while (instance->activeServiceChangeCalls > 0) {
-            celixThreadCondition_wait(&instance->activeServiceChangeCallsCond, &instance->closingLock);
+            serviceTracker_untrackTracked(tracker, tracked);
+            celixThreadMutex_lock(&tracker->mutex);
+            celix_arrayList_remove(tracker->untrackingServices, tracked);
+            celixThreadCondition_broadcast(&tracker->cond);
+            celixThreadMutex_unlock(&tracker->mutex);
         }
-        celixThreadMutex_unlock(&instance->closingLock);
 
 
-#ifdef CELIX_SERVICE_TRACKER_USE_SHUTDOWN_THREAD
-        //NOTE Separate thread is needed to prevent deadlock where closing is triggered from a serviceChange event and the
-        // untrack -> removeServiceListener will try to remove a service listener which is being invoked and is the
-        // actual thread calling the removeServiceListener.
-        //
-        // This can be detached -> because service listener events are ignored (closing=true) and so no callbacks
-        // are made back to the celix framework / tracker owner.
-        serviceTracker_addInstanceFromShutdownList(instance);
-        celix_thread_t localThread;
-        celixThread_create(&localThread, NULL, shutdownServiceTrackerInstanceHandler, instance);
-        celixThread_detach(localThread);
-#else
-        fw_removeServiceListener(instance->context->framework, instance->context->bundle, &instance->listener);
-
-        celixThreadMutex_destroy(&instance->closingLock);
-        celixThreadCondition_destroy(&instance->activeServiceChangeCallsCond);
-        celixThreadMutex_destroy(&instance->mutex);
-        celixThreadRwlock_destroy(&instance->lock);
-        celix_arrayList_destroy(instance->trackedServices);
-        free(instance->filter);
-        free(instance);
-#endif
-    }
+        celixThreadMutex_lock(&tracker->mutex);
+        nrOfTrackedEntries = celix_arrayList_size(tracker->trackedServices);
+        celixThreadMutex_unlock(&tracker->mutex);
+    } while (nrOfTrackedEntries > 0);
+
+
+    fw_removeServiceListener(tracker->context->framework, tracker->context->bundle, &tracker->listener);
 
 	return CELIX_SUCCESS;
 }
 
-service_reference_pt serviceTracker_getServiceReference(service_tracker_pt tracker) {
+service_reference_pt serviceTracker_getServiceReference(service_tracker_t* tracker) {
     //TODO deprecated warning -> not locked
-	celix_tracked_entry_t *tracked;
+
     service_reference_pt result = NULL;
-	unsigned int i;
-
-	celixThreadRwlock_readLock(&tracker->instanceLock);
-	celix_service_tracker_instance_t *instance = tracker->instance;
-	if (instance != NULL) {
-        celixThreadRwlock_readLock(&instance->lock);
-        for (i = 0; i < arrayList_size(instance->trackedServices); ++i) {
-            tracked = (celix_tracked_entry_t *) arrayList_get(instance->trackedServices, i);
-            result = tracked->reference;
-            break;
-        }
-        celixThreadRwlock_unlock(&instance->lock);
+
+    celixThreadMutex_lock(&tracker->mutex);
+    for (int i = 0; i < celix_arrayList_size(tracker->trackedServices); ++i) {
+        celix_tracked_entry_t *tracked = celix_arrayList_get(tracker->trackedServices, i);
+        result = tracked->reference;
+        break;
     }
-    celixThreadRwlock_unlock(&tracker->instanceLock);
+    celixThreadMutex_unlock(&tracker->mutex);
 
 	return result;
 }
 
-array_list_pt serviceTracker_getServiceReferences(service_tracker_pt tracker) {
+array_list_pt serviceTracker_getServiceReferences(service_tracker_t* tracker) {
     //TODO deprecated warning -> not locked

Review comment:
       Is this comment still relevant?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org