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.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.CellScanner;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.HRegionInfo;
35 import org.apache.hadoop.hbase.HRegionLocation;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.TableName;
38 import org.apache.hadoop.hbase.client.HConnection;
39 import org.apache.hadoop.hbase.client.RegionServerCallable;
40 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
41 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
42 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
43 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
44 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
45 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
46 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
47 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
48 import org.apache.hadoop.hbase.util.Pair;
49
50 import com.google.protobuf.ServiceException;
51
52
53
54
55
56
57
58 @InterfaceAudience.Private
59 public class WALEditsReplaySink {
60
61 private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
62 private static final int MAX_BATCH_SIZE = 1024;
63
64 private final Configuration conf;
65 private final HConnection conn;
66 private final TableName tableName;
67 private final MetricsWALEditsReplay metrics;
68 private final AtomicLong totalReplayedEdits = new AtomicLong();
69 private final boolean skipErrors;
70 private final int replayTimeout;
71
72
73
74
75
76
77
78
79 public WALEditsReplaySink(Configuration conf, TableName tableName, HConnection conn)
80 throws IOException {
81 this.conf = conf;
82 this.metrics = new MetricsWALEditsReplay();
83 this.conn = conn;
84 this.tableName = tableName;
85 this.skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
86 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS);
87
88 this.replayTimeout = conf.getInt("hbase.regionserver.logreplay.timeout", 60000);
89 }
90
91
92
93
94
95
96 public void replayEntries(List<Pair<HRegionLocation, HLog.Entry>> entries) throws IOException {
97 if (entries.size() == 0) {
98 return;
99 }
100
101 int batchSize = entries.size();
102 Map<HRegionInfo, List<HLog.Entry>> entriesByRegion =
103 new HashMap<HRegionInfo, List<HLog.Entry>>();
104 HRegionLocation loc = null;
105 HLog.Entry entry = null;
106 List<HLog.Entry> regionEntries = null;
107
108 for (int i = 0; i < batchSize; i++) {
109 loc = entries.get(i).getFirst();
110 entry = entries.get(i).getSecond();
111 if (entriesByRegion.containsKey(loc.getRegionInfo())) {
112 regionEntries = entriesByRegion.get(loc.getRegionInfo());
113 } else {
114 regionEntries = new ArrayList<HLog.Entry>();
115 entriesByRegion.put(loc.getRegionInfo(), regionEntries);
116 }
117 regionEntries.add(entry);
118 }
119
120 long startTime = EnvironmentEdgeManager.currentTimeMillis();
121
122
123 for (Map.Entry<HRegionInfo, List<HLog.Entry>> _entry : entriesByRegion.entrySet()) {
124 HRegionInfo curRegion = _entry.getKey();
125 List<HLog.Entry> allActions = _entry.getValue();
126
127 int totalActions = allActions.size();
128 int replayedActions = 0;
129 int curBatchSize = 0;
130 for (; replayedActions < totalActions;) {
131 curBatchSize = (totalActions > (MAX_BATCH_SIZE + replayedActions)) ? MAX_BATCH_SIZE
132 : (totalActions - replayedActions);
133 replayEdits(loc, curRegion, allActions.subList(replayedActions,
134 replayedActions + curBatchSize));
135 replayedActions += curBatchSize;
136 }
137 }
138
139 long endTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
140 LOG.debug("number of rows:" + entries.size() + " are sent by batch! spent " + endTime
141 + "(ms)!");
142
143 metrics.updateReplayTime(endTime);
144 metrics.updateReplayBatchSize(batchSize);
145
146 this.totalReplayedEdits.addAndGet(batchSize);
147 }
148
149
150
151
152
153 public String getStats() {
154 return this.totalReplayedEdits.get() == 0 ? "" : "Sink: total replayed edits: "
155 + this.totalReplayedEdits;
156 }
157
158 private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo,
159 final List<HLog.Entry> entries) throws IOException {
160 try {
161 RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf, null);
162 ReplayServerCallable<ReplicateWALEntryResponse> callable =
163 new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.tableName, regionLoc,
164 regionInfo, entries);
165 factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, this.replayTimeout);
166 } catch (IOException ie) {
167 if (skipErrors) {
168 LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
169 + "=true so continuing replayEdits with error:" + ie.getMessage());
170 } else {
171 throw ie;
172 }
173 }
174 }
175
176
177
178
179
180 class ReplayServerCallable<R> extends RegionServerCallable<ReplicateWALEntryResponse> {
181 private HRegionInfo regionInfo;
182 private List<HLog.Entry> entries;
183
184 ReplayServerCallable(final HConnection connection, final TableName tableName,
185 final HRegionLocation regionLoc, final HRegionInfo regionInfo,
186 final List<HLog.Entry> entries) {
187 super(connection, tableName, null);
188 this.entries = entries;
189 this.regionInfo = regionInfo;
190 setLocation(regionLoc);
191 }
192
193 @Override
194 public ReplicateWALEntryResponse call() throws IOException {
195 try {
196 replayToServer(this.regionInfo, this.entries);
197 } catch (ServiceException se) {
198 throw ProtobufUtil.getRemoteException(se);
199 }
200 return null;
201 }
202
203 private void replayToServer(HRegionInfo regionInfo, List<HLog.Entry> entries)
204 throws IOException, ServiceException {
205 if (entries.isEmpty()) return;
206
207 HLog.Entry[] entriesArray = new HLog.Entry[entries.size()];
208 entriesArray = entries.toArray(entriesArray);
209 AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName());
210
211 Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
212 ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
213 try {
214 PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond());
215 remoteSvr.replay(controller, p.getFirst());
216 } catch (ServiceException se) {
217 throw ProtobufUtil.getRemoteException(se);
218 }
219 }
220
221 @Override
222 public void prepare(boolean reload) throws IOException {
223 if (!reload) return;
224
225
226
227 boolean skip = false;
228 for (HLog.Entry entry : this.entries) {
229 WALEdit edit = entry.getEdit();
230 List<KeyValue> kvs = edit.getKeyValues();
231 for (KeyValue kv : kvs) {
232
233 setLocation(conn.locateRegion(tableName, kv.getRow()));
234 skip = true;
235 break;
236 }
237
238 if (skip) break;
239 }
240 }
241 }
242 }