1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver.wal;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.HashMap;
24 import java.util.List;
25 import java.util.Map;
26 import java.util.concurrent.atomic.AtomicLong;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.TableName;
33 import org.apache.hadoop.hbase.DoNotRetryIOException;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.hbase.HRegionInfo;
36 import org.apache.hadoop.hbase.HRegionLocation;
37 import org.apache.hadoop.hbase.client.Action;
38 import org.apache.hadoop.hbase.client.HConnection;
39 import org.apache.hadoop.hbase.client.RegionServerCallable;
40 import org.apache.hadoop.hbase.client.Row;
41 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
42 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
43 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
44 import org.apache.hadoop.hbase.protobuf.RequestConverter;
45 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
46 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
47 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
48 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
49 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
50 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
51 import org.apache.hadoop.hbase.util.Pair;
52
53 import com.google.protobuf.ServiceException;
54
55
56
57
58
59
60
61 @InterfaceAudience.Private
62 public class WALEditsReplaySink {
63
64 private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
65 private static final int MAX_BATCH_SIZE = 3000;
66
67 private final Configuration conf;
68 private final HConnection conn;
69 private final TableName tableName;
70 private final MetricsWALEditsReplay metrics;
71 private final AtomicLong totalReplayedEdits = new AtomicLong();
72 private final boolean skipErrors;
73 private final int replayTimeout;
74
75
76
77
78
79
80
81
82 public WALEditsReplaySink(Configuration conf, TableName tableName, HConnection conn)
83 throws IOException {
84 this.conf = conf;
85 this.metrics = new MetricsWALEditsReplay();
86 this.conn = conn;
87 this.tableName = tableName;
88 this.skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
89 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS);
90
91 this.replayTimeout = conf.getInt("hbase.regionserver.logreplay.timeout", 60000);
92 }
93
94
95
96
97
98
99 public void replayEntries(List<Pair<HRegionLocation, Row>> actions) throws IOException {
100 if (actions.size() == 0) {
101 return;
102 }
103
104 int batchSize = actions.size();
105 int dataSize = 0;
106 Map<HRegionInfo, List<Action<Row>>> actionsByRegion =
107 new HashMap<HRegionInfo, List<Action<Row>>>();
108 HRegionLocation loc = null;
109 Row row = null;
110 List<Action<Row>> regionActions = null;
111
112 for (int i = 0; i < batchSize; i++) {
113 loc = actions.get(i).getFirst();
114 row = actions.get(i).getSecond();
115 if (actionsByRegion.containsKey(loc.getRegionInfo())) {
116 regionActions = actionsByRegion.get(loc.getRegionInfo());
117 } else {
118 regionActions = new ArrayList<Action<Row>>();
119 actionsByRegion.put(loc.getRegionInfo(), regionActions);
120 }
121 Action<Row> action = new Action<Row>(row, i);
122 regionActions.add(action);
123 dataSize += row.getRow().length;
124 }
125
126 long startTime = EnvironmentEdgeManager.currentTimeMillis();
127
128
129 for (HRegionInfo curRegion : actionsByRegion.keySet()) {
130 List<Action<Row>> allActions = actionsByRegion.get(curRegion);
131
132 int totalActions = allActions.size();
133 int replayedActions = 0;
134 int curBatchSize = 0;
135 for (; replayedActions < totalActions;) {
136 curBatchSize = (totalActions > (MAX_BATCH_SIZE + replayedActions)) ? MAX_BATCH_SIZE
137 : (totalActions - replayedActions);
138 replayEdits(loc, curRegion, allActions.subList(replayedActions,
139 replayedActions + curBatchSize));
140 replayedActions += curBatchSize;
141 }
142 }
143
144 long endTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
145 LOG.debug("number of rows:" + actions.size() + " are sent by batch! spent " + endTime
146 + "(ms)!");
147
148 metrics.updateReplayTime(endTime);
149 metrics.updateReplayBatchSize(batchSize);
150 metrics.updateReplayDataSize(dataSize);
151
152 this.totalReplayedEdits.addAndGet(batchSize);
153 }
154
155
156
157
158
159 public String getStats() {
160 return this.totalReplayedEdits.get() == 0 ? "" : "Sink: total replayed edits: "
161 + this.totalReplayedEdits;
162 }
163
164 private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo,
165 final List<Action<Row>> actions) throws IOException {
166 try {
167 RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf);
168 ReplayServerCallable<MultiResponse> callable = new ReplayServerCallable<MultiResponse>(
169 this.conn, this.tableName, regionLoc, regionInfo, actions);
170 factory.<MultiResponse> newCaller().callWithRetries(callable, this.replayTimeout);
171 } catch (IOException ie) {
172 if (skipErrors) {
173 LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
174 + "=true so continuing replayEdits with error:" + ie.getMessage());
175 } else {
176 throw ie;
177 }
178 }
179 }
180
181
182
183
184
185 class ReplayServerCallable<R> extends RegionServerCallable<MultiResponse> {
186 private HRegionInfo regionInfo;
187 private List<Action<Row>> actions;
188
189 ReplayServerCallable(final HConnection connection, final TableName tableName,
190 final HRegionLocation regionLoc, final HRegionInfo regionInfo,
191 final List<Action<Row>> actions) {
192 super(connection, tableName, null);
193 this.actions = actions;
194 this.regionInfo = regionInfo;
195 setLocation(regionLoc);
196 }
197
198 @Override
199 public MultiResponse call() throws IOException {
200 try {
201 replayToServer(this.regionInfo, this.actions);
202 } catch (ServiceException se) {
203 throw ProtobufUtil.getRemoteException(se);
204 }
205 return null;
206 }
207
208 private void replayToServer(HRegionInfo regionInfo, List<Action<Row>> actions)
209 throws IOException, ServiceException {
210 AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName());
211 MultiRequest request = RequestConverter.buildMultiRequest(regionInfo.getRegionName(),
212 actions);
213 MultiResponse protoResults = remoteSvr.replay(null, request);
214
215 List<ActionResult> resultList = protoResults.getResultList();
216 for (int i = 0, n = resultList.size(); i < n; i++) {
217 ActionResult result = resultList.get(i);
218 if (result.hasException()) {
219 Throwable t = ProtobufUtil.toException(result.getException());
220 if (!skipErrors) {
221 IOException ie = new IOException();
222 ie.initCause(t);
223
224 throw ie;
225 } else {
226 LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
227 + "=true so continuing replayToServer with error:" + t.getMessage());
228 return;
229 }
230 }
231 }
232 }
233
234 @Override
235 public void prepare(boolean reload) throws IOException {
236 if (!reload) return;
237
238
239
240 for (Action<Row> action : actions) {
241
242 setLocation(conn.locateRegion(tableName, action.getAction().getRow()));
243 break;
244 }
245 }
246 }
247 }