001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    package org.apache.hadoop.hdfs.web;
019    
020    import org.apache.hadoop.fs.*;
021    import org.apache.hadoop.fs.permission.AclEntry;
022    import org.apache.hadoop.fs.permission.AclStatus;
023    import org.apache.hadoop.fs.permission.FsPermission;
024    import org.apache.hadoop.hdfs.DFSUtil;
025    import org.apache.hadoop.hdfs.protocol.*;
026    import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
027    import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
028    import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
029    import org.apache.hadoop.hdfs.server.namenode.INodeId;
030    import org.apache.hadoop.ipc.RemoteException;
031    import org.apache.hadoop.security.token.Token;
032    import org.apache.hadoop.security.token.TokenIdentifier;
033    import org.apache.hadoop.util.DataChecksum;
034    import org.apache.hadoop.util.StringUtils;
035    import org.mortbay.util.ajax.JSON;
036    
037    import java.io.ByteArrayInputStream;
038    import java.io.DataInputStream;
039    import java.io.IOException;
040    import java.util.*;
041    
042    /** JSON Utilities */
043    public class JsonUtil {
044      private static final Object[] EMPTY_OBJECT_ARRAY = {};
045      private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
046    
047      /** Convert a token object to a Json string. */
048      public static String toJsonString(final Token<? extends TokenIdentifier> token
049          ) throws IOException {
050        return toJsonString(Token.class, toJsonMap(token));
051      }
052    
053      private static Map<String, Object> toJsonMap(
054          final Token<? extends TokenIdentifier> token) throws IOException {
055        if (token == null) {
056          return null;
057        }
058    
059        final Map<String, Object> m = new TreeMap<String, Object>();
060        m.put("urlString", token.encodeToUrlString());
061        return m;
062      }
063    
064      /** Convert a Json map to a Token. */
065      public static Token<? extends TokenIdentifier> toToken(
066          final Map<?, ?> m) throws IOException {
067        if (m == null) {
068          return null;
069        }
070    
071        final Token<DelegationTokenIdentifier> token
072            = new Token<DelegationTokenIdentifier>();
073        token.decodeFromUrlString((String)m.get("urlString"));
074        return token;
075      }
076    
077      /** Convert a Json map to a Token of DelegationTokenIdentifier. */
078      @SuppressWarnings("unchecked")
079      public static Token<DelegationTokenIdentifier> toDelegationToken(
080          final Map<?, ?> json) throws IOException {
081        final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName());
082        return (Token<DelegationTokenIdentifier>)toToken(m);
083      }
084    
085      /** Convert a Json map to a Token of BlockTokenIdentifier. */
086      @SuppressWarnings("unchecked")
087      private static Token<BlockTokenIdentifier> toBlockToken(
088          final Map<?, ?> m) throws IOException {
089        return (Token<BlockTokenIdentifier>)toToken(m);
090      }
091    
092      /** Convert a Token[] to a JSON array. */
093      private static Object[] toJsonArray(final Token<? extends TokenIdentifier>[] array
094          ) throws IOException {
095        if (array == null) {
096          return null;
097        } else if (array.length == 0) {
098          return EMPTY_OBJECT_ARRAY;
099        } else {
100          final Object[] a = new Object[array.length];
101          for(int i = 0; i < array.length; i++) {
102            a[i] = toJsonMap(array[i]);
103          }
104          return a;
105        }
106      }
107    
108      /** Convert a token object to a JSON string. */
109      public static String toJsonString(final Token<? extends TokenIdentifier>[] tokens
110          ) throws IOException {
111        if (tokens == null) {
112          return null;
113        }
114    
115        final Map<String, Object> m = new TreeMap<String, Object>();
116        m.put(Token.class.getSimpleName(), toJsonArray(tokens));
117        return toJsonString(Token.class.getSimpleName() + "s", m);
118      }
119    
120      /** Convert an Object[] to a List<Token<?>>.  */
121      private static List<Token<?>> toTokenList(final Object[] objects) throws IOException {
122        if (objects == null) {
123          return null;
124        } else if (objects.length == 0) {
125          return Collections.emptyList();
126        } else {
127          final List<Token<?>> list = new ArrayList<Token<?>>(objects.length);
128          for(int i = 0; i < objects.length; i++) {
129            list.add(toToken((Map<?, ?>)objects[i]));
130          }
131          return list;
132        }
133      }
134    
135      /** Convert a JSON map to a List<Token<?>>. */
136      public static List<Token<?>> toTokenList(final Map<?, ?> json) throws IOException {
137        if (json == null) {
138          return null;
139        }
140    
141        final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName() + "s");
142        return toTokenList((Object[])m.get(Token.class.getSimpleName()));
143      }
144    
145      /** Convert an exception object to a Json string. */
146      public static String toJsonString(final Exception e) {
147        final Map<String, Object> m = new TreeMap<String, Object>();
148        m.put("exception", e.getClass().getSimpleName());
149        m.put("message", e.getMessage());
150        m.put("javaClassName", e.getClass().getName());
151        return toJsonString(RemoteException.class, m);
152      }
153    
154      /** Convert a Json map to a RemoteException. */
155      public static RemoteException toRemoteException(final Map<?, ?> json) {
156        final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
157        final String message = (String)m.get("message");
158        final String javaClassName = (String)m.get("javaClassName");
159        return new RemoteException(javaClassName, message);
160      }
161    
162      private static String toJsonString(final Class<?> clazz, final Object value) {
163        return toJsonString(clazz.getSimpleName(), value);
164      }
165    
166      /** Convert a key-value pair to a Json string. */
167      public static String toJsonString(final String key, final Object value) {
168        final Map<String, Object> m = new TreeMap<String, Object>();
169        m.put(key, value);
170        return JSON.toString(m);
171      }
172    
173      /** Convert a FsPermission object to a string. */
174      private static String toString(final FsPermission permission) {
175        return String.format("%o", permission.toShort());
176      }
177    
178      /** Convert a string to a FsPermission object. */
179      private static FsPermission toFsPermission(final String s, Boolean aclBit) {
180        FsPermission perm = new FsPermission(Short.parseShort(s, 8));
181        return (aclBit != null && aclBit) ? new FsAclPermission(perm) : perm;
182      }
183    
184      static enum PathType {
185        FILE, DIRECTORY, SYMLINK;
186        
187        static PathType valueOf(HdfsFileStatus status) {
188          return status.isDir()? DIRECTORY: status.isSymlink()? SYMLINK: FILE;
189        }
190      }
191    
192      /** Convert a HdfsFileStatus object to a Json string. */
193      public static String toJsonString(final HdfsFileStatus status,
194          boolean includeType) {
195        if (status == null) {
196          return null;
197        }
198        final Map<String, Object> m = new TreeMap<String, Object>();
199        m.put("pathSuffix", status.getLocalName());
200        m.put("type", PathType.valueOf(status));
201        if (status.isSymlink()) {
202          m.put("symlink", status.getSymlink());
203        }
204    
205        m.put("length", status.getLen());
206        m.put("owner", status.getOwner());
207        m.put("group", status.getGroup());
208        FsPermission perm = status.getPermission();
209        m.put("permission", toString(perm));
210        if (perm.getAclBit()) {
211          m.put("aclBit", true);
212        }
213        m.put("accessTime", status.getAccessTime());
214        m.put("modificationTime", status.getModificationTime());
215        m.put("blockSize", status.getBlockSize());
216        m.put("replication", status.getReplication());
217        m.put("fileId", status.getFileId());
218        m.put("childrenNum", status.getChildrenNum());
219        return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
220      }
221    
222      /** Convert a Json map to a HdfsFileStatus object. */
223      public static HdfsFileStatus toFileStatus(final Map<?, ?> json, boolean includesType) {
224        if (json == null) {
225          return null;
226        }
227    
228        final Map<?, ?> m = includesType ? 
229            (Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json;
230        final String localName = (String) m.get("pathSuffix");
231        final PathType type = PathType.valueOf((String) m.get("type"));
232        final byte[] symlink = type != PathType.SYMLINK? null
233            : DFSUtil.string2Bytes((String)m.get("symlink"));
234    
235        final long len = (Long) m.get("length");
236        final String owner = (String) m.get("owner");
237        final String group = (String) m.get("group");
238        final FsPermission permission = toFsPermission((String) m.get("permission"),
239          (Boolean)m.get("aclBit"));
240        final long aTime = (Long) m.get("accessTime");
241        final long mTime = (Long) m.get("modificationTime");
242        final long blockSize = (Long) m.get("blockSize");
243        final short replication = (short) (long) (Long) m.get("replication");
244        final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
245            : INodeId.GRANDFATHER_INODE_ID;
246        Long childrenNumLong = (Long) m.get("childrenNum");
247        final int childrenNum = (childrenNumLong == null) ? -1
248                : childrenNumLong.intValue();
249        return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
250            blockSize, mTime, aTime, permission, owner, group,
251            symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum);
252      }
253    
254      /** Convert an ExtendedBlock to a Json map. */
255      private static Map<String, Object> toJsonMap(final ExtendedBlock extendedblock) {
256        if (extendedblock == null) {
257          return null;
258        }
259    
260        final Map<String, Object> m = new TreeMap<String, Object>();
261        m.put("blockPoolId", extendedblock.getBlockPoolId());
262        m.put("blockId", extendedblock.getBlockId());
263        m.put("numBytes", extendedblock.getNumBytes());
264        m.put("generationStamp", extendedblock.getGenerationStamp());
265        return m;
266      }
267    
268      /** Convert a Json map to an ExtendedBlock object. */
269      private static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
270        if (m == null) {
271          return null;
272        }
273        
274        final String blockPoolId = (String)m.get("blockPoolId");
275        final long blockId = (Long)m.get("blockId");
276        final long numBytes = (Long)m.get("numBytes");
277        final long generationStamp = (Long)m.get("generationStamp");
278        return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
279      }
280      
281      /** Convert a DatanodeInfo to a Json map. */
282      static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
283        if (datanodeinfo == null) {
284          return null;
285        }
286    
287        // TODO: Fix storageID
288        final Map<String, Object> m = new TreeMap<String, Object>();
289        m.put("ipAddr", datanodeinfo.getIpAddr());
290        // 'name' is equivalent to ipAddr:xferPort. Older clients (1.x, 0.23.x) 
291        // expects this instead of the two fields.
292        m.put("name", datanodeinfo.getXferAddr());
293        m.put("hostName", datanodeinfo.getHostName());
294        m.put("storageID", datanodeinfo.getDatanodeUuid());
295        m.put("xferPort", datanodeinfo.getXferPort());
296        m.put("infoPort", datanodeinfo.getInfoPort());
297        m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
298        m.put("ipcPort", datanodeinfo.getIpcPort());
299    
300        m.put("capacity", datanodeinfo.getCapacity());
301        m.put("dfsUsed", datanodeinfo.getDfsUsed());
302        m.put("remaining", datanodeinfo.getRemaining());
303        m.put("blockPoolUsed", datanodeinfo.getBlockPoolUsed());
304        m.put("cacheCapacity", datanodeinfo.getCacheCapacity());
305        m.put("cacheUsed", datanodeinfo.getCacheUsed());
306        m.put("lastUpdate", datanodeinfo.getLastUpdate());
307        m.put("xceiverCount", datanodeinfo.getXceiverCount());
308        m.put("networkLocation", datanodeinfo.getNetworkLocation());
309        m.put("adminState", datanodeinfo.getAdminState().name());
310        return m;
311      }
312    
313      private static int getInt(Map<?, ?> m, String key, final int defaultValue) {
314        Object value = m.get(key);
315        if (value == null) {
316          return defaultValue;
317        }
318        return (int) (long) (Long) value;
319      }
320    
321      private static long getLong(Map<?, ?> m, String key, final long defaultValue) {
322        Object value = m.get(key);
323        if (value == null) {
324          return defaultValue;
325        }
326        return (long) (Long) value;
327      }
328    
329      private static String getString(Map<?, ?> m, String key,
330          final String defaultValue) {
331        Object value = m.get(key);
332        if (value == null) {
333          return defaultValue;
334        }
335        return (String) value;
336      }
337    
338      /** Convert a Json map to an DatanodeInfo object. */
339      static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
340          throws IOException {
341        if (m == null) {
342          return null;
343        }
344    
345        // ipAddr and xferPort are the critical fields for accessing data.
346        // If any one of the two is missing, an exception needs to be thrown.
347    
348        // Handle the case of old servers (1.x, 0.23.x) sending 'name' instead
349        // of ipAddr and xferPort.
350        Object tmpValue = m.get("ipAddr");
351        String ipAddr = (tmpValue == null) ? null : (String)tmpValue;
352        tmpValue = m.get("xferPort");
353        int xferPort = (tmpValue == null) ? -1 : (int)(long)(Long)tmpValue;
354        if (ipAddr == null) {
355          tmpValue = m.get("name");
356          if (tmpValue != null) {
357            String name = (String)tmpValue;
358            int colonIdx = name.indexOf(':');
359            if (colonIdx > 0) {
360              ipAddr = name.substring(0, colonIdx);
361              xferPort = Integer.parseInt(name.substring(colonIdx +1));
362            } else {
363              throw new IOException(
364                  "Invalid value in server response: name=[" + name + "]");
365            }
366          } else {
367            throw new IOException(
368                "Missing both 'ipAddr' and 'name' in server response.");
369          }
370          // ipAddr is non-null & non-empty string at this point.
371        }
372    
373        // Check the validity of xferPort.
374        if (xferPort == -1) {
375          throw new IOException(
376              "Invalid or missing 'xferPort' in server response.");
377        }
378    
379        // TODO: Fix storageID
380        return new DatanodeInfo(
381            ipAddr,
382            (String)m.get("hostName"),
383            (String)m.get("storageID"),
384            xferPort,
385            (int)(long)(Long)m.get("infoPort"),
386            getInt(m, "infoSecurePort", 0),
387            (int)(long)(Long)m.get("ipcPort"),
388    
389            getLong(m, "capacity", 0l),
390            getLong(m, "dfsUsed", 0l),
391            getLong(m, "remaining", 0l),
392            getLong(m, "blockPoolUsed", 0l),
393            getLong(m, "cacheCapacity", 0l),
394            getLong(m, "cacheUsed", 0l),
395            getLong(m, "lastUpdate", 0l),
396            getInt(m, "xceiverCount", 0),
397            getString(m, "networkLocation", ""),
398            AdminStates.valueOf(getString(m, "adminState", "NORMAL")));
399      }
400    
401      /** Convert a DatanodeInfo[] to a Json array. */
402      private static Object[] toJsonArray(final DatanodeInfo[] array) {
403        if (array == null) {
404          return null;
405        } else if (array.length == 0) {
406          return EMPTY_OBJECT_ARRAY;
407        } else {
408          final Object[] a = new Object[array.length];
409          for(int i = 0; i < array.length; i++) {
410            a[i] = toJsonMap(array[i]);
411          }
412          return a;
413        }
414      }
415    
416      /** Convert an Object[] to a DatanodeInfo[]. */
417      private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) 
418          throws IOException {
419        if (objects == null) {
420          return null;
421        } else if (objects.length == 0) {
422          return EMPTY_DATANODE_INFO_ARRAY;
423        } else {
424          final DatanodeInfo[] array = new DatanodeInfo[objects.length];
425          for(int i = 0; i < array.length; i++) {
426            array[i] = toDatanodeInfo((Map<?, ?>) objects[i]);
427          }
428          return array;
429        }
430      }
431      
432      /** Convert a LocatedBlock to a Json map. */
433      private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock
434          ) throws IOException {
435        if (locatedblock == null) {
436          return null;
437        }
438     
439        final Map<String, Object> m = new TreeMap<String, Object>();
440        m.put("blockToken", toJsonMap(locatedblock.getBlockToken()));
441        m.put("isCorrupt", locatedblock.isCorrupt());
442        m.put("startOffset", locatedblock.getStartOffset());
443        m.put("block", toJsonMap(locatedblock.getBlock()));
444        m.put("locations", toJsonArray(locatedblock.getLocations()));
445        m.put("cachedLocations", toJsonArray(locatedblock.getCachedLocations()));
446        return m;
447      }
448    
449      /** Convert a Json map to LocatedBlock. */
450      private static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
451        if (m == null) {
452          return null;
453        }
454    
455        final ExtendedBlock b = toExtendedBlock((Map<?, ?>)m.get("block"));
456        final DatanodeInfo[] locations = toDatanodeInfoArray(
457            (Object[])m.get("locations"));
458        final long startOffset = (Long)m.get("startOffset");
459        final boolean isCorrupt = (Boolean)m.get("isCorrupt");
460        final DatanodeInfo[] cachedLocations = toDatanodeInfoArray(
461            (Object[])m.get("cachedLocations"));
462    
463        final LocatedBlock locatedblock = new LocatedBlock(b, locations,
464            null, null, startOffset, isCorrupt, cachedLocations);
465        locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
466        return locatedblock;
467      }
468    
469      /** Convert a LocatedBlock[] to a Json array. */
470      private static Object[] toJsonArray(final List<LocatedBlock> array
471          ) throws IOException {
472        if (array == null) {
473          return null;
474        } else if (array.size() == 0) {
475          return EMPTY_OBJECT_ARRAY;
476        } else {
477          final Object[] a = new Object[array.size()];
478          for(int i = 0; i < array.size(); i++) {
479            a[i] = toJsonMap(array.get(i));
480          }
481          return a;
482        }
483      }
484    
485      /** Convert an Object[] to a List of LocatedBlock. */
486      private static List<LocatedBlock> toLocatedBlockList(final Object[] objects
487          ) throws IOException {
488        if (objects == null) {
489          return null;
490        } else if (objects.length == 0) {
491          return Collections.emptyList();
492        } else {
493          final List<LocatedBlock> list = new ArrayList<LocatedBlock>(objects.length);
494          for(int i = 0; i < objects.length; i++) {
495            list.add(toLocatedBlock((Map<?, ?>)objects[i]));
496          }
497          return list;
498        }
499      }
500    
501      /** Convert LocatedBlocks to a Json string. */
502      public static String toJsonString(final LocatedBlocks locatedblocks
503          ) throws IOException {
504        if (locatedblocks == null) {
505          return null;
506        }
507    
508        final Map<String, Object> m = new TreeMap<String, Object>();
509        m.put("fileLength", locatedblocks.getFileLength());
510        m.put("isUnderConstruction", locatedblocks.isUnderConstruction());
511    
512        m.put("locatedBlocks", toJsonArray(locatedblocks.getLocatedBlocks()));
513        m.put("lastLocatedBlock", toJsonMap(locatedblocks.getLastLocatedBlock()));
514        m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete());
515        return toJsonString(LocatedBlocks.class, m);
516      }
517    
518      /** Convert a Json map to LocatedBlock. */
519      public static LocatedBlocks toLocatedBlocks(final Map<?, ?> json
520          ) throws IOException {
521        if (json == null) {
522          return null;
523        }
524    
525        final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
526        final long fileLength = (Long)m.get("fileLength");
527        final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
528        final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
529            (Object[])m.get("locatedBlocks"));
530        final LocatedBlock lastLocatedBlock = toLocatedBlock(
531            (Map<?, ?>)m.get("lastLocatedBlock"));
532        final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
533        return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
534            lastLocatedBlock, isLastBlockComplete);
535      }
536    
537      /** Convert a ContentSummary to a Json string. */
538      public static String toJsonString(final ContentSummary contentsummary) {
539        if (contentsummary == null) {
540          return null;
541        }
542    
543        final Map<String, Object> m = new TreeMap<String, Object>();
544        m.put("length", contentsummary.getLength());
545        m.put("fileCount", contentsummary.getFileCount());
546        m.put("directoryCount", contentsummary.getDirectoryCount());
547        m.put("quota", contentsummary.getQuota());
548        m.put("spaceConsumed", contentsummary.getSpaceConsumed());
549        m.put("spaceQuota", contentsummary.getSpaceQuota());
550        return toJsonString(ContentSummary.class, m);
551      }
552    
553      /** Convert a Json map to a ContentSummary. */
554      public static ContentSummary toContentSummary(final Map<?, ?> json) {
555        if (json == null) {
556          return null;
557        }
558    
559        final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
560        final long length = (Long)m.get("length");
561        final long fileCount = (Long)m.get("fileCount");
562        final long directoryCount = (Long)m.get("directoryCount");
563        final long quota = (Long)m.get("quota");
564        final long spaceConsumed = (Long)m.get("spaceConsumed");
565        final long spaceQuota = (Long)m.get("spaceQuota");
566    
567        return new ContentSummary(length, fileCount, directoryCount,
568            quota, spaceConsumed, spaceQuota);
569      }
570    
571      /** Convert a MD5MD5CRC32FileChecksum to a Json string. */
572      public static String toJsonString(final MD5MD5CRC32FileChecksum checksum) {
573        if (checksum == null) {
574          return null;
575        }
576    
577        final Map<String, Object> m = new TreeMap<String, Object>();
578        m.put("algorithm", checksum.getAlgorithmName());
579        m.put("length", checksum.getLength());
580        m.put("bytes", StringUtils.byteToHexString(checksum.getBytes()));
581        return toJsonString(FileChecksum.class, m);
582      }
583    
584      /** Convert a Json map to a MD5MD5CRC32FileChecksum. */
585      public static MD5MD5CRC32FileChecksum toMD5MD5CRC32FileChecksum(
586          final Map<?, ?> json) throws IOException {
587        if (json == null) {
588          return null;
589        }
590    
591        final Map<?, ?> m = (Map<?, ?>)json.get(FileChecksum.class.getSimpleName());
592        final String algorithm = (String)m.get("algorithm");
593        final int length = (int)(long)(Long)m.get("length");
594        final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
595    
596        final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
597        final DataChecksum.Type crcType = 
598            MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm);
599        final MD5MD5CRC32FileChecksum checksum;
600    
601        // Recreate what DFSClient would have returned.
602        switch(crcType) {
603          case CRC32:
604            checksum = new MD5MD5CRC32GzipFileChecksum();
605            break;
606          case CRC32C:
607            checksum = new MD5MD5CRC32CastagnoliFileChecksum();
608            break;
609          default:
610            throw new IOException("Unknown algorithm: " + algorithm);
611        }
612        checksum.readFields(in);
613    
614        //check algorithm name
615        if (!checksum.getAlgorithmName().equals(algorithm)) {
616          throw new IOException("Algorithm not matched. Expected " + algorithm
617              + ", Received " + checksum.getAlgorithmName());
618        }
619        //check length
620        if (length != checksum.getLength()) {
621          throw new IOException("Length not matched: length=" + length
622              + ", checksum.getLength()=" + checksum.getLength());
623        }
624    
625        return checksum;
626      }
627      /** Convert a AclStatus object to a Json string. */
628      public static String toJsonString(final AclStatus status) {
629        if (status == null) {
630          return null;
631        }
632    
633        final Map<String, Object> m = new TreeMap<String, Object>();
634        m.put("owner", status.getOwner());
635        m.put("group", status.getGroup());
636        m.put("stickyBit", status.isStickyBit());
637        m.put("entries", status.getEntries());
638        final Map<String, Map<String, Object>> finalMap =
639            new TreeMap<String, Map<String, Object>>();
640        finalMap.put(AclStatus.class.getSimpleName(), m);
641        return JSON.toString(finalMap);
642      }
643    
644      /** Convert a Json map to a AclStatus object. */
645      public static AclStatus toAclStatus(final Map<?, ?> json) {
646        if (json == null) {
647          return null;
648        }
649    
650        final Map<?, ?> m = (Map<?, ?>) json.get(AclStatus.class.getSimpleName());
651    
652        AclStatus.Builder aclStatusBuilder = new AclStatus.Builder();
653        aclStatusBuilder.owner((String) m.get("owner"));
654        aclStatusBuilder.group((String) m.get("group"));
655        aclStatusBuilder.stickyBit((Boolean) m.get("stickyBit"));
656    
657        final Object[] entries = (Object[]) m.get("entries");
658    
659        List<AclEntry> aclEntryList = new ArrayList<AclEntry>();
660        for (int i = 0; i < entries.length; i++) {
661          AclEntry aclEntry = AclEntry.parseAclEntry((String) entries[i], true);
662          aclEntryList.add(aclEntry);
663        }
664        aclStatusBuilder.addEntries(aclEntryList);
665        return aclStatusBuilder.build();
666      }
667    }