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 package org.apache.hadoop.hdfs;
19 
20 import java.io.IOException;
21 import java.util.ArrayList;
22 
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
25 import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
26 import org.apache.hadoop.fs.FSDataOutputStream;
27 import org.apache.hadoop.fs.FileSystem;
28 import org.apache.hadoop.fs.Path;
29 
30 import junit.framework.TestCase;
31 
32 /**
33  * This class tests DatanodeDescriptor.getBlocksScheduled() at the
34  * NameNode. This counter is supposed to keep track of blocks currently
35  * scheduled to a datanode.
36  */
37 public class TestBlocksScheduledCounter extends TestCase {
38 
testBlocksScheduledCounter()39   public void testBlocksScheduledCounter() throws IOException {
40 
41     MiniDFSCluster cluster = new MiniDFSCluster(new Configuration(), 1,
42                                                 true, null);
43     cluster.waitActive();
44     FileSystem fs = cluster.getFileSystem();
45 
46     //open a file an write a few bytes:
47     FSDataOutputStream out = fs.create(new Path("/testBlockScheduledCounter"));
48     for (int i=0; i<1024; i++) {
49       out.write(i);
50     }
51     // flush to make sure a block is allocated.
52     ((DFSOutputStream)(out.getWrappedStream())).sync();
53 
54     ArrayList<DatanodeDescriptor> dnList = new ArrayList<DatanodeDescriptor>();
55     cluster.getNameNode().getNamesystem().DFSNodesStatus(dnList, dnList);
56     DatanodeDescriptor dn = dnList.get(0);
57 
58     assertEquals(1, dn.getBlocksScheduled());
59 
60     // close the file and the counter should go to zero.
61     out.close();
62     assertEquals(0, dn.getBlocksScheduled());
63   }
64 }
65