|
@@ -19,21 +19,26 @@
|
|
package org.apache.hadoop.ozone.container.keyvalue;
|
|
package org.apache.hadoop.ozone.container.keyvalue;
|
|
|
|
|
|
import com.google.common.primitives.Longs;
|
|
import com.google.common.primitives.Longs;
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.hdds.client.BlockID;
|
|
import org.apache.hadoop.hdds.client.BlockID;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
|
+import org.apache.hadoop.ozone.common.Checksum;
|
|
|
|
+import org.apache.hadoop.ozone.common.ChecksumData;
|
|
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
|
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
|
|
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
|
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
|
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
|
|
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
|
|
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
|
|
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
|
|
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
|
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
|
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
|
|
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
|
|
|
|
+import org.apache.hadoop.ozone.container.ozoneimpl.ContainerScrubberConfiguration;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
|
|
import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
@@ -42,7 +47,9 @@ import org.junit.Test;
|
|
import org.junit.runner.RunWith;
|
|
import org.junit.runner.RunWith;
|
|
import org.junit.runners.Parameterized;
|
|
import org.junit.runners.Parameterized;
|
|
|
|
|
|
|
|
+import java.io.ByteArrayOutputStream;
|
|
import java.io.File;
|
|
import java.io.File;
|
|
|
|
+import java.io.RandomAccessFile;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
@@ -55,7 +62,10 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB;
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
|
+
|
|
|
|
|
|
/**
|
|
/**
|
|
* Basic sanity test for the KeyValueContainerCheck class.
|
|
* Basic sanity test for the KeyValueContainerCheck class.
|
|
@@ -66,7 +76,7 @@ import static org.junit.Assert.assertTrue;
|
|
private KeyValueContainerData containerData;
|
|
private KeyValueContainerData containerData;
|
|
private ChunkManagerImpl chunkManager;
|
|
private ChunkManagerImpl chunkManager;
|
|
private VolumeSet volumeSet;
|
|
private VolumeSet volumeSet;
|
|
- private Configuration conf;
|
|
|
|
|
|
+ private OzoneConfiguration conf;
|
|
private File testRoot;
|
|
private File testRoot;
|
|
|
|
|
|
public TestKeyValueContainerCheck(String metadataImpl) {
|
|
public TestKeyValueContainerCheck(String metadataImpl) {
|
|
@@ -95,12 +105,15 @@ import static org.junit.Assert.assertTrue;
|
|
* Sanity test, when there are no corruptions induced.
|
|
* Sanity test, when there are no corruptions induced.
|
|
* @throws Exception
|
|
* @throws Exception
|
|
*/
|
|
*/
|
|
- @Test public void testKeyValueContainerCheckNoCorruption() throws Exception {
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testKeyValueContainerCheckNoCorruption() throws Exception {
|
|
long containerID = 101;
|
|
long containerID = 101;
|
|
int deletedBlocks = 1;
|
|
int deletedBlocks = 1;
|
|
int normalBlocks = 3;
|
|
int normalBlocks = 3;
|
|
int chunksPerBlock = 4;
|
|
int chunksPerBlock = 4;
|
|
boolean valid = false;
|
|
boolean valid = false;
|
|
|
|
+ ContainerScrubberConfiguration c = conf.getObject(
|
|
|
|
+ ContainerScrubberConfiguration.class);
|
|
|
|
|
|
// test Closed Container
|
|
// test Closed Container
|
|
createContainerWithBlocks(containerID, normalBlocks, deletedBlocks, 65536,
|
|
createContainerWithBlocks(containerID, normalBlocks, deletedBlocks, 65536,
|
|
@@ -120,10 +133,70 @@ import static org.junit.Assert.assertTrue;
|
|
container.close();
|
|
container.close();
|
|
|
|
|
|
// next run checks on a Closed Container
|
|
// next run checks on a Closed Container
|
|
- valid = kvCheck.fullCheck();
|
|
|
|
|
|
+ valid = kvCheck.fullCheck(new DataTransferThrottler(
|
|
|
|
+ c.getBandwidthPerVolume()), null);
|
|
assertTrue(valid);
|
|
assertTrue(valid);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Sanity test, when there are corruptions induced.
|
|
|
|
+ * @throws Exception
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testKeyValueContainerCheckCorruption() throws Exception {
|
|
|
|
+ long containerID = 102;
|
|
|
|
+ int deletedBlocks = 1;
|
|
|
|
+ int normalBlocks = 3;
|
|
|
|
+ int chunksPerBlock = 4;
|
|
|
|
+ boolean valid = false;
|
|
|
|
+ ContainerScrubberConfiguration sc = conf.getObject(
|
|
|
|
+ ContainerScrubberConfiguration.class);
|
|
|
|
+
|
|
|
|
+ // test Closed Container
|
|
|
|
+ createContainerWithBlocks(containerID, normalBlocks, deletedBlocks, 65536,
|
|
|
|
+ chunksPerBlock);
|
|
|
|
+ File chunksPath = new File(containerData.getChunksPath());
|
|
|
|
+ assertTrue(chunksPath.listFiles().length
|
|
|
|
+ == (deletedBlocks + normalBlocks) * chunksPerBlock);
|
|
|
|
+
|
|
|
|
+ container.close();
|
|
|
|
+
|
|
|
|
+ KeyValueContainerCheck kvCheck =
|
|
|
|
+ new KeyValueContainerCheck(containerData.getMetadataPath(), conf,
|
|
|
|
+ containerID);
|
|
|
|
+
|
|
|
|
+ File metaDir = new File(containerData.getMetadataPath());
|
|
|
|
+ File dbFile = KeyValueContainerLocationUtil
|
|
|
|
+ .getContainerDBFile(metaDir, containerID);
|
|
|
|
+ containerData.setDbFile(dbFile);
|
|
|
|
+ try(ReferenceCountedDB db =
|
|
|
|
+ BlockUtils.getDB(containerData, conf);
|
|
|
|
+ KeyValueBlockIterator kvIter = new KeyValueBlockIterator(containerID,
|
|
|
|
+ new File(containerData.getContainerPath()))) {
|
|
|
|
+ BlockData block = kvIter.nextBlock();
|
|
|
|
+ assertTrue(!block.getChunks().isEmpty());
|
|
|
|
+ ContainerProtos.ChunkInfo c = block.getChunks().get(0);
|
|
|
|
+ File chunkFile = ChunkUtils.getChunkFile(containerData,
|
|
|
|
+ ChunkInfo.getFromProtoBuf(c));
|
|
|
|
+ long length = chunkFile.length();
|
|
|
|
+ assertTrue(length > 0);
|
|
|
|
+ // forcefully truncate the file to induce failure.
|
|
|
|
+ try (RandomAccessFile file = new RandomAccessFile(chunkFile, "rws")) {
|
|
|
|
+ file.setLength(length / 2);
|
|
|
|
+ }
|
|
|
|
+ assertEquals(length/2, chunkFile.length());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // metadata check should pass.
|
|
|
|
+ valid = kvCheck.fastCheck();
|
|
|
|
+ assertTrue(valid);
|
|
|
|
+
|
|
|
|
+ // checksum validation should fail.
|
|
|
|
+ valid = kvCheck.fullCheck(new DataTransferThrottler(
|
|
|
|
+ sc.getBandwidthPerVolume()), null);
|
|
|
|
+ assertFalse(valid);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Creates a container with normal and deleted blocks.
|
|
* Creates a container with normal and deleted blocks.
|
|
* First it will insert normal blocks, and then it will insert
|
|
* First it will insert normal blocks, and then it will insert
|
|
@@ -134,12 +207,15 @@ import static org.junit.Assert.assertTrue;
|
|
* @throws Exception
|
|
* @throws Exception
|
|
*/
|
|
*/
|
|
private void createContainerWithBlocks(long containerId, int normalBlocks,
|
|
private void createContainerWithBlocks(long containerId, int normalBlocks,
|
|
- int deletedBlocks, long chunkLen, int chunksPerBlock) throws Exception {
|
|
|
|
|
|
+ int deletedBlocks, int chunkLen, int chunksPerBlock) throws Exception {
|
|
long chunkCount;
|
|
long chunkCount;
|
|
String strBlock = "block";
|
|
String strBlock = "block";
|
|
String strChunk = "-chunkFile";
|
|
String strChunk = "-chunkFile";
|
|
- byte[] chunkData = new byte[(int) chunkLen];
|
|
|
|
long totalBlks = normalBlocks + deletedBlocks;
|
|
long totalBlks = normalBlocks + deletedBlocks;
|
|
|
|
+ Checksum checksum = new Checksum(ContainerProtos.ChecksumType.SHA256,
|
|
|
|
+ chunkLen);
|
|
|
|
+ byte[] chunkData = generateRandomData(chunkLen);
|
|
|
|
+ ChecksumData checksumData = checksum.computeChecksum(chunkData);
|
|
|
|
|
|
containerData = new KeyValueContainerData(containerId,
|
|
containerData = new KeyValueContainerData(containerId,
|
|
(long) StorageUnit.BYTES.toBytes(
|
|
(long) StorageUnit.BYTES.toBytes(
|
|
@@ -166,8 +242,8 @@ import static org.junit.Assert.assertTrue;
|
|
chunkList.clear();
|
|
chunkList.clear();
|
|
for (chunkCount = 0; chunkCount < chunksPerBlock; chunkCount++) {
|
|
for (chunkCount = 0; chunkCount < chunksPerBlock; chunkCount++) {
|
|
String chunkName = strBlock + i + strChunk + chunkCount;
|
|
String chunkName = strBlock + i + strChunk + chunkCount;
|
|
- long offset = chunkCount * chunkLen;
|
|
|
|
- ChunkInfo info = new ChunkInfo(chunkName, offset, chunkLen);
|
|
|
|
|
|
+ ChunkInfo info = new ChunkInfo(chunkName, 0, chunkLen);
|
|
|
|
+ info.setChecksumData(checksumData);
|
|
chunkList.add(info.getProtoBufMessage());
|
|
chunkList.add(info.getProtoBufMessage());
|
|
chunkManager
|
|
chunkManager
|
|
.writeChunk(container, blockID, info, ByteBuffer.wrap(chunkData),
|
|
.writeChunk(container, blockID, info, ByteBuffer.wrap(chunkData),
|
|
@@ -195,4 +271,13 @@ import static org.junit.Assert.assertTrue;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private static byte[] generateRandomData(int length) {
|
|
|
|
+ assertTrue(length % 2 == 0);
|
|
|
|
+ ByteArrayOutputStream os = new ByteArrayOutputStream(length);
|
|
|
|
+ for (int i = 0; i < length; i++) {
|
|
|
|
+ os.write(i % 10);
|
|
|
|
+ }
|
|
|
|
+ return os.toByteArray();
|
|
|
|
+ }
|
|
}
|
|
}
|