|
@@ -347,7 +347,7 @@ public class PBHelper {
|
|
|
if (types == null || types.length == 0) {
|
|
|
return null;
|
|
|
}
|
|
|
- List<StorageTypeProto> list = convertStorageTypes(types);
|
|
|
+ List<StorageTypeProto> list = PBHelperClient.convertStorageTypes(types);
|
|
|
return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
|
|
|
}
|
|
|
|
|
@@ -382,20 +382,6 @@ public class PBHelper {
|
|
|
.getInfoSecurePort() : 0, dn.getIpcPort());
|
|
|
}
|
|
|
|
|
|
- public static DatanodeIDProto convert(DatanodeID dn) {
|
|
|
- // For wire compatibility with older versions we transmit the StorageID
|
|
|
- // which is the same as the DatanodeUuid. Since StorageID is a required
|
|
|
- // field we pass the empty string if the DatanodeUuid is not yet known.
|
|
|
- return DatanodeIDProto.newBuilder()
|
|
|
- .setIpAddr(dn.getIpAddr())
|
|
|
- .setHostName(dn.getHostName())
|
|
|
- .setXferPort(dn.getXferPort())
|
|
|
- .setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
|
|
|
- .setInfoPort(dn.getInfoPort())
|
|
|
- .setInfoSecurePort(dn.getInfoSecurePort())
|
|
|
- .setIpcPort(dn.getIpcPort()).build();
|
|
|
- }
|
|
|
-
|
|
|
// Arrays of DatanodeId
|
|
|
public static DatanodeIDProto[] convert(DatanodeID[] did) {
|
|
|
if (did == null)
|
|
@@ -403,7 +389,7 @@ public class PBHelper {
|
|
|
final int len = did.length;
|
|
|
DatanodeIDProto[] result = new DatanodeIDProto[len];
|
|
|
for (int i = 0; i < len; ++i) {
|
|
|
- result[i] = convert(did[i]);
|
|
|
+ result[i] = PBHelperClient.convert(did[i]);
|
|
|
}
|
|
|
return result;
|
|
|
}
|
|
@@ -434,7 +420,7 @@ public class PBHelper {
|
|
|
.setBlock(convert(blk.getBlock()))
|
|
|
.addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
|
|
|
.addAllStorageUuids(Arrays.asList(blk.getStorageIDs()))
|
|
|
- .addAllStorageTypes(convertStorageTypes(blk.getStorageTypes()))
|
|
|
+ .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()))
|
|
|
.build();
|
|
|
}
|
|
|
|
|
@@ -596,16 +582,6 @@ public class PBHelper {
|
|
|
eb.getGenerationStamp());
|
|
|
}
|
|
|
|
|
|
- public static ExtendedBlockProto convert(final ExtendedBlock b) {
|
|
|
- if (b == null) return null;
|
|
|
- return ExtendedBlockProto.newBuilder().
|
|
|
- setPoolId(b.getBlockPoolId()).
|
|
|
- setBlockId(b.getBlockId()).
|
|
|
- setNumBytes(b.getNumBytes()).
|
|
|
- setGenerationStamp(b.getGenerationStamp()).
|
|
|
- build();
|
|
|
- }
|
|
|
-
|
|
|
public static RecoveringBlockProto convert(RecoveringBlock b) {
|
|
|
if (b == null) {
|
|
|
return null;
|
|
@@ -626,17 +602,6 @@ public class PBHelper {
|
|
|
new RecoveringBlock(block, locs, b.getNewGenStamp());
|
|
|
}
|
|
|
|
|
|
- public static DatanodeInfoProto.AdminState convert(
|
|
|
- final DatanodeInfo.AdminStates inAs) {
|
|
|
- switch (inAs) {
|
|
|
- case NORMAL: return DatanodeInfoProto.AdminState.NORMAL;
|
|
|
- case DECOMMISSION_INPROGRESS:
|
|
|
- return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
|
|
|
- case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
|
|
|
- default: return DatanodeInfoProto.AdminState.NORMAL;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
static public DatanodeInfo convert(DatanodeInfoProto di) {
|
|
|
if (di == null) return null;
|
|
|
return new DatanodeInfo(
|
|
@@ -648,12 +613,6 @@ public class PBHelper {
|
|
|
di.getXceiverCount(), PBHelper.convert(di.getAdminState()));
|
|
|
}
|
|
|
|
|
|
- static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
|
|
|
- if (di == null) return null;
|
|
|
- return convert(di);
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
|
|
|
if (di == null) return null;
|
|
|
DatanodeInfo[] result = new DatanodeInfo[di.length];
|
|
@@ -663,27 +622,6 @@ public class PBHelper {
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
- public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
|
|
|
- DatanodeInfo[] dnInfos) {
|
|
|
- return convert(dnInfos, 0);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Copy from {@code dnInfos} to a target of list of same size starting at
|
|
|
- * {@code startIdx}.
|
|
|
- */
|
|
|
- public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
|
|
|
- DatanodeInfo[] dnInfos, int startIdx) {
|
|
|
- if (dnInfos == null)
|
|
|
- return null;
|
|
|
- ArrayList<HdfsProtos.DatanodeInfoProto> protos = Lists
|
|
|
- .newArrayListWithCapacity(dnInfos.length);
|
|
|
- for (int i = startIdx; i < dnInfos.length; i++) {
|
|
|
- protos.add(convert(dnInfos[i]));
|
|
|
- }
|
|
|
- return protos;
|
|
|
- }
|
|
|
-
|
|
|
public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
|
|
|
DatanodeInfo[] info = new DatanodeInfo[list.size()];
|
|
|
for (int i = 0; i < info.length; i++) {
|
|
@@ -691,32 +629,11 @@ public class PBHelper {
|
|
|
}
|
|
|
return info;
|
|
|
}
|
|
|
-
|
|
|
- public static DatanodeInfoProto convert(DatanodeInfo info) {
|
|
|
- DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
|
|
|
- if (info.getNetworkLocation() != null) {
|
|
|
- builder.setLocation(info.getNetworkLocation());
|
|
|
- }
|
|
|
- builder
|
|
|
- .setId(PBHelper.convert((DatanodeID)info))
|
|
|
- .setCapacity(info.getCapacity())
|
|
|
- .setDfsUsed(info.getDfsUsed())
|
|
|
- .setRemaining(info.getRemaining())
|
|
|
- .setBlockPoolUsed(info.getBlockPoolUsed())
|
|
|
- .setCacheCapacity(info.getCacheCapacity())
|
|
|
- .setCacheUsed(info.getCacheUsed())
|
|
|
- .setLastUpdate(info.getLastUpdate())
|
|
|
- .setLastUpdateMonotonic(info.getLastUpdateMonotonic())
|
|
|
- .setXceiverCount(info.getXceiverCount())
|
|
|
- .setAdminState(PBHelper.convert(info.getAdminState()))
|
|
|
- .build();
|
|
|
- return builder.build();
|
|
|
- }
|
|
|
|
|
|
public static DatanodeStorageReportProto convertDatanodeStorageReport(
|
|
|
DatanodeStorageReport report) {
|
|
|
return DatanodeStorageReportProto.newBuilder()
|
|
|
- .setDatanodeInfo(convert(report.getDatanodeInfo()))
|
|
|
+ .setDatanodeInfo(PBHelperClient.convert(report.getDatanodeInfo()))
|
|
|
.addAllStorageReports(convertStorageReports(report.getStorageReports()))
|
|
|
.build();
|
|
|
}
|
|
@@ -768,7 +685,7 @@ public class PBHelper {
|
|
|
Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
|
|
|
for (int i = 0; i < locs.length; i++) {
|
|
|
DatanodeInfo loc = locs[i];
|
|
|
- builder.addLocs(i, PBHelper.convert(loc));
|
|
|
+ builder.addLocs(i, PBHelperClient.convert(loc));
|
|
|
boolean locIsCached = cachedLocs.contains(loc);
|
|
|
builder.addIsCached(locIsCached);
|
|
|
if (locIsCached) {
|
|
@@ -782,7 +699,7 @@ public class PBHelper {
|
|
|
StorageType[] storageTypes = b.getStorageTypes();
|
|
|
if (storageTypes != null) {
|
|
|
for (int i = 0; i < storageTypes.length; ++i) {
|
|
|
- builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
|
|
|
+ builder.addStorageTypes(PBHelperClient.convertStorageType(storageTypes[i]));
|
|
|
}
|
|
|
}
|
|
|
final String[] storageIDs = b.getStorageIDs();
|
|
@@ -790,8 +707,8 @@ public class PBHelper {
|
|
|
builder.addAllStorageIDs(Arrays.asList(storageIDs));
|
|
|
}
|
|
|
|
|
|
- return builder.setB(PBHelper.convert(b.getBlock()))
|
|
|
- .setBlockToken(PBHelper.convert(b.getBlockToken()))
|
|
|
+ return builder.setB(PBHelperClient.convert(b.getBlock()))
|
|
|
+ .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
|
|
|
.setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
|
|
|
}
|
|
|
|
|
@@ -832,14 +749,6 @@ public class PBHelper {
|
|
|
return lb;
|
|
|
}
|
|
|
|
|
|
- public static TokenProto convert(Token<?> tok) {
|
|
|
- return TokenProto.newBuilder().
|
|
|
- setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
|
|
|
- setPassword(ByteString.copyFrom(tok.getPassword())).
|
|
|
- setKind(tok.getKind().toString()).
|
|
|
- setService(tok.getService().toString()).build();
|
|
|
- }
|
|
|
-
|
|
|
public static Token<BlockTokenIdentifier> convert(
|
|
|
TokenProto blockToken) {
|
|
|
return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
|
|
@@ -891,7 +800,7 @@ public class PBHelper {
|
|
|
DatanodeRegistration registration) {
|
|
|
DatanodeRegistrationProto.Builder builder = DatanodeRegistrationProto
|
|
|
.newBuilder();
|
|
|
- return builder.setDatanodeID(PBHelper.convert((DatanodeID) registration))
|
|
|
+ return builder.setDatanodeID(PBHelperClient.convert((DatanodeID) registration))
|
|
|
.setStorageInfo(PBHelper.convert(registration.getStorageInfo()))
|
|
|
.setKeys(PBHelper.convert(registration.getExportedKeys()))
|
|
|
.setSoftwareVersion(registration.getSoftwareVersion()).build();
|
|
@@ -983,7 +892,7 @@ public class PBHelper {
|
|
|
if (types != null) {
|
|
|
for (StorageType[] ts : types) {
|
|
|
StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
|
|
|
- builder.addAllStorageTypes(convertStorageTypes(ts));
|
|
|
+ builder.addAllStorageTypes(PBHelperClient.convertStorageTypes(ts));
|
|
|
list.add(builder.build());
|
|
|
}
|
|
|
}
|
|
@@ -1014,7 +923,7 @@ public class PBHelper {
|
|
|
DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
|
|
|
for (int i = 0; i < targets.length; i++) {
|
|
|
ret[i] = DatanodeInfosProto.newBuilder()
|
|
|
- .addAllDatanodes(PBHelper.convert(targets[i])).build();
|
|
|
+ .addAllDatanodes(PBHelperClient.convert(targets[i])).build();
|
|
|
}
|
|
|
return Arrays.asList(ret);
|
|
|
}
|
|
@@ -1338,7 +1247,7 @@ public class PBHelper {
|
|
|
fs.getFileBufferSize(),
|
|
|
fs.getEncryptDataTransfer(),
|
|
|
fs.getTrashInterval(),
|
|
|
- PBHelper.convert(fs.getChecksumType()));
|
|
|
+ PBHelperClient.convert(fs.getChecksumType()));
|
|
|
}
|
|
|
|
|
|
public static FsServerDefaultsProto convert(FsServerDefaults fs) {
|
|
@@ -1351,7 +1260,7 @@ public class PBHelper {
|
|
|
.setFileBufferSize(fs.getFileBufferSize())
|
|
|
.setEncryptDataTransfer(fs.getEncryptDataTransfer())
|
|
|
.setTrashInterval(fs.getTrashInterval())
|
|
|
- .setChecksumType(PBHelper.convert(fs.getChecksumType()))
|
|
|
+ .setChecksumType(PBHelperClient.convert(fs.getChecksumType()))
|
|
|
.build();
|
|
|
}
|
|
|
|
|
@@ -1739,7 +1648,7 @@ public class PBHelper {
|
|
|
if (cs.hasTypeQuotaInfos()) {
|
|
|
for (HdfsProtos.StorageTypeQuotaInfoProto info :
|
|
|
cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
|
|
|
- StorageType type = PBHelper.convertStorageType(info.getType());
|
|
|
+ StorageType type = PBHelperClient.convertStorageType(info.getType());
|
|
|
builder.typeConsumed(type, info.getConsumed());
|
|
|
builder.typeQuota(type, info.getQuota());
|
|
|
}
|
|
@@ -1763,7 +1672,7 @@ public class PBHelper {
|
|
|
for (StorageType t: StorageType.getTypesSupportingQuota()) {
|
|
|
HdfsProtos.StorageTypeQuotaInfoProto info =
|
|
|
HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
|
|
|
- setType(convertStorageType(t)).
|
|
|
+ setType(PBHelperClient.convertStorageType(t)).
|
|
|
setConsumed(cs.getTypeConsumed(t)).
|
|
|
setQuota(cs.getTypeQuota(t)).
|
|
|
build();
|
|
@@ -1808,7 +1717,7 @@ public class PBHelper {
|
|
|
public static DatanodeStorageProto convert(DatanodeStorage s) {
|
|
|
return DatanodeStorageProto.newBuilder()
|
|
|
.setState(PBHelper.convertState(s.getState()))
|
|
|
- .setStorageType(PBHelper.convertStorageType(s.getStorageType()))
|
|
|
+ .setStorageType(PBHelperClient.convertStorageType(s.getStorageType()))
|
|
|
.setStorageUuid(s.getStorageID()).build();
|
|
|
}
|
|
|
|
|
@@ -1822,44 +1731,10 @@ public class PBHelper {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public static List<StorageTypeProto> convertStorageTypes(
|
|
|
- StorageType[] types) {
|
|
|
- return convertStorageTypes(types, 0);
|
|
|
- }
|
|
|
-
|
|
|
- public static List<StorageTypeProto> convertStorageTypes(
|
|
|
- StorageType[] types, int startIdx) {
|
|
|
- if (types == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- final List<StorageTypeProto> protos = new ArrayList<StorageTypeProto>(
|
|
|
- types.length);
|
|
|
- for (int i = startIdx; i < types.length; ++i) {
|
|
|
- protos.add(convertStorageType(types[i]));
|
|
|
- }
|
|
|
- return protos;
|
|
|
- }
|
|
|
-
|
|
|
- public static StorageTypeProto convertStorageType(StorageType type) {
|
|
|
- switch(type) {
|
|
|
- case DISK:
|
|
|
- return StorageTypeProto.DISK;
|
|
|
- case SSD:
|
|
|
- return StorageTypeProto.SSD;
|
|
|
- case ARCHIVE:
|
|
|
- return StorageTypeProto.ARCHIVE;
|
|
|
- case RAM_DISK:
|
|
|
- return StorageTypeProto.RAM_DISK;
|
|
|
- default:
|
|
|
- throw new IllegalStateException(
|
|
|
- "BUG: StorageType not found, type=" + type);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
public static DatanodeStorage convert(DatanodeStorageProto s) {
|
|
|
return new DatanodeStorage(s.getStorageUuid(),
|
|
|
PBHelper.convertState(s.getState()),
|
|
|
- PBHelper.convertStorageType(s.getStorageType()));
|
|
|
+ PBHelperClient.convertStorageType(s.getStorageType()));
|
|
|
}
|
|
|
|
|
|
private static State convertState(StorageState state) {
|
|
@@ -1872,22 +1747,6 @@ public class PBHelper {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public static StorageType convertStorageType(StorageTypeProto type) {
|
|
|
- switch(type) {
|
|
|
- case DISK:
|
|
|
- return StorageType.DISK;
|
|
|
- case SSD:
|
|
|
- return StorageType.SSD;
|
|
|
- case ARCHIVE:
|
|
|
- return StorageType.ARCHIVE;
|
|
|
- case RAM_DISK:
|
|
|
- return StorageType.RAM_DISK;
|
|
|
- default:
|
|
|
- throw new IllegalStateException(
|
|
|
- "BUG: StorageTypeProto not found, type=" + type);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
public static StorageType[] convertStorageTypes(
|
|
|
List<StorageTypeProto> storageTypesList, int expectedSize) {
|
|
|
final StorageType[] storageTypes = new StorageType[expectedSize];
|
|
@@ -1896,7 +1755,7 @@ public class PBHelper {
|
|
|
Arrays.fill(storageTypes, StorageType.DEFAULT);
|
|
|
} else {
|
|
|
for (int i = 0; i < storageTypes.length; ++i) {
|
|
|
- storageTypes[i] = convertStorageType(storageTypesList.get(i));
|
|
|
+ storageTypes[i] = PBHelperClient.convertStorageType(storageTypesList.get(i));
|
|
|
}
|
|
|
}
|
|
|
return storageTypes;
|
|
@@ -2080,10 +1939,6 @@ public class PBHelper {
|
|
|
return reportProto;
|
|
|
}
|
|
|
|
|
|
- public static DataChecksum.Type convert(HdfsProtos.ChecksumTypeProto type) {
|
|
|
- return DataChecksum.Type.valueOf(type.getNumber());
|
|
|
- }
|
|
|
-
|
|
|
public static CacheDirectiveInfoProto convert
|
|
|
(CacheDirectiveInfo info) {
|
|
|
CacheDirectiveInfoProto.Builder builder =
|
|
@@ -2256,9 +2111,6 @@ public class PBHelper {
|
|
|
return new CachePoolEntry(info, stats);
|
|
|
}
|
|
|
|
|
|
- public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
|
|
|
- return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
|
|
|
- }
|
|
|
|
|
|
public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
|
|
|
DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
|
|
@@ -2273,17 +2125,6 @@ public class PBHelper {
|
|
|
proto.getConfigVersion(), proto.getUptime());
|
|
|
}
|
|
|
|
|
|
- public static InputStream vintPrefixed(final InputStream input)
|
|
|
- throws IOException {
|
|
|
- final int firstByte = input.read();
|
|
|
- if (firstByte == -1) {
|
|
|
- throw new EOFException("Premature EOF: no length prefix available");
|
|
|
- }
|
|
|
-
|
|
|
- int size = CodedInputStream.readRawVarint32(firstByte, input);
|
|
|
- assert size >= 0;
|
|
|
- return new ExactSizeInputStream(input, size);
|
|
|
- }
|
|
|
|
|
|
private static AclEntryScopeProto convert(AclEntryScope v) {
|
|
|
return AclEntryScopeProto.valueOf(v.ordinal());
|
|
@@ -2507,30 +2348,11 @@ public class PBHelper {
|
|
|
proto.getKeyName());
|
|
|
}
|
|
|
|
|
|
- public static ShortCircuitShmSlotProto convert(SlotId slotId) {
|
|
|
- return ShortCircuitShmSlotProto.newBuilder().
|
|
|
- setShmId(convert(slotId.getShmId())).
|
|
|
- setSlotIdx(slotId.getSlotIdx()).
|
|
|
- build();
|
|
|
- }
|
|
|
-
|
|
|
- public static ShortCircuitShmIdProto convert(ShmId shmId) {
|
|
|
- return ShortCircuitShmIdProto.newBuilder().
|
|
|
- setHi(shmId.getHi()).
|
|
|
- setLo(shmId.getLo()).
|
|
|
- build();
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
public static SlotId convert(ShortCircuitShmSlotProto slotId) {
|
|
|
- return new SlotId(PBHelper.convert(slotId.getShmId()),
|
|
|
+ return new SlotId(PBHelperClient.convert(slotId.getShmId()),
|
|
|
slotId.getSlotIdx());
|
|
|
}
|
|
|
|
|
|
- public static ShmId convert(ShortCircuitShmIdProto shmId) {
|
|
|
- return new ShmId(shmId.getHi(), shmId.getLo());
|
|
|
- }
|
|
|
-
|
|
|
private static Event.CreateEvent.INodeType createTypeConvert(InotifyProtos.INodeType
|
|
|
type) {
|
|
|
switch (type) {
|
|
@@ -3037,18 +2859,6 @@ public class PBHelper {
|
|
|
ezKeyVersionName);
|
|
|
}
|
|
|
|
|
|
- public static List<Boolean> convert(boolean[] targetPinnings, int idx) {
|
|
|
- List<Boolean> pinnings = new ArrayList<Boolean>();
|
|
|
- if (targetPinnings == null) {
|
|
|
- pinnings.add(Boolean.FALSE);
|
|
|
- } else {
|
|
|
- for (; idx < targetPinnings.length; ++idx) {
|
|
|
- pinnings.add(Boolean.valueOf(targetPinnings[idx]));
|
|
|
- }
|
|
|
- }
|
|
|
- return pinnings;
|
|
|
- }
|
|
|
-
|
|
|
public static boolean[] convertBooleanList(
|
|
|
List<Boolean> targetPinningsList) {
|
|
|
final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
|