|
@@ -21,7 +21,10 @@ import com.google.common.base.Strings;
|
|
|
import com.google.common.collect.Lists;
|
|
|
import org.apache.commons.lang3.tuple.ImmutablePair;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
-import org.apache.hadoop.ozone.ksm.helpers.*;
|
|
|
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
|
|
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
|
|
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
|
|
|
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
|
|
import org.apache.hadoop.ozone.common.BlockGroup;
|
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
@@ -146,16 +149,16 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
|
|
|
}
|
|
|
|
|
|
private String getKeyKeyPrefix(String volume, String bucket, String key) {
|
|
|
- String keyStr = getBucketKeyPrefix(volume, bucket);
|
|
|
- keyStr = Strings.isNullOrEmpty(key) ? keyStr + OzoneConsts.KSM_KEY_PREFIX
|
|
|
- : keyStr + OzoneConsts.KSM_KEY_PREFIX + key;
|
|
|
- return keyStr;
|
|
|
+ String keyVB = OzoneConsts.KSM_KEY_PREFIX + volume
|
|
|
+ + OzoneConsts.KSM_KEY_PREFIX + bucket
|
|
|
+ + OzoneConsts.KSM_KEY_PREFIX;
|
|
|
+ return Strings.isNullOrEmpty(key) ? keyVB : keyVB + key;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public byte[] getDBKeyForKey(String volume, String bucket, String key) {
|
|
|
- String keyKeyString = OzoneConsts.KSM_VOLUME_PREFIX + volume
|
|
|
- + OzoneConsts.KSM_BUCKET_PREFIX + bucket + OzoneConsts.KSM_KEY_PREFIX
|
|
|
+ String keyKeyString = OzoneConsts.KSM_KEY_PREFIX + volume
|
|
|
+ + OzoneConsts.KSM_KEY_PREFIX + bucket + OzoneConsts.KSM_KEY_PREFIX
|
|
|
+ key;
|
|
|
return DFSUtil.string2Bytes(keyKeyString);
|
|
|
}
|
|
@@ -223,15 +226,14 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
|
|
|
* @return true if the volume is empty
|
|
|
*/
|
|
|
public boolean isVolumeEmpty(String volume) throws IOException {
|
|
|
- String dbVolumeRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume;
|
|
|
+ String dbVolumeRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume
|
|
|
+ + OzoneConsts.KSM_BUCKET_PREFIX;
|
|
|
byte[] dbVolumeRootKey = DFSUtil.string2Bytes(dbVolumeRootName);
|
|
|
- // Seek to the root of the volume and look for the next key
|
|
|
ImmutablePair<byte[], byte[]> volumeRoot =
|
|
|
- store.peekAround(1, dbVolumeRootKey);
|
|
|
+ store.peekAround(0, dbVolumeRootKey);
|
|
|
if (volumeRoot != null) {
|
|
|
- String firstBucketKey = DFSUtil.bytes2String(volumeRoot.getKey());
|
|
|
- return !firstBucketKey.startsWith(dbVolumeRootName
|
|
|
- + OzoneConsts.KSM_BUCKET_PREFIX);
|
|
|
+ return !DFSUtil.bytes2String(volumeRoot.getKey())
|
|
|
+ .startsWith(dbVolumeRootName);
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
@@ -245,13 +247,13 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
|
|
|
*/
|
|
|
public boolean isBucketEmpty(String volume, String bucket)
|
|
|
throws IOException {
|
|
|
- String keyRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume
|
|
|
- + OzoneConsts.KSM_BUCKET_PREFIX + bucket;
|
|
|
+ String keyRootName = OzoneConsts.KSM_KEY_PREFIX + volume
|
|
|
+ + OzoneConsts.KSM_KEY_PREFIX + bucket + OzoneConsts.KSM_KEY_PREFIX;
|
|
|
byte[] keyRoot = DFSUtil.string2Bytes(keyRootName);
|
|
|
- ImmutablePair<byte[], byte[]> firstKey = store.peekAround(1, keyRoot);
|
|
|
+ ImmutablePair<byte[], byte[]> firstKey = store.peekAround(0, keyRoot);
|
|
|
if (firstKey != null) {
|
|
|
return !DFSUtil.bytes2String(firstKey.getKey())
|
|
|
- .startsWith(keyRootName + OzoneConsts.KSM_KEY_PREFIX);
|
|
|
+ .startsWith(keyRootName);
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
@@ -276,30 +278,27 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
|
|
|
}
|
|
|
|
|
|
|
|
|
- // A bucket must start with /volume/bucket_prefix
|
|
|
- // and exclude keys /volume/bucket_xxx/key_xxx
|
|
|
+ // A bucket starts with /#volume/#bucket_prefix
|
|
|
MetadataKeyFilter filter = (preKey, currentKey, nextKey) -> {
|
|
|
if (currentKey != null) {
|
|
|
String bucketNamePrefix = getBucketKeyPrefix(volumeName, bucketPrefix);
|
|
|
String bucket = DFSUtil.bytes2String(currentKey);
|
|
|
- return bucket.startsWith(bucketNamePrefix) &&
|
|
|
- !bucket.replaceFirst(bucketNamePrefix, "")
|
|
|
- .contains(OzoneConsts.KSM_KEY_PREFIX);
|
|
|
+ return bucket.startsWith(bucketNamePrefix);
|
|
|
}
|
|
|
return false;
|
|
|
};
|
|
|
|
|
|
List<Map.Entry<byte[], byte[]>> rangeResult;
|
|
|
if (!Strings.isNullOrEmpty(startBucket)) {
|
|
|
- //Since we are excluding start key from the result,
|
|
|
+ // Since we are excluding start key from the result,
|
|
|
// the maxNumOfBuckets is incremented.
|
|
|
- rangeResult = store.getRangeKVs(
|
|
|
+ rangeResult = store.getSequentialRangeKVs(
|
|
|
getBucketKey(volumeName, startBucket),
|
|
|
maxNumOfBuckets + 1, filter);
|
|
|
//Remove start key from result.
|
|
|
rangeResult.remove(0);
|
|
|
} else {
|
|
|
- rangeResult = store.getRangeKVs(null, maxNumOfBuckets, filter);
|
|
|
+ rangeResult = store.getSequentialRangeKVs(null, maxNumOfBuckets, filter);
|
|
|
}
|
|
|
|
|
|
for (Map.Entry<byte[], byte[]> entry : rangeResult) {
|