You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by ha...@apache.org on 2016/08/11 04:14:33 UTC

[1/2] incubator-eagle git commit: [EAGLE-442] Support ApplicationProvider to extend extensible guice modules

Repository: incubator-eagle
Updated Branches:
  refs/heads/develop c65138b8b -> 6d5907daa


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java b/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
index 43f5c10..6299e28 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
@@ -16,10 +16,7 @@
  */
 package org.apache.eagle.server;
 
-import com.google.inject.Binding;
-import com.hubspot.dropwizard.guice.GuiceBundle;
 import com.sun.jersey.api.core.PackagesResourceConfig;
-import com.typesafe.config.Config;
 import io.dropwizard.Application;
 import io.dropwizard.assets.AssetsBundle;
 import io.dropwizard.setup.Bootstrap;
@@ -28,30 +25,15 @@ import io.swagger.jaxrs.config.BeanConfig;
 import io.swagger.jaxrs.listing.ApiListingResource;
 import org.apache.eagle.alert.coordinator.CoordinatorListener;
 import org.apache.eagle.alert.resource.SimpleCORSFiler;
-import org.apache.eagle.app.ApplicationGuiceModule;
-import org.apache.eagle.common.module.CommonGuiceModule;
-import org.apache.eagle.common.module.ConfigServiceProvider;
-import org.apache.eagle.metadata.persistence.MetadataStore;
+import org.apache.eagle.server.module.GuideBundleLoader;
 
 import javax.servlet.DispatcherType;
-import java.io.File;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.Arrays;
 import java.util.EnumSet;
 
 class ServerApplication extends Application<ServerConfig> {
-    private GuiceBundle<ServerConfig> guiceBundle;
-
     @Override
     public void initialize(Bootstrap<ServerConfig> bootstrap) {
-        guiceBundle = GuiceBundle.<ServerConfig>newBuilder()
-                .addModule(new CommonGuiceModule())
-                .addModule(MetadataStore.getInstance())
-                .addModule(new ApplicationGuiceModule())
-                .setConfigClass(ServerConfig.class)
-                .build();
-        bootstrap.addBundle(guiceBundle);
+        bootstrap.addBundle(GuideBundleLoader.load());
         bootstrap.addBundle(new AssetsBundle("/assets","/","index.html","/"));
     }
 
@@ -66,18 +48,6 @@ class ServerApplication extends Application<ServerConfig> {
         environment.jersey().register(RESTExceptionMapper.class);
         environment.jersey().setUrlPattern(ServerConfig.getApiBasePath());
 
-        Binding<Config> b = guiceBundle.getInjector().getBinding(Config.class);
-        Config conf = b.getProvider().get();
-        if(conf.hasPath("application.provider.dir")) {
-            File loc = new File(conf.getString("application.provider.dir"));
-            File[] jarFiles = loc.listFiles(file -> file.getPath().toLowerCase().endsWith(".jar"));
-            URL[] urls = new URL[jarFiles.length];
-            for (int i = 0; i < jarFiles.length; i++) {
-                urls[i] = jarFiles[i].toURI().toURL();
-            }
-            URLClassLoader jarFileClassLoader = new URLClassLoader(urls);
-            Thread.currentThread().setContextClassLoader(jarFileClassLoader);
-        }
         // Automatically scan all REST resources
         new PackagesResourceConfig(ServerConfig.getResourcePackage()).getClasses().forEach(environment.jersey()::register);
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java b/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
index 55c5c5d..3bbe335 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/ServerConfig.java
@@ -20,7 +20,7 @@ import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 import io.dropwizard.Configuration;
 
-class ServerConfig extends Configuration {
+public class ServerConfig extends Configuration {
     private final static String SERVER_NAME = "Apache Eagle";
     private final static String SERVER_VERSION = "0.5.0-incubating";
     private final static String API_BASE_PATH = "/rest/*";

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java b/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
new file mode 100644
index 0000000..3dc3eec
--- /dev/null
+++ b/eagle-server/src/main/java/org/apache/eagle/server/module/GuideBundleLoader.java
@@ -0,0 +1,64 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.server.module;
+
+import com.google.inject.Module;
+import com.hubspot.dropwizard.guice.GuiceBundle;
+import org.apache.eagle.app.module.ApplicationExtensionLoader;
+import org.apache.eagle.common.module.GlobalScope;
+import org.apache.eagle.common.module.ModuleRegistry;
+import org.apache.eagle.metadata.persistence.MetadataStore;
+import org.apache.eagle.metadata.persistence.MetadataStoreFactory;
+import org.apache.eagle.server.ServerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class GuideBundleLoader {
+    private final static Logger LOGGER = LoggerFactory.getLogger(GuideBundleLoader.class);
+    public static GuiceBundle<ServerConfig> load(List<Module> modules){
+        MetadataStore metadataStoreModule = MetadataStoreFactory.getInstance();
+        ServeModule serveBaseModule = new ServeModule();
+        GuiceBundle.Builder<ServerConfig> builder = GuiceBundle.newBuilder();
+        ModuleRegistry registry = ApplicationExtensionLoader.load(serveBaseModule);
+        List<Module> metadataExtensions = metadataStoreModule.getModules(registry);
+        int extensionNum = 0;
+        if(metadataExtensions!=null){
+            extensionNum = metadataExtensions.size();
+            metadataExtensions.forEach(builder::addModule);
+        }
+        LOGGER.info("Loaded {} modules (scope: metadata)",extensionNum,metadataStoreModule);
+
+        List<Module> globalExtensions = registry.getModules(GlobalScope.class);
+        extensionNum = 0;
+        if(globalExtensions!=null){
+            extensionNum = globalExtensions.size();
+            globalExtensions.forEach(builder::addModule);
+        }
+        LOGGER.info("Loaded {} modules (scope: global)",extensionNum,metadataStoreModule);
+
+        if(modules!=null) modules.forEach(builder::addModule);
+        return builder.addModule(serveBaseModule)
+                .setConfigClass(ServerConfig.class)
+                .build();
+    }
+
+    public static GuiceBundle<ServerConfig> load(){
+        return load(null);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-server/src/main/java/org/apache/eagle/server/module/ServeModule.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/module/ServeModule.java b/eagle-server/src/main/java/org/apache/eagle/server/module/ServeModule.java
new file mode 100644
index 0000000..3b7d7f9
--- /dev/null
+++ b/eagle-server/src/main/java/org/apache/eagle/server/module/ServeModule.java
@@ -0,0 +1,31 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.server.module;
+
+import com.google.inject.AbstractModule;
+import org.apache.eagle.app.module.ApplicationGuiceModule;
+import org.apache.eagle.common.module.CommonGuiceModule;
+import org.apache.eagle.metadata.persistence.MetadataStoreFactory;
+
+public class ServeModule extends AbstractModule {
+    @Override
+    protected void configure() {
+        install(new CommonGuiceModule());
+        install(new ApplicationGuiceModule());
+        install(MetadataStoreFactory.getInstance());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java b/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
new file mode 100644
index 0000000..4316dfd
--- /dev/null
+++ b/eagle-server/src/test/java/org/apache/eagle/server/GuideBundleLoaderTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.server;
+
+import com.hubspot.dropwizard.guice.GuiceBundle;
+import org.apache.eagle.server.module.GuideBundleLoader;
+import org.junit.Test;
+
+public class GuideBundleLoaderTest {
+    @Test
+    public void testLoad(){
+        GuiceBundle<ServerConfig> bundle = GuideBundleLoader.load();
+    }
+}


[2/2] incubator-eagle git commit: [EAGLE-442] Support ApplicationProvider to extend extensible guice modules

Posted by ha...@apache.org.
[EAGLE-442] Support ApplicationProvider to extend extensible guice modules

## Module Registry Interfaces

* ApplicationProvider
```
ApplicationProvider{
   void register(ModuleRegistry)
}
```

* ModuleRegistry
```
ModuleRegistry {
    void register(ModuleScope scope,Module module);
    default void register(Module moduel) {
        register(GlobalScope.class, module)
    }
}
```

* ModuleScope
- Modules registered in scope of `GlobalScope` will be visible globally
- Modules registered in `MySQLMetadataStore`/`MongodbMetadataStore`/`MemoryMetadataStore` will only be visible when selected MetadataStore is active (set in configuration file)

```
ModuleScope
    |- GlobalScope
    |- MetadataStore
        |- MySQLMetadataStore
        |- MongodbMetadataStore
        |- MemoryMetadataStore
```

## Example

* **Registry**: in `HBaseAuditLogAppProvider`

```
Override
    public void register(ModuleRegistry registry) {
        registry.register(MemoryMetadataStore.class, new AbstractModule() {
            Override
            protected void configure() {
                bind(ISecurityMetadataDAO.class).to(InMemMetadataDaoImpl.class);
            }
        });

        registry.register(MySQLMetadataStore.class, new AbstractModule() {
            Override
            protected void configure() {
                bind(ISecurityMetadataDAO.class).to(JDBCSecurityMetadataDAO.class);
            }
        });
    }
```
* **Inject**: For example, `HbaseMetadataBrowseWebResource` could get current `ISecurityMetadataDAO` with `Inject ISecurityMetadataDAO metadataDAO` as following, instead of using [`MetadataDaoFactory`](https://github.com/apache/incubator-eagle/blob/9b10f22221b60f0f6451e71ac3396b326f9565ef/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/MetadataDaoFactory.java)

```
Inject
    public HbaseMetadataBrowseWebResource(ApplicationEntityService entityService, ISecurityMetadataDAO metadataDAO){
        this.entityService = entityService;
        this.dao = metadataDAO;
    }
```

## More Use Cases
* ExampleResource
* ExampleService
* ExampleApplicationProviderTest#testApplicationExtensions

## JIRA
* https://issues.apache.org/jira/browse/EAGLE-442
* https://issues.apache.org/jira/browse/EAGLE-431

Author: Hao Chen <ha...@apache.org>

Closes #323 from haoch/EAGLE-442.


Project: http://git-wip-us.apache.org/repos/asf/incubator-eagle/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-eagle/commit/6d5907da
Tree: http://git-wip-us.apache.org/repos/asf/incubator-eagle/tree/6d5907da
Diff: http://git-wip-us.apache.org/repos/asf/incubator-eagle/diff/6d5907da

Branch: refs/heads/develop
Commit: 6d5907daaf1b164f1e399df2a50ab87dbf2ca1c9
Parents: c65138b
Author: Hao Chen <ha...@apache.org>
Authored: Thu Aug 11 12:14:31 2016 +0800
Committer: Hao Chen <ha...@apache.org>
Committed: Thu Aug 11 12:14:31 2016 +0800

----------------------------------------------------------------------
 .../eagle/app/ApplicationGuiceModule.java       | 36 --------
 .../app/module/ApplicationExtensionLoader.java  | 38 +++++++++
 .../app/module/ApplicationGuiceModule.java      | 36 ++++++++
 .../eagle/app/service/ApplicationContext.java   | 13 ++-
 .../impl/ApplicationManagementServiceImpl.java  | 16 ++--
 .../apache/eagle/app/sink/KafkaStreamSink.java  |  9 +-
 .../app/spi/AbstractApplicationProvider.java    | 26 +++---
 .../eagle/app/spi/ApplicationProvider.java      |  5 +-
 .../apache/eagle/app/test/AppJUnitRunner.java   | 87 ++++++++++++++++++++
 .../eagle/app/test/AppTestGuiceModule.java      | 17 +++-
 .../eagle/app/test/AppUnitTestRunner.java       | 84 -------------------
 .../app/ApplicationProviderServiceTest.java     |  3 +-
 .../apache/eagle/app/TestStormApplication.java  | 37 ++++++++-
 .../apache/eagle/common/module/GlobalScope.java | 22 +++++
 .../eagle/common/module/ModuleRegistry.java     | 52 ++++++++++++
 .../eagle/common/module/ModuleRegistryImpl.java | 50 +++++++++++
 .../apache/eagle/common/module/ModuleScope.java | 38 +++++++++
 .../eagle/common/module/ModuleRegistryTest.java | 52 ++++++++++++
 .../eagle-metadata/eagle-metadata-base/pom.xml  |  5 ++
 .../metadata/persistence/MetadataStore.java     | 34 +-------
 .../persistence/MetadataStoreFactory.java       | 51 ++++++++++++
 .../service/memory/MemoryMetadataStore.java     |  2 +-
 .../app/example/ExampleApplicationProvider.java | 39 ++++++++-
 .../extensions/ExampleCommonService.java        | 23 ++++++
 .../extensions/ExampleCommonServiceImpl.java    | 27 ++++++
 .../app/example/extensions/ExampleEntity.java   | 29 +++++++
 .../extensions/ExampleEntityService.java        | 23 ++++++
 .../ExampleEntityServiceMemoryImpl.java         | 30 +++++++
 .../app/example/extensions/ExampleResource.java | 58 +++++++++++++
 .../example/ExampleApplicationProviderTest.java | 20 +++--
 .../eagle/app/jpm/JPMApplicationProvider.java   |  7 +-
 .../eagle/app/jpm/JPMApplicationTest.java       |  4 +-
 .../service/JDBCSecurityMetadataDAO.java        |  2 +
 .../security/service/MetadataDaoFactory.java    | 41 ---------
 .../eagle-security-hbase-auditlog/pom.xml       |  5 ++
 .../hbase/HBaseAuditLogAppProvider.java         | 29 ++++++-
 .../hbase/HbaseMetadataBrowseWebResource.java   |  6 +-
 .../hbase/SensitivityMetadataResource.java      |  9 +-
 .../apache/eagle/server/ServerApplication.java  | 34 +-------
 .../org/apache/eagle/server/ServerConfig.java   |  2 +-
 .../eagle/server/module/GuideBundleLoader.java  | 64 ++++++++++++++
 .../apache/eagle/server/module/ServeModule.java | 31 +++++++
 .../eagle/server/GuideBundleLoaderTest.java     | 28 +++++++
 43 files changed, 932 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/ApplicationGuiceModule.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/ApplicationGuiceModule.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/ApplicationGuiceModule.java
deleted file mode 100644
index ca3b8f2..0000000
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/ApplicationGuiceModule.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * 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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.
- */
-package org.apache.eagle.app;
-
-import com.google.inject.AbstractModule;
-import com.google.inject.Singleton;
-import org.apache.eagle.app.service.ApplicationManagementService;
-import org.apache.eagle.app.service.impl.ApplicationManagementServiceImpl;
-import org.apache.eagle.app.service.ApplicationProviderService;
-import org.apache.eagle.app.service.impl.ApplicationProviderServiceImpl;
-import org.apache.eagle.metadata.service.ApplicationDescService;
-
-
-public class ApplicationGuiceModule extends AbstractModule {
-    @Override
-    protected void configure() {
-        bind(ApplicationProviderServiceImpl.class).in(Singleton.class);
-        bind(ApplicationProviderService.class).to(ApplicationProviderServiceImpl.class).in(Singleton.class);
-        bind(ApplicationDescService.class).to(ApplicationProviderServiceImpl.class).in(Singleton.class);
-        bind(ApplicationManagementService.class).to(ApplicationManagementServiceImpl.class).in(Singleton.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationExtensionLoader.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationExtensionLoader.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationExtensionLoader.java
new file mode 100644
index 0000000..061e134
--- /dev/null
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationExtensionLoader.java
@@ -0,0 +1,38 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.module;
+
+import com.google.inject.Guice;
+import com.google.inject.Module;
+import org.apache.eagle.app.service.ApplicationProviderService;
+import org.apache.eagle.common.module.ModuleRegistry;
+import org.apache.eagle.common.module.ModuleRegistryImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ApplicationExtensionLoader {
+    private final static Logger LOGGER = LoggerFactory.getLogger(ApplicationExtensionLoader.class);
+    public static ModuleRegistry load(Module ... context){
+        LOGGER.info("Loading application extension modules");
+        ModuleRegistry registry = new ModuleRegistryImpl();
+        Guice.createInjector(context).getInstance(ApplicationProviderService.class).getProviders().forEach((provider)->{
+            LOGGER.info("Registering modules from {}",provider);
+            provider.register(registry);
+        });
+        return registry;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
new file mode 100644
index 0000000..50efc0f
--- /dev/null
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
@@ -0,0 +1,36 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.module;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Singleton;
+import org.apache.eagle.app.service.ApplicationManagementService;
+import org.apache.eagle.app.service.impl.ApplicationManagementServiceImpl;
+import org.apache.eagle.app.service.ApplicationProviderService;
+import org.apache.eagle.app.service.impl.ApplicationProviderServiceImpl;
+import org.apache.eagle.metadata.service.ApplicationDescService;
+
+
+public class ApplicationGuiceModule extends AbstractModule {
+    @Override
+    protected void configure() {
+        bind(ApplicationProviderServiceImpl.class).in(Singleton.class);
+        bind(ApplicationProviderService.class).to(ApplicationProviderServiceImpl.class).in(Singleton.class);
+        bind(ApplicationDescService.class).to(ApplicationProviderServiceImpl.class).in(Singleton.class);
+        bind(ApplicationManagementService.class).to(ApplicationManagementServiceImpl.class).in(Singleton.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
index d577095..b0d6118 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationContext.java
@@ -58,11 +58,16 @@ public class ApplicationContext implements Serializable, ApplicationLifecycle {
         this.application = application;
         this.metadata = metadata;
         this.runtime = ExecutionRuntimeManager.getInstance().getRuntime(application.getEnvironmentType(),envConfig);
-        Map<String,Object> applicationConfig = metadata.getConfiguration();
-        if(applicationConfig == null) {
-            applicationConfig = Collections.emptyMap();
+        Map<String,Object> executionConfig = metadata.getConfiguration();
+        if(executionConfig == null) {
+            executionConfig = Collections.emptyMap();
         }
-        this.config = ConfigFactory.parseMap(applicationConfig).withFallback(envConfig);
+
+        // TODO: Decouple hardcoded configuration key
+        executionConfig.put("siteId", metadata.getSite().getSiteId());
+        executionConfig.put("mode", metadata.getMode().name());
+        executionConfig.put("appId", metadata.getAppId());
+        this.config = ConfigFactory.parseMap(executionConfig).withFallback(envConfig);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
index c98e7cc..a0b0dc4 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
@@ -71,15 +71,18 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         ApplicationEntity applicationEntity = new ApplicationEntity();
         applicationEntity.setDescriptor(appDesc);
         applicationEntity.setSite(siteEntity);
+        applicationEntity.setMode(operation.getMode());
+        applicationEntity.ensureDefault();
 
         /**
          *  calculate application config based on
          *   1) default values in metadata.xml
-         *   2) user's config value
-         *   3) some metadata, for example siteId, mode, appId
+         *   2) user's config value override default configurations
+         *   3) some metadata, for example siteId, mode, appId in ApplicationContext
          */
         Map<String, Object> appConfig = new HashMap<>();
         ApplicationProvider provider = applicationProviderService.getApplicationProviderByType(operation.getAppType());
+
         List<Property> propertyList = provider.getApplicationDesc().getConfiguration().getProperties();
         for(Property p : propertyList){
             appConfig.put(p.getName(), p.getValue());
@@ -87,19 +90,12 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         if(operation.getConfiguration() != null) {
             appConfig.putAll(operation.getConfiguration());
         }
-        appConfig.put("siteId", operation.getSiteId());
-        appConfig.put("mode", operation.getMode().name());
-        appConfig.put("appId", operation.getAppType());
-
         applicationEntity.setConfiguration(appConfig);
-        applicationEntity.setMode(operation.getMode());
         ApplicationContext applicationContext = new ApplicationContext(
                 applicationProviderService.getApplicationProviderByType(applicationEntity.getDescriptor().getType()).getApplication(),
                 applicationEntity,config);
         applicationContext.onInstall();
-        applicationEntityService.create(applicationEntity);
-
-        return applicationEntity;
+        return applicationEntityService.create(applicationEntity);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
index 9c18d88..5c33c94 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/sink/KafkaStreamSink.java
@@ -18,13 +18,11 @@ package org.apache.eagle.app.sink;
 
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.BasicOutputCollector;
-import backtype.storm.tuple.Tuple;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.typesafe.config.Config;
 import kafka.javaapi.producer.Producer;
 import kafka.producer.KeyedMessage;
 import kafka.producer.ProducerConfig;
-import org.apache.eagle.alert.engine.model.StreamEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -80,6 +78,13 @@ public class KafkaStreamSink extends StormStreamSink<KafkaStreamSinkConfig> {
     }
 
     @Override
+    public void cleanup() {
+        if(this.producer != null){
+            this.producer.close();
+        }
+    }
+
+    @Override
     public void onUninstall() {
         ensureTopicDeleted();
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/AbstractApplicationProvider.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/AbstractApplicationProvider.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/AbstractApplicationProvider.java
index b9034d9..71d2ce6 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/AbstractApplicationProvider.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/AbstractApplicationProvider.java
@@ -21,8 +21,7 @@ import org.apache.eagle.alert.engine.coordinator.StreamDefinition;
 import org.apache.eagle.app.Application;
 import org.apache.eagle.app.config.ApplicationProviderConfig;
 import org.apache.eagle.app.config.ApplicationProviderDescConfig;
-import org.apache.eagle.app.sink.KafkaStreamSink;
-import org.apache.eagle.app.sink.StreamSink;
+import org.apache.eagle.common.module.ModuleRegistry;
 import org.apache.eagle.metadata.model.ApplicationDesc;
 import org.apache.eagle.metadata.model.ApplicationDocs;
 import org.apache.eagle.metadata.model.Configuration;
@@ -36,19 +35,9 @@ public abstract class AbstractApplicationProvider<T extends Application> impleme
     private final static Logger LOG = LoggerFactory.getLogger(AbstractApplicationProvider.class);
     private final ApplicationDesc applicationDesc;
 
-//    Disable programmable ApplicationProvider approach
-//
-//    public AbstractApplicationProvider(){
-//        applicationDesc = new ApplicationDesc();
-//        applicationDesc.setProviderClass(this.getClass());
-//        configure();
-//    }
-//
-//    protected void configure (){
-//        // do nothing by default
-//    }
-
-    protected AbstractApplicationProvider(String applicationDescConfig) {
+    protected abstract String getMetadata();
+    protected AbstractApplicationProvider() {
+        String applicationDescConfig = getMetadata();
         applicationDesc = new ApplicationDesc();
         applicationDesc.setProviderClass(this.getClass());
         ApplicationProviderDescConfig descWrapperConfig = ApplicationProviderDescConfig.loadFromXML(this.getClass(), applicationDescConfig);
@@ -132,4 +121,9 @@ public abstract class AbstractApplicationProvider<T extends Application> impleme
     public ApplicationDesc getApplicationDesc() {
         return applicationDesc;
     }
-}
+
+    @Override
+    public void register(ModuleRegistry registry) {
+        LOG.debug("Registering modules {}",this.getClass().getName());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/ApplicationProvider.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/ApplicationProvider.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/ApplicationProvider.java
index be84f0c..0c68cfc 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/ApplicationProvider.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/spi/ApplicationProvider.java
@@ -20,6 +20,7 @@ import com.typesafe.config.Config;
 import org.apache.eagle.app.Application;
 import org.apache.eagle.app.config.ApplicationProviderConfig;
 import org.apache.eagle.metadata.model.ApplicationDesc;
+import org.apache.eagle.common.module.ModuleRegistry;
 
 public interface ApplicationProvider<T extends Application> {
 
@@ -34,4 +35,6 @@ public interface ApplicationProvider<T extends Application> {
      * @return application instance
      */
     T getApplication();
-}
+
+    void register(ModuleRegistry registry);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppJUnitRunner.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppJUnitRunner.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppJUnitRunner.java
new file mode 100644
index 0000000..7e16bce
--- /dev/null
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppJUnitRunner.java
@@ -0,0 +1,87 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.test;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import org.apache.eagle.app.module.ApplicationExtensionLoader;
+import org.junit.runners.BlockJUnit4ClassRunner;
+import org.junit.runners.model.InitializationError;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class AppJUnitRunner extends BlockJUnit4ClassRunner {
+    private final Injector injector;
+    public AppJUnitRunner(Class<?> klass) throws InitializationError {
+        super(klass);
+        injector = createInjectorFor(getModulesFor(klass));
+    }
+
+    @Override
+    protected Object createTest() throws Exception {
+        final Object obj = super.createTest();
+        injector.injectMembers(this);
+        this.injector.injectMembers(obj);
+        return obj;
+    }
+
+    /**
+     * Create a Guice Injector for the class under test.
+     * @param classes Guice Modules
+     * @return A Guice Injector instance.
+     * @throws InitializationError If couldn't instantiate a module.
+     */
+    private Injector createInjectorFor(final Class<?>[] classes)
+            throws InitializationError {
+        final List<Module> modules = new ArrayList<>();
+
+        AppTestGuiceModule testGuiceModule = new AppTestGuiceModule();
+
+        // Add default modules
+        modules.add(testGuiceModule);
+
+        if(classes!= null) {
+            for (final Class<?> module : Arrays.asList(classes)) {
+                try {
+                    modules.add((Module) module.newInstance());
+                } catch (final ReflectiveOperationException exception) {
+                    throw new InitializationError(exception);
+                }
+            }
+        }
+        return Guice.createInjector(modules);
+    }
+
+    /**
+     * Get the list of Guice Modules request by GuiceModules annotation in the
+     * class under test.
+     * @param klass Class under test.
+     * @return A Class Array of Guice Modules required by this class.
+     * @throws InitializationError If the annotation is not present.
+     */
+    private Class<?>[] getModulesFor(final Class<?> klass)
+            throws InitializationError {
+        final Modules annotation = klass.getAnnotation(Modules.class);
+        if (annotation == null) {
+            return null;
+        }
+        return annotation.value();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppTestGuiceModule.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppTestGuiceModule.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppTestGuiceModule.java
index 051b974..9b30ee4 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppTestGuiceModule.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppTestGuiceModule.java
@@ -18,16 +18,25 @@ package org.apache.eagle.app.test;
 
 import com.google.inject.AbstractModule;
 import com.google.inject.Singleton;
-import org.apache.eagle.app.ApplicationGuiceModule;
+import org.apache.eagle.app.module.ApplicationExtensionLoader;
+import org.apache.eagle.app.module.ApplicationGuiceModule;
 import org.apache.eagle.common.module.CommonGuiceModule;
+import org.apache.eagle.common.module.GlobalScope;
+import org.apache.eagle.common.module.ModuleRegistry;
 import org.apache.eagle.metadata.service.memory.MemoryMetadataStore;
 
 public class AppTestGuiceModule extends AbstractModule{
     @Override
     protected void configure() {
-        install(new CommonGuiceModule());
-        install(new ApplicationGuiceModule());
-        install(new MemoryMetadataStore());
+        CommonGuiceModule common = new CommonGuiceModule();
+        ApplicationGuiceModule app = new ApplicationGuiceModule();
+        MemoryMetadataStore store = new MemoryMetadataStore();
+        install(common);
+        install(app);
+        install(store);
+        ModuleRegistry registry =ApplicationExtensionLoader.load(common,app,store);
+        registry.getModules(store.getClass()).forEach(this::install);
+        registry.getModules(GlobalScope.class).forEach(this::install);
         bind(ServerSimulator.class).to(ServerSimulatorImpl.class).in(Singleton.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppUnitTestRunner.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppUnitTestRunner.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppUnitTestRunner.java
deleted file mode 100644
index 2dda4d6..0000000
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/test/AppUnitTestRunner.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.
- */
-package org.apache.eagle.app.test;
-
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Module;
-import org.junit.runners.BlockJUnit4ClassRunner;
-import org.junit.runners.model.InitializationError;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-public class AppUnitTestRunner extends BlockJUnit4ClassRunner {
-    private final Injector injector;
-    public AppUnitTestRunner(Class<?> klass) throws InitializationError {
-        super(klass);
-        injector = createInjectorFor(getModulesFor(klass));
-    }
-
-    @Override
-    protected Object createTest() throws Exception {
-        final Object obj = super.createTest();
-        injector.injectMembers(this);
-        this.injector.injectMembers(obj);
-        return obj;
-    }
-
-    /**
-     * Create a Guice Injector for the class under test.
-     * @param classes Guice Modules
-     * @return A Guice Injector instance.
-     * @throws InitializationError If couldn't instantiate a module.
-     */
-    private Injector createInjectorFor(final Class<?>[] classes)
-            throws InitializationError {
-        final List<Module> modules = new ArrayList<>();
-
-        // Add default modules
-        modules.add(new AppTestGuiceModule());
-
-        if(classes!= null) {
-            for (final Class<?> module : Arrays.asList(classes)) {
-                try {
-                    modules.add((Module) module.newInstance());
-                } catch (final ReflectiveOperationException exception) {
-                    throw new InitializationError(exception);
-                }
-            }
-        }
-        return Guice.createInjector(modules);
-    }
-
-    /**
-     * Get the list of Guice Modules request by GuiceModules annotation in the
-     * class under test.
-     * @param klass Class under test.
-     * @return A Class Array of Guice Modules required by this class.
-     * @throws InitializationError If the annotation is not present.
-     */
-    private Class<?>[] getModulesFor(final Class<?> klass)
-            throws InitializationError {
-        final Modules annotation = klass.getAnnotation(Modules.class);
-        if (annotation == null) {
-            return null;
-        }
-        return annotation.value();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/ApplicationProviderServiceTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/ApplicationProviderServiceTest.java b/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/ApplicationProviderServiceTest.java
index a3f0ba0..543a834 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/ApplicationProviderServiceTest.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/ApplicationProviderServiceTest.java
@@ -19,6 +19,7 @@ package org.apache.eagle.app;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;
+import org.apache.eagle.app.module.ApplicationGuiceModule;
 import org.apache.eagle.app.service.ApplicationProviderService;
 import org.apache.eagle.app.spi.ApplicationProvider;
 import org.apache.eagle.common.module.CommonGuiceModule;
@@ -32,7 +33,7 @@ import java.util.Collection;
 
 public class ApplicationProviderServiceTest {
     private final static Logger LOGGER = LoggerFactory.getLogger(ApplicationProviderServiceTest.class);
-    private Injector injector = Guice.createInjector(new CommonGuiceModule(),new ApplicationGuiceModule(), MemoryMetadataStore.getInstance());
+    private Injector injector = Guice.createInjector(new CommonGuiceModule(),new ApplicationGuiceModule(), new MemoryMetadataStore());
 
     @Test
     public void testApplicationProviderManagerInit(){

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/TestStormApplication.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/TestStormApplication.java b/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/TestStormApplication.java
index 1d06f14..cd9a42d 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/TestStormApplication.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/test/java/org/apache/eagle/app/TestStormApplication.java
@@ -23,9 +23,13 @@ import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.topology.TopologyBuilder;
 import backtype.storm.topology.base.BaseRichSpout;
 import backtype.storm.tuple.Fields;
+import com.google.inject.AbstractModule;
+import com.google.inject.Inject;
 import com.typesafe.config.Config;
 import org.apache.eagle.app.environment.impl.StormEnvironment;
+import org.apache.eagle.common.module.ModuleRegistry;
 import org.apache.eagle.app.spi.AbstractApplicationProvider;
+import org.apache.eagle.metadata.service.memory.MemoryMetadataStore;
 import org.junit.Ignore;
 
 import java.util.Arrays;
@@ -62,12 +66,41 @@ public class TestStormApplication extends StormApplication{
     }
 
     public final static class Provider extends AbstractApplicationProvider<TestStormApplication> {
-        public Provider(){
-            super("TestApplicationMetadata.xml");
+        @Override
+        protected String getMetadata() {
+            return "TestApplicationMetadata.xml";
         }
+
         @Override
         public TestStormApplication getApplication() {
             return new TestStormApplication();
         }
+
+        @Override
+        public void register(ModuleRegistry registry) {
+            registry.register(MemoryMetadataStore.class, new AbstractModule() {
+                @Override
+                protected void configure() {
+                    bind(ExtendedDao.class).to(ExtendedDaoImpl.class);
+                }
+            });
+        }
+    }
+
+    private interface ExtendedDao{
+        Class<? extends ExtendedDao> getType();
+    }
+
+    private class ExtendedDaoImpl implements ExtendedDao {
+        private final Config config;
+
+        @Inject
+        public ExtendedDaoImpl(Config config){
+            this.config = config;
+        }
+        @Override
+        public Class<? extends ExtendedDao> getType() {
+            return ExtendedDaoImpl.class;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/GlobalScope.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/GlobalScope.java b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/GlobalScope.java
new file mode 100644
index 0000000..e7c73e4
--- /dev/null
+++ b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/GlobalScope.java
@@ -0,0 +1,22 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.common.module;
+
+/**
+ * Global Scope
+ */
+public enum GlobalScope implements ModuleScope {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistry.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistry.java b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistry.java
new file mode 100644
index 0000000..8ae3984
--- /dev/null
+++ b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistry.java
@@ -0,0 +1,52 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.common.module;
+
+import com.google.inject.Module;
+
+import java.util.List;
+
+/**
+ * Module Registry Repository
+ */
+public interface ModuleRegistry {
+    /**
+     * @param modules
+     */
+    void register(Class<? extends ModuleScope> scope, Module ... modules);
+
+    /**
+     *
+     * @param moduleScope
+     * @return
+     */
+    List<Module> getModules(Class<? extends ModuleScope> moduleScope);
+
+    /**
+     * @return
+     */
+    List<Class<? extends ModuleScope>> getScopes();
+
+    /**
+     * @return all modules
+     */
+    List<Module> getModules();
+
+    default void register(Module ... modules){
+        register(GlobalScope.class,modules);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistryImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistryImpl.java b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistryImpl.java
new file mode 100644
index 0000000..3c45dac
--- /dev/null
+++ b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleRegistryImpl.java
@@ -0,0 +1,50 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.common.module;
+
+import com.google.common.collect.LinkedListMultimap;
+import com.google.inject.Module;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class ModuleRegistryImpl implements ModuleRegistry {
+    private final LinkedListMultimap<Class<? extends ModuleScope>, Module> moduleRepo;
+    public ModuleRegistryImpl(){
+        moduleRepo = LinkedListMultimap.create();
+    }
+
+    @Override
+    public void register(Class<? extends ModuleScope> scope, Module... modules) {
+        moduleRepo.putAll(scope, Arrays.asList(modules));
+    }
+
+    @Override
+    public List<Module> getModules(Class<? extends ModuleScope> moduleScope) {
+        return moduleRepo.get(moduleScope);
+    }
+
+    @Override
+    public List<Class<? extends ModuleScope>> getScopes() {
+        return Arrays.asList((Class<? extends ModuleScope>[]) moduleRepo.keys().toArray());
+    }
+
+    @Override
+    public List<Module> getModules() {
+        return moduleRepo.values();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleScope.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleScope.java b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleScope.java
new file mode 100644
index 0000000..b385fba
--- /dev/null
+++ b/eagle-core/eagle-common/src/main/java/org/apache/eagle/common/module/ModuleScope.java
@@ -0,0 +1,38 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.common.module;
+
+import com.google.inject.Module;
+
+import java.util.List;
+
+/**
+ * @see ModuleRegistry
+ */
+public interface ModuleScope {
+    /**
+     * @param registry ModuleRegistry
+     * @return
+     */
+    default List<Module> getModules(ModuleRegistry registry){
+        return registry.getModules(this.getClass());
+    }
+
+    default String getName(){
+        return getClass().getName();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/module/ModuleRegistryTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/module/ModuleRegistryTest.java b/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/module/ModuleRegistryTest.java
new file mode 100644
index 0000000..a7236bb
--- /dev/null
+++ b/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/module/ModuleRegistryTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.common.module;
+
+import com.google.inject.AbstractModule;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ModuleRegistryTest {
+    @Test
+    public void testPutAndGet(){
+        ModuleRegistry registry = new ModuleRegistryImpl();
+        registry.register(TestModuleScope_1.class, new AbstractModule() {
+            @Override
+            protected void configure() {
+
+            }
+        });
+        registry.register(TestModuleScope_2.class, new AbstractModule() {
+            @Override
+            protected void configure() {
+
+            }
+        });
+        registry.register(TestModuleScope_1.class, new AbstractModule() {
+            @Override
+            protected void configure() {
+
+            }
+        });
+        Assert.assertEquals(2,registry.getModules(TestModuleScope_1.class).size());
+        Assert.assertEquals(1,registry.getModules(TestModuleScope_2.class).size());
+        Assert.assertEquals(3,registry.getModules().size());
+    }
+
+    private class TestModuleScope_1 implements ModuleScope {}
+    private class TestModuleScope_2 implements ModuleScope {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-metadata/eagle-metadata-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-metadata/eagle-metadata-base/pom.xml b/eagle-core/eagle-metadata/eagle-metadata-base/pom.xml
index 48abd63..2019b5b 100644
--- a/eagle-core/eagle-metadata/eagle-metadata-base/pom.xml
+++ b/eagle-core/eagle-metadata/eagle-metadata-base/pom.xml
@@ -54,5 +54,10 @@
             <groupId>io.dropwizard</groupId>
             <artifactId>dropwizard-servlets</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStore.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStore.java b/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStore.java
index 0ad8c87..023e14a 100644
--- a/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStore.java
+++ b/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStore.java
@@ -17,36 +17,8 @@
 package org.apache.eagle.metadata.persistence;
 
 import com.google.inject.AbstractModule;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-import org.apache.eagle.metadata.service.memory.MemoryMetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.eagle.common.module.ModuleScope;
 
-public abstract class MetadataStore extends AbstractModule {
-    private final static Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
-    public static final String METADATA_STORE_CONFIG_KEY = "metadata.store";
-
-    private static MetadataStore instance;
-
-    public static MetadataStore getInstance(){
-        String metadataStoreClass = null;
-        if(instance == null) {
-            try {
-                Config config = ConfigFactory.load();
-                if (config.hasPath(METADATA_STORE_CONFIG_KEY)) {
-                    metadataStoreClass = config.getString(METADATA_STORE_CONFIG_KEY);
-                    LOG.info("Using {} = {}",METADATA_STORE_CONFIG_KEY,metadataStoreClass);
-                }else{
-                    metadataStoreClass = MemoryMetadataStore.class.getCanonicalName();
-                    LOG.info("{} is not set, using default {}",METADATA_STORE_CONFIG_KEY,metadataStoreClass);
-                }
-                instance = (MetadataStore) Class.forName(metadataStoreClass).newInstance();
-            } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-                LOG.error("Failed to instantiate {}",metadataStoreClass,e);
-                throw new RuntimeException(e.getMessage(), e.getCause());
-            }
-        }
-        return instance;
-    }
+public abstract class MetadataStore extends AbstractModule implements ModuleScope {
+    
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStoreFactory.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStoreFactory.java b/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStoreFactory.java
new file mode 100644
index 0000000..32eb839
--- /dev/null
+++ b/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/persistence/MetadataStoreFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.metadata.persistence;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import org.apache.eagle.metadata.service.memory.MemoryMetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetadataStoreFactory {
+    private final static Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
+    public static final String METADATA_STORE_CONFIG_KEY = "metadata.store";
+
+    private static MetadataStore instance;
+
+    public static MetadataStore getInstance(){
+        String metadataStoreClass = null;
+        if(instance == null) {
+            try {
+                Config config = ConfigFactory.load();
+                if (config.hasPath(METADATA_STORE_CONFIG_KEY)) {
+                    metadataStoreClass = config.getString(METADATA_STORE_CONFIG_KEY);
+                    LOG.info("Using {} = {}",METADATA_STORE_CONFIG_KEY,metadataStoreClass);
+                }else{
+                    metadataStoreClass = MemoryMetadataStore.class.getCanonicalName();
+                    LOG.info("{} is not set, using default {}",METADATA_STORE_CONFIG_KEY,metadataStoreClass);
+                }
+                instance = (MetadataStore) Class.forName(metadataStoreClass).newInstance();
+            } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+                LOG.error("Failed to instantiate {}",metadataStoreClass,e);
+                throw new RuntimeException(e.getMessage(), e.getCause());
+            }
+        }
+        return instance;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/service/memory/MemoryMetadataStore.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/service/memory/MemoryMetadataStore.java b/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/service/memory/MemoryMetadataStore.java
index 0bdb199..cfb173e 100644
--- a/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/service/memory/MemoryMetadataStore.java
+++ b/eagle-core/eagle-metadata/eagle-metadata-base/src/main/java/org/apache/eagle/metadata/service/memory/MemoryMetadataStore.java
@@ -22,7 +22,7 @@ import org.apache.eagle.metadata.persistence.MetadataStore;
 import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.metadata.service.SiteEntityService;
 
-public class MemoryMetadataStore extends MetadataStore {
+public class MemoryMetadataStore extends MetadataStore{
     @Override
     protected void configure() {
         bind(SiteEntityService.class).to(SiteEntityEntityServiceMemoryImpl.class);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/ExampleApplicationProvider.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/ExampleApplicationProvider.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/ExampleApplicationProvider.java
index 7305817..f3b7939 100644
--- a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/ExampleApplicationProvider.java
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/ExampleApplicationProvider.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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.
@@ -16,18 +16,49 @@
  */
 package org.apache.eagle.app.example;
 
+import com.google.inject.AbstractModule;
+import org.apache.eagle.app.example.extensions.ExampleCommonService;
+import org.apache.eagle.app.example.extensions.ExampleCommonServiceImpl;
+import org.apache.eagle.app.example.extensions.ExampleEntityService;
+import org.apache.eagle.app.example.extensions.ExampleEntityServiceMemoryImpl;
 import org.apache.eagle.app.spi.AbstractApplicationProvider;
+import org.apache.eagle.common.module.GlobalScope;
+import org.apache.eagle.common.module.ModuleRegistry;
+import org.apache.eagle.metadata.service.memory.MemoryMetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Define application provider programmatically
+ * Define application provider pragmatically
  */
 public class ExampleApplicationProvider extends AbstractApplicationProvider<ExampleStormApplication> {
-    public ExampleApplicationProvider() {
-        super("/META-INF/apps/example/metadata.xml");
+    private final static Logger LOGGER = LoggerFactory.getLogger(ExampleApplicationProvider.class);
+    @Override
+    protected String getMetadata() {
+        return "/META-INF/apps/example/metadata.xml";
     }
 
     @Override
     public ExampleStormApplication getApplication() {
         return new ExampleStormApplication();
     }
+
+    @Override
+    public void register(ModuleRegistry registry) {
+        registry.register(MemoryMetadataStore.class, new AbstractModule() {
+            @Override
+            protected void configure() {
+                LOGGER.info("Load memory metadata modules ...");
+                bind(ExampleEntityService.class).to(ExampleEntityServiceMemoryImpl.class);
+            }
+        });
+
+        registry.register(new AbstractModule() {
+            @Override
+            protected void configure() {
+                LOGGER.info("Load global modules ...");
+                bind(ExampleCommonService.class).to(ExampleCommonServiceImpl.class);
+            }
+        });
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonService.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonService.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonService.java
new file mode 100644
index 0000000..0ae5cad
--- /dev/null
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonService.java
@@ -0,0 +1,23 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.example.extensions;
+
+import org.apache.eagle.common.module.ModuleScope;
+
+public interface ExampleCommonService {
+    Class<? extends ModuleScope> getScope();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonServiceImpl.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonServiceImpl.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonServiceImpl.java
new file mode 100644
index 0000000..52d6b10
--- /dev/null
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleCommonServiceImpl.java
@@ -0,0 +1,27 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.example.extensions;
+
+import org.apache.eagle.common.module.GlobalScope;
+import org.apache.eagle.common.module.ModuleScope;
+
+public class ExampleCommonServiceImpl implements ExampleCommonService{
+    @Override
+    public Class<? extends ModuleScope> getScope() {
+        return GlobalScope.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntity.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntity.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntity.java
new file mode 100644
index 0000000..8dc7d43
--- /dev/null
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntity.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.example.extensions;
+
+public class ExampleEntity {
+    private String message;
+
+    public String getMessage() {
+        return message;
+    }
+
+    public void setMessage(String message) {
+        this.message = message;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityService.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityService.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityService.java
new file mode 100644
index 0000000..d3c7cb8
--- /dev/null
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityService.java
@@ -0,0 +1,23 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.example.extensions;
+
+import java.util.List;
+
+public interface ExampleEntityService {
+    List<ExampleEntity> getEntities();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityServiceMemoryImpl.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityServiceMemoryImpl.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityServiceMemoryImpl.java
new file mode 100644
index 0000000..541215f
--- /dev/null
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleEntityServiceMemoryImpl.java
@@ -0,0 +1,30 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.example.extensions;
+
+import java.util.Collections;
+import java.util.List;
+
+public class ExampleEntityServiceMemoryImpl implements ExampleEntityService {
+
+    @Override
+    public List<ExampleEntity> getEntities() {
+        ExampleEntity entity = new ExampleEntity();
+        entity.setMessage("Message from "+ExampleEntityServiceMemoryImpl.class.getCanonicalName());
+        return Collections.singletonList(entity);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleResource.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleResource.java b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleResource.java
new file mode 100644
index 0000000..8fdacc7
--- /dev/null
+++ b/eagle-examples/eagle-app-example/src/main/java/org/apache/eagle/app/example/extensions/ExampleResource.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+package org.apache.eagle.app.example.extensions;
+
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import org.apache.eagle.common.module.ModuleScope;
+
+import javax.ws.rs.Path;
+import java.util.List;
+
+import javax.ws.rs.*;
+import javax.ws.rs.core.MediaType;
+
+@Path("/example")
+public class ExampleResource {
+    @Inject
+    private ExampleEntityService exampleEntityService;
+    @Inject
+    private ExampleCommonService commonService;
+    @Inject
+    private Config config;
+
+    @GET
+    @Produces(MediaType.APPLICATION_JSON)
+    public List<ExampleEntity> getEntities(){
+        return exampleEntityService.getEntities();
+    }
+
+    @GET
+    @Path("/config")
+    @Produces(MediaType.APPLICATION_JSON)
+    public Config getConfig(){
+        return config;
+    }
+
+    @GET
+    @Path("/common")
+    @Produces(MediaType.APPLICATION_JSON)
+    public Class<? extends ModuleScope> getCommonServiceScope(){
+        return commonService.getScope();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-examples/eagle-app-example/src/test/java/org/apache/eagle/app/example/ExampleApplicationProviderTest.java
----------------------------------------------------------------------
diff --git a/eagle-examples/eagle-app-example/src/test/java/org/apache/eagle/app/example/ExampleApplicationProviderTest.java b/eagle-examples/eagle-app-example/src/test/java/org/apache/eagle/app/example/ExampleApplicationProviderTest.java
index 65776ce..e07f487 100644
--- a/eagle-examples/eagle-app-example/src/test/java/org/apache/eagle/app/example/ExampleApplicationProviderTest.java
+++ b/eagle-examples/eagle-app-example/src/test/java/org/apache/eagle/app/example/ExampleApplicationProviderTest.java
@@ -17,28 +17,32 @@
 package org.apache.eagle.app.example;
 
 import com.google.inject.Inject;
+import org.apache.eagle.app.example.extensions.ExampleEntity;
+import org.apache.eagle.app.example.extensions.ExampleResource;
 import org.apache.eagle.app.resource.ApplicationResource;
 import org.apache.eagle.app.service.ApplicationOperations;
 import org.apache.eagle.app.test.ServerSimulator;
-import org.apache.eagle.app.test.AppUnitTestRunner;
+import org.apache.eagle.app.test.AppJUnitRunner;
+import org.apache.eagle.common.module.GlobalScope;
 import org.apache.eagle.metadata.model.ApplicationDesc;
 import org.apache.eagle.metadata.model.ApplicationEntity;
 import org.apache.eagle.metadata.model.SiteEntity;
 import org.apache.eagle.metadata.resource.SiteResource;
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
-@RunWith(AppUnitTestRunner.class)
+@RunWith(AppJUnitRunner.class)
 public class ExampleApplicationProviderTest {
     @Inject private SiteResource siteResource;
     @Inject private ApplicationResource applicationResource;
     @Inject private ServerSimulator simulator;
+    @Inject private ExampleResource exampleResource;
 
     @Test
     public void testApplicationProviderLoading(){
@@ -47,6 +51,14 @@ public class ExampleApplicationProviderTest {
         Assert.assertEquals(1,applicationDescs.size());
     }
 
+    @Test
+    public void testApplicationExtensions(){
+        List<ExampleEntity> entities = exampleResource.getEntities();
+        Assert.assertNotNull(entities);
+        Assert.assertEquals(1,entities.size());
+        Assert.assertEquals(GlobalScope.class,exampleResource.getCommonServiceScope());
+    }
+
     /**
      * register site
      * install app
@@ -57,7 +69,6 @@ public class ExampleApplicationProviderTest {
      * @throws InterruptedException
      */
     @Test
-    @Ignore
     public void testApplicationLifecycle() throws InterruptedException {
         // Create local site
         SiteEntity siteEntity = new SiteEntity();
@@ -90,7 +101,6 @@ public class ExampleApplicationProviderTest {
         simulator.start("EXAMPLE_APPLICATION", getConf());
     }
 
-    @Ignore
     @Test
     public void testApplicationQuickRunWithAppProvider() throws Exception{
         simulator.start(ExampleApplicationProvider.class, getConf());

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplicationProvider.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplicationProvider.java b/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplicationProvider.java
index 1da0c63..9694fd7 100644
--- a/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplicationProvider.java
+++ b/eagle-jpm/eagle-jpm-app/src/main/java/org/apache/eagle/app/jpm/JPMApplicationProvider.java
@@ -19,11 +19,12 @@ package org.apache.eagle.app.jpm;
 import org.apache.eagle.app.spi.AbstractApplicationProvider;
 
 /**
- * Define application provider programmatically
+ * Define application provider pragmatically
  */
 public class JPMApplicationProvider extends AbstractApplicationProvider<JPMApplication> {
-    public JPMApplicationProvider() {
-        super("/META-INF/apps/jpm/metadata.xml");
+    @Override
+    protected String getMetadata() {
+        return "/META-INF/apps/jpm/metadata.xml";
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-jpm/eagle-jpm-app/src/test/java/org/apache/eagle/app/jpm/JPMApplicationTest.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-app/src/test/java/org/apache/eagle/app/jpm/JPMApplicationTest.java b/eagle-jpm/eagle-jpm-app/src/test/java/org/apache/eagle/app/jpm/JPMApplicationTest.java
index c955d36..9c0ae2a 100644
--- a/eagle-jpm/eagle-jpm-app/src/test/java/org/apache/eagle/app/jpm/JPMApplicationTest.java
+++ b/eagle-jpm/eagle-jpm-app/src/test/java/org/apache/eagle/app/jpm/JPMApplicationTest.java
@@ -19,7 +19,7 @@ package org.apache.eagle.app.jpm;
 import com.google.inject.Inject;
 import org.apache.eagle.app.resource.ApplicationResource;
 import org.apache.eagle.app.service.ApplicationOperations;
-import org.apache.eagle.app.test.AppUnitTestRunner;
+import org.apache.eagle.app.test.AppJUnitRunner;
 import org.apache.eagle.metadata.model.ApplicationEntity;
 import org.apache.eagle.metadata.model.SiteEntity;
 import org.apache.eagle.metadata.resource.SiteResource;
@@ -30,7 +30,7 @@ import org.junit.runner.RunWith;
 import java.util.HashMap;
 import java.util.Map;
 
-@RunWith(AppUnitTestRunner.class)
+@RunWith(AppJUnitRunner.class)
 public class JPMApplicationTest {
     @Inject
     private SiteResource siteResource;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/JDBCSecurityMetadataDAO.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/JDBCSecurityMetadataDAO.java b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/JDBCSecurityMetadataDAO.java
index bf05d66..847a598 100644
--- a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/JDBCSecurityMetadataDAO.java
+++ b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/JDBCSecurityMetadataDAO.java
@@ -19,6 +19,7 @@
 
 package org.apache.eagle.security.service;
 
+import com.google.inject.Inject;
 import com.typesafe.config.Config;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,6 +42,7 @@ public class JDBCSecurityMetadataDAO implements ISecurityMetadataDAO  {
     private final String INSERT_STATEMENT = "INSERT INTO hbase_sensitivity_entity (site, hbase_resource, sensitivity_type) VALUES (?, ?, ?)";
 
     // get connection url from config
+    @Inject
     public JDBCSecurityMetadataDAO(Config config){
         this.config = config;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/MetadataDaoFactory.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/MetadataDaoFactory.java b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/MetadataDaoFactory.java
deleted file mode 100644
index 90cecd7..0000000
--- a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/service/MetadataDaoFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.
- */
-package org.apache.eagle.security.service;
-
-import com.typesafe.config.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @since Apr 12, 2016
- *
- */
-public class MetadataDaoFactory {
-    private static final Logger LOG = LoggerFactory.getLogger(MetadataDaoFactory.class);
-
-    public static ISecurityMetadataDAO getMetadataDAO(Config eagleServerConfig) {
-        String storeCls = eagleServerConfig.getString("metadata.store");
-
-        ISecurityMetadataDAO dao = null;
-        if (!storeCls.equalsIgnoreCase("org.apache.eagle.metadata.service.memory.MemoryMetadataStore")) {
-            dao = new InMemMetadataDaoImpl();
-        } else {
-            dao = new JDBCSecurityMetadataDAO(eagleServerConfig);
-        }
-        return dao;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-security/eagle-security-hbase-auditlog/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hbase-auditlog/pom.xml b/eagle-security/eagle-security-hbase-auditlog/pom.xml
index 46d67b5..42b9e86 100644
--- a/eagle-security/eagle-security-hbase-auditlog/pom.xml
+++ b/eagle-security/eagle-security-hbase-auditlog/pom.xml
@@ -44,5 +44,10 @@
             <artifactId>eagle-app-base</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.eagle</groupId>
+            <artifactId>eagle-metadata-jdbc</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-security/eagle-security-hbase-auditlog/src/main/java/org/apache/eagle/security/hbase/HBaseAuditLogAppProvider.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hbase-auditlog/src/main/java/org/apache/eagle/security/hbase/HBaseAuditLogAppProvider.java b/eagle-security/eagle-security-hbase-auditlog/src/main/java/org/apache/eagle/security/hbase/HBaseAuditLogAppProvider.java
index 23acc46..f8f5c90 100644
--- a/eagle-security/eagle-security-hbase-auditlog/src/main/java/org/apache/eagle/security/hbase/HBaseAuditLogAppProvider.java
+++ b/eagle-security/eagle-security-hbase-auditlog/src/main/java/org/apache/eagle/security/hbase/HBaseAuditLogAppProvider.java
@@ -19,18 +19,43 @@
 
 package org.apache.eagle.security.hbase;
 
+import com.google.inject.AbstractModule;
 import org.apache.eagle.app.spi.AbstractApplicationProvider;
+import org.apache.eagle.common.module.ModuleRegistry;
+import org.apache.eagle.metadata.service.memory.MemoryMetadataStore;
+import org.apache.eagle.metadata.store.mysql.MySQLMetadataStore;
+import org.apache.eagle.security.service.ISecurityMetadataDAO;
+import org.apache.eagle.security.service.InMemMetadataDaoImpl;
+import org.apache.eagle.security.service.JDBCSecurityMetadataDAO;
 
 /**
  * Since 8/5/16.
  */
 public class HBaseAuditLogAppProvider extends AbstractApplicationProvider<HBaseAuditLogApplication> {
-    public HBaseAuditLogAppProvider() {
-        super("/META-INF/metadata.xml");
+    @Override
+    protected String getMetadata() {
+        return "/META-INF/metadata.xml";
     }
 
     @Override
     public HBaseAuditLogApplication getApplication() {
         return new HBaseAuditLogApplication();
     }
+
+    @Override
+    public void register(ModuleRegistry registry) {
+        registry.register(MemoryMetadataStore.class, new AbstractModule() {
+            @Override
+            protected void configure() {
+                bind(ISecurityMetadataDAO.class).to(InMemMetadataDaoImpl.class);
+            }
+        });
+
+        registry.register(MySQLMetadataStore.class, new AbstractModule() {
+            @Override
+            protected void configure() {
+                bind(ISecurityMetadataDAO.class).to(JDBCSecurityMetadataDAO.class);
+            }
+        });
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java b/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
index 3fa85d9..441af8f 100644
--- a/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
+++ b/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/HbaseMetadataBrowseWebResource.java
@@ -26,7 +26,6 @@ import org.apache.eagle.security.entity.HbaseResourceEntity;
 import org.apache.eagle.security.resolver.MetadataAccessConfigRepo;
 import org.apache.eagle.security.service.HBaseSensitivityEntity;
 import org.apache.eagle.security.service.ISecurityMetadataDAO;
-import org.apache.eagle.security.service.MetadataDaoFactory;
 import org.apache.eagle.service.common.EagleExceptionWrapper;
 import org.apache.eagle.service.security.hbase.dao.HbaseMetadataDAOImpl;
 import org.apache.hadoop.conf.Configuration;
@@ -48,10 +47,9 @@ public class HbaseMetadataBrowseWebResource {
     private ISecurityMetadataDAO dao;
 
     @Inject
-    public HbaseMetadataBrowseWebResource(ApplicationEntityService entityService, Config eagleServerConfig){
+    public HbaseMetadataBrowseWebResource(ApplicationEntityService entityService, ISecurityMetadataDAO metadataDAO){
         this.entityService = entityService;
-        dao = MetadataDaoFactory.getMetadataDAO(eagleServerConfig);
-
+        this.dao = metadataDAO;
     }
 
     private Map<String, Map<String, String>> getAllSensitivities(){

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/6d5907da/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/SensitivityMetadataResource.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/SensitivityMetadataResource.java b/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/SensitivityMetadataResource.java
index e5d3b31..ccbff29 100644
--- a/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/SensitivityMetadataResource.java
+++ b/eagle-security/eagle-security-hbase-web/src/main/java/org/apache/eagle/service/security/hbase/SensitivityMetadataResource.java
@@ -20,16 +20,12 @@ package org.apache.eagle.service.security.hbase;
 
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
-import com.typesafe.config.Config;
-import org.apache.eagle.metadata.model.ApplicationEntity;
 import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.security.service.HBaseSensitivityEntity;
 import org.apache.eagle.security.service.ISecurityMetadataDAO;
-import org.apache.eagle.security.service.MetadataDaoFactory;
 
 import javax.ws.rs.*;
 import java.util.Collection;
-import java.util.Iterator;
 
 /**
  * Since 6/10/16.
@@ -39,10 +35,11 @@ import java.util.Iterator;
 public class SensitivityMetadataResource {
     private ApplicationEntityService entityService;
     private ISecurityMetadataDAO dao;
+
     @Inject
-    public SensitivityMetadataResource(ApplicationEntityService entityService, Config eagleServerConfig){
+    public SensitivityMetadataResource(ApplicationEntityService entityService, ISecurityMetadataDAO metadataDAO){
         this.entityService = entityService;
-        dao = MetadataDaoFactory.getMetadataDAO(eagleServerConfig);
+        this.dao = metadataDAO;
     }
 
     @Path("/hbase")