|
@@ -21,7 +21,6 @@ import org.apache.hadoop.fs.*;
|
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
|
import org.apache.hadoop.fs.permission.AclStatus;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.XAttrHelper;
|
|
|
import org.apache.hadoop.hdfs.protocol.*;
|
|
@@ -35,7 +34,8 @@ import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
-import org.mortbay.util.ajax.JSON;
|
|
|
+import org.codehaus.jackson.map.ObjectMapper;
|
|
|
+import org.codehaus.jackson.map.ObjectReader;
|
|
|
|
|
|
import com.google.common.collect.Lists;
|
|
|
import com.google.common.collect.Maps;
|
|
@@ -95,59 +95,6 @@ public class JsonUtil {
|
|
|
return (Token<BlockTokenIdentifier>)toToken(m);
|
|
|
}
|
|
|
|
|
|
- /** Convert a Token[] to a JSON array. */
|
|
|
- private static Object[] toJsonArray(final Token<? extends TokenIdentifier>[] array
|
|
|
- ) throws IOException {
|
|
|
- if (array == null) {
|
|
|
- return null;
|
|
|
- } else if (array.length == 0) {
|
|
|
- return EMPTY_OBJECT_ARRAY;
|
|
|
- } else {
|
|
|
- final Object[] a = new Object[array.length];
|
|
|
- for(int i = 0; i < array.length; i++) {
|
|
|
- a[i] = toJsonMap(array[i]);
|
|
|
- }
|
|
|
- return a;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** Convert a token object to a JSON string. */
|
|
|
- public static String toJsonString(final Token<? extends TokenIdentifier>[] tokens
|
|
|
- ) throws IOException {
|
|
|
- if (tokens == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- final Map<String, Object> m = new TreeMap<String, Object>();
|
|
|
- m.put(Token.class.getSimpleName(), toJsonArray(tokens));
|
|
|
- return toJsonString(Token.class.getSimpleName() + "s", m);
|
|
|
- }
|
|
|
-
|
|
|
- /** Convert an Object[] to a List<Token<?>>. */
|
|
|
- private static List<Token<?>> toTokenList(final Object[] objects) throws IOException {
|
|
|
- if (objects == null) {
|
|
|
- return null;
|
|
|
- } else if (objects.length == 0) {
|
|
|
- return Collections.emptyList();
|
|
|
- } else {
|
|
|
- final List<Token<?>> list = new ArrayList<Token<?>>(objects.length);
|
|
|
- for(int i = 0; i < objects.length; i++) {
|
|
|
- list.add(toToken((Map<?, ?>)objects[i]));
|
|
|
- }
|
|
|
- return list;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** Convert a JSON map to a List<Token<?>>. */
|
|
|
- public static List<Token<?>> toTokenList(final Map<?, ?> json) throws IOException {
|
|
|
- if (json == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName() + "s");
|
|
|
- return toTokenList((Object[])m.get(Token.class.getSimpleName()));
|
|
|
- }
|
|
|
-
|
|
|
/** Convert an exception object to a Json string. */
|
|
|
public static String toJsonString(final Exception e) {
|
|
|
final Map<String, Object> m = new TreeMap<String, Object>();
|
|
@@ -173,7 +120,12 @@ public class JsonUtil {
|
|
|
public static String toJsonString(final String key, final Object value) {
|
|
|
final Map<String, Object> m = new TreeMap<String, Object>();
|
|
|
m.put(key, value);
|
|
|
- return JSON.toString(m);
|
|
|
+ ObjectMapper mapper = new ObjectMapper();
|
|
|
+ try {
|
|
|
+ return mapper.writeValueAsString(m);
|
|
|
+ } catch (IOException ignored) {
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
/** Convert a FsPermission object to a string. */
|
|
@@ -233,7 +185,13 @@ public class JsonUtil {
|
|
|
m.put("fileId", status.getFileId());
|
|
|
m.put("childrenNum", status.getChildrenNum());
|
|
|
m.put("storagePolicy", status.getStoragePolicy());
|
|
|
- return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
|
|
|
+ ObjectMapper mapper = new ObjectMapper();
|
|
|
+ try {
|
|
|
+ return includeType ?
|
|
|
+ toJsonString(FileStatus.class, m) : mapper.writeValueAsString(m);
|
|
|
+ } catch (IOException ignored) {
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a HdfsFileStatus object. */
|
|
@@ -249,23 +207,21 @@ public class JsonUtil {
|
|
|
final byte[] symlink = type != PathType.SYMLINK? null
|
|
|
: DFSUtil.string2Bytes((String)m.get("symlink"));
|
|
|
|
|
|
- final long len = (Long) m.get("length");
|
|
|
+ final long len = ((Number) m.get("length")).longValue();
|
|
|
final String owner = (String) m.get("owner");
|
|
|
final String group = (String) m.get("group");
|
|
|
final FsPermission permission = toFsPermission((String) m.get("permission"),
|
|
|
(Boolean)m.get("aclBit"), (Boolean)m.get("encBit"));
|
|
|
- final long aTime = (Long) m.get("accessTime");
|
|
|
- final long mTime = (Long) m.get("modificationTime");
|
|
|
- final long blockSize = (Long) m.get("blockSize");
|
|
|
- final short replication = (short) (long) (Long) m.get("replication");
|
|
|
- final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
|
|
|
- : INodeId.GRANDFATHER_INODE_ID;
|
|
|
- Long childrenNumLong = (Long) m.get("childrenNum");
|
|
|
- final int childrenNum = (childrenNumLong == null) ? -1
|
|
|
- : childrenNumLong.intValue();
|
|
|
+ final long aTime = ((Number) m.get("accessTime")).longValue();
|
|
|
+ final long mTime = ((Number) m.get("modificationTime")).longValue();
|
|
|
+ final long blockSize = ((Number) m.get("blockSize")).longValue();
|
|
|
+ final short replication = ((Number) m.get("replication")).shortValue();
|
|
|
+ final long fileId = m.containsKey("fileId") ?
|
|
|
+ ((Number) m.get("fileId")).longValue() : INodeId.GRANDFATHER_INODE_ID;
|
|
|
+ final int childrenNum = getInt(m, "childrenNum", -1);
|
|
|
final byte storagePolicy = m.containsKey("storagePolicy") ?
|
|
|
- (byte) (long) (Long) m.get("storagePolicy") :
|
|
|
- BlockStoragePolicySuite.ID_UNSPECIFIED;
|
|
|
+ (byte) ((Number) m.get("storagePolicy")).longValue() :
|
|
|
+ BlockStoragePolicySuite.ID_UNSPECIFIED;
|
|
|
return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
|
|
|
blockSize, mTime, aTime, permission, owner, group, symlink,
|
|
|
DFSUtil.string2Bytes(localName), fileId, childrenNum, null, storagePolicy);
|
|
@@ -292,9 +248,10 @@ public class JsonUtil {
|
|
|
}
|
|
|
|
|
|
final String blockPoolId = (String)m.get("blockPoolId");
|
|
|
- final long blockId = (Long)m.get("blockId");
|
|
|
- final long numBytes = (Long)m.get("numBytes");
|
|
|
- final long generationStamp = (Long)m.get("generationStamp");
|
|
|
+ final long blockId = ((Number) m.get("blockId")).longValue();
|
|
|
+ final long numBytes = ((Number) m.get("numBytes")).longValue();
|
|
|
+ final long generationStamp =
|
|
|
+ ((Number) m.get("generationStamp")).longValue();
|
|
|
return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
|
|
|
}
|
|
|
|
|
@@ -335,7 +292,7 @@ public class JsonUtil {
|
|
|
if (value == null) {
|
|
|
return defaultValue;
|
|
|
}
|
|
|
- return (int) (long) (Long) value;
|
|
|
+ return ((Number) value).intValue();
|
|
|
}
|
|
|
|
|
|
private static long getLong(Map<?, ?> m, String key, final long defaultValue) {
|
|
@@ -343,7 +300,7 @@ public class JsonUtil {
|
|
|
if (value == null) {
|
|
|
return defaultValue;
|
|
|
}
|
|
|
- return (Long) value;
|
|
|
+ return ((Number) value).longValue();
|
|
|
}
|
|
|
|
|
|
private static String getString(Map<?, ?> m, String key,
|
|
@@ -355,6 +312,15 @@ public class JsonUtil {
|
|
|
return (String) value;
|
|
|
}
|
|
|
|
|
|
+ static List<?> getList(Map<?, ?> m, String key) {
|
|
|
+ Object list = m.get(key);
|
|
|
+ if (list instanceof List<?>) {
|
|
|
+ return (List<?>) list;
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/** Convert a Json map to an DatanodeInfo object. */
|
|
|
static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
|
|
|
throws IOException {
|
|
@@ -402,9 +368,9 @@ public class JsonUtil {
|
|
|
(String)m.get("hostName"),
|
|
|
(String)m.get("storageID"),
|
|
|
xferPort,
|
|
|
- (int)(long)(Long)m.get("infoPort"),
|
|
|
+ ((Number) m.get("infoPort")).intValue(),
|
|
|
getInt(m, "infoSecurePort", 0),
|
|
|
- (int)(long)(Long)m.get("ipcPort"),
|
|
|
+ ((Number) m.get("ipcPort")).intValue(),
|
|
|
|
|
|
getLong(m, "capacity", 0l),
|
|
|
getLong(m, "dfsUsed", 0l),
|
|
@@ -434,16 +400,17 @@ public class JsonUtil {
|
|
|
}
|
|
|
|
|
|
/** Convert an Object[] to a DatanodeInfo[]. */
|
|
|
- private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects)
|
|
|
+ private static DatanodeInfo[] toDatanodeInfoArray(final List<?> objects)
|
|
|
throws IOException {
|
|
|
if (objects == null) {
|
|
|
return null;
|
|
|
- } else if (objects.length == 0) {
|
|
|
+ } else if (objects.isEmpty()) {
|
|
|
return EMPTY_DATANODE_INFO_ARRAY;
|
|
|
} else {
|
|
|
- final DatanodeInfo[] array = new DatanodeInfo[objects.length];
|
|
|
- for(int i = 0; i < array.length; i++) {
|
|
|
- array[i] = toDatanodeInfo((Map<?, ?>) objects[i]);
|
|
|
+ final DatanodeInfo[] array = new DatanodeInfo[objects.size()];
|
|
|
+ int i = 0;
|
|
|
+ for (Object object : objects) {
|
|
|
+ array[i++] = toDatanodeInfo((Map<?, ?>) object);
|
|
|
}
|
|
|
return array;
|
|
|
}
|
|
@@ -474,11 +441,11 @@ public class JsonUtil {
|
|
|
|
|
|
final ExtendedBlock b = toExtendedBlock((Map<?, ?>)m.get("block"));
|
|
|
final DatanodeInfo[] locations = toDatanodeInfoArray(
|
|
|
- (Object[])m.get("locations"));
|
|
|
- final long startOffset = (Long)m.get("startOffset");
|
|
|
+ getList(m, "locations"));
|
|
|
+ final long startOffset = ((Number) m.get("startOffset")).longValue();
|
|
|
final boolean isCorrupt = (Boolean)m.get("isCorrupt");
|
|
|
final DatanodeInfo[] cachedLocations = toDatanodeInfoArray(
|
|
|
- (Object[])m.get("cachedLocations"));
|
|
|
+ getList(m, "cachedLocations"));
|
|
|
|
|
|
final LocatedBlock locatedblock = new LocatedBlock(b, locations,
|
|
|
null, null, startOffset, isCorrupt, cachedLocations);
|
|
@@ -502,17 +469,17 @@ public class JsonUtil {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Convert an Object[] to a List of LocatedBlock. */
|
|
|
- private static List<LocatedBlock> toLocatedBlockList(final Object[] objects
|
|
|
- ) throws IOException {
|
|
|
+ /** Convert an List of Object to a List of LocatedBlock. */
|
|
|
+ private static List<LocatedBlock> toLocatedBlockList(
|
|
|
+ final List<?> objects) throws IOException {
|
|
|
if (objects == null) {
|
|
|
return null;
|
|
|
- } else if (objects.length == 0) {
|
|
|
+ } else if (objects.isEmpty()) {
|
|
|
return Collections.emptyList();
|
|
|
} else {
|
|
|
- final List<LocatedBlock> list = new ArrayList<LocatedBlock>(objects.length);
|
|
|
- for(int i = 0; i < objects.length; i++) {
|
|
|
- list.add(toLocatedBlock((Map<?, ?>)objects[i]));
|
|
|
+ final List<LocatedBlock> list = new ArrayList<>(objects.size());
|
|
|
+ for (Object object : objects) {
|
|
|
+ list.add(toLocatedBlock((Map<?, ?>) object));
|
|
|
}
|
|
|
return list;
|
|
|
}
|
|
@@ -543,10 +510,10 @@ public class JsonUtil {
|
|
|
}
|
|
|
|
|
|
final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
|
|
|
- final long fileLength = (Long)m.get("fileLength");
|
|
|
+ final long fileLength = ((Number) m.get("fileLength")).longValue();
|
|
|
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
|
|
|
final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
|
|
|
- (Object[])m.get("locatedBlocks"));
|
|
|
+ getList(m, "locatedBlocks"));
|
|
|
final LocatedBlock lastLocatedBlock = toLocatedBlock(
|
|
|
(Map<?, ?>)m.get("lastLocatedBlock"));
|
|
|
final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
|
|
@@ -577,12 +544,12 @@ public class JsonUtil {
|
|
|
}
|
|
|
|
|
|
final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
|
|
|
- final long length = (Long)m.get("length");
|
|
|
- final long fileCount = (Long)m.get("fileCount");
|
|
|
- final long directoryCount = (Long)m.get("directoryCount");
|
|
|
- final long quota = (Long)m.get("quota");
|
|
|
- final long spaceConsumed = (Long)m.get("spaceConsumed");
|
|
|
- final long spaceQuota = (Long)m.get("spaceQuota");
|
|
|
+ final long length = ((Number) m.get("length")).longValue();
|
|
|
+ final long fileCount = ((Number) m.get("fileCount")).longValue();
|
|
|
+ final long directoryCount = ((Number) m.get("directoryCount")).longValue();
|
|
|
+ final long quota = ((Number) m.get("quota")).longValue();
|
|
|
+ final long spaceConsumed = ((Number) m.get("spaceConsumed")).longValue();
|
|
|
+ final long spaceQuota = ((Number) m.get("spaceQuota")).longValue();
|
|
|
|
|
|
return new ContentSummary(length, fileCount, directoryCount,
|
|
|
quota, spaceConsumed, spaceQuota);
|
|
@@ -610,7 +577,7 @@ public class JsonUtil {
|
|
|
|
|
|
final Map<?, ?> m = (Map<?, ?>)json.get(FileChecksum.class.getSimpleName());
|
|
|
final String algorithm = (String)m.get("algorithm");
|
|
|
- final int length = (int)(long)(Long)m.get("length");
|
|
|
+ final int length = ((Number) m.get("length")).intValue();
|
|
|
final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
|
|
|
|
|
|
final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
|
|
@@ -654,7 +621,13 @@ public class JsonUtil {
|
|
|
m.put("owner", status.getOwner());
|
|
|
m.put("group", status.getGroup());
|
|
|
m.put("stickyBit", status.isStickyBit());
|
|
|
- m.put("entries", status.getEntries());
|
|
|
+
|
|
|
+ final List<String> stringEntries = new ArrayList<>();
|
|
|
+ for (AclEntry entry : status.getEntries()) {
|
|
|
+ stringEntries.add(entry.toString());
|
|
|
+ }
|
|
|
+ m.put("entries", stringEntries);
|
|
|
+
|
|
|
FsPermission perm = status.getPermission();
|
|
|
if (perm != null) {
|
|
|
m.put("permission", toString(perm));
|
|
@@ -668,7 +641,13 @@ public class JsonUtil {
|
|
|
final Map<String, Map<String, Object>> finalMap =
|
|
|
new TreeMap<String, Map<String, Object>>();
|
|
|
finalMap.put(AclStatus.class.getSimpleName(), m);
|
|
|
- return JSON.toString(finalMap);
|
|
|
+
|
|
|
+ ObjectMapper mapper = new ObjectMapper();
|
|
|
+ try {
|
|
|
+ return mapper.writeValueAsString(finalMap);
|
|
|
+ } catch (IOException ignored) {
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a AclStatus object. */
|
|
@@ -689,11 +668,11 @@ public class JsonUtil {
|
|
|
(Boolean) m.get("aclBit"), (Boolean) m.get("encBit"));
|
|
|
aclStatusBuilder.setPermission(permission);
|
|
|
}
|
|
|
- final Object[] entries = (Object[]) m.get("entries");
|
|
|
+ final List<?> entries = (List<?>) m.get("entries");
|
|
|
|
|
|
List<AclEntry> aclEntryList = new ArrayList<AclEntry>();
|
|
|
- for (int i = 0; i < entries.length; i++) {
|
|
|
- AclEntry aclEntry = AclEntry.parseAclEntry((String) entries[i], true);
|
|
|
+ for (Object entry : entries) {
|
|
|
+ AclEntry aclEntry = AclEntry.parseAclEntry((String) entry, true);
|
|
|
aclEntryList.add(aclEntry);
|
|
|
}
|
|
|
aclStatusBuilder.addEntries(aclEntryList);
|
|
@@ -732,7 +711,8 @@ public class JsonUtil {
|
|
|
final XAttrCodec encoding) throws IOException {
|
|
|
final Map<String, Object> finalMap = new TreeMap<String, Object>();
|
|
|
finalMap.put("XAttrs", toJsonArray(xAttrs, encoding));
|
|
|
- return JSON.toString(finalMap);
|
|
|
+ ObjectMapper mapper = new ObjectMapper();
|
|
|
+ return mapper.writeValueAsString(finalMap);
|
|
|
}
|
|
|
|
|
|
public static String toJsonString(final List<XAttr> xAttrs)
|
|
@@ -741,10 +721,11 @@ public class JsonUtil {
|
|
|
for (XAttr xAttr : xAttrs) {
|
|
|
names.add(XAttrHelper.getPrefixName(xAttr));
|
|
|
}
|
|
|
- String ret = JSON.toString(names);
|
|
|
+ ObjectMapper mapper = new ObjectMapper();
|
|
|
+ String ret = mapper.writeValueAsString(names);
|
|
|
final Map<String, Object> finalMap = new TreeMap<String, Object>();
|
|
|
finalMap.put("XAttrNames", ret);
|
|
|
- return JSON.toString(finalMap);
|
|
|
+ return mapper.writeValueAsString(finalMap);
|
|
|
}
|
|
|
|
|
|
public static byte[] getXAttr(final Map<?, ?> json, final String name)
|
|
@@ -760,14 +741,13 @@ public class JsonUtil {
|
|
|
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public static Map<String, byte[]> toXAttrs(final Map<?, ?> json)
|
|
|
throws IOException {
|
|
|
if (json == null) {
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
- return toXAttrMap((Object[])json.get("XAttrs"));
|
|
|
+ return toXAttrMap(getList(json, "XAttrs"));
|
|
|
}
|
|
|
|
|
|
public static List<String> toXAttrNames(final Map<?, ?> json)
|
|
@@ -777,27 +757,27 @@ public class JsonUtil {
|
|
|
}
|
|
|
|
|
|
final String namesInJson = (String) json.get("XAttrNames");
|
|
|
- final Object[] xattrs = (Object[]) JSON.parse(namesInJson);
|
|
|
- final List<String> names = Lists.newArrayListWithCapacity(json.keySet()
|
|
|
- .size());
|
|
|
+ ObjectReader reader = new ObjectMapper().reader(List.class);
|
|
|
+ final List<Object> xattrs = reader.readValue(namesInJson);
|
|
|
+ final List<String> names =
|
|
|
+ Lists.newArrayListWithCapacity(json.keySet().size());
|
|
|
|
|
|
- for (int i = 0; i < xattrs.length; i++) {
|
|
|
- names.add((String) (xattrs[i]));
|
|
|
+ for (Object xattr : xattrs) {
|
|
|
+ names.add((String) xattr);
|
|
|
}
|
|
|
return names;
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
- private static Map<String, byte[]> toXAttrMap(final Object[] objects)
|
|
|
+
|
|
|
+ private static Map<String, byte[]> toXAttrMap(final List<?> objects)
|
|
|
throws IOException {
|
|
|
if (objects == null) {
|
|
|
return null;
|
|
|
- } else if (objects.length == 0) {
|
|
|
+ } else if (objects.isEmpty()) {
|
|
|
return Maps.newHashMap();
|
|
|
} else {
|
|
|
final Map<String, byte[]> xAttrs = Maps.newHashMap();
|
|
|
- for(int i = 0; i < objects.length; i++) {
|
|
|
- Map<?, ?> m = (Map<?, ?>) objects[i];
|
|
|
+ for (Object object : objects) {
|
|
|
+ Map<?, ?> m = (Map<?, ?>) object;
|
|
|
String name = (String) m.get("name");
|
|
|
String value = (String) m.get("value");
|
|
|
xAttrs.put(name, decodeXAttrValue(value));
|