Skip to content

Commit a91d24f

Browse files
HDDS-1496. Support partial chunk reads and checksum verification (#804)
1 parent 0b115b6 commit a91d24f

File tree

10 files changed

+1377
-742
lines changed

10 files changed

+1377
-742
lines changed

hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java

Lines changed: 267 additions & 367 deletions
Large diffs are not rendered by default.

hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java

Lines changed: 546 additions & 0 deletions
Large diffs are not rendered by default.
Lines changed: 147 additions & 104 deletions
Original file line numberDiff line numberDiff line change
@@ -1,139 +1,161 @@
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+
1719
package org.apache.hadoop.hdds.scm.storage;
1820

21+
import com.google.common.primitives.Bytes;
1922
import org.apache.hadoop.hdds.client.BlockID;
2023
import 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;
2625
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
2726
import 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;
3031
import org.junit.Assert;
3132
import org.junit.Before;
3233
import org.junit.Test;
3334

3435
import java.io.EOFException;
3536
import java.io.IOException;
3637
import java.util.ArrayList;
38+
import java.util.HashMap;
3739
import java.util.List;
40+
import java.util.Map;
3841
import 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
*/
4448
public 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

Comments
 (0)