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 junit.framework.TestCase; |
---|
21 | import java.io.*; |
---|
22 | import java.util.Random; |
---|
23 | import org.apache.hadoop.conf.Configuration; |
---|
24 | import org.apache.hadoop.fs.ChecksumFileSystem; |
---|
25 | import org.apache.hadoop.fs.FSDataInputStream; |
---|
26 | import org.apache.hadoop.fs.FSInputStream; |
---|
27 | import org.apache.hadoop.fs.FileSystem; |
---|
28 | import org.apache.hadoop.fs.Path; |
---|
29 | |
---|
30 | /** |
---|
31 | * This class tests the presence of seek bug as described |
---|
32 | * in HADOOP-508 |
---|
33 | */ |
---|
34 | public class TestSeekBug extends TestCase { |
---|
35 | static final long seed = 0xDEADBEEFL; |
---|
36 | static final int ONEMB = 1 << 20; |
---|
37 | |
---|
38 | private void writeFile(FileSystem fileSys, Path name) throws IOException { |
---|
39 | // create and write a file that contains 1MB |
---|
40 | DataOutputStream stm = fileSys.create(name); |
---|
41 | byte[] buffer = new byte[ONEMB]; |
---|
42 | Random rand = new Random(seed); |
---|
43 | rand.nextBytes(buffer); |
---|
44 | stm.write(buffer); |
---|
45 | stm.close(); |
---|
46 | } |
---|
47 | |
---|
48 | private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) { |
---|
49 | for (int idx = 0; idx < actual.length; idx++) { |
---|
50 | this.assertEquals(message+" byte "+(from+idx)+" differs. expected "+ |
---|
51 | expected[from+idx]+" actual "+actual[idx], |
---|
52 | actual[idx], expected[from+idx]); |
---|
53 | actual[idx] = 0; |
---|
54 | } |
---|
55 | } |
---|
56 | |
---|
57 | private void seekReadFile(FileSystem fileSys, Path name) throws IOException { |
---|
58 | FSDataInputStream stm = fileSys.open(name, 4096); |
---|
59 | byte[] expected = new byte[ONEMB]; |
---|
60 | Random rand = new Random(seed); |
---|
61 | rand.nextBytes(expected); |
---|
62 | |
---|
63 | // First read 128 bytes to set count in BufferedInputStream |
---|
64 | byte[] actual = new byte[128]; |
---|
65 | stm.read(actual, 0, actual.length); |
---|
66 | // Now read a byte array that is bigger than the internal buffer |
---|
67 | actual = new byte[100000]; |
---|
68 | stm.read(actual, 0, actual.length); |
---|
69 | checkAndEraseData(actual, 128, expected, "First Read Test"); |
---|
70 | // now do a small seek, within the range that is already read |
---|
71 | stm.seek(96036); // 4 byte seek |
---|
72 | actual = new byte[128]; |
---|
73 | stm.read(actual, 0, actual.length); |
---|
74 | checkAndEraseData(actual, 96036, expected, "Seek Bug"); |
---|
75 | // all done |
---|
76 | stm.close(); |
---|
77 | } |
---|
78 | |
---|
79 | /* |
---|
80 | * Read some data, skip a few bytes and read more. HADOOP-922. |
---|
81 | */ |
---|
82 | private void smallReadSeek(FileSystem fileSys, Path name) throws IOException { |
---|
83 | if (fileSys instanceof ChecksumFileSystem) { |
---|
84 | fileSys = ((ChecksumFileSystem)fileSys).getRawFileSystem(); |
---|
85 | } |
---|
86 | // Make the buffer size small to trigger code for HADOOP-922 |
---|
87 | FSDataInputStream stmRaw = fileSys.open(name, 1); |
---|
88 | byte[] expected = new byte[ONEMB]; |
---|
89 | Random rand = new Random(seed); |
---|
90 | rand.nextBytes(expected); |
---|
91 | |
---|
92 | // Issue a simple read first. |
---|
93 | byte[] actual = new byte[128]; |
---|
94 | stmRaw.seek(100000); |
---|
95 | stmRaw.read(actual, 0, actual.length); |
---|
96 | checkAndEraseData(actual, 100000, expected, "First Small Read Test"); |
---|
97 | |
---|
98 | // now do a small seek of 4 bytes, within the same block. |
---|
99 | int newpos1 = 100000 + 128 + 4; |
---|
100 | stmRaw.seek(newpos1); |
---|
101 | stmRaw.read(actual, 0, actual.length); |
---|
102 | checkAndEraseData(actual, newpos1, expected, "Small Seek Bug 1"); |
---|
103 | |
---|
104 | // seek another 256 bytes this time |
---|
105 | int newpos2 = newpos1 + 256; |
---|
106 | stmRaw.seek(newpos2); |
---|
107 | stmRaw.read(actual, 0, actual.length); |
---|
108 | checkAndEraseData(actual, newpos2, expected, "Small Seek Bug 2"); |
---|
109 | |
---|
110 | // all done |
---|
111 | stmRaw.close(); |
---|
112 | } |
---|
113 | |
---|
114 | private void cleanupFile(FileSystem fileSys, Path name) throws IOException { |
---|
115 | assertTrue(fileSys.exists(name)); |
---|
116 | fileSys.delete(name, true); |
---|
117 | assertTrue(!fileSys.exists(name)); |
---|
118 | } |
---|
119 | |
---|
120 | /** |
---|
121 | * Test if the seek bug exists in FSDataInputStream in DFS. |
---|
122 | */ |
---|
123 | public void testSeekBugDFS() throws IOException { |
---|
124 | Configuration conf = new Configuration(); |
---|
125 | MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null); |
---|
126 | FileSystem fileSys = cluster.getFileSystem(); |
---|
127 | try { |
---|
128 | Path file1 = new Path("seektest.dat"); |
---|
129 | writeFile(fileSys, file1); |
---|
130 | seekReadFile(fileSys, file1); |
---|
131 | smallReadSeek(fileSys, file1); |
---|
132 | cleanupFile(fileSys, file1); |
---|
133 | } finally { |
---|
134 | fileSys.close(); |
---|
135 | cluster.shutdown(); |
---|
136 | } |
---|
137 | } |
---|
138 | |
---|
139 | /** |
---|
140 | * Tests if the seek bug exists in FSDataInputStream in LocalFS. |
---|
141 | */ |
---|
142 | public void testSeekBugLocalFS() throws IOException { |
---|
143 | Configuration conf = new Configuration(); |
---|
144 | FileSystem fileSys = FileSystem.getLocal(conf); |
---|
145 | try { |
---|
146 | Path file1 = new Path("build/test/data", "seektest.dat"); |
---|
147 | writeFile(fileSys, file1); |
---|
148 | seekReadFile(fileSys, file1); |
---|
149 | cleanupFile(fileSys, file1); |
---|
150 | } finally { |
---|
151 | fileSys.close(); |
---|
152 | } |
---|
153 | } |
---|
154 | } |
---|