|
@@ -42,42 +42,29 @@ import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
import org.mortbay.util.ajax.JSON;
|
|
import org.mortbay.util.ajax.JSON;
|
|
|
|
|
|
/** JSON Utilities */
|
|
/** JSON Utilities */
|
|
public class JsonUtil {
|
|
public class JsonUtil {
|
|
- private static class ThreadLocalMap extends ThreadLocal<Map<String, Object>> {
|
|
|
|
- @Override
|
|
|
|
- protected Map<String, Object> initialValue() {
|
|
|
|
- return new TreeMap<String, Object>();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public Map<String, Object> get() {
|
|
|
|
- final Map<String, Object> m = super.get();
|
|
|
|
- m.clear();
|
|
|
|
- return m;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static final ThreadLocalMap jsonMap = new ThreadLocalMap();
|
|
|
|
- private static final ThreadLocalMap tokenMap = new ThreadLocalMap();
|
|
|
|
- private static final ThreadLocalMap datanodeInfoMap = new ThreadLocalMap();
|
|
|
|
- private static final ThreadLocalMap extendedBlockMap = new ThreadLocalMap();
|
|
|
|
- private static final ThreadLocalMap locatedBlockMap = new ThreadLocalMap();
|
|
|
|
-
|
|
|
|
|
|
+ private static final Object[] EMPTY_OBJECT_ARRAY = {};
|
|
private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
|
|
private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
|
|
|
|
|
|
/** Convert a token object to a Json string. */
|
|
/** Convert a token object to a Json string. */
|
|
public static String toJsonString(final Token<? extends TokenIdentifier> token
|
|
public static String toJsonString(final Token<? extends TokenIdentifier> token
|
|
) throws IOException {
|
|
) throws IOException {
|
|
|
|
+ return toJsonString(Token.class, toJsonMap(token));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static Map<String, Object> toJsonMap(
|
|
|
|
+ final Token<? extends TokenIdentifier> token) throws IOException {
|
|
if (token == null) {
|
|
if (token == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = tokenMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("urlString", token.encodeToUrlString());
|
|
m.put("urlString", token.encodeToUrlString());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return m;
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a Token. */
|
|
/** Convert a Json map to a Token. */
|
|
@@ -96,46 +83,52 @@ public class JsonUtil {
|
|
/** Convert a Json map to a Token of DelegationTokenIdentifier. */
|
|
/** Convert a Json map to a Token of DelegationTokenIdentifier. */
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
public static Token<DelegationTokenIdentifier> toDelegationToken(
|
|
public static Token<DelegationTokenIdentifier> toDelegationToken(
|
|
- final Map<?, ?> m) throws IOException {
|
|
|
|
|
|
+ final Map<?, ?> json) throws IOException {
|
|
|
|
+ final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName());
|
|
return (Token<DelegationTokenIdentifier>)toToken(m);
|
|
return (Token<DelegationTokenIdentifier>)toToken(m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a Token of BlockTokenIdentifier. */
|
|
/** Convert a Json map to a Token of BlockTokenIdentifier. */
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
- public static Token<BlockTokenIdentifier> toBlockToken(
|
|
|
|
|
|
+ private static Token<BlockTokenIdentifier> toBlockToken(
|
|
final Map<?, ?> m) throws IOException {
|
|
final Map<?, ?> m) throws IOException {
|
|
return (Token<BlockTokenIdentifier>)toToken(m);
|
|
return (Token<BlockTokenIdentifier>)toToken(m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert an exception object to a Json string. */
|
|
/** Convert an exception object to a Json string. */
|
|
public static String toJsonString(final Exception e) {
|
|
public static String toJsonString(final Exception e) {
|
|
- final Map<String, Object> m = jsonMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("className", e.getClass().getName());
|
|
m.put("className", e.getClass().getName());
|
|
m.put("message", e.getMessage());
|
|
m.put("message", e.getMessage());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return toJsonString(RemoteException.class, m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a RemoteException. */
|
|
/** Convert a Json map to a RemoteException. */
|
|
- public static RemoteException toRemoteException(final Map<String, Object> m) {
|
|
|
|
|
|
+ public static RemoteException toRemoteException(final Map<?, ?> json) {
|
|
|
|
+ final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
|
|
final String className = (String)m.get("className");
|
|
final String className = (String)m.get("className");
|
|
final String message = (String)m.get("message");
|
|
final String message = (String)m.get("message");
|
|
return new RemoteException(className, message);
|
|
return new RemoteException(className, message);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private static String toJsonString(final Class<?> clazz, final Object value) {
|
|
|
|
+ return toJsonString(clazz.getSimpleName(), value);
|
|
|
|
+ }
|
|
|
|
+
|
|
/** Convert a key-value pair to a Json string. */
|
|
/** Convert a key-value pair to a Json string. */
|
|
- public static String toJsonString(final Object key, final Object value) {
|
|
|
|
- final Map<String, Object> m = jsonMap.get();
|
|
|
|
- m.put(key instanceof String ? (String) key : key.toString(), value);
|
|
|
|
|
|
+ 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);
|
|
return JSON.toString(m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a FsPermission object to a string. */
|
|
/** Convert a FsPermission object to a string. */
|
|
- public static String toString(final FsPermission permission) {
|
|
|
|
|
|
+ private static String toString(final FsPermission permission) {
|
|
return String.format("%o", permission.toShort());
|
|
return String.format("%o", permission.toShort());
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a string to a FsPermission object. */
|
|
/** Convert a string to a FsPermission object. */
|
|
- public static FsPermission toFsPermission(final String s) {
|
|
|
|
|
|
+ private static FsPermission toFsPermission(final String s) {
|
|
return new FsPermission(Short.parseShort(s, 8));
|
|
return new FsPermission(Short.parseShort(s, 8));
|
|
}
|
|
}
|
|
|
|
|
|
@@ -144,7 +137,7 @@ public class JsonUtil {
|
|
if (status == null) {
|
|
if (status == null) {
|
|
return null;
|
|
return null;
|
|
} else {
|
|
} else {
|
|
- final Map<String, Object> m = jsonMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("localName", status.getLocalName());
|
|
m.put("localName", status.getLocalName());
|
|
m.put("isDir", status.isDir());
|
|
m.put("isDir", status.isDir());
|
|
m.put("isSymlink", status.isSymlink());
|
|
m.put("isSymlink", status.isSymlink());
|
|
@@ -160,21 +153,17 @@ public class JsonUtil {
|
|
m.put("modificationTime", status.getModificationTime());
|
|
m.put("modificationTime", status.getModificationTime());
|
|
m.put("blockSize", status.getBlockSize());
|
|
m.put("blockSize", status.getBlockSize());
|
|
m.put("replication", status.getReplication());
|
|
m.put("replication", status.getReplication());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return toJsonString(HdfsFileStatus.class, m);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
- static Map<String, Object> parse(String jsonString) {
|
|
|
|
- return (Map<String, Object>) JSON.parse(jsonString);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/** Convert a Json map to a HdfsFileStatus object. */
|
|
/** Convert a Json map to a HdfsFileStatus object. */
|
|
- public static HdfsFileStatus toFileStatus(final Map<String, Object> m) {
|
|
|
|
- if (m == null) {
|
|
|
|
|
|
+ public static HdfsFileStatus toFileStatus(final Map<?, ?> json) {
|
|
|
|
+ if (json == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ final Map<?, ?> m = (Map<?, ?>)json.get(HdfsFileStatus.class.getSimpleName());
|
|
final String localName = (String) m.get("localName");
|
|
final String localName = (String) m.get("localName");
|
|
final boolean isDir = (Boolean) m.get("isDir");
|
|
final boolean isDir = (Boolean) m.get("isDir");
|
|
final boolean isSymlink = (Boolean) m.get("isSymlink");
|
|
final boolean isSymlink = (Boolean) m.get("isSymlink");
|
|
@@ -194,22 +183,22 @@ public class JsonUtil {
|
|
symlink, DFSUtil.string2Bytes(localName));
|
|
symlink, DFSUtil.string2Bytes(localName));
|
|
}
|
|
}
|
|
|
|
|
|
- /** Convert a LocatedBlock to a Json string. */
|
|
|
|
- public static String toJsonString(final ExtendedBlock extendedblock) {
|
|
|
|
|
|
+ /** Convert an ExtendedBlock to a Json map. */
|
|
|
|
+ private static Map<String, Object> toJsonMap(final ExtendedBlock extendedblock) {
|
|
if (extendedblock == null) {
|
|
if (extendedblock == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = extendedBlockMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("blockPoolId", extendedblock.getBlockPoolId());
|
|
m.put("blockPoolId", extendedblock.getBlockPoolId());
|
|
m.put("blockId", extendedblock.getBlockId());
|
|
m.put("blockId", extendedblock.getBlockId());
|
|
m.put("numBytes", extendedblock.getNumBytes());
|
|
m.put("numBytes", extendedblock.getNumBytes());
|
|
m.put("generationStamp", extendedblock.getGenerationStamp());
|
|
m.put("generationStamp", extendedblock.getGenerationStamp());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return m;
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to an ExtendedBlock object. */
|
|
/** Convert a Json map to an ExtendedBlock object. */
|
|
- public static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
|
|
|
|
|
|
+ private static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
|
|
if (m == null) {
|
|
if (m == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -221,13 +210,13 @@ public class JsonUtil {
|
|
return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
|
|
return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
|
|
}
|
|
}
|
|
|
|
|
|
- /** Convert a DatanodeInfo to a Json string. */
|
|
|
|
- public static String toJsonString(final DatanodeInfo datanodeinfo) {
|
|
|
|
|
|
+ /** Convert a DatanodeInfo to a Json map. */
|
|
|
|
+ private static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
|
|
if (datanodeinfo == null) {
|
|
if (datanodeinfo == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = datanodeInfoMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("name", datanodeinfo.getName());
|
|
m.put("name", datanodeinfo.getName());
|
|
m.put("storageID", datanodeinfo.getStorageID());
|
|
m.put("storageID", datanodeinfo.getStorageID());
|
|
m.put("infoPort", datanodeinfo.getInfoPort());
|
|
m.put("infoPort", datanodeinfo.getInfoPort());
|
|
@@ -243,11 +232,11 @@ public class JsonUtil {
|
|
m.put("networkLocation", datanodeinfo.getNetworkLocation());
|
|
m.put("networkLocation", datanodeinfo.getNetworkLocation());
|
|
m.put("hostName", datanodeinfo.getHostName());
|
|
m.put("hostName", datanodeinfo.getHostName());
|
|
m.put("adminState", datanodeinfo.getAdminState().name());
|
|
m.put("adminState", datanodeinfo.getAdminState().name());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return m;
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to an DatanodeInfo object. */
|
|
/** Convert a Json map to an DatanodeInfo object. */
|
|
- public static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
|
|
|
|
|
|
+ private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
|
|
if (m == null) {
|
|
if (m == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -269,25 +258,23 @@ public class JsonUtil {
|
|
AdminStates.valueOf((String)m.get("adminState")));
|
|
AdminStates.valueOf((String)m.get("adminState")));
|
|
}
|
|
}
|
|
|
|
|
|
- /** Convert a DatanodeInfo[] to a Json string. */
|
|
|
|
- public static String toJsonString(final DatanodeInfo[] array
|
|
|
|
- ) throws IOException {
|
|
|
|
|
|
+ /** Convert a DatanodeInfo[] to a Json array. */
|
|
|
|
+ private static Object[] toJsonArray(final DatanodeInfo[] array) {
|
|
if (array == null) {
|
|
if (array == null) {
|
|
return null;
|
|
return null;
|
|
} else if (array.length == 0) {
|
|
} else if (array.length == 0) {
|
|
- return "[]";
|
|
|
|
|
|
+ return EMPTY_OBJECT_ARRAY;
|
|
} else {
|
|
} else {
|
|
- final StringBuilder b = new StringBuilder().append('[').append(
|
|
|
|
- toJsonString(array[0]));
|
|
|
|
- for(int i = 1; i < array.length; i++) {
|
|
|
|
- b.append(", ").append(toJsonString(array[i]));
|
|
|
|
|
|
+ final Object[] a = new Object[array.length];
|
|
|
|
+ for(int i = 0; i < array.length; i++) {
|
|
|
|
+ a[i] = toJsonMap(array[i]);
|
|
}
|
|
}
|
|
- return b.append(']').toString();
|
|
|
|
|
|
+ return a;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert an Object[] to a DatanodeInfo[]. */
|
|
/** Convert an Object[] to a DatanodeInfo[]. */
|
|
- public static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
|
|
|
|
|
|
+ private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
|
|
if (objects == null) {
|
|
if (objects == null) {
|
|
return null;
|
|
return null;
|
|
} else if (objects.length == 0) {
|
|
} else if (objects.length == 0) {
|
|
@@ -295,66 +282,63 @@ public class JsonUtil {
|
|
} else {
|
|
} else {
|
|
final DatanodeInfo[] array = new DatanodeInfo[objects.length];
|
|
final DatanodeInfo[] array = new DatanodeInfo[objects.length];
|
|
for(int i = 0; i < array.length; i++) {
|
|
for(int i = 0; i < array.length; i++) {
|
|
- array[i] = (DatanodeInfo)toDatanodeInfo((Map<?, ?>) objects[i]);
|
|
|
|
|
|
+ array[i] = toDatanodeInfo((Map<?, ?>) objects[i]);
|
|
}
|
|
}
|
|
return array;
|
|
return array;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /** Convert a LocatedBlock to a Json string. */
|
|
|
|
- public static String toJsonString(final LocatedBlock locatedblock
|
|
|
|
|
|
+ /** Convert a LocatedBlock to a Json map. */
|
|
|
|
+ private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock
|
|
) throws IOException {
|
|
) throws IOException {
|
|
if (locatedblock == null) {
|
|
if (locatedblock == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = locatedBlockMap.get();
|
|
|
|
- m.put("blockToken", toJsonString(locatedblock.getBlockToken()));
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
|
|
+ m.put("blockToken", toJsonMap(locatedblock.getBlockToken()));
|
|
m.put("isCorrupt", locatedblock.isCorrupt());
|
|
m.put("isCorrupt", locatedblock.isCorrupt());
|
|
m.put("startOffset", locatedblock.getStartOffset());
|
|
m.put("startOffset", locatedblock.getStartOffset());
|
|
- m.put("block", toJsonString(locatedblock.getBlock()));
|
|
|
|
-
|
|
|
|
- m.put("locations", toJsonString(locatedblock.getLocations()));
|
|
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ m.put("block", toJsonMap(locatedblock.getBlock()));
|
|
|
|
+ m.put("locations", toJsonArray(locatedblock.getLocations()));
|
|
|
|
+ return m;
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to LocatedBlock. */
|
|
/** Convert a Json map to LocatedBlock. */
|
|
- public static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
|
|
|
|
|
|
+ private static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
|
|
if (m == null) {
|
|
if (m == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final ExtendedBlock b = toExtendedBlock((Map<?, ?>)JSON.parse((String)m.get("block")));
|
|
|
|
|
|
+ final ExtendedBlock b = toExtendedBlock((Map<?, ?>)m.get("block"));
|
|
final DatanodeInfo[] locations = toDatanodeInfoArray(
|
|
final DatanodeInfo[] locations = toDatanodeInfoArray(
|
|
- (Object[])JSON.parse((String)m.get("locations")));
|
|
|
|
|
|
+ (Object[])m.get("locations"));
|
|
final long startOffset = (Long)m.get("startOffset");
|
|
final long startOffset = (Long)m.get("startOffset");
|
|
final boolean isCorrupt = (Boolean)m.get("isCorrupt");
|
|
final boolean isCorrupt = (Boolean)m.get("isCorrupt");
|
|
|
|
|
|
final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt);
|
|
final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt);
|
|
- locatedblock.setBlockToken(toBlockToken((Map<?, ?>)JSON.parse((String)m.get("blockToken"))));
|
|
|
|
|
|
+ locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
|
|
return locatedblock;
|
|
return locatedblock;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Convert a LocatedBlock[] to a Json string. */
|
|
|
|
- public static String toJsonString(final List<LocatedBlock> array
|
|
|
|
|
|
+ /** Convert a LocatedBlock[] to a Json array. */
|
|
|
|
+ private static Object[] toJsonArray(final List<LocatedBlock> array
|
|
) throws IOException {
|
|
) throws IOException {
|
|
if (array == null) {
|
|
if (array == null) {
|
|
return null;
|
|
return null;
|
|
} else if (array.size() == 0) {
|
|
} else if (array.size() == 0) {
|
|
- return "[]";
|
|
|
|
|
|
+ return EMPTY_OBJECT_ARRAY;
|
|
} else {
|
|
} else {
|
|
- final StringBuilder b = new StringBuilder().append('[').append(
|
|
|
|
- toJsonString(array.get(0)));
|
|
|
|
- for(int i = 1; i < array.size(); i++) {
|
|
|
|
- b.append(",\n ").append(toJsonString(array.get(i)));
|
|
|
|
|
|
+ final Object[] a = new Object[array.size()];
|
|
|
|
+ for(int i = 0; i < array.size(); i++) {
|
|
|
|
+ a[i] = toJsonMap(array.get(0));
|
|
}
|
|
}
|
|
- return b.append(']').toString();
|
|
|
|
|
|
+ return a;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /** Convert an Object[] to a List of LocatedBlock.
|
|
|
|
- * @throws IOException */
|
|
|
|
- public static List<LocatedBlock> toLocatedBlockList(final Object[] objects
|
|
|
|
|
|
+ /** Convert an Object[] to a List of LocatedBlock. */
|
|
|
|
+ private static List<LocatedBlock> toLocatedBlockList(final Object[] objects
|
|
) throws IOException {
|
|
) throws IOException {
|
|
if (objects == null) {
|
|
if (objects == null) {
|
|
return null;
|
|
return null;
|
|
@@ -363,7 +347,7 @@ public class JsonUtil {
|
|
} else {
|
|
} else {
|
|
final List<LocatedBlock> list = new ArrayList<LocatedBlock>(objects.length);
|
|
final List<LocatedBlock> list = new ArrayList<LocatedBlock>(objects.length);
|
|
for(int i = 0; i < objects.length; i++) {
|
|
for(int i = 0; i < objects.length; i++) {
|
|
- list.add((LocatedBlock)toLocatedBlock((Map<?, ?>)objects[i]));
|
|
|
|
|
|
+ list.add(toLocatedBlock((Map<?, ?>)objects[i]));
|
|
}
|
|
}
|
|
return list;
|
|
return list;
|
|
}
|
|
}
|
|
@@ -376,58 +360,58 @@ public class JsonUtil {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = jsonMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("fileLength", locatedblocks.getFileLength());
|
|
m.put("fileLength", locatedblocks.getFileLength());
|
|
m.put("isUnderConstruction", locatedblocks.isUnderConstruction());
|
|
m.put("isUnderConstruction", locatedblocks.isUnderConstruction());
|
|
|
|
|
|
- m.put("locatedBlocks", toJsonString(locatedblocks.getLocatedBlocks()));
|
|
|
|
- m.put("lastLocatedBlock", toJsonString(locatedblocks.getLastLocatedBlock()));
|
|
|
|
|
|
+ m.put("locatedBlocks", toJsonArray(locatedblocks.getLocatedBlocks()));
|
|
|
|
+ m.put("lastLocatedBlock", toJsonMap(locatedblocks.getLastLocatedBlock()));
|
|
m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete());
|
|
m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return toJsonString(LocatedBlocks.class, m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to LocatedBlock. */
|
|
/** Convert a Json map to LocatedBlock. */
|
|
- public static LocatedBlocks toLocatedBlocks(final Map<String, Object> m
|
|
|
|
|
|
+ public static LocatedBlocks toLocatedBlocks(final Map<?, ?> json
|
|
) throws IOException {
|
|
) throws IOException {
|
|
- if (m == null) {
|
|
|
|
|
|
+ if (json == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
|
|
final long fileLength = (Long)m.get("fileLength");
|
|
final long fileLength = (Long)m.get("fileLength");
|
|
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
|
|
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
|
|
final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
|
|
final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
|
|
- (Object[])JSON.parse((String) m.get("locatedBlocks")));
|
|
|
|
|
|
+ (Object[])m.get("locatedBlocks"));
|
|
final LocatedBlock lastLocatedBlock = toLocatedBlock(
|
|
final LocatedBlock lastLocatedBlock = toLocatedBlock(
|
|
- (Map<?, ?>)JSON.parse((String)m.get("lastLocatedBlock")));
|
|
|
|
|
|
+ (Map<?, ?>)m.get("lastLocatedBlock"));
|
|
final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
|
|
final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
|
|
return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
|
|
return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
|
|
lastLocatedBlock, isLastBlockComplete);
|
|
lastLocatedBlock, isLastBlockComplete);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a ContentSummary to a Json string. */
|
|
/** Convert a ContentSummary to a Json string. */
|
|
- public static String toJsonString(final ContentSummary contentsummary
|
|
|
|
- ) throws IOException {
|
|
|
|
|
|
+ public static String toJsonString(final ContentSummary contentsummary) {
|
|
if (contentsummary == null) {
|
|
if (contentsummary == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = jsonMap.get();
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
m.put("length", contentsummary.getLength());
|
|
m.put("length", contentsummary.getLength());
|
|
m.put("fileCount", contentsummary.getFileCount());
|
|
m.put("fileCount", contentsummary.getFileCount());
|
|
m.put("directoryCount", contentsummary.getDirectoryCount());
|
|
m.put("directoryCount", contentsummary.getDirectoryCount());
|
|
m.put("quota", contentsummary.getQuota());
|
|
m.put("quota", contentsummary.getQuota());
|
|
m.put("spaceConsumed", contentsummary.getSpaceConsumed());
|
|
m.put("spaceConsumed", contentsummary.getSpaceConsumed());
|
|
m.put("spaceQuota", contentsummary.getSpaceQuota());
|
|
m.put("spaceQuota", contentsummary.getSpaceQuota());
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ return toJsonString(ContentSummary.class, m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a ContentSummary. */
|
|
/** Convert a Json map to a ContentSummary. */
|
|
- public static ContentSummary toContentSummary(final Map<String, Object> m
|
|
|
|
- ) throws IOException {
|
|
|
|
- if (m == null) {
|
|
|
|
|
|
+ public static ContentSummary toContentSummary(final Map<?, ?> json) {
|
|
|
|
+ if (json == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
|
|
final long length = (Long)m.get("length");
|
|
final long length = (Long)m.get("length");
|
|
final long fileCount = (Long)m.get("fileCount");
|
|
final long fileCount = (Long)m.get("fileCount");
|
|
final long directoryCount = (Long)m.get("directoryCount");
|
|
final long directoryCount = (Long)m.get("directoryCount");
|
|
@@ -440,36 +424,51 @@ public class JsonUtil {
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a MD5MD5CRC32FileChecksum to a Json string. */
|
|
/** Convert a MD5MD5CRC32FileChecksum to a Json string. */
|
|
- public static String toJsonString(final MD5MD5CRC32FileChecksum checksum
|
|
|
|
- ) throws IOException {
|
|
|
|
|
|
+ public static String toJsonString(final MD5MD5CRC32FileChecksum checksum) {
|
|
if (checksum == null) {
|
|
if (checksum == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final Map<String, Object> m = jsonMap.get();
|
|
|
|
- final byte[] bytes = checksum.getBytes();
|
|
|
|
- final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
|
|
|
|
- final int bytesPerCRC = in.readInt();
|
|
|
|
- final long crcPerBlock = in.readLong();
|
|
|
|
- final MD5Hash md5 = MD5Hash.read(in);
|
|
|
|
- m.put("bytesPerCRC", bytesPerCRC);
|
|
|
|
- m.put("crcPerBlock", crcPerBlock);
|
|
|
|
- m.put("md5", "" + md5);
|
|
|
|
- return JSON.toString(m);
|
|
|
|
|
|
+ final Map<String, Object> m = new TreeMap<String, Object>();
|
|
|
|
+ m.put("algorithm", checksum.getAlgorithmName());
|
|
|
|
+ m.put("length", checksum.getLength());
|
|
|
|
+ m.put("bytes", StringUtils.byteToHexString(checksum.getBytes()));
|
|
|
|
+ return toJsonString(MD5MD5CRC32FileChecksum.class, m);
|
|
}
|
|
}
|
|
|
|
|
|
/** Convert a Json map to a MD5MD5CRC32FileChecksum. */
|
|
/** Convert a Json map to a MD5MD5CRC32FileChecksum. */
|
|
public static MD5MD5CRC32FileChecksum toMD5MD5CRC32FileChecksum(
|
|
public static MD5MD5CRC32FileChecksum toMD5MD5CRC32FileChecksum(
|
|
- final Map<String, Object> m) throws IOException {
|
|
|
|
- if (m == null) {
|
|
|
|
|
|
+ final Map<?, ?> json) throws IOException {
|
|
|
|
+ if (json == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- final int bytesPerCRC = (int)(long)(Long)m.get("bytesPerCRC");
|
|
|
|
- final long crcPerBlock = (Long)m.get("crcPerBlock");
|
|
|
|
- final String md5 = (String)m.get("md5");
|
|
|
|
|
|
+ final Map<?, ?> m = (Map<?, ?>)json.get(
|
|
|
|
+ MD5MD5CRC32FileChecksum.class.getSimpleName());
|
|
|
|
+ final String algorithm = (String)m.get("algorithm");
|
|
|
|
+ final int length = (int)(long)(Long)m.get("length");
|
|
|
|
+ final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
|
|
|
|
+
|
|
|
|
+ final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
|
|
|
|
+ final int bytesPerCRC = in.readInt();
|
|
|
|
+ final long crcPerBlock = in.readLong();
|
|
|
|
+ final MD5Hash md5 = MD5Hash.read(in);
|
|
|
|
+ final MD5MD5CRC32FileChecksum checksum = new MD5MD5CRC32FileChecksum(
|
|
|
|
+ bytesPerCRC, crcPerBlock, md5);
|
|
|
|
+
|
|
|
|
+ //check algorithm name
|
|
|
|
+ final String alg = "MD5-of-" + crcPerBlock + "MD5-of-" + bytesPerCRC + "CRC32";
|
|
|
|
+ if (!alg.equals(algorithm)) {
|
|
|
|
+ throw new IOException("Algorithm not matched: algorithm=" + algorithm
|
|
|
|
+ + ", crcPerBlock=" + crcPerBlock
|
|
|
|
+ + ", bytesPerCRC=" + bytesPerCRC);
|
|
|
|
+ }
|
|
|
|
+ //check length
|
|
|
|
+ if (length != checksum.getLength()) {
|
|
|
|
+ throw new IOException("Length not matched: length=" + length
|
|
|
|
+ + ", checksum.getLength()=" + checksum.getLength());
|
|
|
|
+ }
|
|
|
|
|
|
- return new MD5MD5CRC32FileChecksum(bytesPerCRC, crcPerBlock,
|
|
|
|
- new MD5Hash(md5));
|
|
|
|
|
|
+ return checksum;
|
|
}
|
|
}
|
|
}
|
|
}
|