This is an automated email from the ASF dual-hosted git repository.
bharat pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/trunk by this push:
new 106bdc6 HDDS-1120. Add a config to disable checksum verification during read.
106bdc6 is described below
commit 106bdc6c0474480654aa27cd7b8b50342f69e707
Author: Bharat Viswanadham <bharat@apache.org>
AuthorDate: Sat Feb 23 21:37:55 2019 -0800
HDDS-1120. Add a config to disable checksum verification during read.
---
.../hadoop/hdds/scm/storage/BlockInputStream.java | 10 ++-
.../org/apache/hadoop/ozone/OzoneConfigKeys.java | 3 +
.../common/src/main/resources/ozone-default.xml | 9 +++
.../hadoop/ozone/client/io/KeyInputStream.java | 4 +-
.../apache/hadoop/ozone/client/rpc/RpcClient.java | 6 +-
.../client/rpc/TestOzoneRpcClientAbstract.java | 91 ++++++++++++++++++++++
.../web/storage/DistributedStorageHandler.java | 7 +-
.../apache/hadoop/ozone/om/TestChunkStreams.java | 6 +-
8 files changed, 127 insertions(+), 9 deletions(-)
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
index 0e2c471..3ea36d4 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
@@ -66,6 +66,7 @@ public class BlockInputStream extends InputStream implements Seekable {
private long[] chunkOffset;
private List<ByteBuffer> buffers;
private int bufferIndex;
+ private final boolean verifyChecksum;
/**
* Creates a new BlockInputStream.
@@ -75,10 +76,12 @@ public class BlockInputStream extends InputStream implements Seekable
{
* @param xceiverClient client to perform container calls
* @param chunks list of chunks to read
* @param traceID container protocol call traceID
+ * @param verifyChecksum verify checksum
*/
public BlockInputStream(
BlockID blockID, XceiverClientManager xceiverClientManager,
- XceiverClientSpi xceiverClient, List<ChunkInfo> chunks, String traceID) {
+ XceiverClientSpi xceiverClient, List<ChunkInfo> chunks, String traceID,
+ boolean verifyChecksum) {
this.blockID = blockID;
this.traceID = traceID;
this.xceiverClientManager = xceiverClientManager;
@@ -91,6 +94,7 @@ public class BlockInputStream extends InputStream implements Seekable {
initializeChunkOffset();
this.buffers = null;
this.bufferIndex = 0;
+ this.verifyChecksum = verifyChecksum;
}
private void initializeChunkOffset() {
@@ -318,7 +322,9 @@ public class BlockInputStream extends InputStream implements Seekable
{
}
ChecksumData checksumData =
ChecksumData.getFromProtoBuf(chunkInfo.getChecksumData());
- Checksum.verifyChecksum(byteString, checksumData);
+ if (verifyChecksum) {
+ Checksum.verifyChecksum(byteString, checksumData);
+ }
break;
} catch (IOException ioe) {
// we will end up in this situation only if the checksum mismatch
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index 91f53f3..4ac6fda 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -362,6 +362,9 @@ public final class OzoneConfigKeys {
public static final int OZONE_CLIENT_BYTES_PER_CHECKSUM_DEFAULT_BYTES =
1024 * 1024;
public static final int OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE = 256 * 1024;
+ public static final String OZONE_CLIENT_VERIFY_CHECKSUM =
+ "ozone.client.verify.checksum";
+ public static final boolean OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT = true;
public static final String OZONE_ACL_AUTHORIZER_CLASS =
"ozone.acl.authorizer.class";
public static final String OZONE_ACL_AUTHORIZER_CLASS_DEFAULT =
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 7905f4b..e8ae5ed 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1485,6 +1485,15 @@
</property>
<property>
+ <name>ozone.client.verify.checksum</name>
+ <value>true</value>
+ <tag>OZONE, CLIENT, MANAGEMENT</tag>
+ <description>
+ Ozone client to verify checksum of the checksum blocksize data.
+ </description>
+ </property>
+
+ <property>
<name>ozone.om.ratis.enable</name>
<value>false</value>
<tag>OZONE, OM, RATIS, MANAGEMENT</tag>
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
index 642189f..59dd72b 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
@@ -265,7 +265,7 @@ public class KeyInputStream extends InputStream implements Seekable {
XceiverClientManager xceiverClientManager,
StorageContainerLocationProtocolClientSideTranslatorPB
storageContainerLocationClient,
- String requestId) throws IOException {
+ String requestId, boolean verifyChecksum) throws IOException {
long length = 0;
long containerKey;
KeyInputStream groupInputStream = new KeyInputStream();
@@ -311,7 +311,7 @@ public class KeyInputStream extends InputStream implements Seekable {
success = true;
BlockInputStream inputStream = new BlockInputStream(
omKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient,
- chunks, requestId);
+ chunks, requestId, verifyChecksum);
groupInputStream.addStream(inputStream,
omKeyLocationInfo.getLength());
} finally {
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 2c38569..d203e54 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -112,6 +112,7 @@ public class RpcClient implements ClientProtocol {
private final int chunkSize;
private final ChecksumType checksumType;
private final int bytesPerChecksum;
+ private boolean verifyChecksum;
private final UserGroupInformation ugi;
private final OzoneAcl.OzoneACLRights userRights;
private final OzoneAcl.OzoneACLRights groupRights;
@@ -198,6 +199,9 @@ public class RpcClient implements ClientProtocol {
OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE,
OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE_DEFAULT);
checksumType = ChecksumType.valueOf(checksumTypeStr);
+ this.verifyChecksum =
+ conf.getBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM,
+ OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT);
}
private InetSocketAddress getScmAddressForClient() throws IOException {
@@ -648,7 +652,7 @@ public class RpcClient implements ClientProtocol {
LengthInputStream lengthInputStream =
KeyInputStream.getFromOmKeyInfo(
keyInfo, xceiverClientManager, storageContainerLocationClient,
- requestId);
+ requestId, verifyChecksum);
FileEncryptionInfo feInfo = keyInfo.getFileEncryptionInfo();
if (feInfo != null) {
final KeyProvider.KeyVersion decrypted = getDEK(feInfo);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index d0c413a..95c5048 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -31,6 +31,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
@@ -47,6 +48,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.OzoneTestUtils;
import org.apache.hadoop.ozone.client.BucketArgs;
@@ -763,6 +765,95 @@ public abstract class TestOzoneRpcClientAbstract {
}
+
+ @Test
+ public void testReadKeyWithVerifyChecksumFlagEnable() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String keyName = UUID.randomUUID().toString();
+
+ // Create and corrupt key
+ createAndCorruptKey(volumeName, bucketName, keyName);
+
+ // read corrupt key with verify checksum enabled
+ readCorruptedKey(volumeName, bucketName, keyName, true);
+
+ }
+
+
+ @Test
+ public void testReadKeyWithVerifyChecksumFlagDisable() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String keyName = UUID.randomUUID().toString();
+
+ // Create and corrupt key
+ createAndCorruptKey(volumeName, bucketName, keyName);
+
+ // read corrupt key with verify checksum enabled
+ readCorruptedKey(volumeName, bucketName, keyName, false);
+
+ }
+
+ private void createAndCorruptKey(String volumeName, String bucketName,
+ String keyName) throws IOException {
+ String value = "sample value";
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ // Write data into a key
+ OzoneOutputStream out = bucket.createKey(keyName,
+ value.getBytes().length, ReplicationType.RATIS,
+ ReplicationFactor.ONE, new HashMap<>());
+ out.write(value.getBytes());
+ out.close();
+
+ // We need to find the location of the chunk file corresponding to the
+ // data we just wrote.
+ OzoneKey key = bucket.getKey(keyName);
+ long containerID = ((OzoneKeyDetails) key).getOzoneKeyLocations().get(0)
+ .getContainerID();
+
+ // Get the container by traversing the datanodes. Atleast one of the
+ // datanode must have this container.
+ Container container = null;
+ for (HddsDatanodeService hddsDatanode : cluster.getHddsDatanodes()) {
+ container = hddsDatanode.getDatanodeStateMachine().getContainer()
+ .getContainerSet().getContainer(containerID);
+ if (container != null) {
+ break;
+ }
+ }
+ Assert.assertNotNull("Container not found", container);
+ corruptData(container, key);
+ }
+
+
+ private void readCorruptedKey(String volumeName, String bucketName,
+ String keyName, boolean verifyChecksum) throws IOException {
+ try {
+ Configuration configuration = cluster.getConf();
+ configuration.setBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM,
+ verifyChecksum);
+ RpcClient client = new RpcClient(configuration);
+ OzoneInputStream is = client.getKey(volumeName, bucketName, keyName);
+ is.read(new byte[100]);
+ is.close();
+ if (verifyChecksum) {
+ fail("Reading corrupted data should fail, as verify checksum is " +
+ "enabled");
+ }
+ } catch (OzoneChecksumException e) {
+ if (!verifyChecksum) {
+ fail("Reading corrupted data should not fail, as verify checksum is " +
+ "disabled");
+ }
+ }
+ }
+
+
private void readKey(OzoneBucket bucket, String keyName, String data)
throws IOException {
OzoneKey key = bucket.getKey(keyName);
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
index 8197ce8..b1899d9 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
@@ -88,6 +88,7 @@ public final class DistributedStorageHandler implements StorageHandler {
private final long blockSize;
private final ChecksumType checksumType;
private final int bytesPerChecksum;
+ private final boolean verifyChecksum;
/**
* Creates a new DistributedStorageHandler.
@@ -153,7 +154,9 @@ public final class DistributedStorageHandler implements StorageHandler
{
OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE,
OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE_DEFAULT);
this.checksumType = ChecksumType.valueOf(checksumTypeStr);
-
+ this.verifyChecksum =
+ conf.getBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM,
+ OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT);
}
@Override
@@ -479,7 +482,7 @@ public final class DistributedStorageHandler implements StorageHandler
{
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
return KeyInputStream.getFromOmKeyInfo(
keyInfo, xceiverClientManager, storageContainerLocationClient,
- args.getRequestID());
+ args.getRequestID(), verifyChecksum);
}
@Override
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java
index f3ab093..e4e449b 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java
@@ -48,7 +48,8 @@ public class TestChunkStreams {
for (int i = 0; i < 5; i++) {
int tempOffset = offset;
BlockInputStream in =
- new BlockInputStream(null, null, null, new ArrayList<>(), null) {
+ new BlockInputStream(null, null, null, new ArrayList<>(), null,
+ true) {
private long pos = 0;
private ByteArrayInputStream in =
new ByteArrayInputStream(buf, tempOffset, 100);
@@ -104,7 +105,8 @@ public class TestChunkStreams {
for (int i = 0; i < 5; i++) {
int tempOffset = offset;
BlockInputStream in =
- new BlockInputStream(null, null, null, new ArrayList<>(), null) {
+ new BlockInputStream(null, null, null, new ArrayList<>(), null,
+ true) {
private long pos = 0;
private ByteArrayInputStream in =
new ByteArrayInputStream(buf, tempOffset, 100);
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org
|