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