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 2022/08/05 16:38:31 UTC

[GitHub] [celix] pnoltes commented on a diff in pull request #437: Add v2 of shared memory remote service admin

pnoltes commented on code in PR #437:
URL: https://github.com/apache/celix/pull/437#discussion_r938147358


##########
bundles/remote_services/rsa_spi/CMakeLists.txt:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+add_library(c_rsa_spi INTERFACE)
+target_include_directories(c_rsa_spi INTERFACE
+    $<BUILD_INTERFACE:${CMAKE_CURRENT_LIST_DIR}/include>
+    )
+target_link_libraries(c_rsa_spi INTERFACE Celix::framework Celix::deprecated_rsa_spi)

Review Comment:
   Note that I renamed the rsa_spi to deprecated_rsa_spi, because my tough was that the C++ rsa would replace the C rsa. 
   
   Seeing this revival of the C RSA, IMO the deprecated_rsa_spi can be moved into the c_rsa_spi INTERFACE target. 



##########
bundles/remote_services/rsa_spi/include/rsa_request_sender_service.h:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_REQUEST_SENDER_SERVICE_H_
+#define _RSA_REQUEST_SENDER_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <endpoint_description.h>
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_REQUEST_SENDER_SERVICE_NAME "rsa_request_sender_service"
+#define RSA_REQUEST_SENDER_SERVICE_VERSION "1.0.0"
+#define RSA_REQUEST_SENDER_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service send RPC request
+ * @note It can be implemented by RSA bundles, and called by RPC bundles.
+ *
+ */
+typedef struct rsa_request_sender_service {
+    void *handle;/// The Service handle
+    /**
+     * @brief Send the request that from remote service proxy.
+     *
+     * @param[in] handle Service handle
+     * @param[in] endpointDesciption The endpoint desciption of remote service
+     * @param[in,out] metadata The metadata, can be NULL.
+     * @param[in] request The request that from remote service proxy
+     * @param[out] response The response that from remote service. The caller should use free function to free response memory

Review Comment:
   nitpick: "The response that from remote service" -> "The response received from remote service endpoint"
   



##########
bundles/remote_services/rsa_spi/include/rsa_request_sender_service.h:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_REQUEST_SENDER_SERVICE_H_
+#define _RSA_REQUEST_SENDER_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <endpoint_description.h>
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_REQUEST_SENDER_SERVICE_NAME "rsa_request_sender_service"
+#define RSA_REQUEST_SENDER_SERVICE_VERSION "1.0.0"
+#define RSA_REQUEST_SENDER_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service send RPC request
+ * @note It can be implemented by RSA bundles, and called by RPC bundles.
+ *
+ */
+typedef struct rsa_request_sender_service {
+    void *handle;/// The Service handle
+    /**
+     * @brief Send the request that from remote service proxy.

Review Comment:
   Maybe good to mention that will be a remote, synchronized and blocking call.



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/CMakeLists.txt:
##########
@@ -0,0 +1,57 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+find_package(libuuid REQUIRED)
+
+add_celix_bundle(rsa_shm
+    VERSION 1.0.0
+    SYMBOLIC_NAME "apache_celix_remote_service_admin_shm_v2"
+    NAME "Apache Celix Remote Service Admin SHM V2"
+    GROUP "Celix/RSA"
+    SOURCES
+    src/rsa_shm_impl.c
+    src/rsa_shm_activator.c
+    src/rsa_shm_server.c
+    src/rsa_shm_client.c
+    src/rsa_shm_export_registration.c
+    src/rsa_shm_import_registration.c
+    src/thpool/thpool.c
+)
+
+target_include_directories(rsa_shm PRIVATE src src/thpool)
+
+if (ENABLE_TESTING)
+    celix_bundle_private_libs(rsa_shm Celix::shm_pool)

Review Comment:
   Why is this only done when testing is enabled? Now the bundle will change whether testing is enabled or not. 



##########
bundles/remote_services/remote_service_admin_shm_v2/shm_pool/src/tlsf/README.md:
##########
@@ -0,0 +1,92 @@
+# tlsf

Review Comment:
   Also update the LICENCE file so that tlsf is mentioned and also its licence (BSD)



##########
bundles/remote_services/remote_service_admin_shm_v2/shm_pool/src/shm_pool.c:
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <shm_pool.h>
+#include <shm_pool_private.h>
+#include <celix_threads.h>
+#include <tlsf.h>
+#include <stdlib.h>
+#include <stdbool.h>
+#include <stdio.h>
+#include <sys/ipc.h>
+#include <sys/shm.h>
+#include <errno.h>
+#include <assert.h>
+
+
+struct shm_pool{
+    celix_thread_mutex_t mutex;// projects below
+    int shmId;
+    void *shmStartAddr;
+    struct shm_pool_shared_info *sharedInfo;
+    celix_thread_t shmHeartbeatThread;
+    bool heartbeatThreadActive;
+    celix_thread_cond_t heartbeatThreadStoped;
+    tlsf_t allocator;
+};
+
+static void *shmPool_heartbeatThread(void *data);
+
+celix_status_t shmPool_create(size_t size, shm_pool_t **pool) {
+    celix_status_t status = CELIX_SUCCESS;
+    size_t normalizedSharedInfoSize = (sizeof(struct shm_pool_shared_info) % sizeof(void *) == 0) ?
+            sizeof(struct shm_pool_shared_info) : (sizeof(struct shm_pool_shared_info)+sizeof(void *))/sizeof(void *) * sizeof(void *);
+    if (size <= tlsf_size() + normalizedSharedInfoSize || pool == NULL) {
+        fprintf(stderr,"Shm pool: Shm size should be greater than %zu.\n", tlsf_size());
+        status = CELIX_ILLEGAL_ARGUMENT;
+        goto shm_size_invalid;
+    }
+
+    shm_pool_t *shmPool = (shm_pool_t *)malloc(sizeof(*shmPool));
+    assert(shmPool != NULL);
+
+    status = celixThreadMutex_create(&shmPool->mutex, NULL);
+    if(status != CELIX_SUCCESS) {
+        goto shm_pool_mutex_err;
+    }
+
+    shmPool->shmId = shmget(IPC_PRIVATE, size, SHM_R | SHM_W);

Review Comment:
   I expected something like `ftok` to be used here to create a shm entry which can be used for multiple processes. Am I missing something?



##########
bundles/remote_services/rsa_spi/include/rsa_request_handler_service.h:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_REQUEST_HANDLER_SERVICE_H_
+#define _RSA_REQUEST_HANDLER_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_REQUEST_HANDLER_SERVICE_NAME "rsa_request_handler_service"
+#define RSA_REQUEST_HANDLER_SERVICE_VERSION "1.0.0"
+#define RSA_REQUEST_HANDLER_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service handle RPC request
+ * @note It can be implemented by RPC bundles, and called by RSA bundles.
+ *
+ */
+typedef struct rsa_request_handler_service {
+    void *handle;/// The Service handle
+    /**
+     * @brief Handle the request that from remote service proxy.
+     *
+     * @param[in] handle Service handle
+     * @param[in, out] metadata The metadata, can be NULL.
+     * @param[in] request The request that from remote service proxy

Review Comment:
   nitpick:  " The request that from remote service proxy"  -> " The request from remote service proxy"



##########
bundles/remote_services/rsa_common/src/endpoint_description.c:
##########
@@ -88,3 +90,31 @@ static celix_status_t endpointDescription_verifyLongProperty(celix_properties_t
 
     return status;
 }
+
+bool endpointDescription_isInvalid(const endpoint_description_t *description) {
+    if (description == NULL || description->properties == NULL || description->serviceId < 0

Review Comment:
   the if is not needed, just return the expression of the if statement. 



##########
bundles/remote_services/rsa_spi/include/rsa_rpc_service.h:
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_RPC_SERVICE_H_
+#define _RSA_RPC_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <endpoint_description.h>
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_RPC_TYPE_KEY "remote_service.rpc_type"
+
+#define RSA_RPC_SERVICE_NAME "rsa_rpc_service"
+#define RSA_RPC_SERVICE_VERSION "1.0.0"
+#define RSA_RPC_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service use to install endpoint and endpoint proxy
+ * @note It can be implemented by RPC bundles.
+ */
+typedef struct rsa_rpc_service {

Review Comment:
   I like this approach, but I would prefer to align this more the naming used in C++ RSA.
   
   So IMO this should be a rsa_rpc_factory with the following "method" names:
    - `createProxy` 
    - `destroyProxy`
    - `createEndpoint`
    - `destroyEndpoint`
   
   
   I prefer create/destroy of install/uninstall, because for install/uninstall I am thinking about bundles. 



##########
bundles/remote_services/rsa_spi/include/rsa_request_handler_service.h:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_REQUEST_HANDLER_SERVICE_H_
+#define _RSA_REQUEST_HANDLER_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_REQUEST_HANDLER_SERVICE_NAME "rsa_request_handler_service"
+#define RSA_REQUEST_HANDLER_SERVICE_VERSION "1.0.0"
+#define RSA_REQUEST_HANDLER_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service handle RPC request
+ * @note It can be implemented by RPC bundles, and called by RSA bundles.
+ *
+ */
+typedef struct rsa_request_handler_service {
+    void *handle;/// The Service handle
+    /**
+     * @brief Handle the request that from remote service proxy.
+     *
+     * @param[in] handle Service handle
+     * @param[in, out] metadata The metadata, can be NULL.
+     * @param[in] request The request that from remote service proxy
+     * @param[out] response The response that from remote service. The caller should use free function to free response memory

Review Comment:
   nitpick: "The response that from remote service" -> "The response from remote service endpoint"



##########
bundles/remote_services/rsa_spi/include/rsa_request_handler_service.h:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_REQUEST_HANDLER_SERVICE_H_
+#define _RSA_REQUEST_HANDLER_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_REQUEST_HANDLER_SERVICE_NAME "rsa_request_handler_service"
+#define RSA_REQUEST_HANDLER_SERVICE_VERSION "1.0.0"
+#define RSA_REQUEST_HANDLER_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service handle RPC request
+ * @note It can be implemented by RPC bundles, and called by RSA bundles.
+ *

Review Comment:
   First: Nice addition to the SPI.
   
   I also think it good to mention that the rsa_request_handler_service handler raw (unserialized) data so that is does not need to focus on (de)serialization. 



##########
bundles/remote_services/rsa_spi/include/rsa_request_sender_service.h:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_REQUEST_SENDER_SERVICE_H_
+#define _RSA_REQUEST_SENDER_SERVICE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <endpoint_description.h>
+#include <celix_properties.h>
+#include <celix_errno.h>
+#include <sys/uio.h>
+
+#define RSA_REQUEST_SENDER_SERVICE_NAME "rsa_request_sender_service"
+#define RSA_REQUEST_SENDER_SERVICE_VERSION "1.0.0"
+#define RSA_REQUEST_SENDER_SERVICE_USE_RANGE "[1.0.0,2)"
+
+/**
+ * @brief The service send RPC request
+ * @note It can be implemented by RSA bundles, and called by RPC bundles.
+ *
+ */
+typedef struct rsa_request_sender_service {
+    void *handle;/// The Service handle
+    /**
+     * @brief Send the request that from remote service proxy.
+     *
+     * @param[in] handle Service handle
+     * @param[in] endpointDesciption The endpoint desciption of remote service
+     * @param[in,out] metadata The metadata, can be NULL.
+     * @param[in] request The request that from remote service proxy

Review Comment:
   nitpick: "the request that from remote service proxy" -> "The he request for the remote service endpoint"



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_export_registration.c:
##########
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <rsa_shm_export_registration.h>
+#include <rsa_rpc_service.h>
+#include <rsa_request_handler_service.h>
+#include <rsa_shm_constants.h>
+#include <endpoint_description.h>
+#include <remote_constants.h>
+#include <celix_log_helper.h>
+#include <celix_ref.h>
+#include <celix_api.h>
+#include <string.h>
+#include <assert.h>
+#include <stdbool.h>
+
+struct export_reference {
+    endpoint_description_t *endpoint;
+    service_reference_pt reference;
+};
+
+typedef struct export_request_handler_service_entry {
+    struct celix_ref ref;
+    celix_thread_rwlock_t lock; //projects below
+    rsa_request_handler_service_t *reqHandlerSvc;
+}export_request_handler_service_entry_t;
+
+struct export_registration {
+    struct celix_ref ref;
+    celix_bundle_context_t * context;
+    celix_log_helper_t* logHelper;
+    endpoint_description_t * endpointDesc;
+    service_reference_pt reference;
+    long rpcSvcTrkId;
+    rsa_rpc_service_t *rpcSvc;
+    long reqHandlerSvcTrkId;
+    long reqHandlerSvcId;
+    export_request_handler_service_entry_t *reqHandlerSvcEntry;
+};
+
+static void exportRegistration_addRpcSvc(void *handle, void *svc);
+static void exportRegistration_removeRpcSvc(void *handle, void *svc);
+static void exportRegistration_addRequestHandlerSvc(void *handle, void *svc);
+static void exportRegistration_removeRequestHandlerSvc(void *handle, void *svc);
+static void exportRegistration_destroy(export_registration_t *export);
+static export_request_handler_service_entry_t *exportRegistration_createReqHandlerSvcEntry(void);
+static void exportRegistration_retainReqHandlerSvcEntry(export_request_handler_service_entry_t *reqHandlerSvcEntry);
+static void exportRegistration_releaseReqHandlerSvcEntry(export_request_handler_service_entry_t *reqHandlerSvcEntry);
+
+celix_status_t exportRegistration_create(celix_bundle_context_t *context,
+        celix_log_helper_t *logHelper, service_reference_pt reference,
+        endpoint_description_t *endpointDesc, export_registration_t **exportOut) {
+    celix_status_t status = CELIX_SUCCESS;
+    if (context == NULL || logHelper == NULL || reference == NULL
+            || endpointDescription_isInvalid(endpointDesc) || exportOut == NULL) {
+        return CELIX_ILLEGAL_ARGUMENT;
+    }
+    export_registration_t *export = calloc(1, sizeof(*export));
+    assert(export != NULL);
+    celix_ref_init(&export->ref);
+    export->context = context;
+    export->logHelper = logHelper;
+
+    export->endpointDesc = endpointDescription_clone(endpointDesc);
+    assert(export->endpointDesc != NULL);
+
+    export->rpcSvc = NULL;
+    export->reqHandlerSvcTrkId = -1;
+    export->reqHandlerSvcId = -1;
+    export->reqHandlerSvcEntry = NULL;
+    export->reference = reference;
+    status = bundleContext_retainServiceReference(context, reference);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error(logHelper,"RSA export reg: Retain refrence for %s failed. %d.", endpointDesc->service,status);
+        goto err_retaining_service_ref;
+    }
+
+    export->reqHandlerSvcEntry = exportRegistration_createReqHandlerSvcEntry();
+    if (export->reqHandlerSvcEntry == NULL) {
+        celix_logHelper_error(export->logHelper,"RSA export reg: Error creating endpoint svc entry.");
+        status = CELIX_SERVICE_EXCEPTION;
+        goto ep_svc_entry_err;
+    }
+
+    /* If the properties of exported service include 'RSA_RPC_TYPE_KEY',

Review Comment:
   I think the `service.exported.configs` specified in the OSGi remote service spec should be used for this:
   http://docs.osgi.org/specification/osgi.cmpn/7.0.0/service.remoteservices.html
   
   Downside is that the configs is plural, so the string should be tokenized to check whether the RSA is specifically configured. 
   
   



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_msg.h:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_SHM_MSG_H_
+#define _RSA_SHM_MSG_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <pthread.h>
+#include <stddef.h>
+#include <sys/types.h>
+
+typedef enum {
+    REQUESTING = 0,
+    REPLYING = 1,
+    REPLIED = 2,
+    ABEND = 3,
+}MSG_STATE;

Review Comment:
   in celix C types are always lower case and snake case. 
   to msg_state instead of MSG_STATE.



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/thpool/thpool.h:
##########
@@ -0,0 +1,187 @@
+/**********************************

Review Comment:
   I tough that thpool.h was part of celix_utils, but I only see the thpool.h header in libs/utils/include.
   
   I think adding thpool.h where it is needed and maybe making a separate (static) lib is better, so could you remove the thpool.h header from celix_utils?



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_msg.h:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_SHM_MSG_H_
+#define _RSA_SHM_MSG_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <pthread.h>
+#include <stddef.h>
+#include <sys/types.h>
+
+typedef enum {
+    REQUESTING = 0,
+    REPLYING = 1,
+    REPLIED = 2,
+    ABEND = 3,
+}MSG_STATE;
+
+typedef struct rsa_shm_msg_control {
+    MSG_STATE msgState;
+    pthread_mutex_t lock;
+    pthread_cond_t signal;
+    size_t actualReplyedSize;
+}rsa_shm_msg_control_t;
+
+typedef struct rsa_shm_msg {
+    int shmId;
+    ssize_t ctrlDataOffset;
+    size_t ctrlDataSize;
+    ssize_t msgBufferOffset;//Message body includes metadata, request and reserve space

Review Comment:
   If I am reading this correct msgBufferOffset applies for maxBufferSize, metadataSize and requestSize. Maybe rename it so that it does not only align with maxBufferSize. So something like:
   
   ```
   typedef struct rsa_shm_msg {
       int shmId;
       ssize_t ctrlDataOffset;
       size_t ctrlDataSize;
       ssize_t msgBodyOffset;//Message body includes buffer, metadata, request and reserve space
       size_t maxBufferSize;
       size_t metadataSize;
       size_t requestSize;
   }rsa_shm_msg_t;
   ```



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_impl.c:
##########
@@ -0,0 +1,623 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include <rsa_shm_impl.h>
+#include <rsa_shm_server.h>
+#include <rsa_shm_client.h>
+#include <rsa_shm_constants.h>
+#include <rsa_shm_export_registration.h>
+#include <rsa_shm_import_registration.h>
+#include <endpoint_description.h>
+#include <remote_constants.h>
+#include <celix_api.h>
+#include <celix_long_hash_map.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <strings.h>
+#include <unistd.h>
+#include <uuid/uuid.h>
+#include <assert.h>
+
+struct rsa_shm {
+    celix_bundle_context_t *context;
+    celix_log_helper_t *logHelper;
+    celix_thread_mutex_t exportedServicesLock;//It protects exportedServices
+    celix_long_hash_map_t *exportedServices;// Key is service id, value is the list of exported registration.
+    celix_thread_mutex_t importedServicesLock;// It protects importedServices
+    celix_array_list_t *importedServices;
+    rsa_shm_client_manager_t *shmClientManager;
+    rsa_shm_server_t *shmServer;
+    char *shmServerName;
+    long reqSenderSvcId;
+};
+
+
+static celix_status_t rsaShm_receiveMsgCB(void *handle, rsa_shm_server_t *shmServer,
+        celix_properties_t *metadata, const struct iovec *request, struct iovec *response);
+
+static celix_status_t rsaShm_createEndpointDescription(rsa_shm_t *admin,
+        celix_properties_t *exportedProperties, char *interface, endpoint_description_t **description);
+
+celix_status_t rsaShm_create(celix_bundle_context_t *context, celix_log_helper_t *logHelper,
+        rsa_shm_t **admin) {
+    celix_status_t status = CELIX_SUCCESS;
+    if (context == NULL ||  admin == NULL || logHelper == NULL) {
+        return CELIX_ILLEGAL_ARGUMENT;
+    }
+
+    *admin = calloc(1, sizeof(**admin));
+    assert(*admin !=  NULL);
+
+    (*admin)->context = context;
+    (*admin)->logHelper = logHelper;
+    (*admin)->reqSenderSvcId = -1;
+    status = celixThreadMutex_create(&(*admin)->exportedServicesLock, NULL);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error((*admin)->logHelper, "Error creating mutex for exported service. %d", status);
+        goto exported_svc_lock_err;
+    }
+    (*admin)->exportedServices = celix_longHashMap_create();
+    assert((*admin)->exportedServices);
+
+    status = celixThreadMutex_create(&(*admin)->importedServicesLock, NULL);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error((*admin)->logHelper, "Error creating mutex for imported service. %d", status);
+        goto imported_svc_lock_err;
+    }
+    (*admin)->importedServices = celix_arrayList_create();
+    assert((*admin)->importedServices != NULL);
+
+    status = rsaShmClientManager_create(context, (*admin)->logHelper, &(*admin)->shmClientManager);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error((*admin)->logHelper,"Error creating shm client manager. %d", status);
+        goto shm_client_manager_err;
+    }
+
+    const char *fwUuid = NULL;
+    bundleContext_getProperty(context, OSGI_FRAMEWORK_FRAMEWORK_UUID, &fwUuid);
+    if (fwUuid == NULL) {
+        status = CELIX_BUNDLE_EXCEPTION;
+        celix_logHelper_error((*admin)->logHelper,"Error Getting fw uuid for shm rsa admin.");
+        goto fw_uuid_err;
+    }
+    long bundleId = celix_bundleContext_getBundleId(context);
+    if (bundleId < 0) {
+        status = CELIX_BUNDLE_EXCEPTION;
+        celix_logHelper_error((*admin)->logHelper,"Bundle id is invalid.");
+        goto bundle_id_err;
+    }
+    asprintf(&(*admin)->shmServerName, "ShmServ_%s_%ld", fwUuid, bundleId);
+    assert((*admin)->shmServerName != NULL);
+    status = rsaShmServer_create(context, (*admin)->shmServerName,(*admin)->logHelper,
+            rsaShm_receiveMsgCB, *admin, &(*admin)->shmServer);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error((*admin)->logHelper,"Error creating shm server. %d", status);
+        goto shm_server_err;
+    }
+
+    return CELIX_SUCCESS;
+shm_server_err:
+    free((*admin)->shmServerName);
+bundle_id_err:
+fw_uuid_err:
+    rsaShmClientManager_destory((*admin)->shmClientManager);
+shm_client_manager_err:
+    celix_arrayList_destroy((*admin)->importedServices);
+    celixThreadMutex_destroy(&(*admin)->importedServicesLock);
+imported_svc_lock_err:
+    celix_longHashMap_destroy((*admin)->exportedServices);
+    celixThreadMutex_destroy(&(*admin)->exportedServicesLock);
+exported_svc_lock_err:
+    free(*admin);
+    return status;
+}
+
+void rsaShm_setRequestSenderSvcId(rsa_shm_t *admin, long reqSenderSvcId) {
+    admin->reqSenderSvcId = reqSenderSvcId;
+    return;
+}
+
+void rsaShm_destroy(rsa_shm_t *admin) {
+    rsaShmServer_destroy(admin->shmServer);
+    free(admin->shmServerName);
+
+    rsaShmClientManager_destory(admin->shmClientManager);
+
+    // exported/imported services must be clear by topology manager before RSA destroy
+    assert(celix_arrayList_size(admin->importedServices) == 0);
+    celix_arrayList_destroy(admin->importedServices);
+    celixThreadMutex_destroy(&admin->importedServicesLock);
+    assert(celix_longHashMap_size(admin->exportedServices) == 0);
+    celix_longHashMap_destroy(admin->exportedServices);
+    celixThreadMutex_destroy(&admin->exportedServicesLock);
+    free(admin);
+
+    return;
+}
+
+static celix_status_t rsaShm_receiveMsgCB(void *handle, rsa_shm_server_t *shmServer,
+        celix_properties_t *metadata, const struct iovec *request, struct iovec *response) {
+    celix_status_t status = CELIX_SUCCESS;
+    if (handle == NULL || shmServer == NULL || request == NULL || response == NULL) {
+        return CELIX_ILLEGAL_ARGUMENT;
+    }
+    rsa_shm_t *admin = handle;
+
+    long serviceId = celix_properties_getAsLong(metadata, OSGI_RSA_ENDPOINT_SERVICE_ID, -1);
+    if (serviceId < 0) {
+        celix_logHelper_error(admin->logHelper, "Service id is invalid.");
+        status = CELIX_ILLEGAL_ARGUMENT;
+        goto err_getting_service_id;
+    }
+
+    celixThreadMutex_lock(&admin->exportedServicesLock);
+
+    //find exported registration
+    array_list_pt exports = celix_longHashMap_get(admin->exportedServices, serviceId);
+    if (exports == NULL || arrayList_size(exports) <= 0) {
+        status = CELIX_ILLEGAL_STATE;
+        celix_logHelper_error(admin->logHelper, "No export registration found for service id %ld", serviceId);
+        goto err_getting_exports;
+    }
+    export_registration_t *export = arrayList_get(exports, 0);

Review Comment:
   use `celix_arrayList_get` instead of `arrayList_get` and should this not loop over the exports list?



##########
bundles/remote_services/remote_service_admin_shm_v2/shm_pool/src/shm_cache.c:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <shm_cache.h>
+#include <shm_pool_private.h>
+#include <celix_errno.h>
+#include <celix_array_list.h>
+#include <celix_threads.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <assert.h>
+#include <sys/ipc.h>
+#include <sys/shm.h>
+
+typedef struct shm_cache_block {
+    int shmId;
+    void *shmStartAddr;
+    struct shm_pool_shared_info *sharedInfo;
+    uint64_t lastHeartbeatCnt;
+    unsigned int refCnt;
+    size_t maxOffset;
+}shm_cache_block_t;
+
+struct shm_cache{
+    bool shmRdOnly;
+    celix_thread_mutex_t mutex;// projects below
+    celix_array_list_t *shmCacheBlocks;

Review Comment:
   Should this not be hash map with key long (shmId)?



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_server.c:
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include <rsa_shm_server.h>
+#include <rsa_shm_msg.h>
+#include <rsa_shm_constants.h>
+#include <shm_cache.h>
+#include <celix_log_helper.h>
+#include <celix_api.h>
+#include <thpool.h>
+#include <sys/un.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <stdlib.h>
+#include <unistd.h>
+#include <sys/uio.h>
+#include <sys/param.h>
+#include <assert.h>
+#include <stdbool.h>
+#include <errno.h>
+
+#define MAX_RSA_SHM_SERVER_HANDLE_MSG_THREADS_NUM 5
+
+struct rsa_shm_server {
+    celix_bundle_context_t *ctx;
+    char *name;
+    celix_log_helper_t *loghelper;
+    int sfd;
+    shm_cache_t *shmCache;
+    threadpool threadPool;
+    celix_thread_t revMsgThread;
+    bool revMsgThreadActive;
+    rsaShmServer_receiveMsgCB revCB;
+    void *revCBHandle;
+    long msgTimeOutInSec;
+};
+
+struct rsa_shm_server_thpool_work_data {
+    rsa_shm_server_t *server;
+    rsa_shm_msg_control_t *msgCtrl;
+    void *msgBuffer;
+    size_t maxBufferSize;
+    size_t metadataSize;
+    size_t requestSize;
+};
+
+static void *rsaShmServer_receiveMsgThread(void *data);
+
+celix_status_t rsaShmServer_create(celix_bundle_context_t *ctx, const char *name, celix_log_helper_t *loghelper,
+        rsaShmServer_receiveMsgCB receiveCB, void *revHandle, rsa_shm_server_t **shmServerOut) {
+    int status = CELIX_SUCCESS;
+    if (name == NULL || ctx == NULL || strlen(name) >= MAX_RSA_SHM_SERVER_NAME_SIZE
+            || loghelper == NULL || receiveCB == NULL) {
+        return CELIX_ILLEGAL_ARGUMENT;
+    }
+
+    rsa_shm_server_t *server = (rsa_shm_server_t *)calloc(1, sizeof(rsa_shm_server_t));
+    assert(server != NULL);
+    server->ctx = ctx;
+    server->msgTimeOutInSec = celix_bundleContext_getPropertyAsLong(ctx,
+            RSA_SHM_MSG_TIMEOUT_KEY, RSA_SHM_MSG_TIMEOUT_DEFAULT_IN_S);
+    server->name = strdup(name);
+    assert(server->name != NULL);
+    server->loghelper = loghelper;
+    int sfd = socket(AF_UNIX, SOCK_DGRAM, 0);
+    if (sfd == -1) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create socket fd err, errno is %d.", errno);
+        status = CELIX_ERROR_MAKE(CELIX_FACILITY_CERRNO, errno);
+        goto sfd_err;
+    }
+    server->sfd = sfd;
+    struct sockaddr_un svaddr;
+    memset(&svaddr, 0, sizeof(svaddr));
+    svaddr.sun_family = AF_UNIX;
+    strncpy(&svaddr.sun_path[1], name, sizeof(svaddr.sun_path) - 1);
+    if (bind(sfd, (struct sockaddr *) &svaddr, sizeof(struct sockaddr_un)) == -1) {
+        celix_logHelper_error(loghelper, "RsaShmServer: bind socket fd err, errno is %d.", errno);
+        status = CELIX_ERROR_MAKE(CELIX_FACILITY_CERRNO, errno);
+        goto sfd_bind_err;
+    }
+
+    shm_cache_t *shmCache = NULL;
+    status = shmCache_create(false, &shmCache);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create shm cache err; error code is %d.", status);
+        goto create_shm_cache_err;
+    }
+    server->shmCache = shmCache;
+
+    server->threadPool = thpool_init(MAX_RSA_SHM_SERVER_HANDLE_MSG_THREADS_NUM);
+    if (server->threadPool == NULL) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create thread pool err.");
+        status = CELIX_ILLEGAL_STATE;
+        goto create_thpool_err;
+    }
+    server->revCB = receiveCB;
+    server->revCBHandle = revHandle;
+    server->revMsgThreadActive = true;
+    status = celixThread_create(&server->revMsgThread, NULL, rsaShmServer_receiveMsgThread, server);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create receive msg thread err.");
+        goto create_rev_msg_thread_err;
+    }
+    *shmServerOut = server;
+    return CELIX_SUCCESS;
+create_rev_msg_thread_err:
+    thpool_destroy(server->threadPool);
+create_thpool_err:
+    shmCache_destroy(shmCache);
+create_shm_cache_err:
+sfd_bind_err:
+    close(sfd);
+sfd_err:
+    free(server->name);
+    free(server);
+    return status;
+}
+
+void rsaShmServer_destroy(rsa_shm_server_t *server) {
+    if (server == NULL) {
+        return;
+    }
+    server->revMsgThreadActive = false;
+    shutdown(server->sfd,SHUT_RD);
+    celixThread_join(server->revMsgThread, NULL);
+    thpool_destroy(server->threadPool);
+    shmCache_destroy(server->shmCache);
+    close(server->sfd);
+    free(server->name);
+    free(server);
+    return;
+}
+
+
+static void rsaShmServer_terminateMsgHandling(rsa_shm_msg_control_t *ctrl) {
+    assert(ctrl != NULL);
+
+    // weakup client, terminate current interaction
+    pthread_mutex_lock(&ctrl->lock);
+    ctrl->msgState = ABEND;

Review Comment:
   What does ABEND stand for?



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_server.c:
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include <rsa_shm_server.h>
+#include <rsa_shm_msg.h>
+#include <rsa_shm_constants.h>
+#include <shm_cache.h>
+#include <celix_log_helper.h>
+#include <celix_api.h>
+#include <thpool.h>
+#include <sys/un.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <stdlib.h>
+#include <unistd.h>
+#include <sys/uio.h>
+#include <sys/param.h>
+#include <assert.h>
+#include <stdbool.h>
+#include <errno.h>
+
+#define MAX_RSA_SHM_SERVER_HANDLE_MSG_THREADS_NUM 5
+
+struct rsa_shm_server {
+    celix_bundle_context_t *ctx;
+    char *name;
+    celix_log_helper_t *loghelper;
+    int sfd;
+    shm_cache_t *shmCache;
+    threadpool threadPool;
+    celix_thread_t revMsgThread;
+    bool revMsgThreadActive;
+    rsaShmServer_receiveMsgCB revCB;
+    void *revCBHandle;
+    long msgTimeOutInSec;
+};
+
+struct rsa_shm_server_thpool_work_data {
+    rsa_shm_server_t *server;
+    rsa_shm_msg_control_t *msgCtrl;
+    void *msgBuffer;
+    size_t maxBufferSize;
+    size_t metadataSize;
+    size_t requestSize;
+};
+
+static void *rsaShmServer_receiveMsgThread(void *data);
+
+celix_status_t rsaShmServer_create(celix_bundle_context_t *ctx, const char *name, celix_log_helper_t *loghelper,
+        rsaShmServer_receiveMsgCB receiveCB, void *revHandle, rsa_shm_server_t **shmServerOut) {
+    int status = CELIX_SUCCESS;
+    if (name == NULL || ctx == NULL || strlen(name) >= MAX_RSA_SHM_SERVER_NAME_SIZE
+            || loghelper == NULL || receiveCB == NULL) {
+        return CELIX_ILLEGAL_ARGUMENT;
+    }
+
+    rsa_shm_server_t *server = (rsa_shm_server_t *)calloc(1, sizeof(rsa_shm_server_t));
+    assert(server != NULL);
+    server->ctx = ctx;
+    server->msgTimeOutInSec = celix_bundleContext_getPropertyAsLong(ctx,
+            RSA_SHM_MSG_TIMEOUT_KEY, RSA_SHM_MSG_TIMEOUT_DEFAULT_IN_S);
+    server->name = strdup(name);
+    assert(server->name != NULL);
+    server->loghelper = loghelper;
+    int sfd = socket(AF_UNIX, SOCK_DGRAM, 0);
+    if (sfd == -1) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create socket fd err, errno is %d.", errno);
+        status = CELIX_ERROR_MAKE(CELIX_FACILITY_CERRNO, errno);
+        goto sfd_err;
+    }
+    server->sfd = sfd;
+    struct sockaddr_un svaddr;
+    memset(&svaddr, 0, sizeof(svaddr));
+    svaddr.sun_family = AF_UNIX;
+    strncpy(&svaddr.sun_path[1], name, sizeof(svaddr.sun_path) - 1);
+    if (bind(sfd, (struct sockaddr *) &svaddr, sizeof(struct sockaddr_un)) == -1) {
+        celix_logHelper_error(loghelper, "RsaShmServer: bind socket fd err, errno is %d.", errno);
+        status = CELIX_ERROR_MAKE(CELIX_FACILITY_CERRNO, errno);
+        goto sfd_bind_err;
+    }
+
+    shm_cache_t *shmCache = NULL;
+    status = shmCache_create(false, &shmCache);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create shm cache err; error code is %d.", status);
+        goto create_shm_cache_err;
+    }
+    server->shmCache = shmCache;
+
+    server->threadPool = thpool_init(MAX_RSA_SHM_SERVER_HANDLE_MSG_THREADS_NUM);
+    if (server->threadPool == NULL) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create thread pool err.");
+        status = CELIX_ILLEGAL_STATE;
+        goto create_thpool_err;
+    }
+    server->revCB = receiveCB;
+    server->revCBHandle = revHandle;
+    server->revMsgThreadActive = true;
+    status = celixThread_create(&server->revMsgThread, NULL, rsaShmServer_receiveMsgThread, server);
+    if (status != CELIX_SUCCESS) {
+        celix_logHelper_error(loghelper, "RsaShmServer: create receive msg thread err.");
+        goto create_rev_msg_thread_err;
+    }
+    *shmServerOut = server;
+    return CELIX_SUCCESS;
+create_rev_msg_thread_err:
+    thpool_destroy(server->threadPool);
+create_thpool_err:
+    shmCache_destroy(shmCache);
+create_shm_cache_err:
+sfd_bind_err:
+    close(sfd);
+sfd_err:
+    free(server->name);
+    free(server);
+    return status;
+}
+
+void rsaShmServer_destroy(rsa_shm_server_t *server) {
+    if (server == NULL) {
+        return;
+    }
+    server->revMsgThreadActive = false;
+    shutdown(server->sfd,SHUT_RD);
+    celixThread_join(server->revMsgThread, NULL);
+    thpool_destroy(server->threadPool);
+    shmCache_destroy(server->shmCache);
+    close(server->sfd);
+    free(server->name);
+    free(server);
+    return;
+}
+
+
+static void rsaShmServer_terminateMsgHandling(rsa_shm_msg_control_t *ctrl) {
+    assert(ctrl != NULL);
+
+    // weakup client, terminate current interaction
+    pthread_mutex_lock(&ctrl->lock);
+    ctrl->msgState = ABEND;
+    //Signaling the condition variable first, and then unlocking the mutex, because client will free ctrl when msgState is ABEND.
+    pthread_cond_signal(&ctrl->signal);

Review Comment:
   Just to be sure: Are you sure that a signal will interrupt the bloking wait calls (e..g. recvfrom)?
   
   



##########
bundles/remote_services/remote_service_admin_shm_v2/shm_pool/include/shm_cache.h:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _SHM_CACHE_H_
+#define _SHM_CACHE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <celix_errno.h>
+#include <sys/types.h>
+#include <stddef.h>
+#include <stdbool.h>
+
+typedef struct shm_cache shm_cache_t;
+
+/**
+ * @brief Create shared memory cache
+ *
+ * @param[in] shmRdOnly Only read shared memory
+ * @param[out] shmCache The shared memory cache instance
+ * @return @see celix_errno.h
+ */
+celix_status_t shmCache_create(bool shmRdOnly, shm_cache_t **shmCache);
+
+/**
+ * @brief Destroy shared memory cache
+ *
+ * @param[in] shmCache The shared memory cache instance
+ */
+void shmCache_destroy(shm_cache_t *shmCache);
+
+/**
+ * @brief It will be called when shared memory is closed
+ *
+ * @param[in] handle Callback handle
+ * @param[in] shmCache The shared memory cache instance
+ * @param[in] shmId Closed shared memory id
+ */
+typedef void (*shmCache_shmPeerClosedCB)(void *handle, shm_cache_t *shmCache, int shmId);
+
+/**
+ * @brief Set the shared memory closed callback
+ *
+ * @param shmCache The shared memory cache instance
+ * @param shmPeerClosedCB The shared memory closed callback,it will be called when shared memory is closed
+ * @param closedCBHandle Callback handle
+ */
+void shmCache_setShmPeerClosedCB(shm_cache_t *shmCache, shmCache_shmPeerClosedCB shmPeerClosedCB, void *closedCBHandle);
+
+/**
+ * @brief Get shared memory address from shared memory cache.
+ *
+ * @param shmCache The shared memory cache instance
+ * @param shmId Shared memory id
+ * @param memoryOffset shared memory offset
+ * @return Shared memory address/NULL
+ */
+void * shmCache_getMemoryPtr(shm_cache_t *shmCache, int shmId, ssize_t memoryOffset);

Review Comment:
   I have some experience with using complex application based on shm communication. 
   
   In my experience it can be quite difficult to tackle issues when multiple applications are running which are using different middleware versions and result different structure in the shared memory, 
   
   I would advise to not only use a shmId (runtime id), but also add something like a procotol id.
   something like:
   `void * shmCache_getMemoryPtr(shm_cache_t *shmCache, int shmId, unsigned int protocolId, ssize_t memoryOffset);`
   
   The protocolId can be - by convention - an `celix_utils_stringHash` of the bundle symbolic name + "-" + bundle version. I think this can really help in the future if different versions try to communicate.
   
   
   



##########
bundles/remote_services/remote_service_admin_shm_v2/shm_pool/include/shm_cache.h:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _SHM_CACHE_H_
+#define _SHM_CACHE_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <celix_errno.h>
+#include <sys/types.h>
+#include <stddef.h>
+#include <stdbool.h>
+
+typedef struct shm_cache shm_cache_t;
+
+/**
+ * @brief Create shared memory cache
+ *
+ * @param[in] shmRdOnly Only read shared memory
+ * @param[out] shmCache The shared memory cache instance
+ * @return @see celix_errno.h
+ */
+celix_status_t shmCache_create(bool shmRdOnly, shm_cache_t **shmCache);
+
+/**
+ * @brief Destroy shared memory cache
+ *
+ * @param[in] shmCache The shared memory cache instance
+ */
+void shmCache_destroy(shm_cache_t *shmCache);
+
+/**
+ * @brief It will be called when shared memory is closed
+ *
+ * @param[in] handle Callback handle
+ * @param[in] shmCache The shared memory cache instance
+ * @param[in] shmId Closed shared memory id
+ */
+typedef void (*shmCache_shmPeerClosedCB)(void *handle, shm_cache_t *shmCache, int shmId);
+
+/**
+ * @brief Set the shared memory closed callback
+ *
+ * @param shmCache The shared memory cache instance
+ * @param shmPeerClosedCB The shared memory closed callback,it will be called when shared memory is closed
+ * @param closedCBHandle Callback handle
+ */
+void shmCache_setShmPeerClosedCB(shm_cache_t *shmCache, shmCache_shmPeerClosedCB shmPeerClosedCB, void *closedCBHandle);
+
+/**
+ * @brief Get shared memory address from shared memory cache.
+ *
+ * @param shmCache The shared memory cache instance
+ * @param shmId Shared memory id
+ * @param memoryOffset shared memory offset
+ * @return Shared memory address/NULL
+ */
+void * shmCache_getMemoryPtr(shm_cache_t *shmCache, int shmId, ssize_t memoryOffset);
+
+/**
+ * @brief Give back shared memory to shared memory cache
+ *
+ * @param shmCache The shared memory cache instance
+ * @param ptr Shared memory address
+ */
+void shmCache_putMemoryPtr(shm_cache_t *shmCache, void *ptr);

Review Comment:
   I would prefer an other name than shmCache_putMemoryPtr, because put implies you are putting something in the shm cache.
   
   Maybe `shmCache_releaseMemoryPtr`



##########
bundles/remote_services/remote_service_admin_shm_v2/rsa_shm/src/rsa_shm_msg.h:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RSA_SHM_MSG_H_
+#define _RSA_SHM_MSG_H_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+#include <pthread.h>
+#include <stddef.h>
+#include <sys/types.h>
+
+typedef enum {
+    REQUESTING = 0,
+    REPLYING = 1,
+    REPLIED = 2,
+    ABEND = 3,
+}MSG_STATE;

Review Comment:
   and I prefer a component/celix prefix. so rsa_shm_msg_state



-- 
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.

To unsubscribe, e-mail: dev-unsubscribe@celix.apache.org

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