diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/AbstractBlockChecksumComputer.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/AbstractBlockChecksumComputer.java new file mode 100644 index 000000000000..4be13e633148 --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/AbstractBlockChecksumComputer.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.client.checksum; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Base class for computing block checksum which is a function of chunk + * checksums. + */ +public abstract class AbstractBlockChecksumComputer { + private ByteBuffer outByteBuffer; + + /** + * Compute block checksum. The result can be obtained by getOutBytes(). + * @throws IOException + */ + public abstract void compute() throws IOException; + + public ByteBuffer getOutByteBuffer() { + return outByteBuffer; + } + + public void setOutBytes(byte[] bytes) { + this.outByteBuffer = ByteBuffer.wrap(bytes); + } +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/BaseFileChecksumHelper.java similarity index 97% rename from hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java rename to hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/BaseFileChecksumHelper.java index 1afbc8a3cf68..ece725f3f261 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/BaseFileChecksumHelper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.client; +package org.apache.hadoop.ozone.client.checksum; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.FileChecksum; @@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.MD5Hash; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.client.rpc.RpcClient; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedBlockChecksumComputer.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedBlockChecksumComputer.java new file mode 100644 index 000000000000..2d0e198dbcac --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedBlockChecksumComputer.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.client.checksum; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.io.MD5Hash; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; + +/** + * The implementation of AbstractBlockChecksumComputer for replicated blocks. + */ +public class ReplicatedBlockChecksumComputer extends + AbstractBlockChecksumComputer { + + private static final Logger LOG = + LoggerFactory.getLogger(ReplicatedBlockChecksumComputer.class); + + private List chunkInfoList; + + public ReplicatedBlockChecksumComputer( + List chunkInfoList) + throws IOException { + this.chunkInfoList = chunkInfoList; + } + + @Override + public void compute() throws IOException { + computeMd5Crc(); + } + + // compute the block checksum, which is the md5 of chunk checksums + private void computeMd5Crc() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + for (ContainerProtos.ChunkInfo chunkInfo : chunkInfoList) { + ContainerProtos.ChecksumData checksumData = + chunkInfo.getChecksumData(); + List checksums = checksumData.getChecksumsList(); + + for (ByteString checksum : checksums) { + baos.write(checksum.toByteArray()); + } + } + + MD5Hash fileMD5 = MD5Hash.digest(baos.toByteArray()); + setOutBytes(fileMD5.getDigest()); + + LOG.debug("number of chunks={}, md5out={}", + chunkInfoList.size(), fileMD5); + } +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ReplicatedFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java similarity index 87% rename from hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ReplicatedFileChecksumHelper.java rename to hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java index 168458a9802b..5b79eb870ffc 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ReplicatedFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.client; +package org.apache.hadoop.ozone.client.checksum; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.hdds.client.BlockID; @@ -28,11 +28,14 @@ import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.io.MD5Hash; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.rpc.RpcClient; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.security.token.Token; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; /** @@ -88,9 +91,10 @@ private boolean checksumBlock(OmKeyLocationInfo keyLocationInfo) int bytesPerChecksum = checksumData.getBytesPerChecksum(); setBytesPerCRC(bytesPerChecksum); - byte[] blockChecksum = getBlockChecksumFromChunkChecksums( + ByteBuffer blockChecksumByteBuffer = getBlockChecksumFromChunkChecksums( keyLocationInfo, chunkInfos); - String blockChecksumForDebug = populateBlockChecksumBuf(blockChecksum); + String blockChecksumForDebug = + populateBlockChecksumBuf(blockChecksumByteBuffer); LOG.debug("got reply from pipeline {} for block {}: blockChecksum={}, " + "blockChecksumType={}", @@ -148,29 +152,31 @@ protected List getChunkInfos( } // TODO: copy BlockChecksumHelper here - byte[] getBlockChecksumFromChunkChecksums(OmKeyLocationInfo keyLocationInfo, + ByteBuffer getBlockChecksumFromChunkChecksums( + OmKeyLocationInfo keyLocationInfo, List chunkInfoList) throws IOException { + AbstractBlockChecksumComputer blockChecksumComputer = + new ReplicatedBlockChecksumComputer(chunkInfoList); // TODO: support composite CRC - final int lenOfZeroBytes = 32; - byte[] emptyBlockMd5 = new byte[lenOfZeroBytes]; - MD5Hash fileMD5 = MD5Hash.digest(emptyBlockMd5); - return fileMD5.getDigest(); + blockChecksumComputer.compute(); + + return blockChecksumComputer.getOutByteBuffer(); } /** - * Parses out the raw blockChecksum bytes from {@code checksumData} - * according to the blockChecksumType and populates the cumulative + * Parses out the raw blockChecksum bytes from {@code checksumData} byte + * buffer according to the blockChecksumType and populates the cumulative * blockChecksumBuf with it. * * @return a debug-string representation of the parsed checksum if * debug is enabled, otherwise null. */ - String populateBlockChecksumBuf(byte[] checksumData) + String populateBlockChecksumBuf(ByteBuffer checksumData) throws IOException { String blockChecksumForDebug = null; //read md5 - final MD5Hash md5 = new MD5Hash(checksumData); + final MD5Hash md5 = new MD5Hash(checksumData.array()); md5.write(getBlockChecksumBuf()); if (LOG.isDebugEnabled()) { blockChecksumForDebug = md5.toString(); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/package-info.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/package-info.java new file mode 100644 index 000000000000..9345072f63fa --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/package-info.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.client.checksum; + +/** + * This package contains Ozone Client classes. + */ \ No newline at end of file diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/TestReplicatedBlockChecksumComputer.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/TestReplicatedBlockChecksumComputer.java new file mode 100644 index 000000000000..fba0773d15fb --- /dev/null +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/TestReplicatedBlockChecksumComputer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.client.checksum; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.io.MD5Hash; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Unit tests for ReplicatedBlockChecksumComputer class. + */ +public class TestReplicatedBlockChecksumComputer { + @Test + public void testComputeMd5Crc() throws IOException { + final int lenOfZeroBytes = 32; + byte[] emptyChunkChecksum = new byte[lenOfZeroBytes]; + MD5Hash emptyBlockMD5 = MD5Hash.digest(emptyChunkChecksum); + byte[] emptyBlockMD5Hash = emptyBlockMD5.getDigest(); + + ByteString checkSum = ByteString.copyFrom(emptyChunkChecksum); + + ContainerProtos.ChecksumData checksumData = + ContainerProtos.ChecksumData.newBuilder() + .addChecksums(checkSum) + .setBytesPerChecksum(4) + .setType(ContainerProtos.ChecksumType.CRC32) + .build(); + ContainerProtos.ChunkInfo chunkInfo = + ContainerProtos.ChunkInfo.newBuilder() + .setChecksumData(checksumData) + .setChunkName("dummy_chunk") + .setOffset(0) + .setLen(lenOfZeroBytes) + .build(); + List chunkInfoList = + Collections.singletonList(chunkInfo); + AbstractBlockChecksumComputer computer = + new ReplicatedBlockChecksumComputer(chunkInfoList); + + computer.compute(); + + ByteBuffer output = computer.getOutByteBuffer(); + assertArrayEquals(emptyBlockMD5Hash, output.array()); + } +} \ No newline at end of file diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestReplicatedFileChecksumHelper.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/TestReplicatedFileChecksumHelper.java similarity index 65% rename from hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestReplicatedFileChecksumHelper.java rename to hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/TestReplicatedFileChecksumHelper.java index dfb2ddc652c8..198602dd2d33 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestReplicatedFileChecksumHelper.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/TestReplicatedFileChecksumHelper.java @@ -15,12 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.client; +package org.apache.hadoop.ozone.client.checksum; import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.conf.InMemoryConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; @@ -30,44 +33,94 @@ import org.apache.hadoop.hdds.scm.XceiverClientReply; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.ozone.client.MockOmTransport; +import org.apache.hadoop.ozone.client.MockXceiverClientFactory; +import org.apache.hadoop.ozone.client.ObjectStore; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.rpc.RpcClient; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.om.protocolPB.OmTransport; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Time; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.jetbrains.annotations.NotNull; +import org.junit.After; +import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentMatchers; import org.mockito.Mockito; import java.io.IOException; +import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType.CRC32; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; /** * Unit tests for ReplicatedFileChecksumHelper class. */ public class TestReplicatedFileChecksumHelper { + private OzoneClient client; + private ObjectStore store; + private OzoneVolume volume; + private RpcClient rpcClient; + + @Before + public void init() throws IOException { + ConfigurationSource config = new InMemoryConfiguration(); + rpcClient = new RpcClient(config, null) { + + @Override + protected OmTransport createOmTransport( + String omServiceId) + throws IOException { + return new MockOmTransport(); + } + + @NotNull + @Override + protected XceiverClientFactory createXceiverClientFactory( + List x509Certificates) + throws IOException { + return new MockXceiverClientFactory(); + } + }; + client = new OzoneClient(config, rpcClient); + + store = client.getObjectStore(); + } + + @After + public void close() throws IOException { + client.close(); + } + @Test public void testEmptyBlock() throws IOException { // test the file checksum of a file with an empty block. - RpcClient rpcClient = Mockito.mock(RpcClient.class); + RpcClient mockRpcClient = Mockito.mock(RpcClient.class); OzoneManagerProtocol om = Mockito.mock(OzoneManagerProtocol.class); - when(rpcClient.getOzoneManagerClient()).thenReturn(om); + when(mockRpcClient.getOzoneManagerClient()).thenReturn(om); OmKeyInfo omKeyInfo = new OmKeyInfo.Builder() .setVolumeName(null) @@ -84,15 +137,15 @@ public void testEmptyBlock() throws IOException { .setAcls(null) .build(); - when(om.lookupKey(any())).thenReturn(omKeyInfo); + when(om.lookupKey(ArgumentMatchers.any())).thenReturn(omKeyInfo); - OzoneVolume volume = Mockito.mock(OzoneVolume.class); - when(volume.getName()).thenReturn("vol1"); + OzoneVolume mockVolume = Mockito.mock(OzoneVolume.class); + when(mockVolume.getName()).thenReturn("vol1"); OzoneBucket bucket = Mockito.mock(OzoneBucket.class); when(bucket.getName()).thenReturn("bucket1"); ReplicatedFileChecksumHelper helper = new ReplicatedFileChecksumHelper( - volume, bucket, "dummy", 10, rpcClient); + mockVolume, bucket, "dummy", 10, mockRpcClient); helper.compute(); FileChecksum fileChecksum = helper.getFileChecksum(); assertTrue(fileChecksum instanceof MD5MD5CRC32GzipFileChecksum); @@ -101,7 +154,7 @@ public void testEmptyBlock() throws IOException { // test negative length helper = new ReplicatedFileChecksumHelper( - volume, bucket, "dummy", -1, rpcClient); + mockVolume, bucket, "dummy", -1, mockRpcClient); helper.compute(); assertNull(helper.getKeyLocationInfoList()); } @@ -111,7 +164,7 @@ public void testOneBlock() throws IOException { // test the file checksum of a file with one block. OzoneConfiguration conf = new OzoneConfiguration(); - RpcClient rpcClient = Mockito.mock(RpcClient.class); + RpcClient mockRpcClient = Mockito.mock(RpcClient.class); List dns = Arrays.asList( DatanodeDetails.newBuilder().setUuid(UUID.randomUUID()).build()); @@ -124,7 +177,8 @@ public void testOneBlock() throws IOException { .setNodes(dns) .build(); - XceiverClientGrpc client = new XceiverClientGrpc(pipeline, conf) { + XceiverClientGrpc xceiverClientGrpc = + new XceiverClientGrpc(pipeline, conf) { @Override public XceiverClientReply sendCommandAsync( ContainerProtos.ContainerCommandRequestProto request, @@ -133,12 +187,13 @@ public XceiverClientReply sendCommandAsync( } }; XceiverClientFactory factory = Mockito.mock(XceiverClientFactory.class); - when(factory.acquireClientForReadData(any())).thenReturn(client); + when(factory.acquireClientForReadData(ArgumentMatchers.any())). + thenReturn(xceiverClientGrpc); - when(rpcClient.getXceiverClientManager()).thenReturn(factory); + when(mockRpcClient.getXceiverClientManager()).thenReturn(factory); OzoneManagerProtocol om = Mockito.mock(OzoneManagerProtocol.class); - when(rpcClient.getOzoneManagerClient()).thenReturn(om); + when(mockRpcClient.getOzoneManagerClient()).thenReturn(om); BlockID blockID = new BlockID(1, 1); OmKeyLocationInfo omKeyLocationInfo = @@ -164,15 +219,15 @@ public XceiverClientReply sendCommandAsync( .setAcls(null) .build(); - when(om.lookupKey(any())).thenReturn(omKeyInfo); + when(om.lookupKey(ArgumentMatchers.any())).thenReturn(omKeyInfo); - OzoneVolume volume = Mockito.mock(OzoneVolume.class); - when(volume.getName()).thenReturn("vol1"); + OzoneVolume mockVolume = Mockito.mock(OzoneVolume.class); + when(mockVolume.getName()).thenReturn("vol1"); OzoneBucket bucket = Mockito.mock(OzoneBucket.class); when(bucket.getName()).thenReturn("bucket1"); ReplicatedFileChecksumHelper helper = new ReplicatedFileChecksumHelper( - volume, bucket, "dummy", 10, rpcClient); + mockVolume, bucket, "dummy", 10, mockRpcClient); helper.compute(); FileChecksum fileChecksum = helper.getFileChecksum(); @@ -227,4 +282,42 @@ private XceiverClientReply buildValidResponse() { replyFuture.complete(resp); return new XceiverClientReply(replyFuture); } + + private OzoneBucket getOzoneBucket() throws IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + return volume.getBucket(bucketName); + } + + /** + * Write a real key and compute file checksum of it. + * @throws IOException + */ + @Test + public void testPutKeyChecksum() throws IOException { + String value = new String(new byte[1024], UTF_8); + OzoneBucket bucket = getOzoneBucket(); + + for (int i = 0; i < 1; i++) { + String keyName = UUID.randomUUID().toString(); + + try (OzoneOutputStream out = bucket + .createKey(keyName, value.getBytes(UTF_8).length, + ReplicationType.RATIS, ONE, new HashMap<>())) { + out.write(value.getBytes(UTF_8)); + out.write(value.getBytes(UTF_8)); + } + + ReplicatedFileChecksumHelper helper = new ReplicatedFileChecksumHelper( + volume, bucket, keyName, 10, rpcClient); + + helper.compute(); + FileChecksum fileChecksum = helper.getFileChecksum(); + assertTrue(fileChecksum instanceof MD5MD5CRC32GzipFileChecksum); + assertEquals(1, helper.getKeyLocationInfos().size()); + } + } } \ No newline at end of file diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/package-info.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/package-info.java new file mode 100644 index 000000000000..2ed26bf6aaec --- /dev/null +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/checksum/package-info.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.client.checksum; + +/** + * This package contains test classes for Ozone Client checksum APIs. + */ \ No newline at end of file