Ver Fonte

HDDS-1061. DelegationToken: Add certificate serial id to Ozone Delegation Token Identifier. Contributed by Ajay Kumar.

Ajay Kumar há 6 anos atrás
pai
commit
cbf82fabf0

+ 1 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/OMCertificateClient.java

@@ -38,7 +38,7 @@ public class OMCertificateClient extends DefaultCertificateClient {
   private static final Logger LOG =
       LoggerFactory.getLogger(OMCertificateClient.class);
 
-  OMCertificateClient(SecurityConfig securityConfig, String component) {
+  public OMCertificateClient(SecurityConfig securityConfig, String component) {
     super(securityConfig, component, LOG);
   }
 

+ 17 - 4
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneBlockTokenSecretManager.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -30,7 +31,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.security.KeyPair;
 import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.Map;
@@ -148,17 +148,30 @@ public class OzoneBlockTokenSecretManager extends
     }
 
     if (!verifySignature(identifier, createPassword(identifier))) {
-      throw new InvalidToken("Tampared/Inavalid token.");
+      throw new InvalidToken("Tampered/Invalid token.");
     }
     return true;
   }
 
+  /**
+   * Validates if given hash is valid.
+   *
+   * @param identifier
+   * @param password
+   */
+  public boolean verifySignature(OzoneBlockTokenIdentifier identifier,
+      byte[] password) {
+    throw new UnsupportedOperationException("This operation is not " +
+        "supported for block tokens.");
+  }
+
   /**
    * Should be called before this object is used.
+   * @param client
    */
   @Override
-  public synchronized void start(KeyPair keyPair) throws IOException {
-    super.start(keyPair);
+  public synchronized void start(CertificateClient client) throws IOException {
+    super.start(client);
     removeExpiredKeys();
   }
 

+ 40 - 3
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java

@@ -21,6 +21,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.security.OzoneSecretStore.OzoneManagerSecretState;
@@ -36,7 +38,6 @@ import org.slf4j.LoggerFactory;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
-import java.security.KeyPair;
 import java.security.PrivateKey;
 import java.util.Iterator;
 import java.util.Map;
@@ -59,6 +60,7 @@ public class OzoneDelegationTokenSecretManager
   private final OzoneSecretStore store;
   private Thread tokenRemoverThread;
   private final long tokenRemoverScanInterval;
+  private String omCertificateSerialId;
   /**
    * If the delegation token update thread holds this lock, it will not get
    * interrupted.
@@ -162,6 +164,18 @@ public class OzoneDelegationTokenSecretManager
     identifier.setMasterKeyId(getCurrentKey().getKeyId());
     identifier.setSequenceNumber(sequenceNum);
     identifier.setMaxDate(Time.monotonicNow() + getTokenMaxLifetime());
+    identifier.setOmCertSerialId(getOmCertificateSerialId());
+  }
+
+  /**
+   * Get OM certificate serial id.
+   * */
+  private String getOmCertificateSerialId() {
+    if (omCertificateSerialId == null) {
+      omCertificateSerialId =
+          getCertClient().getCertificate().getSerialNumber().toString();
+    }
+    return omCertificateSerialId;
   }
 
   /**
@@ -291,6 +305,28 @@ public class OzoneDelegationTokenSecretManager
     return info;
   }
 
+  /**
+   * Validates if given hash is valid.
+   *
+   * @param identifier
+   * @param password
+   */
+  public boolean verifySignature(OzoneTokenIdentifier identifier,
+      byte[] password) {
+    try {
+      if (identifier.getOmCertSerialId().equals(getOmCertificateSerialId())) {
+        return getCertClient().verifySignature(identifier.getBytes(), password,
+            getCertClient().getCertificate());
+      } else {
+        // TODO: This delegation token was issued by other OM instance. Fetch
+        // certificate from SCM using certificate serial.
+        return false;
+      }
+    } catch (CertificateException e) {
+      return false;
+    }
+  }
+
   // TODO: handle roll private key/certificate
   private synchronized void removeExpiredKeys() {
     long now = Time.monotonicNow();
@@ -358,8 +394,9 @@ public class OzoneDelegationTokenSecretManager
    * Should be called before this object is used.
    */
   @Override
-  public synchronized void start(KeyPair keyPair) throws IOException {
-    super.start(keyPair);
+  public synchronized void start(CertificateClient certClient)
+      throws IOException {
+    super.start(certClient);
     storeKey(getCurrentKey());
     removeExpiredKeys();
     tokenRemoverThread = new Daemon(new ExpiredTokenRemover());

+ 12 - 36
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneSecretManager.java

@@ -21,6 +21,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.SecretManager;
@@ -56,6 +57,7 @@ public abstract class OzoneSecretManager<T extends TokenIdentifier>
   private final long tokenMaxLifetime;
   private final long tokenRenewInterval;
   private final Text service;
+  private CertificateClient certClient;
   private volatile boolean running;
   private OzoneSecretKey currentKey;
   private AtomicInteger currentKeyId;
@@ -125,20 +127,6 @@ public abstract class OzoneSecretManager<T extends TokenIdentifier>
     return password;
   }
 
-  /**
-   * Default implementation for Ozone. Verifies if hash in token is legit.
-   * */
-  @Override
-  public byte[] retrievePassword(T identifier) throws InvalidToken {
-    byte[] password = createPassword(identifier);
-    // TODO: Revisit this when key/certificate rotation is implemented.
-    // i.e Try all valid keys instead of current key only.
-    if (!verifySignature(identifier, password)) {
-      throw new InvalidToken("Tampared/Inavalid token.");
-    }
-    return password;
-  }
-
   /**
    * Renew a delegation token.
    *
@@ -192,25 +180,6 @@ public abstract class OzoneSecretManager<T extends TokenIdentifier>
     return currentKey;
   }
 
-  /**
-   * Validates if given hash is valid.
-   *
-   * @param identifier
-   * @param password
-   */
-  public boolean verifySignature(T identifier, byte[] password) {
-    try {
-      Signature rsaSignature =
-          Signature.getInstance(getDefaultSignatureAlgorithm());
-      rsaSignature.initVerify(currentKey.getPublicKey());
-      rsaSignature.update(identifier.getBytes());
-      return rsaSignature.verify(password);
-    } catch (NoSuchAlgorithmException | SignatureException |
-        InvalidKeyException e) {
-      return false;
-    }
-  }
-
   public String formatTokenId(T id) {
     return "(" + id + ")";
   }
@@ -218,12 +187,15 @@ public abstract class OzoneSecretManager<T extends TokenIdentifier>
   /**
    * Should be called before this object is used.
    *
-   * @param keyPair
+   * @param client
    * @throws IOException
    */
-  public synchronized void start(KeyPair keyPair) throws IOException {
+  public synchronized void start(CertificateClient client)
+      throws IOException {
     Preconditions.checkState(!isRunning());
-    updateCurrentKey(keyPair);
+    this.certClient = client;
+    updateCurrentKey(new KeyPair(certClient.getPublicKey(),
+        certClient.getPrivateKey()));
     setIsRunning(true);
   }
 
@@ -276,5 +248,9 @@ public abstract class OzoneSecretManager<T extends TokenIdentifier>
   public AtomicInteger getTokenSequenceNumber() {
     return tokenSequenceNumber;
   }
+
+  public CertificateClient getCertClient() {
+    return certClient;
+  }
 }
 

+ 27 - 2
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneTokenIdentifier.java

@@ -23,6 +23,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.Text;
@@ -39,6 +40,7 @@ public class OzoneTokenIdentifier extends
     AbstractDelegationTokenIdentifier {
 
   public final static Text KIND_NAME = new Text("OzoneToken");
+  private String omCertSerialId;
 
   /**
    * Create an empty delegation token identifier.
@@ -93,7 +95,9 @@ public class OzoneTokenIdentifier extends
         .setIssueDate(getIssueDate())
         .setMaxDate(getMaxDate())
         .setSequenceNumber(getSequenceNumber())
-        .setMasterKeyId(getMasterKeyId()).build();
+        .setMasterKeyId(getMasterKeyId())
+        .setOmCertSerialId(getOmCertSerialId())
+        .build();
     out.write(token.toByteArray());
   }
 
@@ -113,6 +117,7 @@ public class OzoneTokenIdentifier extends
     setMaxDate(token.getMaxDate());
     setSequenceNumber(token.getSequenceNumber());
     setMasterKeyId(token.getMasterKeyId());
+    setOmCertSerialId(token.getOmCertSerialId());
   }
 
   /**
@@ -130,6 +135,7 @@ public class OzoneTokenIdentifier extends
     identifier.setIssueDate(token.getIssueDate());
     identifier.setSequenceNumber(token.getSequenceNumber());
     identifier.setMasterKeyId(token.getMasterKeyId());
+    identifier.setOmCertSerialId(token.getOmCertSerialId());
     return identifier;
   }
 
@@ -169,7 +175,18 @@ public class OzoneTokenIdentifier extends
     if (!(obj instanceof OzoneTokenIdentifier)) {
       return false;
     }
-    return super.equals(obj);
+    OzoneTokenIdentifier that = (OzoneTokenIdentifier) obj;
+    return new EqualsBuilder()
+        .append(getOmCertSerialId(), that.getOmCertSerialId())
+        .append(getMaxDate(), that.getMaxDate())
+        .append(getIssueDate(), that.getIssueDate())
+        .append(getMasterKeyId(), that.getMasterKeyId())
+        .append(getOwner(), that.getOwner())
+        .append(getRealUser(), that.getRealUser())
+        .append(getRenewer(), that.getRenewer())
+        .append(getKind(), that.getKind())
+        .append(getSequenceNumber(), that.getSequenceNumber())
+        .build();
   }
 
   /**
@@ -214,4 +231,12 @@ public class OzoneTokenIdentifier extends
       return trackingId;
     }
   }
+
+  public String getOmCertSerialId() {
+    return omCertSerialId;
+  }
+
+  public void setOmCertSerialId(String omCertSerialId) {
+    this.omCertSerialId = omCertSerialId;
+  }
 }

+ 1 - 0
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -574,6 +574,7 @@ message OMTokenProto {
     optional uint32 sequenceNumber = 7;
     optional uint32 masterKeyId    = 8;
     optional uint64 expiryDate     = 9;
+    required string omCertSerialId = 10;
 }
 
 message SecretKeyProto {

+ 42 - 3
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestOzoneBlockTokenSecretManager.java

@@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.OMCertificateClient;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -36,6 +38,8 @@ import org.junit.Test;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.security.KeyPair;
+import java.security.PrivateKey;
+import java.security.PublicKey;
 import java.security.Signature;
 import java.security.cert.X509Certificate;
 import java.util.EnumSet;
@@ -50,6 +54,7 @@ public class TestOzoneBlockTokenSecretManager {
   private X509Certificate x509Certificate;
   private long expiryTime;
   private String omCertSerialId;
+  private CertificateClient client;
   private static final String BASEDIR = GenericTestUtils
       .getTempPath(TestOzoneBlockTokenSecretManager.class.getSimpleName());
 
@@ -62,12 +67,35 @@ public class TestOzoneBlockTokenSecretManager {
     keyPair = KeyStoreTestUtil.generateKeyPair("RSA");
     expiryTime = Time.monotonicNow() + 60 * 60 * 24;
     // Create Ozone Master certificate (SCM CA issued cert) and key store.
+    SecurityConfig securityConfig = new SecurityConfig(conf);
     x509Certificate = KeyStoreTestUtil
         .generateCertificate("CN=OzoneMaster", keyPair, 30, "SHA256withRSA");
     omCertSerialId = x509Certificate.getSerialNumber().toString();
-    secretManager = new OzoneBlockTokenSecretManager(new SecurityConfig(conf),
+    secretManager = new OzoneBlockTokenSecretManager(securityConfig,
         expiryTime, omCertSerialId);
-    secretManager.start(keyPair);
+    client = getCertificateClient(securityConfig);
+    client.init();
+    secretManager.start(client);
+  }
+
+  private CertificateClient getCertificateClient(SecurityConfig secConf)
+      throws Exception {
+    return new OMCertificateClient(secConf, "om"){
+      @Override
+      public X509Certificate getCertificate() {
+        return x509Certificate;
+      }
+
+      @Override
+      public PrivateKey getPrivateKey() {
+        return keyPair.getPrivate();
+      }
+
+      @Override
+      public PublicKey getPublicKey() {
+        return keyPair.getPublic();
+      }
+    };
   }
 
   @After
@@ -113,7 +141,7 @@ public class TestOzoneBlockTokenSecretManager {
   private void validateHash(byte[] hash, byte[] identifier) throws Exception {
     Signature rsaSignature =
         Signature.getInstance(secretManager.getDefaultSignatureAlgorithm());
-    rsaSignature.initVerify(keyPair.getPublic());
+    rsaSignature.initVerify(client.getPublicKey());
     rsaSignature.update(identifier);
     Assert.assertTrue(rsaSignature.verify(hash));
   }
@@ -144,4 +172,15 @@ public class TestOzoneBlockTokenSecretManager {
           secretManager.cancelToken(null, null);
         });
   }
+
+  @Test
+  public void testVerifySignatureFailure() throws Exception {
+    OzoneBlockTokenIdentifier id = new OzoneBlockTokenIdentifier(
+        "testUser", "4234", EnumSet.allOf(AccessModeProto.class),
+        Time.now() + 60 * 60 * 24, "123444", 1024);
+    LambdaTestUtils.intercept(UnsupportedOperationException.class, "operation" +
+            " is not supported for block tokens",
+        () -> secretManager.verifySignature(id,
+            client.signData(id.getBytes())));
+  }
 }

+ 73 - 16
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java

@@ -22,6 +22,8 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.OMCertificateClient;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
@@ -37,7 +39,10 @@ import org.junit.Test;
 import java.io.File;
 import java.io.IOException;
 import java.security.KeyPair;
+import java.security.PrivateKey;
+import java.security.PublicKey;
 import java.security.Signature;
+import java.security.cert.X509Certificate;
 
 /**
  * Test class for {@link OzoneDelegationTokenSecretManager}.
@@ -46,7 +51,7 @@ public class TestOzoneDelegationTokenSecretManager {
 
   private OzoneDelegationTokenSecretManager secretManager;
   private SecurityConfig securityConfig;
-  private KeyPair keyPair;
+  private CertificateClient certificateClient;
   private long expiryTime;
   private Text serviceRpcAdd;
   private OzoneConfiguration conf;
@@ -61,12 +66,38 @@ public class TestOzoneDelegationTokenSecretManager {
     conf = new OzoneConfiguration();
     conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, BASEDIR);
     securityConfig = new SecurityConfig(conf);
-    // Create Ozone Master key pair.
-    keyPair = KeyStoreTestUtil.generateKeyPair("RSA");
+    certificateClient = setupCertificateClient();
+    certificateClient.init();
     expiryTime = Time.monotonicNow() + 60 * 60 * 24;
     serviceRpcAdd = new Text("localhost");
   }
 
+  /**
+   * Helper function to create certificate client.
+   * */
+  private CertificateClient setupCertificateClient() throws Exception {
+    KeyPair keyPair = KeyStoreTestUtil.generateKeyPair("RSA");
+    X509Certificate cert = KeyStoreTestUtil
+        .generateCertificate("CN=OzoneMaster", keyPair, 30, "SHA256withRSA");
+
+    return new OMCertificateClient(securityConfig, "test") {
+      @Override
+      public X509Certificate getCertificate() {
+        return cert;
+      }
+
+      @Override
+      public PrivateKey getPrivateKey() {
+        return keyPair.getPrivate();
+      }
+
+      @Override
+      public PublicKey getPublicKey() {
+        return keyPair.getPublic();
+      }
+    };
+  }
+
   @After
   public void tearDown() throws IOException {
     secretManager.stop();
@@ -77,7 +108,7 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testCreateToken() throws Exception {
     secretManager = createSecretManager(conf, tokenMaxLifetime,
         expiryTime, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
         TEST_USER,
         TEST_USER);
@@ -95,7 +126,7 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testRenewTokenSuccess() throws Exception {
     secretManager = createSecretManager(conf, tokenMaxLifetime,
         expiryTime, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
         TEST_USER,
         TEST_USER);
@@ -111,10 +142,9 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testRenewTokenFailure() throws Exception {
     secretManager = createSecretManager(conf, tokenMaxLifetime,
         expiryTime, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
-        TEST_USER,
-        TEST_USER);
+        TEST_USER, TEST_USER);
     LambdaTestUtils.intercept(AccessControlException.class,
         "rougeUser tries to renew a token", () -> {
           secretManager.renewToken(token, "rougeUser");
@@ -128,7 +158,7 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testRenewTokenFailureMaxTime() throws Exception {
     secretManager = createSecretManager(conf, 100,
         100, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
         TEST_USER,
         TEST_USER);
@@ -146,7 +176,7 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testRenewTokenFailureRenewalTime() throws Exception {
     secretManager = createSecretManager(conf, 1000 * 10,
         10, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
         TEST_USER,
         TEST_USER);
@@ -160,7 +190,7 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testCreateIdentifier() throws Exception {
     secretManager = createSecretManager(conf, tokenMaxLifetime,
         expiryTime, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     OzoneTokenIdentifier identifier = secretManager.createIdentifier();
     // Check basic details.
     Assert.assertTrue(identifier.getOwner().equals(new Text("")));
@@ -172,10 +202,9 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testCancelTokenSuccess() throws Exception {
     secretManager = createSecretManager(conf, tokenMaxLifetime,
         expiryTime, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
-        TEST_USER,
-        TEST_USER);
+        TEST_USER, TEST_USER);
     secretManager.cancelToken(token, TEST_USER.toString());
   }
 
@@ -183,7 +212,7 @@ public class TestOzoneDelegationTokenSecretManager {
   public void testCancelTokenFailure() throws Exception {
     secretManager = createSecretManager(conf, tokenMaxLifetime,
         expiryTime, tokenRemoverScanInterval);
-    secretManager.start(keyPair);
+    secretManager.start(certificateClient);
     Token<OzoneTokenIdentifier> token = secretManager.createToken(TEST_USER,
         TEST_USER,
         TEST_USER);
@@ -193,6 +222,34 @@ public class TestOzoneDelegationTokenSecretManager {
         });
   }
 
+  @Test
+  public void testVerifySignatureSuccess() throws Exception {
+    secretManager = createSecretManager(conf, tokenMaxLifetime,
+        expiryTime, tokenRemoverScanInterval);
+    secretManager.start(certificateClient);
+    OzoneTokenIdentifier id = new OzoneTokenIdentifier();
+    id.setOmCertSerialId(certificateClient.getCertificate()
+        .getSerialNumber().toString());
+    id.setMaxDate(Time.now() + 60 * 60 * 24);
+    id.setOwner(new Text("test"));
+    Assert.assertTrue(secretManager.verifySignature(id,
+        certificateClient.signData(id.getBytes())));
+  }
+
+  @Test
+  public void testVerifySignatureFailure() throws Exception {
+    secretManager = createSecretManager(conf, tokenMaxLifetime,
+        expiryTime, tokenRemoverScanInterval);
+    secretManager.start(certificateClient);
+    OzoneTokenIdentifier id = new OzoneTokenIdentifier();
+    // set invalid om cert serial id
+    id.setOmCertSerialId("1927393");
+    id.setMaxDate(Time.now() + 60*60*24);
+    id.setOwner(new Text("test"));
+    Assert.assertFalse(secretManager.verifySignature(id,
+        certificateClient.signData(id.getBytes())));
+  }
+
   /**
    * Validate hash using public key of KeyPair.
    */
@@ -200,7 +257,7 @@ public class TestOzoneDelegationTokenSecretManager {
     Signature rsaSignature =
         Signature.getInstance(securityConfig.getSignatureAlgo(),
             securityConfig.getProvider());
-    rsaSignature.initVerify(keyPair.getPublic());
+    rsaSignature.initVerify(certificateClient.getPublicKey());
     rsaSignature.update(identifier);
     Assert.assertTrue(rsaSignature.verify(hash));
   }

+ 2 - 2
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -633,7 +633,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     if (secConfig.isBlockTokenEnabled() && blockTokenMgr != null) {
       try {
         LOG.info("Starting OM block token secret manager");
-        blockTokenMgr.start(keyPair);
+        blockTokenMgr.start(certClient);
       } catch (IOException e) {
         // Unable to start secret manager.
         LOG.error("Error starting block token secret manager.", e);
@@ -644,7 +644,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     if (delegationTokenMgr != null) {
       try {
         LOG.info("Starting OM delegation token secret manager");
-        delegationTokenMgr.start(keyPair);
+        delegationTokenMgr.start(certClient);
       } catch (IOException e) {
         // Unable to start secret manager.
         LOG.error("Error starting delegation token secret manager.", e);

+ 14 - 9
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneTokenIdentifier.java

@@ -72,12 +72,12 @@ public class TestOzoneTokenIdentifier {
   private static String sslConfsDir;
   private static final String EXCLUDE_CIPHERS =
       "TLS_ECDHE_RSA_WITH_RC4_128_SHA,"
-      + "SSL_DHE_RSA_EXPORT_WITH_DES40_CBC_SHA,  \n"
-      + "SSL_RSA_WITH_DES_CBC_SHA,"
-      + "SSL_DHE_RSA_WITH_DES_CBC_SHA,  "
-      + "SSL_RSA_EXPORT_WITH_RC4_40_MD5,\t \n"
-      + "SSL_RSA_EXPORT_WITH_DES40_CBC_SHA,"
-      + "SSL_RSA_WITH_RC4_128_MD5";
+          + "SSL_DHE_RSA_EXPORT_WITH_DES40_CBC_SHA,  \n"
+          + "SSL_RSA_WITH_DES_CBC_SHA,"
+          + "SSL_DHE_RSA_WITH_DES_CBC_SHA,  "
+          + "SSL_RSA_EXPORT_WITH_RC4_40_MD5,\t \n"
+          + "SSL_RSA_EXPORT_WITH_DES40_CBC_SHA,"
+          + "SSL_RSA_WITH_RC4_128_MD5";
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -87,7 +87,7 @@ public class TestOzoneTokenIdentifier {
   }
 
   private Configuration createConfiguration(boolean clientCert,
-                                            boolean trustStore)
+      boolean trustStore)
       throws Exception {
     Configuration conf = new Configuration();
     KeyStoreTestUtil.setupSSLConfig(KEYSTORES_DIR, sslConfsDir, conf,
@@ -129,6 +129,7 @@ public class TestOzoneTokenIdentifier {
     // Sign the OzoneMaster Token with Ozone Master private key
     PrivateKey privateKey = keyPair.getPrivate();
     OzoneTokenIdentifier tokenId = new OzoneTokenIdentifier();
+    tokenId.setOmCertSerialId("123");
     byte[] signedToken = signTokenAsymmetric(tokenId, privateKey);
 
     // Verify a valid signed OzoneMaster Token with Ozone Master
@@ -140,6 +141,7 @@ public class TestOzoneTokenIdentifier {
     // public key(certificate)
     tokenId = new OzoneTokenIdentifier(new Text("oozie"),
         new Text("rm"), new Text("client"));
+    tokenId.setOmCertSerialId("123");
     LOG.info("Unsigned token {} is {}", tokenId,
         verifyTokenAsymmetric(tokenId, RandomUtils.nextBytes(128), cert));
 
@@ -177,10 +179,12 @@ public class TestOzoneTokenIdentifier {
   }
 
   OzoneTokenIdentifier generateTestToken() {
-    return new OzoneTokenIdentifier(
+    OzoneTokenIdentifier tokenIdentifier = new OzoneTokenIdentifier(
         new Text(RandomStringUtils.randomAlphabetic(6)),
         new Text(RandomStringUtils.randomAlphabetic(5)),
         new Text(RandomStringUtils.randomAlphabetic(4)));
+    tokenIdentifier.setOmCertSerialId("123");
+    return tokenIdentifier;
   }
 
   @Test
@@ -216,7 +220,7 @@ public class TestOzoneTokenIdentifier {
     }
     duration = Time.monotonicNowNanos() - startTime;
     LOG.info("Average token verify time with HmacSha256(RSA/1024 key) "
-            + "is {} ns", duration/testTokenCount);
+        + "is {} ns", duration/testTokenCount);
   }
 
   @Test
@@ -295,6 +299,7 @@ public class TestOzoneTokenIdentifier {
     id.setIssueDate(Time.now());
     id.setMaxDate(Time.now() + 5000);
     id.setSequenceNumber(1);
+    id.setOmCertSerialId("123");
     return id;
   }
 }