ayushtkn commented on code in PR #4720: URL: https://github.com/apache/ozone/pull/4720#discussion_r1203555081
########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeySetTimesRequest.java: ########## @@ -0,0 +1,246 @@ +/** + * 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.key; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; + +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.audit.AuditLogger; +import org.apache.hadoop.ozone.audit.OMAction; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.response.key.OMKeySetTimesResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesResponse; +import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; +import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; + +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; + +/** + * Handle add SetTime request for key. + */ +public class OMKeySetTimesRequest extends OMKeyRequest { + + private static final Logger LOG = + LoggerFactory.getLogger(OMKeySetTimesRequest.class); + + @Override + public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { + OMRequest request = super.preExecute(ozoneManager); + SetTimesRequest setTimesRequest = request.getSetTimesRequest(); + String keyPath = setTimesRequest.getKeyArgs().getKeyName(); + String normalizedKeyPath = + validateAndNormalizeKey(ozoneManager.getEnableFileSystemPaths(), + keyPath, getBucketLayout()); + + OzoneManagerProtocolProtos.KeyArgs keyArgs = + OzoneManagerProtocolProtos.KeyArgs.newBuilder() + .setVolumeName(getVolumeName()) + .setBucketName(getBucketName()) + .setKeyName(normalizedKeyPath) + .build(); + + return request.toBuilder() + .setSetTimesRequest( + setTimesRequest.toBuilder() + .setKeyArgs(keyArgs) + .setMtime(getModificationTime())) + .build(); + } + + private final String volumeName; + private final String bucketName; + private final String keyName; + private final long modificationTime; + + public OMKeySetTimesRequest(OMRequest omRequest, BucketLayout bucketLayout) { + super(omRequest, bucketLayout); + OzoneManagerProtocolProtos.SetTimesRequest setTimesRequest = + getOmRequest().getSetTimesRequest(); + volumeName = setTimesRequest.getKeyArgs().getVolumeName(); + bucketName = setTimesRequest.getKeyArgs().getBucketName(); + keyName = setTimesRequest.getKeyArgs().getKeyName(); + // ignore accessTime + modificationTime = setTimesRequest.getMtime(); + } + + protected String getVolumeName() { + return volumeName; + } + + protected String getBucketName() { + return bucketName; + } + + protected String getKeyName() { + return keyName; + } + + protected long getModificationTime() { + return modificationTime; + } + + OMResponse.Builder onInit() { + return OmResponseUtil.getOMResponseBuilder(getOmRequest()); + } + + OMClientResponse onSuccess(OMResponse.Builder omResponse, + OmKeyInfo omKeyInfo, boolean operationResult) { + omResponse.setSuccess(operationResult); + omResponse.setSetTimesResponse(SetTimesResponse.newBuilder()); + return new OMKeySetTimesResponse(omResponse.build(), omKeyInfo); + } + + /** + * Get the om client response on failure case with lock. + * @param omResponse + * @param exception + * @return OMClientResponse + */ + OMClientResponse onFailure(OMResponse.Builder omResponse, + IOException exception) { + return new OMKeySetTimesResponse(createErrorOMResponse( + omResponse, exception), getBucketLayout()); + } + + void onComplete(Result result, boolean operationResult, + IOException exception, long trxnLogIndex, AuditLogger auditLogger, + Map<String, String> auditMap) { + switch (result) { + case SUCCESS: + if (LOG.isDebugEnabled()) { + LOG.debug("Set mtime: {} to path: {} success!", modificationTime, + getKeyName()); + } + break; + case FAILURE: + LOG.error("Set mtime {} to path {} failed!", modificationTime, + getKeyName(), exception); + break; + default: + LOG.error("Unrecognized Result for OMKeySetTimesRequest: {}", + getOmRequest()); + } + + auditMap.put(OzoneConsts.VOLUME, getVolumeName()); + auditMap.put(OzoneConsts.BUCKET, getBucketName()); + auditMap.put(OzoneConsts.KEY, getKeyName()); + auditMap.put(OzoneConsts.MODIFICATION_TIME, + String.valueOf(getModificationTime())); + auditLog(auditLogger, buildAuditMessage(OMAction.SET_TIMES, auditMap, + exception, getOmRequest().getUserInfo())); + } + + void apply(OmKeyInfo omKeyInfo, long trxnLogIndex) { + // No need to check not null here, this will be never called with null. + long mtime = getModificationTime(); + if (mtime != -1) { Review Comment: shouldn't this be ``mtime>=0`` ########## hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java: ########## @@ -1752,4 +1752,20 @@ public void testFileSystemDeclaresCapability() throws Throwable { assertHasPathCapabilities(fs, root, FS_ACLS); assertHasPathCapabilities(fs, root, FS_CHECKSUMS); } + + @Test + public void testSetTimes() throws Exception { + // Create a file + String testKeyName = "testKey1"; + Path path = new Path(OZONE_URI_DELIMITER, testKeyName); + try (FSDataOutputStream stream = fs.create(path)) { + stream.write(1); + } + + fs.setTimes(path, 1000, 2000); + + FileStatus fileStatus = fs.getFileStatus(path); + // Trash Current directory should still have been created. Review Comment: Trash? ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeySetTimesRequestWithFSO.java: ########## @@ -0,0 +1,161 @@ +/** + * 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.key; + +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +import org.apache.hadoop.ozone.om.request.file.OMFileRequest; +import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.response.key.OMKeySetTimesResponseWithFSO; +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.acl.IAccessAuthorizer; +import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; + +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; + +/** + * Handle set times request for bucket for prefix layout. + */ +public class OMKeySetTimesRequestWithFSO extends OMKeySetTimesRequest { + + private static final Logger LOG = + LoggerFactory.getLogger(OMKeySetTimesRequestWithFSO.class); Review Comment: unused ########## hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java: ########## @@ -706,4 +706,10 @@ private SnapshotDiffReportOzone getSnapshotDiffReportOnceComplete( } return snapshotDiffResponse.getSnapshotDiffReport(); } + + @Override + public void setTimes(String key, long mtime, long atime) throws IOException { + incrementCounter(Statistic.OBJECTS_RENAMED, 1); Review Comment: ``Objects_Renamed`` is this intentional? Can't we use one explicit for SetTimes ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeySetTimesRequest.java: ########## @@ -0,0 +1,246 @@ +/** + * 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.key; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; + +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.audit.AuditLogger; +import org.apache.hadoop.ozone.audit.OMAction; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.response.key.OMKeySetTimesResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesResponse; +import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; +import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; + +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; + +/** + * Handle add SetTime request for key. Review Comment: nit `` setTimes `` ########## hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java: ########## @@ -2516,4 +2516,23 @@ public void testSnapshotDiff() throws Exception { () -> ofs.getSnapshotDiffReport(volumePath1, finalFromSnap, finalToSnap)); } + + @Test + public void testSetTimes() throws Exception { + // Create a file + OzoneBucket bucket1 = + TestDataUtil.createVolumeAndBucket(client, bucketLayout); + Path volumePath1 = new Path(OZONE_URI_DELIMITER, bucket1.getVolumeName()); + Path bucketPath1 = new Path(volumePath1, bucket1.getName()); + Path path = new Path(bucketPath1, "key1"); + try (FSDataOutputStream stream = fs.create(path)) { + stream.write(1); + } + + fs.setTimes(path, 1000, 2000); + + FileStatus fileStatus = fs.getFileStatus(path); + // Trash Current directory should still have been created. Review Comment: trash? ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java: ########## @@ -314,6 +314,11 @@ public static OMClientRequest createClientRequest(OMRequest omRequest, volumeName = keyArgs.getVolumeName(); bucketName = keyArgs.getBucketName(); break; + case SetTimes: + //keyArgs = omRequest.getSetTimesRequest().getKeyArgs(); Review Comment: leftover ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeySetTimesRequest.java: ########## @@ -0,0 +1,246 @@ +/** + * 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.key; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; + +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.audit.AuditLogger; +import org.apache.hadoop.ozone.audit.OMAction; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.response.key.OMKeySetTimesResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesResponse; +import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; +import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; + +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; + +/** + * Handle add SetTime request for key. + */ +public class OMKeySetTimesRequest extends OMKeyRequest { + + private static final Logger LOG = + LoggerFactory.getLogger(OMKeySetTimesRequest.class); + + @Override + public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { + OMRequest request = super.preExecute(ozoneManager); + SetTimesRequest setTimesRequest = request.getSetTimesRequest(); + String keyPath = setTimesRequest.getKeyArgs().getKeyName(); + String normalizedKeyPath = + validateAndNormalizeKey(ozoneManager.getEnableFileSystemPaths(), + keyPath, getBucketLayout()); + + OzoneManagerProtocolProtos.KeyArgs keyArgs = + OzoneManagerProtocolProtos.KeyArgs.newBuilder() + .setVolumeName(getVolumeName()) + .setBucketName(getBucketName()) + .setKeyName(normalizedKeyPath) + .build(); + + return request.toBuilder() + .setSetTimesRequest( + setTimesRequest.toBuilder() + .setKeyArgs(keyArgs) + .setMtime(getModificationTime())) + .build(); + } + + private final String volumeName; + private final String bucketName; + private final String keyName; + private final long modificationTime; + + public OMKeySetTimesRequest(OMRequest omRequest, BucketLayout bucketLayout) { + super(omRequest, bucketLayout); + OzoneManagerProtocolProtos.SetTimesRequest setTimesRequest = + getOmRequest().getSetTimesRequest(); + volumeName = setTimesRequest.getKeyArgs().getVolumeName(); + bucketName = setTimesRequest.getKeyArgs().getBucketName(); + keyName = setTimesRequest.getKeyArgs().getKeyName(); + // ignore accessTime + modificationTime = setTimesRequest.getMtime(); + } + + protected String getVolumeName() { + return volumeName; + } + + protected String getBucketName() { + return bucketName; + } + + protected String getKeyName() { + return keyName; + } + + protected long getModificationTime() { + return modificationTime; + } + + OMResponse.Builder onInit() { + return OmResponseUtil.getOMResponseBuilder(getOmRequest()); + } + + OMClientResponse onSuccess(OMResponse.Builder omResponse, + OmKeyInfo omKeyInfo, boolean operationResult) { + omResponse.setSuccess(operationResult); + omResponse.setSetTimesResponse(SetTimesResponse.newBuilder()); + return new OMKeySetTimesResponse(omResponse.build(), omKeyInfo); + } + + /** + * Get the om client response on failure case with lock. + * @param omResponse + * @param exception + * @return OMClientResponse + */ + OMClientResponse onFailure(OMResponse.Builder omResponse, + IOException exception) { + return new OMKeySetTimesResponse(createErrorOMResponse( + omResponse, exception), getBucketLayout()); + } + + void onComplete(Result result, boolean operationResult, + IOException exception, long trxnLogIndex, AuditLogger auditLogger, Review Comment: ``operationResult`` and ``trxnLogIndex`` are ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java: ########## @@ -823,6 +824,7 @@ public void incNumSetAcl() { numSetAcl.incr(); } + Review Comment: nit can avoid ########## hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java: ########## @@ -1752,4 +1752,20 @@ public void testFileSystemDeclaresCapability() throws Throwable { assertHasPathCapabilities(fs, root, FS_ACLS); assertHasPathCapabilities(fs, root, FS_CHECKSUMS); } + + @Test + public void testSetTimes() throws Exception { + // Create a file + String testKeyName = "testKey1"; + Path path = new Path(OZONE_URI_DELIMITER, testKeyName); + try (FSDataOutputStream stream = fs.create(path)) { + stream.write(1); + } + + fs.setTimes(path, 1000, 2000); + + FileStatus fileStatus = fs.getFileStatus(path); + // Trash Current directory should still have been created. + Assert.assertEquals(1000, fileStatus.getModificationTime()); + } Review Comment: should have added a case for -1 as well, that the value did stay same ########## hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java: ########## @@ -998,4 +998,15 @@ EchoRPCResponse echoRPCReq(byte[] payloadReq, */ boolean recoverLease(String volumeName, String bucketName, String keyName) throws IOException; + + /** + * Update modification time and access time of a file. Review Comment: it is actually just updating the modification time only ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeySetTimesResponse.java: ########## @@ -0,0 +1,83 @@ +/** + * 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.key; + +import java.io.IOException; + +import javax.annotation.Nonnull; + +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .OMResponse; Review Comment: does line limit apply to imports as well? doesn't look very cool though to have import in two lines ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeySetTimesRequest.java: ########## @@ -0,0 +1,246 @@ +/** + * 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.key; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; + +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.audit.AuditLogger; +import org.apache.hadoop.ozone.audit.OMAction; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.response.key.OMKeySetTimesResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetTimesResponse; +import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; +import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; + +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; + +/** + * Handle add SetTime request for key. + */ +public class OMKeySetTimesRequest extends OMKeyRequest { + + private static final Logger LOG = + LoggerFactory.getLogger(OMKeySetTimesRequest.class); + + @Override + public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { + OMRequest request = super.preExecute(ozoneManager); + SetTimesRequest setTimesRequest = request.getSetTimesRequest(); + String keyPath = setTimesRequest.getKeyArgs().getKeyName(); + String normalizedKeyPath = + validateAndNormalizeKey(ozoneManager.getEnableFileSystemPaths(), + keyPath, getBucketLayout()); + + OzoneManagerProtocolProtos.KeyArgs keyArgs = + OzoneManagerProtocolProtos.KeyArgs.newBuilder() + .setVolumeName(getVolumeName()) + .setBucketName(getBucketName()) + .setKeyName(normalizedKeyPath) + .build(); + + return request.toBuilder() + .setSetTimesRequest( + setTimesRequest.toBuilder() + .setKeyArgs(keyArgs) + .setMtime(getModificationTime())) + .build(); + } + + private final String volumeName; + private final String bucketName; + private final String keyName; + private final long modificationTime; + + public OMKeySetTimesRequest(OMRequest omRequest, BucketLayout bucketLayout) { + super(omRequest, bucketLayout); + OzoneManagerProtocolProtos.SetTimesRequest setTimesRequest = + getOmRequest().getSetTimesRequest(); + volumeName = setTimesRequest.getKeyArgs().getVolumeName(); + bucketName = setTimesRequest.getKeyArgs().getBucketName(); + keyName = setTimesRequest.getKeyArgs().getKeyName(); + // ignore accessTime + modificationTime = setTimesRequest.getMtime(); + } + + protected String getVolumeName() { + return volumeName; + } + + protected String getBucketName() { + return bucketName; + } + + protected String getKeyName() { + return keyName; + } + + protected long getModificationTime() { + return modificationTime; + } + + OMResponse.Builder onInit() { + return OmResponseUtil.getOMResponseBuilder(getOmRequest()); + } + + OMClientResponse onSuccess(OMResponse.Builder omResponse, Review Comment: onFailure you added a javadoc, either remove from there or add it for success as well ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeySetTimesResponseWithFSO.java: ########## @@ -0,0 +1,83 @@ +/** + * 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.key; + +import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.request.file.OMFileRequest; +import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.jetbrains.annotations.NotNull; + +import javax.annotation.Nonnull; +import java.io.IOException; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE; + +/** + * Response for Bucket acl request for prefix layout. + */ +@CleanupTableInfo(cleanupTables = { FILE_TABLE, DIRECTORY_TABLE }) +public class OMKeySetTimesResponseWithFSO extends OMKeySetTimesResponse { + + private boolean isDirectory; + private long volumeId; + private long bucketId; + + public OMKeySetTimesResponseWithFSO( + @NotNull OzoneManagerProtocolProtos.OMResponse omResponse, + @NotNull OmKeyInfo omKeyInfo, boolean isDirectory, + @Nonnull BucketLayout bucketLayout, @Nonnull long volumeId, + @Nonnull long bucketId) { + super(omResponse, omKeyInfo, bucketLayout); + this.isDirectory = isDirectory; + this.volumeId = volumeId; + this.bucketId = bucketId; + } + + /** + * For when the request is not successful. + * For a successful request, the other constructor should be used. + * + * @param omResponse + */ + public OMKeySetTimesResponseWithFSO( Review Comment: who uses it? ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeySetTimesResponseWithFSO.java: ########## @@ -0,0 +1,83 @@ +/** + * 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.key; + +import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.request.file.OMFileRequest; +import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.jetbrains.annotations.NotNull; + +import javax.annotation.Nonnull; +import java.io.IOException; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE; + +/** + * Response for Bucket acl request for prefix layout. + */ +@CleanupTableInfo(cleanupTables = { FILE_TABLE, DIRECTORY_TABLE }) +public class OMKeySetTimesResponseWithFSO extends OMKeySetTimesResponse { + + private boolean isDirectory; + private long volumeId; + private long bucketId; + + public OMKeySetTimesResponseWithFSO( + @NotNull OzoneManagerProtocolProtos.OMResponse omResponse, + @NotNull OmKeyInfo omKeyInfo, boolean isDirectory, + @Nonnull BucketLayout bucketLayout, @Nonnull long volumeId, + @Nonnull long bucketId) { Review Comment: ``volumeId`` and ``bucketId`` are primitive long, they can't be ``null`` right? ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeySetTimesResponseWithFSO.java: ########## @@ -0,0 +1,83 @@ +/** + * 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.key; + +import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.request.file.OMFileRequest; +import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.jetbrains.annotations.NotNull; + +import javax.annotation.Nonnull; +import java.io.IOException; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE; + +/** + * Response for Bucket acl request for prefix layout. + */ +@CleanupTableInfo(cleanupTables = { FILE_TABLE, DIRECTORY_TABLE }) +public class OMKeySetTimesResponseWithFSO extends OMKeySetTimesResponse { + + private boolean isDirectory; + private long volumeId; + private long bucketId; + + public OMKeySetTimesResponseWithFSO( + @NotNull OzoneManagerProtocolProtos.OMResponse omResponse, + @NotNull OmKeyInfo omKeyInfo, boolean isDirectory, + @Nonnull BucketLayout bucketLayout, @Nonnull long volumeId, + @Nonnull long bucketId) { + super(omResponse, omKeyInfo, bucketLayout); + this.isDirectory = isDirectory; + this.volumeId = volumeId; + this.bucketId = bucketId; + } + + /** + * For when the request is not successful. + * For a successful request, the other constructor should be used. + * + * @param omResponse + */ + public OMKeySetTimesResponseWithFSO( + @NotNull OzoneManagerProtocolProtos.OMResponse omResponse, + BucketLayout bucketLayout) { + super(omResponse, bucketLayout); + } + + @Override public void addToDBBatch(OMMetadataManager omMetadataManager, Review Comment: nit line break? ########## hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java: ########## @@ -915,6 +915,16 @@ public Path createSnapshot(Path path, String snapshotName) OM_SNAPSHOT_INDICATOR + OZONE_URI_DELIMITER + snapshot); } + @Override + public void setTimes(Path f, long mtime, long atime) throws IOException { + incrementCounter(Statistic.INVOCATION_SET_TIMES, 1); + statistics.incrementWriteOps(1); + LOG.trace("setTimes() path:{}", f); + Path qualifiedPath = f.makeQualified(uri, workingDir); + String key = pathToKey(qualifiedPath); Review Comment: it doesn't call ``checkPath(qualifiedPath);``, that Wrong FS thing must not be getting handled? Can you confirm if it is getting handled somewhere ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeySetTimesResponse.java: ########## @@ -0,0 +1,83 @@ +/** + * 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.key; + +import java.io.IOException; + +import javax.annotation.Nonnull; + +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .OMResponse; +import org.apache.hadoop.hdds.utils.db.BatchOperation; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE; + +/** + * Response for Bucket acl request. + */ +@CleanupTableInfo(cleanupTables = KEY_TABLE) +public class OMKeySetTimesResponse extends OmKeyResponse { + + private OmKeyInfo omKeyInfo; + + public OMKeySetTimesResponse(@Nonnull OMResponse omResponse, + @Nonnull OmKeyInfo omKeyInfo) { + super(omResponse); + this.omKeyInfo = omKeyInfo; + } + + public OMKeySetTimesResponse(@Nonnull OMResponse omResponse, + @Nonnull OmKeyInfo omKeyInfo, + @Nonnull BucketLayout bucketLayout) { + super(omResponse, bucketLayout); + this.omKeyInfo = omKeyInfo; + } + + /** + * For when the request is not successful. + * For a successful request, the other constructor should be used. + */ + public OMKeySetTimesResponse(@Nonnull OMResponse omResponse) { Review Comment: who uses this in the code? ########## hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java: ########## @@ -1364,4 +1364,12 @@ public boolean recoverLease(final Path f) throws IOException { return ozoneClient.getProxy().getOzoneManagerClient().recoverLease( volume.getName(), bucket.getName(), ofsPath.getKeyName()); } + + public void setTimes(String key, long mtime, long atime) throws IOException { + incrementCounter(Statistic.OBJECTS_RENAMED, 1); Review Comment: renamed? -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
