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.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 (HRegionInfo curRegion : entriesByRegion.keySet()) {
124 List<HLog.Entry> allActions = entriesByRegion.get(curRegion);
125
126 int totalActions = allActions.size();
127 int replayedActions = 0;
128 int curBatchSize = 0;
129 for (; replayedActions < totalActions;) {
130 curBatchSize = (totalActions > (MAX_BATCH_SIZE + replayedActions)) ? MAX_BATCH_SIZE
131 : (totalActions - replayedActions);
132 replayEdits(loc, curRegion, allActions.subList(replayedActions,
133 replayedActions + curBatchSize));
134 replayedActions += curBatchSize;
135 }
136 }
137
138 long endTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
139 LOG.debug("number of rows:" + entries.size() + " are sent by batch! spent " + endTime
140 + "(ms)!");
141
142 metrics.updateReplayTime(endTime);
143 metrics.updateReplayBatchSize(batchSize);
144
145 this.totalReplayedEdits.addAndGet(batchSize);
146 }
147
148
149
150
151
152 public String getStats() {
153 return this.totalReplayedEdits.get() == 0 ? "" : "Sink: total replayed edits: "
154 + this.totalReplayedEdits;
155 }
156
157 private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo,
158 final List<HLog.Entry> entries) throws IOException {
159 try {
160 RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf);
161 ReplayServerCallable<ReplicateWALEntryResponse> callable =
162 new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.tableName, regionLoc,
163 regionInfo, entries);
164 factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, this.replayTimeout);
165 } catch (IOException ie) {
166 if (skipErrors) {
167 LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
168 + "=true so continuing replayEdits with error:" + ie.getMessage());
169 } else {
170 throw ie;
171 }
172 }
173 }
174
175
176
177
178
179 class ReplayServerCallable<R> extends RegionServerCallable<ReplicateWALEntryResponse> {
180 private HRegionInfo regionInfo;
181 private List<HLog.Entry> entries;
182
183 ReplayServerCallable(final HConnection connection, final TableName tableName,
184 final HRegionLocation regionLoc, final HRegionInfo regionInfo,
185 final List<HLog.Entry> entries) {
186 super(connection, tableName, null);
187 this.entries = entries;
188 this.regionInfo = regionInfo;
189 setLocation(regionLoc);
190 }
191
192 @Override
193 public ReplicateWALEntryResponse call() throws IOException {
194 try {
195 replayToServer(this.regionInfo, this.entries);
196 } catch (ServiceException se) {
197 throw ProtobufUtil.getRemoteException(se);
198 }
199 return null;
200 }
201
202 private void replayToServer(HRegionInfo regionInfo, List<HLog.Entry> entries)
203 throws IOException, ServiceException {
204 if (entries.isEmpty()) return;
205
206 HLog.Entry[] entriesArray = new HLog.Entry[entries.size()];
207 entriesArray = entries.toArray(entriesArray);
208 AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName());
209
210 Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
211 ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
212 try {
213 PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond());
214 remoteSvr.replay(controller, p.getFirst());
215 } catch (ServiceException se) {
216 throw ProtobufUtil.getRemoteException(se);
217 }
218 }
219
220 @Override
221 public void prepare(boolean reload) throws IOException {
222 if (!reload) return;
223
224
225
226 boolean skip = false;
227 for (HLog.Entry entry : this.entries) {
228 WALEdit edit = entry.getEdit();
229 List<KeyValue> kvs = edit.getKeyValues();
230 for (KeyValue kv : kvs) {
231
232 if (kv.matchingFamily(WALEdit.METAFAMILY)) continue;
233
234 setLocation(conn.locateRegion(tableName, kv.getRow()));
235 skip = true;
236 break;
237 }
238
239 if (skip) break;
240 }
241 }
242 }
243 }