瀏覽代碼

HDDS-807. Period should be an invalid character in bucket names. Contributed by Siddharth Wagle.

Márton Elek 6 年之前
父節點
當前提交
c449cdebe6

+ 11 - 4
hadoop-hdds/docs/content/OzoneFS.md

@@ -51,7 +51,7 @@ Please add the following entry to the core-site.xml.
 </property>
 </property>
 {{< /highlight >}}
 {{< /highlight >}}
 
 
-This will make this bucket to be the default file system for HDFS dfs commands and register the o3fs file system type..
+This will make this bucket to be the default file system for HDFS dfs commands and register the o3fs file system type.
 
 
 You also need to add the ozone-filesystem.jar file to the classpath:
 You also need to add the ozone-filesystem.jar file to the classpath:
 
 
@@ -59,8 +59,6 @@ You also need to add the ozone-filesystem.jar file to the classpath:
 export HADOOP_CLASSPATH=/opt/ozone/share/ozonefs/lib/hadoop-ozone-filesystem-lib-current.*.jar:$HADOOP_CLASSPATH
 export HADOOP_CLASSPATH=/opt/ozone/share/ozonefs/lib/hadoop-ozone-filesystem-lib-current.*.jar:$HADOOP_CLASSPATH
 {{< /highlight >}}
 {{< /highlight >}}
 
 
-
-
 Once the default Filesystem has been setup, users can run commands like ls, put, mkdir, etc.
 Once the default Filesystem has been setup, users can run commands like ls, put, mkdir, etc.
 For example,
 For example,
 
 
@@ -76,7 +74,16 @@ hdfs dfs -mkdir /users
 
 
 
 
 Or put command etc. In other words, all programs like Hive, Spark, and Distcp will work against this file system.
 Or put command etc. In other words, all programs like Hive, Spark, and Distcp will work against this file system.
-Please note that any keys created/deleted in the bucket using methods apart from OzoneFileSystem will show up as diectories and files in the Ozone File System.
+Please note that any keys created/deleted in the bucket using methods apart from OzoneFileSystem will show up as directories and files in the Ozone File System.
+
+Note: Bucket and volume names are not allowed to have a period in them.
+Moreover, the filesystem URI can take a fully qualified form with the OM host and port as a part of the path following the volume name.
+For example,
+
+{{< highlight bash>}}
+hdfs dfs -ls o3fs://bucket.volume.om-host.example.com:5678/key
+{{< /highlight >}}
+
 
 
 ## Legacy mode
 ## Legacy mode
 
 

+ 25 - 0
hadoop-ozone/ozonefs/pom.xml

@@ -165,6 +165,7 @@
     <dependency>
     <dependency>
       <groupId>org.mockito</groupId>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
       <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
       <scope>test</scope>
       <scope>test</scope>
     </dependency>
     </dependency>
     <dependency>
     <dependency>
@@ -188,5 +189,29 @@
       <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
       <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
       <scope>test</scope>
       <scope>test</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-module-junit4</artifactId>
+      <version>1.6.5</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.javassist</groupId>
+          <artifactId>javassist</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-mockito</artifactId>
+      <version>1.6.5</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.hamcrest</groupId>
+          <artifactId>hamcrest-core</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
   </dependencies>
 </project>
 </project>

+ 20 - 5
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java

@@ -17,11 +17,14 @@
  */
  */
 package org.apache.hadoop.fs.ozone;
 package org.apache.hadoop.fs.ozone;
 
 
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
@@ -36,9 +39,6 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
-
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.TokenRenewer;
@@ -100,9 +100,17 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   public OzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr,
   public OzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr,
       String bucketStr, OzoneFSStorageStatistics storageStatistics)
       String bucketStr, OzoneFSStorageStatistics storageStatistics)
       throws IOException {
       throws IOException {
+    this(null, -1, conf, volumeStr, bucketStr, storageStatistics);
+  }
+
+  public OzoneClientAdapterImpl(String omHost, int omPort,
+      OzoneConfiguration conf, String volumeStr, String bucketStr,
+      OzoneFSStorageStatistics storageStatistics) throws IOException {
+
     ClassLoader contextClassLoader =
     ClassLoader contextClassLoader =
         Thread.currentThread().getContextClassLoader();
         Thread.currentThread().getContextClassLoader();
     Thread.currentThread().setContextClassLoader(null);
     Thread.currentThread().setContextClassLoader(null);
+
     try {
     try {
       String replicationTypeConf =
       String replicationTypeConf =
           conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
           conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
@@ -110,8 +118,14 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
 
       int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
       int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
           OzoneConfigKeys.OZONE_REPLICATION_DEFAULT);
           OzoneConfigKeys.OZONE_REPLICATION_DEFAULT);
-      this.ozoneClient =
-          OzoneClientFactory.getRpcClient(conf);
+
+      if (StringUtils.isNotEmpty(omHost) && omPort != -1) {
+        this.ozoneClient =
+            OzoneClientFactory.getRpcClient(omHost, omPort, conf);
+      } else {
+        this.ozoneClient =
+            OzoneClientFactory.getRpcClient(conf);
+      }
       objectStore = ozoneClient.getObjectStore();
       objectStore = ozoneClient.getObjectStore();
       this.volume = objectStore.getVolume(volumeStr);
       this.volume = objectStore.getVolume(volumeStr);
       this.bucket = volume.getBucket(bucketStr);
       this.bucket = volume.getBucket(bucketStr);
@@ -124,6 +138,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
 
   }
   }
 
 
+
   @Override
   @Override
   public void close() throws IOException {
   public void close() throws IOException {
     ozoneClient.close();
     ozoneClient.close();

+ 30 - 7
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java

@@ -34,6 +34,7 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.stream.Stream;
 
 
+import org.apache.commons.lang3.math.NumberUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -88,33 +89,53 @@ public class OzoneFileSystem extends FileSystem {
   private OzoneClientAdapter adapter;
   private OzoneClientAdapter adapter;
   private boolean securityEnabled;
   private boolean securityEnabled;
 
 
-
   private OzoneFSStorageStatistics storageStatistics;
   private OzoneFSStorageStatistics storageStatistics;
 
 
   private static final Pattern URL_SCHEMA_PATTERN =
   private static final Pattern URL_SCHEMA_PATTERN =
-      Pattern.compile("(.+)\\.([^\\.]+)");
+      Pattern.compile("([^\\.]+)\\.([^\\.]+)\\.{0,1}(.*)");
+
+  private static final String URI_EXCEPTION_TEXT = "Ozone file system url " +
+      "should be either one of the two forms: " +
+      "o3fs://bucket.volume/key  OR " +
+      "o3fs://bucket.volume.om-host.example.com:5678/key";
 
 
   @Override
   @Override
   public void initialize(URI name, Configuration conf) throws IOException {
   public void initialize(URI name, Configuration conf) throws IOException {
     super.initialize(name, conf);
     super.initialize(name, conf);
     setConf(conf);
     setConf(conf);
     Objects.requireNonNull(name.getScheme(), "No scheme provided in " + name);
     Objects.requireNonNull(name.getScheme(), "No scheme provided in " + name);
-    assert getScheme().equals(name.getScheme());
+    Preconditions.checkArgument(getScheme().equals(name.getScheme()),
+        "Invalid scheme provided in " + name);
 
 
     String authority = name.getAuthority();
     String authority = name.getAuthority();
 
 
     Matcher matcher = URL_SCHEMA_PATTERN.matcher(authority);
     Matcher matcher = URL_SCHEMA_PATTERN.matcher(authority);
 
 
     if (!matcher.matches()) {
     if (!matcher.matches()) {
-      throw new IllegalArgumentException("Ozone file system url should be "
-          + "in the form o3fs://bucket.volume");
+      throw new IllegalArgumentException(URI_EXCEPTION_TEXT);
     }
     }
     String bucketStr = matcher.group(1);
     String bucketStr = matcher.group(1);
     String volumeStr = matcher.group(2);
     String volumeStr = matcher.group(2);
+    String remaining = matcher.groupCount() == 3 ? matcher.group(3) : null;
+
+    String omHost = null;
+    String omPort = String.valueOf(-1);
+    if (StringUtils.isNotEmpty(remaining)) {
+      String[] parts = remaining.split(":");
+      if (parts.length != 2) {
+        throw new IllegalArgumentException(URI_EXCEPTION_TEXT);
+      }
+      omHost = parts[0];
+      omPort = parts[1];
+      if (!NumberUtils.isParsable(omPort)) {
+        throw new IllegalArgumentException(URI_EXCEPTION_TEXT);
+      }
+    }
 
 
     try {
     try {
       uri = new URIBuilder().setScheme(OZONE_URI_SCHEME)
       uri = new URIBuilder().setScheme(OZONE_URI_SCHEME)
-          .setHost(authority).build();
+        .setHost(authority)
+        .build();
       LOG.trace("Ozone URI for ozfs initialization is " + uri);
       LOG.trace("Ozone URI for ozfs initialization is " + uri);
 
 
       //isolated is the default for ozonefs-lib-legacy which includes the
       //isolated is the default for ozonefs-lib-legacy which includes the
@@ -159,11 +180,13 @@ public class OzoneFileSystem extends FileSystem {
         } else {
         } else {
           ozoneConfiguration = new OzoneConfiguration(conf);
           ozoneConfiguration = new OzoneConfiguration(conf);
         }
         }
+
         SecurityConfig secConfig = new SecurityConfig(ozoneConfiguration);
         SecurityConfig secConfig = new SecurityConfig(ozoneConfiguration);
         if (secConfig.isSecurityEnabled()) {
         if (secConfig.isSecurityEnabled()) {
           this.securityEnabled = true;
           this.securityEnabled = true;
         }
         }
-        this.adapter = new OzoneClientAdapterImpl(ozoneConfiguration,
+        this.adapter = new OzoneClientAdapterImpl(omHost,
+            Integer.parseInt(omPort), ozoneConfiguration,
             volumeStr, bucketStr, storageStatistics);
             volumeStr, bucketStr, storageStatistics);
       }
       }
 
 

+ 111 - 0
hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithMocks.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.ozone;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Ozone File system tests that are light weight and use mocks.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ OzoneClientFactory.class, UserGroupInformation.class })
+public class TestOzoneFileSystemWithMocks {
+
+  @Test
+  public void testFSUriWithHostPortOverrides() throws Exception {
+    Configuration conf = new OzoneConfiguration();
+    OzoneClient ozoneClient = mock(OzoneClient.class);
+    ObjectStore objectStore = mock(ObjectStore.class);
+    OzoneVolume volume = mock(OzoneVolume.class);
+    OzoneBucket bucket = mock(OzoneBucket.class);
+
+    when(ozoneClient.getObjectStore()).thenReturn(objectStore);
+    when(objectStore.getVolume(eq("volume1"))).thenReturn(volume);
+    when(volume.getBucket("bucket1")).thenReturn(bucket);
+
+    PowerMockito.mockStatic(OzoneClientFactory.class);
+    PowerMockito.when(OzoneClientFactory.getRpcClient(eq("local.host"),
+        eq(5899), eq(conf))).thenReturn(ozoneClient);
+
+    UserGroupInformation ugi = mock(UserGroupInformation.class);
+    PowerMockito.mockStatic(UserGroupInformation.class);
+    PowerMockito.when(UserGroupInformation.getCurrentUser()).thenReturn(ugi);
+    when(ugi.getShortUserName()).thenReturn("user1");
+
+    URI uri = new URI("o3fs://bucket1.volume1.local.host:5899");
+
+    FileSystem fileSystem = FileSystem.get(uri, conf);
+    OzoneFileSystem ozfs = (OzoneFileSystem) fileSystem;
+
+    assertEquals(ozfs.getUri().getAuthority(),
+        "bucket1.volume1.local.host:5899");
+    PowerMockito.verifyStatic();
+    OzoneClientFactory.getRpcClient("local.host", 5899, conf);
+  }
+
+  @Test
+  public void testFSUriHostVersionDefault() throws Exception {
+    Configuration conf = new OzoneConfiguration();
+    OzoneClient ozoneClient = mock(OzoneClient.class);
+    ObjectStore objectStore = mock(ObjectStore.class);
+    OzoneVolume volume = mock(OzoneVolume.class);
+    OzoneBucket bucket = mock(OzoneBucket.class);
+
+    when(ozoneClient.getObjectStore()).thenReturn(objectStore);
+    when(objectStore.getVolume(eq("volume1"))).thenReturn(volume);
+    when(volume.getBucket("bucket1")).thenReturn(bucket);
+
+    PowerMockito.mockStatic(OzoneClientFactory.class);
+    PowerMockito.when(OzoneClientFactory.getRpcClient(eq(conf)))
+        .thenReturn(ozoneClient);
+
+    UserGroupInformation ugi = mock(UserGroupInformation.class);
+    PowerMockito.mockStatic(UserGroupInformation.class);
+    PowerMockito.when(UserGroupInformation.getCurrentUser()).thenReturn(ugi);
+    when(ugi.getShortUserName()).thenReturn("user1");
+
+    URI uri = new URI("o3fs://bucket1.volume1/key");
+
+    FileSystem fileSystem = FileSystem.get(uri, conf);
+    OzoneFileSystem ozfs = (OzoneFileSystem) fileSystem;
+
+    assertEquals(ozfs.getUri().getAuthority(), "bucket1.volume1");
+    PowerMockito.verifyStatic();
+    OzoneClientFactory.getRpcClient(conf);
+  }
+}