|
@@ -17,6 +17,8 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.protocol;
|
|
package org.apache.hadoop.hdfs.protocol;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT;
|
|
|
|
+
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
@@ -31,6 +33,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.datanode.Replica;
|
|
import org.apache.hadoop.hdfs.server.datanode.Replica;
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.protobuf.ByteString;
|
|
import com.google.protobuf.ByteString;
|
|
import com.google.protobuf.CodedInputStream;
|
|
import com.google.protobuf.CodedInputStream;
|
|
@@ -63,26 +66,42 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
};
|
|
};
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Prepare an instance to in-place decode the given ByteString buffer
|
|
|
|
|
|
+ * Prepare an instance to in-place decode the given ByteString buffer.
|
|
* @param numBlocks - blocks in the buffer
|
|
* @param numBlocks - blocks in the buffer
|
|
* @param blocksBuf - ByteString encoded varints
|
|
* @param blocksBuf - ByteString encoded varints
|
|
|
|
+ * @param maxDataLength - maximum allowable data size in protobuf message
|
|
* @return BlockListAsLongs
|
|
* @return BlockListAsLongs
|
|
*/
|
|
*/
|
|
public static BlockListAsLongs decodeBuffer(final int numBlocks,
|
|
public static BlockListAsLongs decodeBuffer(final int numBlocks,
|
|
- final ByteString blocksBuf) {
|
|
|
|
- return new BufferDecoder(numBlocks, blocksBuf);
|
|
|
|
|
|
+ final ByteString blocksBuf, final int maxDataLength) {
|
|
|
|
+ return new BufferDecoder(numBlocks, blocksBuf, maxDataLength);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Prepare an instance to in-place decode the given ByteString buffers
|
|
|
|
|
|
+ * Prepare an instance to in-place decode the given ByteString buffers.
|
|
* @param numBlocks - blocks in the buffers
|
|
* @param numBlocks - blocks in the buffers
|
|
* @param blocksBufs - list of ByteString encoded varints
|
|
* @param blocksBufs - list of ByteString encoded varints
|
|
* @return BlockListAsLongs
|
|
* @return BlockListAsLongs
|
|
*/
|
|
*/
|
|
|
|
+ @VisibleForTesting
|
|
public static BlockListAsLongs decodeBuffers(final int numBlocks,
|
|
public static BlockListAsLongs decodeBuffers(final int numBlocks,
|
|
final List<ByteString> blocksBufs) {
|
|
final List<ByteString> blocksBufs) {
|
|
|
|
+ return decodeBuffers(numBlocks, blocksBufs,
|
|
|
|
+ IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Prepare an instance to in-place decode the given ByteString buffers.
|
|
|
|
+ * @param numBlocks - blocks in the buffers
|
|
|
|
+ * @param blocksBufs - list of ByteString encoded varints
|
|
|
|
+ * @param maxDataLength - maximum allowable data size in protobuf message
|
|
|
|
+ * @return BlockListAsLongs
|
|
|
|
+ */
|
|
|
|
+ public static BlockListAsLongs decodeBuffers(final int numBlocks,
|
|
|
|
+ final List<ByteString> blocksBufs, final int maxDataLength) {
|
|
// this doesn't actually copy the data
|
|
// this doesn't actually copy the data
|
|
- return decodeBuffer(numBlocks, ByteString.copyFrom(blocksBufs));
|
|
|
|
|
|
+ return decodeBuffer(numBlocks, ByteString.copyFrom(blocksBufs),
|
|
|
|
+ maxDataLength);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -93,7 +112,21 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
* @return BlockListAsLongs
|
|
* @return BlockListAsLongs
|
|
*/
|
|
*/
|
|
public static BlockListAsLongs decodeLongs(List<Long> blocksList) {
|
|
public static BlockListAsLongs decodeLongs(List<Long> blocksList) {
|
|
- return blocksList.isEmpty() ? EMPTY : new LongsDecoder(blocksList);
|
|
|
|
|
|
+ return decodeLongs(blocksList, IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Prepare an instance to in-place decode the given list of Longs. Note
|
|
|
|
+ * it's much more efficient to decode ByteString buffers and only exists
|
|
|
|
+ * for compatibility.
|
|
|
|
+ * @param blocksList - list of longs
|
|
|
|
+ * @param maxDataLength - maximum allowable data size in protobuf message
|
|
|
|
+ * @return BlockListAsLongs
|
|
|
|
+ */
|
|
|
|
+ public static BlockListAsLongs decodeLongs(List<Long> blocksList,
|
|
|
|
+ int maxDataLength) {
|
|
|
|
+ return blocksList.isEmpty() ? EMPTY :
|
|
|
|
+ new LongsDecoder(blocksList, maxDataLength);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -102,17 +135,22 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
* @param replicas - replicas to encode
|
|
* @param replicas - replicas to encode
|
|
* @return BlockListAsLongs
|
|
* @return BlockListAsLongs
|
|
*/
|
|
*/
|
|
|
|
+ @VisibleForTesting
|
|
public static BlockListAsLongs encode(
|
|
public static BlockListAsLongs encode(
|
|
final Collection<? extends Replica> replicas) {
|
|
final Collection<? extends Replica> replicas) {
|
|
- BlockListAsLongs.Builder builder = builder();
|
|
|
|
|
|
+ BlockListAsLongs.Builder builder = builder(IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
|
|
for (Replica replica : replicas) {
|
|
for (Replica replica : replicas) {
|
|
builder.add(replica);
|
|
builder.add(replica);
|
|
}
|
|
}
|
|
return builder.build();
|
|
return builder.build();
|
|
}
|
|
}
|
|
|
|
|
|
- public static BlockListAsLongs readFrom(InputStream is) throws IOException {
|
|
|
|
|
|
+ public static BlockListAsLongs readFrom(InputStream is, int maxDataLength)
|
|
|
|
+ throws IOException {
|
|
CodedInputStream cis = CodedInputStream.newInstance(is);
|
|
CodedInputStream cis = CodedInputStream.newInstance(is);
|
|
|
|
+ if (maxDataLength != IPC_MAXIMUM_DATA_LENGTH_DEFAULT) {
|
|
|
|
+ cis.setSizeLimit(maxDataLength);
|
|
|
|
+ }
|
|
int numBlocks = -1;
|
|
int numBlocks = -1;
|
|
ByteString blocksBuf = null;
|
|
ByteString blocksBuf = null;
|
|
while (!cis.isAtEnd()) {
|
|
while (!cis.isAtEnd()) {
|
|
@@ -133,7 +171,7 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (numBlocks != -1 && blocksBuf != null) {
|
|
if (numBlocks != -1 && blocksBuf != null) {
|
|
- return decodeBuffer(numBlocks, blocksBuf);
|
|
|
|
|
|
+ return decodeBuffer(numBlocks, blocksBuf, maxDataLength);
|
|
}
|
|
}
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -144,9 +182,14 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
cos.writeBytes(2, getBlocksBuffer());
|
|
cos.writeBytes(2, getBlocksBuffer());
|
|
cos.flush();
|
|
cos.flush();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
public static Builder builder() {
|
|
public static Builder builder() {
|
|
- return new BlockListAsLongs.Builder();
|
|
|
|
|
|
+ return builder(IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static Builder builder(int maxDataLength) {
|
|
|
|
+ return new BlockListAsLongs.Builder(maxDataLength);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -221,10 +264,12 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
private final CodedOutputStream cos;
|
|
private final CodedOutputStream cos;
|
|
private int numBlocks = 0;
|
|
private int numBlocks = 0;
|
|
private int numFinalized = 0;
|
|
private int numFinalized = 0;
|
|
|
|
+ private final int maxDataLength;
|
|
|
|
|
|
- Builder() {
|
|
|
|
|
|
+ Builder(int maxDataLength) {
|
|
out = ByteString.newOutput(64*1024);
|
|
out = ByteString.newOutput(64*1024);
|
|
cos = CodedOutputStream.newInstance(out);
|
|
cos = CodedOutputStream.newInstance(out);
|
|
|
|
+ this.maxDataLength = maxDataLength;
|
|
}
|
|
}
|
|
|
|
|
|
public void add(Replica replica) {
|
|
public void add(Replica replica) {
|
|
@@ -258,7 +303,8 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
// shouldn't happen, ByteString.Output doesn't throw IOE
|
|
// shouldn't happen, ByteString.Output doesn't throw IOE
|
|
throw new IllegalStateException(ioe);
|
|
throw new IllegalStateException(ioe);
|
|
}
|
|
}
|
|
- return new BufferDecoder(numBlocks, numFinalized, out.toByteString());
|
|
|
|
|
|
+ return new BufferDecoder(numBlocks, numFinalized, out.toByteString(),
|
|
|
|
+ maxDataLength);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -273,16 +319,19 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
private final ByteString buffer;
|
|
private final ByteString buffer;
|
|
private final int numBlocks;
|
|
private final int numBlocks;
|
|
private int numFinalized;
|
|
private int numFinalized;
|
|
|
|
+ private final int maxDataLength;
|
|
|
|
|
|
- BufferDecoder(final int numBlocks, final ByteString buf) {
|
|
|
|
- this(numBlocks, -1, buf);
|
|
|
|
|
|
+ BufferDecoder(final int numBlocks, final ByteString buf,
|
|
|
|
+ final int maxDataLength) {
|
|
|
|
+ this(numBlocks, -1, buf, maxDataLength);
|
|
}
|
|
}
|
|
|
|
|
|
BufferDecoder(final int numBlocks, final int numFinalized,
|
|
BufferDecoder(final int numBlocks, final int numFinalized,
|
|
- final ByteString buf) {
|
|
|
|
|
|
+ final ByteString buf, final int maxDataLength) {
|
|
this.numBlocks = numBlocks;
|
|
this.numBlocks = numBlocks;
|
|
this.numFinalized = numFinalized;
|
|
this.numFinalized = numFinalized;
|
|
this.buffer = buf;
|
|
this.buffer = buf;
|
|
|
|
+ this.maxDataLength = maxDataLength;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -349,6 +398,12 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
final CodedInputStream cis = buffer.newCodedInput();
|
|
final CodedInputStream cis = buffer.newCodedInput();
|
|
private int currentBlockIndex = 0;
|
|
private int currentBlockIndex = 0;
|
|
|
|
|
|
|
|
+ {
|
|
|
|
+ if (maxDataLength != IPC_MAXIMUM_DATA_LENGTH_DEFAULT) {
|
|
|
|
+ cis.setSizeLimit(maxDataLength);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public boolean hasNext() {
|
|
public boolean hasNext() {
|
|
return currentBlockIndex < numBlocks;
|
|
return currentBlockIndex < numBlocks;
|
|
@@ -384,12 +439,14 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
private final List<Long> values;
|
|
private final List<Long> values;
|
|
private final int finalizedBlocks;
|
|
private final int finalizedBlocks;
|
|
private final int numBlocks;
|
|
private final int numBlocks;
|
|
|
|
+ private final int maxDataLength;
|
|
|
|
|
|
// set the header
|
|
// set the header
|
|
- LongsDecoder(List<Long> values) {
|
|
|
|
|
|
+ LongsDecoder(List<Long> values, int maxDataLength) {
|
|
this.values = values.subList(2, values.size());
|
|
this.values = values.subList(2, values.size());
|
|
this.finalizedBlocks = values.get(0).intValue();
|
|
this.finalizedBlocks = values.get(0).intValue();
|
|
this.numBlocks = finalizedBlocks + values.get(1).intValue();
|
|
this.numBlocks = finalizedBlocks + values.get(1).intValue();
|
|
|
|
+ this.maxDataLength = maxDataLength;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -399,7 +456,7 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public ByteString getBlocksBuffer() {
|
|
public ByteString getBlocksBuffer() {
|
|
- Builder builder = builder();
|
|
|
|
|
|
+ Builder builder = builder(maxDataLength);
|
|
for (Replica replica : this) {
|
|
for (Replica replica : this) {
|
|
builder.add(replica);
|
|
builder.add(replica);
|
|
}
|
|
}
|