You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by vi...@apache.org on 2021/05/13 23:29:16 UTC

[ozone] branch master updated: HDDS-5165. OM DB checkpoint servlet not accessible in a secure cluster (#2200)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4189088  HDDS-5165. OM DB checkpoint servlet not accessible in a secure cluster (#2200)
4189088 is described below

commit 4189088cc7226a65ddd11c9400061301ad0c2560
Author: Vivek Ratnavel Subramanian <vi...@gmail.com>
AuthorDate: Thu May 13 16:28:55 2021 -0700

    HDDS-5165. OM DB checkpoint servlet not accessible in a secure cluster (#2200)
---
 .../hadoop/hdds/utils/DBCheckpointServlet.java     |  30 ++-
 .../hdds/scm/server/SCMDBCheckpointServlet.java    |   5 +-
 .../hdds/scm/TestSCMDbCheckpointServlet.java       |   9 +-
 .../hadoop/ozone/om/TestOMDbCheckpointServlet.java | 274 +++++++++++++--------
 .../hadoop/ozone/om/OMDBCheckpointServlet.java     |  18 +-
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  13 +
 6 files changed, 225 insertions(+), 124 deletions(-)

diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
index f1ab608..b976244 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -48,6 +48,7 @@ import org.apache.commons.lang3.StringUtils;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FLUSH;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,10 +65,13 @@ public class DBCheckpointServlet extends HttpServlet {
   private transient DBCheckpointMetrics dbMetrics;
 
   private boolean aclEnabled;
-  private Collection<String> ozAdmins;
+  private boolean isSpnegoEnabled;
+  private Collection<String> allowedUsers;
 
   public void initialize(DBStore store, DBCheckpointMetrics metrics,
-      boolean omAclEnabled, Collection<String> ozoneAdmins)
+                         boolean omAclEnabled,
+                         Collection<String> allowedAdminUsers,
+                         boolean isSpnegoAuthEnabled)
       throws ServletException {
 
     dbStore = store;
@@ -78,15 +82,17 @@ public class DBCheckpointServlet extends HttpServlet {
     }
 
     this.aclEnabled = omAclEnabled;
-    this.ozAdmins = ozoneAdmins;
+    this.allowedUsers = allowedAdminUsers;
+    this.isSpnegoEnabled = isSpnegoAuthEnabled;
   }
 
-  private boolean hasPermission(String username) {
-    // Check ACL for dbCheckpoint only when global Ozone ACL is enabled
-    if (aclEnabled) {
-      // Only Ozone admins are allowed
-      return ozAdmins.contains(OZONE_ADMINISTRATORS_WILDCARD)
-          || ozAdmins.contains(username);
+  private boolean hasPermission(UserGroupInformation user) {
+    // Check ACL for dbCheckpoint only when global Ozone ACL and SPNEGO is
+    // enabled
+    if (aclEnabled && isSpnegoEnabled) {
+      return allowedUsers.contains(OZONE_ADMINISTRATORS_WILDCARD)
+          || allowedUsers.contains(user.getShortUserName())
+          || allowedUsers.contains(user.getUserName());
     } else {
       return true;
     }
@@ -121,7 +127,9 @@ public class DBCheckpointServlet extends HttpServlet {
         return;
       } else {
         final String userPrincipalName = userPrincipal.getName();
-        if (!hasPermission(userPrincipalName)) {
+        UserGroupInformation ugi =
+            UserGroupInformation.createRemoteUser(userPrincipalName);
+        if (!hasPermission(ugi)) {
           LOG.error("Permission denied: User principal '{}' does not have"
                   + " access to /dbCheckpoint.\nThis can happen when Ozone"
                   + " Manager is started with a different user.\n"
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDBCheckpointServlet.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDBCheckpointServlet.java
index 7614161..7fa39ba 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDBCheckpointServlet.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDBCheckpointServlet.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -59,6 +59,7 @@ public class SCMDBCheckpointServlet extends DBCheckpointServlet {
     initialize(scm.getScmMetadataStore().getStore(),
         scm.getMetrics().getDBCheckpointMetrics(),
         false,
-        Collections.emptyList());
+        Collections.emptyList(),
+        false);
   }
 }
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
index 6ee2b97..dfd558f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -54,7 +54,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 /**
- * Class used for testing the OM DB Checkpoint provider servlet.
+ * Class used for testing the SCM DB Checkpoint provider servlet.
  */
 public class TestSCMDbCheckpointServlet {
   private MiniOzoneCluster cluster = null;
@@ -75,7 +75,7 @@ public class TestSCMDbCheckpointServlet {
    * <p>
    * Ozone is made active by setting OZONE_ENABLED = true
    *
-   * @throws IOException
+   * @throws Exception
    */
   @Before
   public void init() throws Exception {
@@ -118,7 +118,8 @@ public class TestSCMDbCheckpointServlet {
           scm.getScmMetadataStore().getStore(),
           scmMetrics.getDBCheckpointMetrics(),
           false,
-          Collections.emptyList());
+          Collections.emptyList(),
+          false);
 
       HttpServletRequest requestMock = mock(HttpServletRequest.class);
       HttpServletResponse responseMock = mock(HttpServletResponse.class);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
index b6ac88d..ac7629a 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.ozone.om;
 
 import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
 import javax.servlet.ServletOutputStream;
 import javax.servlet.WriteListener;
 import javax.servlet.http.HttpServletRequest;
@@ -32,7 +31,8 @@ import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.UUID;
+import java.security.Principal;
+import java.util.Collection;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
@@ -42,36 +42,46 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import org.apache.commons.io.FileUtils;
+
+import static org.apache.hadoop.hdds.recon.ReconConfig.ConfigStrings.OZONE_RECON_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FLUSH;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_AUTH_TYPE;
 import static org.apache.hadoop.ozone.om.OMDBCheckpointServlet.writeDBCheckpointToStream;
-import org.junit.AfterClass;
+
+import org.junit.After;
 import org.junit.Assert;
+
 import static org.junit.Assert.assertNotNull;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.Timeout;
 import org.mockito.Matchers;
+
 import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 /**
  * Class used for testing the OM DB Checkpoint provider servlet.
  */
 public class TestOMDbCheckpointServlet {
-  private static MiniOzoneCluster cluster = null;
-  private static OMMetrics omMetrics;
-  private static OzoneConfiguration conf;
-  private static String clusterId;
-  private static String scmId;
-  private static String omId;
+  private OzoneConfiguration conf;
+  private File tempFile;
+  private ServletOutputStream servletOutputStream;
+  private MiniOzoneCluster cluster = null;
+  private OMMetrics omMetrics = null;
+  private HttpServletRequest requestMock = null;
+  private HttpServletResponse responseMock = null;
+  private OMDBCheckpointServlet omDbCheckpointServletMock = null;
 
   @Rule
   public Timeout timeout = Timeout.seconds(240);
@@ -83,114 +93,166 @@ public class TestOMDbCheckpointServlet {
    * <p>
    * Ozone is made active by setting OZONE_ENABLED = true
    *
-   * @throws IOException
+   * @throws Exception
    */
-  @BeforeClass
-  public static void init() throws Exception {
+  @Before
+  public void init() throws Exception {
     conf = new OzoneConfiguration();
-    clusterId = UUID.randomUUID().toString();
-    scmId = UUID.randomUUID().toString();
-    omId = UUID.randomUUID().toString();
-    conf.setBoolean(OZONE_ACL_ENABLED, false);
-    conf.set(OZONE_ADMINISTRATORS, OZONE_ADMINISTRATORS_WILDCARD);
-    conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setClusterId(clusterId)
-        .setScmId(scmId)
-        .setOmId(omId)
-        .setNumDatanodes(1)
-        .build();
-    cluster.waitForClusterToBeReady();
-    omMetrics = cluster.getOzoneManager().getMetrics();
+
+    tempFile = File.createTempFile("testDoGet_" + System
+        .currentTimeMillis(), ".tar.gz");
+
+    FileOutputStream fileOutputStream = new FileOutputStream(tempFile);
+
+    servletOutputStream = new ServletOutputStream() {
+      @Override
+      public boolean isReady() {
+        return true;
+      }
+
+      @Override
+      public void setWriteListener(WriteListener writeListener) {
+      }
+
+      @Override
+      public void write(int b) throws IOException {
+        fileOutputStream.write(b);
+      }
+    };
   }
 
   /**
    * Shutdown MiniDFSCluster.
    */
-  @AfterClass
-  public static void shutdown() {
+  @After
+  public void shutdown() throws InterruptedException {
     if (cluster != null) {
       cluster.shutdown();
     }
+    FileUtils.deleteQuietly(tempFile);
+  }
+
+  private void setupCluster() throws Exception {
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(1)
+        .build();
+    cluster.waitForClusterToBeReady();
+    omMetrics = cluster.getOzoneManager().getMetrics();
+
+    omDbCheckpointServletMock =
+        mock(OMDBCheckpointServlet.class);
+
+    doCallRealMethod().when(omDbCheckpointServletMock).init();
+
+    requestMock = mock(HttpServletRequest.class);
+    // Return current user short name when asked
+    when(requestMock.getRemoteUser())
+        .thenReturn(UserGroupInformation.getCurrentUser().getShortUserName());
+    responseMock = mock(HttpServletResponse.class);
+
+    ServletContext servletContextMock = mock(ServletContext.class);
+    when(omDbCheckpointServletMock.getServletContext())
+        .thenReturn(servletContextMock);
+
+    when(servletContextMock.getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE))
+        .thenReturn(cluster.getOzoneManager());
+    when(requestMock.getParameter(OZONE_DB_CHECKPOINT_REQUEST_FLUSH))
+        .thenReturn("true");
+
+    doCallRealMethod().when(omDbCheckpointServletMock).doGet(requestMock,
+        responseMock);
   }
 
   @Test
-  public void testDoGet() throws ServletException, IOException {
-    File tempFile = null;
-    try {
-      OMDBCheckpointServlet omDbCheckpointServletMock =
-          mock(OMDBCheckpointServlet.class);
-
-      final OzoneManager om = cluster.getOzoneManager();
-
-      doCallRealMethod().when(omDbCheckpointServletMock).init();
-      doCallRealMethod().when(omDbCheckpointServletMock).initialize(
-          om.getMetadataManager().getStore(),
-          om.getMetrics().getDBCheckpointMetrics(),
-          om.getAclsEnabled(),
-          om.getOzoneAdmins(om.getConfiguration()));
-
-      HttpServletRequest requestMock = mock(HttpServletRequest.class);
-      // Return current user short name when asked
-      when(requestMock.getRemoteUser())
-          .thenReturn(UserGroupInformation.getCurrentUser().getShortUserName());
-      HttpServletResponse responseMock = mock(HttpServletResponse.class);
-
-      ServletContext servletContextMock = mock(ServletContext.class);
-      when(omDbCheckpointServletMock.getServletContext())
-          .thenReturn(servletContextMock);
-
-      when(servletContextMock.getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE))
-          .thenReturn(cluster.getOzoneManager());
-      when(requestMock.getParameter(OZONE_DB_CHECKPOINT_REQUEST_FLUSH))
-          .thenReturn("true");
-      doNothing().when(responseMock).setContentType("application/x-tgz");
-      doNothing().when(responseMock).setHeader(Matchers.anyString(),
-          Matchers.anyString());
-
-      tempFile = File.createTempFile("testDoGet_" + System
-          .currentTimeMillis(), ".tar.gz");
-
-      FileOutputStream fileOutputStream = new FileOutputStream(tempFile);
-      when(responseMock.getOutputStream()).thenReturn(
-          new ServletOutputStream() {
-            @Override
-            public boolean isReady() {
-              return true;
-            }
-
-            @Override
-            public void setWriteListener(WriteListener writeListener) {
-            }
-
-            @Override
-            public void write(int b) throws IOException {
-              fileOutputStream.write(b);
-            }
-          });
-
-      doCallRealMethod().when(omDbCheckpointServletMock).doGet(requestMock,
-          responseMock);
-
-      omDbCheckpointServletMock.init();
-      long initialCheckpointCount =
-          omMetrics.getDBCheckpointMetrics().getNumCheckpoints();
-
-      omDbCheckpointServletMock.doGet(requestMock, responseMock);
-
-      Assert.assertTrue(tempFile.length() > 0);
-      Assert.assertTrue(
-          omMetrics.getDBCheckpointMetrics().
-              getLastCheckpointCreationTimeTaken() > 0);
-      Assert.assertTrue(
-          omMetrics.getDBCheckpointMetrics().
-              getLastCheckpointStreamingTimeTaken() > 0);
-      Assert.assertTrue(omMetrics.getDBCheckpointMetrics().
-          getNumCheckpoints() > initialCheckpointCount);
-    } finally {
-      FileUtils.deleteQuietly(tempFile);
-    }
+  public void testDoGet() throws Exception {
+    conf.setBoolean(OZONE_ACL_ENABLED, false);
+    conf.set(OZONE_ADMINISTRATORS, OZONE_ADMINISTRATORS_WILDCARD);
+    conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
+
+    setupCluster();
+
+    final OzoneManager om = cluster.getOzoneManager();
+
+    doCallRealMethod().when(omDbCheckpointServletMock).initialize(
+        om.getMetadataManager().getStore(),
+        om.getMetrics().getDBCheckpointMetrics(),
+        om.getAclsEnabled(),
+        om.getOzoneAdmins(om.getConfiguration()),
+        om.isSpnegoEnabled());
+
+    doNothing().when(responseMock).setContentType("application/x-tgz");
+    doNothing().when(responseMock).setHeader(Matchers.anyString(),
+        Matchers.anyString());
+
+    when(responseMock.getOutputStream()).thenReturn(servletOutputStream);
+
+    omDbCheckpointServletMock.init();
+    long initialCheckpointCount =
+        omMetrics.getDBCheckpointMetrics().getNumCheckpoints();
+
+    omDbCheckpointServletMock.doGet(requestMock, responseMock);
+
+    Assert.assertTrue(tempFile.length() > 0);
+    Assert.assertTrue(
+        omMetrics.getDBCheckpointMetrics().
+            getLastCheckpointCreationTimeTaken() > 0);
+    Assert.assertTrue(
+        omMetrics.getDBCheckpointMetrics().
+            getLastCheckpointStreamingTimeTaken() > 0);
+    Assert.assertTrue(omMetrics.getDBCheckpointMetrics().
+        getNumCheckpoints() > initialCheckpointCount);
+  }
+
+  @Test
+  public void testSpnegoEnabled() throws Exception {
+    conf.setBoolean(OZONE_ACL_ENABLED, true);
+    conf.set(OZONE_ADMINISTRATORS, "");
+    conf.set(OZONE_OM_HTTP_AUTH_TYPE, "kerberos");
+    conf.set(OZONE_RECON_KERBEROS_PRINCIPAL_KEY, "recon/host1@REALM");
+
+    setupCluster();
+
+    final OzoneManager om = cluster.getOzoneManager();
+    Collection<String> allowedUsers = om.getOzoneAdmins(om.getConfiguration());
+    allowedUsers.add("recon");
+
+    doCallRealMethod().when(omDbCheckpointServletMock).initialize(
+        om.getMetadataManager().getStore(),
+        om.getMetrics().getDBCheckpointMetrics(),
+        om.getAclsEnabled(),
+        allowedUsers,
+        om.isSpnegoEnabled());
+
+    omDbCheckpointServletMock.init();
+    omDbCheckpointServletMock.doGet(requestMock, responseMock);
+
+    // Response status should be set to 403 Forbidden since there was no user
+    // principal set in the request
+    verify(responseMock, times(1)).setStatus(HttpServletResponse.SC_FORBIDDEN);
+
+    // Set the principal to DN in request
+    // This should also get denied since only OM and recon
+    // users should be granted access to the servlet
+    Principal userPrincipalMock = mock(Principal.class);
+    when(userPrincipalMock.getName()).thenReturn("dn/localhost@REALM");
+    when(requestMock.getUserPrincipal()).thenReturn(userPrincipalMock);
+
+    omDbCheckpointServletMock.doGet(requestMock, responseMock);
+
+    // Verify that the Response status is set to 403 again for DN user.
+    verify(responseMock, times(2)).setStatus(HttpServletResponse.SC_FORBIDDEN);
+
+    // Now, set the principal to recon in request
+    when(userPrincipalMock.getName()).thenReturn("recon/localhost@REALM");
+
+    when(requestMock.getUserPrincipal()).thenReturn(userPrincipalMock);
+    when(responseMock.getOutputStream()).thenReturn(servletOutputStream);
+
+    omDbCheckpointServletMock.doGet(requestMock, responseMock);
 
+    // Recon user should be able to access the servlet and download the
+    // snapshot
+    Assert.assertTrue(tempFile.length() > 0);
   }
 
   @Test
@@ -229,7 +291,7 @@ public class TestOMDbCheckpointServlet {
 
 class TestDBCheckpoint implements DBCheckpoint {
 
-  private Path checkpointFile;
+  private final Path checkpointFile;
 
   TestDBCheckpoint(Path checkpointFile) {
     this.checkpointFile = checkpointFile;
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
index 39868b4..c951322 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
@@ -20,13 +20,17 @@ package org.apache.hadoop.ozone.om;
 
 import javax.servlet.ServletException;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.recon.ReconConfig;
 import org.apache.hadoop.hdds.utils.DBCheckpointServlet;
 import org.apache.hadoop.ozone.OzoneConsts;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Collection;
 
 /**
  * Provides the current checkpoint Snapshot of the OM DB. (tar.gz)
@@ -59,10 +63,22 @@ public class OMDBCheckpointServlet extends DBCheckpointServlet {
     }
 
     try {
+      OzoneConfiguration conf = om.getConfiguration();
+      // Only Ozone Admins and Recon are allowed
+      Collection<String> allowedUsers = om.getOzoneAdmins(conf);
+      ReconConfig reconConfig = conf.getObject(ReconConfig.class);
+      String reconPrincipal = reconConfig.getKerberosPrincipal();
+      if (!reconPrincipal.isEmpty()) {
+        UserGroupInformation ugi =
+            UserGroupInformation.createRemoteUser(reconPrincipal);
+        allowedUsers.add(ugi.getShortUserName());
+      }
+
       initialize(om.getMetadataManager().getStore(),
           om.getMetrics().getDBCheckpointMetrics(),
           om.getAclsEnabled(),
-          om.getOzoneAdmins(om.getConfiguration()));
+          allowedUsers,
+          om.isSpnegoEnabled());
     } catch (IOException e) {
       LOG.error("Error in getOzoneAdmins: {}", e.getMessage());
     }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 29fe697..d0f81c1 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -216,6 +216,7 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_AUTH_TYPE;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL;
@@ -292,6 +293,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private final Runnable shutdownHook;
   private final File omMetaDir;
   private final boolean isAclEnabled;
+  private final boolean isSpnegoEnabled;
   private IAccessAuthorizer accessAuthorizer;
   private JvmPauseMonitor jvmPauseMonitor;
   private final SecurityConfig secConfig;
@@ -378,6 +380,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     omMetaDir = OMStorage.getOmDbDir(configuration);
     this.isAclEnabled = conf.getBoolean(OZONE_ACL_ENABLED,
         OZONE_ACL_ENABLED_DEFAULT);
+    this.isSpnegoEnabled = conf.get(OZONE_OM_HTTP_AUTH_TYPE, "simple")
+        .equals("kerberos");
     this.scmBlockSize = (long) conf.getStorageSize(OZONE_SCM_BLOCK_SIZE,
         OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
     this.preallocateBlocksMax = conf.getInt(
@@ -1782,6 +1786,15 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   }
 
   /**
+   * Return true if SPNEGO auth is enabled for OM HTTP server, otherwise false.
+   *
+   * @return boolean
+   */
+  public boolean isSpnegoEnabled() {
+    return isSpnegoEnabled;
+  }
+
+  /**
    * {@inheritDoc}
    */
   @Override

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