Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
import org.apache.hadoop.ozone.om.lock.OzoneManagerLock;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.VolumeList;
.UserVolumeInfo;
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.hadoop.hdds.utils.db.Table;
Expand Down Expand Up @@ -225,7 +225,7 @@ List<OmVolumeArgs> listVolumes(String userName, String prefix,
*
* @return UserTable.
*/
Table<String, VolumeList> getUserTable();
Table<String, UserVolumeInfo> getUserTable();

/**
* Returns the Volume Table.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,31 +18,32 @@
package org.apache.hadoop.ozone.om.codec;

import java.io.IOException;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
import org.apache.hadoop.ozone.protocol.proto
.OzoneManagerProtocolProtos.UserVolumeInfo;
import org.apache.hadoop.hdds.utils.db.Codec;

import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;

/**
* Codec to encode VolumeList as byte array.
* Codec to encode UserVolumeInfo as byte array.
*/
public class VolumeListCodec implements Codec<VolumeList> {
public class UserVolumeInfoCodec implements Codec<UserVolumeInfo> {

@Override
public byte[] toPersistedFormat(VolumeList object) throws IOException {
public byte[] toPersistedFormat(UserVolumeInfo object) throws IOException {
Preconditions
.checkNotNull(object, "Null object can't be converted to byte array.");
return object.toByteArray();
}

@Override
public VolumeList fromPersistedFormat(byte[] rawData) throws IOException {
public UserVolumeInfo fromPersistedFormat(byte[] rawData) throws IOException {
Preconditions
.checkNotNull(rawData,
"Null byte array can't converted to real object.");
try {
return VolumeList.parseFrom(rawData);
return UserVolumeInfo.parseFrom(rawData);
} catch (InvalidProtocolBufferException e) {
throw new IllegalArgumentException(
"Can't encode the the raw data from the byte array", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -352,7 +352,7 @@ message CreateVolumeResponse {

}

message VolumeList {
message UserVolumeInfo {
repeated string volumeNames = 1;
optional uint64 objectID = 2;
optional uint64 updateID = 3;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@
import org.apache.hadoop.ozone.om.codec.RepeatedOmKeyInfoCodec;
import org.apache.hadoop.ozone.om.codec.S3SecretValueCodec;
import org.apache.hadoop.ozone.om.codec.TokenIdentifierCodec;
import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
import org.apache.hadoop.ozone.om.codec.UserVolumeInfoCodec;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
Expand All @@ -61,7 +61,8 @@
import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
import org.apache.hadoop.ozone.om.lock.OzoneManagerLock;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
import org.apache.hadoop.ozone.protocol.proto
.OzoneManagerProtocolProtos.UserVolumeInfo;
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;

import com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -92,7 +93,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
* |----------------------------------------------------------------------|
* | Column Family | VALUE |
* |----------------------------------------------------------------------|
* | userTable | user->VolumeList |
* | userTable | /user->UserVolumeInfo |
* |----------------------------------------------------------------------|
* | volumeTable | /volume->VolumeInfo |
* |----------------------------------------------------------------------|
Expand Down Expand Up @@ -170,7 +171,7 @@ protected OmMetadataManagerImpl() {
}

@Override
public Table<String, VolumeList> getUserTable() {
public Table<String, UserVolumeInfo> getUserTable() {
return userTable;
}

Expand Down Expand Up @@ -266,7 +267,7 @@ protected DBStoreBuilder addOMTablesAndCodecs(DBStoreBuilder builder) {
.addCodec(RepeatedOmKeyInfo.class, new RepeatedOmKeyInfoCodec())
.addCodec(OmBucketInfo.class, new OmBucketInfoCodec())
.addCodec(OmVolumeArgs.class, new OmVolumeArgsCodec())
.addCodec(VolumeList.class, new VolumeListCodec())
.addCodec(UserVolumeInfo.class, new UserVolumeInfoCodec())
.addCodec(OmMultipartKeyInfo.class, new OmMultipartKeyInfoCodec())
.addCodec(S3SecretValue.class, new S3SecretValueCodec())
.addCodec(OmPrefixInfo.class, new OmPrefixInfoCodec());
Expand All @@ -279,7 +280,7 @@ protected DBStoreBuilder addOMTablesAndCodecs(DBStoreBuilder builder) {
*/
protected void initializeOmTables() throws IOException {
userTable =
this.store.getTable(USER_TABLE, String.class, VolumeList.class);
this.store.getTable(USER_TABLE, String.class, UserVolumeInfo.class);
checkTableStatus(userTable, USER_TABLE);

TableCacheImpl.CacheCleanupPolicy cleanupPolicy =
Expand Down Expand Up @@ -706,7 +707,7 @@ public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
public List<OmVolumeArgs> listVolumes(String userName,
String prefix, String startKey, int maxKeys) throws IOException {
List<OmVolumeArgs> result = Lists.newArrayList();
VolumeList volumes;
UserVolumeInfo volumes;
if (StringUtil.isBlank(userName)) {
throw new OMException("User name is required to list Volumes.",
ResultCodes.USER_NOT_FOUND);
Expand Down Expand Up @@ -747,15 +748,15 @@ public List<OmVolumeArgs> listVolumes(String userName,
return result;
}

private VolumeList getVolumesByUser(String userNameKey)
private UserVolumeInfo getVolumesByUser(String userNameKey)
throws OMException {
try {
VolumeList volumeList = getUserTable().get(userNameKey);
if (volumeList == null) {
UserVolumeInfo userVolInfo = getUserTable().get(userNameKey);
if (userVolInfo == null) {
// No volume found for this user, return an empty list
return VolumeList.newBuilder().build();
return UserVolumeInfo.newBuilder().build();
} else {
return volumeList;
return userVolInfo;
}
} catch (IOException e) {
throw new OMException("Unable to get volumes info by the given user, "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,10 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
import org.apache.hadoop.ozone.protocol.proto
.OzoneManagerProtocolProtos.OzoneAclInfo;
import org.apache.hadoop.ozone.protocol.proto
.OzoneManagerProtocolProtos.UserVolumeInfo;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.ozone.security.acl.RequestContext;
Expand Down Expand Up @@ -73,11 +75,11 @@ public VolumeManagerImpl(OMMetadataManager metadataManager,
}

// Helpers to add and delete volume from user list
private VolumeList addVolumeToOwnerList(String volume, String owner)
private UserVolumeInfo addVolumeToOwnerList(String volume, String owner)
throws IOException {
// Get the volume list
String dbUserKey = metadataManager.getUserKey(owner);
VolumeList volumeList = metadataManager.getUserTable().get(dbUserKey);
UserVolumeInfo volumeList = metadataManager.getUserTable().get(dbUserKey);
List<String> prevVolList = new ArrayList<>();
if (volumeList != null) {
prevVolList.addAll(volumeList.getVolumeNamesList());
Expand All @@ -92,16 +94,16 @@ private VolumeList addVolumeToOwnerList(String volume, String owner)

// Add the new volume to the list
prevVolList.add(volume);
VolumeList newVolList = VolumeList.newBuilder()
UserVolumeInfo newVolList = UserVolumeInfo.newBuilder()
.addAllVolumeNames(prevVolList).build();

return newVolList;
}

private VolumeList delVolumeFromOwnerList(String volume, String owner)
private UserVolumeInfo delVolumeFromOwnerList(String volume, String owner)
throws IOException {
// Get the volume list
VolumeList volumeList = metadataManager.getUserTable().get(owner);
UserVolumeInfo volumeList = metadataManager.getUserTable().get(owner);
List<String> prevVolList = new ArrayList<>();
if (volumeList != null) {
prevVolList.addAll(volumeList.getVolumeNamesList());
Expand All @@ -112,7 +114,7 @@ private VolumeList delVolumeFromOwnerList(String volume, String owner)

// Remove the volume from the list
prevVolList.remove(volume);
VolumeList newVolList = VolumeList.newBuilder()
UserVolumeInfo newVolList = UserVolumeInfo.newBuilder()
.addAllVolumeNames(prevVolList).build();
return newVolList;
}
Expand Down Expand Up @@ -144,7 +146,7 @@ public void createVolume(OmVolumeArgs omVolumeArgs) throws IOException {
throw new OMException(ResultCodes.VOLUME_ALREADY_EXISTS);
}

VolumeList volumeList = addVolumeToOwnerList(omVolumeArgs.getVolume(),
UserVolumeInfo volumeList = addVolumeToOwnerList(omVolumeArgs.getVolume(),
omVolumeArgs.getOwnerName());

// Set creation time
Expand Down Expand Up @@ -173,7 +175,7 @@ public void createVolume(OmVolumeArgs omVolumeArgs) throws IOException {
}

private void createVolumeCommitToDB(OmVolumeArgs omVolumeArgs,
VolumeList volumeList, String dbVolumeKey, String dbUserKey)
UserVolumeInfo volumeList, String dbVolumeKey, String dbUserKey)
throws IOException {
try (BatchOperation batch = metadataManager.getStore()
.initBatchOperation()) {
Expand Down Expand Up @@ -222,11 +224,12 @@ public void setOwner(String volume, String owner)

acquiredUsersLock = metadataManager.getLock().acquireMultiUserLock(owner,
originalOwner);
VolumeList oldOwnerVolumeList = delVolumeFromOwnerList(volume,
UserVolumeInfo oldOwnerVolumeList = delVolumeFromOwnerList(volume,
originalOwner);

String newOwner = metadataManager.getUserKey(owner);
VolumeList newOwnerVolumeList = addVolumeToOwnerList(volume, newOwner);
UserVolumeInfo newOwnerVolumeList = addVolumeToOwnerList(volume,
newOwner);

volumeArgs.setOwnerName(owner);
setOwnerCommitToDB(oldOwnerVolumeList, newOwnerVolumeList,
Expand All @@ -246,8 +249,8 @@ public void setOwner(String volume, String owner)
}


private void setOwnerCommitToDB(VolumeList oldOwnerVolumeList,
VolumeList newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs,
private void setOwnerCommitToDB(UserVolumeInfo oldOwnerVolumeList,
UserVolumeInfo newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs,
String oldOwner) throws IOException {
try (BatchOperation batch = metadataManager.getStore()
.initBatchOperation()) {
Expand Down Expand Up @@ -370,7 +373,7 @@ public void deleteVolume(String volume) throws IOException {
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
// delete the volume from the owner list
// as well as delete the volume entry
VolumeList newVolumeList = delVolumeFromOwnerList(volume,
UserVolumeInfo newVolumeList = delVolumeFromOwnerList(volume,
volumeArgs.getOwnerName());


Expand All @@ -390,7 +393,7 @@ public void deleteVolume(String volume) throws IOException {
}


private void deleteVolumeCommitToDB(VolumeList newVolumeList,
private void deleteVolumeCommitToDB(UserVolumeInfo newVolumeList,
String volume, String owner) throws IOException {
try (BatchOperation batch = metadataManager.getStore()
.initBatchOperation()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,7 @@
.S3CreateBucketResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.S3CreateVolumeInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.VolumeList;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserVolumeInfo;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
Expand Down Expand Up @@ -176,7 +175,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OmVolumeArgs omVolumeArgs = createOmVolumeArgs(volumeName, userName,
s3CreateBucketRequest.getS3CreateVolumeInfo()
.getCreationTime());
VolumeList volumeList = omMetadataManager.getUserTable().get(
UserVolumeInfo volumeList = omMetadataManager.getUserTable().get(
omMetadataManager.getUserKey(userName));
volumeList = addVolumeToOwnerList(volumeList,
volumeName, userName, ozoneManager.getMaxUserVolumeCount(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,7 @@
.OMResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.VolumeInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.VolumeList;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserVolumeInfo;
import org.apache.hadoop.util.Time;

import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
Expand Down Expand Up @@ -133,7 +132,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
}
}

VolumeList volumeList = null;
UserVolumeInfo volumeList = null;

// acquire lock.
acquiredVolumeLock = omMetadataManager.getLock().acquireLock(VOLUME_LOCK,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
}

OmVolumeArgs omVolumeArgs = null;
OzoneManagerProtocolProtos.VolumeList newVolumeList = null;
OzoneManagerProtocolProtos.UserVolumeInfo newVolumeList = null;

acquiredVolumeLock = omMetadataManager.getLock().acquireLock(VOLUME_LOCK,
volume);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.VolumeList;
.UserVolumeInfo;
import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
import org.apache.hadoop.hdds.utils.db.cache.CacheValue;

Expand All @@ -50,10 +50,10 @@ public OMVolumeRequest(OMRequest omRequest) {
* @param volume - volume which needs to deleted from the volume list.
* @param owner - Name of the Owner.
* @param txID - The transaction ID that is updating this value.
* @return VolumeList - updated volume list for the user.
* @return UserVolumeInfo - updated UserVolumeInfo.
* @throws IOException
*/
protected VolumeList delVolumeFromOwnerList(VolumeList volumeList,
protected UserVolumeInfo delVolumeFromOwnerList(UserVolumeInfo volumeList,
String volume, String owner, long txID) throws IOException {

List<String> prevVolList = new ArrayList<>();
Expand All @@ -68,7 +68,7 @@ protected VolumeList delVolumeFromOwnerList(VolumeList volumeList,

// Remove the volume from the list
prevVolList.remove(volume);
VolumeList newVolList = VolumeList.newBuilder()
UserVolumeInfo newVolList = UserVolumeInfo.newBuilder()
.addAllVolumeNames(prevVolList)
.setObjectID(volumeList.getObjectID())
.setUpdateID(txID)
Expand All @@ -88,7 +88,7 @@ protected VolumeList delVolumeFromOwnerList(VolumeList volumeList,
* @throws OMException - if user has volumes greater than
* maxUserVolumeCount, an exception is thrown.
*/
protected VolumeList addVolumeToOwnerList(VolumeList volumeList,
protected UserVolumeInfo addVolumeToOwnerList(UserVolumeInfo volumeList,
String volume, String owner, long maxUserVolumeCount, long txID)
throws IOException {

Expand All @@ -109,7 +109,7 @@ protected VolumeList addVolumeToOwnerList(VolumeList volumeList,

// Add the new volume to the list
prevVolList.add(volume);
VolumeList newVolList = VolumeList.newBuilder()
UserVolumeInfo newVolList = UserVolumeInfo.newBuilder()
.setObjectID(objectID)
.setUpdateID(txID)
.addAllVolumeNames(prevVolList).build();
Expand All @@ -129,7 +129,7 @@ protected VolumeList addVolumeToOwnerList(VolumeList volumeList,
* @throws IOException
*/
protected void createVolume(final OMMetadataManager omMetadataManager,
OmVolumeArgs omVolumeArgs, VolumeList volumeList, String dbVolumeKey,
OmVolumeArgs omVolumeArgs, UserVolumeInfo volumeList, String dbVolumeKey,
String dbUserKey, long transactionLogIndex) {
// Update cache: Update user and volume cache.
omMetadataManager.getUserTable().addCacheEntry(new CacheKey<>(dbUserKey),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,8 +117,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,

String dbVolumeKey = omMetadataManager.getVolumeKey(volume);

OzoneManagerProtocolProtos.VolumeList oldOwnerVolumeList = null;
OzoneManagerProtocolProtos.VolumeList newOwnerVolumeList = null;
OzoneManagerProtocolProtos.UserVolumeInfo oldOwnerVolumeList = null;
OzoneManagerProtocolProtos.UserVolumeInfo newOwnerVolumeList = null;
OmVolumeArgs omVolumeArgs = null;


Expand Down
Loading