You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by al...@apache.org on 2015/10/10 01:26:03 UTC

incubator-ranger git commit: RANGER-682 Add support for audit to Azure Blob Storage via HDFS audit handler

Repository: incubator-ranger
Updated Branches:
  refs/heads/master 140f7efb8 -> 35aa706ec


RANGER-682 Add support for audit to Azure Blob Storage via HDFS audit handler


Project: http://git-wip-us.apache.org/repos/asf/incubator-ranger/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ranger/commit/35aa706e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ranger/tree/35aa706e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ranger/diff/35aa706e

Branch: refs/heads/master
Commit: 35aa706ecbdd86b25a13f20e79248e59060935b9
Parents: 140f7ef
Author: Alok Lal <al...@apache.org>
Authored: Mon Sep 21 12:06:20 2015 -0700
Committer: Alok Lal <al...@apache.org>
Committed: Fri Oct 9 16:19:54 2015 -0700

----------------------------------------------------------------------
 .../audit/destination/HDFSAuditDestination.java | 37 +++++++++++----
 .../ranger/audit/provider/BaseAuditHandler.java | 19 ++++++--
 .../audit/provider/hdfs/HdfsAuditProvider.java  |  3 ++
 .../audit/provider/hdfs/HdfsLogDestination.java | 26 ++++++++++-
 .../ranger/utils/install/XmlConfigChanger.java  | 47 ++++++++++++++++----
 hbase-agent/conf/ranger-hbase-audit-changes.cfg |  5 +++
 hbase-agent/conf/ranger-hbase-audit.xml         | 40 ++++++++++++++++-
 hbase-agent/scripts/install.properties          | 11 +++++
 hdfs-agent/conf/ranger-hdfs-audit-changes.cfg   |  5 +++
 hdfs-agent/conf/ranger-hdfs-audit.xml           | 36 +++++++++++++++
 hdfs-agent/scripts/install.properties           | 11 +++++
 hive-agent/conf/ranger-hive-audit-changes.cfg   |  5 +++
 hive-agent/conf/ranger-hive-audit.xml           | 36 +++++++++++++++
 hive-agent/scripts/install.properties           | 11 +++++
 kms/scripts/install.properties                  | 11 +++++
 knox-agent/conf/ranger-knox-audit-changes.cfg   |  5 +++
 knox-agent/conf/ranger-knox-audit.xml           | 36 +++++++++++++++
 knox-agent/scripts/install.properties           | 11 +++++
 .../conf/ranger-kafka-audit-changes.cfg         |  5 +++
 plugin-kafka/conf/ranger-kafka-audit.xml        | 36 +++++++++++++++
 plugin-kafka/scripts/install.properties         | 11 +++++
 plugin-kms/conf/ranger-kms-audit-changes.cfg    |  5 +++
 plugin-kms/conf/ranger-kms-audit.xml            | 36 +++++++++++++++
 plugin-solr/conf/ranger-solr-audit-changes.cfg  |  5 +++
 plugin-solr/conf/ranger-solr-audit.xml          | 36 +++++++++++++++
 plugin-solr/scripts/install.properties          | 11 +++++
 plugin-yarn/conf/ranger-yarn-audit-changes.cfg  |  5 +++
 plugin-yarn/conf/ranger-yarn-audit.xml          | 36 +++++++++++++++
 plugin-yarn/scripts/install.properties          | 11 +++++
 storm-agent/conf/ranger-storm-audit-changes.cfg |  5 +++
 storm-agent/conf/ranger-storm-audit.xml         | 36 +++++++++++++++
 storm-agent/scripts/install.properties          | 11 +++++
 32 files changed, 580 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/agents-audit/src/main/java/org/apache/ranger/audit/destination/HDFSAuditDestination.java
----------------------------------------------------------------------
diff --git a/agents-audit/src/main/java/org/apache/ranger/audit/destination/HDFSAuditDestination.java b/agents-audit/src/main/java/org/apache/ranger/audit/destination/HDFSAuditDestination.java
index 49e5fbb..96755be 100644
--- a/agents-audit/src/main/java/org/apache/ranger/audit/destination/HDFSAuditDestination.java
+++ b/agents-audit/src/main/java/org/apache/ranger/audit/destination/HDFSAuditDestination.java
@@ -23,12 +23,9 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.Properties;
+import java.util.*;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -99,7 +96,7 @@ public class HDFSAuditDestination extends AuditDestination {
 		logger.info("logFolder=" + logFolder + ", destName=" + getName());
 		logger.info("logFileNameFormat=" + logFileNameFormat + ", destName="
 				+ getName());
-
+		logger.info("config=" + configProps.toString());
 		initDone = true;
 	}
 
@@ -127,7 +124,15 @@ public class HDFSAuditDestination extends AuditDestination {
 			for (String event : events) {
 				out.println(event);
 			}
-			out.flush();
+			// flush and check the stream for errors
+			if (out.checkError()) {
+				// In theory, this count may NOT be accurate as part of the messages may have been successfully written.
+				// However, in practice, since client does buffering, either all of none would succeed.
+				addDeferredCount(events.size());
+				out.close();
+				logWriter = null;
+				return false;
+			}
 		} catch (Throwable t) {
 			addDeferredCount(events.size());
 			logError("Error writing to log file.", t);
@@ -207,7 +212,7 @@ public class HDFSAuditDestination extends AuditDestination {
 					currentTime.getTime());
 			String parentFolder = MiscUtil.replaceTokens(logFolder,
 					currentTime.getTime());
-			Configuration conf = new Configuration();
+			Configuration conf = createConfiguration();
 
 			String fullPath = parentFolder
 					+ org.apache.hadoop.fs.Path.SEPARATOR + fileName;
@@ -243,6 +248,22 @@ public class HDFSAuditDestination extends AuditDestination {
 		return logWriter;
 	}
 
+	Configuration createConfiguration() {
+		Configuration conf = new Configuration();
+		for (Map.Entry<String, String> entry : configProps.entrySet()) {
+			String key = entry.getKey();
+			String value = entry.getValue();
+			// for ease of install config file may contain properties with empty value, skip those
+			if (StringUtils.isNotEmpty(value)) {
+				conf.set(key, value);
+			}
+			logger.info("Adding property to HDFS config: " + key + " => " + value);
+		}
+
+		logger.info("Returning HDFS Filesystem Config: " + conf.toString());
+		return conf;
+	}
+
 	private void createParents(Path pathLogfile, FileSystem fileSystem)
 			throws Throwable {
 		logger.info("Creating parent folder for " + pathLogfile);

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/agents-audit/src/main/java/org/apache/ranger/audit/provider/BaseAuditHandler.java
----------------------------------------------------------------------
diff --git a/agents-audit/src/main/java/org/apache/ranger/audit/provider/BaseAuditHandler.java b/agents-audit/src/main/java/org/apache/ranger/audit/provider/BaseAuditHandler.java
index 30db18b..6717c92 100644
--- a/agents-audit/src/main/java/org/apache/ranger/audit/provider/BaseAuditHandler.java
+++ b/agents-audit/src/main/java/org/apache/ranger/audit/provider/BaseAuditHandler.java
@@ -25,17 +25,15 @@ import org.apache.ranger.audit.model.AuthzAuditEvent;
 
 import com.google.gson.GsonBuilder;
 
+import java.util.*;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Properties;
 
 public abstract class BaseAuditHandler implements AuditHandler {
 	private static final Log LOG = LogFactory.getLog(BaseAuditHandler.class);
 
 	static final String AUDIT_LOG_FAILURE_REPORT_MIN_INTERVAL_PROP = "xasecure.audit.log.failure.report.min.interval.ms";
 	protected static final String AUDIT_DB_CREDENTIAL_PROVIDER_FILE = "xasecure.audit.credential.provider.file";
+	public static final String PROP_CONFIG = "config";
 
 	private int mLogFailureReportMinIntervalInMs = 60 * 1000;
 
@@ -75,6 +73,7 @@ public abstract class BaseAuditHandler implements AuditHandler {
 	long statusLogIntervalMS = 1 * 60 * 1000;
 
 	protected Properties props = null;
+	protected Map<String, String> configProps = new HashMap<String, String>();
 
 	@Override
 	public void init(Properties props) {
@@ -116,6 +115,18 @@ public abstract class BaseAuditHandler implements AuditHandler {
 		mLogFailureReportMinIntervalInMs = MiscUtil.getIntProperty(props,
 				AUDIT_LOG_FAILURE_REPORT_MIN_INTERVAL_PROP, 60 * 1000);
 
+		String configPropsNamePrefix = propPrefix + "." + PROP_CONFIG + ".";
+		for (Object propNameObj : props.keySet()) {
+			String propName = propNameObj.toString();
+
+			if (!propName.startsWith(configPropsNamePrefix)) {
+				continue;
+			}
+			String configName = propName.substring(configPropsNamePrefix.length());
+			String configValue = props.getProperty(propName);
+			configProps.put(configName, configValue);
+			LOG.info("Found Config property: " + configName + " => " + configValue);
+		}
 	}
 
 	/*

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsAuditProvider.java
----------------------------------------------------------------------
diff --git a/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsAuditProvider.java b/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsAuditProvider.java
index a18e3e9..8cdf869 100644
--- a/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsAuditProvider.java
+++ b/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsAuditProvider.java
@@ -60,6 +60,8 @@ public class HdfsAuditProvider extends BufferedAuditProvider {
 		int    localFileBufferRolloverIntervalSeconds = MiscUtil.parseInteger(hdfsProps.get("local.buffer.rollover.interval.seconds"), 10 * 60);
 		String localFileBufferArchiveDirectory        = hdfsProps.get("local.archive.directory");
 		int    localFileBufferArchiveFileCount        = MiscUtil.parseInteger(hdfsProps.get("local.archive.max.file.count"), 10);
+		// Added for Azure.  Note that exact name of these properties is not known as it contains the variable account name in it.
+		Map<String, String> configProps = MiscUtil.getPropertiesWithPrefix(props, "xasecure.audit.destination.hdfs.config.");
 
 		DebugTracer tracer = new Log4jTracer(LOG);
 
@@ -71,6 +73,7 @@ public class HdfsAuditProvider extends BufferedAuditProvider {
 		mHdfsDestination.setEncoding(encoding);
 		mHdfsDestination.setRolloverIntervalSeconds(hdfsDestinationRolloverIntervalSeconds);
 		mHdfsDestination.setOpenRetryIntervalSeconds(hdfsDestinationOpenRetryIntervalSeconds);
+		mHdfsDestination.setConfigProps(configProps);
 
 		LocalFileLogBuffer<AuditEventBase> mLocalFileBuffer = new LocalFileLogBuffer<AuditEventBase>(tracer);
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsLogDestination.java
----------------------------------------------------------------------
diff --git a/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsLogDestination.java b/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsLogDestination.java
index 49f4e65..a9aa8c1 100644
--- a/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsLogDestination.java
+++ b/agents-audit/src/main/java/org/apache/ranger/audit/provider/hdfs/HdfsLogDestination.java
@@ -24,7 +24,9 @@ import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.UnsupportedEncodingException;
 import java.net.URI;
+import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -55,6 +57,7 @@ public class HdfsLogDestination<T> implements LogDestination<T> {
 	private long               mNextFlushTime      = 0;
 	private long               mLastOpenFailedTime = 0;
 	private boolean            mIsStopInProgress   = false;
+	private Map<String, String> configProps = null;
 
 	public HdfsLogDestination(DebugTracer tracer) {
 		mLogger = tracer;
@@ -272,7 +275,7 @@ public class HdfsLogDestination<T> implements LogDestination<T> {
 
 			// TODO: mechanism to XA-HDFS plugin to disable auditing of access checks to the current HDFS file
 
-			conf        = new Configuration();
+			conf        = createConfiguration();
 			pathLogfile = new Path(mHdfsFilename);
 			fileSystem  = FileSystem.get(uri, conf);
 
@@ -490,4 +493,25 @@ public class HdfsLogDestination<T> implements LogDestination<T> {
 		return sb.toString();
 	}
 
+	public void setConfigProps(Map<String,String> configProps) {
+		this.configProps = configProps;
+	}
+
+	Configuration createConfiguration() {
+		Configuration conf = new Configuration();
+		if (configProps != null) {
+			for (Map.Entry<String, String> entry : configProps.entrySet()) {
+				String key = entry.getKey();
+				String value = entry.getValue();
+				// for ease of install config file may contain properties with empty value, skip those
+				if (StringUtils.isNotEmpty(value)) {
+					conf.set(key, value);
+				}
+				mLogger.info("Adding property to HDFS config: " + key + " => " + value);
+			}
+		}
+
+		mLogger.info("Returning HDFS Filesystem Config: " + conf.toString());
+		return conf;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/agents-installer/src/main/java/org/apache/ranger/utils/install/XmlConfigChanger.java
----------------------------------------------------------------------
diff --git a/agents-installer/src/main/java/org/apache/ranger/utils/install/XmlConfigChanger.java b/agents-installer/src/main/java/org/apache/ranger/utils/install/XmlConfigChanger.java
index 05fbb23..958ae83 100644
--- a/agents-installer/src/main/java/org/apache/ranger/utils/install/XmlConfigChanger.java
+++ b/agents-installer/src/main/java/org/apache/ranger/utils/install/XmlConfigChanger.java
@@ -25,6 +25,8 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Properties;
 
 import javax.xml.parsers.DocumentBuilder;
@@ -45,6 +47,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang.StringUtils;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
@@ -157,7 +160,7 @@ public class XmlConfigChanger {
 		
 	}
 	
-	
+
 
 	
 	public void run() throws ParserConfigurationException, SAXException, IOException, TransformerException {
@@ -177,7 +180,7 @@ public class XmlConfigChanger {
 			
 			@SuppressWarnings("unused")
 			int lineNo = 0 ;
-			
+			Properties variables = new Properties();
 			while ((line = reader.readLine()) != null) {
 				
 				lineNo++ ;
@@ -198,18 +201,21 @@ public class XmlConfigChanger {
 				String[] tokens = line.split("\\s+") ;
 				
 				String propName = tokens[0] ;
-				
+
 				String propValue = null ;
-				
+
 				try {
+					if (propnameContainsVariables(propName)) {
+						propName = replaceProp(propName, variables);
+					}
 					propValue = replaceProp(tokens[1],installProperties) ;
 				} catch (ValidationException e) {
 					// throw new RuntimeException("Unable to replace tokens in the line: \n[" + line + "]\n in file [" + confFile.getAbsolutePath() + "] line number:["  + lineNo + "]" ) ;
 					throw new RuntimeException(e) ;
 				}
-				
-				
-				
+
+
+
 				String actionType = tokens[2] ;
 				String options = (tokens.length > 3 ? tokens[3] : null) ;
 				boolean createIfNotExists = (options != null && options.contains("create-if-not-exists")) ;
@@ -265,6 +271,9 @@ public class XmlConfigChanger {
 						}
 					}
 				}
+				else if ("var".equals(actionType)) {
+					variables.put(propName, propValue);
+				}
 				else {
 					throw new RuntimeException("Unknown Command Found: [" + actionType + "], Supported Types:  add modify del append") ;
 				}
@@ -290,8 +299,28 @@ public class XmlConfigChanger {
 		}
 
 	}
-	
-	
+
+	/**
+	 * Check if prop name contains a substitution variable embedded in it, e.g. %VAR_NAME%.
+	 * @param propName
+	 * @return true if propname contains at least 2 '%' characters in it, else false
+	 */
+	private boolean propnameContainsVariables(String propName) {
+
+		if (propName != null) {
+			int first = propName.indexOf('%');
+			if (first != -1) {
+				// indexof is safe even if 2nd argument is beyond size of string, i.e. if 1st percent was the last character of the string.
+				int second = propName.indexOf('%', first + 1);
+				if (second != -1) {
+					return true;
+				}
+			}
+		}
+		return false;
+	}
+
+
 	private void addProperty(String propName, String val) {
 		NodeList nl = doc.getElementsByTagName(ROOT_NODE_NAME) ;
 		Node rootConfig = nl.item(0) ;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hbase-agent/conf/ranger-hbase-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/hbase-agent/conf/ranger-hbase-audit-changes.cfg b/hbase-agent/conf/ranger-hbase-audit-changes.cfg
index b540e18..e29ccd5 100644
--- a/hbase-agent/conf/ranger-hbase-audit-changes.cfg
+++ b/hbase-agent/conf/ranger-hbase-audit-changes.cfg
@@ -58,5 +58,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hbase-agent/conf/ranger-hbase-audit.xml
----------------------------------------------------------------------
diff --git a/hbase-agent/conf/ranger-hbase-audit.xml b/hbase-agent/conf/ranger-hbase-audit.xml
index c5f416e..5f88224 100644
--- a/hbase-agent/conf/ranger-hbase-audit.xml
+++ b/hbase-agent/conf/ranger-hbase-audit.xml
@@ -160,8 +160,44 @@
 	<property>
 		<name>xasecure.audit.hdfs.config.local.archive.max.file.count</name>
 		<value>10</value>
-	</property>	
-	
+	</property>
+
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hbase-agent/scripts/install.properties
----------------------------------------------------------------------
diff --git a/hbase-agent/scripts/install.properties b/hbase-agent/scripts/install.properties
index 9170f41..5df518a 100644
--- a/hbase-agent/scripts/install.properties
+++ b/hbase-agent/scripts/install.properties
@@ -61,12 +61,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/hbase/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hbase/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hbase/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hdfs-agent/conf/ranger-hdfs-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/hdfs-agent/conf/ranger-hdfs-audit-changes.cfg b/hdfs-agent/conf/ranger-hdfs-audit-changes.cfg
index 93e7b86..9c88450 100644
--- a/hdfs-agent/conf/ranger-hdfs-audit-changes.cfg
+++ b/hdfs-agent/conf/ranger-hdfs-audit-changes.cfg
@@ -55,5 +55,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hdfs-agent/conf/ranger-hdfs-audit.xml
----------------------------------------------------------------------
diff --git a/hdfs-agent/conf/ranger-hdfs-audit.xml b/hdfs-agent/conf/ranger-hdfs-audit.xml
index 4c60c02..a380906 100644
--- a/hdfs-agent/conf/ranger-hdfs-audit.xml
+++ b/hdfs-agent/conf/ranger-hdfs-audit.xml
@@ -162,6 +162,42 @@
 		<value>10</value>
 	</property>	
 
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hdfs-agent/scripts/install.properties
----------------------------------------------------------------------
diff --git a/hdfs-agent/scripts/install.properties b/hdfs-agent/scripts/install.properties
index 289117b..fa21949 100644
--- a/hdfs-agent/scripts/install.properties
+++ b/hdfs-agent/scripts/install.properties
@@ -58,11 +58,22 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/hadoop/hdfs/audit/solr/spool
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hadoop/hdfs/audit/hdfs/spool
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hadoop/hdfs/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hive-agent/conf/ranger-hive-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/hive-agent/conf/ranger-hive-audit-changes.cfg b/hive-agent/conf/ranger-hive-audit-changes.cfg
index daa8b74..4e61c7d 100644
--- a/hive-agent/conf/ranger-hive-audit-changes.cfg
+++ b/hive-agent/conf/ranger-hive-audit-changes.cfg
@@ -56,5 +56,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hive-agent/conf/ranger-hive-audit.xml
----------------------------------------------------------------------
diff --git a/hive-agent/conf/ranger-hive-audit.xml b/hive-agent/conf/ranger-hive-audit.xml
index 867080c..ae1dce9 100644
--- a/hive-agent/conf/ranger-hive-audit.xml
+++ b/hive-agent/conf/ranger-hive-audit.xml
@@ -162,6 +162,42 @@
 		<value>10</value>
 	</property>	
 	
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/hive-agent/scripts/install.properties
----------------------------------------------------------------------
diff --git a/hive-agent/scripts/install.properties b/hive-agent/scripts/install.properties
index 9370600..2e41a37 100644
--- a/hive-agent/scripts/install.properties
+++ b/hive-agent/scripts/install.properties
@@ -58,12 +58,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/hive/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hive/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hive/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/kms/scripts/install.properties
----------------------------------------------------------------------
diff --git a/kms/scripts/install.properties b/kms/scripts/install.properties
index 563a0e0..247e499 100755
--- a/kms/scripts/install.properties
+++ b/kms/scripts/install.properties
@@ -118,12 +118,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/ranger/kms/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/ranger/kms/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/ranger/kms/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/knox-agent/conf/ranger-knox-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/knox-agent/conf/ranger-knox-audit-changes.cfg b/knox-agent/conf/ranger-knox-audit-changes.cfg
index 57858f7..f722e53 100644
--- a/knox-agent/conf/ranger-knox-audit-changes.cfg
+++ b/knox-agent/conf/ranger-knox-audit-changes.cfg
@@ -56,5 +56,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/knox-agent/conf/ranger-knox-audit.xml
----------------------------------------------------------------------
diff --git a/knox-agent/conf/ranger-knox-audit.xml b/knox-agent/conf/ranger-knox-audit.xml
index ccc2691..7b5bfb3 100644
--- a/knox-agent/conf/ranger-knox-audit.xml
+++ b/knox-agent/conf/ranger-knox-audit.xml
@@ -162,6 +162,42 @@
 		<value>10</value>
 	</property>	
 
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/knox-agent/scripts/install.properties
----------------------------------------------------------------------
diff --git a/knox-agent/scripts/install.properties b/knox-agent/scripts/install.properties
index 2fee7ee..506c53c 100644
--- a/knox-agent/scripts/install.properties
+++ b/knox-agent/scripts/install.properties
@@ -61,12 +61,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/knox/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/knox/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/knox/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-kafka/conf/ranger-kafka-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/plugin-kafka/conf/ranger-kafka-audit-changes.cfg b/plugin-kafka/conf/ranger-kafka-audit-changes.cfg
index 0445576..46ee29a 100644
--- a/plugin-kafka/conf/ranger-kafka-audit-changes.cfg
+++ b/plugin-kafka/conf/ranger-kafka-audit-changes.cfg
@@ -49,5 +49,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-kafka/conf/ranger-kafka-audit.xml
----------------------------------------------------------------------
diff --git a/plugin-kafka/conf/ranger-kafka-audit.xml b/plugin-kafka/conf/ranger-kafka-audit.xml
index bff6cbf..5fbbf33 100644
--- a/plugin-kafka/conf/ranger-kafka-audit.xml
+++ b/plugin-kafka/conf/ranger-kafka-audit.xml
@@ -163,6 +163,42 @@
 		<value>10</value>
 	</property>	
 	
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-kafka/scripts/install.properties
----------------------------------------------------------------------
diff --git a/plugin-kafka/scripts/install.properties b/plugin-kafka/scripts/install.properties
index 1e2854e..bc6481b 100644
--- a/plugin-kafka/scripts/install.properties
+++ b/plugin-kafka/scripts/install.properties
@@ -64,12 +64,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/kafka/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/kafka/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/kafka/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-kms/conf/ranger-kms-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/plugin-kms/conf/ranger-kms-audit-changes.cfg b/plugin-kms/conf/ranger-kms-audit-changes.cfg
index f7d3543..5a51455 100644
--- a/plugin-kms/conf/ranger-kms-audit-changes.cfg
+++ b/plugin-kms/conf/ranger-kms-audit-changes.cfg
@@ -58,5 +58,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-kms/conf/ranger-kms-audit.xml
----------------------------------------------------------------------
diff --git a/plugin-kms/conf/ranger-kms-audit.xml b/plugin-kms/conf/ranger-kms-audit.xml
index f1c4187..02c1a79 100755
--- a/plugin-kms/conf/ranger-kms-audit.xml
+++ b/plugin-kms/conf/ranger-kms-audit.xml
@@ -161,6 +161,42 @@
 		<value>10</value>
 	</property>	
 
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-solr/conf/ranger-solr-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/plugin-solr/conf/ranger-solr-audit-changes.cfg b/plugin-solr/conf/ranger-solr-audit-changes.cfg
index 7e77a30..2742bc1 100644
--- a/plugin-solr/conf/ranger-solr-audit-changes.cfg
+++ b/plugin-solr/conf/ranger-solr-audit-changes.cfg
@@ -50,5 +50,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-solr/conf/ranger-solr-audit.xml
----------------------------------------------------------------------
diff --git a/plugin-solr/conf/ranger-solr-audit.xml b/plugin-solr/conf/ranger-solr-audit.xml
index a42716f..f55b623 100644
--- a/plugin-solr/conf/ranger-solr-audit.xml
+++ b/plugin-solr/conf/ranger-solr-audit.xml
@@ -163,6 +163,42 @@
 		<value>10</value>
 	</property>	
 	
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-solr/scripts/install.properties
----------------------------------------------------------------------
diff --git a/plugin-solr/scripts/install.properties b/plugin-solr/scripts/install.properties
index 1f3852e..9bc305b 100644
--- a/plugin-solr/scripts/install.properties
+++ b/plugin-solr/scripts/install.properties
@@ -64,12 +64,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/solr/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/solr/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/solr/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-yarn/conf/ranger-yarn-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/plugin-yarn/conf/ranger-yarn-audit-changes.cfg b/plugin-yarn/conf/ranger-yarn-audit-changes.cfg
index bfc2cd8..b650be1 100644
--- a/plugin-yarn/conf/ranger-yarn-audit-changes.cfg
+++ b/plugin-yarn/conf/ranger-yarn-audit-changes.cfg
@@ -56,5 +56,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-yarn/conf/ranger-yarn-audit.xml
----------------------------------------------------------------------
diff --git a/plugin-yarn/conf/ranger-yarn-audit.xml b/plugin-yarn/conf/ranger-yarn-audit.xml
index c30f963..667419d 100644
--- a/plugin-yarn/conf/ranger-yarn-audit.xml
+++ b/plugin-yarn/conf/ranger-yarn-audit.xml
@@ -163,6 +163,42 @@
 		<value>10</value>
 	</property>	
 	
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/plugin-yarn/scripts/install.properties
----------------------------------------------------------------------
diff --git a/plugin-yarn/scripts/install.properties b/plugin-yarn/scripts/install.properties
index 9358033..01f733d 100644
--- a/plugin-yarn/scripts/install.properties
+++ b/plugin-yarn/scripts/install.properties
@@ -56,12 +56,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/hadoop/yarn/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hadoop/yarn/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/hadoop/yarn/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/storm-agent/conf/ranger-storm-audit-changes.cfg
----------------------------------------------------------------------
diff --git a/storm-agent/conf/ranger-storm-audit-changes.cfg b/storm-agent/conf/ranger-storm-audit-changes.cfg
index bfc2cd8..b650be1 100644
--- a/storm-agent/conf/ranger-storm-audit-changes.cfg
+++ b/storm-agent/conf/ranger-storm-audit-changes.cfg
@@ -56,5 +56,10 @@ xasecure.audit.destination.hdfs					   %XAAUDIT.HDFS.ENABLE%
 xasecure.audit.destination.hdfs.batch.filespool.dir                %XAAUDIT.HDFS.FILE_SPOOL_DIR%                      mod create-if-not-exists
 xasecure.audit.destination.hdfs.dir                		   %XAAUDIT.HDFS.HDFS_DIR%                      mod create-if-not-exists
 
+AZURE.ACCOUNTNAME                                                                                                 %XAAUDIT.HDFS.AZURE_ACCOUNTNAME%            var
+xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script                                           %XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER%     mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.key.%AZURE.ACCOUNTNAME%.blob.core.windows.net             %XAAUDIT.HDFS.AZURE_ACCOUNTKEY%             mod         create-if-not-exists
+xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.%AZURE.ACCOUNTNAME%.blob.core.windows.net     %XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER%    mod         create-if-not-exists
+
 #xasecure.audit.destination.file					   %XAAUDIT.FILE.ENABLE%                      mod create-if-not-exists
 #xasecure.audit.destination.file.dir                		   %XAAUDIT.FILE.DIR%                      mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/storm-agent/conf/ranger-storm-audit.xml
----------------------------------------------------------------------
diff --git a/storm-agent/conf/ranger-storm-audit.xml b/storm-agent/conf/ranger-storm-audit.xml
index 6a5ff69..c0c66d4 100644
--- a/storm-agent/conf/ranger-storm-audit.xml
+++ b/storm-agent/conf/ranger-storm-audit.xml
@@ -162,6 +162,42 @@
 		<value>10</value>
 	</property>	
 	
+	<!-- Audit to HDFS on Azure Datastore (WASB) requires v3 style settings.  Comment the above and uncomment only the
+	following to audit to Azure Blob Datastore via hdfs' WASB schema.
+
+	NOTE: If you specify one audit destination in v3 style then other destinations, if any, must also be specified in v3 style
+	====
+
+	<property>
+		<name>xasecure.audit.destination.hdfs</name>
+		<value>enabled</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.dir</name>
+		<value>wasb://ranger-audit1@youraccount.blob.core.windows.net</value>
+	</property>
+
+	the following 3 correspond to the properties with similar name in core-site.xml, i.e.
+	- fs.azure.account.key.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net and
+	- fs.azure.account.keyprovider.youraccount.blob.core.windows.net => xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net,
+	- fs.azure.shellkeyprovider.script => xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script,
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.key.youraccount.blob.core.windows.net</name>
+		<value>YOUR ENCRYPTED ACCESS KEY</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.account.keyprovider.youraccount.blob.core.windows.net</name>
+		<value>org.apache.hadoop.fs.azure.ShellDecryptionKeyProvider</value>
+	</property>
+
+	<property>
+		<name>xasecure.audit.destination.hdfs.config.fs.azure.shellkeyprovider.script</name>
+		<value>/usr/lib/python2.7/dist-packages/hdinsight_common/decrypt.sh</value>
+	</property>
+	-->
 
 	<!-- Log4j audit provider configuration -->
 	<property>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/35aa706e/storm-agent/scripts/install.properties
----------------------------------------------------------------------
diff --git a/storm-agent/scripts/install.properties b/storm-agent/scripts/install.properties
index 0cee655..82f17af 100644
--- a/storm-agent/scripts/install.properties
+++ b/storm-agent/scripts/install.properties
@@ -58,12 +58,23 @@ XAAUDIT.SOLR.FILE_SPOOL_DIR=/var/log/storm/audit/solr/spool
 #Example
 #XAAUDIT.HDFS.ENABLE=true
 #XAAUDIT.HDFS.HDFS_DIR=hdfs://node-1.example.com:8020/ranger/audit
+#  If using Azure Blob Storage
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+#XAAUDIT.HDFS.HDFS_DIR=wasb://ranger_audit_container@my-azure-account.blob.core.windows.net/ranger/audit
 #XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/storm/audit/hdfs/spool
 
 XAAUDIT.HDFS.ENABLE=false
 XAAUDIT.HDFS.HDFS_DIR=hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit
 XAAUDIT.HDFS.FILE_SPOOL_DIR=/var/log/storm/audit/hdfs/spool
 
+# Following additional propertis are needed When auditing to Azure Blob Storage via HDFS
+# Get these values from your /etc/hadoop/conf/core-site.xml
+#XAAUDIT.HDFS.HDFS_DIR=wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+XAAUDIT.HDFS.AZURE_ACCOUNTNAME=__REPLACE_AZURE_ACCOUNT_NAME
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY=__REPLACE_AZURE_ACCOUNT_KEY
+XAAUDIT.HDFS.AZURE_SHELL_KEY_PROVIDER=__REPLACE_AZURE_SHELL_KEY_PROVIDER
+XAAUDIT.HDFS.AZURE_ACCOUNTKEY_PROVIDER=__REPLACE_AZURE_ACCOUNT_KEY_PROVIDER
+
 # End of V3 properties
 
 #