-
Notifications
You must be signed in to change notification settings - Fork 8.7k
/
ErasureCodingWorker.java
141 lines (125 loc) · 5.2 KB
/
ErasureCodingWorker.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
/**
* 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.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
import org.apache.hadoop.util.Daemon;
import org.slf4j.Logger;
import java.util.Collection;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
/**
* ErasureCodingWorker handles the erasure coding reconstruction work commands.
* These commands would be issued from Namenode as part of Datanode's heart beat
* response. BPOfferService delegates the work to this class for handling EC
* commands.
*/
@InterfaceAudience.Private
public final class ErasureCodingWorker {
private static final Logger LOG = DataNode.LOG;
private final DataNode datanode;
private final Configuration conf;
private ThreadPoolExecutor stripedReconstructionPool;
private ThreadPoolExecutor stripedReadPool;
public ErasureCodingWorker(Configuration conf, DataNode datanode) {
this.datanode = datanode;
this.conf = conf;
initializeStripedReadThreadPool(conf.getInt(
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_THREADS_KEY,
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_THREADS_DEFAULT));
initializeStripedBlkReconstructionThreadPool(conf.getInt(
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_BLK_THREADS_KEY,
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_BLK_THREADS_DEFAULT));
}
private void initializeStripedReadThreadPool(int num) {
LOG.debug("Using striped reads; pool threads={}", num);
stripedReadPool = new ThreadPoolExecutor(1, num, 60, TimeUnit.SECONDS,
new SynchronousQueue<Runnable>(),
new Daemon.DaemonFactory() {
private final AtomicInteger threadIndex = new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
Thread t = super.newThread(r);
t.setName("stripedRead-" + threadIndex.getAndIncrement());
return t;
}
},
new ThreadPoolExecutor.CallerRunsPolicy() {
@Override
public void rejectedExecution(Runnable runnable,
ThreadPoolExecutor e) {
LOG.info("Execution for striped reading rejected, "
+ "Executing in current thread");
// will run in the current thread
super.rejectedExecution(runnable, e);
}
});
stripedReadPool.allowCoreThreadTimeOut(true);
}
private void initializeStripedBlkReconstructionThreadPool(int numThreads) {
LOG.debug("Using striped block reconstruction; pool threads={}",
numThreads);
stripedReconstructionPool = DFSUtilClient.getThreadPoolExecutor(2,
numThreads, 60, "StripedBlockReconstruction-", false);
stripedReconstructionPool.allowCoreThreadTimeOut(true);
}
/**
* Handles the Erasure Coding reconstruction work commands.
*
* @param ecTasks BlockECReconstructionInfo
*
*/
public void processErasureCodingTasks(
Collection<BlockECReconstructionInfo> ecTasks) {
for (BlockECReconstructionInfo reconInfo : ecTasks) {
try {
StripedReconstructionInfo stripedReconInfo =
new StripedReconstructionInfo(
reconInfo.getExtendedBlock(), reconInfo.getErasureCodingPolicy(),
reconInfo.getLiveBlockIndices(), reconInfo.getSourceDnInfos(),
reconInfo.getTargetDnInfos(), reconInfo.getTargetStorageTypes());
final StripedBlockReconstructor task =
new StripedBlockReconstructor(this, stripedReconInfo);
if (task.hasValidTargets()) {
stripedReconstructionPool.submit(task);
} else {
LOG.warn("No missing internal block. Skip reconstruction for task:{}",
reconInfo);
}
} catch (Throwable e) {
LOG.warn("Failed to reconstruct striped block {}",
reconInfo.getExtendedBlock().getLocalBlock(), e);
}
}
}
DataNode getDatanode() {
return datanode;
}
Configuration getConf() {
return conf;
}
ThreadPoolExecutor getStripedReadPool() {
return stripedReadPool;
}
}