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/11/11 20:43:53 UTC

[GitHub] [celix] pnoltes opened a new pull request #293: Feature/add build to svc dep creation

pnoltes opened a new pull request #293:
URL: https://github.com/apache/celix/pull/293


   This PR adds a build functionality to the C++ Dependency Manager, DM Component en Service Dependency.
   
   The build() functionality can be used to gain more control when adding service dependencies to an already created and enabled component. 
   Before this PR a newly created service dependency was activated before additional filters could be added. 
   
   Also removed a lock. The DM is not designed for MT usage and a single recursive mutex did give that impression. 
   
   Not sure if `build()` is the best naming. I tend to prefer `enable()`, but this implies a `disable()`. 
   


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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r525492223



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       function are default inline if you implement them in the class. 
   In my experience the inline is needed if you implement it in a "impl" header (at least for non template functions)




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r543713034



##########
File path: libs/framework/gtest/src/DependencyManagerTestSuite.cc
##########
@@ -0,0 +1,272 @@
+/*
+ * 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 <gtest/gtest.h>
+#include <atomic>
+
+#include "celix_api.h"
+
+class DependencyManagerTestSuite : public ::testing::Test {
+public:
+    celix_framework_t* fw = nullptr;
+    celix_bundle_context_t *ctx = nullptr;
+    celix_properties_t *properties = nullptr;
+
+    DependencyManagerTestSuite() {
+        properties = properties_create();
+        properties_set(properties, "LOGHELPER_ENABLE_STDOUT_FALLBACK", "true");
+        properties_set(properties, "org.osgi.framework.storage.clean", "onFirstInit");
+        properties_set(properties, "org.osgi.framework.storage", ".cacheBundleContextTestFramework");
+
+        fw = celix_frameworkFactory_createFramework(properties);
+        ctx = framework_getContext(fw);
+    }
+
+    ~DependencyManagerTestSuite() override {
+        celix_frameworkFactory_destroyFramework(fw);
+    }
+
+    DependencyManagerTestSuite(DependencyManagerTestSuite&&) = delete;
+    DependencyManagerTestSuite(const DependencyManagerTestSuite&) = delete;
+    DependencyManagerTestSuite& operator=(DependencyManagerTestSuite&&) = delete;
+    DependencyManagerTestSuite& operator=(const DependencyManagerTestSuite&) = delete;
+};
+
+TEST_F(DependencyManagerTestSuite, DmCreateComponent) {
+    auto *mng = celix_bundleContext_getDependencyManager(ctx);
+    auto *cmp = celix_dmComponent_create(ctx, "test1");
+    celix_dependencyManager_add(mng, cmp);
+
+    ASSERT_EQ(1, celix_dependencyManager_nrOfComponents(mng));
+    ASSERT_TRUE(celix_dependencyManager_allComponentsActive(mng));
+
+    cmp = celix_dmComponent_create(ctx, "test2");
+    celix_dependencyManager_add(mng, cmp);
+
+    ASSERT_EQ(2, celix_dependencyManager_nrOfComponents(mng));
+    ASSERT_TRUE(celix_dependencyManager_allComponentsActive(mng));
+}
+
+TEST_F(DependencyManagerTestSuite, TestCheckActive) {
+    auto *mng = celix_bundleContext_getDependencyManager(ctx);
+    auto *cmp = celix_dmComponent_create(ctx, "test1");
+
+    auto *dep = celix_dmServiceDependency_create();
+    celix_dmServiceDependency_setService(dep, "svcname", nullptr, nullptr);
+    celix_dmServiceDependency_setRequired(dep, true);
+    celix_dmComponent_addServiceDependency(cmp, dep); //required dep -> cmp not active
+
+
+    celix_dependencyManager_add(mng, cmp);
+    ASSERT_FALSE(celix_dependencyManager_areComponentsActive(mng));
+}
+
+class TestComponent {
+
+};
+
+TEST_F(DependencyManagerTestSuite, OnlyActiveAfterBuildCheck) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    dm.clear();
+    dm.clear(); //should be ok to call twice
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+}
+
+TEST_F(DependencyManagerTestSuite, StartDmWillBuildCmp) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    dm.start();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    dm.stop();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+}
+
+struct TestService {
+    void *handle;
+};
+
+class Cmp1 : public TestService {
+
+};
+
+class Cmp2 : public TestService {
+public:
+    explicit Cmp2(const std::string& name) {
+        std::cout << "usage arg: " << name;
+    }
+};
+
+TEST_F(DependencyManagerTestSuite, CreateComponentVariant) {
+    celix::dm::DependencyManager dm{ctx};
+
+    dm.createComponent<Cmp1>().addInterface<TestService>(); //lazy
+    dm.createComponent(std::unique_ptr<Cmp1>{new Cmp1}).addInterface<TestService>(); //with unique ptr
+    dm.createComponent(std::make_shared<Cmp1>()).addInterface<TestService>(); //with shared ptr
+    dm.createComponent(Cmp1{}).addInterface<TestService>(); //with value
+
+    //dm.createComponent<Cmp2>(); //Does not compile ->  no default ctor
+    dm.createComponent(std::unique_ptr<Cmp2>{new Cmp2{"a"}}).addInterface<TestService>(); //with unique ptr
+    dm.createComponent(std::make_shared<Cmp2>("b")).addInterface<TestService>();; //with shared ptr
+    dm.createComponent(Cmp2{"c"}).addInterface<TestService>();; //with value
+
+    dm.start();
+}
+
+TEST_F(DependencyManagerTestSuite, AddSvcProvideAfterBuild) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    TestService svc{nullptr};
+    cmp.addCInterface(&svc, "TestService");
+
+    long svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_EQ(-1, svcId); //not build -> not found
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_GT(svcId, -1); //(re)build -> found
+
+    dm.clear();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+    svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_EQ(svcId, -1); //cleared -> not found
+}
+
+TEST_F(DependencyManagerTestSuite, BuildSvcProvide) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<Cmp1>(std::make_shared<Cmp1>(), "test2");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    TestService svc{nullptr};
+    cmp.createProvidedCService(&svc, "CTestService").addProperty("key1", "val1").addProperty("key2", 3);
+
+    long svcId = celix_bundleContext_findService(ctx, "CTestService");
+    EXPECT_EQ(-1, svcId); //not build -> not found
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    svcId = celix_bundleContext_findService(ctx, "CTestService");
+    EXPECT_GT(svcId, -1); //(re)build -> found
+
+    celix_service_filter_options_t opts{};
+    opts.serviceName = "CTestService";
+    opts.filter = "(&(key1=val1)(key2=3))";
+    svcId = celix_bundleContext_findServiceWithOptions(ctx, &opts);
+    EXPECT_GT(svcId, -1); //found, so properties present
+
+    celix::dm::Properties props{};
+    props["key1"] = "value";
+    cmp.createProvidedService<TestService>().setProperties(props).setVersion("1.0.0").build();
+
+    opts.serviceName = "TestService";
+    opts.filter = "(key1=value)";
+    opts.ignoreServiceLanguage = true;
+    svcId = celix_bundleContext_findServiceWithOptions(ctx, &opts);
+    EXPECT_GT(svcId, -1); //found, so properties present
+
+    dm.clear();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+    svcId = celix_bundleContext_findService(ctx, "CTestService");
+    EXPECT_EQ(svcId, -1); //cleared -> not found
+}
+
+TEST_F(DependencyManagerTestSuite, AddSvcDepAfterBuild) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    std::atomic<int> count{0};
+    auto& dep = cmp.createCServiceDependency<TestService>("TestService")
+            .setCallbacks([&count](const TestService*, celix::dm::Properties&&) {
+                count++;
+            });
+
+    TestService svc{nullptr};
+    long svcId = celix_bundleContext_registerService(ctx, &svc, "TestService", nullptr);
+    long svcId2 = celix_bundleContext_registerService(ctx, &svc, "AnotherService", nullptr); //note should not be found.
+
+    ASSERT_EQ(0, count); //service dep not yet build -> so no set call
+
+    dep.build();
+    dep.build(); //should be ok to call twice
+    ASSERT_EQ(1, count); //service dep build -> so count is 1;
+
+    //create another service dep
+    cmp.createCServiceDependency<TestService>("TestService")
+            .setCallbacks([&count](const TestService*, celix::dm::Properties&&) {
+                count++;
+            });
+    ASSERT_EQ(1, count); //new service dep not yet build -> so count still 1
+
+    cmp.build(); //cmp build, which will build svc dep
+    ASSERT_EQ(2, count); //new service dep build -> so count is 2
+
+    celix_bundleContext_unregisterService(ctx, svcId);
+    celix_bundleContext_unregisterService(ctx, svcId2);
+}
+
+TEST_F(DependencyManagerTestSuite, InCompleteBuildShouldNotLeak) {
+    celix::dm::DependencyManager dm{ctx};
+    dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1"); //note not build
+
+    auto& cmp2 = dm.createComponent<Cmp1>(std::make_shared<Cmp1>(), "test2").build();
+    cmp2.createCServiceDependency<TestService>("TestService").setFilter("(key=value"); //note not build
+    cmp2.createServiceDependency<TestService>().setFilter("(key=value)"); //note not build
+
+    TestService svc{nullptr};
+    cmp2.createProvidedCService(&svc, "CTestService").addProperty("key1", "val1"); //note not build
+    cmp2.createProvidedService<TestService>().setVersion("1.0.0").build(); //note not build

Review comment:
       "note not build".
   
   ... but this lines builds :laughing: 

##########
File path: libs/framework/include/celix/dm/ServiceDependency.h
##########
@@ -59,20 +61,20 @@ namespace celix { namespace dm {
             }
         }
     public:
-        BaseServiceDependency(bool v)  : valid{v} {
+        BaseServiceDependency(celix_dm_component_t* c, bool v)  : cCmp{c}, valid{v} {
             if (this->valid) {
                 this->cServiceDep = celix_dmServiceDependency_create();
                 //NOTE using suspend as default strategy
                 celix_dmServiceDependency_setStrategy(this->cServiceDep,  DM_SERVICE_DEPENDENCY_STRATEGY_SUSPEND);
             }
         }
 
-        virtual ~BaseServiceDependency() = default;
+        virtual ~BaseServiceDependency() noexcept;

Review comment:
       Fun fact: implicit noexcept on destructors behave differently than on functions. For normal functions, not specifying noexcept results in the compiler adding `noexcept(false)`, which means that the compiler should assume exceptions can potentially be thrown and some optimizations should not be done (though I would not be surprised if compilers check if a function throws and optimizes anyway.)
   
   Noexcept on destructors are defined differently: if it is missing, the compiler should check if all its parent class and all member variables have noexcept destructors. If so, mark destructor implicitly as `noexcept(true)` otherwise `noexcept(false)`.
   
   Something similar happens for default defined move/copy/regular constructors/operators. If you happen to declare a copy constructor without explicitly telling the compiler if it is noexcept or not, it will try to figure it out for you.
   
   Given that `BaseServiceDependency` has no throwing parents or member variables, it was already marked noexcept. Though I like explicit noexcept. So just wanted to share this tidbit of knowledge.

##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -163,17 +181,33 @@ bool Component<T>::isValid() const {
     return this->bundleContext() != nullptr;
 }
 
+
+template<typename T>
+static
+typename std::enable_if<std::is_default_constructible<T>::value, T*>::type
+createInstance() {
+    return new T{};
+}
+
+template<typename T>
+static
+typename std::enable_if<!std::is_default_constructible<T>::value, T*>::type

Review comment:
       :+1: 

##########
File path: libs/framework/include/celix/dm/ServiceDependency_Impl.h
##########
@@ -22,11 +22,26 @@
 #include <cstring>
 #include "celix_constants.h"
 #include "celix_properties.h"
+#include "ServiceDependency.h"
+
 
 using namespace celix::dm;
 
+inline void BaseServiceDependency::runBuild() {
+    bool alreadyAdded = depAddedToCmp.exchange(true);
+    if (!alreadyAdded) {
+        celix_dmComponent_addServiceDependency(cCmp, cServiceDep);
+    }
+}
+
+inline BaseServiceDependency::~BaseServiceDependency() noexcept {
+    if (!depAddedToCmp) {

Review comment:
       Can you double check if the `!` is correct here?

##########
File path: libs/framework/include/celix/dm/ProvidedService.h
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <string>
+#include <memory>
+
+#include "celix_api.h"
+#include "celix/dm/Properties.h"
+
+namespace celix { namespace dm {
+
+    class BaseProvidedService {
+    public:
+        BaseProvidedService(celix_dm_component_t* _cmp, std::string svcName, void* svc, bool _cppService);
+
+        BaseProvidedService(BaseProvidedService&&) = delete;
+        BaseProvidedService& operator=(BaseProvidedService&&) = delete;
+        BaseProvidedService(const BaseProvidedService&) = delete;
+        BaseProvidedService& operator=(const BaseProvidedService&) = delete;
+
+        const std::string& getName() const;
+        const std::string& getVersion() const;
+        bool isCppService() const;
+        void* getService() const;
+        const celix::dm::Properties& getProperties() const;
+
+        void runBuild();
+    protected:
+        celix_dm_component_t* cCmp;
+        std::string svcName;
+        void* svcPointer;
+        bool cppService;
+        std::string svcVersion{};
+        celix::dm::Properties properties{};
+        bool provideAddedToCmp{false};
+    };
+
+    template<typename T, typename I>
+    class ProvidedService : public BaseProvidedService {
+    public:
+        ProvidedService(celix_dm_component_t* _cmp, std::string svcName, I* svc, bool _cppService);
+
+        /**
+         * Set the version of the interface
+         */
+        ProvidedService<T,I>& setVersion(std::string v);
+
+        /**
+         * Set the properties of the interface. Note this will reset the already set / added properties
+         */
+        ProvidedService<T,I>& setProperties(celix::dm::Properties);
+
+        /**
+         * Add a single property to the service properties.

Review comment:
       I think changing all the addProperty comments to `Add/override ...` makes it more clear.




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



[GitHub] [celix] pnoltes merged pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes merged pull request #293:
URL: https://github.com/apache/celix/pull/293


   


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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r525489750



##########
File path: libs/framework/include/celix/dm/Component.h
##########
@@ -52,6 +49,22 @@ namespace celix { namespace dm {
          * Returns the C bundle context
          */
         celix_bundle_context_t* bundleContext() const { return this->context; }
+
+        void runBuild();
+    protected:
+        std::vector<std::shared_ptr<BaseServiceDependency>> dependencies{};
+
+        // 0 = service name

Review comment:
       I created a ProvidedService class, now this is also more inline with ServiceDependency




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r521640934



##########
File path: libs/framework/gtest/src/dm_tests.cpp
##########
@@ -75,3 +75,113 @@ TEST_F(DepenencyManagerTests, TestCheckActive) {
     celix_dependencyManager_add(mng, cmp);
     ASSERT_FALSE(celix_dependencyManager_areComponentsActive(mng));
 }
+
+class TestComponent {
+
+};
+
+TEST_F(DepenencyManagerTests, OnlyActiveAfterBuildCheck) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice

Review comment:
       :+1: nice tests

##########
File path: libs/framework/include/celix/dm/Component.h
##########
@@ -52,6 +49,22 @@ namespace celix { namespace dm {
          * Returns the C bundle context
          */
         celix_bundle_context_t* bundleContext() const { return this->context; }
+
+        void runBuild();
+    protected:
+        std::vector<std::shared_ptr<BaseServiceDependency>> dependencies{};
+
+        // 0 = service name

Review comment:
       Perhaps worth considering just making a simple struct with names for the variables, rather than tuple. Then these comments are not needed anymore :)

##########
File path: libs/framework/gtest/src/dm_tests.cpp
##########
@@ -75,3 +75,113 @@ TEST_F(DepenencyManagerTests, TestCheckActive) {
     celix_dependencyManager_add(mng, cmp);
     ASSERT_FALSE(celix_dependencyManager_areComponentsActive(mng));
 }
+
+class TestComponent {
+
+};
+
+TEST_F(DepenencyManagerTests, OnlyActiveAfterBuildCheck) {

Review comment:
       Can you rename `DepenencyManagerTests` to `DependencyManagerTests`?

##########
File path: libs/framework/include/celix/dm/Component.h
##########
@@ -243,9 +255,19 @@ namespace celix { namespace dm {
          * @return the DM Component reference for chaining (fluent API)
          */
         Component<T>& removeCallbacks();
+
+        /**
+         * Build the component.
+         *
+         * When building the component all provided services and services dependencies are enabled.
+         * This is not done automatically so that user can firs construct component with their provided

Review comment:
       `firs` -> `first`

##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       I'm generally not in favour of using the inline keyword, the compiler is free to ignore it and usually knows better than programmers anyway. Moreover, templated functions in headers are already inline by default.

##########
File path: libs/framework/include/celix/dm/ServiceDependency.h
##########
@@ -219,13 +229,13 @@ namespace celix { namespace dm {
     class ServiceDependency : public TypedServiceDependency<T> {
         using type = I;
     public:
-        ServiceDependency(const std::string &name = std::string{}, bool valid = true);
+        ServiceDependency(celix_dm_component_t* cCmp, const std::string &name, bool valid = true);
         ~ServiceDependency() override = default;
 
         ServiceDependency(const ServiceDependency&) = delete;
         ServiceDependency& operator=(const ServiceDependency&) = delete;
-        ServiceDependency(ServiceDependency&&) noexcept = default;
-        ServiceDependency& operator=(ServiceDependency&&) noexcept = default;
+        ServiceDependency(ServiceDependency&&) noexcept = delete;
+        ServiceDependency& operator=(ServiceDependency&&) noexcept = delete;

Review comment:
       It's late so maybe I'm missing something, but semantically, I would expect moving of all these service dependencies to be OK. What's the reason you put it on delete?

##########
File path: libs/framework/gtest/src/dm_tests.cpp
##########
@@ -75,3 +75,113 @@ TEST_F(DepenencyManagerTests, TestCheckActive) {
     celix_dependencyManager_add(mng, cmp);
     ASSERT_FALSE(celix_dependencyManager_areComponentsActive(mng));
 }
+
+class TestComponent {
+
+};
+
+TEST_F(DepenencyManagerTests, OnlyActiveAfterBuildCheck) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    dm.clear();
+    dm.clear(); //should be ok to call twice
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+}
+
+TEST_F(DepenencyManagerTests, StartDmWillBuildCmp) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    dm.start();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    dm.stop();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+}
+
+struct TestService {
+    void *handle;
+};
+
+TEST_F(DepenencyManagerTests, AddSvcProvideAfterBuild) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    TestService svc{nullptr};
+    cmp.addCInterface(&svc, "TestService");
+
+    long svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_EQ(-1, svcId); //not build -> not found
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_GE(svcId, -1); //(re)build -> found

Review comment:
       `GE 0` or `GT -1`, I think you meant here.




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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r544357239



##########
File path: libs/framework/gtest/src/DependencyManagerTestSuite.cc
##########
@@ -0,0 +1,272 @@
+/*
+ * 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 <gtest/gtest.h>
+#include <atomic>
+
+#include "celix_api.h"
+
+class DependencyManagerTestSuite : public ::testing::Test {
+public:
+    celix_framework_t* fw = nullptr;
+    celix_bundle_context_t *ctx = nullptr;
+    celix_properties_t *properties = nullptr;
+
+    DependencyManagerTestSuite() {
+        properties = properties_create();
+        properties_set(properties, "LOGHELPER_ENABLE_STDOUT_FALLBACK", "true");
+        properties_set(properties, "org.osgi.framework.storage.clean", "onFirstInit");
+        properties_set(properties, "org.osgi.framework.storage", ".cacheBundleContextTestFramework");
+
+        fw = celix_frameworkFactory_createFramework(properties);
+        ctx = framework_getContext(fw);
+    }
+
+    ~DependencyManagerTestSuite() override {
+        celix_frameworkFactory_destroyFramework(fw);
+    }
+
+    DependencyManagerTestSuite(DependencyManagerTestSuite&&) = delete;
+    DependencyManagerTestSuite(const DependencyManagerTestSuite&) = delete;
+    DependencyManagerTestSuite& operator=(DependencyManagerTestSuite&&) = delete;
+    DependencyManagerTestSuite& operator=(const DependencyManagerTestSuite&) = delete;
+};
+
+TEST_F(DependencyManagerTestSuite, DmCreateComponent) {
+    auto *mng = celix_bundleContext_getDependencyManager(ctx);
+    auto *cmp = celix_dmComponent_create(ctx, "test1");
+    celix_dependencyManager_add(mng, cmp);
+
+    ASSERT_EQ(1, celix_dependencyManager_nrOfComponents(mng));
+    ASSERT_TRUE(celix_dependencyManager_allComponentsActive(mng));
+
+    cmp = celix_dmComponent_create(ctx, "test2");
+    celix_dependencyManager_add(mng, cmp);
+
+    ASSERT_EQ(2, celix_dependencyManager_nrOfComponents(mng));
+    ASSERT_TRUE(celix_dependencyManager_allComponentsActive(mng));
+}
+
+TEST_F(DependencyManagerTestSuite, TestCheckActive) {
+    auto *mng = celix_bundleContext_getDependencyManager(ctx);
+    auto *cmp = celix_dmComponent_create(ctx, "test1");
+
+    auto *dep = celix_dmServiceDependency_create();
+    celix_dmServiceDependency_setService(dep, "svcname", nullptr, nullptr);
+    celix_dmServiceDependency_setRequired(dep, true);
+    celix_dmComponent_addServiceDependency(cmp, dep); //required dep -> cmp not active
+
+
+    celix_dependencyManager_add(mng, cmp);
+    ASSERT_FALSE(celix_dependencyManager_areComponentsActive(mng));
+}
+
+class TestComponent {
+
+};
+
+TEST_F(DependencyManagerTestSuite, OnlyActiveAfterBuildCheck) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    dm.clear();
+    dm.clear(); //should be ok to call twice
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+}
+
+TEST_F(DependencyManagerTestSuite, StartDmWillBuildCmp) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    dm.start();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    dm.stop();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+}
+
+struct TestService {
+    void *handle;
+};
+
+class Cmp1 : public TestService {
+
+};
+
+class Cmp2 : public TestService {
+public:
+    explicit Cmp2(const std::string& name) {
+        std::cout << "usage arg: " << name;
+    }
+};
+
+TEST_F(DependencyManagerTestSuite, CreateComponentVariant) {
+    celix::dm::DependencyManager dm{ctx};
+
+    dm.createComponent<Cmp1>().addInterface<TestService>(); //lazy
+    dm.createComponent(std::unique_ptr<Cmp1>{new Cmp1}).addInterface<TestService>(); //with unique ptr
+    dm.createComponent(std::make_shared<Cmp1>()).addInterface<TestService>(); //with shared ptr
+    dm.createComponent(Cmp1{}).addInterface<TestService>(); //with value
+
+    //dm.createComponent<Cmp2>(); //Does not compile ->  no default ctor
+    dm.createComponent(std::unique_ptr<Cmp2>{new Cmp2{"a"}}).addInterface<TestService>(); //with unique ptr
+    dm.createComponent(std::make_shared<Cmp2>("b")).addInterface<TestService>();; //with shared ptr
+    dm.createComponent(Cmp2{"c"}).addInterface<TestService>();; //with value
+
+    dm.start();
+}
+
+TEST_F(DependencyManagerTestSuite, AddSvcProvideAfterBuild) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    TestService svc{nullptr};
+    cmp.addCInterface(&svc, "TestService");
+
+    long svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_EQ(-1, svcId); //not build -> not found
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_GT(svcId, -1); //(re)build -> found
+
+    dm.clear();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+    svcId = celix_bundleContext_findService(ctx, "TestService");
+    EXPECT_EQ(svcId, -1); //cleared -> not found
+}
+
+TEST_F(DependencyManagerTestSuite, BuildSvcProvide) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<Cmp1>(std::make_shared<Cmp1>(), "test2");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    TestService svc{nullptr};
+    cmp.createProvidedCService(&svc, "CTestService").addProperty("key1", "val1").addProperty("key2", 3);
+
+    long svcId = celix_bundleContext_findService(ctx, "CTestService");
+    EXPECT_EQ(-1, svcId); //not build -> not found
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    svcId = celix_bundleContext_findService(ctx, "CTestService");
+    EXPECT_GT(svcId, -1); //(re)build -> found
+
+    celix_service_filter_options_t opts{};
+    opts.serviceName = "CTestService";
+    opts.filter = "(&(key1=val1)(key2=3))";
+    svcId = celix_bundleContext_findServiceWithOptions(ctx, &opts);
+    EXPECT_GT(svcId, -1); //found, so properties present
+
+    celix::dm::Properties props{};
+    props["key1"] = "value";
+    cmp.createProvidedService<TestService>().setProperties(props).setVersion("1.0.0").build();
+
+    opts.serviceName = "TestService";
+    opts.filter = "(key1=value)";
+    opts.ignoreServiceLanguage = true;
+    svcId = celix_bundleContext_findServiceWithOptions(ctx, &opts);
+    EXPECT_GT(svcId, -1); //found, so properties present
+
+    dm.clear();
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm cleared so no components
+    svcId = celix_bundleContext_findService(ctx, "CTestService");
+    EXPECT_EQ(svcId, -1); //cleared -> not found
+}
+
+TEST_F(DependencyManagerTestSuite, AddSvcDepAfterBuild) {
+    celix::dm::DependencyManager dm{ctx};
+    EXPECT_EQ(0, dm.getNrOfComponents());
+
+    auto& cmp = dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1");
+    EXPECT_EQ(0, dm.getNrOfComponents()); //dm not started yet / comp not build yet
+    EXPECT_TRUE(cmp.isValid());
+
+    cmp.build();
+    cmp.build(); //should be ok to call twice
+    EXPECT_EQ(1, dm.getNrOfComponents()); //cmp "build", so active
+
+    std::atomic<int> count{0};
+    auto& dep = cmp.createCServiceDependency<TestService>("TestService")
+            .setCallbacks([&count](const TestService*, celix::dm::Properties&&) {
+                count++;
+            });
+
+    TestService svc{nullptr};
+    long svcId = celix_bundleContext_registerService(ctx, &svc, "TestService", nullptr);
+    long svcId2 = celix_bundleContext_registerService(ctx, &svc, "AnotherService", nullptr); //note should not be found.
+
+    ASSERT_EQ(0, count); //service dep not yet build -> so no set call
+
+    dep.build();
+    dep.build(); //should be ok to call twice
+    ASSERT_EQ(1, count); //service dep build -> so count is 1;
+
+    //create another service dep
+    cmp.createCServiceDependency<TestService>("TestService")
+            .setCallbacks([&count](const TestService*, celix::dm::Properties&&) {
+                count++;
+            });
+    ASSERT_EQ(1, count); //new service dep not yet build -> so count still 1
+
+    cmp.build(); //cmp build, which will build svc dep
+    ASSERT_EQ(2, count); //new service dep build -> so count is 2
+
+    celix_bundleContext_unregisterService(ctx, svcId);
+    celix_bundleContext_unregisterService(ctx, svcId2);
+}
+
+TEST_F(DependencyManagerTestSuite, InCompleteBuildShouldNotLeak) {
+    celix::dm::DependencyManager dm{ctx};
+    dm.createComponent<TestComponent>(std::make_shared<TestComponent>(), "test1"); //note not build
+
+    auto& cmp2 = dm.createComponent<Cmp1>(std::make_shared<Cmp1>(), "test2").build();
+    cmp2.createCServiceDependency<TestService>("TestService").setFilter("(key=value"); //note not build
+    cmp2.createServiceDependency<TestService>().setFilter("(key=value)"); //note not build
+
+    TestService svc{nullptr};
+    cmp2.createProvidedCService(&svc, "CTestService").addProperty("key1", "val1"); //note not build
+    cmp2.createProvidedService<TestService>().setVersion("1.0.0").build(); //note not build

Review comment:
       nice catch :+1: 




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r521651812



##########
File path: libs/framework/include/celix/dm/DependencyManager.h
##########
@@ -28,46 +30,25 @@
 #include <vector>
 #include <mutex>

Review comment:
       I think this header can be removed as well




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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (94dc9cc) into [master](https://codecov.io/gh/apache/celix/commit/7a26aea42fd33849e54a33d69969e42d096b9dcd?el=desc) (7a26aea) will **increase** coverage by `1.41%`.
   > The diff coverage is `96.72%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   66.69%   68.11%   +1.41%     
   ==========================================
     Files         147      148       +1     
     Lines       29947    30034      +87     
   ==========================================
   + Hits        19974    20457     +483     
   + Misses       9973     9577     -396     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `75.75% <92.85%> (+13.10%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/ProvidedService.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2UuaA==) | `100.00% <100.00%> (ø)` | |
   | [.../framework/include/celix/dm/ProvidedService\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2VfSW1wbC5o) | `100.00% <100.00%> (ø)` | |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `88.00% <100.00%> (+23.29%)` | :arrow_up: |
   | ... and [13 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [7a26aea...94dc9cc](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r525924963



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Rather than experience, let's look at the assembly generated: https://godbolt.org/z/cYasbb
   
   No difference. Inline does not do what you think, it is only useful when getting linkage issues, which is not the case here.
   
   Long explanation available on stackoverflow: https://stackoverflow.com/questions/1759300/when-should-i-write-the-keyword-inline-for-a-function-method




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r525924963



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Rather than experience, let's look at the assembly generated: https://godbolt.org/z/cYasbb
   
   No difference. Inline does not do what you think, it is only useful when getting linkage issues, which is not the case here.
   
   Long explanation available on stackoverflow: https://stackoverflow.com/a/1759575/1460998




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526360622



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Consider the following situation:
   
   example.h
   ```c++
   #pragma once
   void someFunction() {
      printf("funcy!");
   }
   ```
   
   some_file.cpp:
   ```c++
   #include <example.h>
   
   void stuff() {
     someFunction();
   }
   ```
   
   some_other_file.cpp:
   ```c++
   #include <example.h>
   
   void other_stuff() {
     someFunction();
   }
   ```
   
   
   What the compiler does when it tries to compile some_file.cpp is it looks for example.h, finds it, simply pastes the code into the translation unit (still some_file.cpp) and sees that a function `someFunction` is not only defined but also implemented. It generates the `someFunction` assembly and outputs some_file.o.
   
   Then the compiler see some_other_file.cpp and does the same thing. Looks for the header, sees a definition and an implementation, creates the assembly for `someFunction` and outputs it into some_other_file.o.
   
   Now the linker is told to create some_executable from the two object files some_file.o and some_other_file.o. But he sees `someFunction` twice and does not know how to deal with it. Which one is correct? It doesn't know how to determine that and instead of guessing what you meant and creating a random executable, it gives you an error like the following:
   
   ```
   /usr/bin/ld: CMakeFiles/dm_example_cxx_phase3.dir/src/Phase3BaseActivator.cc.o: in function `std::function<void (IPhase2*, std::map<std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >, std::less<std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > >, std::allocator<std::pair<std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > > > >&&)>::operator bool() const':
   /home/oipo-unencrypted/Programming/celix-apache/libs/framework/include/celix/dm/ServiceDependency_Impl.h:240: multiple definition of `celix::dm::DependencyManager::clear()'; CMakeFiles/dm_example_cxx_phase3.dir/src/Phase3Activator.cc.o:/home/oipo-unencrypted/Programming/celix-apache/libs/framework/include/celix/dm/DependencyManager_Impl.h:77: first defined here
   ``` 
   
   The key being "first defined here". It's telling you "which of these two functions do you want me to use? Remove the other one please".
   
   You can either change the header to only have a definition like so:
   example.h
   ```c++
   #pragma once
   void someFunction();
   ```
   example.cpp
   ```
   #include <example.h>
   void someFunction() {
      printf("funcy!");
   }
   ```
   
   Which would only generate the assembly for `someFunction` when the compiler is compiling example.cpp into example.o and never for another translation unit.
   
   Or you can add inline like so:
   example.h
   ```c++
   #pragma once
   inline void someFunction() {
      printf("funcy!");
   }
   ```
   
   This will cause some_file.o to contain the function `stuff()` which doesn't do a function call, but instead simply puts the printf call in the `stuff` function and does the same for `some_stuff` in some_other_file.o. The function `someFunction` simply "disappears" from the generated assembly.




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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (f433fcc) into [master](https://codecov.io/gh/apache/celix/commit/4ebde14e5cd089223339f5a21c07c0cea9ab34f8?el=desc) (4ebde14) will **increase** coverage by `1.50%`.
   > The diff coverage is `96.72%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   68.29%   69.80%   +1.50%     
   ==========================================
     Files         136      137       +1     
     Lines       27379    27466      +87     
   ==========================================
   + Hits        18698    19172     +474     
   + Misses       8681     8294     -387     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `75.75% <92.85%> (+13.10%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/ProvidedService.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2UuaA==) | `100.00% <100.00%> (ø)` | |
   | [.../framework/include/celix/dm/ProvidedService\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2VfSW1wbC5o) | `100.00% <100.00%> (ø)` | |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `88.00% <100.00%> (+23.29%)` | :arrow_up: |
   | [libs/utils/src/hash\_map.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy91dGlscy9zcmMvaGFzaF9tYXAuYw==) | `93.08% <0.00%> (-0.29%)` | :arrow_down: |
   | ... and [11 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [4ebde14...f433fcc](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526416460



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Thanks.
   
   I removed the inline only for the template functions and that indeed works. I was under the impressions that the inline was also explicitly needed for template functions. 
   
   For now, I want to keep the non template function inline. This will keep the celix/dm functionality header only, part of the celix_framework, but without linking the celix_framework to C++ (libstd++).
   
   




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526360622



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Consider the following situation:
   
   example.h
   ```c++
   #pragma once
   void someFunction() {
      printf("funcy!");
   }
   ```
   
   some_file.cpp:
   ```c++
   #include <example.h>
   
   void stuff() {
     someFunction();
   }
   ```
   
   some_other_file.cpp:
   ```c++
   #include <example.h>
   
   void other_stuff() {
     someFunction();
   }
   ```
   
   
   What the compiler does when it tries to compile some_file.cpp is it looks for example.h, finds it, simply pastes the code into the translation unit (still some_file.cpp) and sees that a function `someFunction` is not only defined but also implemented. It generates the `someFunction` assembly and outputs some_file.o.
   
   Then the compiler see some_other_file.cpp and does the same thing. Looks for the header, sees a definition and an implementation, creates the assembly for `someFunction` and outputs it into some_other_file.o.
   
   Now the linker is told to create some_executable from the two object files some_file.o and some_other_file.o. But he sees `someFunction` twice and does not know how to deal with it. Which one is correct? It doesn't know how to determine that and instead of guessing what you meant and creating a random executable, it gives you an error like the following:
   
   ```
   /usr/bin/ld: CMakeFiles/dm_example_cxx_phase3.dir/src/Phase3BaseActivator.cc.o: in function `std::function<void (IPhase2*, std::map<std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >, std::less<std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > >, std::allocator<std::pair<std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > > > >&&)>::operator bool() const':
   /home/oipo-unencrypted/Programming/celix-apache/libs/framework/include/celix/dm/ServiceDependency_Impl.h:240: multiple definition of `celix::dm::DependencyManager::clear()'; CMakeFiles/dm_example_cxx_phase3.dir/src/Phase3Activator.cc.o:/home/oipo-unencrypted/Programming/celix-apache/libs/framework/include/celix/dm/DependencyManager_Impl.h:77: first defined here
   ``` 
   
   The key being "first defined here". It's telling you "which of these two functions do you want me to use? Remove the other one please".
   
   You can either change the header to only have a definition like so:
   example.h
   ```c++
   #pragma once
   void someFunction();
   ```
   example.cpp
   ```c++
   #include <example.h>
   void someFunction() {
      printf("funcy!");
   }
   ```
   
   Which would only generate the assembly for `someFunction` when the compiler is compiling example.cpp into example.o and never for another translation unit.
   
   Or you can add inline like so:
   example.h
   ```c++
   #pragma once
   inline void someFunction() {
      printf("funcy!");
   }
   ```
   
   This will cause some_file.o to contain the function `stuff()` which doesn't do a function call, but instead simply puts the printf call in the `stuff` function and does the same for `some_stuff` in some_other_file.o. The function `someFunction` simply "disappears" from the generated assembly.




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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526272524



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Removing all inline statements from the C++ dep man implementation, leads a lot of errors like:
   ```
   /usr/bin/ld: CMakeFiles/test_framework.dir/src/bundle_context_services_test.cpp.o: in function `std::unique_lock<std::mutex>::~unique_lock()':
   /home/pep/workspace/celix/libs/framework/include/celix/dm/ServiceDependency_Impl.h:28: multiple definition of `celix::dm::BaseServiceDependency::runBuild()'; CMakeFiles/test_framework.dir/src/bundle_context_bundles_tests.cpp.o:/home/pep/workspace/celix/libs/framework/include/celix/dm/ServiceDependency_Impl.h:28: first defined here
   ```
   
   I read the stackoverflow link, it does not cover everything and I have not yet found out exactly why this is needed (I would like to understand it). 
   Closest I got is:
   https://isocpp.org/wiki/faq/templates#separate-template-class-defn-from-decl
   
   As far as I know defining function inside a class will make them implicitly inline:
   https://en.cppreference.com/w/cpp/language/inline
   ```
   A function defined entirely inside a class/struct/union definition, whether it's a member function or a non-member friend function, is implicitly an inline function. 
   ```
   So defining them outside a class, but still in a header - as is done with the _Impl.h headers - require a inline statement to make them effectually the same. 
   Btw I used the _Imp.h construction, because I really prefer to have clean declarations of functions, classes, etc without cluttering it with implementation details. 
   
   
   




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r521651647



##########
File path: libs/framework/include/celix/dm/Component.h
##########
@@ -17,24 +17,21 @@
  * under the License.
  */
 
-#ifndef CELIX_DM_COMPONENT_H
-#define CELIX_DM_COMPONENT_H
+#pragma once
 
 #include "celix/dm/types.h"
 #include "dm_component.h"
 
 #include <map>
 #include <string>
 #include <vector>
+#include <atomic>

Review comment:
       I don't see this header being used anywhere, can this be removed or am I missing something?




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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526272524



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Removing all inline statements from the C++ dep man implementation, leads to a lot of errors like:
   ```
   /usr/bin/ld: CMakeFiles/test_framework.dir/src/bundle_context_services_test.cpp.o: in function `std::unique_lock<std::mutex>::~unique_lock()':
   /home/pep/workspace/celix/libs/framework/include/celix/dm/ServiceDependency_Impl.h:28: multiple definition of `celix::dm::BaseServiceDependency::runBuild()'; CMakeFiles/test_framework.dir/src/bundle_context_bundles_tests.cpp.o:/home/pep/workspace/celix/libs/framework/include/celix/dm/ServiceDependency_Impl.h:28: first defined here
   ```
   
   I read the stackoverflow link, it does not cover everything and I have not yet found out exactly why this it is needed (I would like to understand it). 
   Closest I got is:
   https://isocpp.org/wiki/faq/templates#separate-template-class-defn-from-decl
   
   As far as I know defining function inside a class will make them implicitly inline:
   https://en.cppreference.com/w/cpp/language/inline
   ```
   A function defined entirely inside a class/struct/union definition, whether it's a member function or a non-member friend function, is implicitly an inline function. 
   ```
   So defining them outside a class, but still in a header - as is done with the _Impl.h headers - require a inline statement to make them effectually the same. 
   Btw I used the _Imp.h construction, because I really prefer to have clean declarations of functions, classes, etc without cluttering it with implementation details. 
   
   
   




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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (6d1b2dd) into [master](https://codecov.io/gh/apache/celix/commit/7a26aea42fd33849e54a33d69969e42d096b9dcd?el=desc) (7a26aea) will **increase** coverage by `1.48%`.
   > The diff coverage is `96.96%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   66.69%   68.18%   +1.48%     
   ==========================================
     Files         147      148       +1     
     Lines       29947    30088     +141     
   ==========================================
   + Hits        19974    20516     +542     
   + Misses       9973     9572     -401     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `76.92% <93.54%> (+14.27%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/ProvidedService.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2UuaA==) | `100.00% <100.00%> (ø)` | |
   | [.../framework/include/celix/dm/ProvidedService\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2VfSW1wbC5o) | `100.00% <100.00%> (ø)` | |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `82.35% <100.00%> (+17.64%)` | :arrow_up: |
   | ... and [13 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [7a26aea...6d1b2dd](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526362641



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       In short: 
   1) please remove inline from all templated functions.
   2) two options
     a) keep inline for the non-templated functions, solves the double implementation issue.
     b) create a DependencyManager_Impl.cc and move the non-templated function implementations there, have the impl header only contain templated functions. Faster compilation times and no confusing inline anywhere.




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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (4e70741) into [master](https://codecov.io/gh/apache/celix/commit/4ebde14e5cd089223339f5a21c07c0cea9ab34f8?el=desc) (4ebde14) will **increase** coverage by `1.44%`.
   > The diff coverage is `98.82%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   68.29%   69.74%   +1.44%     
   ==========================================
     Files         136      135       -1     
     Lines       27379    27408      +29     
   ==========================================
   + Hits        18698    19115     +417     
   + Misses       8681     8293     -388     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `71.81% <97.77%> (+9.16%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `88.00% <100.00%> (+23.29%)` | :arrow_up: |
   | [libs/utils/src/hash\_map.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy91dGlscy9zcmMvaGFzaF9tYXAuYw==) | `94.23% <0.00%> (+0.86%)` | :arrow_up: |
   | [...dmin\_websocket/src/pubsub\_websocket\_topic\_sender.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-YnVuZGxlcy9wdWJzdWIvcHVic3ViX2FkbWluX3dlYnNvY2tldC9zcmMvcHVic3ViX3dlYnNvY2tldF90b3BpY19zZW5kZXIuYw==) | `83.24% <0.00%> (+1.11%)` | :arrow_up: |
   | [...sub\_topology\_manager/src/pubsub\_topology\_manager.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-YnVuZGxlcy9wdWJzdWIvcHVic3ViX3RvcG9sb2d5X21hbmFnZXIvc3JjL3B1YnN1Yl90b3BvbG9neV9tYW5hZ2VyLmM=) | `52.80% <0.00%> (+1.14%)` | :arrow_up: |
   | ... and [7 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [4ebde14...4e70741](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (94dc9cc) into [master](https://codecov.io/gh/apache/celix/commit/7a26aea42fd33849e54a33d69969e42d096b9dcd?el=desc) (7a26aea) will **increase** coverage by `1.39%`.
   > The diff coverage is `96.72%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   66.69%   68.09%   +1.39%     
   ==========================================
     Files         147      148       +1     
     Lines       29947    30034      +87     
   ==========================================
   + Hits        19974    20452     +478     
   + Misses       9973     9582     -391     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `75.75% <92.85%> (+13.10%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/ProvidedService.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2UuaA==) | `100.00% <100.00%> (ø)` | |
   | [.../framework/include/celix/dm/ProvidedService\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2VfSW1wbC5o) | `100.00% <100.00%> (ø)` | |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `88.00% <100.00%> (+23.29%)` | :arrow_up: |
   | ... and [11 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [7a26aea...94dc9cc](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526435949



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       :+1: 




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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (6d1b2dd) into [master](https://codecov.io/gh/apache/celix/commit/7a26aea42fd33849e54a33d69969e42d096b9dcd?el=desc) (7a26aea) will **increase** coverage by `1.47%`.
   > The diff coverage is `96.96%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   66.69%   68.17%   +1.47%     
   ==========================================
     Files         147      148       +1     
     Lines       29947    30088     +141     
   ==========================================
   + Hits        19974    20512     +538     
   + Misses       9973     9576     -397     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `76.92% <93.54%> (+14.27%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/ProvidedService.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2UuaA==) | `100.00% <100.00%> (ø)` | |
   | [.../framework/include/celix/dm/ProvidedService\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2VfSW1wbC5o) | `100.00% <100.00%> (ø)` | |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `82.35% <100.00%> (+17.64%)` | :arrow_up: |
   | ... and [11 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [7a26aea...6d1b2dd](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r525924963



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Rather than experience, let's look at the assembly generated: https://godbolt.org/z/cYasbb
   
   No difference. Inline does not do what you think, it is only useful when getting linkage issues, which is not the case here.




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526436603



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       Thanks for bearing with my rants.




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



[GitHub] [celix] codecov-io edited a comment on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (f433fcc) into [master](https://codecov.io/gh/apache/celix/commit/4ebde14e5cd089223339f5a21c07c0cea9ab34f8?el=desc) (4ebde14) will **decrease** coverage by `0.29%`.
   > The diff coverage is `96.72%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   - Coverage   68.29%   67.99%   -0.30%     
   ==========================================
     Files         136      152      +16     
     Lines       27379    31504    +4125     
   ==========================================
   + Hits        18698    21422    +2724     
   - Misses       8681    10082    +1401     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `75.75% <92.85%> (+13.10%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/ProvidedService.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2UuaA==) | `100.00% <100.00%> (ø)` | |
   | [.../framework/include/celix/dm/ProvidedService\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Qcm92aWRlZFNlcnZpY2VfSW1wbC5o) | `100.00% <100.00%> (ø)` | |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `88.00% <100.00%> (+23.29%)` | :arrow_up: |
   | [libs/utils/src/hash\_map.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy91dGlscy9zcmMvaGFzaF9tYXAuYw==) | `93.08% <0.00%> (-0.29%)` | :arrow_down: |
   | ... and [32 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [4ebde14...f433fcc](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r544358850



##########
File path: libs/framework/include/celix/dm/ServiceDependency_Impl.h
##########
@@ -22,11 +22,26 @@
 #include <cstring>
 #include "celix_constants.h"
 #include "celix_properties.h"
+#include "ServiceDependency.h"
+
 
 using namespace celix::dm;
 
+inline void BaseServiceDependency::runBuild() {
+    bool alreadyAdded = depAddedToCmp.exchange(true);
+    if (!alreadyAdded) {
+        celix_dmComponent_addServiceDependency(cCmp, cServiceDep);
+    }
+}
+
+inline BaseServiceDependency::~BaseServiceDependency() noexcept {
+    if (!depAddedToCmp) {

Review comment:
       This is correct. If a dependency is added to a dm component it will be destroyed when the component is destroyed. 
   If a dependency is not added to a dm component (i.e. not build) it needs to be destroyed explicitly. 




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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r544413458



##########
File path: libs/framework/include/celix/dm/ProvidedService.h
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <string>
+#include <memory>
+
+#include "celix_api.h"
+#include "celix/dm/Properties.h"
+
+namespace celix { namespace dm {
+
+    class BaseProvidedService {
+    public:
+        BaseProvidedService(celix_dm_component_t* _cmp, std::string svcName, void* svc, bool _cppService);
+
+        BaseProvidedService(BaseProvidedService&&) = delete;
+        BaseProvidedService& operator=(BaseProvidedService&&) = delete;
+        BaseProvidedService(const BaseProvidedService&) = delete;
+        BaseProvidedService& operator=(const BaseProvidedService&) = delete;
+
+        const std::string& getName() const;
+        const std::string& getVersion() const;
+        bool isCppService() const;
+        void* getService() const;
+        const celix::dm::Properties& getProperties() const;
+
+        void runBuild();
+    protected:
+        celix_dm_component_t* cCmp;
+        std::string svcName;
+        void* svcPointer;
+        bool cppService;
+        std::string svcVersion{};
+        celix::dm::Properties properties{};
+        bool provideAddedToCmp{false};
+    };
+
+    template<typename T, typename I>
+    class ProvidedService : public BaseProvidedService {
+    public:
+        ProvidedService(celix_dm_component_t* _cmp, std::string svcName, I* svc, bool _cppService);
+
+        /**
+         * Set the version of the interface
+         */
+        ProvidedService<T,I>& setVersion(std::string v);
+
+        /**
+         * Set the properties of the interface. Note this will reset the already set / added properties
+         */
+        ProvidedService<T,I>& setProperties(celix::dm::Properties);
+
+        /**
+         * Add a single property to the service properties.

Review comment:
       done




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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r525413649



##########
File path: libs/framework/include/celix/dm/ServiceDependency.h
##########
@@ -219,13 +229,13 @@ namespace celix { namespace dm {
     class ServiceDependency : public TypedServiceDependency<T> {
         using type = I;
     public:
-        ServiceDependency(const std::string &name = std::string{}, bool valid = true);
+        ServiceDependency(celix_dm_component_t* cCmp, const std::string &name, bool valid = true);
         ~ServiceDependency() override = default;
 
         ServiceDependency(const ServiceDependency&) = delete;
         ServiceDependency& operator=(const ServiceDependency&) = delete;
-        ServiceDependency(ServiceDependency&&) noexcept = default;
-        ServiceDependency& operator=(ServiceDependency&&) noexcept = default;
+        ServiceDependency(ServiceDependency&&) noexcept = delete;
+        ServiceDependency& operator=(ServiceDependency&&) noexcept = delete;

Review comment:
       the base of service dependency has raw pointer, to prevent compiler issues the move and copy constructions are deleted




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526353213



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       :laughing: 
   
   You've managed to use inline in both the situation where it is useless and exactly the one type of situation where it is potentially necessary.
   
   Useless because templated functions are inline by default:
   ```c++
   template<typename T>
   inline void DependencyManager::destroyComponent(Component<T> &component) {
       celix_dependencyManager_remove(cDepMan, component.cComponent());
   }
   ```
   
   Totally necessary (if you want to keep it in a header file) because otherwise the function gets defined multiple times:
   ```c++
   inline void DependencyManager::clear() {
       celix_dependencyManager_removeAllComponents(cDepMan.get());
       components.clear();
   }
   ```
   
   So one could make an argument that the `start()`, `build()`, `clear()`, `stop()`, `getNrOfComponents()` etc functions should be put in a .cpp file with only their definitions in the header file, since that would reduce compilation times and also solve the linking problem. This would create a 'function' in assembly that would be called as you would expect.
   
   But inline there causes the function to not exist as a function but rather the assembly of a function to get duplicated everywhere it would normally be called. Whether that actually speeds things up is debatable, but it also means that there are no duplicate functions when linking.




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



[GitHub] [celix] pnoltes commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
pnoltes commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r544413751



##########
File path: libs/framework/include/celix/dm/ServiceDependency.h
##########
@@ -59,20 +61,20 @@ namespace celix { namespace dm {
             }
         }
     public:
-        BaseServiceDependency(bool v)  : valid{v} {
+        BaseServiceDependency(celix_dm_component_t* c, bool v)  : cCmp{c}, valid{v} {
             if (this->valid) {
                 this->cServiceDep = celix_dmServiceDependency_create();
                 //NOTE using suspend as default strategy
                 celix_dmServiceDependency_setStrategy(this->cServiceDep,  DM_SERVICE_DEPENDENCY_STRATEGY_SUSPEND);
             }
         }
 
-        virtual ~BaseServiceDependency() = default;
+        virtual ~BaseServiceDependency() noexcept;

Review comment:
       :+1:




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526353213



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       :laughing: 
   
   You've managed to use inline in both the situation where it is useless and exactly the one type of situation where it is potentially necessary.
   
   Useless because templated functions are inline by default:
   ```c++
   template<typename T>
   inline void DependencyManager::destroyComponent(Component<T> &component) {
       celix_dependencyManager_remove(cDepMan, component.cComponent());
   }
   ```
   
   Totally necessary (if you want to keep it in a header file) because otherwise the function gets defined multiple times:
   ```
   inline void DependencyManager::clear() {
       celix_dependencyManager_removeAllComponents(cDepMan.get());
       components.clear();
   }
   ```
   
   So one could make an argument that the `start()`, `build()`, `clear()`, `stop()`, `getNrOfComponents()` etc functions should be put in a .cpp file with only their definitions in the header file, since that would reduce compilation times and also solve the linking problem. This would create a 'function' in assembly that would be called as you would expect.
   
   But inline there causes the function to not exist as a function but rather the assembly of a function to get duplicated everywhere it would normally be called. Whether that actually speeds things up is debatable, but it also means that there are no duplicate functions when linking.




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



[GitHub] [celix] Oipo commented on a change in pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
Oipo commented on a change in pull request #293:
URL: https://github.com/apache/celix/pull/293#discussion_r526353213



##########
File path: libs/framework/include/celix/dm/Component_Impl.h
##########
@@ -63,7 +88,7 @@ Component<T>& Component<T>::addInterfaceWithName(const std::string &serviceName,
 
 template<class T>
 template<class I>
-Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {
+inline Component<T>& Component<T>::addInterface(const std::string &version, const Properties &properties) {

Review comment:
       :laughing: 
   
   You've managed to use inline in both the situation where it is useless and exactly the one type of situation where it is totally necessary.
   
   Useless because templated functions are inline by default:
   ```c++
   template<typename T>
   inline void DependencyManager::destroyComponent(Component<T> &component) {
       celix_dependencyManager_remove(cDepMan, component.cComponent());
   }
   ```
   
   Totally necessary (if you want to keep it in a header file) because otherwise the function gets defined multiple times:
   ```
   inline void DependencyManager::clear() {
       celix_dependencyManager_removeAllComponents(cDepMan.get());
       components.clear();
   }
   ```
   
   So one could make an argument that the `start()`, `build()`, `clear()`, `stop()`, `getNrOfComponents()` etc functions should be put in a .cpp file with only their definitions in the header file, since that would reduce compilation times and also solve the linking problem. This would create a 'function' in assembly that would be called as you would expect.
   
   But inline there causes the function to not exist as a function but rather the assembly of a function to get duplicated everywhere it would normally be called. Whether that actually speeds things up is debatable, but it also means that there are no duplicate functions when linking.




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



[GitHub] [celix] codecov-io commented on pull request #293: Feature/add build to svc dep creation

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #293:
URL: https://github.com/apache/celix/pull/293#issuecomment-725653571


   # [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=h1) Report
   > Merging [#293](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=desc) (4e70741) into [master](https://codecov.io/gh/apache/celix/commit/4ebde14e5cd089223339f5a21c07c0cea9ab34f8?el=desc) (4ebde14) will **increase** coverage by `1.43%`.
   > The diff coverage is `98.82%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/celix/pull/293/graphs/tree.svg?width=650&height=150&src=pr&token=JdsiThga8P)](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #293      +/-   ##
   ==========================================
   + Coverage   68.29%   69.72%   +1.43%     
   ==========================================
     Files         136      135       -1     
     Lines       27379    27408      +29     
   ==========================================
   + Hits        18698    19111     +413     
   + Misses       8681     8297     -384     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [libs/framework/include/celix/dm/types.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS90eXBlcy5o) | `92.85% <ø> (ø)` | |
   | [libs/framework/include/celix/dm/Component\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnRfSW1wbC5o) | `71.81% <97.77%> (+9.16%)` | :arrow_up: |
   | [libs/framework/include/celix/dm/Component.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9Db21wb25lbnQuaA==) | `100.00% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/DependencyManager\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9EZXBlbmRlbmN5TWFuYWdlcl9JbXBsLmg=) | `100.00% <100.00%> (+12.50%)` | :arrow_up: |
   | [...ibs/framework/include/celix/dm/ServiceDependency.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeS5o) | `84.21% <100.00%> (ø)` | |
   | [...ramework/include/celix/dm/ServiceDependency\_Impl.h](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy9mcmFtZXdvcmsvaW5jbHVkZS9jZWxpeC9kbS9TZXJ2aWNlRGVwZW5kZW5jeV9JbXBsLmg=) | `88.00% <100.00%> (+23.29%)` | :arrow_up: |
   | [libs/utils/src/hash\_map.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-bGlicy91dGlscy9zcmMvaGFzaF9tYXAuYw==) | `93.94% <0.00%> (+0.57%)` | :arrow_up: |
   | [...sub\_topology\_manager/src/pubsub\_topology\_manager.c](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree#diff-YnVuZGxlcy9wdWJzdWIvcHVic3ViX3RvcG9sb2d5X21hbmFnZXIvc3JjL3B1YnN1Yl90b3BvbG9neV9tYW5hZ2VyLmM=) | `52.80% <0.00%> (+1.14%)` | :arrow_up: |
   | ... and [6 more](https://codecov.io/gh/apache/celix/pull/293/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=footer). Last update [4ebde14...4e70741](https://codecov.io/gh/apache/celix/pull/293?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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