Skip to content

Commit 140f55f

Browse files
author
fuchaohong
committed
HDFS-17785. DFSAdmin supports setting the bandwidth for DataNode.
1 parent f34c4be commit 140f55f

File tree

27 files changed

+549
-3
lines changed

27 files changed

+549
-3
lines changed

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2479,6 +2479,20 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
24792479
}
24802480
}
24812481

2482+
/**
2483+
* Requests the namenode to tell all datanodes to use a new bandwidth value.
2484+
* See {@link ClientProtocol#setDataNodeBandwidth(long, String)}
2485+
* for more details.
2486+
*
2487+
* @see ClientProtocol#setDataNodeBandwidth(long, String)
2488+
*/
2489+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
2490+
checkOpen();
2491+
try (TraceScope ignored = tracer.newScope("setDataNodeBandwidth-" + type)) {
2492+
namenode.setDataNodeBandwidth(bandwidth, type);
2493+
}
2494+
}
2495+
24822496
/**
24832497
* @see ClientProtocol#finalizeUpgrade()
24842498
*/

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2094,6 +2094,17 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
20942094
dfs.setBalancerBandwidth(bandwidth);
20952095
}
20962096

2097+
/**
2098+
* Requests the namenode to tell all datanodes to reset the bandwidth of the specified type.
2099+
*
2100+
* @param bandwidth Bandwidth in bytes per second for all datanodes.
2101+
* @param type DataNode bandwidth type.
2102+
* @throws IOException
2103+
*/
2104+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
2105+
dfs.setDataNodeBandwidth(bandwidth, type);
2106+
}
2107+
20972108
/**
20982109
* Get a canonical service name for this file system. If the URI is logical,
20992110
* the hostname part of the URI will be returned.

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ViewDistributedFileSystem.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1057,6 +1057,16 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
10571057
defaultDFS.setBalancerBandwidth(bandwidth);
10581058
}
10591059

1060+
@Override
1061+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
1062+
if (this.vfs == null) {
1063+
super.setDataNodeBandwidth(bandwidth, type);
1064+
return;
1065+
}
1066+
checkDefaultDFS(defaultDFS, "setDataNodeBandwidth");
1067+
defaultDFS.setDataNodeBandwidth(bandwidth, type);
1068+
}
1069+
10601070
@Override
10611071
public String getCanonicalServiceName() {
10621072
if (this.vfs == null) {

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1050,6 +1050,16 @@ CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
10501050
@Idempotent
10511051
void setBalancerBandwidth(long bandwidth) throws IOException;
10521052

1053+
/**
1054+
* Tell all datanodes to reset the bandwidth of the specified type.
1055+
*
1056+
* @param bandwidth Bandwidth in bytes per second for this datanode.
1057+
* @param type DataNode bandwidth type.
1058+
* @throws IOException
1059+
*/
1060+
@Idempotent
1061+
void setDataNodeBandwidth(long bandwidth, String type) throws IOException;
1062+
10531063
/**
10541064
* Get the file info for a specific file or directory.
10551065
* @param src The string representation of the path to the file

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,7 @@
189189
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
190190
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
191191
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
192+
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetDataNodeBandwidthRequestProto;
192193
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
193194
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
194195
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
@@ -976,6 +977,16 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
976977
ipc(() -> rpcProxy.setBalancerBandwidth(null, req));
977978
}
978979

980+
@Override
981+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
982+
SetDataNodeBandwidthRequestProto req =
983+
SetDataNodeBandwidthRequestProto.newBuilder()
984+
.setBandwidth(bandwidth)
985+
.setType(type)
986+
.build();
987+
ipc(() -> rpcProxy.setDataNodeBandwidth(null, req));
988+
}
989+
979990
@Override
980991
public boolean isMethodSupported(String methodName) throws IOException {
981992
return RpcClientUtil.isMethodSupported(rpcProxy,

hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -772,9 +772,17 @@ message SetBalancerBandwidthRequestProto {
772772
required int64 bandwidth = 1;
773773
}
774774

775+
message SetDataNodeBandwidthRequestProto {
776+
required int64 bandwidth = 1;
777+
required string type = 2;
778+
}
779+
775780
message SetBalancerBandwidthResponseProto { // void response
776781
}
777782

783+
message SetDataNodeBandwidthResponseProto { // void response
784+
}
785+
778786
message GetDataEncryptionKeyRequestProto { // no parameters
779787
}
780788

@@ -999,6 +1007,8 @@ service ClientNamenodeProtocol {
9991007
returns(hadoop.common.CancelDelegationTokenResponseProto);
10001008
rpc setBalancerBandwidth(SetBalancerBandwidthRequestProto)
10011009
returns(SetBalancerBandwidthResponseProto);
1010+
rpc setDataNodeBandwidth(SetDataNodeBandwidthRequestProto)
1011+
returns(SetDataNodeBandwidthResponseProto);
10021012
rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
10031013
returns(GetDataEncryptionKeyResponseProto);
10041014
rpc createSnapshot(CreateSnapshotRequestProto)

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1314,6 +1314,16 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
13141314
rpcClient.invokeConcurrent(nss, method, true, false);
13151315
}
13161316

1317+
@Override
1318+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
1319+
rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
1320+
1321+
RemoteMethod method = new RemoteMethod("setDataNodeBandwidth",
1322+
new Class<?>[] {long.class, String.class}, bandwidth, type);
1323+
final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
1324+
rpcClient.invokeConcurrent(nss, method, true, false);
1325+
}
1326+
13171327
/**
13181328
* Recursively get all the locations for the path.
13191329
* For example, there are some mount points:

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1639,6 +1639,11 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
16391639
clientProto.setBalancerBandwidth(bandwidth);
16401640
}
16411641

1642+
@Override // ClientProtocol
1643+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
1644+
clientProto.setDataNodeBandwidth(bandwidth, type);
1645+
}
1646+
16421647
@Override // ClientProtocol
16431648
public ContentSummary getContentSummary(String path) throws IOException {
16441649
return clientProto.getContentSummary(path);

hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2205,6 +2205,15 @@ public void testSetBalancerBandwidth() throws Exception {
22052205
}, 100, 60 * 1000);
22062206
}
22072207

2208+
@Test
2209+
public void testSetDataNodeBandwidth() throws Exception {
2210+
routerProtocol.setDataNodeBandwidth(1000L, "transfer");
2211+
ArrayList<DataNode> datanodes = cluster.getCluster().getDataNodes();
2212+
GenericTestUtils.waitFor(() -> {
2213+
return datanodes.get(0).getTransferBandwidth() == 1000L;
2214+
}, 100, 60 * 1000);
2215+
}
2216+
22082217
@Test
22092218
public void testAddClientIpPortToCallerContext() throws IOException {
22102219
GenericTestUtils.LogCapturer auditLog =

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -225,6 +225,8 @@
225225
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceResponseProto;
226226
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
227227
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthResponseProto;
228+
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetDataNodeBandwidthRequestProto;
229+
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetDataNodeBandwidthResponseProto;
228230
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
229231
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerResponseProto;
230232
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
@@ -415,6 +417,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
415417
protected static final SetBalancerBandwidthResponseProto VOID_SETBALANCERBANDWIDTH_RESPONSE =
416418
SetBalancerBandwidthResponseProto.newBuilder().build();
417419

420+
protected static final SetDataNodeBandwidthResponseProto VOID_SETDATANODEBANDWIDTH_RESPONSE =
421+
SetDataNodeBandwidthResponseProto.newBuilder().build();
422+
418423
protected static final SetAclResponseProto VOID_SETACL_RESPONSE =
419424
SetAclResponseProto.getDefaultInstance();
420425

@@ -1254,6 +1259,18 @@ public SetBalancerBandwidthResponseProto setBalancerBandwidth(
12541259
}
12551260
}
12561261

1262+
@Override
1263+
public SetDataNodeBandwidthResponseProto setDataNodeBandwidth(
1264+
RpcController controller, SetDataNodeBandwidthRequestProto req)
1265+
throws ServiceException {
1266+
try {
1267+
server.setDataNodeBandwidth(req.getBandwidth(), req.getType());
1268+
return VOID_SETDATANODEBANDWIDTH_RESPONSE;
1269+
} catch (IOException e) {
1270+
throw new ServiceException(e);
1271+
}
1272+
}
1273+
12571274
@Override
12581275
public GetDataEncryptionKeyResponseProto getDataEncryptionKey(
12591276
RpcController controller, GetDataEncryptionKeyRequestProto request)

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
4040
import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
4141
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
42+
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DataNodeBandwidthCommandProto;
4243
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
4344
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECReconstructionCommandProto;
4445
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -90,6 +91,7 @@
9091
import org.apache.hadoop.hdfs.server.common.StorageInfo;
9192
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
9293
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
94+
import org.apache.hadoop.hdfs.server.protocol.DataNodeBandwidthCommand;
9395
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
9496
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand;
9597
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
@@ -458,6 +460,8 @@ public static DatanodeCommand convert(DatanodeCommandProto proto) {
458460
switch (proto.getCmdType()) {
459461
case BalancerBandwidthCommand:
460462
return PBHelper.convert(proto.getBalancerCmd());
463+
case DataNodeBandwidthCommand:
464+
return PBHelper.convert(proto.getBandwidthCmd());
461465
case BlockCommand:
462466
return PBHelper.convert(proto.getBlkCmd());
463467
case BlockRecoveryCommand:
@@ -483,6 +487,13 @@ public static BalancerBandwidthCommandProto convert(
483487
.setBandwidth(bbCmd.getBalancerBandwidthValue()).build();
484488
}
485489

490+
public static DataNodeBandwidthCommandProto convert(
491+
DataNodeBandwidthCommand bbCmd) {
492+
return DataNodeBandwidthCommandProto.newBuilder()
493+
.setBandwidth(bbCmd.getDataNodeBandwidthValue())
494+
.setType(bbCmd.getDataNodeBandwidthType()).build();
495+
}
496+
486497
public static KeyUpdateCommandProto convert(KeyUpdateCommand cmd) {
487498
return KeyUpdateCommandProto.newBuilder()
488499
.setKeys(convert(cmd.getExportedKeys())).build();
@@ -572,6 +583,11 @@ public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) {
572583
.setBalancerCmd(
573584
PBHelper.convert((BalancerBandwidthCommand) datanodeCommand));
574585
break;
586+
case DatanodeProtocol.DNA_DATANODEBANDWIDTHUPDATE:
587+
builder.setCmdType(DatanodeCommandProto.Type.DataNodeBandwidthCommand)
588+
.setBandwidthCmd(
589+
PBHelper.convert((DataNodeBandwidthCommand) datanodeCommand));
590+
break;
575591
case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
576592
builder
577593
.setCmdType(DatanodeCommandProto.Type.KeyUpdateCommand)
@@ -709,6 +725,12 @@ public static BalancerBandwidthCommand convert(
709725
return new BalancerBandwidthCommand(balancerCmd.getBandwidth());
710726
}
711727

728+
public static DataNodeBandwidthCommand convert(
729+
DataNodeBandwidthCommandProto bandwidthCmd) {
730+
return new DataNodeBandwidthCommand(bandwidthCmd.getBandwidth(),
731+
bandwidthCmd.getType());
732+
}
733+
712734
public static ReceivedDeletedBlockInfoProto convert(
713735
ReceivedDeletedBlockInfo receivedDeletedBlockInfo) {
714736
ReceivedDeletedBlockInfoProto.Builder builder =

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -193,6 +193,8 @@ public Type getType() {
193193
// specified datanode, this value will be set back to 0.
194194
private long bandwidth;
195195

196+
private Map<String, Long> throttlers = new HashMap<>();
197+
196198
/** A queue of blocks to be replicated by this datanode */
197199
private final BlockQueue<BlockTargetPair> replicateBlocks =
198200
new BlockQueue<>();
@@ -1027,6 +1029,21 @@ public synchronized void setBalancerBandwidth(long bandwidth) {
10271029
this.bandwidth = bandwidth;
10281030
}
10291031

1032+
/**
1033+
* @return bandwidth throttlers for this datanode
1034+
*/
1035+
public synchronized Map<String, Long> getDataNodeBandwidth() {
1036+
return this.throttlers;
1037+
}
1038+
1039+
/**
1040+
* @param bandwidth Bandwidth in bytes per second for this datanode
1041+
* @param type DataNode bandwidth type.
1042+
*/
1043+
public synchronized void setDataNodeBandwidth(long bandwidth, String type) {
1044+
this.throttlers.put(type, bandwidth);
1045+
}
1046+
10301047
@Override
10311048
public String dumpDatanode() {
10321049
StringBuilder sb = new StringBuilder(super.dumpDatanode());

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1974,6 +1974,14 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
19741974
nodeinfo.setBalancerBandwidth(0);
19751975
}
19761976

1977+
// check for datanode bandwidth update
1978+
if (!nodeinfo.getDataNodeBandwidth().isEmpty()) {
1979+
for (Map.Entry<String, Long> entry : nodeinfo.getDataNodeBandwidth().entrySet()){
1980+
cmds.add(new DataNodeBandwidthCommand(entry.getValue(), entry.getKey()));
1981+
}
1982+
nodeinfo.getDataNodeBandwidth().clear();
1983+
}
1984+
19771985
Preconditions.checkNotNull(slowPeerTracker, "slowPeerTracker should not be un-assigned");
19781986

19791987
if (slowPeerTracker.isSlowPeerTrackerEnabled()) {
@@ -2085,7 +2093,25 @@ public void setBalancerBandwidth(long bandwidth) throws IOException {
20852093
}
20862094
}
20872095
}
2088-
2096+
2097+
/**
2098+
* Tell all datanodes to reset the bandwidth of the specified type.
2099+
*
2100+
* A system administrator can tune the datanode bandwidth dynamically by calling
2101+
* "dfsadmin -setDataNodeBandwidth newbandwidth -type <transfer|write|read>"
2102+
*
2103+
* @param bandwidth Bandwidth in bytes per second for all datanodes.
2104+
* @param type DataNode bandwidth type.
2105+
* @throws IOException
2106+
*/
2107+
public void setDataNodeBandwidth(long bandwidth, String type) throws IOException {
2108+
synchronized(this) {
2109+
for (DatanodeDescriptor nodeInfo : datanodeMap.values()) {
2110+
nodeInfo.setDataNodeBandwidth(bandwidth, type);
2111+
}
2112+
}
2113+
}
2114+
20892115
public void markAllDatanodesStaleAndSetKeyUpdateIfNeed() {
20902116
LOG.info("Marking all datanodes as stale and schedule update block token if need.");
20912117
synchronized (this) {

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.apache.hadoop.hdfs.server.protocol.*;
3232
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
3333
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
34+
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
3435
import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
3536
import org.apache.hadoop.util.Lists;
3637
import org.apache.hadoop.util.Sets;
@@ -806,6 +807,40 @@ assert getBlockPoolId().equals(bp) :
806807
dxcs.balanceThrottler.setBandwidth(bandwidth);
807808
}
808809
break;
810+
case DatanodeProtocol.DNA_DATANODEBANDWIDTHUPDATE:
811+
LOG.info("DatanodeCommand action: DNA_DATANODEBANDWIDTHUPDATE");
812+
long dnBandwidth =
813+
((DataNodeBandwidthCommand) cmd).getDataNodeBandwidthValue();
814+
long oldBandwidth;
815+
String type =
816+
((DataNodeBandwidthCommand) cmd).getDataNodeBandwidthType();
817+
if (dnBandwidth >= 0) {
818+
DataXceiverServer dxcs = dn.getXferServer();
819+
if (type.equals("transfer")) {
820+
oldBandwidth = dxcs.getTransferThrottler() == null ? 0 :
821+
dxcs.getTransferThrottler().getBandwidth();
822+
DataTransferThrottler transferThrottler = dnBandwidth == 0 ? null :
823+
new DataTransferThrottler(dnBandwidth);
824+
dxcs.setTransferThrottler(transferThrottler);
825+
} else if (type.equals("write")) {
826+
oldBandwidth = dxcs.getWriteThrottler() == null ? 0 :
827+
dxcs.getWriteThrottler().getBandwidth();
828+
DataTransferThrottler writeThrottler = dnBandwidth == 0 ? null :
829+
new DataTransferThrottler(dnBandwidth);
830+
dxcs.setWriteThrottler(writeThrottler);
831+
} else if (type.equals("read")) {
832+
oldBandwidth = dxcs.getReadThrottler() == null ? 0 :
833+
dxcs.getReadThrottler().getBandwidth();
834+
DataTransferThrottler readThrottler = dnBandwidth == 0 ? null :
835+
new DataTransferThrottler(dnBandwidth);
836+
dxcs.setReadThrottler(readThrottler);
837+
} else {
838+
break;
839+
}
840+
LOG.info("Updated " + type + " throttler bandwidth from "
841+
+ oldBandwidth + " bytes/s to: " + dnBandwidth + " bytes/s.");
842+
}
843+
break;
809844
case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION:
810845
LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY");
811846
Collection<BlockECReconstructionInfo> ecTasks =

0 commit comments

Comments
 (0)