1- /**
2- * Licensed to the Apache Software Foundation (ASF) under one or more
3- * contributor license agreements. See the NOTICE file distributed with this
4- * work for additional information regarding copyright ownership. The ASF
5- * licenses this file to you under the Apache License, Version 2.0 (the
6- * "License"); you may not use this file except in compliance with the License.
7- * You may obtain a copy of the License at
8- * <p>
9- * http://www.apache.org/licenses/LICENSE-2.0
10- * <p>
11- * Unless required by applicable law or agreed to in writing, software
12- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14- * License for the specific language governing permissions and limitations under
15- * the License.
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.
1617 */
18+
1719package org .apache .hadoop .hdds .scm .storage ;
1820
21+ import com .google .common .primitives .Bytes ;
1922import org .apache .hadoop .hdds .client .BlockID ;
2023import org .apache .hadoop .hdds .client .ContainerBlockID ;
21- import org .apache .hadoop .hdds .protocol .DatanodeDetails ;
22- import org .apache .hadoop .hdds .protocol .datanode .proto .ContainerProtos
23- .ChecksumData ;
24- import org .apache .hadoop .hdds .protocol .datanode .proto .ContainerProtos
25- .ChecksumType ;
24+ import org .apache .hadoop .hdds .protocol .datanode .proto .ContainerProtos .ChecksumType ;
2625import org .apache .hadoop .hdds .protocol .datanode .proto .ContainerProtos .ChunkInfo ;
2726import org .apache .hadoop .hdds .scm .XceiverClientManager ;
28- import org .apache .hadoop .hdds .scm .XceiverClientSpi ;
29- import org .apache .ratis .thirdparty .com .google .protobuf .ByteString ;
27+ import org .apache .hadoop .hdds .scm .pipeline .Pipeline ;
28+ import org .apache .hadoop .hdds .security .token .OzoneBlockTokenIdentifier ;
29+ import org .apache .hadoop .ozone .common .Checksum ;
30+ import org .apache .hadoop .security .token .Token ;
3031import org .junit .Assert ;
3132import org .junit .Before ;
3233import org .junit .Test ;
3334
3435import java .io .EOFException ;
3536import java .io .IOException ;
3637import java .util .ArrayList ;
38+ import java .util .HashMap ;
3739import java .util .List ;
40+ import java .util .Map ;
3841import java .util .Random ;
39- import java .util .UUID ;
42+
43+ import static org .apache .hadoop .hdds .scm .storage .TestChunkInputStream .generateRandomData ;
4044
4145/**
42- * Tests {@link BlockInputStream}.
46+ * Tests for {@link BlockInputStream}'s functionality .
4347 */
4448public class TestBlockInputStream {
4549
46- private static BlockInputStream blockInputStream ;
47- private static List <ChunkInfo > chunks ;
48- private static int blockSize ;
50+ private static final int CHUNK_SIZE = 100 ;
51+ private static Checksum checksum ;
4952
50- private static final int CHUNK_SIZE = 20 ;
53+ private BlockInputStream blockStream ;
54+ private byte [] blockData ;
55+ private int blockSize ;
56+ private List <ChunkInfo > chunks ;
57+ private Map <String , byte []> chunkDataMap ;
5158
5259 @ Before
5360 public void setup () throws Exception {
5461 BlockID blockID = new BlockID (new ContainerBlockID (1 , 1 ));
55- chunks = createChunkList (10 );
56- String traceID = UUID .randomUUID ().toString ();
57- blockInputStream = new DummyBlockInputStream (blockID , null , null , chunks ,
58- traceID , false , 0 );
59-
60- blockSize = 0 ;
61- for (ChunkInfo chunk : chunks ) {
62- blockSize += chunk .getLen ();
63- }
62+ checksum = new Checksum (ChecksumType .NONE , CHUNK_SIZE );
63+ createChunkList (5 );
64+
65+ blockStream = new DummyBlockInputStream (blockID , blockSize , null , null ,
66+ false , null , null );
6467 }
6568
6669 /**
6770 * Create a mock list of chunks. The first n-1 chunks of length CHUNK_SIZE
6871 * and the last chunk with length CHUNK_SIZE/2.
69- * @param numChunks
70- * @return
7172 */
72- private static List <ChunkInfo > createChunkList (int numChunks ) {
73- ChecksumData dummyChecksumData = ChecksumData .newBuilder ()
74- .setType (ChecksumType .NONE )
75- .setBytesPerChecksum (100 )
76- .build ();
77- List <ChunkInfo > chunkList = new ArrayList <>(numChunks );
78- int i ;
79- for (i = 0 ; i < numChunks - 1 ; i ++) {
80- String chunkName = "chunk-" + i ;
73+ private void createChunkList (int numChunks )
74+ throws Exception {
75+
76+ chunks = new ArrayList <>(numChunks );
77+ chunkDataMap = new HashMap <>();
78+ blockData = new byte [0 ];
79+ int i , chunkLen ;
80+ byte [] byteData ;
81+ String chunkName ;
82+
83+ for (i = 0 ; i < numChunks ; i ++) {
84+ chunkName = "chunk-" + i ;
85+ chunkLen = CHUNK_SIZE ;
86+ if (i == numChunks - 1 ) {
87+ chunkLen = CHUNK_SIZE / 2 ;
88+ }
89+ byteData = generateRandomData (chunkLen );
8190 ChunkInfo chunkInfo = ChunkInfo .newBuilder ()
8291 .setChunkName (chunkName )
8392 .setOffset (0 )
84- .setLen (CHUNK_SIZE )
85- .setChecksumData (dummyChecksumData )
93+ .setLen (chunkLen )
94+ .setChecksumData (checksum .computeChecksum (
95+ byteData , 0 , chunkLen ).getProtoBufMessage ())
8696 .build ();
87- chunkList .add (chunkInfo );
97+
98+ chunkDataMap .put (chunkName , byteData );
99+ chunks .add (chunkInfo );
100+
101+ blockSize += chunkLen ;
102+ blockData = Bytes .concat (blockData , byteData );
88103 }
89- ChunkInfo chunkInfo = ChunkInfo .newBuilder ()
90- .setChunkName ("chunk-" + i )
91- .setOffset (0 )
92- .setLen (CHUNK_SIZE /2 )
93- .setChecksumData (dummyChecksumData )
94- .build ();
95- chunkList .add (chunkInfo );
96-
97- return chunkList ;
98104 }
99105
100106 /**
101- * A dummy BlockInputStream to test the functionality of BlockInputStream .
107+ * A dummy BlockInputStream to mock read block call to DN .
102108 */
103- private static class DummyBlockInputStream extends BlockInputStream {
109+ private class DummyBlockInputStream extends BlockInputStream {
104110
105- DummyBlockInputStream (BlockID blockID ,
106- XceiverClientManager xceiverClientManager ,
107- XceiverClientSpi xceiverClient ,
108- List <ChunkInfo > chunks ,
109- String traceID ,
111+ DummyBlockInputStream (BlockID blockId ,
112+ long blockLen ,
113+ Pipeline pipeline ,
114+ Token <OzoneBlockTokenIdentifier > token ,
110115 boolean verifyChecksum ,
111- long initialPosition ) throws IOException {
112- super (blockID , xceiverClientManager , xceiverClient , chunks , traceID ,
113- verifyChecksum , initialPosition );
116+ String traceId ,
117+ XceiverClientManager xceiverClientManager ) {
118+ super (blockId , blockLen , pipeline , token , verifyChecksum ,
119+ traceId , xceiverClientManager );
114120 }
115121
116122 @ Override
117- protected ByteString readChunk (final ChunkInfo chunkInfo )
118- throws IOException {
119- return getByteString (chunkInfo .getChunkName (), (int ) chunkInfo .getLen ());
123+ protected List <ChunkInfo > getChunkInfos () {
124+ return chunks ;
120125 }
121126
122127 @ Override
123- protected List <DatanodeDetails > getDatanodeList () {
124- // return an empty dummy list of size 10
125- return new ArrayList <>(10 );
128+ protected void addStream (ChunkInfo chunkInfo ) {
129+ TestChunkInputStream testChunkInputStream = new TestChunkInputStream ();
130+ getChunkStreams ().add (testChunkInputStream .new DummyChunkInputStream (
131+ chunkInfo , null , null , null , false ,
132+ chunkDataMap .get (chunkInfo .getChunkName ()).clone ()));
126133 }
127134
128- /**
129- * Create ByteString with the input data to return when a readChunk call is
130- * placed.
131- */
132- private static ByteString getByteString (String data , int length ) {
133- while (data .length () < length ) {
134- data = data + "0" ;
135- }
136- return ByteString .copyFrom (data .getBytes (), 0 , length );
135+ @ Override
136+ protected synchronized void checkOpen () throws IOException {
137+ // No action needed
138+ }
139+ }
140+
141+ private void seekAndVerify (int pos ) throws Exception {
142+ blockStream .seek (pos );
143+ Assert .assertEquals ("Current position of buffer does not match with the " +
144+ "seeked position" , pos , blockStream .getPos ());
145+ }
146+
147+ /**
148+ * Match readData with the chunkData byte-wise.
149+ * @param readData Data read through ChunkInputStream
150+ * @param inputDataStartIndex first index (inclusive) in chunkData to compare
151+ * with read data
152+ * @param length the number of bytes of data to match starting from
153+ * inputDataStartIndex
154+ */
155+ private void matchWithInputData (byte [] readData , int inputDataStartIndex ,
156+ int length ) {
157+ for (int i = inputDataStartIndex ; i < inputDataStartIndex + length ; i ++) {
158+ Assert .assertEquals (blockData [i ], readData [i - inputDataStartIndex ]);
137159 }
138160 }
139161
@@ -143,25 +165,34 @@ public void testSeek() throws Exception {
143165 int pos = 0 ;
144166 seekAndVerify (pos );
145167 Assert .assertEquals ("ChunkIndex is incorrect" , 0 ,
146- blockInputStream .getChunkIndex ());
168+ blockStream .getChunkIndex ());
147169
170+ // Before BlockInputStream is initialized (initialization happens during
171+ // read operation), seek should update the BlockInputStream#blockPosition
148172 pos = CHUNK_SIZE ;
149173 seekAndVerify (pos );
174+ Assert .assertEquals ("ChunkIndex is incorrect" , 0 ,
175+ blockStream .getChunkIndex ());
176+ Assert .assertEquals (pos , blockStream .getBlockPosition ());
177+
178+ // Initialize the BlockInputStream. After initializtion, the chunkIndex
179+ // should be updated to correspond to the seeked position.
180+ blockStream .initialize ();
150181 Assert .assertEquals ("ChunkIndex is incorrect" , 1 ,
151- blockInputStream .getChunkIndex ());
182+ blockStream .getChunkIndex ());
152183
153- pos = (CHUNK_SIZE * 5 ) + 5 ;
184+ pos = (CHUNK_SIZE * 4 ) + 5 ;
154185 seekAndVerify (pos );
155- Assert .assertEquals ("ChunkIndex is incorrect" , 5 ,
156- blockInputStream .getChunkIndex ());
186+ Assert .assertEquals ("ChunkIndex is incorrect" , 4 ,
187+ blockStream .getChunkIndex ());
157188
158189 try {
159190 // Try seeking beyond the blockSize.
160191 pos = blockSize + 10 ;
161192 seekAndVerify (pos );
162193 Assert .fail ("Seek to position beyond block size should fail." );
163194 } catch (EOFException e ) {
164- // Expected
195+ System . out . println ( e );
165196 }
166197
167198 // Seek to random positions between 0 and the block size.
@@ -173,20 +204,32 @@ public void testSeek() throws Exception {
173204 }
174205
175206 @ Test
176- public void testBlockEOF () throws Exception {
177- // Seek to some position < blockSize and verify EOF is not reached.
178- seekAndVerify (CHUNK_SIZE );
179- Assert .assertFalse (blockInputStream .blockStreamEOF ());
180-
181- // Seek to blockSize-1 and verify that EOF is not reached as the chunk
182- // has not been read from container yet.
183- seekAndVerify (blockSize -1 );
184- Assert .assertFalse (blockInputStream .blockStreamEOF ());
207+ public void testRead () throws Exception {
208+ // read 200 bytes of data starting from position 50. Chunk0 contains
209+ // indices 0 to 99, chunk1 from 100 to 199 and chunk3 from 200 to 299. So
210+ // the read should result in 3 ChunkInputStream reads
211+ seekAndVerify (50 );
212+ byte [] b = new byte [200 ];
213+ blockStream .read (b , 0 , 200 );
214+ matchWithInputData (b , 50 , 200 );
215+
216+ // The new position of the blockInputStream should be the last index read
217+ // + 1.
218+ Assert .assertEquals (250 , blockStream .getPos ());
219+ Assert .assertEquals (2 , blockStream .getChunkIndex ());
185220 }
186221
187- private void seekAndVerify (int pos ) throws Exception {
188- blockInputStream .seek (pos );
189- Assert .assertEquals ("Current position of buffer does not match with the " +
190- "seeked position" , pos , blockInputStream .getPos ());
222+ @ Test
223+ public void testSeekAndRead () throws Exception {
224+ // Seek to a position and read data
225+ seekAndVerify (50 );
226+ byte [] b1 = new byte [100 ];
227+ blockStream .read (b1 , 0 , 100 );
228+ matchWithInputData (b1 , 50 , 100 );
229+
230+ // Next read should start from the position of the last read + 1 i.e. 100
231+ byte [] b2 = new byte [100 ];
232+ blockStream .read (b2 , 0 , 100 );
233+ matchWithInputData (b2 , 150 , 100 );
191234 }
192235}
0 commit comments