source: proiecte/HadoopJUnit/hadoop-0.20.1/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 7.0 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18
19package org.apache.hadoop.hdfs;
20
21import junit.framework.TestCase;
22import java.io.*;
23import java.net.InetSocketAddress;
24import java.util.Iterator;
25import java.util.LinkedList;
26import java.util.TreeMap;
27import java.util.zip.CRC32;
28
29import org.apache.hadoop.conf.Configuration;
30import org.apache.hadoop.fs.FSInputStream;
31import org.apache.hadoop.fs.FileStatus;
32import org.apache.hadoop.fs.FileUtil;
33import org.apache.hadoop.hdfs.protocol.FSConstants;
34import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
35import org.apache.commons.logging.Log;
36import org.apache.commons.logging.LogFactory;
37
38/**
39 * This tests data transfer protocol handling in the Datanode. It sends
40 * various forms of wrong data and verifies that Datanode handles it well.
41 *
42 * This test uses the following two file from src/test/.../dfs directory :
43 *   1) hadoop-version-dfs-dir.tgz : contains DFS directories.
44 *   2) hadoop-dfs-dir.txt : checksums that are compared in this test.
45 * Please read hadoop-dfs-dir.txt for more information. 
46 */
47public class TestDFSUpgradeFromImage extends TestCase {
48 
49  private static final Log LOG = LogFactory.getLog(
50                    "org.apache.hadoop.hdfs.TestDFSUpgradeFromImage");
51 
52  public int numDataNodes = 4;
53 
54  private static class ReferenceFileInfo {
55    String path;
56    long checksum;
57  }
58 
59  LinkedList<ReferenceFileInfo> refList = new LinkedList<ReferenceFileInfo>();
60  Iterator<ReferenceFileInfo> refIter;
61 
62  boolean printChecksum = false;
63 
64  protected void setUp() throws IOException {
65    unpackStorage();
66  }
67
68  public void unpackStorage() throws IOException {
69    String tarFile = System.getProperty("test.cache.data", "build/test/cache") +
70                     "/hadoop-14-dfs-dir.tgz";
71    String dataDir = System.getProperty("test.build.data", "build/test/data");
72    File dfsDir = new File(dataDir, "dfs");
73    if ( dfsDir.exists() && !FileUtil.fullyDelete(dfsDir) ) {
74      throw new IOException("Could not delete dfs directory '" + dfsDir + "'");
75    }
76    FileUtil.unTar(new File(tarFile), new File(dataDir));
77    //Now read the reference info
78   
79    BufferedReader reader = new BufferedReader( 
80                        new FileReader(System.getProperty("test.cache.data", "build/test/cache") +
81                                       "/hadoop-dfs-dir.txt"));
82    String line;
83    while ( (line = reader.readLine()) != null ) {
84     
85      line = line.trim();
86      if (line.length() <= 0 || line.startsWith("#")) {
87        continue;
88      }
89      String[] arr = line.split("\\s+\t\\s+");
90      if (arr.length < 1) {
91        continue;
92      }
93      if (arr[0].equals("printChecksums")) {
94        printChecksum = true;
95        break;
96      }
97      if (arr.length < 2) {
98        continue;
99      }
100      ReferenceFileInfo info = new ReferenceFileInfo();
101      info.path = arr[0];
102      info.checksum = Long.parseLong(arr[1]);
103      refList.add(info);
104    }
105    reader.close();
106  }
107
108  private void verifyChecksum(String path, long checksum) throws IOException {
109    if ( refIter == null ) {
110      refIter = refList.iterator();
111    }
112   
113    if ( printChecksum ) {
114      LOG.info("CRC info for reference file : " + path + " \t " + checksum);
115    } else {
116      if ( !refIter.hasNext() ) {
117        throw new IOException("Checking checksum for " + path +
118                              "Not enough elements in the refList");
119      }
120      ReferenceFileInfo info = refIter.next();
121      // The paths are expected to be listed in the same order
122      // as they are traversed here.
123      assertEquals(info.path, path);
124      assertEquals("Checking checksum for " + path, info.checksum, checksum);
125    }
126  }
127 
128  CRC32 overallChecksum = new CRC32();
129 
130  private void verifyDir(DFSClient client, String dir) 
131                                           throws IOException {
132   
133    FileStatus[] fileArr = client.listPaths(dir);
134    TreeMap<String, Boolean> fileMap = new TreeMap<String, Boolean>();
135   
136    for(FileStatus file : fileArr) {
137      String path = file.getPath().toString();
138      fileMap.put(path, Boolean.valueOf(file.isDir()));
139    }
140   
141    for(Iterator<String> it = fileMap.keySet().iterator(); it.hasNext();) {
142      String path = it.next();
143      boolean isDir = fileMap.get(path);
144     
145      overallChecksum.update(path.getBytes());
146     
147      if ( isDir ) {
148        verifyDir(client, path);
149      } else {
150        // this is not a directory. Checksum the file data.
151        CRC32 fileCRC = new CRC32();
152        FSInputStream in = client.open(path);
153        byte[] buf = new byte[4096];
154        int nRead = 0;
155        while ( (nRead = in.read(buf, 0, buf.length)) > 0 ) {
156          fileCRC.update(buf, 0, nRead);
157        }
158       
159        verifyChecksum(path, fileCRC.getValue());
160      }
161    }
162  }
163 
164  private void verifyFileSystem(DFSClient client) throws IOException {
165 
166    verifyDir(client, "/");
167   
168    verifyChecksum("overallCRC", overallChecksum.getValue());
169   
170    if ( printChecksum ) {
171      throw new IOException("Checksums are written to log as requested. " +
172                            "Throwing this exception to force an error " +
173                            "for this test.");
174    }
175  }
176 
177  public void testUpgradeFromImage() throws IOException {
178    MiniDFSCluster cluster = null;
179    try {
180      Configuration conf = new Configuration();
181      if (System.getProperty("test.build.data") == null) { // to allow test to be run outside of Ant
182        System.setProperty("test.build.data", "build/test/data");
183      }
184      conf.setInt("dfs.datanode.scan.period.hours", -1); // block scanning off
185      cluster = new MiniDFSCluster(0, conf, numDataNodes, false, true,
186                                   StartupOption.UPGRADE, null);
187      cluster.waitActive();
188      DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost",
189                                           cluster.getNameNodePort()), conf);
190      //Safemode will be off only after upgrade is complete. Wait for it.
191      while ( dfsClient.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_GET) ) {
192        LOG.info("Waiting for SafeMode to be OFF.");
193        try {
194          Thread.sleep(1000);
195        } catch (InterruptedException ignored) {}
196      }
197
198      verifyFileSystem(dfsClient);
199    } finally {
200      if (cluster != null) { cluster.shutdown(); }
201    }
202  }
203}
Note: See TracBrowser for help on using the repository browser.