HDDS-1551. Implement Bucket Write Requests to use Cache and DoubleBuffer. (#850)
This commit is contained in:
parent
33c62f8f4e
commit
c965f7f499
|
@ -83,13 +83,16 @@ public class PartialTableCache<CACHEKEY extends CacheKey,
|
|||
iterator.hasNext();) {
|
||||
currentEntry = iterator.next();
|
||||
CACHEKEY cachekey = currentEntry.getCachekey();
|
||||
CacheValue cacheValue = cache.get(cachekey);
|
||||
if (cacheValue.getEpoch() <= epoch) {
|
||||
cache.remove(cachekey);
|
||||
iterator.remove();
|
||||
} else {
|
||||
// If currentEntry epoch is greater than epoch, we have deleted all
|
||||
// entries less than specified epoch. So, we can break.
|
||||
CacheValue cacheValue = cache.computeIfPresent(cachekey, ((k, v) -> {
|
||||
if (v.getEpoch() <= epoch) {
|
||||
iterator.remove();
|
||||
return null;
|
||||
}
|
||||
return v;
|
||||
}));
|
||||
// If currentEntry epoch is greater than epoch, we have deleted all
|
||||
// entries less than specified epoch. So, we can break.
|
||||
if (cacheValue != null && cacheValue.getEpoch() >= epoch) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.ozone.om.protocol;
|
||||
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmDeleteVolumeResponse;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||
|
@ -168,54 +166,4 @@ public interface OzoneManagerHAProtocol {
|
|||
*/
|
||||
void applyDeleteVolume(String volume, String owner,
|
||||
VolumeList newVolumeList) throws IOException;
|
||||
|
||||
/**
|
||||
* Start Create Bucket Transaction.
|
||||
* @param omBucketInfo
|
||||
* @return OmBucketInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
OmBucketInfo startCreateBucket(OmBucketInfo omBucketInfo) throws IOException;
|
||||
|
||||
/**
|
||||
* Apply Create Bucket Changes to OM DB.
|
||||
* @param omBucketInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
void applyCreateBucket(OmBucketInfo omBucketInfo) throws IOException;
|
||||
|
||||
/**
|
||||
* Start Delete Bucket Transaction.
|
||||
* @param volumeName
|
||||
* @param bucketName
|
||||
* @throws IOException
|
||||
*/
|
||||
void startDeleteBucket(String volumeName, String bucketName)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Apply Delete Bucket changes to OM DB.
|
||||
* @param volumeName
|
||||
* @param bucketName
|
||||
* @throws IOException
|
||||
*/
|
||||
void applyDeleteBucket(String volumeName, String bucketName)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Start SetBucket Property Transaction.
|
||||
* @param omBucketArgs
|
||||
* @return OmBucketInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
OmBucketInfo startSetBucketProperty(OmBucketArgs omBucketArgs)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Apply SetBucket Property changes to OM DB.
|
||||
* @param omBucketInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
void applySetBucketProperty(OmBucketInfo omBucketInfo) throws IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -383,7 +383,7 @@ message BucketInfo {
|
|||
repeated OzoneAclInfo acls = 3;
|
||||
required bool isVersionEnabled = 4 [default = false];
|
||||
required StorageTypeProto storageType = 5 [default = DISK];
|
||||
required uint64 creationTime = 6;
|
||||
optional uint64 creationTime = 6;
|
||||
repeated hadoop.hdds.KeyValue metadata = 7;
|
||||
optional BucketEncryptionInfoProto beinfo = 8;
|
||||
}
|
||||
|
@ -553,11 +553,7 @@ message InfoBucketResponse {
|
|||
}
|
||||
|
||||
message SetBucketPropertyRequest {
|
||||
//TODO: See if we can merge bucketArgs and bucketInfo
|
||||
optional BucketArgs bucketArgs = 1;
|
||||
// This will be set during startTransaction, and used to apply to OM DB
|
||||
// during applyTransaction.
|
||||
optional BucketInfo bucketInfo = 2;
|
||||
}
|
||||
|
||||
message SetBucketPropertyResponse {
|
||||
|
|
|
@ -171,11 +171,11 @@ public class TestOmMetrics {
|
|||
Mockito.doNothing().when(mockS3Bm).deleteS3Bucket("random");
|
||||
Mockito.doReturn(true).when(mockS3Bm).createOzoneVolumeIfNeeded(null);
|
||||
|
||||
Mockito.doReturn(null).when(mockBm).createBucket(null);
|
||||
Mockito.doReturn(null).when(mockBm).createBucket(null);
|
||||
Mockito.doNothing().when(mockBm).createBucket(null);
|
||||
Mockito.doNothing().when(mockBm).createBucket(null);
|
||||
Mockito.doNothing().when(mockBm).deleteBucket(null, null);
|
||||
Mockito.doReturn(null).when(mockBm).getBucketInfo(null, null);
|
||||
Mockito.doReturn(null).when(mockBm).setBucketProperty(null);
|
||||
Mockito.doNothing().when(mockBm).setBucketProperty(null);
|
||||
Mockito.doReturn(null).when(mockBm).listBuckets(null, null, null, 0);
|
||||
|
||||
HddsWhiteboxTestUtils.setInternalState(
|
||||
|
|
|
@ -405,7 +405,7 @@ public class TestOzoneManagerHA {
|
|||
// last running OM as it would fail to get a quorum.
|
||||
if (e instanceof RemoteException) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"RaftRetryFailureException", e);
|
||||
"NotLeaderException", e);
|
||||
}
|
||||
} else {
|
||||
throw e;
|
||||
|
@ -446,7 +446,7 @@ public class TestOzoneManagerHA {
|
|||
// last running OM as it would fail to get a quorum.
|
||||
if (e instanceof RemoteException) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"RaftRetryFailureException", e);
|
||||
"NotLeaderException", e);
|
||||
}
|
||||
} else {
|
||||
throw e;
|
||||
|
|
|
@ -30,14 +30,7 @@ public interface BucketManager {
|
|||
* Creates a bucket.
|
||||
* @param bucketInfo - OmBucketInfo for creating bucket.
|
||||
*/
|
||||
OmBucketInfo createBucket(OmBucketInfo bucketInfo) throws IOException;
|
||||
|
||||
/**
|
||||
* Apply Create Bucket changes to OM DB.
|
||||
* @param omBucketInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
void applyCreateBucket(OmBucketInfo omBucketInfo) throws IOException;
|
||||
void createBucket(OmBucketInfo bucketInfo) throws IOException;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -53,14 +46,7 @@ public interface BucketManager {
|
|||
* @param args - BucketArgs.
|
||||
* @throws IOException
|
||||
*/
|
||||
OmBucketInfo setBucketProperty(OmBucketArgs args) throws IOException;
|
||||
|
||||
/**
|
||||
* Apply SetBucket Property changes to OM DB.
|
||||
* @param omBucketInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
void applySetBucketProperty(OmBucketInfo omBucketInfo) throws IOException;
|
||||
void setBucketProperty(OmBucketArgs args) throws IOException;
|
||||
|
||||
/**
|
||||
* Deletes an existing empty bucket from volume.
|
||||
|
@ -70,15 +56,6 @@ public interface BucketManager {
|
|||
*/
|
||||
void deleteBucket(String volumeName, String bucketName) throws IOException;
|
||||
|
||||
/**
|
||||
* Apply Delete Bucket changes to OM DB.
|
||||
* @param volumeName
|
||||
* @param bucketName
|
||||
* @throws IOException
|
||||
*/
|
||||
void applyDeleteBucket(String volumeName, String bucketName)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a list of buckets represented by {@link OmBucketInfo}
|
||||
* in the given volume.
|
||||
|
|
|
@ -102,7 +102,7 @@ public class BucketManagerImpl implements BucketManager {
|
|||
* @param bucketInfo - OmBucketInfo.
|
||||
*/
|
||||
@Override
|
||||
public OmBucketInfo createBucket(OmBucketInfo bucketInfo) throws IOException {
|
||||
public void createBucket(OmBucketInfo bucketInfo) throws IOException {
|
||||
Preconditions.checkNotNull(bucketInfo);
|
||||
String volumeName = bucketInfo.getVolumeName();
|
||||
String bucketName = bucketInfo.getBucketName();
|
||||
|
@ -165,11 +165,8 @@ public class BucketManagerImpl implements BucketManager {
|
|||
}
|
||||
|
||||
OmBucketInfo omBucketInfo = omBucketInfoBuilder.build();
|
||||
if (!isRatisEnabled) {
|
||||
commitCreateBucketInfoToDB(omBucketInfo);
|
||||
}
|
||||
commitBucketInfoToDB(omBucketInfo);
|
||||
LOG.debug("created bucket: {} in volume: {}", bucketName, volumeName);
|
||||
return omBucketInfo;
|
||||
} catch (IOException | DBException ex) {
|
||||
if (!(ex instanceof OMException)) {
|
||||
LOG.error("Bucket creation failed for bucket:{} in volume:{}",
|
||||
|
@ -182,19 +179,7 @@ public class BucketManagerImpl implements BucketManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public void applyCreateBucket(OmBucketInfo omBucketInfo) throws IOException {
|
||||
Preconditions.checkNotNull(omBucketInfo);
|
||||
try {
|
||||
commitCreateBucketInfoToDB(omBucketInfo);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Apply CreateBucket Failed for bucket: {}, volume: {}",
|
||||
omBucketInfo.getBucketName(), omBucketInfo.getVolumeName(), ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
private void commitCreateBucketInfoToDB(OmBucketInfo omBucketInfo)
|
||||
private void commitBucketInfoToDB(OmBucketInfo omBucketInfo)
|
||||
throws IOException {
|
||||
String dbBucketKey =
|
||||
metadataManager.getBucketKey(omBucketInfo.getVolumeName(),
|
||||
|
@ -243,7 +228,7 @@ public class BucketManagerImpl implements BucketManager {
|
|||
* @throws IOException - On Failure.
|
||||
*/
|
||||
@Override
|
||||
public OmBucketInfo setBucketProperty(OmBucketArgs args) throws IOException {
|
||||
public void setBucketProperty(OmBucketArgs args) throws IOException {
|
||||
Preconditions.checkNotNull(args);
|
||||
String volumeName = args.getVolumeName();
|
||||
String bucketName = args.getBucketName();
|
||||
|
@ -296,11 +281,7 @@ public class BucketManagerImpl implements BucketManager {
|
|||
bucketInfoBuilder.setCreationTime(oldBucketInfo.getCreationTime());
|
||||
|
||||
OmBucketInfo omBucketInfo = bucketInfoBuilder.build();
|
||||
|
||||
if (!isRatisEnabled) {
|
||||
commitSetBucketPropertyInfoToDB(omBucketInfo);
|
||||
}
|
||||
return omBucketInfo;
|
||||
commitBucketInfoToDB(omBucketInfo);
|
||||
} catch (IOException | DBException ex) {
|
||||
if (!(ex instanceof OMException)) {
|
||||
LOG.error("Setting bucket property failed for bucket:{} in volume:{}",
|
||||
|
@ -312,23 +293,6 @@ public class BucketManagerImpl implements BucketManager {
|
|||
}
|
||||
}
|
||||
|
||||
public void applySetBucketProperty(OmBucketInfo omBucketInfo)
|
||||
throws IOException {
|
||||
try {
|
||||
commitSetBucketPropertyInfoToDB(omBucketInfo);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Apply SetBucket property failed for bucket:{} in " +
|
||||
"volume:{}", omBucketInfo.getBucketName(),
|
||||
omBucketInfo.getVolumeName(), ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
private void commitSetBucketPropertyInfoToDB(OmBucketInfo omBucketInfo)
|
||||
throws IOException {
|
||||
commitCreateBucketInfoToDB(omBucketInfo);
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the existing ACL list with remove and add ACLs that are passed.
|
||||
* Remove is done before Add.
|
||||
|
@ -377,10 +341,7 @@ public class BucketManagerImpl implements BucketManager {
|
|||
throw new OMException("Bucket is not empty",
|
||||
OMException.ResultCodes.BUCKET_NOT_EMPTY);
|
||||
}
|
||||
|
||||
if (!isRatisEnabled) {
|
||||
commitDeleteBucketInfoToOMDB(bucketKey);
|
||||
}
|
||||
commitDeleteBucketInfoToOMDB(bucketKey);
|
||||
} catch (IOException ex) {
|
||||
if (!(ex instanceof OMException)) {
|
||||
LOG.error("Delete bucket failed for bucket:{} in volume:{}", bucketName,
|
||||
|
@ -392,20 +353,6 @@ public class BucketManagerImpl implements BucketManager {
|
|||
}
|
||||
}
|
||||
|
||||
public void applyDeleteBucket(String volumeName, String bucketName)
|
||||
throws IOException {
|
||||
Preconditions.checkNotNull(volumeName);
|
||||
Preconditions.checkNotNull(bucketName);
|
||||
try {
|
||||
commitDeleteBucketInfoToOMDB(metadataManager.getBucketKey(volumeName,
|
||||
bucketName));
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Apply DeleteBucket Failed for bucket: {}, volume: {}",
|
||||
bucketName, volumeName, ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
private void commitDeleteBucketInfoToOMDB(String dbBucketKey)
|
||||
throws IOException {
|
||||
metadataManager.getBucketTable().delete(dbBucketKey);
|
||||
|
|
|
@ -316,6 +316,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|||
RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class,
|
||||
ProtobufRpcEngine.class);
|
||||
|
||||
metadataManager = new OmMetadataManagerImpl(configuration);
|
||||
startRatisServer();
|
||||
startRatisClient();
|
||||
|
||||
|
@ -328,7 +329,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|||
|
||||
secConfig = new SecurityConfig(configuration);
|
||||
|
||||
metadataManager = new OmMetadataManagerImpl(configuration);
|
||||
volumeManager = new VolumeManagerImpl(metadataManager, configuration);
|
||||
|
||||
// Create the KMS Key Provider
|
||||
|
@ -1273,7 +1273,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|||
|
||||
BlockingService omService = newReflectiveBlockingService(
|
||||
new OzoneManagerProtocolServerSideTranslatorPB(this, omRatisServer,
|
||||
omRatisClient, isRatisEnabled));
|
||||
isRatisEnabled));
|
||||
return startRpcServer(configuration, omNodeRpcAddr,
|
||||
OzoneManagerProtocolPB.class, omService,
|
||||
handlerCount);
|
||||
|
@ -1724,67 +1724,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|||
volumeManager.applyDeleteVolume(volume, owner, newVolumeList);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public OmBucketInfo startCreateBucket(OmBucketInfo omBucketInfo)
|
||||
throws IOException {
|
||||
Preconditions.checkNotNull(omBucketInfo);
|
||||
if(isAclEnabled) {
|
||||
checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.CREATE,
|
||||
omBucketInfo.getVolumeName(), omBucketInfo.getBucketName(), null);
|
||||
}
|
||||
|
||||
return bucketManager.createBucket(omBucketInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyCreateBucket(OmBucketInfo omBucketInfo) throws IOException {
|
||||
// TODO: Need to add metrics and Audit log for HA requests
|
||||
bucketManager.applyCreateBucket(omBucketInfo);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void startDeleteBucket(String volumeName, String bucketName)
|
||||
throws IOException {
|
||||
// TODO: Need to add metrics and Audit log for HA requests
|
||||
if(isAclEnabled) {
|
||||
checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.CREATE,
|
||||
volumeName, bucketName, null);
|
||||
}
|
||||
|
||||
bucketManager.deleteBucket(volumeName, bucketName);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void applyDeleteBucket(String volumeName, String bucketName)
|
||||
throws IOException {
|
||||
// TODO: Need to add metrics and Audit log for HA requests
|
||||
bucketManager.applyDeleteBucket(volumeName, bucketName);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public OmBucketInfo startSetBucketProperty(OmBucketArgs omBucketArgs)
|
||||
throws IOException {
|
||||
Preconditions.checkNotNull(omBucketArgs);
|
||||
// TODO: Need to add metrics and Audit log for HA requests
|
||||
if(isAclEnabled) {
|
||||
checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.CREATE,
|
||||
omBucketArgs.getVolumeName(), omBucketArgs.getBucketName(), null);
|
||||
}
|
||||
return bucketManager.setBucketProperty(omBucketArgs);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void applySetBucketProperty(OmBucketInfo omBucketInfo)
|
||||
throws IOException {
|
||||
// TODO: Need to add metrics and Audit log for HA requests
|
||||
bucketManager.applySetBucketProperty(omBucketInfo);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if current caller has acl permissions.
|
||||
*
|
||||
|
@ -3135,4 +3074,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|||
public OMFailoverProxyProvider getOMFailoverProxyProvider() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public OMMetrics getOmMetrics() {
|
||||
return metrics;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -137,13 +137,7 @@ public class S3BucketManagerImpl implements S3BucketManager {
|
|||
OMException.ResultCodes.S3_BUCKET_NOT_FOUND);
|
||||
}
|
||||
|
||||
if (isRatisEnabled) {
|
||||
bucketManager.deleteBucket(getOzoneVolumeName(bucketName), bucketName);
|
||||
bucketManager.applyDeleteBucket(getOzoneVolumeName(bucketName),
|
||||
bucketName);
|
||||
} else {
|
||||
bucketManager.deleteBucket(getOzoneVolumeName(bucketName), bucketName);
|
||||
}
|
||||
bucketManager.deleteBucket(getOzoneVolumeName(bucketName), bucketName);
|
||||
omMetadataManager.getS3Table().delete(bucketName);
|
||||
} catch(IOException ex) {
|
||||
throw ex;
|
||||
|
@ -202,11 +196,7 @@ public class S3BucketManagerImpl implements S3BucketManager {
|
|||
.setIsVersionEnabled(Boolean.FALSE)
|
||||
.setStorageType(StorageType.DEFAULT)
|
||||
.build();
|
||||
if (isRatisEnabled) {
|
||||
bucketManager.applyCreateBucket(bucketManager.createBucket(bucketInfo));
|
||||
} else {
|
||||
bucketManager.createBucket(bucketInfo);
|
||||
}
|
||||
bucketManager.createBucket(bucketInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
|
|||
public class OzoneManagerDoubleBuffer {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(OzoneManagerDoubleBuffer.class.getName());
|
||||
LoggerFactory.getLogger(OzoneManagerDoubleBuffer.class);
|
||||
|
||||
// Taken unbounded queue, if sync thread is taking too long time, we
|
||||
// might end up taking huge memory to add entries to the buffer.
|
||||
|
@ -109,7 +109,14 @@ public class OzoneManagerDoubleBuffer {
|
|||
LOG.debug("Sync Iteration {} flushed transactions in this " +
|
||||
"iteration{}", flushIterations.get(),
|
||||
flushedTransactionsSize);
|
||||
|
||||
long lastRatisTransactionIndex =
|
||||
readyBuffer.stream().map(DoubleBufferEntry::getTrxLogIndex)
|
||||
.max(Long::compareTo).get();
|
||||
|
||||
readyBuffer.clear();
|
||||
// cleanup cache.
|
||||
cleanupCache(lastRatisTransactionIndex);
|
||||
// TODO: update the last updated index in OzoneManagerStateMachine.
|
||||
}
|
||||
} catch (InterruptedException ex) {
|
||||
|
@ -134,6 +141,14 @@ public class OzoneManagerDoubleBuffer {
|
|||
}
|
||||
}
|
||||
|
||||
private void cleanupCache(long lastRatisTransactionIndex) {
|
||||
// As now only bucket transactions are handled only called cleanupCache
|
||||
// on bucketTable.
|
||||
// TODO: After supporting all write operations we need to call
|
||||
// cleanupCache on the tables only when buffer has entries for that table.
|
||||
omMetadataManager.getBucketTable().cleanupCache(lastRatisTransactionIndex);
|
||||
}
|
||||
|
||||
/**
|
||||
* Stop OM DoubleBuffer flush thread.
|
||||
*/
|
||||
|
|
|
@ -35,12 +35,20 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.StorageUnit;
|
||||
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMNodeDetails;
|
||||
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
|
||||
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.ratis.RaftConfigKeys;
|
||||
import org.apache.ratis.client.RaftClientConfigKeys;
|
||||
import org.apache.ratis.conf.RaftProperties;
|
||||
|
@ -51,10 +59,15 @@ import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
|||
import org.apache.ratis.protocol.ClientId;
|
||||
import org.apache.ratis.protocol.GroupInfoReply;
|
||||
import org.apache.ratis.protocol.GroupInfoRequest;
|
||||
import org.apache.ratis.protocol.Message;
|
||||
import org.apache.ratis.protocol.NotLeaderException;
|
||||
import org.apache.ratis.protocol.RaftClientReply;
|
||||
import org.apache.ratis.protocol.RaftClientRequest;
|
||||
import org.apache.ratis.protocol.RaftGroup;
|
||||
import org.apache.ratis.protocol.RaftGroupId;
|
||||
import org.apache.ratis.protocol.RaftPeer;
|
||||
import org.apache.ratis.protocol.RaftPeerId;
|
||||
import org.apache.ratis.protocol.StateMachineException;
|
||||
import org.apache.ratis.rpc.RpcType;
|
||||
import org.apache.ratis.rpc.SupportedRpcType;
|
||||
import org.apache.ratis.server.RaftServer;
|
||||
|
@ -66,6 +79,8 @@ import org.apache.ratis.util.TimeDuration;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.apache.hadoop.ozone.om.exceptions.OMException.STATUS_CODE;
|
||||
|
||||
/**
|
||||
* Creates a Ratis server endpoint for OM.
|
||||
*/
|
||||
|
@ -80,7 +95,7 @@ public final class OzoneManagerRatisServer {
|
|||
private final RaftGroup raftGroup;
|
||||
private final RaftPeerId raftPeerId;
|
||||
|
||||
private final OzoneManagerServerProtocol ozoneManager;
|
||||
private final OzoneManager ozoneManager;
|
||||
private final OzoneManagerStateMachine omStateMachine;
|
||||
private final ClientId clientId = ClientId.randomId();
|
||||
|
||||
|
@ -97,6 +112,102 @@ public final class OzoneManagerRatisServer {
|
|||
return CALL_ID_COUNTER.getAndIncrement() & Long.MAX_VALUE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Submit request to Ratis server.
|
||||
* @param omRequest
|
||||
* @return OMResponse - response returned to the client.
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public OMResponse submitRequest(OMRequest omRequest) throws ServiceException {
|
||||
RaftClientRequest raftClientRequest =
|
||||
createWriteRaftClientRequest(omRequest);
|
||||
RaftClientReply raftClientReply;
|
||||
try {
|
||||
raftClientReply = server.submitClientRequestAsync(raftClientRequest)
|
||||
.get();
|
||||
} catch (Exception ex) {
|
||||
throw new ServiceException(ex.getMessage(), ex);
|
||||
}
|
||||
|
||||
return processReply(omRequest, raftClientReply);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create Write RaftClient request from OMRequest.
|
||||
* @param omRequest
|
||||
* @return RaftClientRequest - Raft Client request which is submitted to
|
||||
* ratis server.
|
||||
*/
|
||||
private RaftClientRequest createWriteRaftClientRequest(OMRequest omRequest) {
|
||||
return new RaftClientRequest(clientId, server.getId(), raftGroupId,
|
||||
nextCallId(),
|
||||
Message.valueOf(OMRatisHelper.convertRequestToByteString(omRequest)),
|
||||
RaftClientRequest.writeRequestType(), null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the raftClientReply and return OMResponse.
|
||||
* @param omRequest
|
||||
* @param reply
|
||||
* @return OMResponse - response which is returned to client.
|
||||
* @throws ServiceException
|
||||
*/
|
||||
private OMResponse processReply(OMRequest omRequest, RaftClientReply reply)
|
||||
throws ServiceException {
|
||||
// NotLeader exception is thrown only when the raft server to which the
|
||||
// request is submitted is not the leader. This can happen first time
|
||||
// when client is submitting request to OM.
|
||||
NotLeaderException notLeaderException = reply.getNotLeaderException();
|
||||
if (notLeaderException != null) {
|
||||
throw new ServiceException(notLeaderException);
|
||||
}
|
||||
StateMachineException stateMachineException =
|
||||
reply.getStateMachineException();
|
||||
if (stateMachineException != null) {
|
||||
OMResponse.Builder omResponse = OMResponse.newBuilder();
|
||||
omResponse.setCmdType(omRequest.getCmdType());
|
||||
omResponse.setSuccess(false);
|
||||
omResponse.setMessage(stateMachineException.getCause().getMessage());
|
||||
omResponse.setStatus(parseErrorStatus(
|
||||
stateMachineException.getCause().getMessage()));
|
||||
return omResponse.build();
|
||||
}
|
||||
|
||||
try {
|
||||
return OMRatisHelper.getOMResponseFromRaftClientReply(reply);
|
||||
} catch (InvalidProtocolBufferException ex) {
|
||||
if (ex.getMessage() != null) {
|
||||
throw new ServiceException(ex.getMessage(), ex);
|
||||
} else {
|
||||
throw new ServiceException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: Still need to handle RaftRetry failure exception and
|
||||
// NotReplicated exception.
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse errorMessage received from the exception and convert to
|
||||
* {@link OzoneManagerProtocolProtos.Status}.
|
||||
* @param errorMessage
|
||||
* @return OzoneManagerProtocolProtos.Status
|
||||
*/
|
||||
private OzoneManagerProtocolProtos.Status parseErrorStatus(
|
||||
String errorMessage) {
|
||||
if (errorMessage.contains(STATUS_CODE)) {
|
||||
String errorCode = errorMessage.substring(
|
||||
errorMessage.indexOf(STATUS_CODE) + STATUS_CODE.length());
|
||||
LOG.debug("Parsing error message for error code " +
|
||||
errorCode);
|
||||
return OzoneManagerProtocolProtos.Status.valueOf(errorCode.trim());
|
||||
} else {
|
||||
return OzoneManagerProtocolProtos.Status.INTERNAL_ERROR;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns an OM Ratis server.
|
||||
* @param conf configuration
|
||||
|
@ -108,7 +219,7 @@ public final class OzoneManagerRatisServer {
|
|||
* @throws IOException
|
||||
*/
|
||||
private OzoneManagerRatisServer(Configuration conf,
|
||||
OzoneManagerServerProtocol om,
|
||||
OzoneManager om,
|
||||
String raftGroupIdStr, RaftPeerId localRaftPeerId,
|
||||
InetSocketAddress addr, List<RaftPeer> raftPeers)
|
||||
throws IOException {
|
||||
|
@ -157,7 +268,7 @@ public final class OzoneManagerRatisServer {
|
|||
* Creates an instance of OzoneManagerRatisServer.
|
||||
*/
|
||||
public static OzoneManagerRatisServer newOMRatisServer(
|
||||
Configuration ozoneConf, OzoneManagerServerProtocol omProtocol,
|
||||
Configuration ozoneConf, OzoneManager omProtocol,
|
||||
OMNodeDetails omNodeDetails, List<OMNodeDetails> peerNodes)
|
||||
throws IOException {
|
||||
|
||||
|
@ -202,7 +313,7 @@ public final class OzoneManagerRatisServer {
|
|||
return new OzoneManagerStateMachine(this);
|
||||
}
|
||||
|
||||
public OzoneManagerServerProtocol getOzoneManager() {
|
||||
public OzoneManager getOzoneManager() {
|
||||
return ozoneManager;
|
||||
}
|
||||
|
||||
|
@ -219,6 +330,7 @@ public final class OzoneManagerRatisServer {
|
|||
public void stop() {
|
||||
try {
|
||||
server.close();
|
||||
omStateMachine.stop();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.ozone.container.common.transport.server.ratis
|
|||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
|
||||
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.MultipartInfoApplyInitiateRequest;
|
||||
|
@ -67,15 +66,19 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|||
private final SimpleStateMachineStorage storage =
|
||||
new SimpleStateMachineStorage();
|
||||
private final OzoneManagerRatisServer omRatisServer;
|
||||
private final OzoneManagerServerProtocol ozoneManager;
|
||||
private final OzoneManager ozoneManager;
|
||||
private OzoneManagerHARequestHandler handler;
|
||||
private RaftGroupId raftGroupId;
|
||||
private long lastAppliedIndex = 0;
|
||||
private final OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer;
|
||||
|
||||
public OzoneManagerStateMachine(OzoneManagerRatisServer ratisServer) {
|
||||
this.omRatisServer = ratisServer;
|
||||
this.ozoneManager = omRatisServer.getOzoneManager();
|
||||
this.handler = new OzoneManagerHARequestHandlerImpl(ozoneManager);
|
||||
this.ozoneManagerDoubleBuffer =
|
||||
new OzoneManagerDoubleBuffer(ozoneManager.getMetadataManager());
|
||||
this.handler = new OzoneManagerHARequestHandlerImpl(ozoneManager,
|
||||
ozoneManagerDoubleBuffer);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -192,9 +195,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|||
case CreateVolume:
|
||||
case SetVolumeProperty:
|
||||
case DeleteVolume:
|
||||
case CreateBucket:
|
||||
case SetBucketProperty:
|
||||
case DeleteBucket:
|
||||
newOmRequest = handler.handleStartTransaction(omRequest);
|
||||
break;
|
||||
case AllocateBlock:
|
||||
|
@ -403,7 +403,7 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|||
* @throws ServiceException
|
||||
*/
|
||||
private Message runCommand(OMRequest request, long trxLogIndex) {
|
||||
OMResponse response = handler.handleApplyTransaction(request);
|
||||
OMResponse response = handler.handleApplyTransaction(request, trxLogIndex);
|
||||
lastAppliedIndex = trxLogIndex;
|
||||
return OMRatisHelper.convertResponseToMessage(response);
|
||||
}
|
||||
|
@ -439,4 +439,9 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
|
|||
this.raftGroupId = raftGroupId;
|
||||
}
|
||||
|
||||
|
||||
public void stop() {
|
||||
ozoneManagerDoubleBuffer.stop();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
/**
|
||||
* 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.ratis.utils;
|
||||
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.request.bucket.OMBucketCreateRequest;
|
||||
import org.apache.hadoop.ozone.om.request.bucket.OMBucketDeleteRequest;
|
||||
import org.apache.hadoop.ozone.om.request.bucket.OMBucketSetPropertyRequest;
|
||||
import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
||||
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.Status;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Utility class used by OzoneManager HA.
|
||||
*/
|
||||
public final class OzoneManagerRatisUtils {
|
||||
|
||||
private OzoneManagerRatisUtils() {
|
||||
}
|
||||
/**
|
||||
* Create OMClientRequest which enacpsulates the OMRequest.
|
||||
* @param omRequest
|
||||
* @return OMClientRequest
|
||||
* @throws IOException
|
||||
*/
|
||||
public static OMClientRequest createClientRequest(OMRequest omRequest)
|
||||
throws IOException {
|
||||
Type cmdType = omRequest.getCmdType();
|
||||
switch (cmdType) {
|
||||
case CreateBucket:
|
||||
return new OMBucketCreateRequest(omRequest);
|
||||
case DeleteBucket:
|
||||
return new OMBucketDeleteRequest(omRequest);
|
||||
case SetBucketProperty:
|
||||
return new OMBucketSetPropertyRequest(omRequest);
|
||||
default:
|
||||
// TODO: will update once all request types are implemented.
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert exception result to {@link OzoneManagerProtocolProtos.Status}.
|
||||
* @param exception
|
||||
* @return OzoneManagerProtocolProtos.Status
|
||||
*/
|
||||
public static Status exceptionToResponseStatus(IOException exception) {
|
||||
if (exception instanceof OMException) {
|
||||
return Status.values()[((OMException) exception).getResult().ordinal()];
|
||||
} else {
|
||||
return Status.INTERNAL_ERROR;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
/**
|
||||
* 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.ratis.utils;
|
||||
|
||||
/**
|
||||
* Utility class used by OzoneManager HA.
|
||||
*/
|
|
@ -0,0 +1,73 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMRequest;
|
||||
|
||||
/**
|
||||
* OMClientRequest provides methods which every write OM request should
|
||||
* implement.
|
||||
*/
|
||||
public abstract class OMClientRequest {
|
||||
|
||||
private final OMRequest omRequest;
|
||||
|
||||
public OMClientRequest(OMRequest omRequest) {
|
||||
Preconditions.checkNotNull(omRequest);
|
||||
this.omRequest = omRequest;
|
||||
}
|
||||
/**
|
||||
* Perform pre-execute steps on a OMRequest.
|
||||
*
|
||||
* Called from the RPC context, and generates a OMRequest object which has
|
||||
* all the information that will be either persisted
|
||||
* in RocksDB or returned to the caller once this operation
|
||||
* is executed.
|
||||
*
|
||||
* @return OMRequest that will be serialized and handed off to Ratis for
|
||||
* consensus.
|
||||
*/
|
||||
public abstract OMRequest preExecute(OzoneManager ozoneManager)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Validate the OMRequest and update the cache.
|
||||
* This step should verify that the request can be executed, perform
|
||||
* any authorization steps and update the in-memory cache.
|
||||
|
||||
* This step does not persist the changes to the database.
|
||||
*
|
||||
* @return the response that will be returned to the client.
|
||||
*/
|
||||
public abstract OMClientResponse validateAndUpdateCache(
|
||||
OzoneManager ozoneManager, long transactionLogIndex);
|
||||
|
||||
@VisibleForTesting
|
||||
public OMRequest getOmRequest() {
|
||||
return omRequest;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,204 @@
|
|||
/**
|
||||
* 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.bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.crypto.CipherSuite;
|
||||
import org.apache.hadoop.crypto.key.KeyProvider;
|
||||
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
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.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.BucketEncryptionInfoProto;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.BucketInfo;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
|
||||
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.utils.db.cache.CacheKey;
|
||||
import org.apache.hadoop.utils.db.cache.CacheValue;
|
||||
|
||||
import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CryptoProtocolVersionProto.ENCRYPTION_ZONES;
|
||||
|
||||
/**
|
||||
* Handles CreateBucket Request.
|
||||
*/
|
||||
public class OMBucketCreateRequest extends OMClientRequest {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(OMBucketCreateRequest.class);
|
||||
|
||||
public OMBucketCreateRequest(OMRequest omRequest) {
|
||||
super(omRequest);
|
||||
}
|
||||
@Override
|
||||
public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
|
||||
|
||||
// Get original request.
|
||||
CreateBucketRequest createBucketRequest =
|
||||
getOmRequest().getCreateBucketRequest();
|
||||
BucketInfo bucketInfo = createBucketRequest.getBucketInfo();
|
||||
|
||||
// Get KMS provider.
|
||||
KeyProviderCryptoExtension kmsProvider =
|
||||
ozoneManager.getKmsProvider();
|
||||
|
||||
// Create new Bucket request with new bucket info.
|
||||
CreateBucketRequest.Builder newCreateBucketRequest =
|
||||
createBucketRequest.toBuilder();
|
||||
|
||||
BucketInfo.Builder newBucketInfo = bucketInfo.toBuilder();
|
||||
|
||||
// Set creation time.
|
||||
newBucketInfo.setCreationTime(Time.now());
|
||||
|
||||
if (bucketInfo.hasBeinfo()) {
|
||||
newBucketInfo.setBeinfo(getBeinfo(kmsProvider, bucketInfo));
|
||||
}
|
||||
|
||||
newCreateBucketRequest.setBucketInfo(newBucketInfo.build());
|
||||
return getOmRequest().toBuilder().setCreateBucketRequest(
|
||||
newCreateBucketRequest.build()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
|
||||
long transactionLogIndex) {
|
||||
OMMetrics omMetrics = ozoneManager.getMetrics();
|
||||
omMetrics.incNumBucketCreates();
|
||||
|
||||
OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
|
||||
|
||||
BucketInfo bucketInfo = getBucketInfoFromRequest();
|
||||
|
||||
String volumeName = bucketInfo.getVolumeName();
|
||||
String bucketName = bucketInfo.getBucketName();
|
||||
|
||||
OMResponse.Builder omResponse = OMResponse.newBuilder().setCmdType(
|
||||
OzoneManagerProtocolProtos.Type.CreateBucket).setStatus(
|
||||
OzoneManagerProtocolProtos.Status.OK);
|
||||
OmBucketInfo omBucketInfo = null;
|
||||
|
||||
|
||||
metadataManager.getLock().acquireVolumeLock(volumeName);
|
||||
metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
||||
|
||||
try {
|
||||
String volumeKey = metadataManager.getVolumeKey(volumeName);
|
||||
String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
|
||||
|
||||
//Check if the volume exists
|
||||
if (metadataManager.getVolumeTable().get(volumeKey) == null) {
|
||||
LOG.debug("volume: {} not found ", volumeName);
|
||||
throw new OMException("Volume doesn't exist",
|
||||
OMException.ResultCodes.VOLUME_NOT_FOUND);
|
||||
}
|
||||
//Check if bucket already exists
|
||||
if (metadataManager.getBucketTable().get(bucketKey) != null) {
|
||||
LOG.debug("bucket: {} already exists ", bucketName);
|
||||
throw new OMException("Bucket already exist",
|
||||
OMException.ResultCodes.BUCKET_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
omBucketInfo = OmBucketInfo.getFromProtobuf(bucketInfo);
|
||||
LOG.debug("created bucket: {} in volume: {}", bucketName, volumeName);
|
||||
omMetrics.incNumBuckets();
|
||||
|
||||
// Update table cache.
|
||||
metadataManager.getBucketTable().addCacheEntry(new CacheKey<>(bucketKey),
|
||||
new CacheValue<>(Optional.of(omBucketInfo), transactionLogIndex));
|
||||
|
||||
// TODO: check acls.
|
||||
} catch (IOException ex) {
|
||||
omMetrics.incNumBucketCreateFails();
|
||||
LOG.error("Bucket creation failed for bucket:{} in volume:{}",
|
||||
bucketName, volumeName, ex);
|
||||
omResponse.setStatus(
|
||||
OzoneManagerRatisUtils.exceptionToResponseStatus(ex));
|
||||
omResponse.setMessage(ex.getMessage());
|
||||
omResponse.setSuccess(false);
|
||||
} finally {
|
||||
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
||||
metadataManager.getLock().releaseVolumeLock(volumeName);
|
||||
}
|
||||
omResponse.setCreateBucketResponse(
|
||||
CreateBucketResponse.newBuilder().build());
|
||||
return new OMBucketCreateResponse(omBucketInfo, omResponse.build());
|
||||
}
|
||||
|
||||
|
||||
private BucketInfo getBucketInfoFromRequest() {
|
||||
CreateBucketRequest createBucketRequest =
|
||||
getOmRequest().getCreateBucketRequest();
|
||||
return createBucketRequest.getBucketInfo();
|
||||
}
|
||||
|
||||
private BucketEncryptionInfoProto getBeinfo(
|
||||
KeyProviderCryptoExtension kmsProvider, BucketInfo bucketInfo)
|
||||
throws IOException {
|
||||
BucketEncryptionInfoProto bek = bucketInfo.getBeinfo();
|
||||
BucketEncryptionInfoProto.Builder bekb = null;
|
||||
if (kmsProvider == null) {
|
||||
throw new OMException("Invalid KMS provider, check configuration " +
|
||||
CommonConfigurationKeys.HADOOP_SECURITY_KEY_PROVIDER_PATH,
|
||||
OMException.ResultCodes.INVALID_KMS_PROVIDER);
|
||||
}
|
||||
if (bek.getKeyName() == null) {
|
||||
throw new OMException("Bucket encryption key needed.", OMException
|
||||
.ResultCodes.BUCKET_ENCRYPTION_KEY_NOT_FOUND);
|
||||
}
|
||||
// Talk to KMS to retrieve the bucket encryption key info.
|
||||
KeyProvider.Metadata metadata = kmsProvider.getMetadata(
|
||||
bek.getKeyName());
|
||||
if (metadata == null) {
|
||||
throw new OMException("Bucket encryption key " + bek.getKeyName()
|
||||
+ " doesn't exist.",
|
||||
OMException.ResultCodes.BUCKET_ENCRYPTION_KEY_NOT_FOUND);
|
||||
}
|
||||
// If the provider supports pool for EDEKs, this will fill in the pool
|
||||
kmsProvider.warmUpEncryptedKeys(bek.getKeyName());
|
||||
bekb = BucketEncryptionInfoProto.newBuilder()
|
||||
.setKeyName(bek.getKeyName())
|
||||
.setCryptoProtocolVersion(ENCRYPTION_ZONES)
|
||||
.setSuite(OMPBHelper.convert(
|
||||
CipherSuite.convert(metadata.getCipher())));
|
||||
return bekb.build();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
/**
|
||||
* 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.bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
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.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.bucket.OMBucketDeleteResponse;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
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.om.ratis.utils.OzoneManagerRatisUtils;
|
||||
import org.apache.hadoop.utils.db.cache.CacheKey;
|
||||
import org.apache.hadoop.utils.db.cache.CacheValue;
|
||||
|
||||
/**
|
||||
* Handles DeleteBucket Request.
|
||||
*/
|
||||
public class OMBucketDeleteRequest extends OMClientRequest {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(OMBucketDeleteRequest.class);
|
||||
|
||||
public OMBucketDeleteRequest(OMRequest omRequest) {
|
||||
super(omRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
|
||||
// For Delete Bucket there are no preExecute steps
|
||||
return getOmRequest();
|
||||
}
|
||||
|
||||
@Override
|
||||
public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
|
||||
long transactionLogIndex) {
|
||||
OMMetrics omMetrics = ozoneManager.getMetrics();
|
||||
omMetrics.incNumBucketDeletes();
|
||||
OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
|
||||
|
||||
OMRequest omRequest = getOmRequest();
|
||||
String volumeName = omRequest.getDeleteBucketRequest().getVolumeName();
|
||||
String bucketName = omRequest.getDeleteBucketRequest().getBucketName();
|
||||
|
||||
// acquire lock
|
||||
omMetadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
||||
|
||||
// Generate end user response
|
||||
OMResponse.Builder omResponse = OMResponse.newBuilder()
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCmdType(omRequest.getCmdType());
|
||||
|
||||
try {
|
||||
// No need to check volume exists here, as bucket cannot be created
|
||||
// with out volume creation.
|
||||
//Check if bucket exists
|
||||
String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
|
||||
OmBucketInfo omBucketInfo =
|
||||
omMetadataManager.getBucketTable().get(bucketKey);
|
||||
if (omBucketInfo == null) {
|
||||
LOG.debug("bucket: {} not found ", bucketName);
|
||||
throw new OMException("Bucket doesn't exist",
|
||||
OMException.ResultCodes.BUCKET_NOT_FOUND);
|
||||
}
|
||||
//Check if bucket is empty
|
||||
if (!omMetadataManager.isBucketEmpty(volumeName, bucketName)) {
|
||||
LOG.debug("bucket: {} is not empty ", bucketName);
|
||||
throw new OMException("Bucket is not empty",
|
||||
OMException.ResultCodes.BUCKET_NOT_EMPTY);
|
||||
}
|
||||
omMetrics.decNumBuckets();
|
||||
|
||||
// Update table cache.
|
||||
omMetadataManager.getBucketTable().addCacheEntry(
|
||||
new CacheKey<>(bucketKey),
|
||||
new CacheValue<>(Optional.absent(), transactionLogIndex));
|
||||
// TODO: check acls.
|
||||
} catch (IOException ex) {
|
||||
omMetrics.incNumBucketDeleteFails();
|
||||
LOG.error("Delete bucket failed for bucket:{} in volume:{}", bucketName,
|
||||
volumeName, ex);
|
||||
omResponse.setSuccess(false).setMessage(ex.getMessage())
|
||||
.setStatus(OzoneManagerRatisUtils.exceptionToResponseStatus(ex));
|
||||
} finally {
|
||||
omMetadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
||||
}
|
||||
return new OMBucketDeleteResponse(volumeName, bucketName,
|
||||
omResponse.build());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,189 @@
|
|||
/**
|
||||
* 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.bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||
import org.apache.hadoop.ozone.OzoneAcl;
|
||||
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.KeyValueUtil;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.bucket.OMBucketSetPropertyResponse;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.BucketArgs;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
|
||||
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
|
||||
import org.apache.hadoop.utils.db.cache.CacheKey;
|
||||
import org.apache.hadoop.utils.db.cache.CacheValue;
|
||||
|
||||
/**
|
||||
* Handle SetBucketProperty Request.
|
||||
*/
|
||||
public class OMBucketSetPropertyRequest extends OMClientRequest {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(OMBucketSetPropertyRequest.class);
|
||||
|
||||
public OMBucketSetPropertyRequest(OMRequest omRequest) {
|
||||
super(omRequest);
|
||||
}
|
||||
@Override
|
||||
public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
|
||||
return getOmRequest();
|
||||
}
|
||||
|
||||
@Override
|
||||
public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
|
||||
long transactionLogIndex) {
|
||||
|
||||
OMMetrics omMetrics = ozoneManager.getOmMetrics();
|
||||
|
||||
// This will never be null, on a real Ozone cluster. For tests this might
|
||||
// be null. using mockito, to set omMetrics object, but still getting
|
||||
// null. For now added this not null check.
|
||||
//TODO: Removed not null check from here, once tests got fixed.
|
||||
if (omMetrics != null) {
|
||||
omMetrics.incNumBucketUpdates();
|
||||
}
|
||||
|
||||
OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
|
||||
|
||||
BucketArgs bucketArgs =
|
||||
getOmRequest().getSetBucketPropertyRequest().getBucketArgs();
|
||||
OmBucketArgs omBucketArgs = OmBucketArgs.getFromProtobuf(bucketArgs);
|
||||
|
||||
String volumeName = bucketArgs.getVolumeName();
|
||||
String bucketName = bucketArgs.getBucketName();
|
||||
|
||||
// acquire lock
|
||||
omMetadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
||||
|
||||
OMResponse.Builder omResponse = OMResponse.newBuilder().setCmdType(
|
||||
OzoneManagerProtocolProtos.Type.CreateBucket).setStatus(
|
||||
OzoneManagerProtocolProtos.Status.OK);
|
||||
OmBucketInfo omBucketInfo = null;
|
||||
|
||||
try {
|
||||
String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
|
||||
OmBucketInfo oldBucketInfo =
|
||||
omMetadataManager.getBucketTable().get(bucketKey);
|
||||
//Check if bucket exist
|
||||
if (oldBucketInfo == null) {
|
||||
LOG.debug("bucket: {} not found ", bucketName);
|
||||
throw new OMException("Bucket doesn't exist",
|
||||
OMException.ResultCodes.BUCKET_NOT_FOUND);
|
||||
}
|
||||
OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder();
|
||||
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
|
||||
.setBucketName(oldBucketInfo.getBucketName());
|
||||
bucketInfoBuilder.addAllMetadata(KeyValueUtil
|
||||
.getFromProtobuf(bucketArgs.getMetadataList()));
|
||||
|
||||
//Check ACLs to update
|
||||
if (omBucketArgs.getAddAcls() != null ||
|
||||
omBucketArgs.getRemoveAcls() != null) {
|
||||
bucketInfoBuilder.setAcls(getUpdatedAclList(oldBucketInfo.getAcls(),
|
||||
omBucketArgs.getRemoveAcls(), omBucketArgs.getAddAcls()));
|
||||
LOG.debug("Updating ACLs for bucket: {} in volume: {}",
|
||||
bucketName, volumeName);
|
||||
} else {
|
||||
bucketInfoBuilder.setAcls(oldBucketInfo.getAcls());
|
||||
}
|
||||
|
||||
//Check StorageType to update
|
||||
StorageType storageType = omBucketArgs.getStorageType();
|
||||
if (storageType != null) {
|
||||
bucketInfoBuilder.setStorageType(storageType);
|
||||
LOG.debug("Updating bucket storage type for bucket: {} in volume: {}",
|
||||
bucketName, volumeName);
|
||||
} else {
|
||||
bucketInfoBuilder.setStorageType(oldBucketInfo.getStorageType());
|
||||
}
|
||||
|
||||
//Check Versioning to update
|
||||
Boolean versioning = omBucketArgs.getIsVersionEnabled();
|
||||
if (versioning != null) {
|
||||
bucketInfoBuilder.setIsVersionEnabled(versioning);
|
||||
LOG.debug("Updating bucket versioning for bucket: {} in volume: {}",
|
||||
bucketName, volumeName);
|
||||
} else {
|
||||
bucketInfoBuilder
|
||||
.setIsVersionEnabled(oldBucketInfo.getIsVersionEnabled());
|
||||
}
|
||||
bucketInfoBuilder.setCreationTime(oldBucketInfo.getCreationTime());
|
||||
|
||||
omBucketInfo = bucketInfoBuilder.build();
|
||||
|
||||
// Update table cache.
|
||||
omMetadataManager.getBucketTable().addCacheEntry(
|
||||
new CacheKey<>(bucketKey),
|
||||
new CacheValue<>(Optional.of(omBucketInfo), transactionLogIndex));
|
||||
|
||||
// TODO: check acls.
|
||||
} catch (IOException ex) {
|
||||
if (omMetrics != null) {
|
||||
omMetrics.incNumBucketUpdateFails();
|
||||
}
|
||||
LOG.error("Setting bucket property failed for bucket:{} in volume:{}",
|
||||
bucketName, volumeName, ex);
|
||||
omResponse.setSuccess(false).setMessage(ex.getMessage())
|
||||
.setStatus(OzoneManagerRatisUtils.exceptionToResponseStatus(ex));
|
||||
} finally {
|
||||
omMetadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
||||
}
|
||||
return new OMBucketSetPropertyResponse(omBucketInfo, omResponse.build());
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the existing ACL list with remove and add ACLs that are passed.
|
||||
* Remove is done before Add.
|
||||
*
|
||||
* @param existingAcls - old ACL list.
|
||||
* @param removeAcls - ACLs to be removed.
|
||||
* @param addAcls - ACLs to be added.
|
||||
* @return updated ACL list.
|
||||
*/
|
||||
private List< OzoneAcl > getUpdatedAclList(List<OzoneAcl> existingAcls,
|
||||
List<OzoneAcl> removeAcls, List<OzoneAcl> addAcls) {
|
||||
if (removeAcls != null && !removeAcls.isEmpty()) {
|
||||
existingAcls.removeAll(removeAcls);
|
||||
}
|
||||
if (addAcls != null && !addAcls.isEmpty()) {
|
||||
addAcls.stream().filter(acl -> !existingAcls.contains(acl)).forEach(
|
||||
existingAcls::add);
|
||||
}
|
||||
return existingAcls;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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 contains classes related to bucket requests.
|
||||
*/
|
||||
package org.apache.hadoop.ozone.om.request.bucket;
|
|
@ -0,0 +1,21 @@
|
|||
/**
|
||||
* 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 for handling OMRequests.
|
||||
*/
|
|
@ -18,16 +18,23 @@
|
|||
|
||||
package org.apache.hadoop.ozone.om.response;
|
||||
|
||||
import org.apache.commons.lang.NotImplementedException;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* Interface for OM Responses, each OM response should implement this interface.
|
||||
*/
|
||||
public interface OMClientResponse {
|
||||
public abstract class OMClientResponse {
|
||||
|
||||
private OMResponse omResponse;
|
||||
|
||||
public OMClientResponse(OMResponse omResponse) {
|
||||
this.omResponse = omResponse;
|
||||
}
|
||||
|
||||
/**
|
||||
* Implement logic to add the response to batch.
|
||||
|
@ -35,10 +42,15 @@ public interface OMClientResponse {
|
|||
* @param batchOperation
|
||||
* @throws IOException
|
||||
*/
|
||||
default void addToDBBatch(OMMetadataManager omMetadataManager,
|
||||
BatchOperation batchOperation) throws IOException {
|
||||
throw new NotImplementedException("Not implemented, Each OM Response " +
|
||||
"should implement this method");
|
||||
public abstract void addToDBBatch(OMMetadataManager omMetadataManager,
|
||||
BatchOperation batchOperation) throws IOException;
|
||||
|
||||
/**
|
||||
* Return OMResponse.
|
||||
* @return OMResponse
|
||||
*/
|
||||
public OMResponse getOMResponse() {
|
||||
return omResponse;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,19 +22,24 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.VolumeList;
|
||||
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* Response for CreateBucket request.
|
||||
*/
|
||||
public class OMVolumeCreateResponse implements OMClientResponse {
|
||||
public class OMVolumeCreateResponse extends OMClientResponse {
|
||||
|
||||
private OzoneManagerProtocolProtos.VolumeList volumeList;
|
||||
private VolumeList volumeList;
|
||||
private OmVolumeArgs omVolumeArgs;
|
||||
|
||||
public OMVolumeCreateResponse(OmVolumeArgs omVolumeArgs,
|
||||
OzoneManagerProtocolProtos.VolumeList volumeList) {
|
||||
VolumeList volumeList, OMResponse omResponse) {
|
||||
super(omResponse);
|
||||
this.omVolumeArgs = omVolumeArgs;
|
||||
this.volumeList = volumeList;
|
||||
}
|
||||
|
@ -53,12 +58,13 @@ public class OMVolumeCreateResponse implements OMClientResponse {
|
|||
volumeList);
|
||||
}
|
||||
|
||||
public OzoneManagerProtocolProtos.VolumeList getVolumeList() {
|
||||
public VolumeList getVolumeList() {
|
||||
return volumeList;
|
||||
}
|
||||
|
||||
public OmVolumeArgs getOmVolumeArgs() {
|
||||
return omVolumeArgs;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -21,19 +21,23 @@ package org.apache.hadoop.ozone.om.response;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.VolumeList;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* Response for CreateVolume request.
|
||||
*/
|
||||
public class OMVolumeDeleteResponse implements OMClientResponse {
|
||||
public class OMVolumeDeleteResponse extends OMClientResponse {
|
||||
private String volume;
|
||||
private String owner;
|
||||
private OzoneManagerProtocolProtos.VolumeList updatedVolumeList;
|
||||
private VolumeList updatedVolumeList;
|
||||
|
||||
public OMVolumeDeleteResponse(String volume, String owner,
|
||||
OzoneManagerProtocolProtos.VolumeList updatedVolumeList) {
|
||||
VolumeList updatedVolumeList, OMResponse omResponse) {
|
||||
super(omResponse);
|
||||
this.volume = volume;
|
||||
this.owner = owner;
|
||||
this.updatedVolumeList = updatedVolumeList;
|
||||
|
@ -43,8 +47,7 @@ public class OMVolumeDeleteResponse implements OMClientResponse {
|
|||
public void addToDBBatch(OMMetadataManager omMetadataManager,
|
||||
BatchOperation batchOperation) throws IOException {
|
||||
String dbUserKey = omMetadataManager.getUserKey(owner);
|
||||
OzoneManagerProtocolProtos.VolumeList volumeList =
|
||||
updatedVolumeList;
|
||||
VolumeList volumeList = updatedVolumeList;
|
||||
if (updatedVolumeList.getVolumeNamesList().size() == 0) {
|
||||
omMetadataManager.getUserTable().deleteWithBatch(batchOperation,
|
||||
dbUserKey);
|
||||
|
@ -55,5 +58,6 @@ public class OMVolumeDeleteResponse implements OMClientResponse {
|
|||
omMetadataManager.getVolumeTable().deleteWithBatch(batchOperation,
|
||||
omMetadataManager.getVolumeKey(volume));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -16,22 +16,27 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.ozone.om.response;
|
||||
package org.apache.hadoop.ozone.om.response.bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* Response for CreateBucket request.
|
||||
*/
|
||||
public final class OMBucketCreateResponse implements OMClientResponse {
|
||||
public final class OMBucketCreateResponse extends OMClientResponse {
|
||||
|
||||
private final OmBucketInfo omBucketInfo;
|
||||
|
||||
public OMBucketCreateResponse(OmBucketInfo omBucketInfo) {
|
||||
public OMBucketCreateResponse(OmBucketInfo omBucketInfo,
|
||||
OMResponse omResponse) {
|
||||
super(omResponse);
|
||||
this.omBucketInfo = omBucketInfo;
|
||||
}
|
||||
|
||||
|
@ -48,5 +53,6 @@ public final class OMBucketCreateResponse implements OMClientResponse {
|
|||
public OmBucketInfo getOmBucketInfo() {
|
||||
return omBucketInfo;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -16,23 +16,27 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.ozone.om.response;
|
||||
package org.apache.hadoop.ozone.om.response.bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
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.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* Response for DeleteBucket request.
|
||||
*/
|
||||
public final class OMBucketDeleteResponse implements OMClientResponse {
|
||||
public final class OMBucketDeleteResponse extends OMClientResponse {
|
||||
|
||||
private String volumeName;
|
||||
private String bucketName;
|
||||
|
||||
public OMBucketDeleteResponse(
|
||||
String volumeName, String bucketName) {
|
||||
String volumeName, String bucketName,
|
||||
OzoneManagerProtocolProtos.OMResponse omResponse) {
|
||||
super(omResponse);
|
||||
this.volumeName = volumeName;
|
||||
this.bucketName = bucketName;
|
||||
}
|
||||
|
@ -53,5 +57,6 @@ public final class OMBucketDeleteResponse implements OMClientResponse {
|
|||
public String getBucketName() {
|
||||
return bucketName;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,51 @@
|
|||
/**
|
||||
* 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.bucket;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* Response for SetBucketProperty request.
|
||||
*/
|
||||
public class OMBucketSetPropertyResponse extends OMClientResponse {
|
||||
private OmBucketInfo omBucketInfo;
|
||||
|
||||
public OMBucketSetPropertyResponse(OmBucketInfo omBucketInfo,
|
||||
OMResponse omResponse) {
|
||||
super(omResponse);
|
||||
this.omBucketInfo = omBucketInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addToDBBatch(OMMetadataManager omMetadataManager,
|
||||
BatchOperation batchOperation) throws IOException {
|
||||
String dbBucketKey =
|
||||
omMetadataManager.getBucketKey(omBucketInfo.getVolumeName(),
|
||||
omBucketInfo.getBucketName());
|
||||
omMetadataManager.getBucketTable().putWithBatch(batchOperation, dbBucketKey,
|
||||
omBucketInfo);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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 contains classes related to bucket responses.
|
||||
*/
|
||||
package org.apache.hadoop.ozone.om.response.bucket;
|
|
@ -41,8 +41,10 @@ public interface OzoneManagerHARequestHandler extends RequestHandler {
|
|||
/**
|
||||
* Handle Apply Transaction Requests from OzoneManager StateMachine.
|
||||
* @param omRequest
|
||||
* @param transactionLogIndex - ratis transaction log index
|
||||
* @return OMResponse
|
||||
*/
|
||||
OMResponse handleApplyTransaction(OMRequest omRequest);
|
||||
OMResponse handleApplyTransaction(OMRequest omRequest,
|
||||
long transactionLogIndex);
|
||||
|
||||
}
|
||||
|
|
|
@ -19,25 +19,19 @@ package org.apache.hadoop.ozone.protocolPB;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmDeleteVolumeResponse;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeOwnerChangeResponse;
|
||||
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
||||
import org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer;
|
||||
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
|
||||
import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateVolumeRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateVolumeResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.DeleteBucketRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.DeleteBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.DeleteVolumeRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
|
@ -46,10 +40,6 @@ 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
|
||||
.SetBucketPropertyRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.SetBucketPropertyResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.SetVolumePropertyRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
|
@ -70,8 +60,12 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|||
public class OzoneManagerHARequestHandlerImpl
|
||||
extends OzoneManagerRequestHandler implements OzoneManagerHARequestHandler {
|
||||
|
||||
public OzoneManagerHARequestHandlerImpl(OzoneManagerServerProtocol om) {
|
||||
private OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer;
|
||||
|
||||
public OzoneManagerHARequestHandlerImpl(OzoneManager om,
|
||||
OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer) {
|
||||
super(om);
|
||||
this.ozoneManagerDoubleBuffer = ozoneManagerDoubleBuffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -90,15 +84,6 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
case DeleteVolume:
|
||||
newOmRequest = handleDeleteVolumeStart(omRequest);
|
||||
break;
|
||||
case CreateBucket:
|
||||
newOmRequest = handleCreateBucketStart(omRequest);
|
||||
break;
|
||||
case SetBucketProperty:
|
||||
newOmRequest = handleSetBucketPropertyStart(omRequest);
|
||||
break;
|
||||
case DeleteBucket:
|
||||
newOmRequest = handleDeleteBucketRequestStart(omRequest);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unrecognized Command Type:" + cmdType);
|
||||
}
|
||||
|
@ -107,7 +92,8 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
|
||||
|
||||
@Override
|
||||
public OMResponse handleApplyTransaction(OMRequest omRequest) {
|
||||
public OMResponse handleApplyTransaction(OMRequest omRequest,
|
||||
long transactionLogIndex) {
|
||||
LOG.debug("Received OMRequest: {}, ", omRequest);
|
||||
Type cmdType = omRequest.getCmdType();
|
||||
OMResponse.Builder responseBuilder =
|
||||
|
@ -128,17 +114,26 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
handleDeleteVolumeApply(omRequest));
|
||||
break;
|
||||
case CreateBucket:
|
||||
responseBuilder.setCreateBucketResponse(
|
||||
handleCreateBucketApply(omRequest));
|
||||
break;
|
||||
case SetBucketProperty:
|
||||
responseBuilder.setSetBucketPropertyResponse(
|
||||
handleSetBucketPropertyApply(omRequest));
|
||||
break;
|
||||
case DeleteBucket:
|
||||
responseBuilder.setDeleteBucketResponse(
|
||||
handleDeleteBucketApply(omRequest));
|
||||
break;
|
||||
case SetBucketProperty:
|
||||
//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
|
||||
// care by stateMachine. And also integrate both HA and NON HA code
|
||||
// paths.
|
||||
OMClientRequest omClientRequest =
|
||||
OzoneManagerRatisUtils.createClientRequest(omRequest);
|
||||
OMClientResponse omClientResponse =
|
||||
omClientRequest.validateAndUpdateCache(getOzoneManager(),
|
||||
transactionLogIndex);
|
||||
|
||||
// If any error we have got when validateAndUpdateCache, OMResponse
|
||||
// Status is set with Error Code other than OK, in that case don't
|
||||
// add this to double buffer.
|
||||
if (omClientResponse.getOMResponse().getStatus() == Status.OK) {
|
||||
ozoneManagerDoubleBuffer.add(omClientResponse, transactionLogIndex);
|
||||
}
|
||||
return omClientResponse.getOMResponse();
|
||||
default:
|
||||
// As all request types are not changed so we need to call handle
|
||||
// here.
|
||||
|
@ -160,7 +155,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
throws IOException {
|
||||
VolumeInfo volumeInfo = omRequest.getCreateVolumeRequest().getVolumeInfo();
|
||||
OzoneManagerProtocolProtos.VolumeList volumeList =
|
||||
getOzoneManagerServerProtocol().startCreateVolume(
|
||||
getOzoneManager().startCreateVolume(
|
||||
OmVolumeArgs.getFromProtobuf(volumeInfo));
|
||||
|
||||
CreateVolumeRequest createVolumeRequest =
|
||||
|
@ -176,7 +171,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
omRequest.getCreateVolumeRequest().getVolumeInfo();
|
||||
VolumeList volumeList =
|
||||
omRequest.getCreateVolumeRequest().getVolumeList();
|
||||
getOzoneManagerServerProtocol().applyCreateVolume(
|
||||
getOzoneManager().applyCreateVolume(
|
||||
OmVolumeArgs.getFromProtobuf(volumeInfo),
|
||||
volumeList);
|
||||
return CreateVolumeResponse.newBuilder().build();
|
||||
|
@ -191,7 +186,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
if (setVolumePropertyRequest.hasQuotaInBytes()) {
|
||||
long quota = setVolumePropertyRequest.getQuotaInBytes();
|
||||
OmVolumeArgs omVolumeArgs =
|
||||
getOzoneManagerServerProtocol().startSetQuota(volume, quota);
|
||||
getOzoneManager().startSetQuota(volume, quota);
|
||||
SetVolumePropertyRequest newSetVolumePropertyRequest =
|
||||
SetVolumePropertyRequest.newBuilder().setVolumeName(volume)
|
||||
.setVolumeInfo(omVolumeArgs.getProtobuf()).build();
|
||||
|
@ -201,7 +196,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
} else {
|
||||
String owner = setVolumePropertyRequest.getOwnerName();
|
||||
OmVolumeOwnerChangeResponse omVolumeOwnerChangeResponse =
|
||||
getOzoneManagerServerProtocol().startSetOwner(volume, owner);
|
||||
getOzoneManager().startSetOwner(volume, owner);
|
||||
// If volumeLists become large and as ratis writes the request to disk we
|
||||
// might take more space if the lists become very big in size. We might
|
||||
// need to revisit this if it becomes problem
|
||||
|
@ -230,11 +225,11 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
omRequest.getSetVolumePropertyRequest();
|
||||
|
||||
if (setVolumePropertyRequest.hasQuotaInBytes()) {
|
||||
getOzoneManagerServerProtocol().applySetQuota(
|
||||
getOzoneManager().applySetQuota(
|
||||
OmVolumeArgs.getFromProtobuf(
|
||||
setVolumePropertyRequest.getVolumeInfo()));
|
||||
} else {
|
||||
getOzoneManagerServerProtocol().applySetOwner(
|
||||
getOzoneManager().applySetOwner(
|
||||
setVolumePropertyRequest.getOriginalOwner(),
|
||||
setVolumePropertyRequest.getOldOwnerVolumeList(),
|
||||
setVolumePropertyRequest.getNewOwnerVolumeList(),
|
||||
|
@ -252,7 +247,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
String volume = deleteVolumeRequest.getVolumeName();
|
||||
|
||||
OmDeleteVolumeResponse omDeleteVolumeResponse =
|
||||
getOzoneManagerServerProtocol().startDeleteVolume(volume);
|
||||
getOzoneManager().startDeleteVolume(volume);
|
||||
|
||||
DeleteVolumeRequest newDeleteVolumeRequest =
|
||||
DeleteVolumeRequest.newBuilder().setVolumeList(
|
||||
|
@ -272,97 +267,10 @@ public class OzoneManagerHARequestHandlerImpl
|
|||
DeleteVolumeRequest deleteVolumeRequest =
|
||||
omRequest.getDeleteVolumeRequest();
|
||||
|
||||
getOzoneManagerServerProtocol().applyDeleteVolume(
|
||||
getOzoneManager().applyDeleteVolume(
|
||||
deleteVolumeRequest.getVolumeName(), deleteVolumeRequest.getOwner(),
|
||||
deleteVolumeRequest.getVolumeList());
|
||||
|
||||
return DeleteVolumeResponse.newBuilder().build();
|
||||
}
|
||||
|
||||
private OMRequest handleCreateBucketStart(OMRequest omRequest)
|
||||
throws IOException {
|
||||
|
||||
CreateBucketRequest createBucketRequest =
|
||||
omRequest.getCreateBucketRequest();
|
||||
|
||||
OmBucketInfo omBucketInfo =
|
||||
getOzoneManagerServerProtocol().startCreateBucket(
|
||||
OmBucketInfo.getFromProtobuf(createBucketRequest.getBucketInfo()));
|
||||
|
||||
CreateBucketRequest newCreateBucketRequest =
|
||||
CreateBucketRequest.newBuilder().setBucketInfo(
|
||||
omBucketInfo.getProtobuf()).build();
|
||||
return omRequest.toBuilder().setCreateBucketRequest(newCreateBucketRequest)
|
||||
.build();
|
||||
|
||||
}
|
||||
|
||||
|
||||
private CreateBucketResponse handleCreateBucketApply(OMRequest omRequest)
|
||||
throws IOException {
|
||||
CreateBucketRequest createBucketRequest =
|
||||
omRequest.getCreateBucketRequest();
|
||||
|
||||
getOzoneManagerServerProtocol().applyCreateBucket(
|
||||
OmBucketInfo.getFromProtobuf(createBucketRequest.getBucketInfo()));
|
||||
|
||||
return CreateBucketResponse.newBuilder().build();
|
||||
}
|
||||
|
||||
|
||||
private OMRequest handleDeleteBucketRequestStart(OMRequest omRequest)
|
||||
throws IOException {
|
||||
|
||||
DeleteBucketRequest deleteBucketRequest =
|
||||
omRequest.getDeleteBucketRequest();
|
||||
getOzoneManagerServerProtocol().startDeleteBucket(
|
||||
deleteBucketRequest.getVolumeName(),
|
||||
deleteBucketRequest.getBucketName());
|
||||
|
||||
return omRequest;
|
||||
}
|
||||
|
||||
private DeleteBucketResponse handleDeleteBucketApply(OMRequest omRequest)
|
||||
throws IOException {
|
||||
|
||||
DeleteBucketRequest deleteBucketRequest =
|
||||
omRequest.getDeleteBucketRequest();
|
||||
|
||||
getOzoneManagerServerProtocol().applyDeleteBucket(
|
||||
deleteBucketRequest.getVolumeName(),
|
||||
deleteBucketRequest.getBucketName());
|
||||
|
||||
return DeleteBucketResponse.newBuilder().build();
|
||||
}
|
||||
|
||||
private OMRequest handleSetBucketPropertyStart(
|
||||
OMRequest omRequest) throws IOException {
|
||||
SetBucketPropertyRequest setBucketPropertyRequest =
|
||||
omRequest.getSetBucketPropertyRequest();
|
||||
|
||||
OmBucketInfo omBucketInfo =
|
||||
getOzoneManagerServerProtocol().startSetBucketProperty(
|
||||
OmBucketArgs.getFromProtobuf(setBucketPropertyRequest.getBucketArgs()));
|
||||
|
||||
SetBucketPropertyRequest newSetBucketPropertyRequest =
|
||||
SetBucketPropertyRequest.newBuilder()
|
||||
.setBucketInfo(omBucketInfo.getProtobuf()).build();
|
||||
|
||||
return omRequest.toBuilder().setSetBucketPropertyRequest(
|
||||
newSetBucketPropertyRequest).build();
|
||||
}
|
||||
|
||||
private SetBucketPropertyResponse handleSetBucketPropertyApply(
|
||||
OMRequest omRequest) throws IOException {
|
||||
SetBucketPropertyRequest setBucketPropertyRequest =
|
||||
omRequest.getSetBucketPropertyRequest();
|
||||
|
||||
getOzoneManagerServerProtocol().applySetBucketProperty(
|
||||
OmBucketInfo.getFromProtobuf(setBucketPropertyRequest.getBucketInfo()));
|
||||
|
||||
return SetBucketPropertyResponse.newBuilder().build();
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -16,13 +16,16 @@
|
|||
*/
|
||||
package org.apache.hadoop.ozone.protocolPB;
|
||||
|
||||
|
||||
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
||||
import org.apache.hadoop.ozone.OmUtils;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.exceptions.NotLeaderException;
|
||||
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
||||
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
|
||||
import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisClient;
|
||||
import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
|
||||
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
|
||||
import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
||||
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;
|
||||
|
||||
|
@ -33,6 +36,7 @@ import org.apache.ratis.protocol.RaftPeerId;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
|
@ -45,9 +49,9 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
|
|||
private static final Logger LOG = LoggerFactory
|
||||
.getLogger(OzoneManagerProtocolServerSideTranslatorPB.class);
|
||||
private final OzoneManagerRatisServer omRatisServer;
|
||||
private final OzoneManagerRatisClient omRatisClient;
|
||||
private final RequestHandler handler;
|
||||
private final boolean isRatisEnabled;
|
||||
private final OzoneManager ozoneManager;
|
||||
|
||||
/**
|
||||
* Constructs an instance of the server handler.
|
||||
|
@ -55,12 +59,13 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
|
|||
* @param impl OzoneManagerProtocolPB
|
||||
*/
|
||||
public OzoneManagerProtocolServerSideTranslatorPB(
|
||||
OzoneManagerServerProtocol impl, OzoneManagerRatisServer ratisServer,
|
||||
OzoneManagerRatisClient ratisClient, boolean enableRatis) {
|
||||
OzoneManager impl, OzoneManagerRatisServer ratisServer,
|
||||
boolean enableRatis) {
|
||||
this.ozoneManager = impl;
|
||||
handler = new OzoneManagerRequestHandler(impl);
|
||||
this.omRatisServer = ratisServer;
|
||||
this.omRatisClient = ratisClient;
|
||||
this.isRatisEnabled = enableRatis;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -80,6 +85,17 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
|
|||
if (OmUtils.isReadOnly(request)) {
|
||||
return submitReadRequestToOM(request);
|
||||
} else {
|
||||
// PreExecute if needed.
|
||||
try {
|
||||
OMClientRequest omClientRequest =
|
||||
OzoneManagerRatisUtils.createClientRequest(request);
|
||||
if (omClientRequest != null) {
|
||||
request = omClientRequest.preExecute(ozoneManager);
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
// As some of the preExecute returns error. So handle here.
|
||||
return createErrorResponse(request, ex);
|
||||
}
|
||||
return submitRequestToRatis(request);
|
||||
}
|
||||
} else {
|
||||
|
@ -89,12 +105,46 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
|
|||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create OMResponse from the specified OMRequest and exception.
|
||||
* @param omRequest
|
||||
* @param exception
|
||||
* @return OMResponse
|
||||
*/
|
||||
private OMResponse createErrorResponse(
|
||||
OMRequest omRequest, IOException exception) {
|
||||
OzoneManagerProtocolProtos.Type cmdType = omRequest.getCmdType();
|
||||
switch (cmdType) {
|
||||
case CreateBucket:
|
||||
OMResponse.Builder omResponse = OMResponse.newBuilder()
|
||||
.setStatus(
|
||||
OzoneManagerRatisUtils.exceptionToResponseStatus(exception))
|
||||
.setCmdType(cmdType)
|
||||
.setSuccess(false);
|
||||
if (exception.getMessage() != null) {
|
||||
omResponse.setMessage(exception.getMessage());
|
||||
}
|
||||
return omResponse.build();
|
||||
case DeleteBucket:
|
||||
case SetBucketProperty:
|
||||
// In these cases, we can return null. As this method is called when
|
||||
// some error occurred in preExecute. For these request types
|
||||
// preExecute is do nothing.
|
||||
return null;
|
||||
default:
|
||||
// We shall never come here.
|
||||
return null;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Submits request to OM's Ratis server.
|
||||
*/
|
||||
private OMResponse submitRequestToRatis(OMRequest request)
|
||||
throws ServiceException {
|
||||
return omRatisClient.sendCommand(request);
|
||||
//TODO: Need to remove OzoneManagerRatisClient, as now we are using
|
||||
// RatisServer Api's.
|
||||
return omRatisServer.submitRequest(request);
|
||||
}
|
||||
|
||||
private OMResponse submitReadRequestToOM(OMRequest request)
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|||
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ozone.OzoneAcl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
|
@ -43,7 +44,6 @@ import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|||
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
|
||||
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AddAclResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
|
||||
|
@ -138,9 +138,9 @@ import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.
|
|||
public class OzoneManagerRequestHandler implements RequestHandler {
|
||||
static final Logger LOG =
|
||||
LoggerFactory.getLogger(OzoneManagerRequestHandler.class);
|
||||
private final OzoneManagerServerProtocol impl;
|
||||
private final OzoneManager impl;
|
||||
|
||||
public OzoneManagerRequestHandler(OzoneManagerServerProtocol om) {
|
||||
public OzoneManagerRequestHandler(OzoneManager om) {
|
||||
this.impl = om;
|
||||
}
|
||||
|
||||
|
@ -1089,10 +1089,6 @@ public class OzoneManagerRequestHandler implements RequestHandler {
|
|||
.build();
|
||||
}
|
||||
|
||||
protected OzoneManagerServerProtocol getOzoneManagerServerProtocol() {
|
||||
return impl;
|
||||
}
|
||||
|
||||
private ListStatusResponse listStatus(
|
||||
ListStatusRequest request) throws IOException {
|
||||
KeyArgs keyArgs = request.getKeyArgs();
|
||||
|
@ -1112,4 +1108,8 @@ public class OzoneManagerRequestHandler implements RequestHandler {
|
|||
}
|
||||
return listStatusResponseBuilder.build();
|
||||
}
|
||||
|
||||
protected OzoneManager getOzoneManager() {
|
||||
return impl;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,11 @@ import java.io.IOException;
|
|||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -102,17 +107,24 @@ public class TestOzoneManagerDoubleBufferWithDummyResponse {
|
|||
OmBucketInfo omBucketInfo =
|
||||
OmBucketInfo.newBuilder().setVolumeName(volumeName)
|
||||
.setBucketName(bucketName).setCreationTime(Time.now()).build();
|
||||
return new OMDummyCreateBucketResponse(omBucketInfo);
|
||||
return new OMDummyCreateBucketResponse(omBucketInfo,
|
||||
OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCreateBucketResponse(CreateBucketResponse.newBuilder().build())
|
||||
.build());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* DummyCreatedBucket Response class used in testing.
|
||||
*/
|
||||
public static class OMDummyCreateBucketResponse implements OMClientResponse {
|
||||
public static class OMDummyCreateBucketResponse extends OMClientResponse {
|
||||
private final OmBucketInfo omBucketInfo;
|
||||
|
||||
public OMDummyCreateBucketResponse(OmBucketInfo omBucketInfo) {
|
||||
public OMDummyCreateBucketResponse(OmBucketInfo omBucketInfo,
|
||||
OMResponse omResponse) {
|
||||
super(omResponse);
|
||||
this.omBucketInfo = omBucketInfo;
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.UUID;
|
|||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -37,9 +38,18 @@ import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
|||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||
import org.apache.hadoop.ozone.om.response.OMVolumeCreateResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
|
||||
import org.apache.hadoop.ozone.om.response.OMBucketCreateResponse;
|
||||
import org.apache.hadoop.ozone.om.response.OMBucketDeleteResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateVolumeResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.DeleteBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.VolumeList;
|
||||
import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
|
||||
import org.apache.hadoop.ozone.om.response.bucket.OMBucketDeleteResponse;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
@ -376,7 +386,12 @@ public class TestOzoneManagerDoubleBufferWithOMResponse {
|
|||
|
||||
VolumeList volumeList = VolumeList.newBuilder()
|
||||
.addVolumeNames(volumeName).build();
|
||||
return new OMVolumeCreateResponse(omVolumeArgs, volumeList);
|
||||
return new OMVolumeCreateResponse(omVolumeArgs, volumeList,
|
||||
OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateVolume)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCreateVolumeResponse(CreateVolumeResponse.newBuilder().build())
|
||||
.build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -390,7 +405,11 @@ public class TestOzoneManagerDoubleBufferWithOMResponse {
|
|||
OmBucketInfo omBucketInfo =
|
||||
OmBucketInfo.newBuilder().setVolumeName(volumeName)
|
||||
.setBucketName(bucketName).setCreationTime(Time.now()).build();
|
||||
return new OMBucketCreateResponse(omBucketInfo);
|
||||
return new OMBucketCreateResponse(omBucketInfo, OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCreateBucketResponse(CreateBucketResponse.newBuilder().build())
|
||||
.build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -401,7 +420,12 @@ public class TestOzoneManagerDoubleBufferWithOMResponse {
|
|||
*/
|
||||
private OMBucketDeleteResponse deleteBucket(String volumeName,
|
||||
String bucketName) {
|
||||
return new OMBucketDeleteResponse(volumeName, bucketName);
|
||||
return new OMBucketDeleteResponse(volumeName, bucketName,
|
||||
OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.DeleteBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setDeleteBucketResponse(DeleteBucketResponse.newBuilder().build())
|
||||
.build());
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -31,7 +31,10 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|||
import org.apache.hadoop.ozone.OmUtils;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OMNodeDetails;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
|
@ -41,22 +44,32 @@ import org.apache.ratis.util.LifeCycle;
|
|||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.mockito.Mockito;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Test OM Ratis server.
|
||||
*/
|
||||
public class TestOzoneManagerRatisServer {
|
||||
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OzoneConfiguration conf;
|
||||
private OzoneManagerRatisServer omRatisServer;
|
||||
private OzoneManagerRatisClient omRatisClient;
|
||||
private String omID;
|
||||
private String clientId = UUID.randomUUID().toString();
|
||||
private static final long LEADER_ELECTION_TIMEOUT = 500L;
|
||||
private OMMetadataManager omMetadataManager;
|
||||
private OzoneManager ozoneManager;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
|
@ -80,7 +93,13 @@ public class TestOzoneManagerRatisServer {
|
|||
.setOMServiceId(OzoneConsts.OM_SERVICE_ID_DEFAULT)
|
||||
.build();
|
||||
// Starts a single node Ratis server
|
||||
omRatisServer = OzoneManagerRatisServer.newOMRatisServer(conf, null,
|
||||
ozoneManager = Mockito.mock(OzoneManager.class);
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
|
||||
omRatisServer = OzoneManagerRatisServer.newOMRatisServer(conf, ozoneManager,
|
||||
omNodeDetails, Collections.emptyList());
|
||||
omRatisServer.start();
|
||||
omRatisClient = OzoneManagerRatisClient.newOzoneManagerRatisClient(omID,
|
||||
|
@ -165,8 +184,8 @@ public class TestOzoneManagerRatisServer {
|
|||
.build();
|
||||
// Starts a single node Ratis server
|
||||
OzoneManagerRatisServer newOmRatisServer = OzoneManagerRatisServer
|
||||
.newOMRatisServer(newConf, null,
|
||||
omNodeDetails, Collections.emptyList());
|
||||
.newOMRatisServer(newConf, ozoneManager, omNodeDetails,
|
||||
Collections.emptyList());
|
||||
newOmRatisServer.start();
|
||||
OzoneManagerRatisClient newOmRatisClient = OzoneManagerRatisClient
|
||||
.newOzoneManagerRatisClient(
|
||||
|
|
|
@ -26,7 +26,10 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OMNodeDetails;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||
|
@ -75,6 +78,8 @@ public class TestOzoneManagerStateMachine {
|
|||
private OzoneManagerHARequestHandler requestHandler;
|
||||
private RaftGroupId raftGroupId;
|
||||
private OzoneManagerStateMachine ozoneManagerStateMachine;
|
||||
private OMMetadataManager omMetadataManager;
|
||||
private OzoneManager ozoneManager;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
@ -97,8 +102,14 @@ public class TestOzoneManagerStateMachine {
|
|||
.setOMNodeId(omID)
|
||||
.setOMServiceId(OzoneConsts.OM_SERVICE_ID_DEFAULT)
|
||||
.build();
|
||||
ozoneManager = Mockito.mock(OzoneManager.class);
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
temporaryFolder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
|
||||
// Starts a single node Ratis server
|
||||
omRatisServer = OzoneManagerRatisServer.newOMRatisServer(conf, null,
|
||||
omRatisServer = OzoneManagerRatisServer.newOMRatisServer(conf, ozoneManager,
|
||||
omNodeDetails, Collections.emptyList());
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* Helper class to test OMClientRequest classes.
|
||||
*/
|
||||
public final class TestOMRequestUtils {
|
||||
|
||||
private TestOMRequestUtils() {
|
||||
//Do nothing
|
||||
}
|
||||
|
||||
/**
|
||||
* Add's volume and bucket creation entries to OM DB.
|
||||
* @param volumeName
|
||||
* @param bucketName
|
||||
* @param omMetadataManager
|
||||
* @throws Exception
|
||||
*/
|
||||
public static void addVolumeAndBucketToDB(String volumeName,
|
||||
String bucketName, OMMetadataManager omMetadataManager) throws Exception {
|
||||
|
||||
addVolumeToDB(volumeName, omMetadataManager);
|
||||
|
||||
OmBucketInfo omBucketInfo =
|
||||
OmBucketInfo.newBuilder().setVolumeName(volumeName)
|
||||
.setBucketName(bucketName).setCreationTime(Time.now()).build();
|
||||
|
||||
omMetadataManager.getBucketTable().put(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName), omBucketInfo);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add's volume creation entry to OM DB.
|
||||
* @param volumeName
|
||||
* @param omMetadataManager
|
||||
* @throws Exception
|
||||
*/
|
||||
public static void addVolumeToDB(String volumeName,
|
||||
OMMetadataManager omMetadataManager) throws Exception {
|
||||
OmVolumeArgs omVolumeArgs =
|
||||
OmVolumeArgs.newBuilder().setCreationTime(Time.now())
|
||||
.setVolume(volumeName).setAdminName(UUID.randomUUID().toString())
|
||||
.setOwnerName(UUID.randomUUID().toString()).build();
|
||||
omMetadataManager.getVolumeTable().put(
|
||||
omMetadataManager.getVolumeKey(volumeName), omVolumeArgs);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,270 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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.bucket;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OMMetrics;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
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
|
||||
.StorageTypeProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Tests OMBucketCreateRequest class, which handles CreateBucket request.
|
||||
*/
|
||||
public class TestOMBucketCreateRequest {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OzoneManager ozoneManager;
|
||||
private OMMetrics omMetrics;
|
||||
private OMMetadataManager omMetadataManager;
|
||||
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
ozoneManager = Mockito.mock(OzoneManager.class);
|
||||
omMetrics = OMMetrics.create();
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
when(ozoneManager.getMetrics()).thenReturn(omMetrics);
|
||||
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
|
||||
}
|
||||
|
||||
@After
|
||||
public void stop() {
|
||||
omMetrics.unRegister();
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testPreExecute() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
doPreExecute(volumeName, bucketName);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCache() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
OMBucketCreateRequest omBucketCreateRequest = doPreExecute(volumeName,
|
||||
bucketName);
|
||||
|
||||
doValidateAndUpdateCache(volumeName, bucketName,
|
||||
omBucketCreateRequest.getOmRequest());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCacheWithNoVolume() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
OMRequest originalRequest = createBucketRequest(bucketName, volumeName,
|
||||
false, StorageTypeProto.SSD);
|
||||
|
||||
OMBucketCreateRequest omBucketCreateRequest =
|
||||
new OMBucketCreateRequest(originalRequest);
|
||||
|
||||
String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
|
||||
|
||||
// As we have not still called validateAndUpdateCache, get() should
|
||||
// return null.
|
||||
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(bucketKey));
|
||||
|
||||
OMClientResponse omClientResponse =
|
||||
omBucketCreateRequest.validateAndUpdateCache(ozoneManager, 1);
|
||||
|
||||
OMResponse omResponse = omClientResponse.getOMResponse();
|
||||
Assert.assertNotNull(omResponse.getCreateBucketResponse());
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Status.VOLUME_NOT_FOUND,
|
||||
omResponse.getStatus());
|
||||
|
||||
// As request is invalid bucket table should not have entry.
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(bucketKey));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCacheWithBucketAlreadyExists()
|
||||
throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
OMBucketCreateRequest omBucketCreateRequest =
|
||||
doPreExecute(volumeName, bucketName);
|
||||
|
||||
doValidateAndUpdateCache(volumeName, bucketName,
|
||||
omBucketCreateRequest.getOmRequest());
|
||||
|
||||
// Try create same bucket again
|
||||
OMClientResponse omClientResponse =
|
||||
omBucketCreateRequest.validateAndUpdateCache(ozoneManager, 2);
|
||||
|
||||
OMResponse omResponse = omClientResponse.getOMResponse();
|
||||
Assert.assertNotNull(omResponse.getCreateBucketResponse());
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_ALREADY_EXISTS,
|
||||
omResponse.getStatus());
|
||||
}
|
||||
|
||||
|
||||
private OMBucketCreateRequest doPreExecute(String volumeName,
|
||||
String bucketName) throws Exception {
|
||||
addCreateVolumeToTable(volumeName, omMetadataManager);
|
||||
OMRequest originalRequest = createBucketRequest(bucketName, volumeName,
|
||||
false, StorageTypeProto.SSD);
|
||||
|
||||
OMBucketCreateRequest omBucketCreateRequest =
|
||||
new OMBucketCreateRequest(originalRequest);
|
||||
|
||||
OMRequest modifiedRequest = omBucketCreateRequest.preExecute(ozoneManager);
|
||||
verifyRequest(modifiedRequest, originalRequest);
|
||||
return new OMBucketCreateRequest(modifiedRequest);
|
||||
}
|
||||
|
||||
private void doValidateAndUpdateCache(String volumeName, String bucketName,
|
||||
OMRequest modifiedRequest) throws Exception {
|
||||
String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
|
||||
|
||||
// As we have not still called validateAndUpdateCache, get() should
|
||||
// return null.
|
||||
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(bucketKey));
|
||||
OMBucketCreateRequest omBucketCreateRequest =
|
||||
new OMBucketCreateRequest(modifiedRequest);
|
||||
|
||||
|
||||
OMClientResponse omClientResponse =
|
||||
omBucketCreateRequest.validateAndUpdateCache(ozoneManager, 1);
|
||||
|
||||
// As now after validateAndUpdateCache it should add entry to cache, get
|
||||
// should return non null value.
|
||||
OmBucketInfo omBucketInfo =
|
||||
omMetadataManager.getBucketTable().get(bucketKey);
|
||||
Assert.assertNotNull(omMetadataManager.getBucketTable().get(bucketKey));
|
||||
|
||||
// verify table data with actual request data.
|
||||
Assert.assertEquals(OmBucketInfo.getFromProtobuf(
|
||||
modifiedRequest.getCreateBucketRequest().getBucketInfo()),
|
||||
omBucketInfo);
|
||||
|
||||
// verify OMResponse.
|
||||
verifySuccessCreateBucketResponse(omClientResponse.getOMResponse());
|
||||
|
||||
}
|
||||
|
||||
|
||||
private void verifyRequest(OMRequest modifiedOmRequest,
|
||||
OMRequest originalRequest) {
|
||||
OzoneManagerProtocolProtos.BucketInfo original =
|
||||
originalRequest.getCreateBucketRequest().getBucketInfo();
|
||||
OzoneManagerProtocolProtos.BucketInfo updated =
|
||||
modifiedOmRequest.getCreateBucketRequest().getBucketInfo();
|
||||
|
||||
Assert.assertEquals(original.getBucketName(), updated.getBucketName());
|
||||
Assert.assertEquals(original.getVolumeName(), updated.getVolumeName());
|
||||
Assert.assertEquals(original.getIsVersionEnabled(),
|
||||
updated.getIsVersionEnabled());
|
||||
Assert.assertEquals(original.getStorageType(), updated.getStorageType());
|
||||
Assert.assertEquals(original.getMetadataList(), updated.getMetadataList());
|
||||
Assert.assertNotEquals(original.getCreationTime(),
|
||||
updated.getCreationTime());
|
||||
}
|
||||
|
||||
public static void verifySuccessCreateBucketResponse(OMResponse omResponse) {
|
||||
Assert.assertNotNull(omResponse.getCreateBucketResponse());
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Type.CreateBucket,
|
||||
omResponse.getCmdType());
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
|
||||
omResponse.getStatus());
|
||||
}
|
||||
|
||||
public static void addCreateVolumeToTable(String volumeName,
|
||||
OMMetadataManager omMetadataManager) throws Exception {
|
||||
OmVolumeArgs omVolumeArgs =
|
||||
OmVolumeArgs.newBuilder().setCreationTime(Time.now())
|
||||
.setVolume(volumeName).setAdminName(UUID.randomUUID().toString())
|
||||
.setOwnerName(UUID.randomUUID().toString()).build();
|
||||
omMetadataManager.getVolumeTable().put(
|
||||
omMetadataManager.getVolumeKey(volumeName), omVolumeArgs);
|
||||
}
|
||||
|
||||
|
||||
public static OMRequest createBucketRequest(String bucketName,
|
||||
String volumeName, boolean isVersionEnabled,
|
||||
StorageTypeProto storageTypeProto) {
|
||||
OzoneManagerProtocolProtos.BucketInfo bucketInfo =
|
||||
OzoneManagerProtocolProtos.BucketInfo.newBuilder()
|
||||
.setBucketName(bucketName)
|
||||
.setVolumeName(volumeName)
|
||||
.setIsVersionEnabled(isVersionEnabled)
|
||||
.setStorageType(storageTypeProto)
|
||||
.addAllMetadata(getMetadataList()).build();
|
||||
OzoneManagerProtocolProtos.CreateBucketRequest.Builder req =
|
||||
OzoneManagerProtocolProtos.CreateBucketRequest.newBuilder();
|
||||
req.setBucketInfo(bucketInfo);
|
||||
return OMRequest.newBuilder().setCreateBucketRequest(req)
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateBucket)
|
||||
.setClientId(UUID.randomUUID().toString()).build();
|
||||
}
|
||||
|
||||
public static List< HddsProtos.KeyValue> getMetadataList() {
|
||||
List<HddsProtos.KeyValue> metadataList = new ArrayList<>();
|
||||
metadataList.add(HddsProtos.KeyValue.newBuilder().setKey("key1").setValue(
|
||||
"value1").build());
|
||||
metadataList.add(HddsProtos.KeyValue.newBuilder().setKey("key2").setValue(
|
||||
"value2").build());
|
||||
return metadataList;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,152 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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.bucket;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OMMetrics;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.DeleteBucketRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMRequest;
|
||||
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Tests OMBucketDeleteRequest class which handles DeleteBucket request.
|
||||
*/
|
||||
public class TestOMBucketDeleteRequest {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OzoneManager ozoneManager;
|
||||
private OMMetrics omMetrics;
|
||||
private OMMetadataManager omMetadataManager;
|
||||
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
ozoneManager = Mockito.mock(OzoneManager.class);
|
||||
omMetrics = OMMetrics.create();
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
when(ozoneManager.getMetrics()).thenReturn(omMetrics);
|
||||
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
|
||||
}
|
||||
|
||||
@After
|
||||
public void stop() {
|
||||
omMetrics.unRegister();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreExecute() throws Exception {
|
||||
OMRequest omRequest =
|
||||
createDeleteBucketRequest(UUID.randomUUID().toString(),
|
||||
UUID.randomUUID().toString());
|
||||
|
||||
OMBucketDeleteRequest omBucketDeleteRequest =
|
||||
new OMBucketDeleteRequest(omRequest);
|
||||
|
||||
// As preExecute of DeleteBucket request is do nothing, requests should
|
||||
// be same.
|
||||
Assert.assertEquals(omRequest,
|
||||
omBucketDeleteRequest.preExecute(ozoneManager));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCache() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
OMRequest omRequest =
|
||||
createDeleteBucketRequest(volumeName, bucketName);
|
||||
|
||||
OMBucketDeleteRequest omBucketDeleteRequest =
|
||||
new OMBucketDeleteRequest(omRequest);
|
||||
|
||||
// Create Volume and bucket entries in DB.
|
||||
TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
|
||||
omMetadataManager);
|
||||
|
||||
omBucketDeleteRequest.validateAndUpdateCache(ozoneManager, 1);
|
||||
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName)));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCacheFailure() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
OMRequest omRequest =
|
||||
createDeleteBucketRequest(volumeName, bucketName);
|
||||
|
||||
OMBucketDeleteRequest omBucketDeleteRequest =
|
||||
new OMBucketDeleteRequest(omRequest);
|
||||
|
||||
|
||||
OMClientResponse omClientResponse =
|
||||
omBucketDeleteRequest.validateAndUpdateCache(ozoneManager, 1);
|
||||
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName)));
|
||||
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND,
|
||||
omClientResponse.getOMResponse().getStatus());
|
||||
|
||||
TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
|
||||
omMetadataManager);
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
private OMRequest createDeleteBucketRequest(String volumeName,
|
||||
String bucketName) {
|
||||
return OMRequest.newBuilder().setDeleteBucketRequest(
|
||||
DeleteBucketRequest.newBuilder()
|
||||
.setBucketName(bucketName).setVolumeName(volumeName))
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.DeleteBucket)
|
||||
.setClientId(UUID.randomUUID().toString()).build();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,162 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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.bucket;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OMMetrics;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
|
||||
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.
|
||||
BucketArgs;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMRequest;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.SetBucketPropertyRequest;
|
||||
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Tests OMBucketSetPropertyRequest class which handles OMSetBucketProperty
|
||||
* request.
|
||||
*/
|
||||
public class TestOMBucketSetPropertyRequest {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OzoneManager ozoneManager;
|
||||
private OMMetrics omMetrics;
|
||||
private OMMetadataManager omMetadataManager;
|
||||
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
ozoneManager = Mockito.mock(OzoneManager.class);
|
||||
omMetrics = OMMetrics.create();
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
when(ozoneManager.getMetrics()).thenReturn(omMetrics);
|
||||
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
|
||||
}
|
||||
|
||||
@After
|
||||
public void stop() {
|
||||
omMetrics.unRegister();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreExecute() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
|
||||
OMRequest omRequest = createSetBucketPropertyRequest(volumeName,
|
||||
bucketName, true);
|
||||
|
||||
OMBucketSetPropertyRequest omBucketSetPropertyRequest =
|
||||
new OMBucketSetPropertyRequest(omRequest);
|
||||
|
||||
Assert.assertEquals(omRequest,
|
||||
omBucketSetPropertyRequest.preExecute(ozoneManager));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCache() throws Exception {
|
||||
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
|
||||
OMRequest omRequest = createSetBucketPropertyRequest(volumeName,
|
||||
bucketName, true);
|
||||
|
||||
// Create with default BucketInfo values
|
||||
TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
|
||||
omMetadataManager);
|
||||
|
||||
OMBucketSetPropertyRequest omBucketSetPropertyRequest =
|
||||
new OMBucketSetPropertyRequest(omRequest);
|
||||
|
||||
OMClientResponse omClientResponse =
|
||||
omBucketSetPropertyRequest.validateAndUpdateCache(ozoneManager, 1);
|
||||
|
||||
Assert.assertEquals(true,
|
||||
omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName))
|
||||
.getIsVersionEnabled());
|
||||
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
|
||||
omClientResponse.getOMResponse().getStatus());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidateAndUpdateCacheFails() throws Exception {
|
||||
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
|
||||
OMRequest omRequest = createSetBucketPropertyRequest(volumeName,
|
||||
bucketName, true);
|
||||
|
||||
|
||||
OMBucketSetPropertyRequest omBucketSetPropertyRequest =
|
||||
new OMBucketSetPropertyRequest(omRequest);
|
||||
|
||||
OMClientResponse omClientResponse =
|
||||
omBucketSetPropertyRequest.validateAndUpdateCache(ozoneManager, 1);
|
||||
|
||||
Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND,
|
||||
omClientResponse.getOMResponse().getStatus());
|
||||
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName)));
|
||||
|
||||
}
|
||||
|
||||
private OMRequest createSetBucketPropertyRequest(String volumeName,
|
||||
String bucketName, boolean isVersionEnabled) {
|
||||
return OMRequest.newBuilder().setSetBucketPropertyRequest(
|
||||
SetBucketPropertyRequest.newBuilder().setBucketArgs(
|
||||
BucketArgs.newBuilder().setBucketName(bucketName)
|
||||
.setVolumeName(volumeName)
|
||||
.setIsVersionEnabled(isVersionEnabled).build()))
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.SetBucketProperty)
|
||||
.setClientId(UUID.randomUUID().toString()).build();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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 contains test classes for bucket requests.
|
||||
*/
|
||||
package org.apache.hadoop.ozone.om.request.bucket;
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
/**
|
||||
* Helper class to test OMClientResponse classes.
|
||||
*/
|
||||
public final class TestOMResponseUtils {
|
||||
|
||||
// No one can instantiate, this is just utility class with all static methods.
|
||||
private TestOMResponseUtils() {
|
||||
}
|
||||
|
||||
public static OmBucketInfo createBucket(String volume, String bucket) {
|
||||
return OmBucketInfo.newBuilder().setVolumeName(volume).setBucketName(bucket)
|
||||
.setCreationTime(Time.now()).setIsVersionEnabled(true).addMetadata(
|
||||
"key1", "value1").build();
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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.bucket;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.TestOMResponseUtils;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* This class tests OMBucketCreateResponse.
|
||||
*/
|
||||
public class TestOMBucketCreateResponse {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OMMetadataManager omMetadataManager;
|
||||
private BatchOperation batchOperation;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
batchOperation = omMetadataManager.getStore().initBatchOperation();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddToDBBatch() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
OmBucketInfo omBucketInfo = TestOMResponseUtils.createBucket(
|
||||
volumeName, bucketName);
|
||||
OMBucketCreateResponse omBucketCreateResponse =
|
||||
new OMBucketCreateResponse(omBucketInfo, OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCreateBucketResponse(
|
||||
CreateBucketResponse.newBuilder().build()).build());
|
||||
|
||||
omBucketCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
|
||||
|
||||
// Do manual commit and see whether addToBatch is successful or not.
|
||||
omMetadataManager.getStore().commitBatchOperation(batchOperation);
|
||||
|
||||
Assert.assertEquals(omBucketInfo,
|
||||
omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName)));
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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.bucket;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.TestOMResponseUtils;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.DeleteBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* This class tests OMBucketDeleteResponse.
|
||||
*/
|
||||
public class TestOMBucketDeleteResponse {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OMMetadataManager omMetadataManager;
|
||||
private BatchOperation batchOperation;
|
||||
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
batchOperation = omMetadataManager.getStore().initBatchOperation();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddToDBBatch() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
OmBucketInfo omBucketInfo = TestOMResponseUtils.createBucket(
|
||||
volumeName, bucketName);
|
||||
OMBucketCreateResponse omBucketCreateResponse =
|
||||
new OMBucketCreateResponse(omBucketInfo, OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCreateBucketResponse(
|
||||
CreateBucketResponse.newBuilder().build()).build());
|
||||
|
||||
OMBucketDeleteResponse omBucketDeleteResponse =
|
||||
new OMBucketDeleteResponse(volumeName, bucketName,
|
||||
OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.DeleteBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setDeleteBucketResponse(
|
||||
DeleteBucketResponse.getDefaultInstance()).build());
|
||||
|
||||
omBucketCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
|
||||
omBucketDeleteResponse.addToDBBatch(omMetadataManager, batchOperation);
|
||||
|
||||
// Do manual commit and see whether addToBatch is successful or not.
|
||||
omMetadataManager.getStore().commitBatchOperation(batchOperation);
|
||||
|
||||
Assert.assertNull(omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName)));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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.bucket;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.response.TestOMResponseUtils;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.CreateBucketResponse;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||
.OMResponse;
|
||||
import org.apache.hadoop.utils.db.BatchOperation;
|
||||
|
||||
/**
|
||||
* This class tests OMBucketSetPropertyResponse.
|
||||
*/
|
||||
public class TestOMBucketSetPropertyResponse {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
private OMMetadataManager omMetadataManager;
|
||||
private BatchOperation batchOperation;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
|
||||
ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
|
||||
folder.newFolder().getAbsolutePath());
|
||||
omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
|
||||
batchOperation = omMetadataManager.getStore().initBatchOperation();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddToDBBatch() throws Exception {
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
|
||||
OmBucketInfo omBucketInfo = TestOMResponseUtils.createBucket(
|
||||
volumeName, bucketName);
|
||||
OMBucketSetPropertyResponse omBucketCreateResponse =
|
||||
new OMBucketSetPropertyResponse(omBucketInfo, OMResponse.newBuilder()
|
||||
.setCmdType(OzoneManagerProtocolProtos.Type.CreateBucket)
|
||||
.setStatus(OzoneManagerProtocolProtos.Status.OK)
|
||||
.setCreateBucketResponse(
|
||||
CreateBucketResponse.newBuilder().build()).build());
|
||||
|
||||
omBucketCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
|
||||
|
||||
// Do manual commit and see whether addToBatch is successful or not.
|
||||
omMetadataManager.getStore().commitBatchOperation(batchOperation);
|
||||
|
||||
Assert.assertEquals(omBucketInfo,
|
||||
omMetadataManager.getBucketTable().get(
|
||||
omMetadataManager.getBucketKey(volumeName, bucketName)));
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* 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
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* 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 contains test classes for bucket responses.
|
||||
*/
|
||||
package org.apache.hadoop.ozone.om.response.bucket;
|
|
@ -0,0 +1,16 @@
|
|||
# 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
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# 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.
|
||||
mock-maker-inline
|
Loading…
Reference in New Issue