source: proiecte/HadoopJUnit/hadoop-0.20.1/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.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: 5.8 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 */
18package org.apache.hadoop.hdfs;
19
20import java.io.IOException;
21
22import org.apache.commons.logging.impl.Log4JLogger;
23import org.apache.hadoop.conf.Configuration;
24import org.apache.hadoop.fs.FSDataInputStream;
25import org.apache.hadoop.fs.FSDataOutputStream;
26import org.apache.hadoop.fs.FileSystem;
27import org.apache.hadoop.fs.Path;
28import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
29import org.apache.hadoop.hdfs.server.datanode.DataNode;
30import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
31import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
32import org.apache.hadoop.security.UnixUserGroupInformation;
33import org.apache.hadoop.security.UserGroupInformation;
34import org.apache.log4j.Level;
35
36public class TestLeaseRecovery2 extends junit.framework.TestCase {
37  {
38    ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
39    ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
40    ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
41  }
42
43  static final long BLOCK_SIZE = 1024;
44  static final int FILE_SIZE = 1024*16;
45  static final short REPLICATION_NUM = (short)3;
46  static byte[] buffer = new byte[FILE_SIZE];
47
48  public void testBlockSynchronization() throws Exception {
49    final long softLease = 1000;
50    final long hardLease = 60 * 60 *1000;
51    final short repl = 3;
52    final Configuration conf = new Configuration();
53    final int bufferSize = conf.getInt("io.file.buffer.size", 4096);
54    conf.setLong("dfs.block.size", BLOCK_SIZE);
55    conf.setInt("dfs.heartbeat.interval", 1);
56  //  conf.setInt("io.bytes.per.checksum", 16);
57
58    MiniDFSCluster cluster = null;
59    byte[] actual = new byte[FILE_SIZE];
60
61    try {
62      cluster = new MiniDFSCluster(conf, 5, true, null);
63      cluster.waitActive();
64
65      //create a file
66      DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
67      // create a random file name
68      String filestr = "/foo" + AppendTestUtil.nextInt();
69      System.out.println("filestr=" + filestr);
70      Path filepath = new Path(filestr);
71      FSDataOutputStream stm = dfs.create(filepath, true,
72          bufferSize, repl, BLOCK_SIZE);
73      assertTrue(dfs.dfs.exists(filestr));
74
75      // write random number of bytes into it.
76      int size = AppendTestUtil.nextInt(FILE_SIZE);
77      System.out.println("size=" + size);
78      stm.write(buffer, 0, size);
79
80      // sync file
81      AppendTestUtil.LOG.info("sync");
82      stm.sync();
83      AppendTestUtil.LOG.info("leasechecker.interruptAndJoin()");
84      dfs.dfs.leasechecker.interruptAndJoin();
85
86      // set the soft limit to be 1 second so that the
87      // namenode triggers lease recovery on next attempt to write-for-open.
88      cluster.setLeasePeriod(softLease, hardLease);
89
90      // try to re-open the file before closing the previous handle. This
91      // should fail but will trigger lease recovery.
92      {
93        Configuration conf2 = new Configuration(conf);
94        String username = UserGroupInformation.getCurrentUGI().getUserName()+"_1";
95        UnixUserGroupInformation.saveToConf(conf2,
96            UnixUserGroupInformation.UGI_PROPERTY_NAME,
97            new UnixUserGroupInformation(username, new String[]{"supergroup"}));
98        FileSystem dfs2 = FileSystem.get(conf2);
99 
100        boolean done = false;
101        for(int i = 0; i < 10 && !done; i++) {
102          AppendTestUtil.LOG.info("i=" + i);
103          try {
104            dfs2.create(filepath, false, bufferSize, repl, BLOCK_SIZE);
105            fail("Creation of an existing file should never succeed.");
106          } catch (IOException ioe) {
107            final String message = ioe.getMessage();
108            if (message.contains("file exists")) {
109              AppendTestUtil.LOG.info("done", ioe);
110              done = true;
111            }
112            else if (message.contains(AlreadyBeingCreatedException.class.getSimpleName())) {
113              AppendTestUtil.LOG.info("GOOD! got " + message);
114            }
115            else {
116              AppendTestUtil.LOG.warn("UNEXPECTED IOException", ioe);
117            }
118          }
119
120          if (!done) {
121            AppendTestUtil.LOG.info("sleep " + 5000 + "ms");
122            try {Thread.sleep(5000);} catch (InterruptedException e) {}
123          }
124        }
125        assertTrue(done);
126      }
127
128      AppendTestUtil.LOG.info("Lease for file " +  filepath + " is recovered. "
129          + "Validating its contents now...");
130
131      // verify that file-size matches
132      assertTrue("File should be " + size + " bytes, but is actually " +
133                 " found to be " + dfs.getFileStatus(filepath).getLen() +
134                 " bytes",
135                 dfs.getFileStatus(filepath).getLen() == size);
136
137      // verify that there is enough data to read.
138      System.out.println("File size is good. Now validating sizes from datanodes...");
139      FSDataInputStream stmin = dfs.open(filepath);
140      stmin.readFully(0, actual, 0, size);
141      stmin.close();
142    }
143    finally {
144      try {
145        if (cluster != null) {cluster.shutdown();}
146      } catch (Exception e) {
147        // ignore
148      }
149    }
150  }
151}
Note: See TracBrowser for help on using the repository browser.