|
@@ -20,27 +20,24 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.nio.file.Paths;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
import org.apache.hadoop.hdds.client.BlockID;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
-import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.ozone.OmUtils;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.ozone.common.BlockGroup;
|
|
|
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;
|
|
|
+import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
+import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
|
|
|
+import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
|
|
|
+import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.utils.db.DBStore;
|
|
@@ -123,32 +120,32 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Table<byte[], byte[]> getUserTable() {
|
|
|
+ public Table<String, VolumeList> getUserTable() {
|
|
|
return userTable;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Table<byte[], byte[]> getVolumeTable() {
|
|
|
+ public Table<String, OmVolumeArgs> getVolumeTable() {
|
|
|
return volumeTable;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Table<byte[], byte[]> getBucketTable() {
|
|
|
+ public Table<String, OmBucketInfo> getBucketTable() {
|
|
|
return bucketTable;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Table<byte[], byte[]> getKeyTable() {
|
|
|
+ public Table<String, OmKeyInfo> getKeyTable() {
|
|
|
return keyTable;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Table<byte[], byte[]> getDeletedTable() {
|
|
|
+ public Table<String, OmKeyInfo> getDeletedTable() {
|
|
|
return deletedTable;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Table<byte[], byte[]> getOpenKeyTable() {
|
|
|
+ public Table<String, OmKeyInfo> getOpenKeyTable() {
|
|
|
return openKeyTable;
|
|
|
}
|
|
|
|
|
@@ -178,6 +175,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
// db, so we need to create the store object and initialize DB.
|
|
|
if (store == null) {
|
|
|
File metaDir = OmUtils.getOmDbDir(configuration);
|
|
|
+
|
|
|
this.store = DBStoreBuilder.newBuilder(configuration)
|
|
|
.setName(OM_DB_NAME)
|
|
|
.setPath(Paths.get(metaDir.getPath()))
|
|
@@ -188,28 +186,39 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
.addTable(DELETED_TABLE)
|
|
|
.addTable(OPEN_KEY_TABLE)
|
|
|
.addTable(S3_TABLE)
|
|
|
+ .addCodec(OmKeyInfo.class, new OmKeyInfoCodec())
|
|
|
+ .addCodec(OmBucketInfo.class, new OmBucketInfoCodec())
|
|
|
+ .addCodec(OmVolumeArgs.class, new OmVolumeArgsCodec())
|
|
|
+ .addCodec(VolumeList.class, new VolumeListCodec())
|
|
|
.build();
|
|
|
|
|
|
- userTable = this.store.getTable(USER_TABLE);
|
|
|
+ userTable =
|
|
|
+ this.store.getTable(USER_TABLE, String.class, VolumeList.class);
|
|
|
checkTableStatus(userTable, USER_TABLE);
|
|
|
-
|
|
|
- volumeTable = this.store.getTable(VOLUME_TABLE);
|
|
|
+ this.store.getTable(VOLUME_TABLE, String.class,
|
|
|
+ String.class);
|
|
|
+ volumeTable =
|
|
|
+ this.store.getTable(VOLUME_TABLE, String.class, OmVolumeArgs.class);
|
|
|
checkTableStatus(volumeTable, VOLUME_TABLE);
|
|
|
|
|
|
- bucketTable = this.store.getTable(BUCKET_TABLE);
|
|
|
+ bucketTable =
|
|
|
+ this.store.getTable(BUCKET_TABLE, String.class, OmBucketInfo.class);
|
|
|
checkTableStatus(bucketTable, BUCKET_TABLE);
|
|
|
|
|
|
- keyTable = this.store.getTable(KEY_TABLE);
|
|
|
+ keyTable = this.store.getTable(KEY_TABLE, String.class, OmKeyInfo.class);
|
|
|
checkTableStatus(keyTable, KEY_TABLE);
|
|
|
|
|
|
- deletedTable = this.store.getTable(DELETED_TABLE);
|
|
|
+ deletedTable =
|
|
|
+ this.store.getTable(DELETED_TABLE, String.class, OmKeyInfo.class);
|
|
|
checkTableStatus(deletedTable, DELETED_TABLE);
|
|
|
|
|
|
- openKeyTable = this.store.getTable(OPEN_KEY_TABLE);
|
|
|
+ openKeyTable =
|
|
|
+ this.store.getTable(OPEN_KEY_TABLE, String.class, OmKeyInfo.class);
|
|
|
checkTableStatus(openKeyTable, OPEN_KEY_TABLE);
|
|
|
|
|
|
s3Table = this.store.getTable(S3_TABLE);
|
|
|
checkTableStatus(s3Table, S3_TABLE);
|
|
|
+
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -241,8 +250,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
* @param volume - Volume name
|
|
|
*/
|
|
|
@Override
|
|
|
- public byte[] getVolumeKey(String volume) {
|
|
|
- return DFSUtil.string2Bytes(OzoneConsts.OM_KEY_PREFIX + volume);
|
|
|
+ public String getVolumeKey(String volume) {
|
|
|
+ return OzoneConsts.OM_KEY_PREFIX + volume;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -251,8 +260,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
* @param user - User name
|
|
|
*/
|
|
|
@Override
|
|
|
- public byte[] getUserKey(String user) {
|
|
|
- return DFSUtil.string2Bytes(user);
|
|
|
+ public String getUserKey(String user) {
|
|
|
+ return user;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -262,18 +271,18 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
* @param bucket - Bucket name
|
|
|
*/
|
|
|
@Override
|
|
|
- public byte[] getBucketKey(String volume, String bucket) {
|
|
|
+ public String getBucketKey(String volume, String bucket) {
|
|
|
StringBuilder builder =
|
|
|
new StringBuilder().append(OM_KEY_PREFIX).append(volume);
|
|
|
|
|
|
if (StringUtils.isNotBlank(bucket)) {
|
|
|
builder.append(OM_KEY_PREFIX).append(bucket);
|
|
|
}
|
|
|
- return DFSUtil.string2Bytes(builder.toString());
|
|
|
+ return builder.toString();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public byte[] getOzoneKeyBytes(String volume, String bucket, String key) {
|
|
|
+ public String getOzoneKey(String volume, String bucket, String key) {
|
|
|
StringBuilder builder = new StringBuilder()
|
|
|
.append(OM_KEY_PREFIX).append(volume);
|
|
|
// TODO : Throw if the Bucket is null?
|
|
@@ -281,15 +290,15 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
if (StringUtil.isNotBlank(key)) {
|
|
|
builder.append(OM_KEY_PREFIX).append(key);
|
|
|
}
|
|
|
- return DFSUtil.string2Bytes(builder.toString());
|
|
|
+ return builder.toString();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public byte[] getOpenKeyBytes(String volume, String bucket,
|
|
|
- String key, long id) {
|
|
|
+ public String getOpenKey(String volume, String bucket,
|
|
|
+ String key, long id) {
|
|
|
String openKey = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket +
|
|
|
OM_KEY_PREFIX + key + OM_KEY_PREFIX + id;
|
|
|
- return DFSUtil.string2Bytes(openKey);
|
|
|
+ return openKey;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -349,11 +358,12 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
*/
|
|
|
@Override
|
|
|
public boolean isVolumeEmpty(String volume) throws IOException {
|
|
|
- byte[] volumePrefix = getVolumeKey(volume + OM_KEY_PREFIX);
|
|
|
- try (TableIterator<byte[], Table.KeyValue> bucketIter = bucketTable
|
|
|
- .iterator()) {
|
|
|
- Table.KeyValue<byte[], byte[]> kv = bucketIter.seek(volumePrefix);
|
|
|
- if (kv != null && startsWith(kv.getKey(), volumePrefix)) {
|
|
|
+ String volumePrefix = getVolumeKey(volume + OM_KEY_PREFIX);
|
|
|
+
|
|
|
+ try (TableIterator<String, ? extends KeyValue<String, OmBucketInfo>>
|
|
|
+ bucketIter = bucketTable.iterator()) {
|
|
|
+ KeyValue<String, OmBucketInfo> kv = bucketIter.seek(volumePrefix);
|
|
|
+ if (kv != null && kv.getKey().startsWith(volumePrefix)) {
|
|
|
return false; // we found at least one bucket with this volume prefix.
|
|
|
}
|
|
|
}
|
|
@@ -371,10 +381,11 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
@Override
|
|
|
public boolean isBucketEmpty(String volume, String bucket)
|
|
|
throws IOException {
|
|
|
- byte[] keyPrefix = getBucketKey(volume, bucket + OM_KEY_PREFIX);
|
|
|
- try (TableIterator<byte[], Table.KeyValue> keyIter = keyTable.iterator()) {
|
|
|
- Table.KeyValue<byte[], byte[]> kv = keyIter.seek(keyPrefix);
|
|
|
- if (kv != null && startsWith(kv.getKey(), keyPrefix)) {
|
|
|
+ String keyPrefix = getBucketKey(volume, bucket);
|
|
|
+ try (TableIterator<String, ? extends KeyValue<String, OmKeyInfo>> keyIter =
|
|
|
+ keyTable.iterator()) {
|
|
|
+ KeyValue<String, OmKeyInfo> kv = keyIter.seek(keyPrefix);
|
|
|
+ if (kv != null && kv.getKey().startsWith(keyPrefix)) {
|
|
|
return false; // we found at least one key with this vol/bucket prefix.
|
|
|
}
|
|
|
}
|
|
@@ -394,14 +405,13 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
|
|
}
|
|
|
|
|
|
- byte[] volumeNameBytes = getVolumeKey(volumeName);
|
|
|
+ String volumeNameBytes = getVolumeKey(volumeName);
|
|
|
if (volumeTable.get(volumeNameBytes) == null) {
|
|
|
throw new OMException("Volume " + volumeName + " not found.",
|
|
|
ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- byte[] startKey;
|
|
|
+ String startKey;
|
|
|
boolean skipStartKey = false;
|
|
|
if (StringUtil.isNotBlank(startBucket)) {
|
|
|
// if the user has specified a start key, we need to seek to that key
|
|
@@ -417,26 +427,26 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
startKey = getBucketKey(volumeName, bucketPrefix);
|
|
|
}
|
|
|
|
|
|
- byte[] seekPrefix;
|
|
|
+ String seekPrefix;
|
|
|
if (StringUtil.isNotBlank(bucketPrefix)) {
|
|
|
seekPrefix = getBucketKey(volumeName, bucketPrefix);
|
|
|
} else {
|
|
|
seekPrefix = getVolumeKey(volumeName + OM_KEY_PREFIX);
|
|
|
}
|
|
|
int currentCount = 0;
|
|
|
- try (TableIterator<byte[], Table.KeyValue> bucketIter = bucketTable
|
|
|
- .iterator()) {
|
|
|
- Table.KeyValue<byte[], byte[]> kv = bucketIter.seek(startKey);
|
|
|
+
|
|
|
+ try (TableIterator<String, ? extends KeyValue<String, OmBucketInfo>>
|
|
|
+ bucketIter = bucketTable.iterator()) {
|
|
|
+ KeyValue<String, OmBucketInfo> kv = bucketIter.seek(startKey);
|
|
|
while (currentCount < maxNumOfBuckets && bucketIter.hasNext()) {
|
|
|
kv = bucketIter.next();
|
|
|
// Skip the Start Bucket if needed.
|
|
|
if (kv != null && skipStartKey &&
|
|
|
- Arrays.equals(kv.getKey(), startKey)) {
|
|
|
+ kv.getKey().equals(startKey)) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (kv != null && startsWith(kv.getKey(), seekPrefix)) {
|
|
|
- result.add(OmBucketInfo.getFromProtobuf(
|
|
|
- BucketInfo.parseFrom(kv.getValue())));
|
|
|
+ if (kv != null && kv.getKey().startsWith(seekPrefix)) {
|
|
|
+ result.add(kv.getValue());
|
|
|
currentCount++;
|
|
|
} else {
|
|
|
// The SeekPrefix does not match any more, we can break out of the
|
|
@@ -462,43 +472,42 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
|
|
}
|
|
|
|
|
|
- byte[] bucketNameBytes = getBucketKey(volumeName, bucketName);
|
|
|
+ String bucketNameBytes = getBucketKey(volumeName, bucketName);
|
|
|
if (getBucketTable().get(bucketNameBytes) == null) {
|
|
|
throw new OMException("Bucket " + bucketName + " not found.",
|
|
|
ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
|
|
}
|
|
|
|
|
|
- byte[] seekKey;
|
|
|
+ String seekKey;
|
|
|
boolean skipStartKey = false;
|
|
|
if (StringUtil.isNotBlank(startKey)) {
|
|
|
// Seek to the specified key.
|
|
|
- seekKey = getOzoneKeyBytes(volumeName, bucketName, startKey);
|
|
|
+ seekKey = getOzoneKey(volumeName, bucketName, startKey);
|
|
|
skipStartKey = true;
|
|
|
} else {
|
|
|
// This allows us to seek directly to the first key with the right prefix.
|
|
|
- seekKey = getOzoneKeyBytes(volumeName, bucketName, keyPrefix);
|
|
|
+ seekKey = getOzoneKey(volumeName, bucketName, keyPrefix);
|
|
|
}
|
|
|
|
|
|
- byte[] seekPrefix;
|
|
|
+ String seekPrefix;
|
|
|
if (StringUtil.isNotBlank(keyPrefix)) {
|
|
|
- seekPrefix = getOzoneKeyBytes(volumeName, bucketName, keyPrefix);
|
|
|
+ seekPrefix = getOzoneKey(volumeName, bucketName, keyPrefix);
|
|
|
} else {
|
|
|
seekPrefix = getBucketKey(volumeName, bucketName + OM_KEY_PREFIX);
|
|
|
}
|
|
|
int currentCount = 0;
|
|
|
- try (TableIterator<byte[], ? extends KeyValue<byte[], byte[]>> keyIter =
|
|
|
+ try (TableIterator<String, ? extends KeyValue<String, OmKeyInfo>> keyIter =
|
|
|
getKeyTable()
|
|
|
.iterator()) {
|
|
|
- Table.KeyValue<byte[], byte[]> kv = keyIter.seek(seekKey);
|
|
|
+ KeyValue<String, OmKeyInfo> kv = keyIter.seek(seekKey);
|
|
|
while (currentCount < maxKeys && keyIter.hasNext()) {
|
|
|
kv = keyIter.next();
|
|
|
// Skip the Start key if needed.
|
|
|
- if (kv != null && skipStartKey && Arrays.equals(kv.getKey(), seekKey)) {
|
|
|
+ if (kv != null && skipStartKey && kv.getKey().equals(seekKey)) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (kv != null && startsWith(kv.getKey(), seekPrefix)) {
|
|
|
- result.add(OmKeyInfo.getFromProtobuf(
|
|
|
- KeyInfo.parseFrom(kv.getValue())));
|
|
|
+ if (kv != null && kv.getKey().startsWith(seekPrefix)) {
|
|
|
+ result.add(kv.getValue());
|
|
|
currentCount++;
|
|
|
} else {
|
|
|
// The SeekPrefix does not match any more, we can break out of the
|
|
@@ -538,8 +547,9 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
continue;
|
|
|
}
|
|
|
if (startKeyFound && result.size() < maxKeys) {
|
|
|
- byte[] volumeInfo = getVolumeTable().get(this.getVolumeKey(volumeName));
|
|
|
- if (volumeInfo == null) {
|
|
|
+ OmVolumeArgs volumeArgs =
|
|
|
+ getVolumeTable().get(this.getVolumeKey(volumeName));
|
|
|
+ if (volumeArgs == null) {
|
|
|
// Could not get volume info by given volume name,
|
|
|
// since the volume name is loaded from db,
|
|
|
// this probably means om db is corrupted or some entries are
|
|
@@ -547,8 +557,6 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
throw new OMException("Volume info not found for " + volumeName,
|
|
|
ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
|
|
}
|
|
|
- VolumeInfo info = VolumeInfo.parseFrom(volumeInfo);
|
|
|
- OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(info);
|
|
|
result.add(volumeArgs);
|
|
|
}
|
|
|
}
|
|
@@ -556,49 +564,42 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
- private VolumeList getVolumesByUser(String userName)
|
|
|
+ private VolumeList getVolumesByUser(String userNameKey)
|
|
|
throws OMException {
|
|
|
- return getVolumesByUser(getUserKey(userName));
|
|
|
- }
|
|
|
-
|
|
|
- private VolumeList getVolumesByUser(byte[] userNameKey)
|
|
|
- throws OMException {
|
|
|
- VolumeList volumes = null;
|
|
|
try {
|
|
|
- byte[] volumesInBytes = getUserTable().get(userNameKey);
|
|
|
- if (volumesInBytes == null) {
|
|
|
+ VolumeList volumeList = getUserTable().get(userNameKey);
|
|
|
+ if (volumeList == null) {
|
|
|
// No volume found for this user, return an empty list
|
|
|
return VolumeList.newBuilder().build();
|
|
|
+ } else {
|
|
|
+ return volumeList;
|
|
|
}
|
|
|
- volumes = VolumeList.parseFrom(volumesInBytes);
|
|
|
} catch (IOException e) {
|
|
|
throw new OMException("Unable to get volumes info by the given user, "
|
|
|
+ "metadata might be corrupted", e,
|
|
|
ResultCodes.FAILED_METADATA_ERROR);
|
|
|
}
|
|
|
- return volumes;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public List<BlockGroup> getPendingDeletionKeys(final int keyCount)
|
|
|
throws IOException {
|
|
|
List<BlockGroup> keyBlocksList = Lists.newArrayList();
|
|
|
- try (TableIterator<byte[], ? extends KeyValue<byte[], byte[]>> keyIter =
|
|
|
+ try (TableIterator<String, ? extends KeyValue<String, OmKeyInfo>> keyIter =
|
|
|
getDeletedTable()
|
|
|
.iterator()) {
|
|
|
int currentCount = 0;
|
|
|
while (keyIter.hasNext() && currentCount < keyCount) {
|
|
|
- KeyValue<byte[], byte[]> kv = keyIter.next();
|
|
|
+ KeyValue<String, OmKeyInfo> kv = keyIter.next();
|
|
|
if (kv != null) {
|
|
|
- OmKeyInfo info =
|
|
|
- OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(kv.getValue()));
|
|
|
+ OmKeyInfo info = kv.getValue();
|
|
|
// Get block keys as a list.
|
|
|
OmKeyLocationInfoGroup latest = info.getLatestVersionLocations();
|
|
|
List<BlockID> item = latest.getLocationList().stream()
|
|
|
.map(b -> new BlockID(b.getContainerID(), b.getLocalID()))
|
|
|
.collect(Collectors.toList());
|
|
|
BlockGroup keyBlocks = BlockGroup.newBuilder()
|
|
|
- .setKeyName(DFSUtil.bytes2String(kv.getKey()))
|
|
|
+ .setKeyName(kv.getKey())
|
|
|
.addAllBlockIDs(item)
|
|
|
.build();
|
|
|
keyBlocksList.add(keyBlocks);
|
|
@@ -614,27 +615,6 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
List<BlockGroup> keyBlocksList = Lists.newArrayList();
|
|
|
long now = Time.now();
|
|
|
// TODO: Fix the getExpiredOpenKeys, Not part of this patch.
|
|
|
- List<Map.Entry<byte[], byte[]>> rangeResult = Collections.emptyList();
|
|
|
-
|
|
|
- for (Map.Entry<byte[], byte[]> entry : rangeResult) {
|
|
|
- OmKeyInfo info =
|
|
|
- OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue()));
|
|
|
- long lastModify = info.getModificationTime();
|
|
|
- if (now - lastModify < this.openKeyExpireThresholdMS) {
|
|
|
- // consider as may still be active, not hanging.
|
|
|
- continue;
|
|
|
- }
|
|
|
- // Get block keys as a list.
|
|
|
- List<BlockID> item = info.getLatestVersionLocations()
|
|
|
- .getBlocksLatestVersionOnly().stream()
|
|
|
- .map(b -> new BlockID(b.getContainerID(), b.getLocalID()))
|
|
|
- .collect(Collectors.toList());
|
|
|
- BlockGroup keyBlocks = BlockGroup.newBuilder()
|
|
|
- .setKeyName(DFSUtil.bytes2String(entry.getKey()))
|
|
|
- .addAllBlockIDs(item)
|
|
|
- .build();
|
|
|
- keyBlocksList.add(keyBlocks);
|
|
|
- }
|
|
|
return keyBlocksList;
|
|
|
}
|
|
|
|