001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.io.hfile; 019 020import static org.junit.Assert.assertArrayEquals; 021import static org.junit.Assert.assertFalse; 022import static org.junit.Assert.assertTrue; 023import static org.junit.Assert.fail; 024import static org.mockito.Mockito.mock; 025import static org.mockito.Mockito.verify; 026import static org.mockito.Mockito.verifyNoMoreInteractions; 027import static org.mockito.Mockito.when; 028 029import java.io.IOException; 030import java.nio.ByteBuffer; 031 032import org.apache.hadoop.fs.FSDataInputStream; 033import org.apache.hadoop.fs.FSDataOutputStream; 034import org.apache.hadoop.fs.FileSystem; 035import org.apache.hadoop.fs.Path; 036import org.apache.hadoop.hbase.HBaseClassTestRule; 037import org.apache.hadoop.hbase.HBaseTestingUtility; 038import org.apache.hadoop.hbase.io.util.BlockIOUtils; 039import org.apache.hadoop.hbase.nio.ByteBuff; 040import org.apache.hadoop.hbase.nio.MultiByteBuff; 041import org.apache.hadoop.hbase.nio.SingleByteBuff; 042import org.apache.hadoop.hbase.testclassification.IOTests; 043import org.apache.hadoop.hbase.testclassification.SmallTests; 044import org.apache.hadoop.hbase.util.Bytes; 045import org.junit.ClassRule; 046import org.junit.Rule; 047import org.junit.Test; 048import org.junit.experimental.categories.Category; 049import org.junit.rules.ExpectedException; 050 051@Category({ IOTests.class, SmallTests.class }) 052public class TestBlockIOUtils { 053 054 @ClassRule 055 public static final HBaseClassTestRule CLASS_RULE = 056 HBaseClassTestRule.forClass(TestBlockIOUtils.class); 057 058 @Rule 059 public ExpectedException exception = ExpectedException.none(); 060 061 private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 062 063 @Test 064 public void testIsByteBufferReadable() throws IOException { 065 FileSystem fs = TEST_UTIL.getTestFileSystem(); 066 Path p = new Path(TEST_UTIL.getDataTestDirOnTestFS(), "testIsByteBufferReadable"); 067 try (FSDataOutputStream out = fs.create(p)) { 068 out.writeInt(23); 069 } 070 try (FSDataInputStream is = fs.open(p)) { 071 assertFalse(BlockIOUtils.isByteBufferReadable(is)); 072 } 073 } 074 075 @Test 076 public void testReadFully() throws IOException { 077 FileSystem fs = TEST_UTIL.getTestFileSystem(); 078 Path p = new Path(TEST_UTIL.getDataTestDirOnTestFS(), "testReadFully"); 079 String s = "hello world"; 080 try (FSDataOutputStream out = fs.create(p)) { 081 out.writeBytes(s); 082 } 083 ByteBuff buf = new SingleByteBuff(ByteBuffer.allocate(11)); 084 try (FSDataInputStream in = fs.open(p)) { 085 BlockIOUtils.readFully(buf, in, 11); 086 } 087 buf.rewind(); 088 byte[] heapBuf = new byte[s.length()]; 089 buf.get(heapBuf, 0, heapBuf.length); 090 assertArrayEquals(Bytes.toBytes(s), heapBuf); 091 } 092 093 @Test 094 public void testReadWithExtra() throws IOException { 095 FileSystem fs = TEST_UTIL.getTestFileSystem(); 096 Path p = new Path(TEST_UTIL.getDataTestDirOnTestFS(), "testReadWithExtra"); 097 String s = "hello world"; 098 try (FSDataOutputStream out = fs.create(p)) { 099 out.writeBytes(s); 100 } 101 ByteBuff buf = new SingleByteBuff(ByteBuffer.allocate(8)); 102 try (FSDataInputStream in = fs.open(p)) { 103 assertTrue(BlockIOUtils.readWithExtra(buf, in, 6, 2)); 104 } 105 buf.rewind(); 106 byte[] heapBuf = new byte[buf.capacity()]; 107 buf.get(heapBuf, 0, heapBuf.length); 108 assertArrayEquals(Bytes.toBytes("hello wo"), heapBuf); 109 110 buf = new MultiByteBuff(ByteBuffer.allocate(4), ByteBuffer.allocate(4), ByteBuffer.allocate(4)); 111 try (FSDataInputStream in = fs.open(p)) { 112 assertTrue(BlockIOUtils.readWithExtra(buf, in, 8, 3)); 113 } 114 buf.rewind(); 115 heapBuf = new byte[11]; 116 buf.get(heapBuf, 0, heapBuf.length); 117 assertArrayEquals(Bytes.toBytes("hello world"), heapBuf); 118 119 buf.position(0).limit(12); 120 try (FSDataInputStream in = fs.open(p)) { 121 try { 122 BlockIOUtils.readWithExtra(buf, in, 12, 0); 123 fail("Should only read 11 bytes"); 124 } catch (IOException e) { 125 126 } 127 } 128 } 129 130 @Test 131 public void testPositionalReadNoExtra() throws IOException { 132 long position = 0; 133 int bufOffset = 0; 134 int necessaryLen = 10; 135 int extraLen = 0; 136 int totalLen = necessaryLen + extraLen; 137 byte[] buf = new byte[totalLen]; 138 ByteBuff bb = new SingleByteBuff(ByteBuffer.wrap(buf, 0, totalLen)); 139 FSDataInputStream in = mock(FSDataInputStream.class); 140 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(totalLen); 141 boolean ret = BlockIOUtils.preadWithExtra(bb, in, position, necessaryLen, extraLen); 142 assertFalse("Expect false return when no extra bytes requested", ret); 143 verify(in).read(position, buf, bufOffset, totalLen); 144 verifyNoMoreInteractions(in); 145 } 146 147 @Test 148 public void testPositionalReadShortReadOfNecessaryBytes() throws IOException { 149 long position = 0; 150 int bufOffset = 0; 151 int necessaryLen = 10; 152 int extraLen = 0; 153 int totalLen = necessaryLen + extraLen; 154 byte[] buf = new byte[totalLen]; 155 ByteBuff bb = new SingleByteBuff(ByteBuffer.wrap(buf, 0, totalLen)); 156 FSDataInputStream in = mock(FSDataInputStream.class); 157 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(5); 158 when(in.read(5, buf, 5, 5)).thenReturn(5); 159 boolean ret = BlockIOUtils.preadWithExtra(bb, in, position, necessaryLen, extraLen); 160 assertFalse("Expect false return when no extra bytes requested", ret); 161 verify(in).read(position, buf, bufOffset, totalLen); 162 verify(in).read(5, buf, 5, 5); 163 verifyNoMoreInteractions(in); 164 } 165 166 @Test 167 public void testPositionalReadExtraSucceeded() throws IOException { 168 long position = 0; 169 int bufOffset = 0; 170 int necessaryLen = 10; 171 int extraLen = 5; 172 int totalLen = necessaryLen + extraLen; 173 byte[] buf = new byte[totalLen]; 174 ByteBuff bb = new SingleByteBuff(ByteBuffer.wrap(buf, 0, totalLen)); 175 FSDataInputStream in = mock(FSDataInputStream.class); 176 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(totalLen); 177 boolean ret = BlockIOUtils.preadWithExtra(bb, in, position, necessaryLen, extraLen); 178 assertTrue("Expect true return when reading extra bytes succeeds", ret); 179 verify(in).read(position, buf, bufOffset, totalLen); 180 verifyNoMoreInteractions(in); 181 } 182 183 @Test 184 public void testPositionalReadExtraFailed() throws IOException { 185 long position = 0; 186 int bufOffset = 0; 187 int necessaryLen = 10; 188 int extraLen = 5; 189 int totalLen = necessaryLen + extraLen; 190 byte[] buf = new byte[totalLen]; 191 ByteBuff bb = new SingleByteBuff(ByteBuffer.wrap(buf, 0, totalLen)); 192 FSDataInputStream in = mock(FSDataInputStream.class); 193 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(necessaryLen); 194 boolean ret = BlockIOUtils.preadWithExtra(bb, in, position, necessaryLen, extraLen); 195 assertFalse("Expect false return when reading extra bytes fails", ret); 196 verify(in).read(position, buf, bufOffset, totalLen); 197 verifyNoMoreInteractions(in); 198 } 199 200 @Test 201 public void testPositionalReadShortReadCompletesNecessaryAndExtraBytes() 202 throws IOException { 203 long position = 0; 204 int bufOffset = 0; 205 int necessaryLen = 10; 206 int extraLen = 5; 207 int totalLen = necessaryLen + extraLen; 208 byte[] buf = new byte[totalLen]; 209 ByteBuff bb = new SingleByteBuff(ByteBuffer.wrap(buf, 0, totalLen)); 210 FSDataInputStream in = mock(FSDataInputStream.class); 211 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(5); 212 when(in.read(5, buf, 5, 10)).thenReturn(10); 213 boolean ret = BlockIOUtils.preadWithExtra(bb, in, position, necessaryLen, extraLen); 214 assertTrue("Expect true return when reading extra bytes succeeds", ret); 215 verify(in).read(position, buf, bufOffset, totalLen); 216 verify(in).read(5, buf, 5, 10); 217 verifyNoMoreInteractions(in); 218 } 219 220 @Test 221 public void testPositionalReadPrematureEOF() throws IOException { 222 long position = 0; 223 int bufOffset = 0; 224 int necessaryLen = 10; 225 int extraLen = 0; 226 int totalLen = necessaryLen + extraLen; 227 byte[] buf = new byte[totalLen]; 228 ByteBuff bb = new SingleByteBuff(ByteBuffer.wrap(buf, 0, totalLen)); 229 FSDataInputStream in = mock(FSDataInputStream.class); 230 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(9); 231 when(in.read(position, buf, bufOffset, totalLen)).thenReturn(-1); 232 exception.expect(IOException.class); 233 exception.expectMessage("EOF"); 234 BlockIOUtils.preadWithExtra(bb, in, position, necessaryLen, extraLen); 235 } 236}