HDFS-9719. Refactoring ErasureCodingWorker into smaller reusable constructs. Contributed by Kai Zheng.
This commit is contained in:
parent
8d29e2451f
commit
3c18a53cbd
|
@ -141,6 +141,12 @@ public class StripedBlockUtil {
|
||||||
return locatedBlock;
|
return locatedBlock;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static ExtendedBlock constructInternalBlock(
|
||||||
|
ExtendedBlock blockGroup, ErasureCodingPolicy ecPolicy,
|
||||||
|
int idxInBlockGroup) {
|
||||||
|
return constructInternalBlock(blockGroup, ecPolicy.getCellSize(),
|
||||||
|
ecPolicy.getNumDataUnits(), idxInBlockGroup);
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* This method creates an internal {@link ExtendedBlock} at the given index
|
* This method creates an internal {@link ExtendedBlock} at the given index
|
||||||
* of a block group.
|
* of a block group.
|
||||||
|
@ -154,21 +160,28 @@ public class StripedBlockUtil {
|
||||||
return block;
|
return block;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static long getInternalBlockLength(long dataSize,
|
||||||
|
ErasureCodingPolicy ecPolicy,
|
||||||
|
int idxInBlockGroup) {
|
||||||
|
return getInternalBlockLength(dataSize, ecPolicy.getCellSize(),
|
||||||
|
ecPolicy.getNumDataUnits(), idxInBlockGroup);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the size of an internal block at the given index of a block group
|
* Get the size of an internal block at the given index of a block group
|
||||||
*
|
*
|
||||||
* @param dataSize Size of the block group only counting data blocks
|
* @param dataSize Size of the block group only counting data blocks
|
||||||
* @param cellSize The size of a striping cell
|
* @param cellSize The size of a striping cell
|
||||||
* @param numDataBlocks The number of data blocks
|
* @param numDataBlocks The number of data blocks
|
||||||
* @param i The logical index in the striped block group
|
* @param idxInBlockGroup The logical index in the striped block group
|
||||||
* @return The size of the internal block at the specified index
|
* @return The size of the internal block at the specified index
|
||||||
*/
|
*/
|
||||||
public static long getInternalBlockLength(long dataSize,
|
public static long getInternalBlockLength(long dataSize,
|
||||||
int cellSize, int numDataBlocks, int i) {
|
int cellSize, int numDataBlocks, int idxInBlockGroup) {
|
||||||
Preconditions.checkArgument(dataSize >= 0);
|
Preconditions.checkArgument(dataSize >= 0);
|
||||||
Preconditions.checkArgument(cellSize > 0);
|
Preconditions.checkArgument(cellSize > 0);
|
||||||
Preconditions.checkArgument(numDataBlocks > 0);
|
Preconditions.checkArgument(numDataBlocks > 0);
|
||||||
Preconditions.checkArgument(i >= 0);
|
Preconditions.checkArgument(idxInBlockGroup >= 0);
|
||||||
// Size of each stripe (only counting data blocks)
|
// Size of each stripe (only counting data blocks)
|
||||||
final int stripeSize = cellSize * numDataBlocks;
|
final int stripeSize = cellSize * numDataBlocks;
|
||||||
// If block group ends at stripe boundary, each internal block has an equal
|
// If block group ends at stripe boundary, each internal block has an equal
|
||||||
|
@ -180,7 +193,8 @@ public class StripedBlockUtil {
|
||||||
|
|
||||||
final int numStripes = (int) ((dataSize - 1) / stripeSize + 1);
|
final int numStripes = (int) ((dataSize - 1) / stripeSize + 1);
|
||||||
return (numStripes - 1L)*cellSize
|
return (numStripes - 1L)*cellSize
|
||||||
+ lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
|
+ lastCellSize(lastStripeDataLen, cellSize,
|
||||||
|
numDataBlocks, idxInBlockGroup);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,202 @@
|
||||||
|
/**
|
||||||
|
* 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.hdfs.server.datanode.erasurecode;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.ChecksumException;
|
||||||
|
import org.apache.hadoop.hdfs.BlockReader;
|
||||||
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
||||||
|
import org.apache.hadoop.hdfs.DFSUtilClient.CorruptedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.RemoteBlockReader2;
|
||||||
|
import org.apache.hadoop.hdfs.net.Peer;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.net.Socket;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* StripedBlockReader is used to read block data from one source DN, it contains
|
||||||
|
* a block reader, read buffer and striped block index.
|
||||||
|
* Only allocate StripedBlockReader once for one source, and the StripedReader
|
||||||
|
* has the same array order with sources. Typically we only need to allocate
|
||||||
|
* minimum number (minRequiredSources) of StripedReader, and allocate
|
||||||
|
* new for new source DN if some existing DN invalid or slow.
|
||||||
|
* If some source DN is corrupt, set the corresponding blockReader to
|
||||||
|
* null and will never read from it again.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class StripedBlockReader {
|
||||||
|
private static final Logger LOG = DataNode.LOG;
|
||||||
|
|
||||||
|
private StripedReader stripedReader;
|
||||||
|
private final DataNode datanode;
|
||||||
|
private final Configuration conf;
|
||||||
|
|
||||||
|
private final short index; // internal block index
|
||||||
|
private final ExtendedBlock block;
|
||||||
|
private final DatanodeInfo source;
|
||||||
|
private BlockReader blockReader;
|
||||||
|
private ByteBuffer buffer;
|
||||||
|
|
||||||
|
StripedBlockReader(StripedReader stripedReader, DataNode datanode,
|
||||||
|
Configuration conf, short index, ExtendedBlock block,
|
||||||
|
DatanodeInfo source, long offsetInBlock) {
|
||||||
|
this.stripedReader = stripedReader;
|
||||||
|
this.datanode = datanode;
|
||||||
|
this.conf = conf;
|
||||||
|
|
||||||
|
this.index = index;
|
||||||
|
this.source = source;
|
||||||
|
this.block = block;
|
||||||
|
|
||||||
|
BlockReader tmpBlockReader = createBlockReader(offsetInBlock);
|
||||||
|
if (tmpBlockReader != null) {
|
||||||
|
this.blockReader = tmpBlockReader;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
ByteBuffer getReadBuffer() {
|
||||||
|
if (buffer == null) {
|
||||||
|
this.buffer = stripedReader.allocateReadBuffer();
|
||||||
|
}
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
void resetBlockReader(long offsetInBlock) {
|
||||||
|
this.blockReader = createBlockReader(offsetInBlock);
|
||||||
|
}
|
||||||
|
|
||||||
|
private BlockReader createBlockReader(long offsetInBlock) {
|
||||||
|
if (offsetInBlock >= block.getNumBytes()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
InetSocketAddress dnAddr =
|
||||||
|
stripedReader.getSocketAddress4Transfer(source);
|
||||||
|
Token<BlockTokenIdentifier> blockToken = datanode.getBlockAccessToken(
|
||||||
|
block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ));
|
||||||
|
/*
|
||||||
|
* This can be further improved if the replica is local, then we can
|
||||||
|
* read directly from DN and need to check the replica is FINALIZED
|
||||||
|
* state, notice we should not use short-circuit local read which
|
||||||
|
* requires config for domain-socket in UNIX or legacy config in
|
||||||
|
* Windows. The network distance value isn't used for this scenario.
|
||||||
|
*
|
||||||
|
* TODO: add proper tracer
|
||||||
|
*/
|
||||||
|
return RemoteBlockReader2.newBlockReader(
|
||||||
|
"dummy", block, blockToken, offsetInBlock,
|
||||||
|
block.getNumBytes() - offsetInBlock, true,
|
||||||
|
"", newConnectedPeer(block, dnAddr, blockToken, source), source,
|
||||||
|
null, stripedReader.getCachingStrategy(), datanode.getTracer(), -1);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.debug("Exception while creating remote block reader, datanode {}",
|
||||||
|
source, e);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
|
||||||
|
Token<BlockTokenIdentifier> blockToken,
|
||||||
|
DatanodeID datanodeId)
|
||||||
|
throws IOException {
|
||||||
|
Peer peer = null;
|
||||||
|
boolean success = false;
|
||||||
|
Socket sock = null;
|
||||||
|
final int socketTimeout = datanode.getDnConf().getSocketTimeout();
|
||||||
|
try {
|
||||||
|
sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||||
|
NetUtils.connect(sock, addr, socketTimeout);
|
||||||
|
peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(),
|
||||||
|
sock, datanode.getDataEncryptionKeyFactoryForBlock(b),
|
||||||
|
blockToken, datanodeId, socketTimeout);
|
||||||
|
success = true;
|
||||||
|
return peer;
|
||||||
|
} finally {
|
||||||
|
if (!success) {
|
||||||
|
IOUtils.cleanup(null, peer);
|
||||||
|
IOUtils.closeSocket(sock);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Callable<Void> readFromBlock(final int length,
|
||||||
|
final CorruptedBlocks corruptedBlocks) {
|
||||||
|
return new Callable<Void>() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Void call() throws Exception {
|
||||||
|
try {
|
||||||
|
getReadBuffer().limit(length);
|
||||||
|
actualReadFromBlock();
|
||||||
|
return null;
|
||||||
|
} catch (ChecksumException e) {
|
||||||
|
LOG.warn("Found Checksum error for {} from {} at {}", block,
|
||||||
|
source, e.getPos());
|
||||||
|
corruptedBlocks.addCorruptedBlock(block, source);
|
||||||
|
throw e;
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.info(e.getMessage());
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Perform actual reading of bytes from block.
|
||||||
|
*/
|
||||||
|
private void actualReadFromBlock() throws IOException {
|
||||||
|
int len = buffer.remaining();
|
||||||
|
int n = 0;
|
||||||
|
while (n < len) {
|
||||||
|
int nread = blockReader.read(buffer);
|
||||||
|
if (nread <= 0) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
n += nread;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// close block reader
|
||||||
|
void closeBlockReader() {
|
||||||
|
IOUtils.closeStream(blockReader);
|
||||||
|
blockReader = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
short getIndex() {
|
||||||
|
return index;
|
||||||
|
}
|
||||||
|
|
||||||
|
BlockReader getBlockReader() {
|
||||||
|
return blockReader;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,196 @@
|
||||||
|
/**
|
||||||
|
* 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.hdfs.server.datanode.erasurecode;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.StorageType;
|
||||||
|
import org.apache.hadoop.hdfs.DFSPacket;
|
||||||
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
|
||||||
|
import java.io.BufferedOutputStream;
|
||||||
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.net.Socket;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A striped block writer that writes reconstructed data to the remote target
|
||||||
|
* datanode.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class StripedBlockWriter {
|
||||||
|
private final StripedWriter stripedWriter;
|
||||||
|
private final DataNode datanode;
|
||||||
|
private final Configuration conf;
|
||||||
|
|
||||||
|
private final ExtendedBlock block;
|
||||||
|
private final DatanodeInfo target;
|
||||||
|
private final StorageType storageType;
|
||||||
|
|
||||||
|
private Socket targetSocket;
|
||||||
|
private DataOutputStream targetOutputStream;
|
||||||
|
private DataInputStream targetInputStream;
|
||||||
|
private ByteBuffer targetBuffer;
|
||||||
|
private long blockOffset4Target = 0;
|
||||||
|
private long seqNo4Target = 0;
|
||||||
|
|
||||||
|
StripedBlockWriter(StripedWriter stripedWriter, DataNode datanode,
|
||||||
|
Configuration conf, ExtendedBlock block,
|
||||||
|
DatanodeInfo target, StorageType storageType)
|
||||||
|
throws IOException {
|
||||||
|
this.stripedWriter = stripedWriter;
|
||||||
|
this.datanode = datanode;
|
||||||
|
this.conf = conf;
|
||||||
|
|
||||||
|
this.block = block;
|
||||||
|
this.target = target;
|
||||||
|
this.storageType = storageType;
|
||||||
|
|
||||||
|
this.targetBuffer = stripedWriter.allocateWriteBuffer();
|
||||||
|
|
||||||
|
init();
|
||||||
|
}
|
||||||
|
|
||||||
|
ByteBuffer getTargetBuffer() {
|
||||||
|
return targetBuffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initialize output/input streams for transferring data to target
|
||||||
|
* and send create block request.
|
||||||
|
*/
|
||||||
|
private void init() throws IOException {
|
||||||
|
Socket socket = null;
|
||||||
|
DataOutputStream out = null;
|
||||||
|
DataInputStream in = null;
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
InetSocketAddress targetAddr =
|
||||||
|
stripedWriter.getSocketAddress4Transfer(target);
|
||||||
|
socket = datanode.newSocket();
|
||||||
|
NetUtils.connect(socket, targetAddr,
|
||||||
|
datanode.getDnConf().getSocketTimeout());
|
||||||
|
socket.setSoTimeout(datanode.getDnConf().getSocketTimeout());
|
||||||
|
|
||||||
|
Token<BlockTokenIdentifier> blockToken =
|
||||||
|
datanode.getBlockAccessToken(block,
|
||||||
|
EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
|
||||||
|
|
||||||
|
long writeTimeout = datanode.getDnConf().getSocketWriteTimeout();
|
||||||
|
OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout);
|
||||||
|
InputStream unbufIn = NetUtils.getInputStream(socket);
|
||||||
|
DataEncryptionKeyFactory keyFactory =
|
||||||
|
datanode.getDataEncryptionKeyFactoryForBlock(block);
|
||||||
|
IOStreamPair saslStreams = datanode.getSaslClient().socketSend(
|
||||||
|
socket, unbufOut, unbufIn, keyFactory, blockToken, target);
|
||||||
|
|
||||||
|
unbufOut = saslStreams.out;
|
||||||
|
unbufIn = saslStreams.in;
|
||||||
|
|
||||||
|
out = new DataOutputStream(new BufferedOutputStream(unbufOut,
|
||||||
|
DFSUtilClient.getSmallBufferSize(conf)));
|
||||||
|
in = new DataInputStream(unbufIn);
|
||||||
|
|
||||||
|
DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());
|
||||||
|
new Sender(out).writeBlock(block, storageType,
|
||||||
|
blockToken, "", new DatanodeInfo[]{target},
|
||||||
|
new StorageType[]{storageType}, source,
|
||||||
|
BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0,
|
||||||
|
stripedWriter.getChecksum(), stripedWriter.getCachingStrategy(),
|
||||||
|
false, false, null);
|
||||||
|
|
||||||
|
targetSocket = socket;
|
||||||
|
targetOutputStream = out;
|
||||||
|
targetInputStream = in;
|
||||||
|
success = true;
|
||||||
|
} finally {
|
||||||
|
if (!success) {
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
IOUtils.closeStream(in);
|
||||||
|
IOUtils.closeStream(socket);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send data to targets.
|
||||||
|
*/
|
||||||
|
void transferData2Target(byte[] packetBuf) throws IOException {
|
||||||
|
if (targetBuffer.remaining() == 0) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
stripedWriter.getChecksum().calculateChunkedSums(
|
||||||
|
targetBuffer.array(), 0, targetBuffer.remaining(),
|
||||||
|
stripedWriter.getChecksumBuf(), 0);
|
||||||
|
|
||||||
|
int ckOff = 0;
|
||||||
|
while (targetBuffer.remaining() > 0) {
|
||||||
|
DFSPacket packet = new DFSPacket(packetBuf,
|
||||||
|
stripedWriter.getMaxChunksPerPacket(),
|
||||||
|
blockOffset4Target, seqNo4Target++,
|
||||||
|
stripedWriter.getChecksumSize(), false);
|
||||||
|
int maxBytesToPacket = stripedWriter.getMaxChunksPerPacket()
|
||||||
|
* stripedWriter.getBytesPerChecksum();
|
||||||
|
int toWrite = targetBuffer.remaining() > maxBytesToPacket ?
|
||||||
|
maxBytesToPacket : targetBuffer.remaining();
|
||||||
|
int ckLen = ((toWrite - 1) / stripedWriter.getBytesPerChecksum() + 1)
|
||||||
|
* stripedWriter.getChecksumSize();
|
||||||
|
packet.writeChecksum(stripedWriter.getChecksumBuf(), ckOff, ckLen);
|
||||||
|
ckOff += ckLen;
|
||||||
|
packet.writeData(targetBuffer, toWrite);
|
||||||
|
|
||||||
|
// Send packet
|
||||||
|
packet.writeTo(targetOutputStream);
|
||||||
|
|
||||||
|
blockOffset4Target += toWrite;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// send an empty packet to mark the end of the block
|
||||||
|
void endTargetBlock(byte[] packetBuf) throws IOException {
|
||||||
|
DFSPacket packet = new DFSPacket(packetBuf, 0,
|
||||||
|
blockOffset4Target, seqNo4Target++,
|
||||||
|
stripedWriter.getChecksumSize(), true);
|
||||||
|
packet.writeTo(targetOutputStream);
|
||||||
|
targetOutputStream.flush();
|
||||||
|
}
|
||||||
|
|
||||||
|
void close() {
|
||||||
|
IOUtils.closeStream(targetOutputStream);
|
||||||
|
IOUtils.closeStream(targetInputStream);
|
||||||
|
IOUtils.closeStream(targetSocket);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,466 @@
|
||||||
|
/**
|
||||||
|
* 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.hdfs.server.datanode.erasurecode;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
|
import org.apache.hadoop.hdfs.DFSUtilClient.CorruptedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
|
||||||
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
||||||
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
|
||||||
|
import org.apache.hadoop.util.DataChecksum;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.BitSet;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.CompletionService;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Manage striped readers that performs reading of block data from remote to
|
||||||
|
* serve input data for the erasure decoding.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class StripedReader {
|
||||||
|
private static final Logger LOG = DataNode.LOG;
|
||||||
|
|
||||||
|
private final int stripedReadTimeoutInMills;
|
||||||
|
private final int stripedReadBufferSize;
|
||||||
|
|
||||||
|
private StripedReconstructor reconstructor;
|
||||||
|
private final DataNode datanode;
|
||||||
|
private final Configuration conf;
|
||||||
|
|
||||||
|
private final int dataBlkNum;
|
||||||
|
private final int parityBlkNum;
|
||||||
|
|
||||||
|
|
||||||
|
private DataChecksum checksum;
|
||||||
|
// Striped read buffer size
|
||||||
|
private int bufferSize;
|
||||||
|
private int[] successList;
|
||||||
|
|
||||||
|
private final int minRequiredSources;
|
||||||
|
// The buffers and indices for striped blocks whose length is 0
|
||||||
|
private ByteBuffer[] zeroStripeBuffers;
|
||||||
|
private short[] zeroStripeIndices;
|
||||||
|
|
||||||
|
// sources
|
||||||
|
private final byte[] liveIndices;
|
||||||
|
private final DatanodeInfo[] sources;
|
||||||
|
|
||||||
|
private final List<StripedBlockReader> readers;
|
||||||
|
|
||||||
|
private final Map<Future<Void>, Integer> futures = new HashMap<>();
|
||||||
|
private final CompletionService<Void> readService;
|
||||||
|
|
||||||
|
StripedReader(StripedReconstructor reconstructor, DataNode datanode,
|
||||||
|
Configuration conf,
|
||||||
|
BlockECReconstructionInfo reconstructionInfo) {
|
||||||
|
stripedReadTimeoutInMills = conf.getInt(
|
||||||
|
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_KEY,
|
||||||
|
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
|
||||||
|
stripedReadBufferSize = conf.getInt(
|
||||||
|
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY,
|
||||||
|
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_DEFAULT);
|
||||||
|
|
||||||
|
this.reconstructor = reconstructor;
|
||||||
|
this.datanode = datanode;
|
||||||
|
this.conf = conf;
|
||||||
|
|
||||||
|
ErasureCodingPolicy ecPolicy = reconstructionInfo.getErasureCodingPolicy();
|
||||||
|
dataBlkNum = ecPolicy.getNumDataUnits();
|
||||||
|
parityBlkNum = ecPolicy.getNumParityUnits();
|
||||||
|
|
||||||
|
ExtendedBlock blockGroup = reconstructionInfo.getExtendedBlock();
|
||||||
|
int cellsNum = (int)((blockGroup.getNumBytes() - 1) / ecPolicy.getCellSize()
|
||||||
|
+ 1);
|
||||||
|
minRequiredSources = Math.min(cellsNum, dataBlkNum);
|
||||||
|
|
||||||
|
if (minRequiredSources < dataBlkNum) {
|
||||||
|
int zeroStripNum = dataBlkNum - minRequiredSources;
|
||||||
|
zeroStripeBuffers = new ByteBuffer[zeroStripNum];
|
||||||
|
zeroStripeIndices = new short[zeroStripNum];
|
||||||
|
}
|
||||||
|
|
||||||
|
liveIndices = reconstructionInfo.getLiveBlockIndices();
|
||||||
|
sources = reconstructionInfo.getSourceDnInfos();
|
||||||
|
|
||||||
|
readers = new ArrayList<>(sources.length);
|
||||||
|
readService = reconstructor.createReadService();
|
||||||
|
|
||||||
|
Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
|
||||||
|
"No enough live striped blocks.");
|
||||||
|
Preconditions.checkArgument(liveIndices.length == sources.length,
|
||||||
|
"liveBlockIndices and source datanodes should match");
|
||||||
|
}
|
||||||
|
|
||||||
|
void init() throws IOException {
|
||||||
|
initReaders();
|
||||||
|
|
||||||
|
initBufferSize();
|
||||||
|
|
||||||
|
initZeroStrip();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initReaders() throws IOException {
|
||||||
|
// Store the array indices of source DNs we have read successfully.
|
||||||
|
// In each iteration of read, the successList list may be updated if
|
||||||
|
// some source DN is corrupted or slow. And use the updated successList
|
||||||
|
// list of DNs for next iteration read.
|
||||||
|
successList = new int[minRequiredSources];
|
||||||
|
|
||||||
|
StripedBlockReader reader;
|
||||||
|
int nSuccess = 0;
|
||||||
|
for (int i = 0; i < sources.length && nSuccess < minRequiredSources; i++) {
|
||||||
|
reader = createReader(i, 0);
|
||||||
|
readers.add(reader);
|
||||||
|
if (reader.getBlockReader() != null) {
|
||||||
|
initOrVerifyChecksum(reader);
|
||||||
|
successList[nSuccess++] = i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (nSuccess < minRequiredSources) {
|
||||||
|
String error = "Can't find minimum sources required by "
|
||||||
|
+ "reconstruction, block id: "
|
||||||
|
+ reconstructor.getBlockGroup().getBlockId();
|
||||||
|
throw new IOException(error);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
StripedBlockReader createReader(int idxInSources, long offsetInBlock) {
|
||||||
|
return new StripedBlockReader(this, datanode,
|
||||||
|
conf, liveIndices[idxInSources],
|
||||||
|
reconstructor.getBlock(liveIndices[idxInSources]),
|
||||||
|
sources[idxInSources], offsetInBlock);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initBufferSize() {
|
||||||
|
int bytesPerChecksum = checksum.getBytesPerChecksum();
|
||||||
|
// The bufferSize is flat to divide bytesPerChecksum
|
||||||
|
int readBufferSize = stripedReadBufferSize;
|
||||||
|
bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
|
||||||
|
readBufferSize - readBufferSize % bytesPerChecksum;
|
||||||
|
}
|
||||||
|
|
||||||
|
// init checksum from block reader
|
||||||
|
private void initOrVerifyChecksum(StripedBlockReader reader) {
|
||||||
|
if (checksum == null) {
|
||||||
|
checksum = reader.getBlockReader().getDataChecksum();
|
||||||
|
} else {
|
||||||
|
assert reader.getBlockReader().getDataChecksum().equals(checksum);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ByteBuffer allocateReadBuffer() {
|
||||||
|
return ByteBuffer.allocate(getBufferSize());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initZeroStrip() {
|
||||||
|
if (zeroStripeBuffers != null) {
|
||||||
|
for (int i = 0; i < zeroStripeBuffers.length; i++) {
|
||||||
|
zeroStripeBuffers[i] = reconstructor.allocateBuffer(bufferSize);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
BitSet bitset = reconstructor.getLiveBitSet();
|
||||||
|
int k = 0;
|
||||||
|
for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
|
||||||
|
if (!bitset.get(i)) {
|
||||||
|
if (reconstructor.getBlockLen(i) <= 0) {
|
||||||
|
zeroStripeIndices[k++] = (short)i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getReadLength(int index, int reconstructLength) {
|
||||||
|
// the reading length should not exceed the length for reconstruction
|
||||||
|
long blockLen = reconstructor.getBlockLen(index);
|
||||||
|
long remaining = blockLen - reconstructor.getPositionInBlock();
|
||||||
|
return (int) Math.min(remaining, reconstructLength);
|
||||||
|
}
|
||||||
|
|
||||||
|
ByteBuffer[] getInputBuffers(int toReconstructLen) {
|
||||||
|
ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
|
||||||
|
|
||||||
|
for (int i = 0; i < successList.length; i++) {
|
||||||
|
int index = successList[i];
|
||||||
|
StripedBlockReader reader = getReader(index);
|
||||||
|
ByteBuffer buffer = reader.getReadBuffer();
|
||||||
|
paddingBufferToLen(buffer, toReconstructLen);
|
||||||
|
inputs[reader.getIndex()] = (ByteBuffer)buffer.flip();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (successList.length < dataBlkNum) {
|
||||||
|
for (int i = 0; i < zeroStripeBuffers.length; i++) {
|
||||||
|
ByteBuffer buffer = zeroStripeBuffers[i];
|
||||||
|
paddingBufferToLen(buffer, toReconstructLen);
|
||||||
|
int index = zeroStripeIndices[i];
|
||||||
|
inputs[index] = (ByteBuffer)buffer.flip();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return inputs;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void paddingBufferToLen(ByteBuffer buffer, int len) {
|
||||||
|
if (len > buffer.limit()) {
|
||||||
|
buffer.limit(len);
|
||||||
|
}
|
||||||
|
int toPadding = len - buffer.position();
|
||||||
|
for (int i = 0; i < toPadding; i++) {
|
||||||
|
buffer.put((byte) 0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read from minimum source DNs required for reconstruction in the iteration.
|
||||||
|
* First try the success list which we think they are the best DNs
|
||||||
|
* If source DN is corrupt or slow, try to read some other source DN,
|
||||||
|
* and will update the success list.
|
||||||
|
*
|
||||||
|
* Remember the updated success list and return it for following
|
||||||
|
* operations and next iteration read.
|
||||||
|
*
|
||||||
|
* @param reconstructLength the length to reconstruct.
|
||||||
|
* @return updated success list of source DNs we do real read
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
void readMinimumSources(int reconstructLength) throws IOException {
|
||||||
|
CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
|
||||||
|
try {
|
||||||
|
successList = doReadMinimumSources(reconstructLength, corruptedBlocks);
|
||||||
|
} finally {
|
||||||
|
// report corrupted blocks to NN
|
||||||
|
datanode.reportCorruptedBlocks(corruptedBlocks);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
int[] doReadMinimumSources(int reconstructLength,
|
||||||
|
CorruptedBlocks corruptedBlocks)
|
||||||
|
throws IOException {
|
||||||
|
Preconditions.checkArgument(reconstructLength >= 0 &&
|
||||||
|
reconstructLength <= bufferSize);
|
||||||
|
int nSuccess = 0;
|
||||||
|
int[] newSuccess = new int[minRequiredSources];
|
||||||
|
BitSet usedFlag = new BitSet(sources.length);
|
||||||
|
/*
|
||||||
|
* Read from minimum source DNs required, the success list contains
|
||||||
|
* source DNs which we think best.
|
||||||
|
*/
|
||||||
|
for (int i = 0; i < minRequiredSources; i++) {
|
||||||
|
StripedBlockReader reader = readers.get(successList[i]);
|
||||||
|
int toRead = getReadLength(liveIndices[successList[i]],
|
||||||
|
reconstructLength);
|
||||||
|
if (toRead > 0) {
|
||||||
|
Callable<Void> readCallable =
|
||||||
|
reader.readFromBlock(toRead, corruptedBlocks);
|
||||||
|
Future<Void> f = readService.submit(readCallable);
|
||||||
|
futures.put(f, successList[i]);
|
||||||
|
} else {
|
||||||
|
// If the read length is 0, we don't need to do real read
|
||||||
|
reader.getReadBuffer().position(0);
|
||||||
|
newSuccess[nSuccess++] = successList[i];
|
||||||
|
}
|
||||||
|
usedFlag.set(successList[i]);
|
||||||
|
}
|
||||||
|
|
||||||
|
while (!futures.isEmpty()) {
|
||||||
|
try {
|
||||||
|
StripingChunkReadResult result =
|
||||||
|
StripedBlockUtil.getNextCompletedStripedRead(
|
||||||
|
readService, futures, stripedReadTimeoutInMills);
|
||||||
|
int resultIndex = -1;
|
||||||
|
if (result.state == StripingChunkReadResult.SUCCESSFUL) {
|
||||||
|
resultIndex = result.index;
|
||||||
|
} else if (result.state == StripingChunkReadResult.FAILED) {
|
||||||
|
// If read failed for some source DN, we should not use it anymore
|
||||||
|
// and schedule read from another source DN.
|
||||||
|
StripedBlockReader failedReader = readers.get(result.index);
|
||||||
|
failedReader.closeBlockReader();
|
||||||
|
resultIndex = scheduleNewRead(usedFlag,
|
||||||
|
reconstructLength, corruptedBlocks);
|
||||||
|
} else if (result.state == StripingChunkReadResult.TIMEOUT) {
|
||||||
|
// If timeout, we also schedule a new read.
|
||||||
|
resultIndex = scheduleNewRead(usedFlag,
|
||||||
|
reconstructLength, corruptedBlocks);
|
||||||
|
}
|
||||||
|
if (resultIndex >= 0) {
|
||||||
|
newSuccess[nSuccess++] = resultIndex;
|
||||||
|
if (nSuccess >= minRequiredSources) {
|
||||||
|
// cancel remaining reads if we read successfully from minimum
|
||||||
|
// number of source DNs required by reconstruction.
|
||||||
|
cancelReads(futures.keySet());
|
||||||
|
futures.clear();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
LOG.info("Read data interrupted.", e);
|
||||||
|
cancelReads(futures.keySet());
|
||||||
|
futures.clear();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (nSuccess < minRequiredSources) {
|
||||||
|
String error = "Can't read data from minimum number of sources "
|
||||||
|
+ "required by reconstruction, block id: " +
|
||||||
|
reconstructor.getBlockGroup().getBlockId();
|
||||||
|
throw new IOException(error);
|
||||||
|
}
|
||||||
|
|
||||||
|
return newSuccess;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Schedule a read from some new source DN if some DN is corrupted
|
||||||
|
* or slow, this is called from the read iteration.
|
||||||
|
* Initially we may only have <code>minRequiredSources</code> number of
|
||||||
|
* StripedBlockReader.
|
||||||
|
* If the position is at the end of target block, don't need to do
|
||||||
|
* real read, and return the array index of source DN, otherwise -1.
|
||||||
|
*
|
||||||
|
* @param used the used source DNs in this iteration.
|
||||||
|
* @return the array index of source DN if don't need to do real read.
|
||||||
|
*/
|
||||||
|
private int scheduleNewRead(BitSet used, int reconstructLength,
|
||||||
|
CorruptedBlocks corruptedBlocks) {
|
||||||
|
StripedBlockReader reader = null;
|
||||||
|
// step1: initially we may only have <code>minRequiredSources</code>
|
||||||
|
// number of StripedBlockReader, and there may be some source DNs we never
|
||||||
|
// read before, so will try to create StripedBlockReader for one new source
|
||||||
|
// DN and try to read from it. If found, go to step 3.
|
||||||
|
int m = readers.size();
|
||||||
|
int toRead = 0;
|
||||||
|
while (reader == null && m < sources.length) {
|
||||||
|
reader = createReader(m, reconstructor.getPositionInBlock());
|
||||||
|
readers.add(reader);
|
||||||
|
toRead = getReadLength(liveIndices[m], reconstructLength);
|
||||||
|
if (toRead > 0) {
|
||||||
|
if (reader.getBlockReader() == null) {
|
||||||
|
reader = null;
|
||||||
|
m++;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
used.set(m);
|
||||||
|
return m;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// step2: if there is no new source DN we can use, try to find a source
|
||||||
|
// DN we ever read from but because some reason, e.g., slow, it
|
||||||
|
// is not in the success DN list at the begin of this iteration, so
|
||||||
|
// we have not tried it in this iteration. Now we have a chance to
|
||||||
|
// revisit it again.
|
||||||
|
for (int i = 0; reader == null && i < readers.size(); i++) {
|
||||||
|
if (!used.get(i)) {
|
||||||
|
StripedBlockReader stripedReader = readers.get(i);
|
||||||
|
toRead = getReadLength(liveIndices[i], reconstructLength);
|
||||||
|
if (toRead > 0) {
|
||||||
|
stripedReader.closeBlockReader();
|
||||||
|
stripedReader.resetBlockReader(reconstructor.getPositionInBlock());
|
||||||
|
if (stripedReader.getBlockReader() != null) {
|
||||||
|
stripedReader.getReadBuffer().position(0);
|
||||||
|
m = i;
|
||||||
|
reader = stripedReader;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
used.set(i);
|
||||||
|
stripedReader.getReadBuffer().position(0);
|
||||||
|
return i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// step3: schedule if find a correct source DN and need to do real read.
|
||||||
|
if (reader != null) {
|
||||||
|
Callable<Void> readCallable =
|
||||||
|
reader.readFromBlock(toRead, corruptedBlocks);
|
||||||
|
Future<Void> f = readService.submit(readCallable);
|
||||||
|
futures.put(f, m);
|
||||||
|
used.set(m);
|
||||||
|
}
|
||||||
|
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Cancel all reads.
|
||||||
|
private static void cancelReads(Collection<Future<Void>> futures) {
|
||||||
|
for (Future<Void> future : futures) {
|
||||||
|
future.cancel(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void close() {
|
||||||
|
for (StripedBlockReader reader : readers) {
|
||||||
|
reader.closeBlockReader();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
StripedBlockReader getReader(int i) {
|
||||||
|
return readers.get(i);
|
||||||
|
}
|
||||||
|
|
||||||
|
int getBufferSize() {
|
||||||
|
return bufferSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataChecksum getChecksum() {
|
||||||
|
return checksum;
|
||||||
|
}
|
||||||
|
|
||||||
|
void clearBuffers() {
|
||||||
|
if (zeroStripeBuffers != null) {
|
||||||
|
for (ByteBuffer zeroStripeBuffer : zeroStripeBuffers) {
|
||||||
|
zeroStripeBuffer.clear();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for (StripedBlockReader reader : readers) {
|
||||||
|
if (reader.getReadBuffer() != null) {
|
||||||
|
reader.getReadBuffer().clear();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
|
||||||
|
return reconstructor.getSocketAddress4Transfer(dnInfo);
|
||||||
|
}
|
||||||
|
|
||||||
|
CachingStrategy getCachingStrategy() {
|
||||||
|
return reconstructor.getCachingStrategy();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,273 @@
|
||||||
|
/**
|
||||||
|
* 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.hdfs.server.datanode.erasurecode;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
|
||||||
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
||||||
|
import org.apache.hadoop.io.erasurecode.CodecUtil;
|
||||||
|
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
|
||||||
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
import org.apache.hadoop.util.DataChecksum;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.BitSet;
|
||||||
|
import java.util.concurrent.CompletionService;
|
||||||
|
import java.util.concurrent.ExecutorCompletionService;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* StripedReconstructor reconstruct one or more missed striped block in the
|
||||||
|
* striped block group, the minimum number of live striped blocks should be
|
||||||
|
* no less than data block number.
|
||||||
|
*
|
||||||
|
* | <- Striped Block Group -> |
|
||||||
|
* blk_0 blk_1 blk_2(*) blk_3 ... <- A striped block group
|
||||||
|
* | | | |
|
||||||
|
* v v v v
|
||||||
|
* +------+ +------+ +------+ +------+
|
||||||
|
* |cell_0| |cell_1| |cell_2| |cell_3| ...
|
||||||
|
* +------+ +------+ +------+ +------+
|
||||||
|
* |cell_4| |cell_5| |cell_6| |cell_7| ...
|
||||||
|
* +------+ +------+ +------+ +------+
|
||||||
|
* |cell_8| |cell_9| |cell10| |cell11| ...
|
||||||
|
* +------+ +------+ +------+ +------+
|
||||||
|
* ... ... ... ...
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* We use following steps to reconstruct striped block group, in each round, we
|
||||||
|
* reconstruct <code>bufferSize</code> data until finish, the
|
||||||
|
* <code>bufferSize</code> is configurable and may be less or larger than
|
||||||
|
* cell size:
|
||||||
|
* step1: read <code>bufferSize</code> data from minimum number of sources
|
||||||
|
* required by reconstruction.
|
||||||
|
* step2: decode data for targets.
|
||||||
|
* step3: transfer data to targets.
|
||||||
|
*
|
||||||
|
* In step1, try to read <code>bufferSize</code> data from minimum number
|
||||||
|
* of sources , if there is corrupt or stale sources, read from new source
|
||||||
|
* will be scheduled. The best sources are remembered for next round and
|
||||||
|
* may be updated in each round.
|
||||||
|
*
|
||||||
|
* In step2, typically if source blocks we read are all data blocks, we
|
||||||
|
* need to call encode, and if there is one parity block, we need to call
|
||||||
|
* decode. Notice we only read once and reconstruct all missed striped block
|
||||||
|
* if they are more than one.
|
||||||
|
*
|
||||||
|
* In step3, send the reconstructed data to targets by constructing packet
|
||||||
|
* and send them directly. Same as continuous block replication, we
|
||||||
|
* don't check the packet ack. Since the datanode doing the reconstruction work
|
||||||
|
* are one of the source datanodes, so the reconstructed data are sent
|
||||||
|
* remotely.
|
||||||
|
*
|
||||||
|
* There are some points we can do further improvements in next phase:
|
||||||
|
* 1. we can read the block file directly on the local datanode,
|
||||||
|
* currently we use remote block reader. (Notice short-circuit is not
|
||||||
|
* a good choice, see inline comments).
|
||||||
|
* 2. We need to check the packet ack for EC reconstruction? Since EC
|
||||||
|
* reconstruction is more expensive than continuous block replication,
|
||||||
|
* it needs to read from several other datanodes, should we make sure the
|
||||||
|
* reconstructed result received by targets?
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class StripedReconstructor implements Runnable {
|
||||||
|
private static final Logger LOG = DataNode.LOG;
|
||||||
|
|
||||||
|
private final ErasureCodingWorker worker;
|
||||||
|
private final DataNode datanode;
|
||||||
|
private final Configuration conf;
|
||||||
|
|
||||||
|
private final ErasureCodingPolicy ecPolicy;
|
||||||
|
|
||||||
|
private RawErasureDecoder decoder;
|
||||||
|
|
||||||
|
private final ExtendedBlock blockGroup;
|
||||||
|
private final BitSet liveBitSet;
|
||||||
|
|
||||||
|
// position in striped internal block
|
||||||
|
private long positionInBlock;
|
||||||
|
|
||||||
|
private StripedReader stripedReader;
|
||||||
|
|
||||||
|
private StripedWriter stripedWriter;
|
||||||
|
|
||||||
|
private final CachingStrategy cachingStrategy;
|
||||||
|
|
||||||
|
StripedReconstructor(ErasureCodingWorker worker,
|
||||||
|
BlockECReconstructionInfo reconstructionInfo) {
|
||||||
|
this.worker = worker;
|
||||||
|
this.datanode = worker.getDatanode();
|
||||||
|
this.conf = worker.getConf();
|
||||||
|
|
||||||
|
ecPolicy = reconstructionInfo.getErasureCodingPolicy();
|
||||||
|
|
||||||
|
blockGroup = reconstructionInfo.getExtendedBlock();
|
||||||
|
byte[] liveIndices = reconstructionInfo.getLiveBlockIndices();
|
||||||
|
liveBitSet = new BitSet(ecPolicy.getNumDataUnits() +
|
||||||
|
ecPolicy.getNumParityUnits());
|
||||||
|
for (int i = 0; i < liveIndices.length; i++) {
|
||||||
|
liveBitSet.set(liveIndices[i]);
|
||||||
|
}
|
||||||
|
|
||||||
|
stripedReader = new StripedReader(this, datanode,
|
||||||
|
conf, reconstructionInfo);
|
||||||
|
stripedWriter = new StripedWriter(this, datanode,
|
||||||
|
conf, reconstructionInfo);
|
||||||
|
|
||||||
|
cachingStrategy = CachingStrategy.newDefaultStrategy();
|
||||||
|
|
||||||
|
positionInBlock = 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
BitSet getLiveBitSet() {
|
||||||
|
return liveBitSet;
|
||||||
|
}
|
||||||
|
|
||||||
|
ByteBuffer allocateBuffer(int length) {
|
||||||
|
return ByteBuffer.allocate(length);
|
||||||
|
}
|
||||||
|
|
||||||
|
ExtendedBlock getBlock(int i) {
|
||||||
|
return StripedBlockUtil.constructInternalBlock(blockGroup, ecPolicy, i);
|
||||||
|
}
|
||||||
|
|
||||||
|
long getBlockLen(int i) {
|
||||||
|
return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
|
||||||
|
ecPolicy, i);
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean hasValidTargets() {
|
||||||
|
return stripedWriter.hasValidTargets();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
datanode.incrementXmitsInProgress();
|
||||||
|
try {
|
||||||
|
stripedReader.init();
|
||||||
|
|
||||||
|
stripedWriter.init();
|
||||||
|
|
||||||
|
reconstructAndTransfer();
|
||||||
|
|
||||||
|
stripedWriter.endTargetBlocks();
|
||||||
|
|
||||||
|
// Currently we don't check the acks for packets, this is similar as
|
||||||
|
// block replication.
|
||||||
|
} catch (Throwable e) {
|
||||||
|
LOG.warn("Failed to reconstruct striped block: {}", blockGroup, e);
|
||||||
|
} finally {
|
||||||
|
datanode.decrementXmitsInProgress();
|
||||||
|
|
||||||
|
stripedReader.close();
|
||||||
|
|
||||||
|
stripedWriter.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void reconstructAndTransfer() throws IOException {
|
||||||
|
while (positionInBlock < stripedWriter.getMaxTargetLength()) {
|
||||||
|
long remaining = stripedWriter.getMaxTargetLength() - positionInBlock;
|
||||||
|
final int toReconstructLen =
|
||||||
|
(int) Math.min(stripedReader.getBufferSize(), remaining);
|
||||||
|
// step1: read from minimum source DNs required for reconstruction.
|
||||||
|
// The returned success list is the source DNs we do real read from
|
||||||
|
stripedReader.readMinimumSources(toReconstructLen);
|
||||||
|
|
||||||
|
// step2: decode to reconstruct targets
|
||||||
|
reconstructTargets(toReconstructLen);
|
||||||
|
|
||||||
|
// step3: transfer data
|
||||||
|
if (stripedWriter.transferData2Targets() == 0) {
|
||||||
|
String error = "Transfer failed for all targets.";
|
||||||
|
throw new IOException(error);
|
||||||
|
}
|
||||||
|
|
||||||
|
positionInBlock += toReconstructLen;
|
||||||
|
|
||||||
|
clearBuffers();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Initialize decoder
|
||||||
|
private void initDecoderIfNecessary() {
|
||||||
|
if (decoder == null) {
|
||||||
|
decoder = CodecUtil.createRSRawDecoder(conf, ecPolicy.getNumDataUnits(),
|
||||||
|
ecPolicy.getNumParityUnits());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reconstructTargets(int toReconstructLen) {
|
||||||
|
initDecoderIfNecessary();
|
||||||
|
|
||||||
|
ByteBuffer[] inputs = stripedReader.getInputBuffers(toReconstructLen);
|
||||||
|
|
||||||
|
int[] erasedIndices = stripedWriter.getRealTargetIndices();
|
||||||
|
ByteBuffer[] outputs = stripedWriter.getRealTargetBuffers(toReconstructLen);
|
||||||
|
|
||||||
|
decoder.decode(inputs, erasedIndices, outputs);
|
||||||
|
|
||||||
|
stripedWriter.updateRealTargetBuffers(toReconstructLen);
|
||||||
|
}
|
||||||
|
|
||||||
|
long getPositionInBlock() {
|
||||||
|
return positionInBlock;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clear all associated buffers.
|
||||||
|
*/
|
||||||
|
private void clearBuffers() {
|
||||||
|
stripedReader.clearBuffers();
|
||||||
|
|
||||||
|
stripedWriter.clearBuffers();
|
||||||
|
}
|
||||||
|
|
||||||
|
InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
|
||||||
|
return NetUtils.createSocketAddr(dnInfo.getXferAddr(
|
||||||
|
datanode.getDnConf().getConnectToDnViaHostname()));
|
||||||
|
}
|
||||||
|
|
||||||
|
int getBufferSize() {
|
||||||
|
return stripedReader.getBufferSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
DataChecksum getChecksum() {
|
||||||
|
return stripedReader.getChecksum();
|
||||||
|
}
|
||||||
|
|
||||||
|
CachingStrategy getCachingStrategy() {
|
||||||
|
return cachingStrategy;
|
||||||
|
}
|
||||||
|
|
||||||
|
CompletionService<Void> createReadService() {
|
||||||
|
return new ExecutorCompletionService<>(worker.getStripedReadPool());
|
||||||
|
}
|
||||||
|
|
||||||
|
ExtendedBlock getBlockGroup() {
|
||||||
|
return blockGroup;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,313 @@
|
||||||
|
/**
|
||||||
|
* 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.hdfs.server.datanode.erasurecode;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.StorageType;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
|
||||||
|
import org.apache.hadoop.util.DataChecksum;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.BitSet;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Manage striped writers that writes to a target with reconstructed data.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class StripedWriter {
|
||||||
|
private static final Logger LOG = DataNode.LOG;
|
||||||
|
private final static int WRITE_PACKET_SIZE = 64 * 1024;
|
||||||
|
|
||||||
|
private final StripedReconstructor reconstructor;
|
||||||
|
private final DataNode datanode;
|
||||||
|
private final Configuration conf;
|
||||||
|
|
||||||
|
private final int dataBlkNum;
|
||||||
|
private final int parityBlkNum;
|
||||||
|
|
||||||
|
private boolean[] targetsStatus;
|
||||||
|
|
||||||
|
// targets
|
||||||
|
private final DatanodeInfo[] targets;
|
||||||
|
private final short[] targetIndices;
|
||||||
|
private boolean hasValidTargets;
|
||||||
|
private final StorageType[] targetStorageTypes;
|
||||||
|
private long maxTargetLength;
|
||||||
|
|
||||||
|
private StripedBlockWriter[] writers;
|
||||||
|
|
||||||
|
private int maxChunksPerPacket;
|
||||||
|
private byte[] packetBuf;
|
||||||
|
private byte[] checksumBuf;
|
||||||
|
private int bytesPerChecksum;
|
||||||
|
private int checksumSize;
|
||||||
|
|
||||||
|
StripedWriter(StripedReconstructor reconstructor,
|
||||||
|
DataNode datanode,
|
||||||
|
Configuration conf,
|
||||||
|
BlockECReconstructionInfo reconstructionInfo) {
|
||||||
|
this.reconstructor = reconstructor;
|
||||||
|
this.datanode = datanode;
|
||||||
|
this.conf = conf;
|
||||||
|
|
||||||
|
ErasureCodingPolicy ecPolicy = reconstructionInfo.getErasureCodingPolicy();
|
||||||
|
dataBlkNum = ecPolicy.getNumDataUnits();
|
||||||
|
parityBlkNum = ecPolicy.getNumParityUnits();
|
||||||
|
|
||||||
|
targets = reconstructionInfo.getTargetDnInfos();
|
||||||
|
targetStorageTypes = reconstructionInfo.getTargetStorageTypes();
|
||||||
|
|
||||||
|
writers = new StripedBlockWriter[targets.length];
|
||||||
|
|
||||||
|
targetIndices = new short[targets.length];
|
||||||
|
Preconditions.checkArgument(targetIndices.length <= parityBlkNum,
|
||||||
|
"Too much missed striped blocks.");
|
||||||
|
initTargetIndices();
|
||||||
|
|
||||||
|
maxTargetLength = 0L;
|
||||||
|
for (short targetIndex : targetIndices) {
|
||||||
|
maxTargetLength = Math.max(maxTargetLength,
|
||||||
|
reconstructor.getBlockLen(targetIndex));
|
||||||
|
}
|
||||||
|
|
||||||
|
// targetsStatus store whether some target is success, it will record
|
||||||
|
// any failed target once, if some target failed (invalid DN or transfer
|
||||||
|
// failed), will not transfer data to it any more.
|
||||||
|
targetsStatus = new boolean[targets.length];
|
||||||
|
}
|
||||||
|
|
||||||
|
void init() throws IOException {
|
||||||
|
DataChecksum checksum = reconstructor.getChecksum();
|
||||||
|
checksumSize = checksum.getChecksumSize();
|
||||||
|
bytesPerChecksum = checksum.getBytesPerChecksum();
|
||||||
|
int chunkSize = bytesPerChecksum + checksumSize;
|
||||||
|
maxChunksPerPacket = Math.max(
|
||||||
|
(WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN) / chunkSize, 1);
|
||||||
|
int maxPacketSize = chunkSize * maxChunksPerPacket
|
||||||
|
+ PacketHeader.PKT_MAX_HEADER_LEN;
|
||||||
|
|
||||||
|
packetBuf = new byte[maxPacketSize];
|
||||||
|
int tmpLen = checksumSize *
|
||||||
|
(reconstructor.getBufferSize() / bytesPerChecksum);
|
||||||
|
checksumBuf = new byte[tmpLen];
|
||||||
|
|
||||||
|
if (initTargetStreams() == 0) {
|
||||||
|
String error = "All targets are failed.";
|
||||||
|
throw new IOException(error);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initTargetIndices() {
|
||||||
|
BitSet bitset = reconstructor.getLiveBitSet();
|
||||||
|
|
||||||
|
int m = 0;
|
||||||
|
int k = 0;
|
||||||
|
hasValidTargets = false;
|
||||||
|
for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
|
||||||
|
if (!bitset.get(i)) {
|
||||||
|
if (reconstructor.getBlockLen(i) > 0) {
|
||||||
|
if (m < targets.length) {
|
||||||
|
targetIndices[m++] = (short)i;
|
||||||
|
hasValidTargets = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send reconstructed data to targets.
|
||||||
|
*/
|
||||||
|
int transferData2Targets() {
|
||||||
|
int nSuccess = 0;
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
if (targetsStatus[i]) {
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
writers[i].transferData2Target(packetBuf);
|
||||||
|
nSuccess++;
|
||||||
|
success = true;
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.warn(e.getMessage());
|
||||||
|
}
|
||||||
|
targetsStatus[i] = success;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return nSuccess;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send an empty packet to mark the end of the block.
|
||||||
|
*/
|
||||||
|
void endTargetBlocks() {
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
if (targetsStatus[i]) {
|
||||||
|
try {
|
||||||
|
writers[i].endTargetBlock(packetBuf);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.warn(e.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initialize output/input streams for transferring data to target
|
||||||
|
* and send create block request.
|
||||||
|
*/
|
||||||
|
int initTargetStreams() {
|
||||||
|
int nSuccess = 0;
|
||||||
|
for (short i = 0; i < targets.length; i++) {
|
||||||
|
try {
|
||||||
|
writers[i] = createWriter(i);
|
||||||
|
nSuccess++;
|
||||||
|
targetsStatus[i] = true;
|
||||||
|
} catch (Throwable e) {
|
||||||
|
LOG.warn(e.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return nSuccess;
|
||||||
|
}
|
||||||
|
|
||||||
|
private StripedBlockWriter createWriter(short index) throws IOException {
|
||||||
|
return new StripedBlockWriter(this, datanode, conf,
|
||||||
|
reconstructor.getBlock(targetIndices[index]), targets[index],
|
||||||
|
targetStorageTypes[index]);
|
||||||
|
}
|
||||||
|
|
||||||
|
ByteBuffer allocateWriteBuffer() {
|
||||||
|
return reconstructor.allocateBuffer(reconstructor.getBufferSize());
|
||||||
|
}
|
||||||
|
|
||||||
|
int getTargets() {
|
||||||
|
return targets.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getRealTargets() {
|
||||||
|
int m = 0;
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
if (targetsStatus[i]) {
|
||||||
|
m++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return m;
|
||||||
|
}
|
||||||
|
|
||||||
|
int[] getRealTargetIndices() {
|
||||||
|
int realTargets = getRealTargets();
|
||||||
|
int[] results = new int[realTargets];
|
||||||
|
int m = 0;
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
if (targetsStatus[i]) {
|
||||||
|
results[m++] = targetIndices[i];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return results;
|
||||||
|
}
|
||||||
|
|
||||||
|
ByteBuffer[] getRealTargetBuffers(int toReconstructLen) {
|
||||||
|
int numGood = getRealTargets();
|
||||||
|
ByteBuffer[] outputs = new ByteBuffer[numGood];
|
||||||
|
int m = 0;
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
if (targetsStatus[i]) {
|
||||||
|
writers[i].getTargetBuffer().limit(toReconstructLen);
|
||||||
|
outputs[m++] = writers[i].getTargetBuffer();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return outputs;
|
||||||
|
}
|
||||||
|
|
||||||
|
void updateRealTargetBuffers(int toReconstructLen) {
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
if (targetsStatus[i]) {
|
||||||
|
long blockLen = reconstructor.getBlockLen(targetIndices[i]);
|
||||||
|
long remaining = blockLen - reconstructor.getPositionInBlock();
|
||||||
|
if (remaining <= 0) {
|
||||||
|
writers[i].getTargetBuffer().limit(0);
|
||||||
|
} else if (remaining < toReconstructLen) {
|
||||||
|
writers[i].getTargetBuffer().limit((int)remaining);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
long getMaxTargetLength() {
|
||||||
|
return maxTargetLength;
|
||||||
|
}
|
||||||
|
|
||||||
|
byte[] getChecksumBuf() {
|
||||||
|
return checksumBuf;
|
||||||
|
}
|
||||||
|
|
||||||
|
int getBytesPerChecksum() {
|
||||||
|
return bytesPerChecksum;
|
||||||
|
}
|
||||||
|
|
||||||
|
int getChecksumSize() {
|
||||||
|
return checksumSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataChecksum getChecksum() {
|
||||||
|
return reconstructor.getChecksum();
|
||||||
|
}
|
||||||
|
|
||||||
|
int getMaxChunksPerPacket() {
|
||||||
|
return maxChunksPerPacket;
|
||||||
|
}
|
||||||
|
|
||||||
|
CachingStrategy getCachingStrategy() {
|
||||||
|
return reconstructor.getCachingStrategy();
|
||||||
|
}
|
||||||
|
|
||||||
|
InetSocketAddress getSocketAddress4Transfer(DatanodeInfo target) {
|
||||||
|
return reconstructor.getSocketAddress4Transfer(target);
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean hasValidTargets() {
|
||||||
|
return hasValidTargets;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clear all buffers.
|
||||||
|
*/
|
||||||
|
void clearBuffers() {
|
||||||
|
for (StripedBlockWriter writer : writers) {
|
||||||
|
ByteBuffer targetBuffer = writer.getTargetBuffer();
|
||||||
|
if (targetBuffer != null) {
|
||||||
|
targetBuffer.clear();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void close() {
|
||||||
|
for (int i = 0; i < targets.length; i++) {
|
||||||
|
writers[i].close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,26 @@
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Datanode side striping + erasure coding related task processing.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
package org.apache.hadoop.hdfs.server.datanode.erasurecode;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@ -230,22 +230,23 @@ public class TestReconstructStripedFile {
|
||||||
private int generateErrors(Map<ExtendedBlock, DataNode> corruptTargets,
|
private int generateErrors(Map<ExtendedBlock, DataNode> corruptTargets,
|
||||||
ReconstructionType type)
|
ReconstructionType type)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
int stoppedDN = 0;
|
int stoppedDNs = 0;
|
||||||
for (Map.Entry<ExtendedBlock, DataNode> target : corruptTargets.entrySet()) {
|
for (Map.Entry<ExtendedBlock, DataNode> target :
|
||||||
if (stoppedDN == 0 || type != ReconstructionType.DataOnly
|
corruptTargets.entrySet()) {
|
||||||
|
if (stoppedDNs == 0 || type != ReconstructionType.DataOnly
|
||||||
|| random.nextBoolean()) {
|
|| random.nextBoolean()) {
|
||||||
// stop at least one DN to trigger reconstruction
|
// stop at least one DN to trigger reconstruction
|
||||||
LOG.info("Note: stop DataNode " + target.getValue().getDisplayName()
|
LOG.info("Note: stop DataNode " + target.getValue().getDisplayName()
|
||||||
+ " with internal block " + target.getKey());
|
+ " with internal block " + target.getKey());
|
||||||
shutdownDataNode(target.getValue());
|
shutdownDataNode(target.getValue());
|
||||||
stoppedDN++;
|
stoppedDNs++;
|
||||||
} else { // corrupt the data on the DN
|
} else { // corrupt the data on the DN
|
||||||
LOG.info("Note: corrupt data on " + target.getValue().getDisplayName()
|
LOG.info("Note: corrupt data on " + target.getValue().getDisplayName()
|
||||||
+ " with internal block " + target.getKey());
|
+ " with internal block " + target.getKey());
|
||||||
cluster.corruptReplica(target.getValue(), target.getKey());
|
cluster.corruptReplica(target.getValue(), target.getKey());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return stoppedDN;
|
return stoppedDNs;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
Loading…
Reference in New Issue