jojochuang commented on code in PR #3790:
URL: https://github.com/apache/ozone/pull/3790#discussion_r1022195052
##########
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java:
##########
@@ -384,6 +388,33 @@ private Stream<ECBlockOutputStream> dataStreams() {
.filter(Objects::nonNull);
}
+ public ByteString calculateChecksum() throws IOException {
+ if (blockOutputStreams == null) {
+ throw new IOException("Block Output Stream is null");
+ }
+
+ List<ContainerProtos.ChunkInfo> chunkInfos = new ArrayList<>();
+ // First chunk should always have the additional chunks in a partial
stripe.
+ int currentIdx = blockOutputStreams[0]
+ .getContainerBlockData().getChunksCount();
+ for (ECBlockOutputStream stream: blockOutputStreams) {
+ List<ContainerProtos.ChunkInfo> chunks =
+ stream.getContainerBlockData().getChunksList();
+ if (chunks.size() > currentIdx - 1) {
+ chunkInfos.add(chunks.get(currentIdx - 1));
Review Comment:
We should be able to eliminate the temporary array list chunkInfos, for sake
of performance. As a future item, we should do some kind of performance
benchmarks to understand the impact of file checksum.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileChecksum.java:
##########
@@ -0,0 +1,155 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+
+/**
+ * Test FileChecksum API.
+ */
+public class TestOzoneFileChecksum {
+
+ @Rule
+ public Timeout timeout = Timeout.seconds(100);
+
+ private static OzoneConfiguration conf;
+ private static MiniOzoneCluster cluster = null;
+ private static FileSystem fs;
+ private static RootedOzoneFileSystem ofs;
+ private static BasicRootedOzoneClientAdapterImpl adapter;
+ private static String rootPath;
+
+ @BeforeAll
+ public static void setup() throws IOException,
+ InterruptedException, TimeoutException {
+ conf = new OzoneConfiguration();
+ cluster = MiniOzoneCluster.newBuilder(conf)
+ .setNumDatanodes(5)
+ .build();
+ cluster.waitForClusterToBeReady();
+ rootPath = String.format("%s://%s/",
+ OzoneConsts.OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY));
+ conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);
+ fs = FileSystem.get(conf);
+ ofs = (RootedOzoneFileSystem) fs;
+ adapter = (BasicRootedOzoneClientAdapterImpl) ofs.getAdapter();
+ }
+
+ @AfterAll
+ public static void teardown() {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ IOUtils.closeQuietly(fs);
+ }
+
+ /**
+ * Test EC checksum with Replicated checksum.
Review Comment:
is this comment right?Also if we create the TestOzoneFileChecksum class to
test EC file checksum, it sounds reasonable to add the replicated file checksum
tests here. Just some thoughts.
I think, the only place FileSystem.getFileChecksum() is invoked for
replicated files is in TestOzoneShellHA.
##########
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java:
##########
@@ -384,6 +388,33 @@ private Stream<ECBlockOutputStream> dataStreams() {
.filter(Objects::nonNull);
}
+ public ByteString calculateChecksum() throws IOException {
+ if (blockOutputStreams == null) {
+ throw new IOException("Block Output Stream is null");
+ }
+
+ List<ContainerProtos.ChunkInfo> chunkInfos = new ArrayList<>();
+ // First chunk should always have the additional chunks in a partial
stripe.
+ int currentIdx = blockOutputStreams[0]
+ .getContainerBlockData().getChunksCount();
+ for (ECBlockOutputStream stream: blockOutputStreams) {
+ List<ContainerProtos.ChunkInfo> chunks =
+ stream.getContainerBlockData().getChunksList();
+ if (chunks.size() > currentIdx - 1) {
+ chunkInfos.add(chunks.get(currentIdx - 1));
+ }
+ }
+
+ ByteString checksum = ByteString.EMPTY;
+ for (ContainerProtos.ChunkInfo info : chunkInfos) {
+ for (ByteString byteString : info.getChecksumData().getChecksumsList()) {
+ checksum = checksum.concat(byteString);
Review Comment:
According to protobuf javadoc, ByteString.concat() involves no copying. So
this is okay.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileChecksum.java:
##########
@@ -0,0 +1,155 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+
+/**
+ * Test FileChecksum API.
+ */
+public class TestOzoneFileChecksum {
+
+ @Rule
+ public Timeout timeout = Timeout.seconds(100);
+
+ private static OzoneConfiguration conf;
+ private static MiniOzoneCluster cluster = null;
+ private static FileSystem fs;
+ private static RootedOzoneFileSystem ofs;
+ private static BasicRootedOzoneClientAdapterImpl adapter;
+ private static String rootPath;
+
+ @BeforeAll
+ public static void setup() throws IOException,
+ InterruptedException, TimeoutException {
+ conf = new OzoneConfiguration();
+ cluster = MiniOzoneCluster.newBuilder(conf)
+ .setNumDatanodes(5)
+ .build();
+ cluster.waitForClusterToBeReady();
+ rootPath = String.format("%s://%s/",
+ OzoneConsts.OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY));
+ conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);
+ fs = FileSystem.get(conf);
+ ofs = (RootedOzoneFileSystem) fs;
+ adapter = (BasicRootedOzoneClientAdapterImpl) ofs.getAdapter();
+ }
+
+ @AfterAll
+ public static void teardown() {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ IOUtils.closeQuietly(fs);
+ }
+
+ /**
+ * Test EC checksum with Replicated checksum.
+ */
+ @ParameterizedTest
+ @ValueSource(doubles = {0.5, 1, 1.5, 2, 3, 7, 8})
+ public void testFileChecksum(double size) throws IOException {
Review Comment:
Should we also verify the case when there's failure?
##########
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.fs.PathIOException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+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.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.CrcUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The helper class to compute file checksum for EC files.
+ */
+public class ECFileChecksumHelper extends BaseFileChecksumHelper {
+ private int blockIdx;
+
+ public ECFileChecksumHelper(OzoneVolume volume, OzoneBucket bucket,
+ String keyName, long length, OzoneClientConfig.ChecksumCombineMode
+ checksumCombineMode, ClientProtocol rpcClient, OmKeyInfo keyInfo)
+ throws IOException {
+ super(volume, bucket, keyName, length, checksumCombineMode, rpcClient,
+ keyInfo);
+ }
+
+ @Override
+ protected void checksumBlocks() throws IOException {
+ long currentLength = 0;
+ for (blockIdx = 0;
+ blockIdx < getKeyLocationInfoList().size() && getRemaining() >= 0;
+ blockIdx++) {
+ OmKeyLocationInfo keyLocationInfo =
+ getKeyLocationInfoList().get(blockIdx);
+
+ if (currentLength > getLength()) {
+ return;
+ }
+
+ if (!checksumBlock(keyLocationInfo)) {
+ throw new PathIOException(getSrc(),
+ "Fail to get block checksum for " + keyLocationInfo
+ + ", checksum combine mode : {}" + getCombineMode());
+ }
+
+ currentLength += keyLocationInfo.getLength();
+ }
+ }
+
+ private boolean checksumBlock(OmKeyLocationInfo keyLocationInfo)
+ throws IOException {
+ // for each block, send request
+ List<ContainerProtos.ChunkInfo> chunkInfos =
+ getChunkInfos(keyLocationInfo);
+ if (chunkInfos.size() == 0) {
+ return false;
+ }
+
+ long blockNumBytes = keyLocationInfo.getLength();
+
+ if (getRemaining() < blockNumBytes) {
+ blockNumBytes = getRemaining();
+ }
+ setRemaining(getRemaining() - blockNumBytes);
+
+ ContainerProtos.ChecksumData checksumData =
+ chunkInfos.get(0).getChecksumData();
+ setChecksumType(checksumData.getType());
+ int bytesPerChecksum = checksumData.getBytesPerChecksum();
+ setBytesPerCRC(bytesPerChecksum);
+
+ ByteBuffer blockChecksumByteBuffer =
+ getBlockChecksumFromChunkChecksums(chunkInfos);
+ String blockChecksumForDebug =
+ populateBlockChecksumBuf(blockChecksumByteBuffer);
+
+ LOG.debug("Got reply from EC pipeline {} for block {}: blockChecksum={}, "
+
+ "blockChecksumType={}",
+ keyLocationInfo.getPipeline(), keyLocationInfo.getBlockID(),
+ blockChecksumForDebug, checksumData.getType());
+ return true;
+ }
+
+ private String populateBlockChecksumBuf(
+ ByteBuffer blockChecksumByteBuffer) throws IOException {
+ String blockChecksumForDebug = null;
+ switch (getCombineMode()) {
+ case MD5MD5CRC:
+ final MD5Hash md5 = new MD5Hash(blockChecksumByteBuffer.array());
+ md5.write(getBlockChecksumBuf());
+ if (LOG.isDebugEnabled()) {
+ blockChecksumForDebug = md5.toString();
+ }
+ break;
+ case COMPOSITE_CRC:
+ byte[] crcBytes = blockChecksumByteBuffer.array();
+ if (LOG.isDebugEnabled()) {
+ blockChecksumForDebug = CrcUtil.toSingleCrcString(crcBytes);
+ }
+ getBlockChecksumBuf().write(crcBytes);
+ break;
+ default:
+ throw new IOException(
+ "Unknown combine mode: " + getCombineMode());
+ }
+
+ return blockChecksumForDebug;
+ }
+
+ private ByteBuffer getBlockChecksumFromChunkChecksums(
+ List<ContainerProtos.ChunkInfo> chunkInfos) throws IOException {
+
+ AbstractBlockChecksumComputer blockChecksumComputer =
+ new ECBlockChecksumComputer(chunkInfos, getKeyInfo());
+ blockChecksumComputer.compute(getCombineMode());
+
+ return blockChecksumComputer.getOutByteBuffer();
+ }
+
+ private List<ContainerProtos.ChunkInfo> getChunkInfos(OmKeyLocationInfo
+ keyLocationInfo) throws IOException {
+ // To read an EC block, we create a STANDALONE pipeline that contains the
+ // single location for the block index we want to read. The EC blocks are
+ // indexed from 1 to N, however the data locations are stored in the
+ // dataLocations array indexed from zero.
+ Token<OzoneBlockTokenIdentifier> token = keyLocationInfo.getToken();
+ BlockID blockID = keyLocationInfo.getBlockID();
+
+ Pipeline pipeline = keyLocationInfo.getPipeline();
+
+ List<DatanodeDetails> nodes = pipeline.getNodes();
+ List<DatanodeDetails> newNodes = new ArrayList<>();
+ ECReplicationConfig repConfig = (ECReplicationConfig)
+ pipeline.getReplicationConfig();
+ int totalNodes = repConfig.getRequiredNodes();
+ int parity = repConfig.getParity();
+
+ // Filtering the nodes that has the checksumBytes
+ for (int i = 0; i < nodes.size(); i++) {
+ if (i > 0 && i < totalNodes - parity) {
+ continue;
+ }
+ newNodes.add(nodes.get(i));
+ }
+
+ pipeline = Pipeline.newBuilder(pipeline)
+ .setReplicationConfig(StandaloneReplicationConfig
Review Comment:
Not familiar with EC implementation. Why is standalone type used instead of
EC type?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]