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 2018/06/06 17:40:28 UTC

[reef] branch master updated: [REEF-2016] Fix CreateUriForPath methods of different IFileSystems to not append prefix if given path already contains it

This is an automated email from the ASF dual-hosted git repository.

weimer pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/reef.git


The following commit(s) were added to refs/heads/master by this push:
     new 6d46de4  [REEF-2016] Fix CreateUriForPath methods of different IFileSystems to not append prefix if given path already contains it
6d46de4 is described below

commit 6d46de48dcde52ea6d8f41f8545f46e6f55778b7
Author: Dwaipayan Mukhopadhyay <dw...@gmail.com>
AuthorDate: Wed Jun 6 10:40:25 2018 -0700

    [REEF-2016] Fix CreateUriForPath methods of different IFileSystems to not append prefix if given path already contains it
    
    
      * Fix CreateUriForPath methods of different Ifilesystems to not append prefix is given path already ontains it
    
    JIRA:
      [REEF-2016](https://issues.apache.org/jira/browse/REEF-2016)
    
    Pull Request:
      This closes #1456
---
 .../TestAzureBlockBlobFileSystem.cs                | 61 +++++++++++++++++-----
 .../TestAzureDataLakeFileSystem.cs                 | 47 ++++++++++++++---
 .../TestHadoopFileSystem.cs                        | 21 ++++++++
 .../AzureBlob/AzureBlockBlobFileSystem.cs          | 42 ++++++++++++++-
 .../AzureDataLake/AzureDataLakeFileSystem.cs       | 31 ++++++++---
 .../FileSystem/Hadoop/HadoopFileSystem.cs          | 22 +++++---
 6 files changed, 189 insertions(+), 35 deletions(-)

diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
index 6623051..c5b23b9 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
@@ -35,12 +35,22 @@ namespace Org.Apache.REEF.IO.Tests
     /// </summary>
     public sealed class TestAzureBlockBlobFileSystem
     {
-        private static readonly Uri FakeUri = new Uri("http://fake.com/container/file");
+        private static readonly Uri FakeUri = new Uri("http://storageacct.blob.core.windows.net/container/file");
+        private TestContext testContext;
+
+        private static Uri BaseUri
+        {
+            get { return new Uri(FakeUri.GetLeftPart(UriPartial.Authority)); }
+        }
+
+        public TestAzureBlockBlobFileSystem()
+        {
+            this.testContext = new TestContext();
+        }
 
         [Fact]
         public void TestCreate()
         {
-            var testContext = new TestContext();
             Stream stream = testContext.GetAzureFileSystem().Create(FakeUri);
             testContext.TestCloudBlockBlob.Received(1).Create();
             Assert.Equal(testContext.TestCreateStream, stream);
@@ -49,7 +59,6 @@ namespace Org.Apache.REEF.IO.Tests
         [Fact]
         public void TestOpen()
         {
-            var testContext = new TestContext();
             Stream stream = testContext.GetAzureFileSystem().Open(FakeUri);
             testContext.TestCloudBlockBlob.Received(1).Open();
             Assert.Equal(testContext.TestOpenStream, stream);
@@ -58,7 +67,6 @@ namespace Org.Apache.REEF.IO.Tests
         [Fact]
         public void TestDelete()
         {
-            var testContext = new TestContext();
             testContext.GetAzureFileSystem().Delete(FakeUri);
             testContext.TestCloudBlockBlob.Received(1).Delete();
         }
@@ -66,7 +74,6 @@ namespace Org.Apache.REEF.IO.Tests
         [Fact]
         public void TestExists()
         {
-            var testContext = new TestContext();
             testContext.GetAzureFileSystem().Exists(FakeUri);
             testContext.TestCloudBlockBlob.Received(1).Exists();
         }
@@ -74,7 +81,6 @@ namespace Org.Apache.REEF.IO.Tests
         [Fact]
         public void TestCopyToLocal()
         {
-            var testContext = new TestContext();
             testContext.GetAzureFileSystem().CopyToLocal(FakeUri, "local");
             testContext.TestCloudBlockBlob.Received(1).DownloadToFile("local", FileMode.CreateNew);
         }
@@ -82,7 +88,6 @@ namespace Org.Apache.REEF.IO.Tests
         [Fact]
         public void TestCopyFromLocal()
         {
-            var testContext = new TestContext();
             testContext.GetAzureFileSystem().CopyFromLocal("local", FakeUri);
             testContext.TestCloudBlockBlob.Received(1).UploadFromFile("local", FileMode.Open);
         }
@@ -113,7 +118,6 @@ namespace Org.Apache.REEF.IO.Tests
         [Fact]
         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");
@@ -124,10 +128,43 @@ namespace Org.Apache.REEF.IO.Tests
         }
 
         [Fact]
-        public void TestCreateUriForPath()
+        public void TestCreateUriForAbsolutePathInvalid()
         {
-            var testContext = new TestContext();
-            Assert.Equal(FakeUri, testContext.GetAzureFileSystem().CreateUriForPath(FakeUri.LocalPath));
+            Assert.Throws<ArgumentException>(() => testContext.GetAzureFileSystem().CreateUriForPath("http://www.invalidstorageaccount.com/container/folder1/file1.txt"));
+        }
+
+        [Fact]
+        public void TestCreateUriForAbsolutePath()
+        {
+            Uri uri = new Uri("http://storageacct.blob.core.windows.net/container/folder1/folder2/file.txt");
+            Uri resultUri = testContext.GetAzureFileSystem().CreateUriForPath(uri.AbsoluteUri);
+            Assert.Equal<Uri>(uri, resultUri);
+        }
+
+        [Fact]
+        public void TestCreateUriForRelativePathValid()
+        {
+            string relativePath = "container/folder1/folder2/file.txt";
+            Uri expectedUri = new Uri(BaseUri, relativePath);
+            Uri resultUri = testContext.GetAzureFileSystem().CreateUriForPath(relativePath);
+            Assert.Equal<Uri>(expectedUri, resultUri);
+        }
+
+        [Fact]
+        public void TestCreateUriForRelativePathWithContainerNameTooSmall()
+        {
+            // Container name must be atleast 3 characters.
+            string relativePath = "c1/folder1/folder2/file.txt";
+            Uri expectedUri = new Uri(BaseUri, relativePath);
+            Assert.Throws<ArgumentException>(() => testContext.GetAzureFileSystem().CreateUriForPath(relativePath));
+        }
+
+        [Fact]
+        public void TestCreateUriForRelativePathWithInvalidContainerName()
+        {
+            // Container name cannot contain a colon character.
+            string relativePath = "c:/folder1/folder2/file.txt";
+            Assert.Throws<ArgumentException>(() => testContext.GetAzureFileSystem().CreateUriForPath(relativePath));
         }
 
         private sealed class TestContext
@@ -149,7 +186,7 @@ namespace Org.Apache.REEF.IO.Tests
                 var injector = TangFactory.GetTang().NewInjector(conf);
                 injector.BindVolatileInstance(TestCloudBlobClient);
                 var fs = injector.GetInstance<AzureBlockBlobFileSystem>();
-                TestCloudBlobClient.BaseUri.ReturnsForAnyArgs(new Uri(FakeUri.GetLeftPart(UriPartial.Authority)));
+                TestCloudBlobClient.BaseUri.ReturnsForAnyArgs(BaseUri);
                 TestCloudBlockBlob.Open().Returns(TestOpenStream);
                 TestCloudBlockBlob.Create().Returns(TestCreateStream);
                 TestCloudBlockBlob.Blob.ReturnsForAnyArgs(new CloudBlockBlob(FakeUri));
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs
index 821e6ae..7199f9e 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs
@@ -28,9 +28,21 @@ namespace Org.Apache.REEF.IO.Tests
 {
     public sealed class TestAzureDataLakeFileSystem
     {
-        private static readonly Uri FakeBaseUri = new Uri("http://fakeadls.com");
-        private static readonly Uri FakeDirUri = new Uri(FakeBaseUri, "dir");
-        private static readonly Uri FakeFileUri = new Uri($"{FakeDirUri}/fakefile");
+        private Uri FakeBaseUri
+        {
+            get { return new Uri("adl://" + new TestContext().AdlAccountFQDN); }
+        }
+
+        private Uri FakeDirUri
+        {
+            get { return new Uri(FakeBaseUri, "dir"); }
+        }
+
+        private Uri FakeFileUri
+        {
+            get { return new Uri($"{FakeDirUri}/fakefile"); }
+        }
+
         private readonly TestContext _context = new TestContext();
         private readonly AzureDataLakeFileSystem _fs;
 
@@ -195,11 +207,25 @@ namespace Org.Apache.REEF.IO.Tests
         }
 
         [Fact]
-        public void TestCreateUriForPath()
+        public void TestCreateUriForPathNoPrefix()
         {
             string dirStructure = FakeFileUri.AbsolutePath;
             Uri createdUri = _fs.CreateUriForPath(dirStructure);
-            Assert.Equal(createdUri, new Uri($"adl://{_context.AdlAccountName}{dirStructure}"));
+            Assert.Equal(new Uri(FakeBaseUri, dirStructure), createdUri);
+        }
+
+        [Fact]
+        public void TestCreateUriForPathWithPrefix()
+        {
+            Uri createdUri = _fs.CreateUriForPath(FakeFileUri.AbsolutePath);
+            Assert.Equal(FakeFileUri, createdUri);
+        }
+
+        [Fact]
+        public void TestCreateUriForPathWithInvalidPrefix()
+        {
+            // Scheme for adl accounts must start with adl and not http
+            Assert.Throws<ArgumentException>(() => _fs.CreateUriForPath("http://invalidadlaccount.net/driver3.txt"));
         }
 
         [Fact]
@@ -210,13 +236,18 @@ namespace Org.Apache.REEF.IO.Tests
 
         private sealed class TestContext
         {
-            public readonly string AdlAccountName = "adlAccount";
+            public readonly string AdlAccountName = "adlaccount";
             public readonly AdlsClient MockAdlsClient = Microsoft.Azure.DataLake.Store.MockAdlsFileSystem.MockAdlsClient.GetMockClient();
 
+            public string AdlAccountFQDN
+            {
+                get { return $"{AdlAccountName}.azuredatalakestore.net"; }
+            }
+
             public AzureDataLakeFileSystem GetAdlsFileSystem()
             {
                 var conf = AzureDataLakeFileSystemConfiguration.ConfigurationModule
-                     .Set(AzureDataLakeFileSystemConfiguration.DataLakeStorageAccountName, "adlsAccountName")
+                     .Set(AzureDataLakeFileSystemConfiguration.DataLakeStorageAccountName, AdlAccountFQDN)
                     .Set(AzureDataLakeFileSystemConfiguration.Tenant, "tenant")
                     .Set(AzureDataLakeFileSystemConfiguration.ClientId, "clientId")
                     .Set(AzureDataLakeFileSystemConfiguration.SecretKey, "secretKey")
@@ -225,7 +256,7 @@ namespace Org.Apache.REEF.IO.Tests
                 var testDataLakeStoreClient = Substitute.For<IDataLakeStoreClient>();
                 injector.BindVolatileInstance(testDataLakeStoreClient);
                 testDataLakeStoreClient.GetAdlsClient().ReturnsForAnyArgs(MockAdlsClient);
-                testDataLakeStoreClient.AccountFQDN.Returns(AdlAccountName);
+                testDataLakeStoreClient.AccountFQDN.Returns(AdlAccountFQDN);
                 var fs = injector.GetInstance<AzureDataLakeFileSystem>();
                 return fs;
             }
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestHadoopFileSystem.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestHadoopFileSystem.cs
index 889fe10..94297a5 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/TestHadoopFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestHadoopFileSystem.cs
@@ -18,6 +18,7 @@
 using System;
 using System.IO;
 using System.Linq;
+using System.Text.RegularExpressions;
 using Org.Apache.REEF.IO.FileSystem;
 using Org.Apache.REEF.IO.FileSystem.Hadoop;
 using Org.Apache.REEF.Tang.Annotations;
@@ -142,6 +143,26 @@ namespace Org.Apache.REEF.IO.Tests
             Assert.Throws<NotImplementedException>(() => _fileSystem.Create(GetTempUri()));
         }
 
+        [Fact(Skip = "These tests need to be run in an environment with HDFS installed.")]
+        public void CreateUriForPathNoPrefix()
+        {
+            Uri uri = _fileSystem.CreateUriForPath("/tmp/TestHadoop");
+            Assert.True(new Regex("hdfs://[a-z]+:\\d+/tmp/TestHadoop").Match(uri.AbsoluteUri).Success);
+        }
+
+        [Fact(Skip = "These tests need to be run in an environment with HDFS installed.")]
+        public void TestCreateUriForPathInvalid()
+        {
+            Assert.Throws<ArgumentException>(() => _fileSystem.CreateUriForPath("http://www.invalidhadoopfs.com/container/folder1/file1.txt"));
+        }
+
+        [Fact(Skip = "These tests need to be run in an environment with HDFS installed.")]
+        public void CreateUriForPathWithPrefix()
+        {
+            string uriString = "hdfs://localhost:9000/tmp/TestHadoop";
+            Assert.Equal(new Uri(uriString), _fileSystem.CreateUriForPath(uriString));
+        }
+
         /// <summary>
         /// This test is to make sure with the HadoopFileSystemConfiguration, HadoopFileSystem can be injected.
         /// </summary>
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
index d333839..9292f42 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
@@ -35,6 +35,8 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
     {
         private readonly ICloudBlobClient _client;
 
+        private const char UrlPathSeparator = '/';
+
         [Inject]
         private AzureBlockBlobFileSystem(ICloudBlobClient client)
         {
@@ -186,12 +188,48 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
         /// <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.
+        /// If path is null or the prefix doesn't match the base uri in the FileSystem, throw ArgumentException.
         /// </summary>
-        /// <param name="path">The relative path to the remote file, including the container</param>
+        /// <param name="path">The relative or absolute 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('/'));
+            if (path == null)
+            {
+                throw new ArgumentNullException(nameof(path), "Specified path is null");
+            }
+
+            Uri resultUri = null;
+            try
+            {
+                resultUri = new Uri(path);
+            }
+            catch (UriFormatException)
+            {
+                resultUri = new Uri(_client.BaseUri, path);
+            }
+
+            if (!resultUri.AbsoluteUri.StartsWith(_client.BaseUri.AbsoluteUri))
+            {
+                throw new ArgumentException($"Given URI must begin with valid prefix ({_client.BaseUri.AbsoluteUri})", nameof(path));
+            }
+
+            if (resultUri.Segments.Count() < 2)
+            {
+                throw new ArgumentException("Input path must have a container name", nameof(path));
+            }
+
+            string containerName = resultUri.Segments[1].Trim(UrlPathSeparator);
+            NameValidator.ValidateContainerName(containerName);
+            NameValidator.ValidateBlobName(resultUri.PathAndQuery);
+
+            // If the last segment does not end with a '/', we require it to be a valid file name.
+            if (!resultUri.PathAndQuery.EndsWith(UrlPathSeparator.ToString()))
+            {
+                NameValidator.ValidateFileName(resultUri.Segments[resultUri.Segments.Length - 1]);
+            }
+
+            return resultUri;
         }
 
         /// <summary>
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs
index 74dd70e..f6ad141 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs
@@ -34,6 +34,11 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
         private readonly IDataLakeStoreClient _client;
         private readonly AdlsClient _adlsClient;
 
+        private string UriPrefix
+        {
+            get { return $"adl://{_client.AccountFQDN}"; }
+        }
+
         [Inject]
         private AzureDataLakeFileSystem(IDataLakeStoreClient client)
         {
@@ -205,7 +210,7 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
         /// <summary>
         /// Create Uri from a given file path.
         /// The file path can be full with prefix or relative without prefix.
-        /// If null is passed as the path, ArgumentException will be thrown.
+        /// If path is null or the prefix doesn't match the prefix in the FileSystem, throw ArgumentException.
         /// </summary>
         /// <exception cref="ArgumentNullException">If specified path is null</exception>
         public Uri CreateUriForPath(string path)
@@ -214,7 +219,24 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
             {
                 throw new ArgumentNullException(nameof(path), "Specified path is null");
             }
-            return new Uri($"{GetUriPrefix()}/{path.TrimStart('/')}");
+
+            Uri resultUri = null;
+
+            try
+            {
+                resultUri = new Uri(path);
+            }
+            catch (UriFormatException)
+            {
+                resultUri = new Uri(new Uri(this.UriPrefix), path);
+            }
+
+            if (!resultUri.AbsoluteUri.StartsWith(this.UriPrefix))
+            {
+                throw new ArgumentException($"Given URI must begin with valid prefix ({this.UriPrefix})", nameof(path));
+            }
+
+            return resultUri;
         }
 
         /// <summary>
@@ -236,10 +258,5 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
 
             return new FileStatus(entrySummary.LastModifiedTime.Value, entrySummary.Length);
         }
-
-        private string GetUriPrefix()
-        {
-            return $"adl://{_client.AccountFQDN}";
-        }
     }
 }
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HadoopFileSystem.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HadoopFileSystem.cs
index f7f8274..cd80b62 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HadoopFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HadoopFileSystem.cs
@@ -55,22 +55,32 @@ namespace Org.Apache.REEF.IO.FileSystem.Hadoop
         /// <summary>
         /// Create Uri from a given file name
         /// If the path already contains prefix, use it directly and verify the prefix after it is created.
-        /// Otherwise add the prefix in fron of the relative path.
+        /// Otherwise add the prefix in front of the relative path.
         /// If path is null or the prefix doesn't match the prefix in the FileSystem, throw ArgumentException
         /// </summary>
         /// <param name="path"></param>
         /// <returns></returns>
         public Uri CreateUriForPath(string path)
         {
-            Logger.Log(Level.Info, string.Format(CultureInfo.CurrentCulture, "CreateUriForPath with path: {0}, _uriPrefix: {1}.", path, _uriPrefix));
             if (path == null)
             {
-                throw new ArgumentException("null path passed in CreateUriForPath");
+                throw new ArgumentNullException(nameof(path));
             }
 
-            Uri uri;
-            uri = new Uri(path);
-            Logger.Log(Level.Info, string.Format(CultureInfo.CurrentCulture, "Uri {0} created in CreateUriForPath.", uri));
+            Uri uri = null;
+            try
+            {
+                uri = new Uri(path);
+            }
+            catch (UriFormatException)
+            {
+                uri = new Uri(new Uri(_uriPrefix), path);
+            }
+
+            if (!uri.AbsoluteUri.StartsWith(_uriPrefix))
+            {
+                throw new ArgumentException($"Given URI does not begin with valid prefix ({_uriPrefix})");
+            }
 
             return uri;
         }

-- 
To stop receiving notification emails like this one, please contact
weimer@apache.org.