소스 검색

HDDS-1969. Implement OM GetDelegationToken request to use Cache and DoubleBuffer. (#1296)

Bharat Viswanadham 6 년 전
부모
커밋
8943e1340d

+ 29 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java

@@ -24,6 +24,7 @@ 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.OMConfigKeys;
 import org.apache.hadoop.ozone.om.S3SecretManager;
 import org.apache.hadoop.ozone.om.S3SecretManagerImpl;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
@@ -71,6 +72,8 @@ public class OzoneDelegationTokenSecretManager
    */
   private Object noInterruptsLock = new Object();
 
+  private boolean isRatisEnabled;
+
   /**
    * Create a secret manager.
    *
@@ -93,6 +96,9 @@ public class OzoneDelegationTokenSecretManager
     this.s3SecretManager = (S3SecretManagerImpl) s3SecretManager;
     this.store = new OzoneSecretStore(conf,
         this.s3SecretManager.getOmMetadataManager());
+    isRatisEnabled = conf.getBoolean(
+        OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
     loadTokenSecretState(store.loadState());
   }
 
@@ -131,7 +137,13 @@ public class OzoneDelegationTokenSecretManager
     byte[] password = createPassword(identifier.getBytes(),
         getCurrentKey().getPrivateKey());
     long expiryTime = identifier.getIssueDate() + getTokenRenewInterval();
-    addToTokenStore(identifier, password, expiryTime);
+
+    // For HA ratis will take care of updating.
+    // This will be removed, when HA/Non-HA code is merged.
+    if (!isRatisEnabled) {
+      addToTokenStore(identifier, password, expiryTime);
+    }
+
     Token<OzoneTokenIdentifier> token = new Token<>(identifier.getBytes(),
         password, identifier.getKind(), getService());
     if (LOG.isDebugEnabled()) {
@@ -140,6 +152,22 @@ public class OzoneDelegationTokenSecretManager
     return token;
   }
 
+  /**
+   * Add delegation token in to in-memory map of tokens.
+   * @param token
+   * @param ozoneTokenIdentifier
+   * @return renewTime - If updated successfully, return renewTime.
+   */
+  public long updateToken(Token<OzoneTokenIdentifier> token,
+      OzoneTokenIdentifier ozoneTokenIdentifier) {
+    long renewTime =
+        ozoneTokenIdentifier.getIssueDate() + getTokenRenewInterval();
+    TokenInfo tokenInfo = new TokenInfo(renewTime, token.getPassword(),
+        ozoneTokenIdentifier.getTrackingId());
+    currentTokens.put(ozoneTokenIdentifier, tokenInfo);
+    return renewTime;
+  }
+
   /**
    * Stores given identifier in token store.
    *

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

@@ -142,6 +142,7 @@ message OMRequest {
   optional hadoop.common.GetDelegationTokenRequestProto getDelegationTokenRequest = 61;
   optional hadoop.common.RenewDelegationTokenRequestProto renewDelegationTokenRequest= 62;
   optional hadoop.common.CancelDelegationTokenRequestProto cancelDelegationTokenRequest = 63;
+  optional UpdateGetDelegationTokenRequest updateGetDelegationTokenRequest = 64;
 
   optional GetFileStatusRequest             getFileStatusRequest           = 70;
   optional CreateDirectoryRequest           createDirectoryRequest         = 71;
@@ -307,6 +308,14 @@ message UserInfo {
     optional string remoteAddress = 3;
 }
 
+/**
+  This will be used during OM HA, once leader generates token sends this
+  request via ratis to persist to OM DB. This request will be internally used
+   by OM for replicating token across a quorum of OMs.
+*/
+message UpdateGetDelegationTokenRequest {
+    required GetDelegationTokenResponseProto getDelegationTokenResponse = 1;
+}
 
 /**
     Creates a volume

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

@@ -3429,4 +3429,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
         .getUpdatesSince(dbUpdatesRequest.getSequenceNumber());
 
   }
+
+  public OzoneDelegationTokenSecretManager getDelegationTokenMgr() {
+    return delegationTokenMgr;
+  }
+
 }

+ 3 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.ozone.om.request.s3.multipart.S3InitiateMultipartUpload
 import org.apache.hadoop.ozone.om.request.s3.multipart.S3MultipartUploadAbortRequest;
 import org.apache.hadoop.ozone.om.request.s3.multipart.S3MultipartUploadCommitPartRequest;
 import org.apache.hadoop.ozone.om.request.s3.multipart.S3MultipartUploadCompleteRequest;
+import org.apache.hadoop.ozone.om.request.security.OMGetDelegationTokenRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeCreateRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeDeleteRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeSetOwnerRequest;
@@ -130,6 +131,8 @@ public final class OzoneManagerRatisUtils {
     case RemoveAcl:
     case SetAcl:
       return getOMAclRequest(omRequest);
+    case GetDelegationToken:
+      return new OMGetDelegationTokenRequest(omRequest);
     default:
       // TODO: will update once all request types are implemented.
       return null;

+ 156 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/security/OMGetDelegationTokenRequest.java

@@ -0,0 +1,156 @@
+/**
+ * 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.ozone.om.request.security;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.security.OMDelegationTokenResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetDelegationTokenResponseProto;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UpdateGetDelegationTokenRequest;
+import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
+import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
+import org.apache.hadoop.security.proto.SecurityProtos;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Handle GetDelegationToken Request.
+ */
+public class OMGetDelegationTokenRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMGetDelegationTokenRequest.class);
+
+  public OMGetDelegationTokenRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    GetDelegationTokenRequestProto getDelegationTokenRequest =
+        getOmRequest().getGetDelegationTokenRequest();
+
+    // Call OM to create token
+    Token<OzoneTokenIdentifier> token = ozoneManager
+        .getDelegationToken(new Text(getDelegationTokenRequest.getRenewer()));
+
+
+    // Client issues GetDelegationToken request, when received by OM leader will
+    // it generate Token. Original GetDelegationToken request is converted to
+    // UpdateGetDelegationToken request with the generated token information.
+    // This updated request will be submitted to Ratis. In this way delegation
+    // token created by leader, will be replicated across all OMs.
+    // And also original GetDelegationToken request from client does not need
+    // any proto changes.
+
+    // Create UpdateGetDelegationTokenRequest with token response.
+    OMRequest.Builder omRequest = OMRequest.newBuilder()
+        .setUserInfo(getUserInfo())
+        .setUpdateGetDelegationTokenRequest(
+            UpdateGetDelegationTokenRequest.newBuilder()
+                .setGetDelegationTokenResponse(
+                    GetDelegationTokenResponseProto.newBuilder()
+                    .setResponse(SecurityProtos.GetDelegationTokenResponseProto
+                        .newBuilder().setToken(OMPBHelper
+                        .convertToTokenProto(token)).build()).build()))
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    if (getOmRequest().hasTraceID()) {
+      omRequest.setTraceID(getOmRequest().getTraceID());
+    }
+
+    return omRequest.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    UpdateGetDelegationTokenRequest updateGetDelegationTokenRequest =
+        getOmRequest().getUpdateGetDelegationTokenRequest();
+
+    SecurityProtos.TokenProto tokenProto = updateGetDelegationTokenRequest
+        .getGetDelegationTokenResponse().getResponse().getToken();
+
+    Token<OzoneTokenIdentifier> ozoneTokenIdentifierToken =
+        OMPBHelper.convertToDelegationToken(tokenProto);
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMClientResponse omClientResponse = null;
+    OMResponse.Builder omResponse =
+        OMResponse.newBuilder()
+            .setCmdType(OzoneManagerProtocolProtos.Type.GetDelegationToken)
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setSuccess(true);
+    try {
+      OzoneTokenIdentifier ozoneTokenIdentifier =
+          ozoneTokenIdentifierToken.decodeIdentifier();
+
+      // Update in memory map of token.
+      long renewTime = ozoneManager.getDelegationTokenMgr()
+          .updateToken(ozoneTokenIdentifierToken, ozoneTokenIdentifier);
+
+     // Update Cache.
+      omMetadataManager.getDelegationTokenTable().addCacheEntry(
+          new CacheKey<>(ozoneTokenIdentifier),
+          new CacheValue<>(Optional.of(renewTime), transactionLogIndex));
+
+      omClientResponse =
+          new OMDelegationTokenResponse(ozoneTokenIdentifier, renewTime,
+              omResponse.setGetDelegationTokenResponse(
+                  updateGetDelegationTokenRequest
+                      .getGetDelegationTokenResponse()).build());
+    } catch (IOException ex) {
+      LOG.error("Error in Updating DelegationToken {} to DB",
+          ozoneTokenIdentifierToken, ex);
+      omClientResponse = new OMDelegationTokenResponse(null, -1L,
+          createErrorOMResponse(omResponse, ex));
+    } finally {
+      if (omClientResponse != null) {
+        omClientResponse.setFlushFuture(
+            ozoneManagerDoubleBufferHelper.add(omClientResponse,
+                transactionLogIndex));
+      }
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Updated delegation token to OM DB: {}",
+          ozoneTokenIdentifierToken);
+    }
+
+    return omClientResponse;
+  }
+}

+ 22 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/security/package-info.java

@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package contains classes which handle security requests.
+ */
+package org.apache.hadoop.ozone.om.request.security;

+ 54 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/security/OMDelegationTokenResponse.java

@@ -0,0 +1,54 @@
+/**
+ * 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.ozone.om.response.security;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import java.io.IOException;
+
+/**
+ * Handle response for DelegationToken request.
+ */
+public class OMDelegationTokenResponse extends OMClientResponse {
+
+  private OzoneTokenIdentifier ozoneTokenIdentifier;
+  private long renewTime;
+  public OMDelegationTokenResponse(OzoneTokenIdentifier ozoneTokenIdentifier,
+      long renewTime, OMResponse omResponse) {
+    super(omResponse);
+    this.ozoneTokenIdentifier = ozoneTokenIdentifier;
+    this.renewTime = renewTime;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+      omMetadataManager.getDelegationTokenTable().putWithBatch(batchOperation,
+          ozoneTokenIdentifier, renewTime);
+    }
+  }
+}
+

+ 22 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/security/package-info.java

@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package contains classes which handle security request responses.
+ */
+package org.apache.hadoop.ozone.om.response.security;

+ 1 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerHARequestHandlerImpl.java

@@ -74,6 +74,7 @@ public class OzoneManagerHARequestHandlerImpl
     case AddAcl:
     case RemoveAcl:
     case SetAcl:
+    case GetDelegationToken:
       //TODO: We don't need to pass transactionID, this will be removed when
       // complete write requests is changed to new model. And also we can
       // return OMClientResponse, then adding to doubleBuffer can be taken