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