You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by da...@apache.org on 2016/09/08 05:20:46 UTC

incubator-eagle git commit: EAGLE-447 convert eagle-maprfs app to use new app framework

Repository: incubator-eagle
Updated Branches:
  refs/heads/master c2a7576e2 -> 2958bc8b4


EAGLE-447 convert eagle-maprfs app to use new app framework

https://issues.apache.org/jira/browse/EAGLE-447

Author: @DadanielZ <da...@apache.org>
Reviewer: @DadanielZ <da...@apache.org>

Closes #410


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

Branch: refs/heads/master
Commit: 2958bc8b4655312a094720d87ce3634b05e9f219
Parents: c2a7576
Author: DadanielZ <da...@apache.org>
Authored: Wed Sep 7 21:55:19 2016 -0700
Committer: DadanielZ <da...@apache.org>
Committed: Wed Sep 7 21:55:19 2016 -0700

----------------------------------------------------------------------
 .../security/hdfs/MAPRFSAuditLogParser.java     |  62 ++---
 .../AbstractHdfsAuditLogApplication.java        |   2 +-
 .../auditlog/MapRFSAuditLogAppProvider.java     |  29 +++
 ...urity.auditlog.MapRFSAuditLogAppProvider.xml | 246 +++++++++++++++++++
 ...org.apache.eagle.app.spi.ApplicationProvider |  37 +++
 .../src/main/resources/application.conf         |  71 ++----
 .../auditlog/MapRFSAuditLogAppProviderTest.java |  35 +++
 .../hdfs/resolver/MAPRFSCommandResolver.java    |   6 +
 .../hdfs/resolver/MAPRFSResourceResolver.java   |  33 ++-
 .../hdfs/resolver/MAPRFSVolumeResolver.java     |  35 ++-
 .../hdfs/resolver/MAPRStatusCodeResolver.java   |   9 +-
 .../hdfs/rest/MAPRFSResourceWebResource.java    |  35 ++-
 .../security/hdfs/rest/MapRNameResolver.java    |  31 ++-
 .../security/hdfs/rest/HttpRequestTest.java     |  30 ---
 .../hdfs/rest/MapRNameResolverTest.java         |  30 ---
 .../security/hdfs/rest/HttpRequestTest.java     |  30 +++
 .../hdfs/rest/MapRNameResolverTest.java         |  30 +++
 17 files changed, 592 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/hdfs/MAPRFSAuditLogParser.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/hdfs/MAPRFSAuditLogParser.java b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/hdfs/MAPRFSAuditLogParser.java
index 458de12..1dd2c78 100644
--- a/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/hdfs/MAPRFSAuditLogParser.java
+++ b/eagle-security/eagle-security-common/src/main/java/org/apache/eagle/security/hdfs/MAPRFSAuditLogParser.java
@@ -31,37 +31,41 @@ public final class MAPRFSAuditLogParser {
     public MAPRFSAuditLogParser(){
     }
 
-    public MAPRFSAuditLogObject parse(String log) throws JSONException, ParseException {
+    public MAPRFSAuditLogObject parse(String log) throws JSONException {
         JSONObject jsonObject = new JSONObject(log);
-        String timestamp = jsonObject.getJSONObject("timestamp").getString("$date");
-        String cmd = jsonObject.getString("operation");
-        String user = jsonObject.getString("uid");
-        String ip = jsonObject.getString("ipAddress");
-        String status = jsonObject.getString("status");
-        String volumeID = jsonObject.getString("volumeId");
-        String src;
-        String dst;
-        if(jsonObject.has("srcFid")){
-            src = jsonObject.getString("srcFid");
-        }else{
-            src = "null";
-        }
+        MAPRFSAuditLogObject entity = new MAPRFSAuditLogObject();
+        try{
+            String timestamp = jsonObject.getJSONObject("timestamp").getString("$date");
+            String cmd = jsonObject.getString("operation");
+            String user = jsonObject.getString("uid");
+            String ip = jsonObject.getString("ipAddress");
+            String status = jsonObject.getString("status");
+            String volumeID = jsonObject.getString("volumeId");
+            String src;
+            String dst;
+            if(jsonObject.has("srcFid")){
+                src = jsonObject.getString("srcFid");
+            }else{
+                src = "null";
+            }
 
-        if(jsonObject.has("dstFid")){
-            dst = jsonObject.getString("dstFid");
-        }else{
-            dst = "null";
+            if(jsonObject.has("dstFid")){
+                dst = jsonObject.getString("dstFid");
+            }else{
+                dst = "null";
+            }
+            entity.user = user;
+            entity.cmd = cmd;
+            entity.src = src;
+            entity.dst = dst;
+            entity.host = ip;
+            entity.status = status;
+            entity.volume = volumeID;
+            entity.timestamp = DateTimeUtil.maprhumanDateToMilliseconds(timestamp);
+        } catch (Exception e){
+            LOG.error("Failed to parse mapr audit log message", e);
+        } finally {
+            return entity;
         }
-
-        MAPRFSAuditLogObject entity = new MAPRFSAuditLogObject();
-        entity.user = user;
-        entity.cmd = cmd;
-        entity.src = src;
-        entity.dst = dst;
-        entity.host = ip;
-        entity.status = status;
-        entity.volume = volumeID;
-        entity.timestamp = DateTimeUtil.maprhumanDateToMilliseconds(timestamp);
-        return entity;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-hdfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/AbstractHdfsAuditLogApplication.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-hdfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/AbstractHdfsAuditLogApplication.java b/eagle-security/eagle-security-hdfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/AbstractHdfsAuditLogApplication.java
index bd23643..b985daf 100644
--- a/eagle-security/eagle-security-hdfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/AbstractHdfsAuditLogApplication.java
+++ b/eagle-security/eagle-security-hdfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/AbstractHdfsAuditLogApplication.java
@@ -62,7 +62,7 @@ public abstract class AbstractHdfsAuditLogApplication extends StormApplication {
         builder.setSpout("ingest", spout, numOfSpoutTasks);
 
 
-        HdfsAuditLogParserBolt parserBolt = new HdfsAuditLogParserBolt();
+        BaseRichBolt parserBolt = getParserBolt();
         BoltDeclarer boltDeclarer = builder.setBolt("parserBolt", parserBolt, numOfParserTasks);
 
         Boolean useDefaultPartition = !config.hasPath("eagleProps.useDefaultPartition") || config.getBoolean("eagleProps.useDefaultPartition");

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProvider.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProvider.java b/eagle-security/eagle-security-maprfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProvider.java
new file mode 100644
index 0000000..3532378
--- /dev/null
+++ b/eagle-security/eagle-security-maprfs-auditlog/src/main/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProvider.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
+ *
+ *    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.auditlog;
+
+
+import org.apache.eagle.app.spi.AbstractApplicationProvider;
+
+public class MapRFSAuditLogAppProvider extends AbstractApplicationProvider<MapRFSAuditLogApplication> {
+    @Override
+    public MapRFSAuditLogApplication getApplication() {
+        return new MapRFSAuditLogApplication();
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.MapRFSAuditLogAppProvider.xml
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.MapRFSAuditLogAppProvider.xml b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.MapRFSAuditLogAppProvider.xml
new file mode 100644
index 0000000..c54dd28
--- /dev/null
+++ b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/providers/org.apache.eagle.security.auditlog.MapRFSAuditLogAppProvider.xml
@@ -0,0 +1,246 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ /*
+  ~  *
+  ~  *  * 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.
+  ~  *
+  ~  */
+  -->
+
+<application>
+    <type>MapRFSAuditLogApplication</type>
+    <name>MapRFS Audit Log Monitoring Application</name>
+    <version>0.5.0-incubating</version>
+    <appClass>org.apache.eagle.security.auditlog.MapRFSAuditLogApplication</appClass>
+    <viewPath>/apps/example</viewPath>
+    <configuration>
+        <!-- topology related configurations -->
+        <property>
+            <name>topology.numOfSpoutTasks</name>
+            <displayName>topology.numOfSpoutTasks</displayName>
+            <value>2</value>
+            <description>number of spout tasks</description>
+        </property>
+        <property>
+            <name>topology.numOfParserTasks</name>
+            <displayName>topology.numOfParserTasks</displayName>
+            <value>2</value>
+            <description>number of parser tasks</description>
+        </property>
+        <property>
+            <name>topology.numOfJoinTasks</name>
+            <displayName>topology.numOfJoinTasks</displayName>
+            <value>2</value>
+            <description>number of external join tasks</description>
+        </property>
+        <property>
+            <name>topology.numOfSinkTasks</name>
+            <displayName>topology.numOfSinkTasks</displayName>
+            <value>2</value>
+            <description>number of sink tasks</description>
+        </property>
+
+        <!-- data source configurations -->
+        <property>
+            <name>dataSourceConfig.topic</name>
+            <displayName>dataSourceConfig.topic</displayName>
+            <value>maprfs_audit_log</value>
+            <description>data source topic</description>
+        </property>
+        <property>
+            <name>dataSourceConfig.zkConnection</name>
+            <displayName>dataSourceConfig.zkConnection</displayName>
+            <value>server.eagle.apache.org</value>
+            <description>zk connection</description>
+        </property>
+        <property>
+            <name>dataSourceConfig.txZkServers</name>
+            <displayName>dataSourceConfig.txZkServers</displayName>
+            <value>server.eagle.apache.org:5181</value>
+            <description>zookeeper server for offset transaction</description>
+        </property>
+        <property>
+            <name>dataSourceConfig.schemeCls</name>
+            <displayName>dataSourceConfig.schemeCls</displayName>
+            <value>storm.kafka.StringScheme</value>
+            <description>scheme class</description>
+        </property>
+
+        <!-- data enrich configurations -->
+        <property>
+            <name>dataEnrich.dataJoinPollIntervalSec</name>
+            <displayName>eagleProps.dataJoinPollIntervalSec</displayName>
+            <value>30</value>
+            <description>interval in seconds for polling</description>
+        </property>
+
+        <!-- eagle service configurations -->
+        <property>
+            <name>eagleService.host</name>
+            <displayName>eagleService.host</displayName>
+            <value>localhost</value>
+            <description>eagle service host</description>
+        </property>
+        <property>
+            <name>eagleService.port</name>
+            <displayName>eagleService.port</displayName>
+            <value>9090</value>
+            <description>eagle service port</description>
+        </property>
+        <property>
+            <name>eagleService.username</name>
+            <displayName>eagleService.username</displayName>
+            <value>admin</value>
+            <description>eagle service username</description>
+        </property>
+        <property>
+            <name>eagleService.password</name>
+            <displayName>eagleService.password</displayName>
+            <value>secret</value>
+            <description>eagle service password</description>
+        </property>
+
+        <!-- data sink configurations -->
+        <property>
+            <name>dataSinkConfig.topic</name>
+            <displayName>dataSinkConfig.topic</displayName>
+            <value>maprfs_audit_log_enriched</value>
+            <description>topic for kafka data sink</description>
+        </property>
+        <property>
+            <name>dataSinkConfig.brokerList</name>
+            <displayName>dataSinkConfig.brokerList</displayName>
+            <value>server.eagle.apache.org:9092</value>
+            <description>kafka broker list</description>
+        </property>
+        <property>
+            <name>dataSinkConfig.serializerClass</name>
+            <displayName>dataSinkConfig.serializerClass</displayName>
+            <value>kafka.serializer.StringEncoder</value>
+            <description>serializer class Kafka message value</description>
+        </property>
+        <property>
+            <name>dataSinkConfig.keySerializerClass</name>
+            <displayName>dataSinkConfig.keySerializerClass</displayName>
+            <value>kafka.serializer.StringEncoder</value>
+            <description>serializer class Kafka message key</description>
+        </property>
+
+        <!-- web app related configurations -->
+        <property>
+            <name>fs.defaultFS</name>
+            <displayName>fs.defaultFS</displayName>
+            <value>hdfs://server.eagle.apache.org:7222</value>
+            <description>hdfs endpoint</description>
+        </property>
+        <property>
+            <name>mapr.webhttps</name>
+            <displayName>mapr.webhttps</displayName>
+            <value>https://server.eagle.apache.org:8443</value>
+            <description>mapr web service</description>
+        </property>
+        <property>
+            <name>mapr.username</name>
+            <displayName>mapr.username</displayName>
+            <value>mapr</value>
+            <description>mapr user name to login we UI</description>
+        </property>
+        <property>
+            <name>mapr.password</name>
+            <displayName>mapr.password</displayName>
+            <value>mapr</value>
+            <description>mapr user password to login we UI</description>
+        </property>
+    </configuration>
+    <streams>
+        <stream>
+            <streamId>maprfs_audit_log_enriched_stream</streamId>
+            <description>maprfs Audit Log Enriched Stream</description>
+            <validate>true</validate>
+            <timeseries>true</timeseries>
+            <columns>
+                <column>
+                    <name>src</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>dst</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>host</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>timestamp</name>
+                    <type>long</type>
+                </column>
+                <column>
+                    <name>status</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>user</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>cmd</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>sensitivityType</name>
+                    <type>string</type>
+                </column>
+                <column>
+                    <name>volume</name>
+                    <type>string</type>
+                </column>
+            </columns>
+        </stream>
+    </streams>
+    <docs>
+        <install>
+# Step 1: Create source kafka topic named "${site}_example_source_topic"
+
+./bin/kafka-topics.sh --create --topic example_source_topic --replication-factor 1 --replication 1
+
+# Step 2: Set up data collector to flow data into kafka topic in
+
+./bin/logstash -f log_collector.conf
+
+## `log_collector.conf` sample as following:
+
+input {
+
+}
+filter {
+
+}
+output{
+
+}
+
+# Step 3: start application
+
+# Step 4: monitor with featured portal or alert with policies
+        </install>
+        <uninstall>
+# Step 1: stop and uninstall application
+# Step 2: delete kafka topic named "${site}_example_source_topic"
+# Step 3: stop logstash
+        </uninstall>
+    </docs>
+</application>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
new file mode 100644
index 0000000..8ff0115
--- /dev/null
+++ b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/META-INF/services/org.apache.eagle.app.spi.ApplicationProvider
@@ -0,0 +1,37 @@
+#
+# /*
+#  *
+#  *  * 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.
+#  *
+#  */
+#
+
+# 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.
+
+org.apache.eagle.security.auditlog.MapRFSAuditLogAppProvider

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/application.conf b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/application.conf
index 52a31a5..5327c74 100644
--- a/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/application.conf
+++ b/eagle-security/eagle-security-maprfs-auditlog/src/main/resources/application.conf
@@ -14,56 +14,33 @@
 # limitations under the License.
 
 {
-  "envContextConfig" : {
-    "env" : "storm",
-    "mode" : "local",
-    "topologyName" : "maprFSAuditLogProcessTopology",
-    "stormConfigFile" : "security-auditlog-storm.yaml",
-    "parallelismConfig" : {
-      "kafkaMsgConsumer" : 1,
-      "maprFSAuditLogAlertExecutor*" : 1
-    }
+  "appId" : "maprFSAuditLog",
+  "mode" : "LOCAL",
+  "siteId" : "testsite",
+  "topology" : {
+    "numOfTotalWorkers" : 2,
+    "numOfSpoutTasks" : 2,
+    "numOfParserTasks" : 2,
+    "numOfSensitivityJoinTasks" : 2,
+    "numOfIPZoneJoinTasks" : 2,
+    "numOfSinkTasks" : 2
   },
   "dataSourceConfig": {
     "topic" : "maprtest",
-    "zkConnection" : "192.168.6.148:5181",
-    "zkConnectionTimeoutMS" : 15000,
-    "consumerGroupId" : "EagleConsumer",
-    "fetchSize" : 1048586,
-    "deserializerClass" : "org.apache.eagle.security.auditlog.MapRFSAuditLogKafkaDeserializer",
-    "transactionZKServers" : "192.168.6.148",
-    "transactionZKPort" : 5181,
-    "transactionZKRoot" : "/consumers",
-    "transactionStateUpdateMS" : 2000
+    "zkConnection" : "server.eagle.apache.org:5181",
+    "txZkServers" : "server.eagle.apache.org:5181",
+    "schemeCls" : "storm.kafka.StringScheme"
   },
-  "alertExecutorConfigs" : {
-     "maprFSAuditLogAlertExecutor" : {
-       "parallelism" : 1,
-       "partitioner" : "org.apache.eagle.policy.DefaultPolicyPartitioner"
-       "needValidation" : "true"
-     }
+  "eagleService": {
+    "host": "localhost",
+    "port": 9090,
+    "username": "admin",
+    "password": "secret"
   },
-  "eagleProps" : {
-    "site" : "sandbox",
-    "application": "maprFSAuditLog",
-  	"dataJoinPollIntervalSec" : 30,
-    "mailHost" : "mailHost.com",
-    "mailSmtpPort":"25",
-    "mailDebug" : "true",
-    "balancePartitionEnabled" : false,
-    #"partitionRefreshIntervalInMin" : 60,
-    #"kafkaStatisticRangeInMin" : 60,
-    "eagleService": {
-      "host": "localhost",
-      "port": 9099,
-      "username": "admin",
-      "password": "secret"
-    },
-    "readHdfsUserCommandPatternFrom" : "file"
-  },
-  "dynamicConfigSource" : {
-  	"enabled" : true,
-  	"initDelayMillis" : 0,
-  	"delayMillis" : 30000
+  "dataSinkConfig": {
+    "topic" : "maprfs_audit_log_parsed",
+    "brokerList" : "server.eagle.apache.org:9092",
+    "serializerClass" : "kafka.serializer.StringEncoder",
+    "keySerializerClass" : "kafka.serializer.StringEncoder"
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-auditlog/src/test/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProviderTest.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-auditlog/src/test/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProviderTest.java b/eagle-security/eagle-security-maprfs-auditlog/src/test/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProviderTest.java
new file mode 100644
index 0000000..5f0d903
--- /dev/null
+++ b/eagle-security/eagle-security-maprfs-auditlog/src/test/java/org/apache/eagle/security/auditlog/MapRFSAuditLogAppProviderTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.security.auditlog;
+
+import com.google.inject.Inject;
+import org.apache.eagle.app.test.AppJUnitRunner;
+import org.apache.eagle.app.test.ApplicationSimulator;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(AppJUnitRunner.class)
+public class MapRFSAuditLogAppProviderTest {
+    @Inject
+    private
+    ApplicationSimulator simulator;
+
+    @Test
+    public void testStartAsManagedApplication(){
+        simulator.start(MapRFSAuditLogAppProvider.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSCommandResolver.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSCommandResolver.java b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSCommandResolver.java
index edb0737..ce1b132 100644
--- a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSCommandResolver.java
+++ b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSCommandResolver.java
@@ -16,6 +16,8 @@
  */
 package org.apache.eagle.service.security.hdfs.resolver;
 
+import com.typesafe.config.Config;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.service.alert.resolver.AttributeResolvable;
 import org.apache.eagle.service.alert.resolver.AttributeResolveException;
 import org.apache.eagle.service.alert.resolver.BadAttributeResolveRequestException;
@@ -41,6 +43,10 @@ public class MAPRFSCommandResolver implements AttributeResolvable<GenericAttribu
 
     private final static List<String> commands = Arrays.asList(cmdStrs);
 
+    public MAPRFSCommandResolver(ApplicationEntityService entityService, Config eagleServerConfig){
+
+    }
+
     public List<String> resolve(GenericAttributeResolveRequest request) throws AttributeResolveException {
         String query = request.getQuery().trim();
         List<String> res = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSResourceResolver.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSResourceResolver.java b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSResourceResolver.java
index 243e2fa..d0f0116 100644
--- a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSResourceResolver.java
+++ b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSResourceResolver.java
@@ -16,7 +16,10 @@
  */
 package org.apache.eagle.service.security.hdfs.resolver;
 
+import com.google.inject.Inject;
 import com.typesafe.config.Config;
+import org.apache.eagle.metadata.model.ApplicationEntity;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.security.resolver.MetadataAccessConfigRepo;
 import org.apache.eagle.service.alert.resolver.AttributeResolvable;
 import org.apache.eagle.service.alert.resolver.AttributeResolveException;
@@ -24,6 +27,8 @@ import org.apache.eagle.service.alert.resolver.BadAttributeResolveRequestExcepti
 import org.apache.eagle.service.alert.resolver.GenericAttributeResolveRequest;
 import org.apache.eagle.service.security.hdfs.HDFSFileSystem;
 import org.apache.eagle.service.security.hdfs.MAPRFSResourceConstants;
+import org.apache.eagle.service.security.hdfs.rest.HDFSResourceWebResource;
+import org.apache.eagle.service.security.hdfs.rest.MAPRFSResourceWebResource;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.slf4j.Logger;
@@ -31,6 +36,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -43,6 +49,14 @@ import java.util.regex.Pattern;
  */
 public class MAPRFSResourceResolver implements AttributeResolvable<GenericAttributeResolveRequest,String> {
     private final static Logger LOG = LoggerFactory.getLogger(MAPRFSResourceResolver.class);
+
+    private ApplicationEntityService entityService;
+
+    @Inject
+    public MAPRFSResourceResolver(ApplicationEntityService entityService, Config eagleServerConfig){
+        this.entityService = entityService;
+    }
+
     /**
      * MAPRFS Resource Resolve API
      *
@@ -52,10 +66,9 @@ public class MAPRFSResourceResolver implements AttributeResolvable<GenericAttrib
     public List<String> resolve(GenericAttributeResolveRequest request)
             throws AttributeResolveException {
         List<String> result = new ArrayList<>();
-        MetadataAccessConfigRepo repo = new MetadataAccessConfigRepo();
         try {
-            Config config = repo.getConfig(MAPRFSResourceConstants.MAPRFS_APPLICATION, request.getSite().trim());
-            Configuration conf = repo.convert(config);
+            Map<String, Object> config = getAppConfig(request.getSite(), MAPRFSResourceWebResource.MAPRFS_APPLICATION);
+            Configuration conf = convert(config);
             HDFSFileSystem fileSystem = new HDFSFileSystem(conf);
             String query = request.getQuery().trim();
             List<FileStatus> fileStatuses = null;
@@ -84,6 +97,20 @@ public class MAPRFSResourceResolver implements AttributeResolvable<GenericAttrib
         }
     }
 
+    private Map<String, Object> getAppConfig(String site, String appType){
+        ApplicationEntity entity = entityService.getBySiteIdAndAppType(site, appType);
+        return entity.getConfiguration();
+    }
+
+    private Configuration convert(Map<String, Object> originalConfig) throws Exception {
+        Configuration config = new Configuration();
+        for (Map.Entry<String, Object> entry : originalConfig.entrySet()) {
+            config.set(entry.getKey().toString(), entry.getValue().toString());
+        }
+        return config;
+    }
+
+
     /**
      * Validate the Passed Request Object
      * It should have Site Id and File Path

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSVolumeResolver.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSVolumeResolver.java b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSVolumeResolver.java
index 3b7fb03..28cb151 100644
--- a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSVolumeResolver.java
+++ b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRFSVolumeResolver.java
@@ -16,7 +16,10 @@
  */
 package org.apache.eagle.service.security.hdfs.resolver;
 
+import com.google.inject.Inject;
 import com.typesafe.config.Config;
+import org.apache.eagle.metadata.model.ApplicationEntity;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.security.resolver.MetadataAccessConfigRepo;
 import org.apache.eagle.service.alert.resolver.AttributeResolvable;
 import org.apache.eagle.service.alert.resolver.AttributeResolveException;
@@ -24,6 +27,7 @@ import org.apache.eagle.service.alert.resolver.BadAttributeResolveRequestExcepti
 import org.apache.eagle.service.alert.resolver.GenericAttributeResolveRequest;
 import org.apache.eagle.service.security.hdfs.MAPRFSResourceConstants;
 import org.apache.eagle.service.security.hdfs.rest.HttpRequest;
+import org.apache.eagle.service.security.hdfs.rest.MAPRFSResourceWebResource;
 import org.apache.hadoop.conf.Configuration;
 import org.json.JSONArray;
 import org.json.JSONObject;
@@ -32,28 +36,34 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Pattern;
 
 public class MAPRFSVolumeResolver implements AttributeResolvable<GenericAttributeResolveRequest,String> {
 
     private final static Logger LOG = LoggerFactory.getLogger(MAPRFSVolumeResolver.class);
 
+    private ApplicationEntityService entityService;
+    @Inject
+    public MAPRFSVolumeResolver(ApplicationEntityService entityService, Config eagleServerConfig){
+        this.entityService = entityService;
+    }
+
     @Override
     public List<String> resolve(GenericAttributeResolveRequest request) throws AttributeResolveException {
         String query = request.getQuery().trim();
         List<String> volumelist = new ArrayList<>();
-        MetadataAccessConfigRepo repo = new MetadataAccessConfigRepo();
         //Call MAPR REST API to get volumes
         try {
-            Config config = repo.getConfig(MAPRFSResourceConstants.MAPRFS_APPLICATION, request.getSite().trim());
-            Configuration conf = repo.convert(config);
+            Map<String, Object> config = getAppConfig(request.getSite(), MAPRFSResourceWebResource.MAPRFS_APPLICATION);
+
             //get user name, password, make request to mapr rest service
             String username;
             String password;
-            username = config.getString(MAPRFSResourceConstants.MAPRFS_USERNAME);
-            password = config.getString(MAPRFSResourceConstants.MAPRFS_PASSWORD);
+            username = (String) config.get(MAPRFSResourceConstants.MAPRFS_USERNAME);
+            password = (String) config.get(MAPRFSResourceConstants.MAPRFS_PASSWORD);
             //constuct url to query mapr volume
-            String restUrl = config.getString(MAPRFSResourceConstants.MAPRFS_WEBUI_HTTPS) + MAPRFSResourceConstants.MAPRFS_LIST_VOLUME;
+            String restUrl = (String) config.get(MAPRFSResourceConstants.MAPRFS_WEBUI_HTTPS) + MAPRFSResourceConstants.MAPRFS_LIST_VOLUME;
 
             JSONObject response = HttpRequest.executeGet(restUrl,username,password);
             volumelist = extractVolumeList(response);
@@ -74,6 +84,19 @@ public class MAPRFSVolumeResolver implements AttributeResolvable<GenericAttribut
         }
     }
 
+    private Map<String, Object> getAppConfig(String site, String appType){
+        ApplicationEntity entity = entityService.getBySiteIdAndAppType(site, appType);
+        return entity.getConfiguration();
+    }
+
+    private Configuration convert(Map<String, Object> originalConfig) throws Exception {
+        Configuration config = new Configuration();
+        for (Map.Entry<String, Object> entry : originalConfig.entrySet()) {
+            config.set(entry.getKey().toString(), entry.getValue().toString());
+        }
+        return config;
+    }
+
     @Override
     public void validateRequest(GenericAttributeResolveRequest request) throws BadAttributeResolveRequestException {
         ;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRStatusCodeResolver.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRStatusCodeResolver.java b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRStatusCodeResolver.java
index db1511d..20c94f3 100644
--- a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRStatusCodeResolver.java
+++ b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/resolver/MAPRStatusCodeResolver.java
@@ -17,6 +17,9 @@
 
 package org.apache.eagle.service.security.hdfs.resolver;
 
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.service.alert.resolver.AttributeResolvable;
 import org.apache.eagle.service.alert.resolver.AttributeResolveException;
 import org.apache.eagle.service.alert.resolver.BadAttributeResolveRequestException;
@@ -49,13 +52,17 @@ public class MAPRStatusCodeResolver implements AttributeResolvable<GenericAttrib
 
     private final static List<String> statusList = Arrays.asList(statusCodes);
 
-    public MAPRStatusCodeResolver () {
+
+    @Inject
+    public MAPRStatusCodeResolver(){
         //construct hashmap for status code query
         for(int i = 0; i < statusCodes.length; i++){
             statusCodeMap.put(statusCodes[i],String.valueOf(i));
         }
+
     }
 
+
     //conver human readable status code to id
     public String getStatusCodeID(String code){
         String id = "STATUS CODE ID NOT FOUND";

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MAPRFSResourceWebResource.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MAPRFSResourceWebResource.java b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MAPRFSResourceWebResource.java
index 783d3e9..24a07c0 100644
--- a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MAPRFSResourceWebResource.java
+++ b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MAPRFSResourceWebResource.java
@@ -17,9 +17,14 @@
 package org.apache.eagle.service.security.hdfs.rest;
 
 
+import com.google.inject.Inject;
 import com.typesafe.config.Config;
+import org.apache.eagle.metadata.model.ApplicationEntity;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.security.entity.FileStatusEntity;
 import org.apache.eagle.security.resolver.MetadataAccessConfigRepo;
+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.hdfs.HDFSFileSystem;
 import org.apache.eagle.service.security.hdfs.HDFSResourceSensitivityDataJoiner;
@@ -33,6 +38,7 @@ import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 /**
  * REST Web Service to browse files and Paths in MAPRFS
@@ -41,6 +47,15 @@ import java.util.List;
 public class MAPRFSResourceWebResource
 {
     private static Logger LOG = LoggerFactory.getLogger(MAPRFSResourceWebResource.class);
+    final public static String MAPRFS_APPLICATION = "maprFSAuditLog";
+    private ApplicationEntityService entityService;
+    private ISecurityMetadataDAO dao;
+
+    @Inject
+    public MAPRFSResourceWebResource(ApplicationEntityService entityService, Config eagleServerConfig){
+        this.entityService = entityService;
+        dao = MetadataDaoFactory.getMetadataDAO(eagleServerConfig);
+    }
 
     @GET
     @Consumes(MediaType.APPLICATION_JSON)
@@ -50,17 +65,16 @@ public class MAPRFSResourceWebResource
         LOG.info("Starting MAPRFS Resource Browsing.  Query Parameters ==> Site :"+site+"  Path : "+filePath );
         HDFSResourceWebResponse response = new HDFSResourceWebResponse();
         HDFSResourceWebRequestValidator validator = new HDFSResourceWebRequestValidator();
-        MetadataAccessConfigRepo repo = new MetadataAccessConfigRepo();
         List<FileStatusEntity> result = new ArrayList<>();
         List<FileStatus> fileStatuses = null;
         try {
             validator.validate(site, filePath); // First Step would be validating Request
-            Config config = repo.getConfig(MAPRFSResourceConstants.MAPRFS_APPLICATION, site);
-            Configuration conf = repo.convert(config);
+            Map<String, Object> config = getAppConfig(site, MAPRFS_APPLICATION);
+            Configuration conf = convert(config);
             HDFSFileSystem fileSystem = new HDFSFileSystem(conf);
             fileStatuses = fileSystem.browse(filePath);
             // Join with File Sensitivity Info
-            HDFSResourceSensitivityDataJoiner joiner = new HDFSResourceSensitivityDataJoiner(null);
+            HDFSResourceSensitivityDataJoiner joiner = new HDFSResourceSensitivityDataJoiner(dao);
             result = joiner.joinFileSensitivity(site, fileStatuses);
             LOG.info("Successfully browsed files in MAPRFS .");
         } catch( Exception ex ) {
@@ -70,4 +84,17 @@ public class MAPRFSResourceWebResource
         response.setObj(result);
         return response;
     }
+
+    private Map<String, Object> getAppConfig(String site, String appType){
+        ApplicationEntity entity = entityService.getBySiteIdAndAppType(site, appType);
+        return entity.getConfiguration();
+    }
+
+    private Configuration convert(Map<String, Object> originalConfig) throws Exception {
+        Configuration config = new Configuration();
+        for (Map.Entry<String, Object> entry : originalConfig.entrySet()) {
+            config.set(entry.getKey().toString(), entry.getValue().toString());
+        }
+        return config;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolver.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolver.java b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolver.java
index c62c0dc..b8e7a20 100644
--- a/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolver.java
+++ b/eagle-security/eagle-security-maprfs-web/src/main/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolver.java
@@ -15,10 +15,15 @@
  * limitations under the License.
  */
 package org.apache.eagle.service.security.hdfs.rest;
+import com.google.inject.Inject;
 import com.mapr.fs.clicommands.MapRCliCommands;
 import com.typesafe.config.Config;
 import org.apache.commons.io.output.ByteArrayOutputStream;
+import org.apache.eagle.metadata.model.ApplicationEntity;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
 import org.apache.eagle.security.resolver.MetadataAccessConfigRepo;
+import org.apache.eagle.security.service.ISecurityMetadataDAO;
+import org.apache.eagle.security.service.MetadataDaoFactory;
 import org.apache.eagle.service.security.hdfs.MAPRFSResourceConstants;
 import org.apache.eagle.service.security.hdfs.resolver.MAPRStatusCodeResolver;
 import org.apache.hadoop.conf.Configuration;
@@ -30,9 +35,12 @@ import org.slf4j.LoggerFactory;
 import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
 import java.io.PrintStream;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.apache.eagle.service.security.hdfs.MAPRFSResourceConstants.MAPRFS_APPLICATION;
+
 
 @Path(MAPRFSResourceConstants.MAPRFS_NAME_RESOLVER)
 public class MapRNameResolver {
@@ -63,7 +71,12 @@ public class MapRNameResolver {
 
     private MapRCliCommands mprcmd;
 
+    private ApplicationEntityService entityService;
 
+    @Inject
+    public MapRNameResolver(ApplicationEntityService entityService, Config eagleServerConfig){
+        this.entityService = entityService;
+    }
     /**
      * rest api : convert file/folder name to id
      * @param fName file/folder name
@@ -76,10 +89,9 @@ public class MapRNameResolver {
     public String getFid(@QueryParam("fName") String fName, @QueryParam("site") String site) {
         String ans = "FID NOT FOUND";
         try{
-            MetadataAccessConfigRepo repo = new MetadataAccessConfigRepo();
-            Config typeSafeConfig= repo.getConfig(MAPRFSResourceConstants.MAPRFS_APPLICATION, site);
+            Map<String, Object> config = getAppConfig(site, MAPRFS_APPLICATION);
             String defaultFS;
-            defaultFS = typeSafeConfig.getString("fs.defaultFS");
+            defaultFS = (String) config.get("fs.defaultFS");
             Configuration conf = new Configuration();
             conf.set("fs.defaultFS",defaultFS);
             mprcmd = new MapRCliCommands(conf);
@@ -130,14 +142,13 @@ public class MapRNameResolver {
         String ans= "VOLUME ID NOT FOUDN";
         try{
             // call mapr rest api to get corresponding id;
-            MetadataAccessConfigRepo repo = new MetadataAccessConfigRepo();
-            Config typeSafeConfig= repo.getConfig(MAPRFSResourceConstants.MAPRFS_APPLICATION, site);
+            Map<String, Object> config = getAppConfig(site, MAPRFS_APPLICATION);
             String username;
             String password;
-            username = typeSafeConfig.getString(MAPRFSResourceConstants.MAPRFS_USERNAME);
-            password = typeSafeConfig.getString(MAPRFSResourceConstants.MAPRFS_PASSWORD);
+            username = (String) config.get(MAPRFSResourceConstants.MAPRFS_USERNAME);
+            password = (String) config.get(MAPRFSResourceConstants.MAPRFS_PASSWORD);
             // call
-            String restUrl = typeSafeConfig.getString(MAPRFSResourceConstants.MAPRFS_WEBUI_HTTPS) + MAPRFSResourceConstants.MAPRFS_VOLUME_INFO + "?name=" + vName;
+            String restUrl = (String) config.get(MAPRFSResourceConstants.MAPRFS_WEBUI_HTTPS) + MAPRFSResourceConstants.MAPRFS_VOLUME_INFO + "?name=" + vName;
 
             JSONObject response = HttpRequest.executeGet(restUrl,username,password);
             ans = extractVolumeId(response);
@@ -175,6 +186,10 @@ public class MapRNameResolver {
     }
 
 
+    private Map<String, Object> getAppConfig(String site, String appType){
+        ApplicationEntity entity = entityService.getBySiteIdAndAppType(site, appType);
+        return entity.getConfiguration();
+    }
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java b/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java
deleted file mode 100644
index 990085c..0000000
--- a/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java
+++ /dev/null
@@ -1,30 +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.service.security.hdfs.rest;
-
-import org.json.JSONObject;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class HttpRequestTest {
-    @Test
-    public void TestExecuteGet() throws Exception {
-        String restUrl = "https://sandbox.mapr.com:8443/rest/volume/list";
-        JSONObject res = HttpRequest.executeGet(restUrl,"username", "password");
-        Assert.assertEquals("OK",res.getString("status"));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java b/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java
deleted file mode 100644
index f3a6abc..0000000
--- a/eagle-security/eagle-security-maprfs-web/src/main/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-package org.apache.eagle.service.security.hdfs.rest;/*
- * 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.
- */
-
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class MapRNameResolverTest {
-    @Test
-    public void testResolver () {
-        MapRNameResolver resolver = new MapRNameResolver();
-        Assert.assertEquals("2049.2692.136572", resolver.getFid("/test0","mapr"));
-        Assert.assertEquals("0", resolver.getSid("success", "mapr"));
-        Assert.assertEquals("mapr.cluster.root","186635570");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java b/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.java
new file mode 100644
index 0000000..c639290
--- /dev/null
+++ b/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/HttpRequestTest.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
+ *
+ *    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.service.security.hdfs.rest;
+
+import org.json.JSONObject;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class HttpRequestTest {
+    @Test
+    public void TestExecuteGet() throws Exception {
+//        String restUrl = "https://sandbox.mapr.com:8443/rest/volume/list";
+//        JSONObject res = HttpRequest.executeGet(restUrl,"username", "password");
+//        Assert.assertEquals("OK",res.getString("status"));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/2958bc8b/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java
----------------------------------------------------------------------
diff --git a/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java b/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java
new file mode 100644
index 0000000..726926b
--- /dev/null
+++ b/eagle-security/eagle-security-maprfs-web/src/test/java/org/apache/eagle/service/security/hdfs/rest/MapRNameResolverTest.java
@@ -0,0 +1,30 @@
+package org.apache.eagle.service.security.hdfs.rest;/*
+ * 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.
+ */
+
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MapRNameResolverTest {
+    @Test
+    public void testResolver () {
+//        MapRNameResolver resolver = new MapRNameResolver(null, null);
+//        Assert.assertEquals("2049.2692.136572", resolver.getFid("/test0","mapr"));
+//        Assert.assertEquals("0", resolver.getSid("success", "mapr"));
+//        Assert.assertEquals("mapr.cluster.root","186635570");
+    }
+}
\ No newline at end of file