|
@@ -22,17 +22,18 @@ import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmDeleteVolumeResponse;
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmVolumeOwnerChangeResponse;
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
|
|
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.VolumeList;
|
|
-import org.apache.hadoop.utils.RocksDBStore;
|
|
|
|
import org.apache.hadoop.utils.db.BatchOperation;
|
|
import org.apache.hadoop.utils.db.BatchOperation;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME_DEFAULT;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME_DEFAULT;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
|
-import org.rocksdb.RocksDBException;
|
|
|
|
|
|
+
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
@@ -45,6 +46,7 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
|
|
|
|
private final OMMetadataManager metadataManager;
|
|
private final OMMetadataManager metadataManager;
|
|
private final int maxUserVolumeCount;
|
|
private final int maxUserVolumeCount;
|
|
|
|
+ private final boolean isRatisEnabled;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Constructor.
|
|
* Constructor.
|
|
@@ -52,15 +54,18 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
public VolumeManagerImpl(OMMetadataManager metadataManager,
|
|
public VolumeManagerImpl(OMMetadataManager metadataManager,
|
|
- OzoneConfiguration conf) throws IOException {
|
|
|
|
|
|
+ OzoneConfiguration conf) {
|
|
this.metadataManager = metadataManager;
|
|
this.metadataManager = metadataManager;
|
|
this.maxUserVolumeCount = conf.getInt(OZONE_OM_USER_MAX_VOLUME,
|
|
this.maxUserVolumeCount = conf.getInt(OZONE_OM_USER_MAX_VOLUME,
|
|
OZONE_OM_USER_MAX_VOLUME_DEFAULT);
|
|
OZONE_OM_USER_MAX_VOLUME_DEFAULT);
|
|
|
|
+ isRatisEnabled = conf.getBoolean(
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
}
|
|
}
|
|
|
|
|
|
// Helpers to add and delete volume from user list
|
|
// Helpers to add and delete volume from user list
|
|
- private void addVolumeToOwnerList(String volume, String owner,
|
|
|
|
- BatchOperation batchOperation) throws IOException {
|
|
|
|
|
|
+ private VolumeList addVolumeToOwnerList(String volume, String owner)
|
|
|
|
+ throws IOException {
|
|
// Get the volume list
|
|
// Get the volume list
|
|
String dbUserKey = metadataManager.getUserKey(owner);
|
|
String dbUserKey = metadataManager.getUserKey(owner);
|
|
VolumeList volumeList = metadataManager.getUserTable().get(dbUserKey);
|
|
VolumeList volumeList = metadataManager.getUserTable().get(dbUserKey);
|
|
@@ -72,22 +77,22 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
// Check the volume count
|
|
// Check the volume count
|
|
if (prevVolList.size() >= maxUserVolumeCount) {
|
|
if (prevVolList.size() >= maxUserVolumeCount) {
|
|
LOG.debug("Too many volumes for user:{}", owner);
|
|
LOG.debug("Too many volumes for user:{}", owner);
|
|
- throw new OMException(ResultCodes.USER_TOO_MANY_VOLUMES);
|
|
|
|
|
|
+ throw new OMException("Too many volumes for user:" + owner,
|
|
|
|
+ ResultCodes.USER_TOO_MANY_VOLUMES);
|
|
}
|
|
}
|
|
|
|
|
|
// Add the new volume to the list
|
|
// Add the new volume to the list
|
|
prevVolList.add(volume);
|
|
prevVolList.add(volume);
|
|
VolumeList newVolList = VolumeList.newBuilder()
|
|
VolumeList newVolList = VolumeList.newBuilder()
|
|
.addAllVolumeNames(prevVolList).build();
|
|
.addAllVolumeNames(prevVolList).build();
|
|
- metadataManager.getUserTable().putWithBatch(batchOperation,
|
|
|
|
- dbUserKey, newVolList);
|
|
|
|
|
|
+
|
|
|
|
+ return newVolList;
|
|
}
|
|
}
|
|
|
|
|
|
- private void delVolumeFromOwnerList(String volume, String owner,
|
|
|
|
- BatchOperation batch) throws RocksDBException, IOException {
|
|
|
|
|
|
+ private VolumeList delVolumeFromOwnerList(String volume, String owner)
|
|
|
|
+ throws IOException {
|
|
// Get the volume list
|
|
// Get the volume list
|
|
- String dbUserKey = metadataManager.getUserKey(owner);
|
|
|
|
- VolumeList volumeList = metadataManager.getUserTable().get(dbUserKey);
|
|
|
|
|
|
+ VolumeList volumeList = metadataManager.getUserTable().get(owner);
|
|
List<String> prevVolList = new ArrayList<>();
|
|
List<String> prevVolList = new ArrayList<>();
|
|
if (volumeList != null) {
|
|
if (volumeList != null) {
|
|
prevVolList.addAll(volumeList.getVolumeNamesList());
|
|
prevVolList.addAll(volumeList.getVolumeNamesList());
|
|
@@ -98,58 +103,90 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
|
|
|
|
// Remove the volume from the list
|
|
// Remove the volume from the list
|
|
prevVolList.remove(volume);
|
|
prevVolList.remove(volume);
|
|
- if (prevVolList.size() == 0) {
|
|
|
|
- metadataManager.getUserTable().deleteWithBatch(batch, dbUserKey);
|
|
|
|
- } else {
|
|
|
|
- VolumeList newVolList = VolumeList.newBuilder()
|
|
|
|
- .addAllVolumeNames(prevVolList).build();
|
|
|
|
- metadataManager.getUserTable().putWithBatch(batch,
|
|
|
|
- dbUserKey, newVolList);
|
|
|
|
- }
|
|
|
|
|
|
+ VolumeList newVolList = VolumeList.newBuilder()
|
|
|
|
+ .addAllVolumeNames(prevVolList).build();
|
|
|
|
+ return newVolList;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Creates a volume.
|
|
* Creates a volume.
|
|
- * @param args - OmVolumeArgs.
|
|
|
|
|
|
+ * @param omVolumeArgs - OmVolumeArgs.
|
|
|
|
+ * @return VolumeList
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- public void createVolume(OmVolumeArgs args) throws IOException {
|
|
|
|
- Preconditions.checkNotNull(args);
|
|
|
|
- metadataManager.getLock().acquireUserLock(args.getOwnerName());
|
|
|
|
- metadataManager.getLock().acquireVolumeLock(args.getVolume());
|
|
|
|
|
|
+ public VolumeList createVolume(OmVolumeArgs omVolumeArgs) throws IOException {
|
|
|
|
+ Preconditions.checkNotNull(omVolumeArgs);
|
|
|
|
+ metadataManager.getLock().acquireUserLock(omVolumeArgs.getOwnerName());
|
|
|
|
+ metadataManager.getLock().acquireVolumeLock(omVolumeArgs.getVolume());
|
|
try {
|
|
try {
|
|
- String dbVolumeKey = metadataManager.getVolumeKey(args.getVolume());
|
|
|
|
|
|
+ String dbVolumeKey = metadataManager.getVolumeKey(
|
|
|
|
+ omVolumeArgs.getVolume());
|
|
|
|
+ String dbUserKey = metadataManager.getUserKey(
|
|
|
|
+ omVolumeArgs.getOwnerName());
|
|
OmVolumeArgs volumeInfo =
|
|
OmVolumeArgs volumeInfo =
|
|
metadataManager.getVolumeTable().get(dbVolumeKey);
|
|
metadataManager.getVolumeTable().get(dbVolumeKey);
|
|
|
|
|
|
// Check of the volume already exists
|
|
// Check of the volume already exists
|
|
if (volumeInfo != null) {
|
|
if (volumeInfo != null) {
|
|
- LOG.debug("volume:{} already exists", args.getVolume());
|
|
|
|
|
|
+ LOG.debug("volume:{} already exists", omVolumeArgs.getVolume());
|
|
throw new OMException(ResultCodes.VOLUME_ALREADY_EXISTS);
|
|
throw new OMException(ResultCodes.VOLUME_ALREADY_EXISTS);
|
|
}
|
|
}
|
|
|
|
|
|
- try (BatchOperation batch = metadataManager.getStore()
|
|
|
|
- .initBatchOperation()) {
|
|
|
|
- // Write the vol info
|
|
|
|
- metadataManager.getVolumeTable().putWithBatch(batch,
|
|
|
|
- dbVolumeKey, args);
|
|
|
|
|
|
+ VolumeList volumeList = addVolumeToOwnerList(omVolumeArgs.getVolume(),
|
|
|
|
+ omVolumeArgs.getOwnerName());
|
|
|
|
|
|
- // Add volume to user list
|
|
|
|
- addVolumeToOwnerList(args.getVolume(), args.getOwnerName(), batch);
|
|
|
|
- metadataManager.getStore().commitBatchOperation(batch);
|
|
|
|
|
|
+ // Set creation time
|
|
|
|
+ omVolumeArgs.setCreationTime(System.currentTimeMillis());
|
|
|
|
+
|
|
|
|
+ if (!isRatisEnabled) {
|
|
|
|
+ createVolumeCommitToDB(omVolumeArgs, volumeList, dbVolumeKey,
|
|
|
|
+ dbUserKey);
|
|
}
|
|
}
|
|
- LOG.debug("created volume:{} user:{}", args.getVolume(),
|
|
|
|
- args.getOwnerName());
|
|
|
|
|
|
+ LOG.debug("created volume:{} user:{}", omVolumeArgs.getVolume(),
|
|
|
|
+ omVolumeArgs.getOwnerName());
|
|
|
|
+ return volumeList;
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
if (!(ex instanceof OMException)) {
|
|
if (!(ex instanceof OMException)) {
|
|
LOG.error("Volume creation failed for user:{} volume:{}",
|
|
LOG.error("Volume creation failed for user:{} volume:{}",
|
|
- args.getOwnerName(), args.getVolume(), ex);
|
|
|
|
- } else {
|
|
|
|
- throw (IOException) ex;
|
|
|
|
|
|
+ omVolumeArgs.getOwnerName(), omVolumeArgs.getVolume(), ex);
|
|
}
|
|
}
|
|
|
|
+ throw ex;
|
|
} finally {
|
|
} finally {
|
|
- metadataManager.getLock().releaseVolumeLock(args.getVolume());
|
|
|
|
- metadataManager.getLock().releaseUserLock(args.getOwnerName());
|
|
|
|
|
|
+ metadataManager.getLock().releaseVolumeLock(omVolumeArgs.getVolume());
|
|
|
|
+ metadataManager.getLock().releaseUserLock(omVolumeArgs.getOwnerName());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void applyCreateVolume(OmVolumeArgs omVolumeArgs,
|
|
|
|
+ VolumeList volumeList) throws IOException {
|
|
|
|
+ // Do we need to hold lock in apply Transactions requests?
|
|
|
|
+ String dbVolumeKey = metadataManager.getVolumeKey(omVolumeArgs.getVolume());
|
|
|
|
+ String dbUserKey = metadataManager.getUserKey(omVolumeArgs.getOwnerName());
|
|
|
|
+ try {
|
|
|
|
+ createVolumeCommitToDB(omVolumeArgs, volumeList, dbVolumeKey, dbUserKey);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ LOG.error("Volume creation failed for user:{} volume:{}",
|
|
|
|
+ omVolumeArgs.getOwnerName(), omVolumeArgs.getVolume(), ex);
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void createVolumeCommitToDB(OmVolumeArgs omVolumeArgs,
|
|
|
|
+ VolumeList volumeList, String dbVolumeKey, String dbUserKey)
|
|
|
|
+ throws IOException {
|
|
|
|
+ try (BatchOperation batch = metadataManager.getStore()
|
|
|
|
+ .initBatchOperation()) {
|
|
|
|
+ // Write the vol info
|
|
|
|
+ metadataManager.getVolumeTable().putWithBatch(batch, dbVolumeKey,
|
|
|
|
+ omVolumeArgs);
|
|
|
|
+ metadataManager.getUserTable().putWithBatch(batch, dbUserKey,
|
|
|
|
+ volumeList);
|
|
|
|
+ // Add volume to user list
|
|
|
|
+ metadataManager.getStore().commitBatchOperation(batch);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ throw ex;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -161,7 +198,8 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- public void setOwner(String volume, String owner) throws IOException {
|
|
|
|
|
|
+ public OmVolumeOwnerChangeResponse setOwner(String volume, String owner)
|
|
|
|
+ throws IOException {
|
|
Preconditions.checkNotNull(volume);
|
|
Preconditions.checkNotNull(volume);
|
|
Preconditions.checkNotNull(owner);
|
|
Preconditions.checkNotNull(owner);
|
|
metadataManager.getLock().acquireUserLock(owner);
|
|
metadataManager.getLock().acquireUserLock(owner);
|
|
@@ -179,49 +217,84 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
|
|
|
|
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
|
|
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
|
|
|
|
|
|
- try (BatchOperation batch = metadataManager.getStore()
|
|
|
|
- .initBatchOperation()) {
|
|
|
|
- delVolumeFromOwnerList(volume, volumeArgs.getOwnerName(), batch);
|
|
|
|
- addVolumeToOwnerList(volume, owner, batch);
|
|
|
|
-
|
|
|
|
- OmVolumeArgs newVolumeArgs =
|
|
|
|
- OmVolumeArgs.newBuilder().setVolume(volumeArgs.getVolume())
|
|
|
|
- .setAdminName(volumeArgs.getAdminName())
|
|
|
|
- .setOwnerName(owner)
|
|
|
|
- .setQuotaInBytes(volumeArgs.getQuotaInBytes())
|
|
|
|
- .setCreationTime(volumeArgs.getCreationTime())
|
|
|
|
- .build();
|
|
|
|
-
|
|
|
|
- metadataManager.getVolumeTable().putWithBatch(batch,
|
|
|
|
- dbVolumeKey, newVolumeArgs);
|
|
|
|
- metadataManager.getStore().commitBatchOperation(batch);
|
|
|
|
|
|
+ String originalOwner =
|
|
|
|
+ metadataManager.getUserKey(volumeArgs.getOwnerName());
|
|
|
|
+ VolumeList oldOwnerVolumeList = delVolumeFromOwnerList(volume,
|
|
|
|
+ originalOwner);
|
|
|
|
+
|
|
|
|
+ String newOwner = metadataManager.getUserKey(owner);
|
|
|
|
+ VolumeList newOwnerVolumeList = addVolumeToOwnerList(volume, newOwner);
|
|
|
|
+
|
|
|
|
+ volumeArgs.setOwnerName(owner);
|
|
|
|
+ if (!isRatisEnabled) {
|
|
|
|
+ setOwnerCommitToDB(oldOwnerVolumeList, newOwnerVolumeList,
|
|
|
|
+ volumeArgs, owner);
|
|
}
|
|
}
|
|
- } catch (RocksDBException | IOException ex) {
|
|
|
|
|
|
+ return new OmVolumeOwnerChangeResponse(oldOwnerVolumeList,
|
|
|
|
+ newOwnerVolumeList, volumeArgs, originalOwner);
|
|
|
|
+ } catch (IOException ex) {
|
|
if (!(ex instanceof OMException)) {
|
|
if (!(ex instanceof OMException)) {
|
|
LOG.error("Changing volume ownership failed for user:{} volume:{}",
|
|
LOG.error("Changing volume ownership failed for user:{} volume:{}",
|
|
owner, volume, ex);
|
|
owner, volume, ex);
|
|
}
|
|
}
|
|
- if(ex instanceof RocksDBException) {
|
|
|
|
- throw RocksDBStore.toIOException("Volume creation failed.",
|
|
|
|
- (RocksDBException) ex);
|
|
|
|
- } else {
|
|
|
|
- throw (IOException) ex;
|
|
|
|
- }
|
|
|
|
|
|
+ throw ex;
|
|
} finally {
|
|
} finally {
|
|
metadataManager.getLock().releaseVolumeLock(volume);
|
|
metadataManager.getLock().releaseVolumeLock(volume);
|
|
metadataManager.getLock().releaseUserLock(owner);
|
|
metadataManager.getLock().releaseUserLock(owner);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void applySetOwner(String oldOwner, VolumeList oldOwnerVolumeList,
|
|
|
|
+ VolumeList newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs)
|
|
|
|
+ throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ setOwnerCommitToDB(oldOwnerVolumeList, newOwnerVolumeList,
|
|
|
|
+ newOwnerVolumeArgs, oldOwner);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ LOG.error("Changing volume ownership failed for user:{} volume:{}",
|
|
|
|
+ newOwnerVolumeArgs.getOwnerName(), newOwnerVolumeArgs.getVolume(),
|
|
|
|
+ ex);
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ private void setOwnerCommitToDB(VolumeList oldOwnerVolumeList,
|
|
|
|
+ VolumeList newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs,
|
|
|
|
+ String oldOwner) throws IOException {
|
|
|
|
+ try (BatchOperation batch = metadataManager.getStore()
|
|
|
|
+ .initBatchOperation()) {
|
|
|
|
+ if (oldOwnerVolumeList.getVolumeNamesList().size() == 0) {
|
|
|
|
+ metadataManager.getUserTable().deleteWithBatch(batch, oldOwner);
|
|
|
|
+ } else {
|
|
|
|
+ metadataManager.getUserTable().putWithBatch(batch, oldOwner,
|
|
|
|
+ oldOwnerVolumeList);
|
|
|
|
+ }
|
|
|
|
+ metadataManager.getUserTable().putWithBatch(batch,
|
|
|
|
+ newOwnerVolumeArgs.getOwnerName(),
|
|
|
|
+ newOwnerVolumeList);
|
|
|
|
+
|
|
|
|
+ String dbVolumeKey =
|
|
|
|
+ metadataManager.getVolumeKey(newOwnerVolumeArgs.getVolume());
|
|
|
|
+ metadataManager.getVolumeTable().putWithBatch(batch,
|
|
|
|
+ dbVolumeKey, newOwnerVolumeArgs);
|
|
|
|
+ metadataManager.getStore().commitBatchOperation(batch);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Changes the Quota on a volume.
|
|
* Changes the Quota on a volume.
|
|
*
|
|
*
|
|
* @param volume - Name of the volume.
|
|
* @param volume - Name of the volume.
|
|
* @param quota - Quota in bytes.
|
|
* @param quota - Quota in bytes.
|
|
|
|
+ *
|
|
|
|
+ * @return OmVolumeArgs
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- public void setQuota(String volume, long quota) throws IOException {
|
|
|
|
|
|
+ public OmVolumeArgs setQuota(String volume, long quota) throws IOException {
|
|
Preconditions.checkNotNull(volume);
|
|
Preconditions.checkNotNull(volume);
|
|
metadataManager.getLock().acquireVolumeLock(volume);
|
|
metadataManager.getLock().acquireVolumeLock(volume);
|
|
try {
|
|
try {
|
|
@@ -235,15 +308,13 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
|
|
|
|
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
|
|
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
|
|
|
|
|
|
- OmVolumeArgs newVolumeArgs =
|
|
|
|
- OmVolumeArgs.newBuilder()
|
|
|
|
- .setVolume(volumeArgs.getVolume())
|
|
|
|
- .setAdminName(volumeArgs.getAdminName())
|
|
|
|
- .setOwnerName(volumeArgs.getOwnerName())
|
|
|
|
- .setQuotaInBytes(quota)
|
|
|
|
- .setCreationTime(volumeArgs.getCreationTime()).build();
|
|
|
|
|
|
|
|
- metadataManager.getVolumeTable().put(dbVolumeKey, newVolumeArgs);
|
|
|
|
|
|
+ volumeArgs.setQuotaInBytes(quota);
|
|
|
|
+
|
|
|
|
+ if (!isRatisEnabled) {
|
|
|
|
+ metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs);
|
|
|
|
+ }
|
|
|
|
+ return volumeArgs;
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
if (!(ex instanceof OMException)) {
|
|
if (!(ex instanceof OMException)) {
|
|
LOG.error("Changing volume quota failed for volume:{} quota:{}", volume,
|
|
LOG.error("Changing volume quota failed for volume:{} quota:{}", volume,
|
|
@@ -255,6 +326,19 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void applySetQuota(OmVolumeArgs omVolumeArgs) throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ String dbVolumeKey = metadataManager.getVolumeKey(
|
|
|
|
+ omVolumeArgs.getVolume());
|
|
|
|
+ metadataManager.getVolumeTable().put(dbVolumeKey, omVolumeArgs);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ LOG.error("Changing volume quota failed for volume:{} quota:{}",
|
|
|
|
+ omVolumeArgs.getVolume(), omVolumeArgs.getQuotaInBytes(), ex);
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Gets the volume information.
|
|
* Gets the volume information.
|
|
* @param volume - Volume name.
|
|
* @param volume - Volume name.
|
|
@@ -290,10 +374,12 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
* Deletes an existing empty volume.
|
|
* Deletes an existing empty volume.
|
|
*
|
|
*
|
|
* @param volume - Name of the volume.
|
|
* @param volume - Name of the volume.
|
|
|
|
+ *
|
|
|
|
+ * @return OmDeleteVolumeResponse
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- public void deleteVolume(String volume) throws IOException {
|
|
|
|
|
|
+ public OmDeleteVolumeResponse deleteVolume(String volume) throws IOException {
|
|
Preconditions.checkNotNull(volume);
|
|
Preconditions.checkNotNull(volume);
|
|
String owner;
|
|
String owner;
|
|
metadataManager.getLock().acquireVolumeLock(volume);
|
|
metadataManager.getLock().acquireVolumeLock(volume);
|
|
@@ -305,7 +391,6 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
metadataManager.getLock().acquireUserLock(owner);
|
|
metadataManager.getLock().acquireUserLock(owner);
|
|
metadataManager.getLock().acquireVolumeLock(volume);
|
|
metadataManager.getLock().acquireVolumeLock(volume);
|
|
try {
|
|
try {
|
|
-
|
|
|
|
String dbVolumeKey = metadataManager.getVolumeKey(volume);
|
|
String dbVolumeKey = metadataManager.getVolumeKey(volume);
|
|
OmVolumeArgs volumeArgs =
|
|
OmVolumeArgs volumeArgs =
|
|
metadataManager.getVolumeTable().get(dbVolumeKey);
|
|
metadataManager.getVolumeTable().get(dbVolumeKey);
|
|
@@ -322,28 +407,54 @@ public class VolumeManagerImpl implements VolumeManager {
|
|
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
|
|
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
|
|
// delete the volume from the owner list
|
|
// delete the volume from the owner list
|
|
// as well as delete the volume entry
|
|
// as well as delete the volume entry
|
|
- try (BatchOperation batch = metadataManager.getStore()
|
|
|
|
- .initBatchOperation()) {
|
|
|
|
- delVolumeFromOwnerList(volume, volumeArgs.getOwnerName(), batch);
|
|
|
|
- metadataManager.getVolumeTable().deleteWithBatch(batch, dbVolumeKey);
|
|
|
|
- metadataManager.getStore().commitBatchOperation(batch);
|
|
|
|
|
|
+ VolumeList newVolumeList = delVolumeFromOwnerList(volume,
|
|
|
|
+ volumeArgs.getOwnerName());
|
|
|
|
+
|
|
|
|
+ if (!isRatisEnabled) {
|
|
|
|
+ deleteVolumeCommitToDB(newVolumeList,
|
|
|
|
+ volume, owner);
|
|
}
|
|
}
|
|
- } catch (RocksDBException| IOException ex) {
|
|
|
|
|
|
+ return new OmDeleteVolumeResponse(volume, owner, newVolumeList);
|
|
|
|
+ } catch (IOException ex) {
|
|
if (!(ex instanceof OMException)) {
|
|
if (!(ex instanceof OMException)) {
|
|
LOG.error("Delete volume failed for volume:{}", volume, ex);
|
|
LOG.error("Delete volume failed for volume:{}", volume, ex);
|
|
}
|
|
}
|
|
- if(ex instanceof RocksDBException) {
|
|
|
|
- throw RocksDBStore.toIOException("Volume creation failed.",
|
|
|
|
- (RocksDBException) ex);
|
|
|
|
- } else {
|
|
|
|
- throw (IOException) ex;
|
|
|
|
- }
|
|
|
|
|
|
+ throw ex;
|
|
} finally {
|
|
} finally {
|
|
metadataManager.getLock().releaseVolumeLock(volume);
|
|
metadataManager.getLock().releaseVolumeLock(volume);
|
|
metadataManager.getLock().releaseUserLock(owner);
|
|
metadataManager.getLock().releaseUserLock(owner);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void applyDeleteVolume(String volume, String owner,
|
|
|
|
+ VolumeList newVolumeList) throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ deleteVolumeCommitToDB(newVolumeList, volume, owner);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ LOG.error("Delete volume failed for volume:{}", volume,
|
|
|
|
+ ex);
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void deleteVolumeCommitToDB(VolumeList newVolumeList,
|
|
|
|
+ String volume, String owner) throws IOException {
|
|
|
|
+ try (BatchOperation batch = metadataManager.getStore()
|
|
|
|
+ .initBatchOperation()) {
|
|
|
|
+ String dbUserKey = metadataManager.getUserKey(owner);
|
|
|
|
+ if (newVolumeList.getVolumeNamesList().size() == 0) {
|
|
|
|
+ metadataManager.getUserTable().deleteWithBatch(batch, dbUserKey);
|
|
|
|
+ } else {
|
|
|
|
+ metadataManager.getUserTable().putWithBatch(batch, dbUserKey,
|
|
|
|
+ newVolumeList);
|
|
|
|
+ }
|
|
|
|
+ metadataManager.getVolumeTable().deleteWithBatch(batch,
|
|
|
|
+ metadataManager.getVolumeKey(volume));
|
|
|
|
+ metadataManager.getStore().commitBatchOperation(batch);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Checks if the specified user with a role can access this volume.
|
|
* Checks if the specified user with a role can access this volume.
|
|
*
|
|
*
|