You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by xy...@apache.org on 2019/03/15 17:10:30 UTC

[hadoop] branch ozone-0.4 updated (8bd13ff -> 522976e)

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

xyao pushed a change to branch ozone-0.4
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


    from 8bd13ff  HDDS-1098. Introduce Retry Policy in Ozone Client. Contributed by Shashikant Banerjee.
     new 3fd9c0d  HDDS-807. Period should be an invalid character in bucket names. Contributed by Siddharth Wagle.
     new 522976e  HDDS-1259. OzoneFS classpath separation is broken by the token validation. Contributed by Elek Marton.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 hadoop-hdds/docs/content/OzoneFS.md                |  15 ++-
 hadoop-ozone/ozonefs/pom.xml                       |  25 +++++
 .../hadoop/fs/ozone/OzoneClientAdapterImpl.java    |  52 ++++++++--
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java    |  53 ++++++----
 .../fs/ozone/TestOzoneFileSystemWithMocks.java     | 111 +++++++++++++++++++++
 5 files changed, 220 insertions(+), 36 deletions(-)
 create mode 100644 hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithMocks.java


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 01/02: HDDS-807. Period should be an invalid character in bucket names. Contributed by Siddharth Wagle.

Posted by xy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xyao pushed a commit to branch ozone-0.4
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 3fd9c0db17168215db90717d63fe63a01cd0fc05
Author: Márton Elek <el...@apache.org>
AuthorDate: Tue Mar 12 19:14:55 2019 +0100

    HDDS-807. Period should be an invalid character in bucket names. Contributed by Siddharth Wagle.
    
    (cherry picked from commit c449cdebe68fb5fe2a7bd83715f6c3f4d5b523d9)
---
 hadoop-hdds/docs/content/OzoneFS.md                |  15 ++-
 hadoop-ozone/ozonefs/pom.xml                       |  25 +++++
 .../hadoop/fs/ozone/OzoneClientAdapterImpl.java    |  25 ++++-
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java    |  37 +++++--
 .../fs/ozone/TestOzoneFileSystemWithMocks.java     | 111 +++++++++++++++++++++
 5 files changed, 197 insertions(+), 16 deletions(-)

diff --git a/hadoop-hdds/docs/content/OzoneFS.md b/hadoop-hdds/docs/content/OzoneFS.md
index 23d12b3..2ca0406 100644
--- a/hadoop-hdds/docs/content/OzoneFS.md
+++ b/hadoop-hdds/docs/content/OzoneFS.md
@@ -51,7 +51,7 @@ Please add the following entry to the core-site.xml.
 </property>
 {{< /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:
 
@@ -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
 {{< /highlight >}}
 
-
-
 Once the default Filesystem has been setup, users can run commands like ls, put, mkdir, etc.
 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.
-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
 
diff --git a/hadoop-ozone/ozonefs/pom.xml b/hadoop-ozone/ozonefs/pom.xml
index a3681d6..e7354c0 100644
--- a/hadoop-ozone/ozonefs/pom.xml
+++ b/hadoop-ozone/ozonefs/pom.xml
@@ -165,6 +165,7 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -188,5 +189,29 @@
       <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
       <scope>test</scope>
     </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>
 </project>
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
index 3b034ed..1ea1261 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
@@ -17,11 +17,14 @@
  */
 package org.apache.hadoop.fs.ozone;
 
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.HashMap;
 import java.util.Iterator;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 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.OzoneVolume;
 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.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
@@ -100,9 +100,17 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   public OzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr,
       String bucketStr, OzoneFSStorageStatistics storageStatistics)
       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 =
         Thread.currentThread().getContextClassLoader();
     Thread.currentThread().setContextClassLoader(null);
+
     try {
       String replicationTypeConf =
           conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
@@ -110,8 +118,14 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
       int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
           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();
       this.volume = objectStore.getVolume(volumeStr);
       this.bucket = volume.getBucket(bucketStr);
@@ -124,6 +138,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
   }
 
+
   @Override
   public void close() throws IOException {
     ozoneClient.close();
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
index 13b7dda..3cfbebf 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
+++ b/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.Stream;
 
+import org.apache.commons.lang3.math.NumberUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -88,33 +89,53 @@ public class OzoneFileSystem extends FileSystem {
   private OzoneClientAdapter adapter;
   private boolean securityEnabled;
 
-
   private OzoneFSStorageStatistics storageStatistics;
 
   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
   public void initialize(URI name, Configuration conf) throws IOException {
     super.initialize(name, conf);
     setConf(conf);
     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();
 
     Matcher matcher = URL_SCHEMA_PATTERN.matcher(authority);
 
     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 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 {
       uri = new URIBuilder().setScheme(OZONE_URI_SCHEME)
-          .setHost(authority).build();
+        .setHost(authority)
+        .build();
       LOG.trace("Ozone URI for ozfs initialization is " + uri);
 
       //isolated is the default for ozonefs-lib-legacy which includes the
@@ -159,11 +180,13 @@ public class OzoneFileSystem extends FileSystem {
         } else {
           ozoneConfiguration = new OzoneConfiguration(conf);
         }
+
         SecurityConfig secConfig = new SecurityConfig(ozoneConfiguration);
         if (secConfig.isSecurityEnabled()) {
           this.securityEnabled = true;
         }
-        this.adapter = new OzoneClientAdapterImpl(ozoneConfiguration,
+        this.adapter = new OzoneClientAdapterImpl(omHost,
+            Integer.parseInt(omPort), ozoneConfiguration,
             volumeStr, bucketStr, storageStatistics);
       }
 
diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithMocks.java b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithMocks.java
new file mode 100644
index 0000000..7109327
--- /dev/null
+++ b/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);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 02/02: HDDS-1259. OzoneFS classpath separation is broken by the token validation. Contributed by Elek Marton.

Posted by xy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xyao pushed a commit to branch ozone-0.4
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 522976e405301fe8dd23f4751ba40edd27244c1d
Author: Elek, Márton <el...@users.noreply.github.com>
AuthorDate: Fri Mar 15 17:43:01 2019 +0100

    HDDS-1259. OzoneFS classpath separation is broken by the token validation. Contributed by Elek Marton.
    
     Closes #604
    
    (cherry picked from commit dc21655f2a477196ccc5173666b73d11865eeaf5)
---
 .../hadoop/fs/ozone/OzoneClientAdapterImpl.java    | 29 +++++++++++++++++-----
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java    | 18 ++------------
 2 files changed, 25 insertions(+), 22 deletions(-)

diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
index 1ea1261..1dbfa95 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -60,7 +61,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   private ReplicationType replicationType;
   private ReplicationFactor replicationFactor;
   private OzoneFSStorageStatistics storageStatistics;
-
+  private boolean securityEnabled;
   /**
    * Create new OzoneClientAdapter implementation.
    *
@@ -104,12 +105,24 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   }
 
   public OzoneClientAdapterImpl(String omHost, int omPort,
-      OzoneConfiguration conf, String volumeStr, String bucketStr,
+      Configuration hadoopConf, String volumeStr, String bucketStr,
       OzoneFSStorageStatistics storageStatistics) throws IOException {
 
     ClassLoader contextClassLoader =
         Thread.currentThread().getContextClassLoader();
     Thread.currentThread().setContextClassLoader(null);
+    OzoneConfiguration conf;
+    if (hadoopConf instanceof OzoneConfiguration) {
+      conf = (OzoneConfiguration) hadoopConf;
+    } else {
+      conf = new OzoneConfiguration(hadoopConf);
+    }
+
+    SecurityConfig secConfig = new SecurityConfig(conf);
+
+    if (secConfig.isSecurityEnabled()) {
+      this.securityEnabled = true;
+    }
 
     try {
       String replicationTypeConf =
@@ -276,10 +289,14 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   @Override
   public Token<OzoneTokenIdentifier> getDelegationToken(String renewer)
       throws IOException {
-    Token<OzoneTokenIdentifier> token =
-        ozoneClient.getObjectStore().getDelegationToken(new Text(renewer));
-    token.setKind(OzoneTokenIdentifier.KIND_NAME);
-    return token;
+    if (!securityEnabled) {
+      return null;
+    } else {
+      Token<OzoneTokenIdentifier> token =
+          ozoneClient.getObjectStore().getDelegationToken(new Text(renewer));
+      token.setKind(OzoneTokenIdentifier.KIND_NAME);
+      return token;
+    }
   }
 
   /**
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
index 3cfbebf..97f5c8e 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
@@ -48,8 +48,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
@@ -87,7 +85,6 @@ public class OzoneFileSystem extends FileSystem {
   private Path workingDir;
 
   private OzoneClientAdapter adapter;
-  private boolean securityEnabled;
 
   private OzoneFSStorageStatistics storageStatistics;
 
@@ -174,19 +171,9 @@ public class OzoneFileSystem extends FileSystem {
               OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr);
         }
       } else {
-        OzoneConfiguration ozoneConfiguration;
-        if (conf instanceof OzoneConfiguration) {
-          ozoneConfiguration = (OzoneConfiguration) conf;
-        } else {
-          ozoneConfiguration = new OzoneConfiguration(conf);
-        }
 
-        SecurityConfig secConfig = new SecurityConfig(ozoneConfiguration);
-        if (secConfig.isSecurityEnabled()) {
-          this.securityEnabled = true;
-        }
         this.adapter = new OzoneClientAdapterImpl(omHost,
-            Integer.parseInt(omPort), ozoneConfiguration,
+            Integer.parseInt(omPort), conf,
             volumeStr, bucketStr, storageStatistics);
       }
 
@@ -701,8 +688,7 @@ public class OzoneFileSystem extends FileSystem {
 
   @Override
   public Token<?> getDelegationToken(String renewer) throws IOException {
-    return securityEnabled? adapter.getDelegationToken(renewer) :
-        super.getDelegationToken(renewer);
+    return adapter.getDelegationToken(renewer);
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org