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 19 package org.apache.hadoop.hdfs.web; 20 21 import java.io.IOException; 22 import java.net.InetSocketAddress; 23 import java.net.ServerSocket; 24 import java.net.Socket; 25 import java.util.concurrent.Callable; 26 import java.util.concurrent.ExecutorService; 27 import java.util.concurrent.Executors; 28 import java.util.concurrent.Future; 29 import java.util.concurrent.TimeUnit; 30 import java.util.regex.Matcher; 31 import java.util.regex.Pattern; 32 33 import org.apache.hadoop.conf.Configuration; 34 import org.apache.hadoop.fs.FSDataOutputStream; 35 import org.apache.hadoop.fs.FileSystem; 36 import org.apache.hadoop.fs.Path; 37 import org.apache.hadoop.net.NetUtils; 38 import org.junit.AfterClass; 39 import org.junit.Assert; 40 import org.junit.BeforeClass; 41 import org.junit.Test; 42 43 44 public class TestWebHdfsContentLength { 45 private static ServerSocket listenSocket; 46 private static String bindAddr; 47 private static Path p; 48 private static FileSystem fs; 49 50 private static final Pattern contentLengthPattern = Pattern.compile( 51 "^(Content-Length|Transfer-Encoding):\\s*(.*)", Pattern.MULTILINE); 52 53 private static String errResponse = 54 "HTTP/1.1 500 Boom\r\n" + 55 "Content-Length: 0\r\n" + 56 "Connection: close\r\n\r\n"; 57 private static String redirectResponse; 58 59 private static ExecutorService executor; 60 61 @BeforeClass setup()62 public static void setup() throws IOException { 63 listenSocket = new ServerSocket(); 64 listenSocket.bind(null); 65 bindAddr = NetUtils.getHostPortString( 66 (InetSocketAddress)listenSocket.getLocalSocketAddress()); 67 redirectResponse = 68 "HTTP/1.1 307 Redirect\r\n" + 69 "Location: http://"+bindAddr+"/path\r\n" + 70 "Connection: close\r\n\r\n"; 71 72 p = new Path("webhdfs://"+bindAddr+"/path"); 73 fs = p.getFileSystem(new Configuration()); 74 executor = Executors.newSingleThreadExecutor(); 75 } 76 77 @AfterClass teardown()78 public static void teardown() throws IOException { 79 if (listenSocket != null) { 80 listenSocket.close(); 81 } 82 if (executor != null) { 83 executor.shutdownNow(); 84 } 85 } 86 87 @Test testGetOp()88 public void testGetOp() throws Exception { 89 Future<String> future = contentLengthFuture(errResponse); 90 try { 91 fs.getFileStatus(p); 92 Assert.fail(); 93 } catch (IOException ioe) {} // expected 94 Assert.assertEquals(null, getContentLength(future)); 95 } 96 97 @Test testGetOpWithRedirect()98 public void testGetOpWithRedirect() { 99 Future<String> future1 = contentLengthFuture(redirectResponse); 100 Future<String> future2 = contentLengthFuture(errResponse); 101 try { 102 fs.open(p).read(); 103 Assert.fail(); 104 } catch (IOException ioe) {} // expected 105 Assert.assertEquals(null, getContentLength(future1)); 106 Assert.assertEquals(null, getContentLength(future2)); 107 } 108 109 @Test testPutOp()110 public void testPutOp() { 111 Future<String> future = contentLengthFuture(errResponse); 112 try { 113 fs.mkdirs(p); 114 Assert.fail(); 115 } catch (IOException ioe) {} // expected 116 Assert.assertEquals("0", getContentLength(future)); 117 } 118 119 @Test testPutOpWithRedirect()120 public void testPutOpWithRedirect() { 121 Future<String> future1 = contentLengthFuture(redirectResponse); 122 Future<String> future2 = contentLengthFuture(errResponse); 123 try { 124 FSDataOutputStream os = fs.create(p); 125 os.write(new byte[]{0}); 126 os.close(); 127 Assert.fail(); 128 } catch (IOException ioe) {} // expected 129 Assert.assertEquals("0", getContentLength(future1)); 130 Assert.assertEquals("chunked", getContentLength(future2)); 131 } 132 133 @Test testPostOp()134 public void testPostOp() { 135 Future<String> future = contentLengthFuture(errResponse); 136 try { 137 fs.concat(p, new Path[]{p}); 138 Assert.fail(); 139 } catch (IOException ioe) {} // expected 140 Assert.assertEquals("0", getContentLength(future)); 141 } 142 143 @Test testPostOpWithRedirect()144 public void testPostOpWithRedirect() { 145 // POST operation with redirect 146 Future<String> future1 = contentLengthFuture(redirectResponse); 147 Future<String> future2 = contentLengthFuture(errResponse); 148 try { 149 FSDataOutputStream os = fs.append(p); 150 os.write(new byte[]{0}); 151 os.close(); 152 Assert.fail(); 153 } catch (IOException ioe) {} // expected 154 Assert.assertEquals("0", getContentLength(future1)); 155 Assert.assertEquals("chunked", getContentLength(future2)); 156 } 157 158 @Test testDelete()159 public void testDelete() { 160 Future<String> future = contentLengthFuture(errResponse); 161 try { 162 fs.delete(p, false); 163 Assert.fail(); 164 } catch (IOException ioe) {} // expected 165 Assert.assertEquals(null, getContentLength(future)); 166 } 167 getContentLength(Future<String> future)168 private String getContentLength(Future<String> future) { 169 String request = null; 170 try { 171 request = future.get(2, TimeUnit.SECONDS); 172 } catch (Exception e) { 173 Assert.fail(e.toString()); 174 } 175 Matcher matcher = contentLengthPattern.matcher(request); 176 return matcher.find() ? matcher.group(2) : null; 177 } 178 contentLengthFuture(final String response)179 private Future<String> contentLengthFuture(final String response) { 180 return executor.submit(new Callable<String>() { 181 @Override 182 public String call() throws Exception { 183 Socket client = listenSocket.accept(); 184 client.setSoTimeout(2000); 185 try { 186 client.getOutputStream().write(response.getBytes()); 187 client.shutdownOutput(); 188 byte[] buf = new byte[4*1024]; // much bigger than request 189 int n = client.getInputStream().read(buf); 190 return new String(buf, 0, n); 191 } finally { 192 client.close(); 193 } 194 } 195 }); 196 } 197 } 198