[ 
https://issues.apache.org/jira/browse/HDDS-1620?focusedWorklogId=258291&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-258291
 ]

ASF GitHub Bot logged work on HDDS-1620:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Jun/19 02:34
            Start Date: 12/Jun/19 02:34
    Worklog Time Spent: 10m 
      Work Description: bharatviswa504 commented on pull request #884: 
HDDS-1620. Implement Volume Write Requests to use Cache and DoubleBuffer.
URL: https://github.com/apache/hadoop/pull/884#discussion_r292727143
 
 

 ##########
 File path: 
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java
 ##########
 @@ -0,0 +1,188 @@
+/**
+ * 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.volume;
+
+import java.io.IOException;
+import java.util.Map;
+
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
+import org.apache.hadoop.ozone.om.response.volume.OMVolumeSetOwnerResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+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
+    .SetVolumePropertyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .SetVolumePropertyResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+
+/**
+ * Handle set owner request for volume.
+ */
+public class OMVolumeSetOwnerRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMVolumeSetOwnerRequest.class);
+
+  public OMVolumeSetOwnerRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+
+    SetVolumePropertyRequest setVolumePropertyRequest =
+        getOmRequest().getSetVolumePropertyRequest();
+
+    Preconditions.checkNotNull(setVolumePropertyRequest);
+
+    OMResponse.Builder omResponse = OMResponse.newBuilder().setCmdType(
+        OzoneManagerProtocolProtos.Type.SetVolumeProperty).setStatus(
+        OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    // In production this will never happen, this request will be called only
+    // when we have ownerName in setVolumePropertyRequest.
+    if (!setVolumePropertyRequest.hasOwnerName()) {
+      omResponse.setStatus(OzoneManagerProtocolProtos.Status.INVALID_REQUEST)
+          .setSuccess(false);
+      return new OMVolumeSetOwnerResponse(null, null, null, null,
+          omResponse.build());
+    }
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumVolumeUpdates();
+    String volume = setVolumePropertyRequest.getVolumeName();
+    String newOwner = setVolumePropertyRequest.getOwnerName();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+    OzoneManagerProtocolProtos.UserInfo userInfo = 
getOmRequest().getUserInfo();
+
+    Map<String, String> auditMap = buildVolumeAuditMap(volume);
+    auditMap.put(OzoneConsts.OWNER, newOwner);
+    try {
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE_ACL,
+            volume, null, null);
+      }
+    } catch (IOException ex) {
+      LOG.error("Changing volume ownership failed for user:{} volume:{}",
+          newOwner, volume);
+      omMetrics.incNumVolumeUpdateFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.SET_OWNER, auditMap,
+          ex, userInfo));
+      return new OMVolumeCreateResponse(null, null,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+
+    long maxUserVolumeCount = ozoneManager.getMaxUserVolumeCount();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String dbVolumeKey = omMetadataManager.getVolumeKey(volume);
+    String oldOwner = null;
+    OzoneManagerProtocolProtos.VolumeList oldOwnerVolumeList = null;
+    OzoneManagerProtocolProtos.VolumeList newOwnerVolumeList = null;
+    OmVolumeArgs omVolumeArgs = null;
+    IOException exception = null;
+
+    omMetadataManager.getLock().acquireUserLock(newOwner);
+    omMetadataManager.getLock().acquireVolumeLock(volume);
+    try {
+      omVolumeArgs = omMetadataManager.getVolumeTable().get(dbVolumeKey);
+
+      if (omVolumeArgs == null) {
+        LOG.debug("Changing volume ownership failed for user:{} volume:{}",
+            newOwner, volume);
+        throw new OMException("Volume " + volume + " is not found",
+            OMException.ResultCodes.VOLUME_NOT_FOUND);
+      }
+
+      oldOwner = omVolumeArgs.getOwnerName();
+      oldOwnerVolumeList = VolumeRequestHelper.delVolumeFromOwnerList(
+          omMetadataManager, volume, oldOwner);
+
+      newOwnerVolumeList = VolumeRequestHelper.addVolumeToOwnerList(
+          omMetadataManager, volume, newOwner, maxUserVolumeCount);
+
+      // Set owner with new owner name.
+      omVolumeArgs.setOwnerName(newOwner);
+
+      // Update cache.
+      omMetadataManager.getUserTable().addCacheEntry(
+          new CacheKey<>(omMetadataManager.getUserKey(newOwner)),
+              new CacheValue<>(Optional.of(newOwnerVolumeList),
+                  transactionLogIndex));
+      omMetadataManager.getUserTable().addCacheEntry(
 
 Review comment:
   Example: 
   SetOwner V1 , old Owner ozone, newOwner hadoop 
   CreateVolume v7 ozone
   
   If we don't acquire the lock for oldOwner before reading from the table. We 
might see an issue. Thanks for catching it.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 258291)
    Time Spent: 14h  (was: 13h 50m)

> Implement Volume Write Requests to use Cache and DoubleBuffer
> -------------------------------------------------------------
>
>                 Key: HDDS-1620
>                 URL: https://issues.apache.org/jira/browse/HDDS-1620
>             Project: Hadoop Distributed Data Store
>          Issue Type: Sub-task
>          Components: Ozone Manager
>            Reporter: Bharat Viswanadham
>            Assignee: Bharat Viswanadham
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 14h
>  Remaining Estimate: 0h
>
> Implement Volume write requests to use OM Cache, double buffer. 
> In this Jira will add the changes to implement volume operations, and 
> HA/Non-HA will have a different code path, but once all requests are 
> implemented will have a single code path.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to