1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 package org.apache.hadoop.hdfs.web;
19 
20 import org.apache.hadoop.fs.*;
21 import org.apache.hadoop.fs.permission.AclEntry;
22 import org.apache.hadoop.fs.permission.AclStatus;
23 import org.apache.hadoop.fs.permission.FsPermission;
24 import org.apache.hadoop.hdfs.DFSUtil;
25 import org.apache.hadoop.hdfs.XAttrHelper;
26 import org.apache.hadoop.hdfs.protocol.*;
27 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
28 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
29 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
30 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
31 import org.apache.hadoop.hdfs.server.namenode.INodeId;
32 import org.apache.hadoop.ipc.RemoteException;
33 import org.apache.hadoop.security.token.Token;
34 import org.apache.hadoop.security.token.TokenIdentifier;
35 import org.apache.hadoop.util.DataChecksum;
36 import org.apache.hadoop.util.StringUtils;
37 import org.codehaus.jackson.map.ObjectMapper;
38 import org.codehaus.jackson.map.ObjectReader;
39 
40 import com.google.common.collect.Lists;
41 import com.google.common.collect.Maps;
42 
43 import java.io.ByteArrayInputStream;
44 import java.io.DataInputStream;
45 import java.io.IOException;
46 import java.util.*;
47 
48 /** JSON Utilities */
49 public class JsonUtil {
50   private static final Object[] EMPTY_OBJECT_ARRAY = {};
51   private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
52 
53   /** Convert a token object to a Json string. */
toJsonString(final Token<? extends TokenIdentifier> token )54   public static String toJsonString(final Token<? extends TokenIdentifier> token
55       ) throws IOException {
56     return toJsonString(Token.class, toJsonMap(token));
57   }
58 
toJsonMap( final Token<? extends TokenIdentifier> token)59   private static Map<String, Object> toJsonMap(
60       final Token<? extends TokenIdentifier> token) throws IOException {
61     if (token == null) {
62       return null;
63     }
64 
65     final Map<String, Object> m = new TreeMap<String, Object>();
66     m.put("urlString", token.encodeToUrlString());
67     return m;
68   }
69 
70   /** Convert a Json map to a Token. */
toToken( final Map<?, ?> m)71   public static Token<? extends TokenIdentifier> toToken(
72       final Map<?, ?> m) throws IOException {
73     if (m == null) {
74       return null;
75     }
76 
77     final Token<DelegationTokenIdentifier> token
78         = new Token<DelegationTokenIdentifier>();
79     token.decodeFromUrlString((String)m.get("urlString"));
80     return token;
81   }
82 
83   /** Convert a Json map to a Token of DelegationTokenIdentifier. */
84   @SuppressWarnings("unchecked")
toDelegationToken( final Map<?, ?> json)85   public static Token<DelegationTokenIdentifier> toDelegationToken(
86       final Map<?, ?> json) throws IOException {
87     final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName());
88     return (Token<DelegationTokenIdentifier>)toToken(m);
89   }
90 
91   /** Convert a Json map to a Token of BlockTokenIdentifier. */
92   @SuppressWarnings("unchecked")
toBlockToken( final Map<?, ?> m)93   private static Token<BlockTokenIdentifier> toBlockToken(
94       final Map<?, ?> m) throws IOException {
95     return (Token<BlockTokenIdentifier>)toToken(m);
96   }
97 
98   /** Convert an exception object to a Json string. */
toJsonString(final Exception e)99   public static String toJsonString(final Exception e) {
100     final Map<String, Object> m = new TreeMap<String, Object>();
101     m.put("exception", e.getClass().getSimpleName());
102     m.put("message", e.getMessage());
103     m.put("javaClassName", e.getClass().getName());
104     return toJsonString(RemoteException.class, m);
105   }
106 
107   /** Convert a Json map to a RemoteException. */
toRemoteException(final Map<?, ?> json)108   public static RemoteException toRemoteException(final Map<?, ?> json) {
109     final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
110     final String message = (String)m.get("message");
111     final String javaClassName = (String)m.get("javaClassName");
112     return new RemoteException(javaClassName, message);
113   }
114 
toJsonString(final Class<?> clazz, final Object value)115   private static String toJsonString(final Class<?> clazz, final Object value) {
116     return toJsonString(clazz.getSimpleName(), value);
117   }
118 
119   /** Convert a key-value pair to a Json string. */
toJsonString(final String key, final Object value)120   public static String toJsonString(final String key, final Object value) {
121     final Map<String, Object> m = new TreeMap<String, Object>();
122     m.put(key, value);
123     ObjectMapper mapper = new ObjectMapper();
124     try {
125       return mapper.writeValueAsString(m);
126     } catch (IOException ignored) {
127     }
128     return null;
129   }
130 
131   /** Convert a FsPermission object to a string. */
toString(final FsPermission permission)132   private static String toString(final FsPermission permission) {
133     return String.format("%o", permission.toShort());
134   }
135 
136   /** Convert a string to a FsPermission object. */
toFsPermission(final String s, Boolean aclBit, Boolean encBit)137   private static FsPermission toFsPermission(final String s, Boolean aclBit,
138       Boolean encBit) {
139     FsPermission perm = new FsPermission(Short.parseShort(s, 8));
140     final boolean aBit = (aclBit != null) ? aclBit : false;
141     final boolean eBit = (encBit != null) ? encBit : false;
142     if (aBit || eBit) {
143       return new FsPermissionExtension(perm, aBit, eBit);
144     } else {
145       return perm;
146     }
147   }
148 
149   static enum PathType {
150     FILE, DIRECTORY, SYMLINK;
151 
valueOf(HdfsFileStatus status)152     static PathType valueOf(HdfsFileStatus status) {
153       return status.isDir()? DIRECTORY: status.isSymlink()? SYMLINK: FILE;
154     }
155   }
156 
157   /** Convert a HdfsFileStatus object to a Json string. */
toJsonString(final HdfsFileStatus status, boolean includeType)158   public static String toJsonString(final HdfsFileStatus status,
159       boolean includeType) {
160     if (status == null) {
161       return null;
162     }
163     final Map<String, Object> m = new TreeMap<String, Object>();
164     m.put("pathSuffix", status.getLocalName());
165     m.put("type", PathType.valueOf(status));
166     if (status.isSymlink()) {
167       m.put("symlink", status.getSymlink());
168     }
169 
170     m.put("length", status.getLen());
171     m.put("owner", status.getOwner());
172     m.put("group", status.getGroup());
173     FsPermission perm = status.getPermission();
174     m.put("permission", toString(perm));
175     if (perm.getAclBit()) {
176       m.put("aclBit", true);
177     }
178     if (perm.getEncryptedBit()) {
179       m.put("encBit", true);
180     }
181     m.put("accessTime", status.getAccessTime());
182     m.put("modificationTime", status.getModificationTime());
183     m.put("blockSize", status.getBlockSize());
184     m.put("replication", status.getReplication());
185     m.put("fileId", status.getFileId());
186     m.put("childrenNum", status.getChildrenNum());
187     m.put("storagePolicy", status.getStoragePolicy());
188     ObjectMapper mapper = new ObjectMapper();
189     try {
190       return includeType ?
191           toJsonString(FileStatus.class, m) : mapper.writeValueAsString(m);
192     } catch (IOException ignored) {
193     }
194     return null;
195   }
196 
197   /** Convert a Json map to a HdfsFileStatus object. */
toFileStatus(final Map<?, ?> json, boolean includesType)198   public static HdfsFileStatus toFileStatus(final Map<?, ?> json, boolean includesType) {
199     if (json == null) {
200       return null;
201     }
202 
203     final Map<?, ?> m = includesType ?
204         (Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json;
205     final String localName = (String) m.get("pathSuffix");
206     final PathType type = PathType.valueOf((String) m.get("type"));
207     final byte[] symlink = type != PathType.SYMLINK? null
208         : DFSUtil.string2Bytes((String)m.get("symlink"));
209 
210     final long len = ((Number) m.get("length")).longValue();
211     final String owner = (String) m.get("owner");
212     final String group = (String) m.get("group");
213     final FsPermission permission = toFsPermission((String) m.get("permission"),
214       (Boolean)m.get("aclBit"), (Boolean)m.get("encBit"));
215     final long aTime = ((Number) m.get("accessTime")).longValue();
216     final long mTime = ((Number) m.get("modificationTime")).longValue();
217     final long blockSize = ((Number) m.get("blockSize")).longValue();
218     final short replication = ((Number) m.get("replication")).shortValue();
219     final long fileId = m.containsKey("fileId") ?
220         ((Number) m.get("fileId")).longValue() : INodeId.GRANDFATHER_INODE_ID;
221     final int childrenNum = getInt(m, "childrenNum", -1);
222     final byte storagePolicy = m.containsKey("storagePolicy") ?
223         (byte) ((Number) m.get("storagePolicy")).longValue() :
224         BlockStoragePolicySuite.ID_UNSPECIFIED;
225     return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
226         blockSize, mTime, aTime, permission, owner, group, symlink,
227         DFSUtil.string2Bytes(localName), fileId, childrenNum, null, storagePolicy);
228   }
229 
230   /** Convert an ExtendedBlock to a Json map. */
toJsonMap(final ExtendedBlock extendedblock)231   private static Map<String, Object> toJsonMap(final ExtendedBlock extendedblock) {
232     if (extendedblock == null) {
233       return null;
234     }
235 
236     final Map<String, Object> m = new TreeMap<String, Object>();
237     m.put("blockPoolId", extendedblock.getBlockPoolId());
238     m.put("blockId", extendedblock.getBlockId());
239     m.put("numBytes", extendedblock.getNumBytes());
240     m.put("generationStamp", extendedblock.getGenerationStamp());
241     return m;
242   }
243 
244   /** Convert a Json map to an ExtendedBlock object. */
toExtendedBlock(final Map<?, ?> m)245   private static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
246     if (m == null) {
247       return null;
248     }
249 
250     final String blockPoolId = (String)m.get("blockPoolId");
251     final long blockId = ((Number) m.get("blockId")).longValue();
252     final long numBytes = ((Number) m.get("numBytes")).longValue();
253     final long generationStamp =
254         ((Number) m.get("generationStamp")).longValue();
255     return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
256   }
257 
258   /** Convert a DatanodeInfo to a Json map. */
toJsonMap(final DatanodeInfo datanodeinfo)259   static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
260     if (datanodeinfo == null) {
261       return null;
262     }
263 
264     // TODO: Fix storageID
265     final Map<String, Object> m = new TreeMap<String, Object>();
266     m.put("ipAddr", datanodeinfo.getIpAddr());
267     // 'name' is equivalent to ipAddr:xferPort. Older clients (1.x, 0.23.x)
268     // expects this instead of the two fields.
269     m.put("name", datanodeinfo.getXferAddr());
270     m.put("hostName", datanodeinfo.getHostName());
271     m.put("storageID", datanodeinfo.getDatanodeUuid());
272     m.put("xferPort", datanodeinfo.getXferPort());
273     m.put("infoPort", datanodeinfo.getInfoPort());
274     m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
275     m.put("ipcPort", datanodeinfo.getIpcPort());
276 
277     m.put("capacity", datanodeinfo.getCapacity());
278     m.put("dfsUsed", datanodeinfo.getDfsUsed());
279     m.put("remaining", datanodeinfo.getRemaining());
280     m.put("blockPoolUsed", datanodeinfo.getBlockPoolUsed());
281     m.put("cacheCapacity", datanodeinfo.getCacheCapacity());
282     m.put("cacheUsed", datanodeinfo.getCacheUsed());
283     m.put("lastUpdate", datanodeinfo.getLastUpdate());
284     m.put("lastUpdateMonotonic", datanodeinfo.getLastUpdateMonotonic());
285     m.put("xceiverCount", datanodeinfo.getXceiverCount());
286     m.put("networkLocation", datanodeinfo.getNetworkLocation());
287     m.put("adminState", datanodeinfo.getAdminState().name());
288     return m;
289   }
290 
getInt(Map<?, ?> m, String key, final int defaultValue)291   private static int getInt(Map<?, ?> m, String key, final int defaultValue) {
292     Object value = m.get(key);
293     if (value == null) {
294       return defaultValue;
295     }
296     return ((Number) value).intValue();
297   }
298 
getLong(Map<?, ?> m, String key, final long defaultValue)299   private static long getLong(Map<?, ?> m, String key, final long defaultValue) {
300     Object value = m.get(key);
301     if (value == null) {
302       return defaultValue;
303     }
304     return ((Number) value).longValue();
305   }
306 
getString(Map<?, ?> m, String key, final String defaultValue)307   private static String getString(Map<?, ?> m, String key,
308       final String defaultValue) {
309     Object value = m.get(key);
310     if (value == null) {
311       return defaultValue;
312     }
313     return (String) value;
314   }
315 
getList(Map<?, ?> m, String key)316   static List<?> getList(Map<?, ?> m, String key) {
317     Object list = m.get(key);
318     if (list instanceof List<?>) {
319       return (List<?>) list;
320     } else {
321       return null;
322     }
323   }
324 
325   /** Convert a Json map to an DatanodeInfo object. */
toDatanodeInfo(final Map<?, ?> m)326   static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
327       throws IOException {
328     if (m == null) {
329       return null;
330     }
331 
332     // ipAddr and xferPort are the critical fields for accessing data.
333     // If any one of the two is missing, an exception needs to be thrown.
334 
335     // Handle the case of old servers (1.x, 0.23.x) sending 'name' instead
336     // of ipAddr and xferPort.
337     int xferPort = getInt(m, "xferPort", -1);
338     Object tmpValue = m.get("ipAddr");
339     String ipAddr = (tmpValue == null) ? null : (String)tmpValue;
340     if (ipAddr == null) {
341       tmpValue = m.get("name");
342       if (tmpValue != null) {
343         String name = (String)tmpValue;
344         int colonIdx = name.indexOf(':');
345         if (colonIdx > 0) {
346           ipAddr = name.substring(0, colonIdx);
347           xferPort = Integer.parseInt(name.substring(colonIdx +1));
348         } else {
349           throw new IOException(
350               "Invalid value in server response: name=[" + name + "]");
351         }
352       } else {
353         throw new IOException(
354             "Missing both 'ipAddr' and 'name' in server response.");
355       }
356       // ipAddr is non-null & non-empty string at this point.
357     }
358 
359     // Check the validity of xferPort.
360     if (xferPort == -1) {
361       throw new IOException(
362           "Invalid or missing 'xferPort' in server response.");
363     }
364 
365     // TODO: Fix storageID
366     return new DatanodeInfo(
367         ipAddr,
368         (String)m.get("hostName"),
369         (String)m.get("storageID"),
370         xferPort,
371         ((Number) m.get("infoPort")).intValue(),
372         getInt(m, "infoSecurePort", 0),
373         ((Number) m.get("ipcPort")).intValue(),
374 
375         getLong(m, "capacity", 0l),
376         getLong(m, "dfsUsed", 0l),
377         getLong(m, "remaining", 0l),
378         getLong(m, "blockPoolUsed", 0l),
379         getLong(m, "cacheCapacity", 0l),
380         getLong(m, "cacheUsed", 0l),
381         getLong(m, "lastUpdate", 0l),
382         getLong(m, "lastUpdateMonotonic", 0l),
383         getInt(m, "xceiverCount", 0),
384         getString(m, "networkLocation", ""),
385         AdminStates.valueOf(getString(m, "adminState", "NORMAL")));
386   }
387 
388   /** Convert a DatanodeInfo[] to a Json array. */
toJsonArray(final DatanodeInfo[] array)389   private static Object[] toJsonArray(final DatanodeInfo[] array) {
390     if (array == null) {
391       return null;
392     } else if (array.length == 0) {
393       return EMPTY_OBJECT_ARRAY;
394     } else {
395       final Object[] a = new Object[array.length];
396       for(int i = 0; i < array.length; i++) {
397         a[i] = toJsonMap(array[i]);
398       }
399       return a;
400     }
401   }
402 
403   /** Convert an Object[] to a DatanodeInfo[]. */
toDatanodeInfoArray(final List<?> objects)404   private static DatanodeInfo[] toDatanodeInfoArray(final List<?> objects)
405       throws IOException {
406     if (objects == null) {
407       return null;
408     } else if (objects.isEmpty()) {
409       return EMPTY_DATANODE_INFO_ARRAY;
410     } else {
411       final DatanodeInfo[] array = new DatanodeInfo[objects.size()];
412       int i = 0;
413       for (Object object : objects) {
414         array[i++] = toDatanodeInfo((Map<?, ?>) object);
415       }
416       return array;
417     }
418   }
419 
420   /** Convert a LocatedBlock to a Json map. */
toJsonMap(final LocatedBlock locatedblock )421   private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock
422       ) throws IOException {
423     if (locatedblock == null) {
424       return null;
425     }
426 
427     final Map<String, Object> m = new TreeMap<String, Object>();
428     m.put("blockToken", toJsonMap(locatedblock.getBlockToken()));
429     m.put("isCorrupt", locatedblock.isCorrupt());
430     m.put("startOffset", locatedblock.getStartOffset());
431     m.put("block", toJsonMap(locatedblock.getBlock()));
432     m.put("locations", toJsonArray(locatedblock.getLocations()));
433     m.put("cachedLocations", toJsonArray(locatedblock.getCachedLocations()));
434     return m;
435   }
436 
437   /** Convert a Json map to LocatedBlock. */
toLocatedBlock(final Map<?, ?> m)438   private static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
439     if (m == null) {
440       return null;
441     }
442 
443     final ExtendedBlock b = toExtendedBlock((Map<?, ?>)m.get("block"));
444     final DatanodeInfo[] locations = toDatanodeInfoArray(
445         getList(m, "locations"));
446     final long startOffset = ((Number) m.get("startOffset")).longValue();
447     final boolean isCorrupt = (Boolean)m.get("isCorrupt");
448     final DatanodeInfo[] cachedLocations = toDatanodeInfoArray(
449         getList(m, "cachedLocations"));
450 
451     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
452         null, null, startOffset, isCorrupt, cachedLocations);
453     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
454     return locatedblock;
455   }
456 
457   /** Convert a LocatedBlock[] to a Json array. */
toJsonArray(final List<LocatedBlock> array )458   private static Object[] toJsonArray(final List<LocatedBlock> array
459       ) throws IOException {
460     if (array == null) {
461       return null;
462     } else if (array.size() == 0) {
463       return EMPTY_OBJECT_ARRAY;
464     } else {
465       final Object[] a = new Object[array.size()];
466       for(int i = 0; i < array.size(); i++) {
467         a[i] = toJsonMap(array.get(i));
468       }
469       return a;
470     }
471   }
472 
473   /** Convert an List of Object to a List of LocatedBlock. */
toLocatedBlockList( final List<?> objects)474   private static List<LocatedBlock> toLocatedBlockList(
475       final List<?> objects) throws IOException {
476     if (objects == null) {
477       return null;
478     } else if (objects.isEmpty()) {
479       return Collections.emptyList();
480     } else {
481       final List<LocatedBlock> list = new ArrayList<>(objects.size());
482       for (Object object : objects) {
483         list.add(toLocatedBlock((Map<?, ?>) object));
484       }
485       return list;
486     }
487   }
488 
489   /** Convert LocatedBlocks to a Json string. */
toJsonString(final LocatedBlocks locatedblocks )490   public static String toJsonString(final LocatedBlocks locatedblocks
491       ) throws IOException {
492     if (locatedblocks == null) {
493       return null;
494     }
495 
496     final Map<String, Object> m = new TreeMap<String, Object>();
497     m.put("fileLength", locatedblocks.getFileLength());
498     m.put("isUnderConstruction", locatedblocks.isUnderConstruction());
499 
500     m.put("locatedBlocks", toJsonArray(locatedblocks.getLocatedBlocks()));
501     m.put("lastLocatedBlock", toJsonMap(locatedblocks.getLastLocatedBlock()));
502     m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete());
503     return toJsonString(LocatedBlocks.class, m);
504   }
505 
506   /** Convert a Json map to LocatedBlock. */
toLocatedBlocks(final Map<?, ?> json )507   public static LocatedBlocks toLocatedBlocks(final Map<?, ?> json
508       ) throws IOException {
509     if (json == null) {
510       return null;
511     }
512 
513     final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
514     final long fileLength = ((Number) m.get("fileLength")).longValue();
515     final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
516     final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
517         getList(m, "locatedBlocks"));
518     final LocatedBlock lastLocatedBlock = toLocatedBlock(
519         (Map<?, ?>)m.get("lastLocatedBlock"));
520     final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
521     return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
522         lastLocatedBlock, isLastBlockComplete, null);
523   }
524 
525   /** Convert a ContentSummary to a Json string. */
toJsonString(final ContentSummary contentsummary)526   public static String toJsonString(final ContentSummary contentsummary) {
527     if (contentsummary == null) {
528       return null;
529     }
530 
531     final Map<String, Object> m = new TreeMap<String, Object>();
532     m.put("length", contentsummary.getLength());
533     m.put("fileCount", contentsummary.getFileCount());
534     m.put("directoryCount", contentsummary.getDirectoryCount());
535     m.put("quota", contentsummary.getQuota());
536     m.put("spaceConsumed", contentsummary.getSpaceConsumed());
537     m.put("spaceQuota", contentsummary.getSpaceQuota());
538     return toJsonString(ContentSummary.class, m);
539   }
540 
541   /** Convert a Json map to a ContentSummary. */
toContentSummary(final Map<?, ?> json)542   public static ContentSummary toContentSummary(final Map<?, ?> json) {
543     if (json == null) {
544       return null;
545     }
546 
547     final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
548     final long length = ((Number) m.get("length")).longValue();
549     final long fileCount = ((Number) m.get("fileCount")).longValue();
550     final long directoryCount = ((Number) m.get("directoryCount")).longValue();
551     final long quota = ((Number) m.get("quota")).longValue();
552     final long spaceConsumed = ((Number) m.get("spaceConsumed")).longValue();
553     final long spaceQuota = ((Number) m.get("spaceQuota")).longValue();
554 
555     return new ContentSummary.Builder().length(length).fileCount(fileCount).
556         directoryCount(directoryCount).quota(quota).spaceConsumed(spaceConsumed).
557         spaceQuota(spaceQuota).build();
558   }
559 
560   /** Convert a MD5MD5CRC32FileChecksum to a Json string. */
toJsonString(final MD5MD5CRC32FileChecksum checksum)561   public static String toJsonString(final MD5MD5CRC32FileChecksum checksum) {
562     if (checksum == null) {
563       return null;
564     }
565 
566     final Map<String, Object> m = new TreeMap<String, Object>();
567     m.put("algorithm", checksum.getAlgorithmName());
568     m.put("length", checksum.getLength());
569     m.put("bytes", StringUtils.byteToHexString(checksum.getBytes()));
570     return toJsonString(FileChecksum.class, m);
571   }
572 
573   /** Convert a Json map to a MD5MD5CRC32FileChecksum. */
toMD5MD5CRC32FileChecksum( final Map<?, ?> json)574   public static MD5MD5CRC32FileChecksum toMD5MD5CRC32FileChecksum(
575       final Map<?, ?> json) throws IOException {
576     if (json == null) {
577       return null;
578     }
579 
580     final Map<?, ?> m = (Map<?, ?>)json.get(FileChecksum.class.getSimpleName());
581     final String algorithm = (String)m.get("algorithm");
582     final int length = ((Number) m.get("length")).intValue();
583     final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
584 
585     final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
586     final DataChecksum.Type crcType =
587         MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm);
588     final MD5MD5CRC32FileChecksum checksum;
589 
590     // Recreate what DFSClient would have returned.
591     switch(crcType) {
592       case CRC32:
593         checksum = new MD5MD5CRC32GzipFileChecksum();
594         break;
595       case CRC32C:
596         checksum = new MD5MD5CRC32CastagnoliFileChecksum();
597         break;
598       default:
599         throw new IOException("Unknown algorithm: " + algorithm);
600     }
601     checksum.readFields(in);
602 
603     //check algorithm name
604     if (!checksum.getAlgorithmName().equals(algorithm)) {
605       throw new IOException("Algorithm not matched. Expected " + algorithm
606           + ", Received " + checksum.getAlgorithmName());
607     }
608     //check length
609     if (length != checksum.getLength()) {
610       throw new IOException("Length not matched: length=" + length
611           + ", checksum.getLength()=" + checksum.getLength());
612     }
613 
614     return checksum;
615   }
616   /** Convert a AclStatus object to a Json string. */
toJsonString(final AclStatus status)617   public static String toJsonString(final AclStatus status) {
618     if (status == null) {
619       return null;
620     }
621 
622     final Map<String, Object> m = new TreeMap<String, Object>();
623     m.put("owner", status.getOwner());
624     m.put("group", status.getGroup());
625     m.put("stickyBit", status.isStickyBit());
626 
627     final List<String> stringEntries = new ArrayList<>();
628     for (AclEntry entry : status.getEntries()) {
629       stringEntries.add(entry.toString());
630     }
631     m.put("entries", stringEntries);
632 
633     FsPermission perm = status.getPermission();
634     if (perm != null) {
635       m.put("permission", toString(perm));
636       if (perm.getAclBit()) {
637         m.put("aclBit", true);
638       }
639       if (perm.getEncryptedBit()) {
640         m.put("encBit", true);
641       }
642     }
643     final Map<String, Map<String, Object>> finalMap =
644         new TreeMap<String, Map<String, Object>>();
645     finalMap.put(AclStatus.class.getSimpleName(), m);
646 
647     ObjectMapper mapper = new ObjectMapper();
648     try {
649       return mapper.writeValueAsString(finalMap);
650     } catch (IOException ignored) {
651     }
652     return null;
653   }
654 
655   /** Convert a Json map to a AclStatus object. */
toAclStatus(final Map<?, ?> json)656   public static AclStatus toAclStatus(final Map<?, ?> json) {
657     if (json == null) {
658       return null;
659     }
660 
661     final Map<?, ?> m = (Map<?, ?>) json.get(AclStatus.class.getSimpleName());
662 
663     AclStatus.Builder aclStatusBuilder = new AclStatus.Builder();
664     aclStatusBuilder.owner((String) m.get("owner"));
665     aclStatusBuilder.group((String) m.get("group"));
666     aclStatusBuilder.stickyBit((Boolean) m.get("stickyBit"));
667     String permString = (String) m.get("permission");
668     if (permString != null) {
669       final FsPermission permission = toFsPermission(permString,
670           (Boolean) m.get("aclBit"), (Boolean) m.get("encBit"));
671       aclStatusBuilder.setPermission(permission);
672     }
673     final List<?> entries = (List<?>) m.get("entries");
674 
675     List<AclEntry> aclEntryList = new ArrayList<AclEntry>();
676     for (Object entry : entries) {
677       AclEntry aclEntry = AclEntry.parseAclEntry((String) entry, true);
678       aclEntryList.add(aclEntry);
679     }
680     aclStatusBuilder.addEntries(aclEntryList);
681     return aclStatusBuilder.build();
682   }
683 
toJsonMap(final XAttr xAttr, final XAttrCodec encoding)684   private static Map<String, Object> toJsonMap(final XAttr xAttr,
685       final XAttrCodec encoding) throws IOException {
686     if (xAttr == null) {
687       return null;
688     }
689 
690     final Map<String, Object> m = new TreeMap<String, Object>();
691     m.put("name", XAttrHelper.getPrefixName(xAttr));
692     m.put("value", xAttr.getValue() != null ?
693         XAttrCodec.encodeValue(xAttr.getValue(), encoding) : null);
694     return m;
695   }
696 
toJsonArray(final List<XAttr> array, final XAttrCodec encoding)697   private static Object[] toJsonArray(final List<XAttr> array,
698       final XAttrCodec encoding) throws IOException {
699     if (array == null) {
700       return null;
701     } else if (array.size() == 0) {
702       return EMPTY_OBJECT_ARRAY;
703     } else {
704       final Object[] a = new Object[array.size()];
705       for(int i = 0; i < array.size(); i++) {
706         a[i] = toJsonMap(array.get(i), encoding);
707       }
708       return a;
709     }
710   }
711 
toJsonString(final List<XAttr> xAttrs, final XAttrCodec encoding)712   public static String toJsonString(final List<XAttr> xAttrs,
713       final XAttrCodec encoding) throws IOException {
714     final Map<String, Object> finalMap = new TreeMap<String, Object>();
715     finalMap.put("XAttrs", toJsonArray(xAttrs, encoding));
716     ObjectMapper mapper = new ObjectMapper();
717     return mapper.writeValueAsString(finalMap);
718   }
719 
toJsonString(final List<XAttr> xAttrs)720   public static String toJsonString(final List<XAttr> xAttrs)
721       throws IOException {
722     final List<String> names = Lists.newArrayListWithCapacity(xAttrs.size());
723     for (XAttr xAttr : xAttrs) {
724       names.add(XAttrHelper.getPrefixName(xAttr));
725     }
726     ObjectMapper mapper = new ObjectMapper();
727     String ret = mapper.writeValueAsString(names);
728     final Map<String, Object> finalMap = new TreeMap<String, Object>();
729     finalMap.put("XAttrNames", ret);
730     return mapper.writeValueAsString(finalMap);
731   }
732 
getXAttr(final Map<?, ?> json, final String name)733   public static byte[] getXAttr(final Map<?, ?> json, final String name)
734       throws IOException {
735     if (json == null) {
736       return null;
737     }
738 
739     Map<String, byte[]> xAttrs = toXAttrs(json);
740     if (xAttrs != null) {
741       return xAttrs.get(name);
742     }
743 
744     return null;
745   }
746 
toXAttrs(final Map<?, ?> json)747   public static Map<String, byte[]> toXAttrs(final Map<?, ?> json)
748       throws IOException {
749     if (json == null) {
750       return null;
751     }
752     return toXAttrMap(getList(json, "XAttrs"));
753   }
754 
toXAttrNames(final Map<?, ?> json)755   public static List<String> toXAttrNames(final Map<?, ?> json)
756       throws IOException {
757     if (json == null) {
758       return null;
759     }
760 
761     final String namesInJson = (String) json.get("XAttrNames");
762     ObjectReader reader = new ObjectMapper().reader(List.class);
763     final List<Object> xattrs = reader.readValue(namesInJson);
764     final List<String> names =
765       Lists.newArrayListWithCapacity(json.keySet().size());
766 
767     for (Object xattr : xattrs) {
768       names.add((String) xattr);
769     }
770     return names;
771   }
772 
toXAttrMap(final List<?> objects)773   private static Map<String, byte[]> toXAttrMap(final List<?> objects)
774       throws IOException {
775     if (objects == null) {
776       return null;
777     } else if (objects.isEmpty()) {
778       return Maps.newHashMap();
779     } else {
780       final Map<String, byte[]> xAttrs = Maps.newHashMap();
781       for (Object object : objects) {
782         Map<?, ?> m = (Map<?, ?>) object;
783         String name = (String) m.get("name");
784         String value = (String) m.get("value");
785         xAttrs.put(name, decodeXAttrValue(value));
786       }
787       return xAttrs;
788     }
789   }
790 
decodeXAttrValue(String value)791   private static byte[] decodeXAttrValue(String value) throws IOException {
792     if (value != null) {
793       return XAttrCodec.decodeValue(value);
794     } else {
795       return new byte[0];
796     }
797   }
798 }
799