Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions hadoop-ozone/client/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,198 @@
/*
* 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;

import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
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.protocol.ClientProtocol;
import org.apache.hadoop.ozone.client.rpc.RpcClient;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.List;

/**
* The base class to support file checksum.
*/
public abstract class BaseFileChecksumHelper {
static final Logger LOG =
LoggerFactory.getLogger(BaseFileChecksumHelper.class);

private OzoneVolume volume;
private OzoneBucket bucket;
private String keyName;
private final long length;
private ClientProtocol rpcClient;

private XceiverClientFactory xceiverClientFactory;
private final DataOutputBuffer blockChecksumBuf = new DataOutputBuffer();
private FileChecksum fileChecksum;
private List<OmKeyLocationInfo> keyLocationInfos;
private long remaining = 0L;
private int bytesPerCRC = -1;
private long crcPerBlock = 0;

// initialization
BaseFileChecksumHelper(
OzoneVolume volume, OzoneBucket bucket, String keyName,
long length, ClientProtocol rpcClient) throws IOException {

this.volume = volume;
this.bucket = bucket;
this.keyName = keyName;
this.length = length;
this.rpcClient = rpcClient;
this.xceiverClientFactory =
((RpcClient)rpcClient).getXceiverClientManager();
if (this.length > 0) {
fetchBlocks();
}
}

protected String getSrc() {
return "Volume: " + volume.getName() + " Bucket: " + bucket.getName() + " "
+ keyName;
}

protected long getLength() {
return length;
}

protected ClientProtocol getRpcClient() {
return rpcClient;
}

protected XceiverClientFactory getXceiverClientFactory() {
return xceiverClientFactory;
}

protected DataOutputBuffer getBlockChecksumBuf() {
return blockChecksumBuf;
}

protected List<OmKeyLocationInfo> getKeyLocationInfoList() {
return keyLocationInfos;
}

protected long getRemaining() {
return remaining;
}

protected void setRemaining(long remaining) {
this.remaining = remaining;
}

int getBytesPerCRC() {
return bytesPerCRC;
}

protected void setBytesPerCRC(int bytesPerCRC) {
this.bytesPerCRC = bytesPerCRC;
}

/**
* Request the blocks created in the most recent version from Ozone Manager.
*
* @throws IOException
*/
private void fetchBlocks() throws IOException {
OzoneManagerProtocol ozoneManagerClient =
getRpcClient().getOzoneManagerClient();
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volume.getName())
.setBucketName(bucket.getName())
.setKeyName(keyName)
.setRefreshPipeline(true)
.setSortDatanodesInPipeline(true)
.setLatestVersionLocation(true)
.build();
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);

// use OmKeyArgs to call Om.lookup() and get OmKeyInfo
keyLocationInfos = keyInfo
.getLatestVersionLocations().getBlocksLatestVersionOnly();
}

/**
* Compute file checksum given the list of chunk checksums requested earlier.
* @throws IOException
*/
public void compute() throws IOException {
/**
* request length is 0 or the file is empty, return one with the
* magic entry that matches the md5 of a 32 byte zero-padded byte array.
*/
if (keyLocationInfos == null || keyLocationInfos.isEmpty()) {
// Explicitly specified here in case the default DataOutputBuffer
// buffer length value is changed in future.
final int lenOfZeroBytes = 32;
byte[] emptyBlockMd5 = new byte[lenOfZeroBytes];
MD5Hash fileMD5 = MD5Hash.digest(emptyBlockMd5);
fileChecksum = new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
} else {
checksumBlocks();
fileChecksum = makeFinalResult();
}
}

@VisibleForTesting
List<OmKeyLocationInfo> getKeyLocationInfos() {
return keyLocationInfos;
}


/**
* Compute block checksums block by block and append the raw bytes of the
* block checksums into getBlockChecksumBuf().
*
* @throws IOException
*/
protected abstract void checksumBlocks() throws IOException;

/**
* Make final file checksum result given the per-block or per-block-group
* checksums collected into getBlockChecksumBuf().
*/
private FileChecksum makeFinalResult() throws IOException {
// TODO: support composite CRC
return makeMd5CrcResult();
}

private FileChecksum makeMd5CrcResult() {
// TODO: support CRC32C
//compute file MD5
final MD5Hash fileMD5 = MD5Hash.digest(getBlockChecksumBuf().getData());
// assume CRC32 for now
return new MD5MD5CRC32GzipFileChecksum(getBytesPerCRC(),
crcPerBlock, fileMD5);
}

public FileChecksum getFileChecksum() {
return fileChecksum;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,181 @@
/*
* 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;

import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
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.rpc.RpcClient;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.security.token.Token;

import java.io.IOException;
import java.util.List;

/**
* The helper class to compute file checksum for replicated files.
*/
public class ReplicatedFileChecksumHelper extends BaseFileChecksumHelper {
private int blockIdx;

ReplicatedFileChecksumHelper(
OzoneVolume volume, OzoneBucket bucket, String keyName, long length,
RpcClient rpcClient) throws IOException {
super(volume, bucket, keyName, length, rpcClient);
}

@Override
protected void checksumBlocks() throws IOException {
long currentLength = 0;
for (blockIdx = 0;
blockIdx < getKeyLocationInfoList().size() && getRemaining() >= 0;
blockIdx++) {
OmKeyLocationInfo keyLocationInfo =
getKeyLocationInfoList().get(blockIdx);
currentLength += keyLocationInfo.getLength();
if (currentLength > getLength()) {
return;
}

if (!checksumBlock(keyLocationInfo)) {
throw new PathIOException(
getSrc(), "Fail to get block MD5 for " + keyLocationInfo);
}
}
}

/**
* Return true when sounds good to continue or retry, false when severe
* condition or totally failed.
*/
private boolean checksumBlock(OmKeyLocationInfo keyLocationInfo)
throws IOException {

long blockNumBytes = keyLocationInfo.getLength();

if (getRemaining() < blockNumBytes) {
blockNumBytes = getRemaining();
}
setRemaining(getRemaining() - blockNumBytes);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the role of remaining?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We obtain the the metadata of blocks after blocks, until remaining is < 0. Remaining is decremented by the block length each time.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems we are not setting remaining to key length initially.

// for each block, send request
List<ContainerProtos.ChunkInfo> chunkInfos =
getChunkInfos(keyLocationInfo);
ContainerProtos.ChecksumData checksumData =
chunkInfos.get(0).getChecksumData();
int bytesPerChecksum = checksumData.getBytesPerChecksum();
setBytesPerCRC(bytesPerChecksum);

byte[] blockChecksum = getBlockChecksumFromChunkChecksums(
keyLocationInfo, chunkInfos);
String blockChecksumForDebug = populateBlockChecksumBuf(blockChecksum);

LOG.debug("got reply from pipeline {} for block {}: blockChecksum={}, " +
"blockChecksumType={}",
keyLocationInfo.getPipeline(), keyLocationInfo.getBlockID(),
blockChecksumForDebug, checksumData.getType());
return true;
}

// copied from BlockInputStream
/**
* Send RPC call to get the block info from the container.
* @return List of chunks in this block.
*/
protected List<ContainerProtos.ChunkInfo> getChunkInfos(
OmKeyLocationInfo keyLocationInfo) throws IOException {
// irrespective of the container state, we will always read via Standalone
// protocol.
Token<OzoneBlockTokenIdentifier> token = keyLocationInfo.getToken();
Pipeline pipeline = keyLocationInfo.getPipeline();
BlockID blockID = keyLocationInfo.getBlockID();
if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE) {
pipeline = Pipeline.newBuilder(pipeline)
.setReplicationConfig(new StandaloneReplicationConfig(
ReplicationConfig
.getLegacyFactor(pipeline.getReplicationConfig())))
.build();
}

boolean success = false;
List<ContainerProtos.ChunkInfo> chunks;
XceiverClientSpi xceiverClientSpi = null;
try {
if (LOG.isDebugEnabled()) {
LOG.debug("Initializing BlockInputStream for get key to access {}",
blockID.getContainerID());
}
xceiverClientSpi =
getXceiverClientFactory().acquireClientForReadData(pipeline);

ContainerProtos.DatanodeBlockID datanodeBlockID = blockID
.getDatanodeBlockIDProtobuf();
ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls
.getBlock(xceiverClientSpi, datanodeBlockID, token);

chunks = response.getBlockData().getChunksList();
success = true;
} finally {
if (!success && xceiverClientSpi != null) {
getXceiverClientFactory().releaseClientForReadData(
xceiverClientSpi, false);
}
}

return chunks;
}

// TODO: copy BlockChecksumHelper here
byte[] getBlockChecksumFromChunkChecksums(OmKeyLocationInfo keyLocationInfo,
List<ContainerProtos.ChunkInfo> chunkInfoList)
throws IOException {
// TODO: support composite CRC
final int lenOfZeroBytes = 32;
byte[] emptyBlockMd5 = new byte[lenOfZeroBytes];
MD5Hash fileMD5 = MD5Hash.digest(emptyBlockMd5);
return fileMD5.getDigest();
}

/**
* Parses out the raw blockChecksum bytes from {@code checksumData}
* 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)
throws IOException {
String blockChecksumForDebug = null;
//read md5
final MD5Hash md5 = new MD5Hash(checksumData);
md5.write(getBlockChecksumBuf());
if (LOG.isDebugEnabled()) {
blockChecksumForDebug = md5.toString();
}

return blockChecksumForDebug;
}
}
Loading