You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by we...@apache.org on 2015/11/20 22:01:44 UTC

incubator-reef git commit: [REEF-952] IFileSystem for Azure BlockBlobs

Repository: incubator-reef
Updated Branches:
  refs/heads/master 6e9485d4f -> 3c35b7eef


[REEF-952] IFileSystem for Azure BlockBlobs

This addressed the issue by
  * Adding Azure Storage dependency to REEF.IO.
  * Implementing IFileSystem for Block Blobs.
  * Implementing and testing on "Fakes" and actual blob storage.
  * Creating proxy classes for "Fakes."

JIRA:
  [REEF-952](https://issues.apache.org/jira/browse/REEF-952)

Pull Request:
  This closes #645


Project: http://git-wip-us.apache.org/repos/asf/incubator-reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-reef/commit/3c35b7ee
Tree: http://git-wip-us.apache.org/repos/asf/incubator-reef/tree/3c35b7ee
Diff: http://git-wip-us.apache.org/repos/asf/incubator-reef/diff/3c35b7ee

Branch: refs/heads/master
Commit: 3c35b7eef2cfff55bc390b37b56f27d48331c940
Parents: 6e9485d
Author: Andrew Chung <af...@gmail.com>
Authored: Thu Nov 12 16:50:46 2015 -0800
Committer: Markus Weimer <we...@apache.org>
Committed: Fri Nov 20 13:00:18 2015 -0800

----------------------------------------------------------------------
 .../Org.Apache.REEF.IO.Tests.csproj             |  11 +
 .../TestAzureBlockBlobFileSystem.cs             | 159 ++++++++++++
 .../TestAzureBlockBlobFileSystemE2E.cs          | 246 +++++++++++++++++++
 .../cs/Org.Apache.REEF.IO.Tests/packages.config |   3 +-
 .../FileSystem/AzureBlob/AzureBlobType.cs       |  30 +++
 .../AzureBlob/AzureBlockBlobFileSystem.cs       | 214 ++++++++++++++++
 .../AzureBlockBlobFileSystemConfiguration.cs    |  51 ++++
 ...eBlockBlobFileSystemConfigurationProvider.cs |  55 +++++
 .../AzureBlob/AzureCloudBlobClient.cs           |  73 ++++++
 .../AzureBlob/AzureCloudBlobContainer.cs        |  44 ++++
 .../AzureBlob/AzureCloudBlobDirectory.cs        |  45 ++++
 .../FileSystem/AzureBlob/AzureCloudBlockBlob.cs |  97 ++++++++
 .../FileSystem/AzureBlob/ICloudBlobClient.cs    |  64 +++++
 .../FileSystem/AzureBlob/ICloudBlobContainer.cs |  39 +++
 .../FileSystem/AzureBlob/ICloudBlobDirectory.cs |  49 ++++
 .../FileSystem/AzureBlob/ICloudBlockBlob.cs     |  91 +++++++
 .../Parameters/AzureStorageConnectionString.cs  |  29 +++
 .../RetryPolicy/DefaultAzureBlobRetryPolicy.cs  |  46 ++++
 .../RetryPolicy/IAzureBlobRetryPolicy.cs        |  27 ++
 .../Org.Apache.REEF.IO.csproj                   |  51 +++-
 .../Properties/AssemblyInfo.cs                  |   6 +
 lang/cs/Org.Apache.REEF.IO/packages.config      |   9 +-
 .../Org.Apache.REEF.Tests.csproj                |  30 ++-
 lang/cs/Org.Apache.REEF.Tests/packages.config   |  15 +-
 24 files changed, 1471 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj b/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
index bd824b6..9dabc5d 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
@@ -41,10 +41,21 @@ under the License.
     <Reference Include="Microsoft.CSharp" />
     <Reference Include="System.Data" />
     <Reference Include="System.Xml" />
+    <Reference Include="NSubstitute, Version=1.8.2.0, Culture=neutral, PublicKeyToken=92dd2e9066daa5ca, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\NSubstitute.1.8.2.0\lib\net45\NSubstitute.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.WindowsAzure.Storage, Version=6.1.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <SpecificVersion>True</SpecificVersion>
+      <HintPath>$(PackagesDir)\WindowsAzure.Storage.6.1.0\lib\net40\Microsoft.WindowsAzure.Storage.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
   </ItemGroup>
   <ItemGroup>
     <Compile Include="FileSystemTestUtilities.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="TestAzureBlockBlobFileSystem.cs" />
+    <Compile Include="TestAzureBlockBlobFileSystemE2E.cs" />
     <Compile Include="TestFilePartitionInputDataSet.cs" />
     <Compile Include="TestHadoopFileSystem.cs" />
     <Compile Include="TestLocalFileSystem.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
new file mode 100644
index 0000000..eabe3f7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
@@ -0,0 +1,159 @@
+// 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.
+
+using System;
+using System.IO;
+using System.Linq;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Microsoft.WindowsAzure.Storage.RetryPolicies;
+using NSubstitute;
+using Org.Apache.REEF.IO.FileSystem;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.RetryPolicy;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.IO.Tests
+{
+    /// <summary>
+    /// Test class for unit testing the AzureBlockBlobFileSystem.
+    /// Some methods are currently not unit tested due to the complexity of "faking out"
+    /// the methods. They are instead tested E2E in TestAzureBlockBlobFileSystemE2E
+    /// </summary>
+    [TestClass]
+    public sealed class TestAzureBlockBlobFileSystem
+    {
+        private readonly static Uri FakeUri = new Uri("http://fake.com");
+
+        [TestMethod, ExpectedException(typeof(NotSupportedException))]
+        public void TestCreateNotSupported()
+        {
+            new TestContext().GetAzureFileSystem().Create(FakeUri);
+        }
+
+        [TestMethod, ExpectedException(typeof(NotSupportedException))]
+        public void TestOpenNotSupported()
+        {
+            new TestContext().GetAzureFileSystem().Open(FakeUri);
+        }
+
+        [TestMethod]
+        public void TestDelete()
+        {
+            var testContext = new TestContext();
+            testContext.GetAzureFileSystem().Delete(new Uri(FakeUri, "container/file"));
+            testContext.TestCloudBlockBlob.Received(1).Delete();
+        }
+
+        [TestMethod]
+        public void TestExists()
+        {
+            var testContext = new TestContext();
+            testContext.GetAzureFileSystem().Exists(FakeUri);
+            testContext.TestCloudBlockBlob.Received(1).Exists();
+        }
+
+        [TestMethod]
+        public void TestCopyToLocal()
+        {
+            var testContext = new TestContext();
+            testContext.GetAzureFileSystem().CopyToLocal(FakeUri, "local");
+            testContext.TestCloudBlockBlob.Received(1).DownloadToFile("local", FileMode.CreateNew);
+        }
+
+        [TestMethod]
+        public void TestCopyFromLocal()
+        {
+            var testContext = new TestContext();
+            testContext.GetAzureFileSystem().CopyFromLocal("local", FakeUri);
+            testContext.TestCloudBlockBlob.Received(1).UploadFromFile("local", FileMode.Open);
+        }
+
+        [TestMethod]
+        public void TestCreateDirectory()
+        {
+            var testContext = new TestContext();
+            testContext.GetAzureFileSystem().CreateDirectory(FakeUri);
+            testContext.TestCloudBlobClient.DidNotReceiveWithAnyArgs();
+        }
+
+        [TestMethod, ExpectedException(typeof(StorageException))]
+        public void TestDeleteDirectoryFails()
+        {
+            new TestContext().GetAzureFileSystem().DeleteDirectory(FakeUri);
+        }
+
+        [TestMethod]
+        public void TestDeleteDirectoryAtContainer()
+        {
+            var testContext = new TestContext();
+            testContext.GetAzureFileSystem().DeleteDirectory(new Uri("http://test.com/test"));
+            testContext.TestCloudBlobClient.Received(1).GetContainerReference("test");
+            testContext.TestCloudBlobContainer.Received(1).DeleteIfExists();
+        }
+
+        [TestMethod]
+        public void TestDeleteDirectoryRecursive()
+        {
+            var testContext = new TestContext();
+            testContext.TestCloudBlobDirectory.ListBlobs(true).ReturnsForAnyArgs(Enumerable.Repeat(testContext.TestCloudBlob, 5));
+            testContext.GetAzureFileSystem().DeleteDirectory(new Uri("http://test.com/container/directory/directory"));
+            testContext.TestCloudBlobClient.Received(1).GetContainerReference("container");
+            testContext.TestCloudBlobContainer.Received(1).GetDirectoryReference("directory");
+            testContext.TestCloudBlobDirectory.Received(1).GetDirectoryReference("directory");
+            testContext.TestCloudBlobDirectory.Received(1).ListBlobs(true);
+            testContext.TestCloudBlob.Received(5).DeleteIfExists();
+        }
+
+        [TestMethod]
+        public void TestCreateUriForPath()
+        {
+            var testContext = new TestContext();
+            const string dirStructure = "container/directory";
+            Assert.AreEqual(new Uri(FakeUri, dirStructure), testContext.GetAzureFileSystem().CreateUriForPath(dirStructure));
+        }
+
+        private sealed class TestContext
+        {
+            public readonly ICloudBlobClient TestCloudBlobClient = Substitute.For<ICloudBlobClient>();
+            public readonly ICloudBlob TestCloudBlob = Substitute.For<ICloudBlob>();
+            public readonly ICloudBlockBlob TestCloudBlockBlob = Substitute.For<ICloudBlockBlob>();
+            public readonly ICloudBlobContainer TestCloudBlobContainer = Substitute.For<ICloudBlobContainer>();
+            public readonly ICloudBlobDirectory TestCloudBlobDirectory = Substitute.For<ICloudBlobDirectory>();
+
+            public IFileSystem GetAzureFileSystem()
+            {
+                var conf = AzureBlockBlobFileSystemConfiguration.ConfigurationModule
+                    .Set(AzureBlockBlobFileSystemConfiguration.ConnectionString, "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;")
+                    .Build();
+
+                var injector = TangFactory.GetTang().NewInjector(conf);
+                injector.BindVolatileInstance(TestCloudBlobClient);
+                var fs = injector.GetInstance<AzureBlockBlobFileSystem>();
+                TestCloudBlobClient.BaseUri.ReturnsForAnyArgs(FakeUri);
+                TestCloudBlobClient.GetBlockBlobReference(FakeUri).ReturnsForAnyArgs(TestCloudBlockBlob);
+                TestCloudBlobClient.GetContainerReference("container").ReturnsForAnyArgs(TestCloudBlobContainer);
+                TestCloudBlobContainer.GetDirectoryReference("directory").ReturnsForAnyArgs(TestCloudBlobDirectory);
+                TestCloudBlobDirectory.GetDirectoryReference("directory").ReturnsForAnyArgs(TestCloudBlobDirectory);
+                return fs;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
new file mode 100644
index 0000000..79d7d94
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
@@ -0,0 +1,246 @@
+// 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.
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Org.Apache.REEF.IO.FileSystem;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+
+namespace Org.Apache.REEF.IO.Tests
+{
+    /// <summary>
+    /// E2E tests for AzureBlockBlobFileSystem.
+    /// These tests require the person running the test to fill in credentials.
+    /// </summary>
+    [TestClass, Ignore]
+    public sealed class TestAzureBlockBlobFileSystemE2E
+    {
+        private const string HelloFile = "hello";
+        private IFileSystem _fileSystem;
+        private CloudBlobContainer _container;
+
+        [TestInitialize]
+        public void TestInitialize()
+        {
+            // Fill in before running test!
+            const string connectionString = "DefaultEndpointsProtocol=http;AccountName=myAccount;AccountKey=myKey;";
+            var defaultContainerName = "reef-test-container-" + Guid.NewGuid();
+            var conf = AzureBlockBlobFileSystemConfiguration.ConfigurationModule
+                .Set(AzureBlockBlobFileSystemConfiguration.ConnectionString, connectionString)
+                .Build();
+
+            _fileSystem= TangFactory.GetTang().NewInjector(conf).GetInstance<AzureBlockBlobFileSystem>();
+            _container = CloudStorageAccount.Parse(connectionString).CreateCloudBlobClient().GetContainerReference(defaultContainerName);
+            _container.CreateIfNotExists();
+        }
+
+        [TestCleanup]
+        public void TestCleanup()
+        {
+            if (_container != null)
+            {
+                _container.DeleteIfExists();
+            }
+        }
+
+        [TestMethod]
+        public void TestDeleteE2E()
+        {
+            var blob = _container.GetBlockBlobReference(HelloFile);
+            UploadFromString(blob, "hello");
+            Assert.IsTrue(blob.Exists());
+            _fileSystem.Delete(PathToFile(HelloFile));
+            Assert.IsFalse(blob.Exists());
+        }
+
+        [TestMethod]
+        public void TestExistsE2E()
+        {
+            var helloFilePath = PathToFile(HelloFile);
+            var blob = _container.GetBlockBlobReference(HelloFile);
+            UploadFromString(blob, "hello");
+            Assert.IsTrue(_fileSystem.Exists(helloFilePath));
+            blob.DeleteIfExists();
+            Assert.IsFalse(_fileSystem.Exists(helloFilePath));
+        }
+
+        [TestMethod]
+        public void TestCopyE2E()
+        {
+            const string srcFileName = "src";
+            const string destFileName = "dest";
+            var srcFilePath = PathToFile(srcFileName);
+            var destFilePath = PathToFile(destFileName);
+            ICloudBlob srcBlob = _container.GetBlockBlobReference(srcFileName);
+            UploadFromString(srcBlob, "hello");
+            Assert.IsTrue(srcBlob.Exists());
+            ICloudBlob destBlob = _container.GetBlockBlobReference(destFileName);
+            Assert.IsFalse(destBlob.Exists());
+            _fileSystem.Copy(srcFilePath, destFilePath);
+            destBlob = _container.GetBlobReferenceFromServer(destFileName);
+            Assert.IsTrue(destBlob.Exists());
+            srcBlob = _container.GetBlobReferenceFromServer(srcFileName);
+            Assert.IsTrue(srcBlob.Exists());
+            Assert.AreEqual(_container.GetBlockBlobReference(srcFileName).DownloadText(), _container.GetBlockBlobReference(destFileName).DownloadText());
+        }
+
+        [TestMethod]
+        public void TestCopyToLocalE2E()
+        {
+            var helloFilePath = PathToFile(HelloFile);
+            var blob = _container.GetBlockBlobReference(HelloFile);
+            var tempFilePath = GetTempFilePath();
+            const string text = "hello";
+            try
+            {
+                UploadFromString(blob, text);
+                _fileSystem.CopyToLocal(helloFilePath, tempFilePath);
+                Assert.IsTrue(File.Exists(tempFilePath));
+                Assert.AreEqual(text, File.ReadAllText(tempFilePath));
+            }
+            finally
+            {
+                File.Delete(tempFilePath);
+            }
+        }
+
+        [TestMethod]
+        public void TestCopyFromLocalE2E()
+        {
+            var helloFilePath = PathToFile(HelloFile);
+            ICloudBlob blob = _container.GetBlockBlobReference(HelloFile);
+            Assert.IsFalse(blob.Exists());
+            var tempFilePath = GetTempFilePath();
+            const string text = "hello";
+            try
+            {
+                File.WriteAllText(tempFilePath, text);
+                _fileSystem.CopyFromLocal(tempFilePath, helloFilePath);
+                blob = _container.GetBlobReferenceFromServer(HelloFile);
+                Assert.IsTrue(blob.Exists());
+                using (var stream = new MemoryStream())
+                {
+                    blob.DownloadToStream(stream);
+                    stream.Seek(0, SeekOrigin.Begin);
+
+                    using (var sr = new StreamReader(stream))
+                    {
+                        var matchingText = sr.ReadToEnd();
+                        Assert.AreEqual(text, matchingText);
+                    }
+                }
+            }
+            finally
+            {
+                File.Delete(tempFilePath);
+            }
+        }
+
+        
+
+        [TestMethod]
+        public void TestDeleteDirectoryAtContainerE2E()
+        {
+            _fileSystem.DeleteDirectory(_container.Uri);
+            Assert.IsFalse(_container.Exists());
+        }
+
+        [TestMethod]
+        public void TestDeleteDirectoryFirstLevelE2E()
+        {
+            const string directory = "dir";
+            var blockBlobs = new List<CloudBlockBlob>(); 
+
+            for (var i = 0; i < 3; i++)
+            {
+                var filePath = directory + '/' + i;
+                var blockBlob = _container.GetBlockBlobReference(filePath);
+                UploadFromString(blockBlob, "hello");
+                Assert.IsTrue(blockBlob.Exists());
+                blockBlobs.Add(blockBlob);
+            }
+
+            _fileSystem.DeleteDirectory(PathToFile(directory));
+
+            foreach (var blockBlob in blockBlobs)
+            {
+                Assert.IsFalse(blockBlob.Exists());
+            }
+
+            Assert.IsTrue(_container.Exists());
+        }
+
+        [TestMethod]
+        public void TestDeleteDirectorySecondLevelE2E()
+        {
+            const string directory1 = "dir1";
+            const string directory2 = "dir2";
+            var blockBlobs1 = new List<CloudBlockBlob>();
+            var blockBlobs2 = new List<CloudBlockBlob>();
+
+            for (var i = 0; i < 3; i++)
+            {
+                var filePath1 = directory1 + '/' + i;
+                var filePath2 = directory1 + '/' + directory2 + '/' + i;
+                var blockBlob1 = _container.GetBlockBlobReference(filePath1);
+                var blockBlob2 = _container.GetBlockBlobReference(filePath2);
+                UploadFromString(blockBlob1, "hello");
+                UploadFromString(blockBlob2, "hello");
+                Assert.IsTrue(blockBlob1.Exists());
+                Assert.IsTrue(blockBlob2.Exists());
+                blockBlobs1.Add(blockBlob1);
+                blockBlobs2.Add(blockBlob2);
+            }
+
+            _fileSystem.DeleteDirectory(PathToFile(directory1 + '/' + directory2));
+
+            foreach (var blockBlob in blockBlobs2)
+            {
+                Assert.IsFalse(blockBlob.Exists());
+            }
+
+            foreach (var blockBlob in blockBlobs1)
+            {
+                Assert.IsTrue(blockBlob.Exists());
+            }
+
+            Assert.IsTrue(_container.Exists());
+        }
+
+        private static string GetTempFilePath()
+        {
+            return Path.Combine(Path.GetTempPath(), Guid.NewGuid().ToString());
+        }
+
+        private static void UploadFromString(ICloudBlob blob, string str)
+        {
+            var byteArray = Encoding.UTF8.GetBytes(str);
+            blob.UploadFromByteArray(byteArray, 0, byteArray.Length);
+        }
+
+        private Uri PathToFile(string filePath)
+        {
+            return _fileSystem.CreateUriForPath(_container.Name + '/' + filePath);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
index d952982..3e75428 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -18,5 +18,6 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
+  <package id="NSubstitute" version="1.9.2.0" targetFramework="net45" />
   <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlobType.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlobType.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlobType.cs
new file mode 100644
index 0000000..5ff20e4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlobType.cs
@@ -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.
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// The type of blob supported for the <see cref="IFileSystem"/>.
+    /// For more details on each type of blob, see https://msdn.microsoft.com/en-us/library/azure/ee691964.aspx.
+    /// </summary>
+    public enum AzureBlobType
+    {
+        Block,
+        Page,
+        Append
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
new file mode 100644
index 0000000..84845ce
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
@@ -0,0 +1,214 @@
+// 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.
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Linq;
+using System.Threading.Tasks;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// An IFileSystem implementation for Azure Blob storage.
+    /// This particular File System implementation only supports 
+    /// block blob operations
+    /// </summary>
+    internal sealed class AzureBlockBlobFileSystem : IFileSystem
+    {
+        private readonly ICloudBlobClient _client;
+
+        [Inject]
+        private AzureBlockBlobFileSystem(ICloudBlobClient client)
+        {
+            _client = client;
+        }
+
+        /// <summary>
+        /// Not supported for Azure Blobs, will throw <see cref="NotSupportedException"/>.
+        /// </summary>
+        public Stream Open(Uri fileUri)
+        {
+            throw new NotSupportedException("Open is not supported for AzureBlockBlobFileSystem.");
+        }
+
+        /// <summary>
+        /// Not supported for Azure Blobs, will throw <see cref="NotSupportedException"/>.
+        /// </summary>
+        public Stream Create(Uri fileUri)
+        {
+            throw new NotSupportedException("Open is not supported for AzureBlockBlobFileSystem.");
+        }
+
+        /// <summary>
+        /// Deletes a CloudBlockBlob
+        /// <see cref="IFileSystem.Delete"/>
+        /// </summary>
+        public void Delete(Uri fileUri)
+        {
+            _client.GetBlockBlobReference(fileUri).Delete();
+        }
+
+        /// <summary>
+        /// Checks if a CloudBlockBlob exists.
+        /// <see cref="IFileSystem.Exists"/>
+        /// </summary>
+        public bool Exists(Uri fileUri)
+        {
+            return _client.GetBlockBlobReference(fileUri).Exists();
+        }
+
+        /// <summary>
+        /// Copies to a CloudBlockBlob from another CloudBlockBlob.
+        /// <see cref="IFileSystem.Copy"/>
+        /// </summary>
+        public void Copy(Uri sourceUri, Uri destinationUri)
+        {
+            _client.GetBlockBlobReference(destinationUri).StartCopy(sourceUri);
+            var blockBlob = _client.GetBlockBlobReference(destinationUri);
+            blockBlob.FetchAttributes();
+
+            while (blockBlob.CopyState.Status == CopyStatus.Pending)
+            {
+                Task.Delay(TimeSpan.FromMilliseconds(50));
+                blockBlob.FetchAttributes();
+            }
+        }
+
+        /// <summary>
+        /// Copies from a CloudBlockBlob to a local file.
+        /// <see cref="IFileSystem.CopyToLocal"/>
+        /// </summary>
+        public void CopyToLocal(Uri remoteFileUri, string localName)
+        {
+            _client.GetBlockBlobReference(remoteFileUri).DownloadToFile(localName, FileMode.CreateNew);
+        }
+
+        /// <summary>
+        // Copies to a CloudBlockBlob from a local file.
+        /// <see cref="IFileSystem.CopyFromLocal"/>
+        /// </summary>
+        public void CopyFromLocal(string localFileName, Uri remoteFileUri)
+        {
+            _client.GetBlockBlobReference(remoteFileUri).UploadFromFile(localFileName, FileMode.Open);
+        }
+
+        /// <summary>
+        /// Azure blobs does not have the sense of the existence of a "directory."
+        /// We will not throw an error, but we will not do anything either.
+        /// </summary>
+        public void CreateDirectory(Uri directoryUri)
+        {
+        }
+
+        /// <summary>
+        /// Recursively deletes blobs under a specified "directory URI."
+        /// If only the container is specified, the entire container is deleted.
+        /// </summary>
+        public void DeleteDirectory(Uri directoryUri)
+        {
+            var uriSplit = directoryUri.AbsolutePath.Split(new[] {"/"}, StringSplitOptions.RemoveEmptyEntries);
+            if (!uriSplit.Any())
+            {
+                throw new StorageException(string.Format("URI {0} must contain at least the container.", directoryUri));
+            }
+
+            var containerName = uriSplit[0];
+            var container = _client.GetContainerReference(containerName);
+            if (uriSplit.Length == 1)
+            {
+                container.DeleteIfExists();
+                return;
+            }
+
+            var directory = container.GetDirectoryReference(uriSplit[1]);
+
+            for (var i = 2; i < uriSplit.Length; i++)
+            {
+                directory = directory.GetDirectoryReference(uriSplit[i]);
+            }
+
+            foreach (var blob in directory.ListBlobs(true).OfType<ICloudBlob>())
+            {
+                blob.DeleteIfExists();
+            }
+        }
+
+        /// <summary>
+        /// Gets the children of the blob "directory."
+        /// </summary>
+        public IEnumerable<Uri> GetChildren(Uri directoryUri)
+        {
+            BlobContinuationToken blobContinuationToken = null;
+            var path = directoryUri.AbsolutePath.Trim('/');
+
+            do
+            {
+                var listing = _client.ListBlobsSegmented(path, false,
+                    BlobListingDetails.None, null,
+                    blobContinuationToken, new BlobRequestOptions(), new OperationContext());
+
+                if (listing.Results != null)
+                {
+                    foreach (var listBlobItem in listing.Results)
+                    {
+                        yield return listBlobItem.Uri;
+                    }
+                }
+
+                blobContinuationToken = listing.ContinuationToken;
+            } while (blobContinuationToken != null);
+        }
+
+        /// <summary>
+        /// Creates a Uri using the relative path to the remote file (including the container),
+        /// getting the absolute URI from the Blob client's base URI.
+        /// </summary>
+        /// <param name="path">The relative path to the remote file, including the container</param>
+        /// <returns>The URI to the remote file</returns>
+        public Uri CreateUriForPath(string path)
+        {
+            return new Uri(_client.BaseUri.AbsoluteUri.TrimEnd('/') + '/' + path.Trim('/'));
+        }
+
+        /// <summary>
+        /// Gets the FileStatus based on reports from Blob.
+        /// </summary>
+        public FileStatus GetFileStatus(Uri remoteFileUri)
+        {
+            var blobReference = _client.GetBlockBlobReference(remoteFileUri);
+            if (!blobReference.Exists())
+            {
+                throw new StorageException("Unable to find blob at " + remoteFileUri);
+            }
+
+            blobReference.FetchAttributes();
+
+            var lastModifiedTime = blobReference.Properties.LastModified;
+            if (!lastModifiedTime.HasValue)
+            {
+                throw new StorageException("Blob at " + remoteFileUri + " does not have a last modified" +
+                                           "time. It may have been deleted.");
+            }
+
+            return new FileStatus(lastModifiedTime.Value.DateTime, blobReference.Properties.Length);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfiguration.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfiguration.cs
new file mode 100644
index 0000000..0b25862
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfiguration.cs
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   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.
+
+using Org.Apache.REEF.Common.Client.Parameters;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.Parameters;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.RetryPolicy;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// Configuration Module for the Azure Block Blob (WASB) file system implementation of IFileSystem.
+    /// </summary>
+    /// <remarks>
+    /// Stream-based operations are not supported.
+    /// </remarks>
+    public sealed class AzureBlockBlobFileSystemConfiguration : ConfigurationModuleBuilder
+    {
+        public static readonly RequiredParameter<string> ConnectionString = new RequiredParameter<string>();
+
+        public static readonly OptionalImpl<IAzureBlobRetryPolicy> RetryPolicy = new OptionalImpl<IAzureBlobRetryPolicy>(); 
+
+        /// <summary>
+        /// Set AzureBlockBlobFileSystemProvider to DriverConfigurationProviders.
+        /// Set all the parameters needed for injecting AzureBlockBlobFileSystemProvider.
+        /// </summary>
+        public static readonly ConfigurationModule ConfigurationModule = new AzureBlockBlobFileSystemConfiguration()
+            .BindSetEntry<DriverConfigurationProviders, AzureBlockBlobFileSystemConfigurationProvider, IConfigurationProvider>(
+                GenericType<DriverConfigurationProviders>.Class, GenericType<AzureBlockBlobFileSystemConfigurationProvider>.Class)
+            .BindImplementation(GenericType<IFileSystem>.Class, GenericType<AzureBlockBlobFileSystem>.Class)
+            .BindNamedParameter(GenericType<AzureStorageConnectionString>.Class, ConnectionString)
+            .BindImplementation(GenericType<IAzureBlobRetryPolicy>.Class, RetryPolicy)
+            .Build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfigurationProvider.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfigurationProvider.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfigurationProvider.cs
new file mode 100644
index 0000000..de490c0
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystemConfigurationProvider.cs
@@ -0,0 +1,55 @@
+// 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.
+
+using Org.Apache.REEF.Common.Evaluator.Parameters;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.Parameters;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.RetryPolicy;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// This provider provides configuration for Azure Block Blob File System
+    /// The client that is going to use AzureBlockBlobFileSystem in its driver and evaluators should set 
+    /// configuration data through AzureBlockBlobFileSystem configuration module in the client's configuration
+    /// </summary>
+    internal sealed class AzureBlockBlobFileSystemConfigurationProvider : IConfigurationProvider
+    {
+        private readonly IConfiguration _configuration;
+
+        [Inject]
+        private AzureBlockBlobFileSystemConfigurationProvider(
+            [Parameter(typeof(AzureStorageConnectionString))] string connectionString,
+            IAzureBlobRetryPolicy retryPolicy)
+        {
+            _configuration = TangFactory.GetTang().NewConfigurationBuilder()
+                .BindImplementation(GenericType<IFileSystem>.Class, GenericType<AzureBlockBlobFileSystem>.Class)
+                .BindImplementation(typeof(IAzureBlobRetryPolicy), retryPolicy.GetType())
+                .BindStringNamedParam<AzureStorageConnectionString>(connectionString)
+                .BindSetEntry<EvaluatorConfigurationProviders, AzureBlockBlobFileSystemConfigurationProvider, IConfigurationProvider>()
+                .Build();
+        }
+
+        public IConfiguration GetConfiguration()
+        {
+            return _configuration;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobClient.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobClient.cs
new file mode 100644
index 0000000..797f44c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobClient.cs
@@ -0,0 +1,73 @@
+// 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.
+
+using System;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Auth;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.Parameters;
+using Org.Apache.REEF.IO.FileSystem.AzureBlob.RetryPolicy;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy class for CloudBlobClient, mainly in order to fake for unit testing.
+    /// </summary>
+    internal sealed class AzureCloudBlobClient : ICloudBlobClient
+    {
+        private readonly CloudBlobClient _client;
+
+        public StorageCredentials Credentials { get { return _client.Credentials; } }
+
+        [Inject]
+        private AzureCloudBlobClient([Parameter(typeof(AzureStorageConnectionString))] string connectionString,
+                                     IAzureBlobRetryPolicy retryPolicy)
+        {
+            _client = CloudStorageAccount.Parse(connectionString).CreateCloudBlobClient();
+            _client.DefaultRequestOptions.RetryPolicy = retryPolicy;
+        }
+
+        public Uri BaseUri
+        {
+            get { return _client.BaseUri; }
+        }
+
+        public ICloudBlob GetBlobReferenceFromServer(Uri blobUri)
+        {
+            return _client.GetBlobReferenceFromServer(blobUri);
+        }
+
+        public ICloudBlobContainer GetContainerReference(string containerName)
+        {
+            return new AzureCloudBlobContainer(_client.GetContainerReference(containerName));
+        }
+
+        public ICloudBlockBlob GetBlockBlobReference(Uri uri)
+        {
+            return new AzureCloudBlockBlob(uri, _client.Credentials);
+        }
+
+        public BlobResultSegment ListBlobsSegmented(string prefix, bool useFlatListing, BlobListingDetails blobListingDetails,
+            int? maxResults, BlobContinuationToken continuationToken, BlobRequestOptions blobRequestOptions,
+            OperationContext operationContext)
+        {
+            return _client.ListBlobsSegmented(prefix, useFlatListing, blobListingDetails, maxResults, continuationToken,
+                blobRequestOptions, operationContext);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobContainer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobContainer.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobContainer.cs
new file mode 100644
index 0000000..fb9ff81
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobContainer.cs
@@ -0,0 +1,44 @@
+// 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.
+
+using Microsoft.WindowsAzure.Storage.Blob;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy class for CloudBlobContainer, mainly in order to fake for unit testing.
+    /// </summary>
+    internal sealed class AzureCloudBlobContainer : ICloudBlobContainer
+    {
+        private readonly CloudBlobContainer _container;
+
+        public AzureCloudBlobContainer(CloudBlobContainer container)
+        {
+            _container = container;
+        }
+
+        public void DeleteIfExists()
+        {
+            _container.DeleteIfExists();
+        }
+
+        public ICloudBlobDirectory GetDirectoryReference(string directoryName)
+        {
+            return new AzureCloudBlobDirectory(_container.GetDirectoryReference(directoryName));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobDirectory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobDirectory.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobDirectory.cs
new file mode 100644
index 0000000..599c02a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlobDirectory.cs
@@ -0,0 +1,45 @@
+// 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.
+
+using System.Collections.Generic;
+using Microsoft.WindowsAzure.Storage.Blob;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy class for CloudBlobDirectory, mainly in order to fake for testing.
+    /// </summary>
+    internal sealed class AzureCloudBlobDirectory : ICloudBlobDirectory
+    {
+        private readonly CloudBlobDirectory _directory;
+
+        public AzureCloudBlobDirectory(CloudBlobDirectory directory)
+        {
+            _directory = directory;
+        }
+
+        public ICloudBlobDirectory GetDirectoryReference(string directoryName)
+        {
+            return new AzureCloudBlobDirectory(_directory.GetDirectoryReference(directoryName));
+        }
+
+        public IEnumerable<IListBlobItem> ListBlobs(bool useFlatListing = false)
+        {
+            return _directory.ListBlobs(useFlatListing);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
new file mode 100644
index 0000000..4e860a9
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
@@ -0,0 +1,97 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   http://www.apache.org/licenses/LICENSE-2.0
+// 
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using System;
+using System.IO;
+using Microsoft.WindowsAzure.Storage.Auth;
+using Microsoft.WindowsAzure.Storage.Blob;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy class for <see cref="CloudBlockBlob"/>, created mainly
+    /// for unit testing purposes.
+    /// </summary>
+    internal sealed class AzureCloudBlockBlob : ICloudBlockBlob
+    {
+        private readonly CloudBlockBlob _blob;
+
+        public ICloudBlob Blob
+        {
+            get
+            {
+                return _blob;
+            }
+        }
+
+        public BlobProperties Properties
+        {
+            get
+            {
+                return _blob.Properties;
+            }
+        }
+
+        public CopyState CopyState
+        {
+            get
+            {
+                return _blob.CopyState;
+            }
+        }
+
+        public AzureCloudBlockBlob(Uri uri, StorageCredentials credentials)
+        {
+            _blob = new CloudBlockBlob(uri, credentials);
+        }
+
+        public bool Exists()
+        {
+            return _blob.Exists();
+        }
+
+        public void Delete()
+        {
+            _blob.Delete();
+        }
+
+        public void DeleteIfExists()
+        {
+            _blob.DeleteIfExists();
+        }
+
+        public string StartCopy(Uri source)
+        {
+            return _blob.StartCopy(source);
+        }
+
+        public void DownloadToFile(string path, FileMode mode)
+        {
+            _blob.DownloadToFile(path, mode);
+        }
+
+        public void UploadFromFile(string path, FileMode mode)
+        {
+            _blob.UploadFromFile(path, mode);
+        }
+
+        public void FetchAttributes()
+        {
+            _blob.FetchAttributes();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobClient.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobClient.cs
new file mode 100644
index 0000000..0acc2c3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobClient.cs
@@ -0,0 +1,64 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   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.
+
+using System;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Auth;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy interface for CloudBlobClient, mainly in order to fake for unit testing.
+    /// </summary>
+    [DefaultImplementation(typeof(AzureCloudBlobClient))]
+    internal interface ICloudBlobClient
+    {
+        /// <summary>
+        /// Returns the StorageCredentials used to instantiate the <see cref="ICloudBlobClient"/>.
+        /// </summary>
+        StorageCredentials Credentials { get; }
+
+        /// <summary>
+        /// Returns the Base URI for the <see cref="ICloudBlobClient"/>.
+        /// </summary>
+        Uri BaseUri { get; }
+
+        /// <summary>
+        /// Gets a container reference for a storage account.
+        /// </summary>
+        /// <param name="containerName">Name of the container</param>
+        /// <returns>A reference to the blob container</returns>
+        ICloudBlobContainer GetContainerReference(string containerName);
+
+        /// <summary>
+        /// Gets a reference to a block blob. Note that the metadata of the blob
+        /// will not be filled in, since this method does not do a round trip
+        /// to the server.
+        /// </summary>
+        /// <param name="uri">The absolute URI to the block blob</param>
+        /// <returns>A reference to the block blob</returns>
+        ICloudBlockBlob GetBlockBlobReference(Uri uri);
+
+        /// <summary>
+        /// Paginates a blob listing with prefix.
+        /// </summary>
+        BlobResultSegment ListBlobsSegmented(string prefix, bool useFlatListing, BlobListingDetails blobListingDetails, int? maxResults, 
+            BlobContinuationToken continuationToken, BlobRequestOptions blobRequestOptions, OperationContext operationContext);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobContainer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobContainer.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobContainer.cs
new file mode 100644
index 0000000..ce3cf2c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobContainer.cs
@@ -0,0 +1,39 @@
+// 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.
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy interface for CloudBlobContainer, mainly in order to fake for testing.
+    /// </summary>
+    internal interface ICloudBlobContainer
+    {
+        /// <summary>
+        /// Deletes the <see cref="ICloudBlobContainer"/> if it exists.
+        /// Does a round trip to the Blob Server.
+        /// </summary>
+        void DeleteIfExists();
+
+        /// <summary>
+        /// Gets a reference to a blob "directory." Note that Azure Blob does not actually support
+        /// the concept of directories, so in reality this is more of a convenience method.
+        /// </summary>
+        /// <param name="directoryName">Name of the "directory"</param>
+        /// <returns>The reference to a "directory"</returns>
+        ICloudBlobDirectory GetDirectoryReference(string directoryName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobDirectory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobDirectory.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobDirectory.cs
new file mode 100644
index 0000000..f9d3c1b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlobDirectory.cs
@@ -0,0 +1,49 @@
+// 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.
+
+using System.Collections.Generic;
+using Microsoft.WindowsAzure.Storage.Blob;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy interface for CloudBlobDirectory, mainly in order to fake for testing.
+    /// </summary>
+    internal interface ICloudBlobDirectory
+    {
+        /// <summary>
+        /// Gets a reference to a Blob "directory." Note that Azure Blob does not actually support
+        /// the concept of directories, so in reality this is more of a convenience method.
+        /// </summary>
+        /// <param name="directoryName">The name of the blob "directory"</param>
+        /// <returns>A reference to the blob "directory"</returns>
+        ICloudBlobDirectory GetDirectoryReference(string directoryName);
+
+        /// <summary>
+        /// Lists the blobs.
+        /// </summary>
+        /// <param name="useFlatListing">
+        /// If true, recursively lists all blobs.
+        /// Otherwise, "directories" can be listed as well.
+        /// </param>
+        /// <returns>
+        /// An <see cref="IEnumerable{T}"/> of blob items that can be either a type of blob
+        /// or blob directories.
+        /// </returns>
+        IEnumerable<IListBlobItem> ListBlobs(bool useFlatListing = false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
new file mode 100644
index 0000000..2941e6e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
@@ -0,0 +1,91 @@
+// 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.
+
+using System;
+using System.IO;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Blob;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
+{
+    /// <summary>
+    /// A proxy interface for <see cref="CloudBlockBlob"/>,
+    /// mainly used for unit testing purposes.
+    /// </summary>
+    internal interface ICloudBlockBlob
+    {
+        /// <summary>
+        /// The actual, underlying <see cref="ICloudBlob"/>. Mainly a test hook.
+        /// </summary>
+        ICloudBlob Blob { get; }
+
+        /// <summary>
+        /// The <see cref="BlobProperties"/> of the blob. Note that this metadata
+        /// will only be fetched if <see cref="FetchAttributes"/> is called first.
+        /// </summary>
+        BlobProperties Properties { get; }
+
+        /// <summary>
+        /// The <see cref="CopyState"/> of the blob. Note that this metadata
+        /// will only be fetched if <see cref="FetchAttributes"/> is called first.
+        /// </summary>
+        CopyState CopyState { get; }
+
+        /// <summary>
+        /// Makes a round trip to the server to test if the blob exists.
+        /// </summary>
+        /// <returns>True if exists. False otherwise.</returns>
+        bool Exists();
+
+        /// <summary>
+        /// Deletes the <see cref="ICloudBlockBlob"/> from the server.
+        /// </summary>
+        /// <exception cref="StorageException">If blob does not exist</exception>
+        void Delete();
+
+        /// <summary>
+        /// Deletes the <see cref="ICloudBlockBlob"/> from the server, only if it exists.
+        /// </summary>
+        void DeleteIfExists();
+
+        /// <summary>
+        /// Starts the process to copy a <see cref="ICloudBlockBlob"/> to another <see cref="ICloudBlockBlob"/>.
+        /// </summary>
+        /// <param name="source">The URI of the source <see cref="ICloudBlockBlob"/></param>
+        /// <returns>The TaskID of the copy operation</returns>
+        string StartCopy(Uri source);
+
+        /// <summary>
+        /// Downloads the <see cref="ICloudBlockBlob"/> to a local file.
+        /// </summary>
+        /// <param name="path">Path to local file</param>
+        /// <param name="mode">Mode of the file</param>
+        void DownloadToFile(string path, FileMode mode);
+
+        /// <summary>
+        /// Uploads to an <see cref="ICloudBlockBlob"/> from a local file.
+        /// </summary>
+        /// <param name="path">Path to local file</param>
+        /// <param name="mode">Mode of the file</param>
+        void UploadFromFile(string path, FileMode mode);
+
+        /// <summary>
+        /// Makes a round trip to the server to fetch the metadata of the <see cref="ICloudBlockBlob"/>.
+        /// </summary>
+        void FetchAttributes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/Parameters/AzureStorageConnectionString.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/Parameters/AzureStorageConnectionString.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/Parameters/AzureStorageConnectionString.cs
new file mode 100644
index 0000000..c7e0540
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/Parameters/AzureStorageConnectionString.cs
@@ -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.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob.Parameters
+{
+    [NamedParameter("The connection string for Azure Storage")]
+    public sealed class AzureStorageConnectionString : Name<string>
+    {
+        private AzureStorageConnectionString()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/DefaultAzureBlobRetryPolicy.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/DefaultAzureBlobRetryPolicy.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/DefaultAzureBlobRetryPolicy.cs
new file mode 100644
index 0000000..b6fa604
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/DefaultAzureBlobRetryPolicy.cs
@@ -0,0 +1,46 @@
+// 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.
+
+using System;
+using Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.RetryPolicies;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob.RetryPolicy
+{
+    internal class DefaultAzureBlobRetryPolicy : IAzureBlobRetryPolicy
+    {
+        private readonly IRetryPolicy _retryPolicy;
+
+        [Inject]
+        private DefaultAzureBlobRetryPolicy()
+        {
+            _retryPolicy = new ExponentialRetry();
+        } 
+
+        public IRetryPolicy CreateInstance()
+        {
+            return new ExponentialRetry();
+        }
+
+        public bool ShouldRetry(int currentRetryCount, int statusCode, Exception lastException, out TimeSpan retryInterval,
+            OperationContext operationContext)
+        {
+            return _retryPolicy.ShouldRetry(currentRetryCount,statusCode, lastException, out retryInterval, operationContext);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/IAzureBlobRetryPolicy.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/IAzureBlobRetryPolicy.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/IAzureBlobRetryPolicy.cs
new file mode 100644
index 0000000..40dba19
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/RetryPolicy/IAzureBlobRetryPolicy.cs
@@ -0,0 +1,27 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   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.
+
+using Microsoft.WindowsAzure.Storage.RetryPolicies;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureBlob.RetryPolicy
+{
+    [DefaultImplementation(typeof(DefaultAzureBlobRetryPolicy))]
+    public interface IAzureBlobRetryPolicy : IRetryPolicy
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj b/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
index 7ed8e77..90f4f29 100644
--- a/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
+++ b/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
@@ -32,8 +32,36 @@ under the License.
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <ItemGroup>
+    <Reference Include="Microsoft.Azure.KeyVault.Core, Version=1.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Azure.KeyVault.Core.1.0.0\lib\net40\Microsoft.Azure.KeyVault.Core.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.Edm, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Data.Edm.5.6.4\lib\net40\Microsoft.Data.Edm.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.OData, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Data.OData.5.6.4\lib\net40\Microsoft.Data.OData.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.Services.Client, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Data.Services.Client.5.6.4\lib\net40\Microsoft.Data.Services.Client.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.WindowsAzure.Storage, Version=6.1.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\WindowsAzure.Storage.6.1.0\lib\net40\Microsoft.WindowsAzure.Storage.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Newtonsoft.Json, Version=6.0.0.0, Culture=neutral, PublicKeyToken=30ad4fe6b2a6aeed, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Newtonsoft.Json.6.0.8\lib\net45\Newtonsoft.Json.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Spatial, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\System.Spatial.5.6.4\lib\net40\System.Spatial.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System.Xml.Linq" />
     <Reference Include="System.Data.DataSetExtensions" />
     <Reference Include="Microsoft.CSharp" />
@@ -41,6 +69,19 @@ under the License.
     <Reference Include="System.Xml" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="FileSystem\AzureBlob\AzureBlockBlobFileSystemConfiguration.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureBlobType.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureBlockBlobFileSystemConfigurationProvider.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureCloudBlockBlob.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureBlockBlobFileSystem.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureCloudBlobClient.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureCloudBlobDirectory.cs" />
+    <Compile Include="FileSystem\AzureBlob\ICloudBlockBlob.cs" />
+    <Compile Include="FileSystem\AzureBlob\RetryPolicy\IAzureBlobRetryPolicy.cs" />
+    <Compile Include="FileSystem\AzureBlob\RetryPolicy\DefaultAzureBlobRetryPolicy.cs" />
+    <Compile Include="FileSystem\AzureBlob\ICloudBlobClient.cs" />
+    <Compile Include="FileSystem\AzureBlob\AzureCloudBlobContainer.cs" />
+    <Compile Include="FileSystem\AzureBlob\Parameters\AzureStorageConnectionString.cs" />
     <Compile Include="FileSystem\FileStatus.cs" />
     <Compile Include="FileSystem\Hadoop\CommandResult.cs" />
     <Compile Include="FileSystem\Hadoop\HDFSConfigurationWithoutDriverBinding.cs" />
@@ -51,7 +92,9 @@ under the License.
     <Compile Include="FileSystem\Hadoop\Parameters\CommandTimeOut.cs" />
     <Compile Include="FileSystem\Hadoop\Parameters\HadoopHome.cs" />
     <Compile Include="FileSystem\Hadoop\Parameters\NumberOfRetries.cs" />
+    <Compile Include="FileSystem\AzureBlob\ICloudBlobDirectory.cs" />
     <Compile Include="FileSystem\IFileSystem.cs" />
+    <Compile Include="FileSystem\AzureBlob\ICloudBlobContainer.cs" />
     <Compile Include="FileSystem\Local\LocalFileSystem.cs" />
     <Compile Include="FileSystem\Local\LocalFileSystemConfiguration.cs" />
     <Compile Include="PartitionedData\FileSystem\FileSystemInputPartition.cs" />
@@ -76,7 +119,9 @@ under the License.
   </ItemGroup>
   <ItemGroup>
     <None Include="Org.Apache.REEF.IO.nuspec" />
-    <None Include="packages.config" />
+    <None Include="packages.config">
+      <SubType>Designer</SubType>
+    </None>
   </ItemGroup>
   <ItemGroup>
     <ProjectReference Include="$(SolutionDir)\Org.Apache.REEF.Tang\Org.Apache.REEF.Tang.csproj">
@@ -92,5 +137,7 @@ under the License.
       <Name>Org.Apache.REEF.Common</Name>
     </ProjectReference>
   </ItemGroup>
-  <ItemGroup />
+  <ItemGroup>
+    <WCFMetadata Include="Service References\" />
+  </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/Properties/AssemblyInfo.cs b/lang/cs/Org.Apache.REEF.IO/Properties/AssemblyInfo.cs
index e8641d5..9edf44f 100644
--- a/lang/cs/Org.Apache.REEF.IO/Properties/AssemblyInfo.cs
+++ b/lang/cs/Org.Apache.REEF.IO/Properties/AssemblyInfo.cs
@@ -44,3 +44,9 @@ using System.Runtime.InteropServices;
  "9c03469d0f93a9f5d45aa2c883f50cd158759e93673f759ec4657fd84cc79d2db38ef1a2d914cc" +
  "b7c717846a897e11dd22eb260a7ce2da2dccf0263ea63e2b3f7dac24f28882aa568ef544341d17" +
  "618392a1095f4049ad079d4f4f0b429bb535699155fd6a7652ec7d6c1f1ba2b560f11ef3a86b5945d288cf")]
+// Allow NSubstitute to create proxy implementations
+[assembly: InternalsVisibleTo("DynamicProxyGenAssembly2, PublicKey=002400000480000" +
+ "0940000000602000000240000525341310004000001000100c547cac37abd99c8db225ef2f6c8a36" +
+ "02f3b3606cc9891605d02baa56104f4cfc0734aa39b93bf7852f7d9266654753cc297e7d2edfe0ba" +
+ "c1cdcf9f717241550e0a7b191195b7667bb4f64bcb8e2121380fd1d9d46ad2d92d2d15605093924c" +
+ "ceaf74c4861eff62abf69b9291ed0a340e113be11e6a7d3113e92484cf7045cc7")]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.IO/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/packages.config b/lang/cs/Org.Apache.REEF.IO/packages.config
index d952982..2814084 100644
--- a/lang/cs/Org.Apache.REEF.IO/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO/packages.config
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -18,5 +18,12 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
+  <package id="Microsoft.Azure.KeyVault.Core" version="1.0.0" targetFramework="net45" />
+  <package id="Microsoft.Data.Edm" version="5.6.4" targetFramework="net45" />
+  <package id="Microsoft.Data.OData" version="5.6.4" targetFramework="net45" />
+  <package id="Microsoft.Data.Services.Client" version="5.6.4" targetFramework="net45" />
+  <package id="Newtonsoft.Json" version="6.0.8" targetFramework="net45" />
   <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="System.Spatial" version="5.6.4" targetFramework="net45" />
+  <package id="WindowsAzure.Storage" version="6.1.0" targetFramework="net45" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
index 9ca9452..377cc64 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
@@ -33,18 +33,44 @@ under the License.
     <BuildPackage>false</BuildPackage>
   </PropertyGroup>
   <ItemGroup>
+    <Reference Include="Microsoft.Azure.KeyVault.Core, Version=1.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Azure.KeyVault.Core.1.0.0\lib\net40\Microsoft.Azure.KeyVault.Core.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.Edm, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Data.Edm.5.6.4\lib\net40\Microsoft.Data.Edm.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.OData, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Data.OData.5.6.4\lib\net40\Microsoft.Data.OData.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.Services.Client, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Data.Services.Client.5.6.4\lib\net40\Microsoft.Data.Services.Client.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="Microsoft.VisualStudio.QualityTools.UnitTestFramework, Version=10.1.0.0, Culture=neutral, PublicKeyToken=b03f5f7f11d50a3a, processorArchitecture=MSIL" />
-    <Reference Include="Microsoft.WindowsAzure.Storage">
-      <HintPath>$(PackagesDir)\WindowsAzure.Storage.4.3.0\lib\net40\Microsoft.WindowsAzure.Storage.dll</HintPath>
+    <Reference Include="Microsoft.WindowsAzure.Storage, Version=6.1.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\WindowsAzure.Storage.6.1.0\lib\net40\Microsoft.WindowsAzure.Storage.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Newtonsoft.Json, Version=6.0.0.0, Culture=neutral, PublicKeyToken=30ad4fe6b2a6aeed, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Newtonsoft.Json.6.0.8\lib\net45\Newtonsoft.Json.dll</HintPath>
+      <Private>True</Private>
     </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Data" />
     <Reference Include="System.Reactive.Core">
       <HintPath>$(PackagesDir)\Rx-Core.$(RxVersion)\lib\net45\System.Reactive.Core.dll</HintPath>
     </Reference>
     <Reference Include="System.Reactive.Interfaces">
       <HintPath>$(PackagesDir)\Rx-Interfaces.$(RxVersion)\lib\net45\System.Reactive.Interfaces.dll</HintPath>
     </Reference>
+    <Reference Include="System.Spatial, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\System.Spatial.5.6.4\lib\net40\System.Spatial.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
   </ItemGroup>
   <ItemGroup>
     <Compile Include="Functional\Bridge\HelloSimpleEventHandlers.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3c35b7ee/lang/cs/Org.Apache.REEF.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/packages.config b/lang/cs/Org.Apache.REEF.Tests/packages.config
index f31f794..6085d73 100644
--- a/lang/cs/Org.Apache.REEF.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Tests/packages.config
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -18,14 +18,15 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="Microsoft.Data.Edm" version="5.6.3" targetFramework="net45" />
-  <package id="Microsoft.Data.OData" version="5.6.3" targetFramework="net45" />
-  <package id="Microsoft.Data.Services.Client" version="5.6.3" targetFramework="net45" />
-  <package id="Microsoft.WindowsAzure.ConfigurationManager" version="2.0.3" targetFramework="net45" />
+  <package id="Microsoft.Azure.KeyVault.Core" version="1.0.0" targetFramework="net45" />
+  <package id="Microsoft.Data.Edm" version="5.6.4" targetFramework="net45" />
+  <package id="Microsoft.Data.OData" version="5.6.4" targetFramework="net45" />
+  <package id="Microsoft.Data.Services.Client" version="5.6.4" targetFramework="net45" />
+  <package id="Newtonsoft.Json" version="6.0.8" targetFramework="net45" />
   <package id="protobuf-net" version="2.0.0.668" targetFramework="net45" />
   <package id="Rx-Core" version="2.2.5" targetFramework="net45" />
   <package id="Rx-Interfaces" version="2.2.5" targetFramework="net45" />
   <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
-  <package id="System.Spatial" version="5.6.3" targetFramework="net45" />
-  <package id="WindowsAzure.Storage" version="4.3.0" targetFramework="net45" />
+  <package id="System.Spatial" version="5.6.4" targetFramework="net45" />
+  <package id="WindowsAzure.Storage" version="6.1.0" targetFramework="net45" />
 </packages>
\ No newline at end of file