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 java.io.ByteArrayInputStream;
021    import java.io.DataInputStream;
022    import java.io.IOException;
023    import java.util.ArrayList;
024    import java.util.Collections;
025    import java.util.List;
026    import java.util.Map;
027    import java.util.TreeMap;
028    
029    import org.apache.hadoop.fs.ContentSummary;
030    import org.apache.hadoop.fs.FileChecksum;
031    import org.apache.hadoop.fs.FileStatus;
032    import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
033    import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
034    import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
035    import org.apache.hadoop.fs.permission.FsPermission;
036    import org.apache.hadoop.hdfs.DFSUtil;
037    import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
038    import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
039    import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
040    import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
041    import org.apache.hadoop.hdfs.protocol.LocatedBlock;
042    import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
043    import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
044    import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
045    import org.apache.hadoop.hdfs.server.namenode.INodeId;
046    import org.apache.hadoop.ipc.RemoteException;
047    import org.apache.hadoop.security.token.Token;
048    import org.apache.hadoop.security.token.TokenIdentifier;
049    import org.apache.hadoop.util.DataChecksum;
050    import org.apache.hadoop.util.StringUtils;
051    import org.mortbay.util.ajax.JSON;
052    
053    /** JSON Utilities */
054    public class JsonUtil {
055      private static final Object[] EMPTY_OBJECT_ARRAY = {};
056      private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
057    
058      /** Convert a token object to a Json string. */
059      public static String toJsonString(final Token<? extends TokenIdentifier> token
060          ) throws IOException {
061        return toJsonString(Token.class, toJsonMap(token));
062      }
063    
064      private static Map<String, Object> toJsonMap(
065          final Token<? extends TokenIdentifier> token) throws IOException {
066        if (token == null) {
067          return null;
068        }
069    
070        final Map<String, Object> m = new TreeMap<String, Object>();
071        m.put("urlString", token.encodeToUrlString());
072        return m;
073      }
074    
075      /** Convert a Json map to a Token. */
076      public static Token<? extends TokenIdentifier> toToken(
077          final Map<?, ?> m) throws IOException {
078        if (m == null) {
079          return null;
080        }
081    
082        final Token<DelegationTokenIdentifier> token
083            = new Token<DelegationTokenIdentifier>();
084        token.decodeFromUrlString((String)m.get("urlString"));
085        return token;
086      }
087    
088      /** Convert a Json map to a Token of DelegationTokenIdentifier. */
089      @SuppressWarnings("unchecked")
090      public static Token<DelegationTokenIdentifier> toDelegationToken(
091          final Map<?, ?> json) throws IOException {
092        final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName());
093        return (Token<DelegationTokenIdentifier>)toToken(m);
094      }
095    
096      /** Convert a Json map to a Token of BlockTokenIdentifier. */
097      @SuppressWarnings("unchecked")
098      private static Token<BlockTokenIdentifier> toBlockToken(
099          final Map<?, ?> m) throws IOException {
100        return (Token<BlockTokenIdentifier>)toToken(m);
101      }
102    
103      /** Convert a Token[] to a JSON array. */
104      private static Object[] toJsonArray(final Token<? extends TokenIdentifier>[] array
105          ) throws IOException {
106        if (array == null) {
107          return null;
108        } else if (array.length == 0) {
109          return EMPTY_OBJECT_ARRAY;
110        } else {
111          final Object[] a = new Object[array.length];
112          for(int i = 0; i < array.length; i++) {
113            a[i] = toJsonMap(array[i]);
114          }
115          return a;
116        }
117      }
118    
119      /** Convert a token object to a JSON string. */
120      public static String toJsonString(final Token<? extends TokenIdentifier>[] tokens
121          ) throws IOException {
122        if (tokens == null) {
123          return null;
124        }
125    
126        final Map<String, Object> m = new TreeMap<String, Object>();
127        m.put(Token.class.getSimpleName(), toJsonArray(tokens));
128        return toJsonString(Token.class.getSimpleName() + "s", m);
129      }
130    
131      /** Convert an Object[] to a List<Token<?>>.  */
132      private static List<Token<?>> toTokenList(final Object[] objects) throws IOException {
133        if (objects == null) {
134          return null;
135        } else if (objects.length == 0) {
136          return Collections.emptyList();
137        } else {
138          final List<Token<?>> list = new ArrayList<Token<?>>(objects.length);
139          for(int i = 0; i < objects.length; i++) {
140            list.add(toToken((Map<?, ?>)objects[i]));
141          }
142          return list;
143        }
144      }
145    
146      /** Convert a JSON map to a List<Token<?>>. */
147      public static List<Token<?>> toTokenList(final Map<?, ?> json) throws IOException {
148        if (json == null) {
149          return null;
150        }
151    
152        final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName() + "s");
153        return toTokenList((Object[])m.get(Token.class.getSimpleName()));
154      }
155    
156      /** Convert an exception object to a Json string. */
157      public static String toJsonString(final Exception e) {
158        final Map<String, Object> m = new TreeMap<String, Object>();
159        m.put("exception", e.getClass().getSimpleName());
160        m.put("message", e.getMessage());
161        m.put("javaClassName", e.getClass().getName());
162        return toJsonString(RemoteException.class, m);
163      }
164    
165      /** Convert a Json map to a RemoteException. */
166      public static RemoteException toRemoteException(final Map<?, ?> json) {
167        final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
168        final String message = (String)m.get("message");
169        final String javaClassName = (String)m.get("javaClassName");
170        return new RemoteException(javaClassName, message);
171      }
172    
173      private static String toJsonString(final Class<?> clazz, final Object value) {
174        return toJsonString(clazz.getSimpleName(), value);
175      }
176    
177      /** Convert a key-value pair to a Json string. */
178      public static String toJsonString(final String key, final Object value) {
179        final Map<String, Object> m = new TreeMap<String, Object>();
180        m.put(key, value);
181        return JSON.toString(m);
182      }
183    
184      /** Convert a FsPermission object to a string. */
185      private static String toString(final FsPermission permission) {
186        return String.format("%o", permission.toShort());
187      }
188    
189      /** Convert a string to a FsPermission object. */
190      private static FsPermission toFsPermission(final String s) {
191        return new FsPermission(Short.parseShort(s, 8));
192      }
193    
194      static enum PathType {
195        FILE, DIRECTORY, SYMLINK;
196        
197        static PathType valueOf(HdfsFileStatus status) {
198          return status.isDir()? DIRECTORY: status.isSymlink()? SYMLINK: FILE;
199        }
200      }
201    
202      /** Convert a HdfsFileStatus object to a Json string. */
203      public static String toJsonString(final HdfsFileStatus status,
204          boolean includeType) {
205        if (status == null) {
206          return null;
207        }
208        final Map<String, Object> m = new TreeMap<String, Object>();
209        m.put("pathSuffix", status.getLocalName());
210        m.put("type", PathType.valueOf(status));
211        if (status.isSymlink()) {
212          m.put("symlink", status.getSymlink());
213        }
214    
215        m.put("length", status.getLen());
216        m.put("owner", status.getOwner());
217        m.put("group", status.getGroup());
218        m.put("permission", toString(status.getPermission()));
219        m.put("accessTime", status.getAccessTime());
220        m.put("modificationTime", status.getModificationTime());
221        m.put("blockSize", status.getBlockSize());
222        m.put("replication", status.getReplication());
223        m.put("fileId", status.getFileId());
224        m.put("childrenNum", status.getChildrenNum());
225        return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
226      }
227    
228      /** Convert a Json map to a HdfsFileStatus object. */
229      public static HdfsFileStatus toFileStatus(final Map<?, ?> json, boolean includesType) {
230        if (json == null) {
231          return null;
232        }
233    
234        final Map<?, ?> m = includesType ? 
235            (Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json;
236        final String localName = (String) m.get("pathSuffix");
237        final PathType type = PathType.valueOf((String) m.get("type"));
238        final byte[] symlink = type != PathType.SYMLINK? null
239            : DFSUtil.string2Bytes((String)m.get("symlink"));
240    
241        final long len = (Long) m.get("length");
242        final String owner = (String) m.get("owner");
243        final String group = (String) m.get("group");
244        final FsPermission permission = toFsPermission((String) m.get("permission"));
245        final long aTime = (Long) m.get("accessTime");
246        final long mTime = (Long) m.get("modificationTime");
247        final long blockSize = (Long) m.get("blockSize");
248        final short replication = (short) (long) (Long) m.get("replication");
249        final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
250            : INodeId.GRANDFATHER_INODE_ID;
251        Long childrenNumLong = (Long) m.get("childrenNum");
252        final int childrenNum = (childrenNumLong == null) ? -1
253                : childrenNumLong.intValue();
254        return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
255            blockSize, mTime, aTime, permission, owner, group,
256            symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum);
257      }
258    
259      /** Convert an ExtendedBlock to a Json map. */
260      private static Map<String, Object> toJsonMap(final ExtendedBlock extendedblock) {
261        if (extendedblock == null) {
262          return null;
263        }
264    
265        final Map<String, Object> m = new TreeMap<String, Object>();
266        m.put("blockPoolId", extendedblock.getBlockPoolId());
267        m.put("blockId", extendedblock.getBlockId());
268        m.put("numBytes", extendedblock.getNumBytes());
269        m.put("generationStamp", extendedblock.getGenerationStamp());
270        return m;
271      }
272    
273      /** Convert a Json map to an ExtendedBlock object. */
274      private static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
275        if (m == null) {
276          return null;
277        }
278        
279        final String blockPoolId = (String)m.get("blockPoolId");
280        final long blockId = (Long)m.get("blockId");
281        final long numBytes = (Long)m.get("numBytes");
282        final long generationStamp = (Long)m.get("generationStamp");
283        return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
284      }
285      
286      /** Convert a DatanodeInfo to a Json map. */
287      private static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
288        if (datanodeinfo == null) {
289          return null;
290        }
291    
292        final Map<String, Object> m = new TreeMap<String, Object>();
293        m.put("ipAddr", datanodeinfo.getIpAddr());
294        m.put("hostName", datanodeinfo.getHostName());
295        m.put("storageID", datanodeinfo.getStorageID());
296        m.put("xferPort", datanodeinfo.getXferPort());
297        m.put("infoPort", datanodeinfo.getInfoPort());
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("lastUpdate", datanodeinfo.getLastUpdate());
305        m.put("xceiverCount", datanodeinfo.getXceiverCount());
306        m.put("networkLocation", datanodeinfo.getNetworkLocation());
307        m.put("adminState", datanodeinfo.getAdminState().name());
308        return m;
309      }
310    
311      /** Convert a Json map to an DatanodeInfo object. */
312      private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
313        if (m == null) {
314          return null;
315        }
316    
317        return new DatanodeInfo(
318            (String)m.get("ipAddr"),
319            (String)m.get("hostName"),
320            (String)m.get("storageID"),
321            (int)(long)(Long)m.get("xferPort"),
322            (int)(long)(Long)m.get("infoPort"),
323            (int)(long)(Long)m.get("ipcPort"),
324    
325            (Long)m.get("capacity"),
326            (Long)m.get("dfsUsed"),
327            (Long)m.get("remaining"),
328            (Long)m.get("blockPoolUsed"),
329            (Long)m.get("lastUpdate"),
330            (int)(long)(Long)m.get("xceiverCount"),
331            (String)m.get("networkLocation"),
332            AdminStates.valueOf((String)m.get("adminState")));
333      }
334    
335      /** Convert a DatanodeInfo[] to a Json array. */
336      private static Object[] toJsonArray(final DatanodeInfo[] array) {
337        if (array == null) {
338          return null;
339        } else if (array.length == 0) {
340          return EMPTY_OBJECT_ARRAY;
341        } else {
342          final Object[] a = new Object[array.length];
343          for(int i = 0; i < array.length; i++) {
344            a[i] = toJsonMap(array[i]);
345          }
346          return a;
347        }
348      }
349    
350      /** Convert an Object[] to a DatanodeInfo[]. */
351      private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
352        if (objects == null) {
353          return null;
354        } else if (objects.length == 0) {
355          return EMPTY_DATANODE_INFO_ARRAY;
356        } else {
357          final DatanodeInfo[] array = new DatanodeInfo[objects.length];
358          for(int i = 0; i < array.length; i++) {
359            array[i] = toDatanodeInfo((Map<?, ?>) objects[i]);
360          }
361          return array;
362        }
363      }
364      
365      /** Convert a LocatedBlock to a Json map. */
366      private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock
367          ) throws IOException {
368        if (locatedblock == null) {
369          return null;
370        }
371     
372        final Map<String, Object> m = new TreeMap<String, Object>();
373        m.put("blockToken", toJsonMap(locatedblock.getBlockToken()));
374        m.put("isCorrupt", locatedblock.isCorrupt());
375        m.put("startOffset", locatedblock.getStartOffset());
376        m.put("block", toJsonMap(locatedblock.getBlock()));
377        m.put("locations", toJsonArray(locatedblock.getLocations()));
378        return m;
379      }
380    
381      /** Convert a Json map to LocatedBlock. */
382      private static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
383        if (m == null) {
384          return null;
385        }
386    
387        final ExtendedBlock b = toExtendedBlock((Map<?, ?>)m.get("block"));
388        final DatanodeInfo[] locations = toDatanodeInfoArray(
389            (Object[])m.get("locations"));
390        final long startOffset = (Long)m.get("startOffset");
391        final boolean isCorrupt = (Boolean)m.get("isCorrupt");
392    
393        final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt);
394        locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
395        return locatedblock;
396      }
397    
398      /** Convert a LocatedBlock[] to a Json array. */
399      private static Object[] toJsonArray(final List<LocatedBlock> array
400          ) throws IOException {
401        if (array == null) {
402          return null;
403        } else if (array.size() == 0) {
404          return EMPTY_OBJECT_ARRAY;
405        } else {
406          final Object[] a = new Object[array.size()];
407          for(int i = 0; i < array.size(); i++) {
408            a[i] = toJsonMap(array.get(i));
409          }
410          return a;
411        }
412      }
413    
414      /** Convert an Object[] to a List of LocatedBlock. */
415      private static List<LocatedBlock> toLocatedBlockList(final Object[] objects
416          ) throws IOException {
417        if (objects == null) {
418          return null;
419        } else if (objects.length == 0) {
420          return Collections.emptyList();
421        } else {
422          final List<LocatedBlock> list = new ArrayList<LocatedBlock>(objects.length);
423          for(int i = 0; i < objects.length; i++) {
424            list.add(toLocatedBlock((Map<?, ?>)objects[i]));
425          }
426          return list;
427        }
428      }
429    
430      /** Convert LocatedBlocks to a Json string. */
431      public static String toJsonString(final LocatedBlocks locatedblocks
432          ) throws IOException {
433        if (locatedblocks == null) {
434          return null;
435        }
436    
437        final Map<String, Object> m = new TreeMap<String, Object>();
438        m.put("fileLength", locatedblocks.getFileLength());
439        m.put("isUnderConstruction", locatedblocks.isUnderConstruction());
440    
441        m.put("locatedBlocks", toJsonArray(locatedblocks.getLocatedBlocks()));
442        m.put("lastLocatedBlock", toJsonMap(locatedblocks.getLastLocatedBlock()));
443        m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete());
444        return toJsonString(LocatedBlocks.class, m);
445      }
446    
447      /** Convert a Json map to LocatedBlock. */
448      public static LocatedBlocks toLocatedBlocks(final Map<?, ?> json
449          ) throws IOException {
450        if (json == null) {
451          return null;
452        }
453    
454        final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
455        final long fileLength = (Long)m.get("fileLength");
456        final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
457        final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
458            (Object[])m.get("locatedBlocks"));
459        final LocatedBlock lastLocatedBlock = toLocatedBlock(
460            (Map<?, ?>)m.get("lastLocatedBlock"));
461        final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
462        return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
463            lastLocatedBlock, isLastBlockComplete);
464      }
465    
466      /** Convert a ContentSummary to a Json string. */
467      public static String toJsonString(final ContentSummary contentsummary) {
468        if (contentsummary == null) {
469          return null;
470        }
471    
472        final Map<String, Object> m = new TreeMap<String, Object>();
473        m.put("length", contentsummary.getLength());
474        m.put("fileCount", contentsummary.getFileCount());
475        m.put("directoryCount", contentsummary.getDirectoryCount());
476        m.put("quota", contentsummary.getQuota());
477        m.put("spaceConsumed", contentsummary.getSpaceConsumed());
478        m.put("spaceQuota", contentsummary.getSpaceQuota());
479        return toJsonString(ContentSummary.class, m);
480      }
481    
482      /** Convert a Json map to a ContentSummary. */
483      public static ContentSummary toContentSummary(final Map<?, ?> json) {
484        if (json == null) {
485          return null;
486        }
487    
488        final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
489        final long length = (Long)m.get("length");
490        final long fileCount = (Long)m.get("fileCount");
491        final long directoryCount = (Long)m.get("directoryCount");
492        final long quota = (Long)m.get("quota");
493        final long spaceConsumed = (Long)m.get("spaceConsumed");
494        final long spaceQuota = (Long)m.get("spaceQuota");
495    
496        return new ContentSummary(length, fileCount, directoryCount,
497            quota, spaceConsumed, spaceQuota);
498      }
499    
500      /** Convert a MD5MD5CRC32FileChecksum to a Json string. */
501      public static String toJsonString(final MD5MD5CRC32FileChecksum checksum) {
502        if (checksum == null) {
503          return null;
504        }
505    
506        final Map<String, Object> m = new TreeMap<String, Object>();
507        m.put("algorithm", checksum.getAlgorithmName());
508        m.put("length", checksum.getLength());
509        m.put("bytes", StringUtils.byteToHexString(checksum.getBytes()));
510        return toJsonString(FileChecksum.class, m);
511      }
512    
513      /** Convert a Json map to a MD5MD5CRC32FileChecksum. */
514      public static MD5MD5CRC32FileChecksum toMD5MD5CRC32FileChecksum(
515          final Map<?, ?> json) throws IOException {
516        if (json == null) {
517          return null;
518        }
519    
520        final Map<?, ?> m = (Map<?, ?>)json.get(FileChecksum.class.getSimpleName());
521        final String algorithm = (String)m.get("algorithm");
522        final int length = (int)(long)(Long)m.get("length");
523        final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
524    
525        final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
526        final DataChecksum.Type crcType = 
527            MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm);
528        final MD5MD5CRC32FileChecksum checksum;
529    
530        // Recreate what DFSClient would have returned.
531        switch(crcType) {
532          case CRC32:
533            checksum = new MD5MD5CRC32GzipFileChecksum();
534            break;
535          case CRC32C:
536            checksum = new MD5MD5CRC32CastagnoliFileChecksum();
537            break;
538          default:
539            throw new IOException("Unknown algorithm: " + algorithm);
540        }
541        checksum.readFields(in);
542    
543        //check algorithm name
544        if (!checksum.getAlgorithmName().equals(algorithm)) {
545          throw new IOException("Algorithm not matched. Expected " + algorithm
546              + ", Received " + checksum.getAlgorithmName());
547        }
548        //check length
549        if (length != checksum.getLength()) {
550          throw new IOException("Length not matched: length=" + length
551              + ", checksum.getLength()=" + checksum.getLength());
552        }
553    
554        return checksum;
555      }
556    }