|
@@ -17,36 +17,38 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.ozone.web.localstorage;
|
|
|
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
|
|
-import org.apache.hadoop.ozone.web.handlers.ListArgs;
|
|
|
-import org.apache.hadoop.ozone.web.response.KeyInfo;
|
|
|
-import org.apache.hadoop.ozone.web.response.ListKeys;
|
|
|
-import org.slf4j.LoggerFactory;
|
|
|
-import org.slf4j.Logger;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import org.apache.commons.codec.digest.DigestUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
+import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.ozone.container.common.utils.LevelDBStore;
|
|
|
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
|
|
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
|
|
|
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
|
|
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
|
|
+import org.apache.hadoop.ozone.web.handlers.ListArgs;
|
|
|
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
|
|
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
|
|
|
import org.apache.hadoop.ozone.web.request.OzoneAcl;
|
|
|
import org.apache.hadoop.ozone.web.response.BucketInfo;
|
|
|
+import org.apache.hadoop.ozone.web.response.KeyInfo;
|
|
|
import org.apache.hadoop.ozone.web.response.ListBuckets;
|
|
|
+import org.apache.hadoop.ozone.web.response.ListKeys;
|
|
|
import org.apache.hadoop.ozone.web.response.ListVolumes;
|
|
|
import org.apache.hadoop.ozone.web.response.VolumeInfo;
|
|
|
import org.apache.hadoop.ozone.web.response.VolumeOwner;
|
|
|
-import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.iq80.leveldb.DBException;
|
|
|
-import org.apache.commons.codec.digest.DigestUtils;
|
|
|
+import org.iq80.leveldb.DBIterator;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
+import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
-import java.io.FileOutputStream;
|
|
|
import java.nio.charset.Charset;
|
|
|
import java.text.SimpleDateFormat;
|
|
|
import java.util.Date;
|
|
@@ -375,21 +377,180 @@ public final class OzoneMetadataManager {
|
|
|
* @return - ListVolumes
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
|
- public ListVolumes listVolumes(UserArgs args) throws OzoneException {
|
|
|
+ public ListVolumes listVolumes(ListArgs args) throws OzoneException {
|
|
|
lock.readLock().lock();
|
|
|
try {
|
|
|
- byte[] volumeList = userDB.get(args.getUserName().getBytes(encoding));
|
|
|
+ Preconditions.checkState(args.getArgs() instanceof UserArgs);
|
|
|
+
|
|
|
+ if (args.isRootScan()) {
|
|
|
+ return listAllVolumes(args);
|
|
|
+ }
|
|
|
+
|
|
|
+ UserArgs uArgs = (UserArgs) args.getArgs();
|
|
|
+ byte[] volumeList = userDB.get(uArgs.getUserName().getBytes(encoding));
|
|
|
if (volumeList == null) {
|
|
|
- throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args);
|
|
|
+ throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, uArgs);
|
|
|
+ }
|
|
|
+
|
|
|
+ String prefix = args.getPrefix();
|
|
|
+ int maxCount = args.getMaxKeys();
|
|
|
+ String prevKey = args.getPrevKey();
|
|
|
+ if (prevKey != null) {
|
|
|
+ // Format is username/volumeName, in local mode we don't use the
|
|
|
+ // user name since we have a userName DB.
|
|
|
+ String[] volName = args.getPrevKey().split("/");
|
|
|
+ if (volName.length < 2) {
|
|
|
+ throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, uArgs);
|
|
|
+ }
|
|
|
+ prevKey = volName[1];
|
|
|
}
|
|
|
- return ListVolumes.parse(new String(volumeList, encoding));
|
|
|
+ return getFilteredVolumes(volumeList, prefix, prevKey, maxCount);
|
|
|
} catch (IOException | DBException ex) {
|
|
|
- throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args.getArgs(), ex);
|
|
|
} finally {
|
|
|
lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns a List of Volumes that meet the prefix, prevkey and maxCount
|
|
|
+ * constraints.
|
|
|
+ *
|
|
|
+ * @param volumeList - Byte Array of Volume Info.
|
|
|
+ * @param prefix - prefix string.
|
|
|
+ * @param prevKey - PrevKey
|
|
|
+ * @param maxCount - Maximum Count.
|
|
|
+ * @return ListVolumes.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private ListVolumes getFilteredVolumes(byte[] volumeList, String prefix,
|
|
|
+ String prevKey, int maxCount) throws
|
|
|
+ IOException {
|
|
|
+ ListVolumes volumes = ListVolumes.parse(new String(volumeList,
|
|
|
+ encoding));
|
|
|
+ int currentCount = 0;
|
|
|
+ ListIterator<VolumeInfo> iter = volumes.getVolumes().listIterator();
|
|
|
+ ListVolumes filteredVolumes = new ListVolumes();
|
|
|
+ while (currentCount < maxCount && iter.hasNext()) {
|
|
|
+ VolumeInfo vInfo = iter.next();
|
|
|
+ if (isMatchingPrefix(prefix, vInfo) && isAfterKey(prevKey, vInfo)) {
|
|
|
+ filteredVolumes.addVolume(vInfo);
|
|
|
+ currentCount++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return filteredVolumes;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns all volumes in a cluster.
|
|
|
+ *
|
|
|
+ * @param args - ListArgs.
|
|
|
+ * @return ListVolumes.
|
|
|
+ * @throws OzoneException
|
|
|
+ */
|
|
|
+ public ListVolumes listAllVolumes(ListArgs args) throws OzoneException,
|
|
|
+ IOException {
|
|
|
+ String prefix = args.getPrefix();
|
|
|
+ String prevKey = args.getPrevKey();
|
|
|
+ int maxCount = args.getMaxKeys();
|
|
|
+ String userName = null;
|
|
|
+ DBIterator iterator = this.userDB.getDB().iterator();
|
|
|
+
|
|
|
+ if (prevKey != null) {
|
|
|
+ // Format is username/volumeName
|
|
|
+
|
|
|
+ String[] volName = args.getPrevKey().split("/");
|
|
|
+ if (volName.length < 2) {
|
|
|
+ throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
|
|
|
+ }
|
|
|
+ seekToUser(iterator, volName[0]);
|
|
|
+ userName = new String(iterator.peekNext().getKey(), encoding);
|
|
|
+ prevKey = volName[1];
|
|
|
+ } else {
|
|
|
+ userName = getFirstUser(iterator);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (userName == null || userName.isEmpty()) {
|
|
|
+ throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
|
|
|
+ }
|
|
|
+
|
|
|
+ ListVolumes returnSet = new ListVolumes();
|
|
|
+ int count = maxCount - returnSet.getVolumes().size();
|
|
|
+
|
|
|
+ // we need to iterate through users until we get maxcount volumes
|
|
|
+ // or no more volumes are left.
|
|
|
+ while (iterator.hasNext() && count > 0) {
|
|
|
+
|
|
|
+ userName = new String(iterator.next().getKey(), encoding);
|
|
|
+
|
|
|
+ byte[] volumeList = userDB.get(userName.getBytes(encoding));
|
|
|
+ if (volumeList == null) {
|
|
|
+ throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
|
|
|
+ }
|
|
|
+
|
|
|
+ returnSet.getVolumes().addAll(
|
|
|
+ getFilteredVolumes(volumeList, prefix, prevKey, count).getVolumes());
|
|
|
+ count = maxCount - returnSet.getVolumes().size();
|
|
|
+ }
|
|
|
+ return returnSet;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the first user name from the UserDB.
|
|
|
+ *
|
|
|
+ * @return - UserName.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ String getFirstUser(DBIterator iterator) throws IOException {
|
|
|
+ iterator.seekToFirst();
|
|
|
+ if (iterator.hasNext()) {
|
|
|
+ return new String(iterator.peekNext().getKey(), encoding);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Reposition the DB cursor to the user name.
|
|
|
+ *
|
|
|
+ * @param iterator - Current Iterator.
|
|
|
+ * @param userName - userName to seek to
|
|
|
+ * @return - DBIterator.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ DBIterator seekToUser(DBIterator iterator, String userName) throws
|
|
|
+ IOException {
|
|
|
+ iterator.seek(userName.getBytes(encoding));
|
|
|
+ return iterator;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Checks if a name starts with a matching prefix.
|
|
|
+ *
|
|
|
+ * @param prefix - prefix string.
|
|
|
+ * @param vInfo - volume info.
|
|
|
+ * @return true or false.
|
|
|
+ */
|
|
|
+ private boolean isMatchingPrefix(String prefix, VolumeInfo vInfo) {
|
|
|
+ if (prefix == null || prefix.isEmpty()) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return vInfo.getVolumeName().startsWith(prefix);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Checks if the key is after the prevKey.
|
|
|
+ *
|
|
|
+ * @param prevKey - String prevKey.
|
|
|
+ * @param vInfo - volume Info.
|
|
|
+ * @return - true or false.
|
|
|
+ */
|
|
|
+ private boolean isAfterKey(String prevKey, VolumeInfo vInfo) {
|
|
|
+ if (prevKey == null || prevKey.isEmpty()) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return prevKey.compareTo(vInfo.getVolumeName()) < 0;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Deletes a volume if it exists and is empty.
|
|
|
*
|
|
@@ -683,26 +844,31 @@ public final class OzoneMetadataManager {
|
|
|
* @return List of buckets
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
|
- public ListBuckets listBuckets(VolumeArgs args) throws OzoneException {
|
|
|
+ public ListBuckets listBuckets(ListArgs args) throws OzoneException {
|
|
|
lock.readLock().lock();
|
|
|
try {
|
|
|
- String userVolKey = args.getUserName() + "/" + args.getVolumeName();
|
|
|
+ Preconditions.checkState(args.getArgs() instanceof VolumeArgs);
|
|
|
+ VolumeArgs vArgs = (VolumeArgs) args.getArgs();
|
|
|
+ String userVolKey = vArgs.getUserName() + "/" + vArgs.getVolumeName();
|
|
|
|
|
|
+ // TODO : Query using Prefix and PrevKey
|
|
|
byte[] bucketBytes = userDB.get(userVolKey.getBytes(encoding));
|
|
|
if (bucketBytes == null) {
|
|
|
- throw ErrorTable.newError(ErrorTable.INVALID_VOLUME_NAME, args);
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_VOLUME_NAME,
|
|
|
+ args.getArgs());
|
|
|
}
|
|
|
return ListBuckets.parse(new String(bucketBytes, encoding));
|
|
|
} catch (IOException ex) {
|
|
|
- throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args.getArgs(), ex);
|
|
|
} finally {
|
|
|
lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Creates a key and returns a stream to which this key can be written to.
|
|
|
- * @param args KeyArgs
|
|
|
+ * Creates a key and returns a stream to which this key can be written to.
|
|
|
+ *
|
|
|
+ * @param args KeyArgs
|
|
|
* @return - A stream into which key can be written to.
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
@@ -721,13 +887,13 @@ public final class OzoneMetadataManager {
|
|
|
// only if the upload is successful.
|
|
|
if (f.exists()) {
|
|
|
LOG.debug("we are overwriting a file. This is by design.");
|
|
|
- if(!f.delete()) {
|
|
|
+ if (!f.delete()) {
|
|
|
LOG.error("Unable to delete the file: {}", fullPath);
|
|
|
throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // f.createNewFile();
|
|
|
+ // f.createNewFile();
|
|
|
FileOutputStream fsStream = new FileOutputStream(f);
|
|
|
inProgressObjects.put(fsStream, fullPath);
|
|
|
|
|
@@ -739,14 +905,11 @@ public final class OzoneMetadataManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
/**
|
|
|
- * commit keys moves an In progress object into the metadata store
|
|
|
- * so that key is visible in the metadata operations from that point
|
|
|
- * onwards.
|
|
|
+ * commit keys moves an In progress object into the metadata store so that key
|
|
|
+ * is visible in the metadata operations from that point onwards.
|
|
|
*
|
|
|
* @param args Object args
|
|
|
- *
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
|
public void commitKey(KeyArgs args, OutputStream stream)
|
|
@@ -813,7 +976,7 @@ public final class OzoneMetadataManager {
|
|
|
keyInfo.toDBString().getBytes(encoding));
|
|
|
|
|
|
metadataDB.put(args.getParentName().getBytes(encoding),
|
|
|
- bInfo.toDBString().getBytes(encoding));
|
|
|
+ bInfo.toDBString().getBytes(encoding));
|
|
|
|
|
|
userDB.put(args.getParentName().getBytes(encoding),
|
|
|
keyList.toDBString().getBytes(encoding));
|
|
@@ -829,7 +992,6 @@ public final class OzoneMetadataManager {
|
|
|
* deletes an key from a given bucket.
|
|
|
*
|
|
|
* @param args - ObjectArgs
|
|
|
- *
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
|
public void deleteKey(KeyArgs args) throws OzoneException {
|
|
@@ -867,7 +1029,7 @@ public final class OzoneMetadataManager {
|
|
|
File f = new File(fullPath);
|
|
|
|
|
|
if (f.exists()) {
|
|
|
- if(!f.delete()) {
|
|
|
+ if (!f.delete()) {
|
|
|
throw ErrorTable.newError(ErrorTable.KEY_OPERATION_CONFLICT, args);
|
|
|
}
|
|
|
} else {
|
|
@@ -877,7 +1039,7 @@ public final class OzoneMetadataManager {
|
|
|
|
|
|
metadataDB.delete(args.getResourceName().getBytes(encoding));
|
|
|
metadataDB.put(args.getParentName().getBytes(encoding),
|
|
|
- bInfo.toDBString().getBytes(encoding));
|
|
|
+ bInfo.toDBString().getBytes(encoding));
|
|
|
userDB.put(args.getParentName().getBytes(encoding),
|
|
|
keyList.toDBString().getBytes(encoding));
|
|
|
} catch (IOException e) {
|
|
@@ -891,9 +1053,7 @@ public final class OzoneMetadataManager {
|
|
|
* Returns a Stream for the file.
|
|
|
*
|
|
|
* @param args - Object args
|
|
|
- *
|
|
|
* @return Stream
|
|
|
- *
|
|
|
* @throws IOException
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
@@ -918,24 +1078,28 @@ public final class OzoneMetadataManager {
|
|
|
|
|
|
/**
|
|
|
* Returns keys in a bucket.
|
|
|
+ *
|
|
|
* @param args
|
|
|
- * @return List of keys.
|
|
|
+ * @return List of keys.
|
|
|
* @throws IOException
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
|
public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
|
|
|
lock.readLock().lock();
|
|
|
+ // TODO : Support Prefix and PrevKey lookup.
|
|
|
try {
|
|
|
- byte[] bucketInfo = metadataDB.get(args.getResourceName()
|
|
|
+ Preconditions.checkState(args.getArgs() instanceof BucketArgs);
|
|
|
+ BucketArgs bArgs = (BucketArgs) args.getArgs();
|
|
|
+ byte[] bucketInfo = metadataDB.get(bArgs.getResourceName()
|
|
|
.getBytes(encoding));
|
|
|
if (bucketInfo == null) {
|
|
|
- throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, bArgs);
|
|
|
}
|
|
|
|
|
|
- byte[] bucketListBytes = userDB.get(args.getResourceName()
|
|
|
+ byte[] bucketListBytes = userDB.get(bArgs.getResourceName()
|
|
|
.getBytes(encoding));
|
|
|
if (bucketListBytes == null) {
|
|
|
- throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, bArgs);
|
|
|
}
|
|
|
return ListKeys.parse(new String(bucketListBytes, encoding));
|
|
|
} finally {
|