View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.lang.Math;
24  import java.lang.annotation.Retention;
25  import java.lang.annotation.RetentionPolicy;
26  import java.net.InetSocketAddress;
27  import java.net.UnknownHostException;
28  import java.util.ArrayList;
29  import java.util.Collections;
30  import java.util.HashMap;
31  import java.util.Iterator;
32  import java.util.List;
33  import java.util.Map;
34  import java.util.NavigableMap;
35  import java.util.Set;
36  import java.util.TreeSet;
37  import java.util.concurrent.ConcurrentHashMap;
38  import java.util.concurrent.atomic.AtomicLong;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.Cell;
44  import org.apache.hadoop.hbase.CellScannable;
45  import org.apache.hadoop.hbase.CellScanner;
46  import org.apache.hadoop.hbase.CellUtil;
47  import org.apache.hadoop.hbase.DoNotRetryIOException;
48  import org.apache.hadoop.hbase.DroppedSnapshotException;
49  import org.apache.hadoop.hbase.HBaseIOException;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.HTableDescriptor;
53  import org.apache.hadoop.hbase.MetaTableAccessor;
54  import org.apache.hadoop.hbase.NotServingRegionException;
55  import org.apache.hadoop.hbase.ServerName;
56  import org.apache.hadoop.hbase.TableName;
57  import org.apache.hadoop.hbase.UnknownScannerException;
58  import org.apache.hadoop.hbase.classification.InterfaceAudience;
59  import org.apache.hadoop.hbase.client.Append;
60  import org.apache.hadoop.hbase.client.ConnectionUtils;
61  import org.apache.hadoop.hbase.client.Delete;
62  import org.apache.hadoop.hbase.client.Get;
63  import org.apache.hadoop.hbase.client.Increment;
64  import org.apache.hadoop.hbase.client.Mutation;
65  import org.apache.hadoop.hbase.client.Put;
66  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
67  import org.apache.hadoop.hbase.client.Result;
68  import org.apache.hadoop.hbase.client.RowMutations;
69  import org.apache.hadoop.hbase.client.Scan;
70  import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
71  import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
72  import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
73  import org.apache.hadoop.hbase.exceptions.OperationConflictException;
74  import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
75  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
76  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
77  import org.apache.hadoop.hbase.io.hfile.HFile;
78  import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
79  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
80  import org.apache.hadoop.hbase.ipc.PriorityFunction;
81  import org.apache.hadoop.hbase.ipc.RpcCallContext;
82  import org.apache.hadoop.hbase.ipc.RpcServer;
83  import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
84  import org.apache.hadoop.hbase.ipc.RpcServerInterface;
85  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
86  import org.apache.hadoop.hbase.ipc.ServerRpcController;
87  import org.apache.hadoop.hbase.master.MasterRpcServices;
88  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
89  import org.apache.hadoop.hbase.protobuf.RequestConverter;
90  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
91  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
92  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
93  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
94  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
95  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
96  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
97  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
98  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
99  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
100 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
101 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
102 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
103 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
104 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
105 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileResponse;
106 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse;
108 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
109 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
110 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
111 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
112 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
113 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
114 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
115 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
116 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
117 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse;
118 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
119 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse;
120 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
121 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
122 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
123 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
124 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
125 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
126 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
127 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
128 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
129 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
130 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition;
131 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
132 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
133 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
135 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
136 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
137 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
139 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
140 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
141 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
142 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
143 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
144 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
145 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
146 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
147 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
148 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
149 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
150 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
151 import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
152 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
153 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
154 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
155 import org.apache.hadoop.hbase.wal.WAL;
156 import org.apache.hadoop.hbase.wal.WALKey;
157 import org.apache.hadoop.hbase.wal.WALSplitter;
158 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
159 import org.apache.hadoop.hbase.util.ByteStringer;
160 import org.apache.hadoop.hbase.util.Bytes;
161 import org.apache.hadoop.hbase.util.Counter;
162 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
163 import org.apache.hadoop.hbase.util.Pair;
164 import org.apache.hadoop.hbase.util.Strings;
165 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
166 import org.apache.hadoop.net.DNS;
167 import org.apache.zookeeper.KeeperException;
168 
169 import com.google.protobuf.ByteString;
170 import com.google.protobuf.Message;
171 import com.google.protobuf.RpcController;
172 import com.google.protobuf.ServiceException;
173 import com.google.protobuf.TextFormat;
174 
175 /**
176  * Implements the regionserver RPC services.
177  */
178 @InterfaceAudience.Private
179 @SuppressWarnings("deprecation")
180 public class RSRpcServices implements HBaseRPCErrorHandler,
181     AdminService.BlockingInterface, ClientService.BlockingInterface, PriorityFunction {
182   protected static final Log LOG = LogFactory.getLog(RSRpcServices.class);
183 
184   /** RPC scheduler to use for the region server. */
185   public static final String REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS =
186     "hbase.region.server.rpc.scheduler.factory.class";
187 
188   // Request counter. (Includes requests that are not serviced by regions.)
189   final Counter requestCount = new Counter();
190   // Server to handle client requests.
191   final RpcServerInterface rpcServer;
192   final InetSocketAddress isa;
193 
194   private final HRegionServer regionServer;
195   private final long maxScannerResultSize;
196 
197   // The reference to the priority extraction function
198   private final PriorityFunction priority;
199 
200   private final AtomicLong scannerIdGen = new AtomicLong(0L);
201   private final ConcurrentHashMap<String, RegionScannerHolder> scanners =
202     new ConcurrentHashMap<String, RegionScannerHolder>();
203 
204   /**
205    * The lease timeout period for client scanners (milliseconds).
206    */
207   private final int scannerLeaseTimeoutPeriod;
208 
209   /**
210    * Holder class which holds the RegionScanner and nextCallSeq together.
211    */
212   private static class RegionScannerHolder {
213     private RegionScanner s;
214     private long nextCallSeq = 0L;
215     private HRegion r;
216 
217     public RegionScannerHolder(RegionScanner s, HRegion r) {
218       this.s = s;
219       this.r = r;
220     }
221   }
222 
223   /**
224    * Instantiated as a scanner lease. If the lease times out, the scanner is
225    * closed
226    */
227   private class ScannerListener implements LeaseListener {
228     private final String scannerName;
229 
230     ScannerListener(final String n) {
231       this.scannerName = n;
232     }
233 
234     @Override
235     public void leaseExpired() {
236       RegionScannerHolder rsh = scanners.remove(this.scannerName);
237       if (rsh != null) {
238         RegionScanner s = rsh.s;
239         LOG.info("Scanner " + this.scannerName + " lease expired on region "
240           + s.getRegionInfo().getRegionNameAsString());
241         try {
242           HRegion region = regionServer.getRegion(s.getRegionInfo().getRegionName());
243           if (region != null && region.getCoprocessorHost() != null) {
244             region.getCoprocessorHost().preScannerClose(s);
245           }
246 
247           s.close();
248           if (region != null && region.getCoprocessorHost() != null) {
249             region.getCoprocessorHost().postScannerClose(s);
250           }
251         } catch (IOException e) {
252           LOG.error("Closing scanner for "
253             + s.getRegionInfo().getRegionNameAsString(), e);
254         }
255       } else {
256         LOG.warn("Scanner " + this.scannerName + " lease expired, but no related" +
257           " scanner found, hence no chance to close that related scanner!");
258       }
259     }
260   }
261 
262   private static ResultOrException getResultOrException(
263       final ClientProtos.Result r, final int index, final ClientProtos.RegionLoadStats stats) {
264     return getResultOrException(ResponseConverter.buildActionResult(r, stats), index);
265   }
266 
267   private static ResultOrException getResultOrException(final Exception e, final int index) {
268     return getResultOrException(ResponseConverter.buildActionResult(e), index);
269   }
270 
271   private static ResultOrException getResultOrException(
272       final ResultOrException.Builder builder, final int index) {
273     return builder.setIndex(index).build();
274   }
275 
276   /**
277    * Starts the nonce operation for a mutation, if needed.
278    * @param mutation Mutation.
279    * @param nonceGroup Nonce group from the request.
280    * @returns Nonce used (can be NO_NONCE).
281    */
282   private long startNonceOperation(final MutationProto mutation, long nonceGroup)
283       throws IOException, OperationConflictException {
284     if (regionServer.nonceManager == null || !mutation.hasNonce()) return HConstants.NO_NONCE;
285     boolean canProceed = false;
286     try {
287       canProceed = regionServer.nonceManager.startOperation(
288         nonceGroup, mutation.getNonce(), regionServer);
289     } catch (InterruptedException ex) {
290       throw new InterruptedIOException("Nonce start operation interrupted");
291     }
292     if (!canProceed) {
293       // TODO: instead, we could convert append/increment to get w/mvcc
294       String message = "The operation with nonce {" + nonceGroup + ", " + mutation.getNonce()
295         + "} on row [" + Bytes.toString(mutation.getRow().toByteArray())
296         + "] may have already completed";
297       throw new OperationConflictException(message);
298     }
299     return mutation.getNonce();
300   }
301 
302   /**
303    * Ends nonce operation for a mutation, if needed.
304    * @param mutation Mutation.
305    * @param nonceGroup Nonce group from the request. Always 0 in initial implementation.
306    * @param success Whether the operation for this nonce has succeeded.
307    */
308   private void endNonceOperation(final MutationProto mutation,
309       long nonceGroup, boolean success) {
310     if (regionServer.nonceManager != null && mutation.hasNonce()) {
311       regionServer.nonceManager.endOperation(nonceGroup, mutation.getNonce(), success);
312     }
313   }
314 
315   /**
316    * @return True if current call supports cellblocks
317    */
318   private boolean isClientCellBlockSupport() {
319     RpcCallContext context = RpcServer.getCurrentCall();
320     return context != null && context.isClientCellBlockSupport();
321   }
322 
323   private void addResult(final MutateResponse.Builder builder,
324       final Result result, final PayloadCarryingRpcController rpcc) {
325     if (result == null) return;
326     if (isClientCellBlockSupport()) {
327       builder.setResult(ProtobufUtil.toResultNoData(result));
328       rpcc.setCellScanner(result.cellScanner());
329     } else {
330       ClientProtos.Result pbr = ProtobufUtil.toResult(result);
331       builder.setResult(pbr);
332     }
333   }
334 
335   private void addResults(final ScanResponse.Builder builder, final List<Result> results,
336       final RpcController controller, boolean isDefaultRegion) {
337     builder.setStale(!isDefaultRegion);
338     if (results == null || results.isEmpty()) return;
339     if (isClientCellBlockSupport()) {
340       for (Result res : results) {
341         builder.addCellsPerResult(res.size());
342       }
343       ((PayloadCarryingRpcController)controller).
344         setCellScanner(CellUtil.createCellScanner(results));
345     } else {
346       for (Result res: results) {
347         ClientProtos.Result pbr = ProtobufUtil.toResult(res);
348         builder.addResults(pbr);
349       }
350     }
351   }
352 
353   /**
354    * Mutate a list of rows atomically.
355    *
356    * @param region
357    * @param actions
358    * @param cellScanner if non-null, the mutation data -- the Cell content.
359    * @throws IOException
360    */
361   private ClientProtos.RegionLoadStats mutateRows(final HRegion region,
362       final List<ClientProtos.Action> actions,
363       final CellScanner cellScanner) throws IOException {
364     if (!region.getRegionInfo().isMetaTable()) {
365       regionServer.cacheFlusher.reclaimMemStoreMemory();
366     }
367     RowMutations rm = null;
368     for (ClientProtos.Action action: actions) {
369       if (action.hasGet()) {
370         throw new DoNotRetryIOException("Atomic put and/or delete only, not a Get=" +
371           action.getGet());
372       }
373       MutationType type = action.getMutation().getMutateType();
374       if (rm == null) {
375         rm = new RowMutations(action.getMutation().getRow().toByteArray());
376       }
377       switch (type) {
378       case PUT:
379         rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
380         break;
381       case DELETE:
382         rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
383         break;
384       default:
385           throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
386       }
387     }
388     region.mutateRow(rm);
389     return region.getRegionStats();
390   }
391 
392   /**
393    * Mutate a list of rows atomically.
394    *
395    * @param region
396    * @param actions
397    * @param cellScanner if non-null, the mutation data -- the Cell content.
398    * @param row
399    * @param family
400    * @param qualifier
401    * @param compareOp
402    * @param comparator @throws IOException
403    */
404   private boolean checkAndRowMutate(final HRegion region, final List<ClientProtos.Action> actions,
405       final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
406       CompareOp compareOp, ByteArrayComparable comparator) throws IOException {
407     if (!region.getRegionInfo().isMetaTable()) {
408       regionServer.cacheFlusher.reclaimMemStoreMemory();
409     }
410     RowMutations rm = null;
411     for (ClientProtos.Action action: actions) {
412       if (action.hasGet()) {
413         throw new DoNotRetryIOException("Atomic put and/or delete only, not a Get=" +
414             action.getGet());
415       }
416       MutationType type = action.getMutation().getMutateType();
417       if (rm == null) {
418         rm = new RowMutations(action.getMutation().getRow().toByteArray());
419       }
420       switch (type) {
421       case PUT:
422         rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
423         break;
424       case DELETE:
425         rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
426         break;
427       default:
428         throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
429       }
430     }
431     return region.checkAndRowMutate(row, family, qualifier, compareOp, comparator, rm, Boolean.TRUE);
432   }
433 
434   /**
435    * Execute an append mutation.
436    *
437    * @param region
438    * @param m
439    * @param cellScanner
440    * @return result to return to client if default operation should be
441    * bypassed as indicated by RegionObserver, null otherwise
442    * @throws IOException
443    */
444   private Result append(final HRegion region, final MutationProto m,
445       final CellScanner cellScanner, long nonceGroup) throws IOException {
446     long before = EnvironmentEdgeManager.currentTime();
447     Append append = ProtobufUtil.toAppend(m, cellScanner);
448     Result r = null;
449     if (region.getCoprocessorHost() != null) {
450       r = region.getCoprocessorHost().preAppend(append);
451     }
452     if (r == null) {
453       long nonce = startNonceOperation(m, nonceGroup);
454       boolean success = false;
455       try {
456         r = region.append(append, nonceGroup, nonce);
457         success = true;
458       } finally {
459         endNonceOperation(m, nonceGroup, success);
460       }
461       if (region.getCoprocessorHost() != null) {
462         region.getCoprocessorHost().postAppend(append, r);
463       }
464     }
465     if (regionServer.metricsRegionServer != null) {
466       regionServer.metricsRegionServer.updateAppend(
467         EnvironmentEdgeManager.currentTime() - before);
468     }
469     return r;
470   }
471 
472   /**
473    * Execute an increment mutation.
474    *
475    * @param region
476    * @param mutation
477    * @return the Result
478    * @throws IOException
479    */
480   private Result increment(final HRegion region, final MutationProto mutation,
481       final CellScanner cells, long nonceGroup) throws IOException {
482     long before = EnvironmentEdgeManager.currentTime();
483     Increment increment = ProtobufUtil.toIncrement(mutation, cells);
484     Result r = null;
485     if (region.getCoprocessorHost() != null) {
486       r = region.getCoprocessorHost().preIncrement(increment);
487     }
488     if (r == null) {
489       long nonce = startNonceOperation(mutation, nonceGroup);
490       boolean success = false;
491       try {
492         r = region.increment(increment, nonceGroup, nonce);
493         success = true;
494       } finally {
495         endNonceOperation(mutation, nonceGroup, success);
496       }
497       if (region.getCoprocessorHost() != null) {
498         r = region.getCoprocessorHost().postIncrement(increment, r);
499       }
500     }
501     if (regionServer.metricsRegionServer != null) {
502       regionServer.metricsRegionServer.updateIncrement(
503         EnvironmentEdgeManager.currentTime() - before);
504     }
505     return r;
506   }
507 
508   /**
509    * Run through the regionMutation <code>rm</code> and per Mutation, do the work, and then when
510    * done, add an instance of a {@link ResultOrException} that corresponds to each Mutation.
511    * @param region
512    * @param actions
513    * @param cellScanner
514    * @param builder
515    * @param cellsToReturn  Could be null. May be allocated in this method.  This is what this
516    * method returns as a 'result'.
517    * @return Return the <code>cellScanner</code> passed
518    */
519   private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
520       final RegionAction actions, final CellScanner cellScanner,
521       final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup) {
522     // Gather up CONTIGUOUS Puts and Deletes in this mutations List.  Idea is that rather than do
523     // one at a time, we instead pass them in batch.  Be aware that the corresponding
524     // ResultOrException instance that matches each Put or Delete is then added down in the
525     // doBatchOp call.  We should be staying aligned though the Put and Delete are deferred/batched
526     List<ClientProtos.Action> mutations = null;
527     for (ClientProtos.Action action: actions.getActionList()) {
528       ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null;
529       try {
530         Result r = null;
531         if (action.hasGet()) {
532           Get get = ProtobufUtil.toGet(action.getGet());
533           r = region.get(get);
534         } else if (action.hasServiceCall()) {
535           resultOrExceptionBuilder = ResultOrException.newBuilder();
536           try {
537             Message result = execServiceOnRegion(region, action.getServiceCall());
538             ClientProtos.CoprocessorServiceResult.Builder serviceResultBuilder =
539                 ClientProtos.CoprocessorServiceResult.newBuilder();
540             resultOrExceptionBuilder.setServiceResult(
541                 serviceResultBuilder.setValue(
542                   serviceResultBuilder.getValueBuilder()
543                     .setName(result.getClass().getName())
544                     .setValue(result.toByteString())));
545           } catch (IOException ioe) {
546             resultOrExceptionBuilder.setException(ResponseConverter.buildException(ioe));
547           }
548         } else if (action.hasMutation()) {
549           MutationType type = action.getMutation().getMutateType();
550           if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null &&
551               !mutations.isEmpty()) {
552             // Flush out any Puts or Deletes already collected.
553             doBatchOp(builder, region, mutations, cellScanner);
554             mutations.clear();
555           }
556           switch (type) {
557           case APPEND:
558             r = append(region, action.getMutation(), cellScanner, nonceGroup);
559             break;
560           case INCREMENT:
561             r = increment(region, action.getMutation(), cellScanner,  nonceGroup);
562             break;
563           case PUT:
564           case DELETE:
565             // Collect the individual mutations and apply in a batch
566             if (mutations == null) {
567               mutations = new ArrayList<ClientProtos.Action>(actions.getActionCount());
568             }
569             mutations.add(action);
570             break;
571           default:
572             throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
573           }
574         } else {
575           throw new HBaseIOException("Unexpected Action type");
576         }
577         if (r != null) {
578           ClientProtos.Result pbResult = null;
579           if (isClientCellBlockSupport()) {
580             pbResult = ProtobufUtil.toResultNoData(r);
581             //  Hard to guess the size here.  Just make a rough guess.
582             if (cellsToReturn == null) cellsToReturn = new ArrayList<CellScannable>();
583             cellsToReturn.add(r);
584           } else {
585             pbResult = ProtobufUtil.toResult(r);
586           }
587           resultOrExceptionBuilder =
588             ClientProtos.ResultOrException.newBuilder().setResult(pbResult);
589         }
590         // Could get to here and there was no result and no exception.  Presumes we added
591         // a Put or Delete to the collecting Mutations List for adding later.  In this
592         // case the corresponding ResultOrException instance for the Put or Delete will be added
593         // down in the doBatchOp method call rather than up here.
594       } catch (IOException ie) {
595         resultOrExceptionBuilder = ResultOrException.newBuilder().
596           setException(ResponseConverter.buildException(ie));
597       }
598       if (resultOrExceptionBuilder != null) {
599         // Propagate index.
600         resultOrExceptionBuilder.setIndex(action.getIndex());
601         builder.addResultOrException(resultOrExceptionBuilder.build());
602       }
603     }
604     // Finish up any outstanding mutations
605     if (mutations != null && !mutations.isEmpty()) {
606       doBatchOp(builder, region, mutations, cellScanner);
607     }
608     return cellsToReturn;
609   }
610 
611   /**
612    * Execute a list of Put/Delete mutations.
613    *
614    * @param builder
615    * @param region
616    * @param mutations
617    */
618   private void doBatchOp(final RegionActionResult.Builder builder, final HRegion region,
619       final List<ClientProtos.Action> mutations, final CellScanner cells) {
620     Mutation[] mArray = new Mutation[mutations.size()];
621     long before = EnvironmentEdgeManager.currentTime();
622     boolean batchContainsPuts = false, batchContainsDelete = false;
623     try {
624       int i = 0;
625       for (ClientProtos.Action action: mutations) {
626         MutationProto m = action.getMutation();
627         Mutation mutation;
628         if (m.getMutateType() == MutationType.PUT) {
629           mutation = ProtobufUtil.toPut(m, cells);
630           batchContainsPuts = true;
631         } else {
632           mutation = ProtobufUtil.toDelete(m, cells);
633           batchContainsDelete = true;
634         }
635         mArray[i++] = mutation;
636       }
637 
638       if (!region.getRegionInfo().isMetaTable()) {
639         regionServer.cacheFlusher.reclaimMemStoreMemory();
640       }
641 
642       OperationStatus codes[] = region.batchMutate(mArray);
643       for (i = 0; i < codes.length; i++) {
644         int index = mutations.get(i).getIndex();
645         Exception e = null;
646         switch (codes[i].getOperationStatusCode()) {
647           case BAD_FAMILY:
648             e = new NoSuchColumnFamilyException(codes[i].getExceptionMsg());
649             builder.addResultOrException(getResultOrException(e, index));
650             break;
651 
652           case SANITY_CHECK_FAILURE:
653             e = new FailedSanityCheckException(codes[i].getExceptionMsg());
654             builder.addResultOrException(getResultOrException(e, index));
655             break;
656 
657           default:
658             e = new DoNotRetryIOException(codes[i].getExceptionMsg());
659             builder.addResultOrException(getResultOrException(e, index));
660             break;
661 
662           case SUCCESS:
663             builder.addResultOrException(getResultOrException(
664               ClientProtos.Result.getDefaultInstance(), index, region.getRegionStats()));
665             break;
666         }
667       }
668     } catch (IOException ie) {
669       for (int i = 0; i < mutations.size(); i++) {
670         builder.addResultOrException(getResultOrException(ie, mutations.get(i).getIndex()));
671       }
672     }
673     if (regionServer.metricsRegionServer != null) {
674       long after = EnvironmentEdgeManager.currentTime();
675       if (batchContainsPuts) {
676         regionServer.metricsRegionServer.updatePut(after - before);
677       }
678       if (batchContainsDelete) {
679         regionServer.metricsRegionServer.updateDelete(after - before);
680       }
681     }
682   }
683 
684   /**
685    * Execute a list of Put/Delete mutations. The function returns OperationStatus instead of
686    * constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse.
687    * @param region
688    * @param mutations
689    * @param replaySeqId
690    * @return an array of OperationStatus which internally contains the OperationStatusCode and the
691    *         exceptionMessage if any
692    * @throws IOException
693    */
694   private OperationStatus [] doReplayBatchOp(final HRegion region,
695       final List<WALSplitter.MutationReplay> mutations, long replaySeqId) throws IOException {
696 
697     long before = EnvironmentEdgeManager.currentTime();
698     boolean batchContainsPuts = false, batchContainsDelete = false;
699     try {
700       for (Iterator<WALSplitter.MutationReplay> it = mutations.iterator(); it.hasNext();) {
701         WALSplitter.MutationReplay m = it.next();
702 
703         if (m.type == MutationType.PUT) {
704           batchContainsPuts = true;
705         } else {
706           batchContainsDelete = true;
707         }
708 
709         NavigableMap<byte[], List<Cell>> map = m.mutation.getFamilyCellMap();
710         List<Cell> metaCells = map.get(WALEdit.METAFAMILY);
711         if (metaCells != null && !metaCells.isEmpty()) {
712           for (Cell metaCell : metaCells) {
713             CompactionDescriptor compactionDesc = WALEdit.getCompaction(metaCell);
714             if (compactionDesc != null) {
715               region.completeCompactionMarker(compactionDesc);
716             }
717           }
718           it.remove();
719         }
720       }
721       requestCount.add(mutations.size());
722       if (!region.getRegionInfo().isMetaTable()) {
723         regionServer.cacheFlusher.reclaimMemStoreMemory();
724       }
725       return region.batchReplay(mutations.toArray(
726         new WALSplitter.MutationReplay[mutations.size()]), replaySeqId);
727     } finally {
728       if (regionServer.metricsRegionServer != null) {
729         long after = EnvironmentEdgeManager.currentTime();
730           if (batchContainsPuts) {
731           regionServer.metricsRegionServer.updatePut(after - before);
732         }
733         if (batchContainsDelete) {
734           regionServer.metricsRegionServer.updateDelete(after - before);
735         }
736       }
737     }
738   }
739 
740   private void closeAllScanners() {
741     // Close any outstanding scanners. Means they'll get an UnknownScanner
742     // exception next time they come in.
743     for (Map.Entry<String, RegionScannerHolder> e : scanners.entrySet()) {
744       try {
745         e.getValue().s.close();
746       } catch (IOException ioe) {
747         LOG.warn("Closing scanner " + e.getKey(), ioe);
748       }
749     }
750   }
751 
752   public RSRpcServices(HRegionServer rs) throws IOException {
753     regionServer = rs;
754 
755     RpcSchedulerFactory rpcSchedulerFactory;
756     try {
757       Class<?> rpcSchedulerFactoryClass = rs.conf.getClass(
758           REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
759           SimpleRpcSchedulerFactory.class);
760       rpcSchedulerFactory = ((RpcSchedulerFactory) rpcSchedulerFactoryClass.newInstance());
761     } catch (InstantiationException e) {
762       throw new IllegalArgumentException(e);
763     } catch (IllegalAccessException e) {
764       throw new IllegalArgumentException(e);
765     }
766     // Server to handle client requests.
767     InetSocketAddress initialIsa;
768     InetSocketAddress bindAddress;
769     if(this instanceof MasterRpcServices) {
770       String hostname = getHostname(rs.conf, true);
771       int port = rs.conf.getInt(HConstants.MASTER_PORT,
772           HConstants.DEFAULT_REGIONSERVER_PORT);
773       // Creation of a HSA will force a resolve.
774       initialIsa = new InetSocketAddress(hostname, port);
775       bindAddress = new InetSocketAddress(rs.conf.get("hbase.master.ipc.address", hostname), port);
776     } else {
777       String hostname = getHostname(rs.conf, false);
778       int port = rs.conf.getInt(HConstants.REGIONSERVER_PORT,
779         HConstants.DEFAULT_REGIONSERVER_PORT);
780       // Creation of a HSA will force a resolve.
781       initialIsa = new InetSocketAddress(hostname, port);
782       bindAddress = new InetSocketAddress(
783         rs.conf.get("hbase.regionserver.ipc.address", hostname), port);
784     }
785     if (initialIsa.getAddress() == null) {
786       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
787     }
788     priority = new AnnotationReadingPriorityFunction(this);
789     String name = rs.getProcessName() + "/" + initialIsa.toString();
790     // Set how many times to retry talking to another server over HConnection.
791     ConnectionUtils.setServerSideHConnectionRetriesConfig(rs.conf, name, LOG);
792     rpcServer = new RpcServer(rs, name, getServices(),
793       bindAddress, // use final bindAddress for this server.
794       rs.conf,
795       rpcSchedulerFactory.create(rs.conf, this, rs));
796 
797     scannerLeaseTimeoutPeriod = rs.conf.getInt(
798       HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
799       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
800     maxScannerResultSize = rs.conf.getLong(
801       HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY,
802       HConstants.DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE);
803 
804     // Set our address, however we need the final port that was given to rpcServer
805     isa = new InetSocketAddress(initialIsa.getHostName(), rpcServer.getListenerAddress().getPort());
806     rpcServer.setErrorHandler(this);
807     rs.setName(name);
808   }
809 
810   public static String getHostname(Configuration conf, boolean isMaster)
811       throws UnknownHostException {
812     String masterOrRS = isMaster ? "master" : "regionserver";
813     return Strings.domainNamePointerToHostName(DNS.getDefaultHost(
814         conf.get("hbase." + masterOrRS + ".dns.interface", "default"),
815         conf.get("hbase." + masterOrRS + ".dns.nameserver", "default")));
816   }
817 
818   RegionScanner getScanner(long scannerId) {
819     String scannerIdString = Long.toString(scannerId);
820     RegionScannerHolder scannerHolder = scanners.get(scannerIdString);
821     if (scannerHolder != null) {
822       return scannerHolder.s;
823     }
824     return null;
825   }
826 
827   /**
828    * Get the vtime associated with the scanner.
829    * Currently the vtime is the number of "next" calls.
830    */
831   long getScannerVirtualTime(long scannerId) {
832     String scannerIdString = Long.toString(scannerId);
833     RegionScannerHolder scannerHolder = scanners.get(scannerIdString);
834     if (scannerHolder != null) {
835       return scannerHolder.nextCallSeq;
836     }
837     return 0L;
838   }
839 
840   long addScanner(RegionScanner s, HRegion r) throws LeaseStillHeldException {
841     long scannerId = this.scannerIdGen.incrementAndGet();
842     String scannerName = String.valueOf(scannerId);
843 
844     RegionScannerHolder existing =
845       scanners.putIfAbsent(scannerName, new RegionScannerHolder(s, r));
846     assert existing == null : "scannerId must be unique within regionserver's whole lifecycle!";
847 
848     regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
849         new ScannerListener(scannerName));
850     return scannerId;
851   }
852 
853   /**
854    * Find the HRegion based on a region specifier
855    *
856    * @param regionSpecifier the region specifier
857    * @return the corresponding region
858    * @throws IOException if the specifier is not null,
859    *    but failed to find the region
860    */
861   HRegion getRegion(
862       final RegionSpecifier regionSpecifier) throws IOException {
863     return regionServer.getRegionByEncodedName(regionSpecifier.getValue().toByteArray(),
864         ProtobufUtil.getRegionEncodedName(regionSpecifier));
865   }
866 
867   PriorityFunction getPriority() {
868     return priority;
869   }
870 
871   Configuration getConfiguration() {
872     return regionServer.getConfiguration();
873   }
874 
875   void start() {
876     rpcServer.start();
877   }
878 
879   void stop() {
880     closeAllScanners();
881     rpcServer.stop();
882   }
883 
884   /**
885    * Called to verify that this server is up and running.
886    *
887    * @throws IOException
888    */
889   protected void checkOpen() throws IOException {
890     if (regionServer.isStopped() || regionServer.isAborted()) {
891       throw new RegionServerStoppedException("Server " + regionServer.serverName
892         + " not running" + (regionServer.isAborted() ? ", aborting" : ""));
893     }
894     if (!regionServer.fsOk) {
895       throw new RegionServerStoppedException("File system not available");
896     }
897     if (!regionServer.isOnline()) {
898       throw new ServerNotRunningYetException("Server is not running yet");
899     }
900   }
901 
902   /**
903    * @return list of blocking services and their security info classes that this server supports
904    */
905   protected List<BlockingServiceAndInterface> getServices() {
906     List<BlockingServiceAndInterface> bssi = new ArrayList<BlockingServiceAndInterface>(2);
907     bssi.add(new BlockingServiceAndInterface(
908       ClientService.newReflectiveBlockingService(this),
909       ClientService.BlockingInterface.class));
910     bssi.add(new BlockingServiceAndInterface(
911       AdminService.newReflectiveBlockingService(this),
912       AdminService.BlockingInterface.class));
913     return bssi;
914   }
915 
916   @Retention(RetentionPolicy.RUNTIME)
917   protected @interface QosPriority {
918     int priority() default HConstants.NORMAL_QOS;
919   }
920 
921   public InetSocketAddress getSocketAddress() {
922     return isa;
923   }
924 
925   @Override
926   public int getPriority(RequestHeader header, Message param) {
927     return priority.getPriority(header, param);
928   }
929 
930   @Override
931   public long getDeadline(RequestHeader header, Message param) {
932     return priority.getDeadline(header, param);
933   }
934 
935   /*
936    * Check if an OOME and, if so, abort immediately to avoid creating more objects.
937    *
938    * @param e
939    *
940    * @return True if we OOME'd and are aborting.
941    */
942   @Override
943   public boolean checkOOME(final Throwable e) {
944     boolean stop = false;
945     try {
946       if (e instanceof OutOfMemoryError
947           || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
948           || (e.getMessage() != null && e.getMessage().contains(
949               "java.lang.OutOfMemoryError"))) {
950         stop = true;
951         LOG.fatal("Run out of memory; " + getClass().getSimpleName()
952           + " will abort itself immediately", e);
953       }
954     } finally {
955       if (stop) {
956         Runtime.getRuntime().halt(1);
957       }
958     }
959     return stop;
960   }
961 
962   /**
963    * Close a region on the region server.
964    *
965    * @param controller the RPC controller
966    * @param request the request
967    * @throws ServiceException
968    */
969   @Override
970   @QosPriority(priority=HConstants.ADMIN_QOS)
971   public CloseRegionResponse closeRegion(final RpcController controller,
972       final CloseRegionRequest request) throws ServiceException {
973     final ServerName sn = (request.hasDestinationServer() ?
974       ProtobufUtil.toServerName(request.getDestinationServer()) : null);
975 
976     try {
977       checkOpen();
978       if (request.hasServerStartCode()) {
979         // check that we are the same server that this RPC is intended for.
980         long serverStartCode = request.getServerStartCode();
981         if (regionServer.serverName.getStartcode() !=  serverStartCode) {
982           throw new ServiceException(new DoNotRetryIOException("This RPC was intended for a " +
983               "different server with startCode: " + serverStartCode + ", this server is: "
984               + regionServer.serverName));
985         }
986       }
987       final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());
988 
989       // Can be null if we're calling close on a region that's not online
990       final HRegion region = regionServer.getFromOnlineRegions(encodedRegionName);
991       if ((region  != null) && (region .getCoprocessorHost() != null)) {
992         region.getCoprocessorHost().preClose(false);
993       }
994 
995       requestCount.increment();
996       LOG.info("Close " + encodedRegionName + ", moving to " + sn);
997       CloseRegionCoordination.CloseRegionDetails crd = regionServer.getCoordinatedStateManager()
998         .getCloseRegionCoordination().parseFromProtoRequest(request);
999 
1000       boolean closed = regionServer.closeRegion(encodedRegionName, false, crd, sn);
1001       CloseRegionResponse.Builder builder = CloseRegionResponse.newBuilder().setClosed(closed);
1002       return builder.build();
1003     } catch (IOException ie) {
1004       throw new ServiceException(ie);
1005     }
1006   }
1007 
1008   /**
1009    * Compact a region on the region server.
1010    *
1011    * @param controller the RPC controller
1012    * @param request the request
1013    * @throws ServiceException
1014    */
1015   @Override
1016   @QosPriority(priority=HConstants.ADMIN_QOS)
1017   public CompactRegionResponse compactRegion(final RpcController controller,
1018       final CompactRegionRequest request) throws ServiceException {
1019     try {
1020       checkOpen();
1021       requestCount.increment();
1022       HRegion region = getRegion(request.getRegion());
1023       region.startRegionOperation(Operation.COMPACT_REGION);
1024       LOG.info("Compacting " + region.getRegionNameAsString());
1025       boolean major = false;
1026       byte [] family = null;
1027       Store store = null;
1028       if (request.hasFamily()) {
1029         family = request.getFamily().toByteArray();
1030         store = region.getStore(family);
1031         if (store == null) {
1032           throw new ServiceException(new IOException("column family " + Bytes.toString(family)
1033             + " does not exist in region " + region.getRegionNameAsString()));
1034         }
1035       }
1036       if (request.hasMajor()) {
1037         major = request.getMajor();
1038       }
1039       if (major) {
1040         if (family != null) {
1041           store.triggerMajorCompaction();
1042         } else {
1043           region.triggerMajorCompaction();
1044         }
1045       }
1046 
1047       String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):"";
1048       if (LOG.isTraceEnabled()) {
1049         LOG.trace("User-triggered compaction requested for region "
1050           + region.getRegionNameAsString() + familyLogMsg);
1051       }
1052       String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
1053       if(family != null) {
1054         regionServer.compactSplitThread.requestCompaction(region, store, log,
1055           Store.PRIORITY_USER, null);
1056       } else {
1057         regionServer.compactSplitThread.requestCompaction(region, log,
1058           Store.PRIORITY_USER, null);
1059       }
1060       return CompactRegionResponse.newBuilder().build();
1061     } catch (IOException ie) {
1062       throw new ServiceException(ie);
1063     }
1064   }
1065 
1066   /**
1067    * Flush a region on the region server.
1068    *
1069    * @param controller the RPC controller
1070    * @param request the request
1071    * @throws ServiceException
1072    */
1073   @Override
1074   @QosPriority(priority=HConstants.ADMIN_QOS)
1075   public FlushRegionResponse flushRegion(final RpcController controller,
1076       final FlushRegionRequest request) throws ServiceException {
1077     try {
1078       checkOpen();
1079       requestCount.increment();
1080       HRegion region = getRegion(request.getRegion());
1081       LOG.info("Flushing " + region.getRegionNameAsString());
1082       boolean shouldFlush = true;
1083       if (request.hasIfOlderThanTs()) {
1084         shouldFlush = region.getLastFlushTime() < request.getIfOlderThanTs();
1085       }
1086       FlushRegionResponse.Builder builder = FlushRegionResponse.newBuilder();
1087       if (shouldFlush) {
1088         long startTime = EnvironmentEdgeManager.currentTime();
1089         HRegion.FlushResult flushResult = region.flushcache();
1090         if (flushResult.isFlushSucceeded()) {
1091           long endTime = EnvironmentEdgeManager.currentTime();
1092           regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
1093         }
1094         boolean result = flushResult.isCompactionNeeded();
1095         if (result) {
1096           regionServer.compactSplitThread.requestSystemCompaction(region,
1097             "Compaction through user triggered flush");
1098         }
1099         builder.setFlushed(result);
1100       }
1101       builder.setLastFlushTime(region.getLastFlushTime());
1102       return builder.build();
1103     } catch (DroppedSnapshotException ex) {
1104       // Cache flush can fail in a few places. If it fails in a critical
1105       // section, we get a DroppedSnapshotException and a replay of wal
1106       // is required. Currently the only way to do this is a restart of
1107       // the server.
1108       regionServer.abort("Replay of WAL required. Forcing server shutdown", ex);
1109       throw new ServiceException(ex);
1110     } catch (IOException ie) {
1111       throw new ServiceException(ie);
1112     }
1113   }
1114 
1115   @Override
1116   @QosPriority(priority=HConstants.ADMIN_QOS)
1117   public GetOnlineRegionResponse getOnlineRegion(final RpcController controller,
1118       final GetOnlineRegionRequest request) throws ServiceException {
1119     try {
1120       checkOpen();
1121       requestCount.increment();
1122       Map<String, HRegion> onlineRegions = regionServer.onlineRegions;
1123       List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
1124       for (HRegion region: onlineRegions.values()) {
1125         list.add(region.getRegionInfo());
1126       }
1127       Collections.sort(list);
1128       return ResponseConverter.buildGetOnlineRegionResponse(list);
1129     } catch (IOException ie) {
1130       throw new ServiceException(ie);
1131     }
1132   }
1133 
1134   @Override
1135   @QosPriority(priority=HConstants.ADMIN_QOS)
1136   public GetRegionInfoResponse getRegionInfo(final RpcController controller,
1137       final GetRegionInfoRequest request) throws ServiceException {
1138     try {
1139       checkOpen();
1140       requestCount.increment();
1141       HRegion region = getRegion(request.getRegion());
1142       HRegionInfo info = region.getRegionInfo();
1143       GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
1144       builder.setRegionInfo(HRegionInfo.convert(info));
1145       if (request.hasCompactionState() && request.getCompactionState()) {
1146         builder.setCompactionState(region.getCompactionState());
1147       }
1148       builder.setIsRecovering(region.isRecovering());
1149       return builder.build();
1150     } catch (IOException ie) {
1151       throw new ServiceException(ie);
1152     }
1153   }
1154 
1155   /**
1156    * Get some information of the region server.
1157    *
1158    * @param controller the RPC controller
1159    * @param request the request
1160    * @throws ServiceException
1161    */
1162   @Override
1163   @QosPriority(priority=HConstants.ADMIN_QOS)
1164   public GetServerInfoResponse getServerInfo(final RpcController controller,
1165       final GetServerInfoRequest request) throws ServiceException {
1166     try {
1167       checkOpen();
1168     } catch (IOException ie) {
1169       throw new ServiceException(ie);
1170     }
1171     requestCount.increment();
1172     int infoPort = regionServer.infoServer != null ? regionServer.infoServer.getPort() : -1;
1173     return ResponseConverter.buildGetServerInfoResponse(regionServer.serverName, infoPort);
1174   }
1175 
1176   @Override
1177   @QosPriority(priority=HConstants.ADMIN_QOS)
1178   public GetStoreFileResponse getStoreFile(final RpcController controller,
1179       final GetStoreFileRequest request) throws ServiceException {
1180     try {
1181       checkOpen();
1182       HRegion region = getRegion(request.getRegion());
1183       requestCount.increment();
1184       Set<byte[]> columnFamilies;
1185       if (request.getFamilyCount() == 0) {
1186         columnFamilies = region.getStores().keySet();
1187       } else {
1188         columnFamilies = new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
1189         for (ByteString cf: request.getFamilyList()) {
1190           columnFamilies.add(cf.toByteArray());
1191         }
1192       }
1193       int nCF = columnFamilies.size();
1194       List<String>  fileList = region.getStoreFileList(
1195         columnFamilies.toArray(new byte[nCF][]));
1196       GetStoreFileResponse.Builder builder = GetStoreFileResponse.newBuilder();
1197       builder.addAllStoreFile(fileList);
1198       return builder.build();
1199     } catch (IOException ie) {
1200       throw new ServiceException(ie);
1201     }
1202   }
1203 
1204   /**
1205    * Merge regions on the region server.
1206    *
1207    * @param controller the RPC controller
1208    * @param request the request
1209    * @return merge regions response
1210    * @throws ServiceException
1211    */
1212   @Override
1213   @QosPriority(priority = HConstants.ADMIN_QOS)
1214   public MergeRegionsResponse mergeRegions(final RpcController controller,
1215       final MergeRegionsRequest request) throws ServiceException {
1216     try {
1217       checkOpen();
1218       requestCount.increment();
1219       HRegion regionA = getRegion(request.getRegionA());
1220       HRegion regionB = getRegion(request.getRegionB());
1221       boolean forcible = request.getForcible();
1222       regionA.startRegionOperation(Operation.MERGE_REGION);
1223       regionB.startRegionOperation(Operation.MERGE_REGION);
1224       LOG.info("Receiving merging request for  " + regionA + ", " + regionB
1225           + ",forcible=" + forcible);
1226       long startTime = EnvironmentEdgeManager.currentTime();
1227       HRegion.FlushResult flushResult = regionA.flushcache();
1228       if (flushResult.isFlushSucceeded()) {
1229         long endTime = EnvironmentEdgeManager.currentTime();
1230         regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
1231       }
1232       startTime = EnvironmentEdgeManager.currentTime();
1233       flushResult = regionB.flushcache();
1234       if (flushResult.isFlushSucceeded()) {
1235         long endTime = EnvironmentEdgeManager.currentTime();
1236         regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
1237       }
1238       regionServer.compactSplitThread.requestRegionsMerge(regionA, regionB, forcible);
1239       return MergeRegionsResponse.newBuilder().build();
1240     } catch (IOException ie) {
1241       throw new ServiceException(ie);
1242     }
1243   }
1244 
1245   /**
1246    * Open asynchronously a region or a set of regions on the region server.
1247    *
1248    * The opening is coordinated by ZooKeeper, and this method requires the znode to be created
1249    *  before being called. As a consequence, this method should be called only from the master.
1250    * <p>
1251    * Different manages states for the region are:<ul>
1252    *  <li>region not opened: the region opening will start asynchronously.</li>
1253    *  <li>a close is already in progress: this is considered as an error.</li>
1254    *  <li>an open is already in progress: this new open request will be ignored. This is important
1255    *  because the Master can do multiple requests if it crashes.</li>
1256    *  <li>the region is already opened:  this new open request will be ignored./li>
1257    *  </ul>
1258    * </p>
1259    * <p>
1260    * Bulk assign: If there are more than 1 region to open, it will be considered as a bulk assign.
1261    * For a single region opening, errors are sent through a ServiceException. For bulk assign,
1262    * errors are put in the response as FAILED_OPENING.
1263    * </p>
1264    * @param controller the RPC controller
1265    * @param request the request
1266    * @throws ServiceException
1267    */
1268   @Override
1269   @QosPriority(priority=HConstants.ADMIN_QOS)
1270   public OpenRegionResponse openRegion(final RpcController controller,
1271       final OpenRegionRequest request) throws ServiceException {
1272     requestCount.increment();
1273     if (request.hasServerStartCode()) {
1274       // check that we are the same server that this RPC is intended for.
1275       long serverStartCode = request.getServerStartCode();
1276       if (regionServer.serverName.getStartcode() !=  serverStartCode) {
1277         throw new ServiceException(new DoNotRetryIOException("This RPC was intended for a " +
1278             "different server with startCode: " + serverStartCode + ", this server is: "
1279             + regionServer.serverName));
1280       }
1281     }
1282 
1283     OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();
1284     final int regionCount = request.getOpenInfoCount();
1285     final Map<TableName, HTableDescriptor> htds =
1286         new HashMap<TableName, HTableDescriptor>(regionCount);
1287     final boolean isBulkAssign = regionCount > 1;
1288     try {
1289       checkOpen();
1290     } catch (IOException ie) {
1291       TableName tableName = null;
1292       if (regionCount == 1) {
1293         RegionInfo ri = request.getOpenInfo(0).getRegion();
1294         if (ri != null) {
1295           tableName = ProtobufUtil.toTableName(ri.getTableName());
1296         }
1297       }
1298       if (!TableName.META_TABLE_NAME.equals(tableName)) {
1299         throw new ServiceException(ie);
1300       }
1301       // We are assigning meta, wait a little for regionserver to finish initialization.
1302       int timeout = regionServer.conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
1303         HConstants.DEFAULT_HBASE_RPC_TIMEOUT) >> 2; // Quarter of RPC timeout
1304       long endTime = System.currentTimeMillis() + timeout;
1305       synchronized (regionServer.online) {
1306         try {
1307           while (System.currentTimeMillis() <= endTime
1308               && !regionServer.isStopped() && !regionServer.isOnline()) {
1309             regionServer.online.wait(regionServer.msgInterval);
1310           }
1311           checkOpen();
1312         } catch (InterruptedException t) {
1313           Thread.currentThread().interrupt();
1314           throw new ServiceException(t);
1315         } catch (IOException e) {
1316           throw new ServiceException(e);
1317         }
1318       }
1319     }
1320     for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
1321       final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
1322       OpenRegionCoordination coordination = regionServer.getCoordinatedStateManager().
1323         getOpenRegionCoordination();
1324       OpenRegionCoordination.OpenRegionDetails ord =
1325         coordination.parseFromProtoRequest(regionOpenInfo);
1326 
1327       HTableDescriptor htd;
1328       try {
1329         final HRegion onlineRegion = regionServer.getFromOnlineRegions(region.getEncodedName());
1330         if (onlineRegion != null) {
1331           //Check if the region can actually be opened.
1332           if (onlineRegion.getCoprocessorHost() != null) {
1333             onlineRegion.getCoprocessorHost().preOpen();
1334           }
1335           // See HBASE-5094. Cross check with hbase:meta if still this RS is owning
1336           // the region.
1337           Pair<HRegionInfo, ServerName> p = MetaTableAccessor.getRegion(
1338             regionServer.getConnection(), region.getRegionName());
1339           if (regionServer.serverName.equals(p.getSecond())) {
1340             Boolean closing = regionServer.regionsInTransitionInRS.get(region.getEncodedNameAsBytes());
1341             // Map regionsInTransitionInRSOnly has an entry for a region only if the region
1342             // is in transition on this RS, so here closing can be null. If not null, it can
1343             // be true or false. True means the region is opening on this RS; while false
1344             // means the region is closing. Only return ALREADY_OPENED if not closing (i.e.
1345             // not in transition any more, or still transition to open.
1346             if (!Boolean.FALSE.equals(closing)
1347                 && regionServer.getFromOnlineRegions(region.getEncodedName()) != null) {
1348               LOG.warn("Attempted open of " + region.getEncodedName()
1349                 + " but already online on this server");
1350               builder.addOpeningState(RegionOpeningState.ALREADY_OPENED);
1351               continue;
1352             }
1353           } else {
1354             LOG.warn("The region " + region.getEncodedName() + " is online on this server"
1355               + " but hbase:meta does not have this server - continue opening.");
1356             regionServer.removeFromOnlineRegions(onlineRegion, null);
1357           }
1358         }
1359         LOG.info("Open " + region.getRegionNameAsString());
1360         htd = htds.get(region.getTable());
1361         if (htd == null) {
1362           htd = regionServer.tableDescriptors.get(region.getTable());
1363           htds.put(region.getTable(), htd);
1364         }
1365 
1366         final Boolean previous = regionServer.regionsInTransitionInRS.putIfAbsent(
1367           region.getEncodedNameAsBytes(), Boolean.TRUE);
1368 
1369         if (Boolean.FALSE.equals(previous)) {
1370           // There is a close in progress. We need to mark this open as failed in ZK.
1371 
1372           coordination.tryTransitionFromOfflineToFailedOpen(regionServer, region, ord);
1373 
1374           throw new RegionAlreadyInTransitionException("Received OPEN for the region:"
1375             + region.getRegionNameAsString() + " , which we are already trying to CLOSE ");
1376         }
1377 
1378         if (Boolean.TRUE.equals(previous)) {
1379           // An open is in progress. This is supported, but let's log this.
1380           LOG.info("Receiving OPEN for the region:" +
1381             region.getRegionNameAsString() + " , which we are already trying to OPEN"
1382               + " - ignoring this new request for this region.");
1383         }
1384 
1385         // We are opening this region. If it moves back and forth for whatever reason, we don't
1386         // want to keep returning the stale moved record while we are opening/if we close again.
1387         regionServer.removeFromMovedRegions(region.getEncodedName());
1388 
1389         if (previous == null) {
1390           // check if the region to be opened is marked in recovering state in ZK
1391           if (ZKSplitLog.isRegionMarkedRecoveringInZK(regionServer.getZooKeeper(),
1392               region.getEncodedName())) {
1393             // Check if current region open is for distributedLogReplay. This check is to support
1394             // rolling restart/upgrade where we want to Master/RS see same configuration
1395             if (!regionOpenInfo.hasOpenForDistributedLogReplay()
1396                   || regionOpenInfo.getOpenForDistributedLogReplay()) {
1397               regionServer.recoveringRegions.put(region.getEncodedName(), null);
1398             } else {
1399               // Remove stale recovery region from ZK when we open region not for recovering which
1400               // could happen when turn distributedLogReplay off from on.
1401               List<String> tmpRegions = new ArrayList<String>();
1402               tmpRegions.add(region.getEncodedName());
1403               ZKSplitLog.deleteRecoveringRegionZNodes(regionServer.getZooKeeper(),
1404                 tmpRegions);
1405             }
1406           }
1407           // If there is no action in progress, we can submit a specific handler.
1408           // Need to pass the expected version in the constructor.
1409           if (region.isMetaRegion()) {
1410             regionServer.service.submit(new OpenMetaHandler(
1411               regionServer, regionServer, region, htd, coordination, ord));
1412           } else {
1413             regionServer.updateRegionFavoredNodesMapping(region.getEncodedName(),
1414               regionOpenInfo.getFavoredNodesList());
1415             regionServer.service.submit(new OpenRegionHandler(
1416               regionServer, regionServer, region, htd, coordination, ord));
1417           }
1418         }
1419 
1420         builder.addOpeningState(RegionOpeningState.OPENED);
1421 
1422       } catch (KeeperException zooKeeperEx) {
1423         LOG.error("Can't retrieve recovering state from zookeeper", zooKeeperEx);
1424         throw new ServiceException(zooKeeperEx);
1425       } catch (IOException ie) {
1426         LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie);
1427         if (isBulkAssign) {
1428           builder.addOpeningState(RegionOpeningState.FAILED_OPENING);
1429         } else {
1430           throw new ServiceException(ie);
1431         }
1432       }
1433     }
1434     return builder.build();
1435   }
1436 
1437   /**
1438    * Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is
1439    * that the given mutations will be durable on the receiving RS if this method returns without any
1440    * exception.
1441    * @param controller the RPC controller
1442    * @param request the request
1443    * @throws ServiceException
1444    */
1445   @Override
1446   @QosPriority(priority = HConstants.REPLAY_QOS)
1447   public ReplicateWALEntryResponse replay(final RpcController controller,
1448       final ReplicateWALEntryRequest request) throws ServiceException {
1449     long before = EnvironmentEdgeManager.currentTime();
1450     CellScanner cells = ((PayloadCarryingRpcController) controller).cellScanner();
1451     try {
1452       checkOpen();
1453       List<WALEntry> entries = request.getEntryList();
1454       if (entries == null || entries.isEmpty()) {
1455         // empty input
1456         return ReplicateWALEntryResponse.newBuilder().build();
1457       }
1458       HRegion region = regionServer.getRegionByEncodedName(
1459         entries.get(0).getKey().getEncodedRegionName().toStringUtf8());
1460       RegionCoprocessorHost coprocessorHost = region.getCoprocessorHost();
1461       List<Pair<WALKey, WALEdit>> walEntries = new ArrayList<Pair<WALKey, WALEdit>>();
1462       for (WALEntry entry : entries) {
1463         if (regionServer.nonceManager != null) {
1464           long nonceGroup = entry.getKey().hasNonceGroup()
1465             ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
1466           long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
1467           regionServer.nonceManager.reportOperationFromWal(nonceGroup, nonce, entry.getKey().getWriteTime());
1468         }
1469         Pair<WALKey, WALEdit> walEntry = (coprocessorHost == null) ? null :
1470           new Pair<WALKey, WALEdit>();
1471         List<WALSplitter.MutationReplay> edits = WALSplitter.getMutationsFromWALEntry(entry,
1472           cells, walEntry);
1473         if (coprocessorHost != null) {
1474           // Start coprocessor replay here. The coprocessor is for each WALEdit instead of a
1475           // KeyValue.
1476           if (coprocessorHost.preWALRestore(region.getRegionInfo(), walEntry.getFirst(),
1477             walEntry.getSecond())) {
1478             // if bypass this log entry, ignore it ...
1479             continue;
1480           }
1481           walEntries.add(walEntry);
1482         }
1483         if(edits!=null && !edits.isEmpty()) {
1484           long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
1485             entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
1486           OperationStatus[] result = doReplayBatchOp(region, edits, replaySeqId);
1487           // check if it's a partial success
1488           for (int i = 0; result != null && i < result.length; i++) {
1489             if (result[i] != OperationStatus.SUCCESS) {
1490               throw new IOException(result[i].getExceptionMsg());
1491             }
1492           }
1493         }
1494       }
1495 
1496       //sync wal at the end because ASYNC_WAL is used above
1497       region.syncWal();
1498 
1499       if (coprocessorHost != null) {
1500         for (Pair<WALKey, WALEdit> wal : walEntries) {
1501           coprocessorHost.postWALRestore(region.getRegionInfo(), wal.getFirst(),
1502             wal.getSecond());
1503         }
1504       }
1505       return ReplicateWALEntryResponse.newBuilder().build();
1506     } catch (IOException ie) {
1507       throw new ServiceException(ie);
1508     } finally {
1509       if (regionServer.metricsRegionServer != null) {
1510         regionServer.metricsRegionServer.updateReplay(
1511           EnvironmentEdgeManager.currentTime() - before);
1512       }
1513     }
1514   }
1515 
1516   /**
1517    * Replicate WAL entries on the region server.
1518    *
1519    * @param controller the RPC controller
1520    * @param request the request
1521    * @throws ServiceException
1522    */
1523   @Override
1524   @QosPriority(priority=HConstants.REPLICATION_QOS)
1525   public ReplicateWALEntryResponse replicateWALEntry(final RpcController controller,
1526       final ReplicateWALEntryRequest request) throws ServiceException {
1527     try {
1528       if (regionServer.replicationSinkHandler != null) {
1529         checkOpen();
1530         requestCount.increment();
1531         List<WALEntry> entries = request.getEntryList();
1532         CellScanner cellScanner = ((PayloadCarryingRpcController)controller).cellScanner();
1533         regionServer.getRegionServerCoprocessorHost().preReplicateLogEntries(entries, cellScanner);
1534         regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner);
1535         regionServer.getRegionServerCoprocessorHost().postReplicateLogEntries(entries, cellScanner);
1536       }
1537       return ReplicateWALEntryResponse.newBuilder().build();
1538     } catch (IOException ie) {
1539       throw new ServiceException(ie);
1540     }
1541   }
1542 
1543   /**
1544    * Roll the WAL writer of the region server.
1545    * @param controller the RPC controller
1546    * @param request the request
1547    * @throws ServiceException
1548    */
1549   @Override
1550   public RollWALWriterResponse rollWALWriter(final RpcController controller,
1551       final RollWALWriterRequest request) throws ServiceException {
1552     try {
1553       checkOpen();
1554       requestCount.increment();
1555       regionServer.getRegionServerCoprocessorHost().preRollWALWriterRequest();
1556       regionServer.walRoller.requestRollAll();
1557       regionServer.getRegionServerCoprocessorHost().postRollWALWriterRequest();
1558       RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();
1559       return builder.build();
1560     } catch (IOException ie) {
1561       throw new ServiceException(ie);
1562     }
1563   }
1564 
1565   /**
1566    * Split a region on the region server.
1567    *
1568    * @param controller the RPC controller
1569    * @param request the request
1570    * @throws ServiceException
1571    */
1572   @Override
1573   @QosPriority(priority=HConstants.ADMIN_QOS)
1574   public SplitRegionResponse splitRegion(final RpcController controller,
1575       final SplitRegionRequest request) throws ServiceException {
1576     try {
1577       checkOpen();
1578       requestCount.increment();
1579       HRegion region = getRegion(request.getRegion());
1580       region.startRegionOperation(Operation.SPLIT_REGION);
1581       LOG.info("Splitting " + region.getRegionNameAsString());
1582       long startTime = EnvironmentEdgeManager.currentTime();
1583       HRegion.FlushResult flushResult = region.flushcache();
1584       if (flushResult.isFlushSucceeded()) {
1585         long endTime = EnvironmentEdgeManager.currentTime();
1586         regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
1587       }
1588       byte[] splitPoint = null;
1589       if (request.hasSplitPoint()) {
1590         splitPoint = request.getSplitPoint().toByteArray();
1591       }
1592       region.forceSplit(splitPoint);
1593       regionServer.compactSplitThread.requestSplit(region, region.checkSplit());
1594       return SplitRegionResponse.newBuilder().build();
1595     } catch (IOException ie) {
1596       throw new ServiceException(ie);
1597     }
1598   }
1599 
1600   /**
1601    * Stop the region server.
1602    *
1603    * @param controller the RPC controller
1604    * @param request the request
1605    * @throws ServiceException
1606    */
1607   @Override
1608   @QosPriority(priority=HConstants.ADMIN_QOS)
1609   public StopServerResponse stopServer(final RpcController controller,
1610       final StopServerRequest request) throws ServiceException {
1611     requestCount.increment();
1612     String reason = request.getReason();
1613     regionServer.stop(reason);
1614     return StopServerResponse.newBuilder().build();
1615   }
1616 
1617   @Override
1618   public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
1619       UpdateFavoredNodesRequest request) throws ServiceException {
1620     List<UpdateFavoredNodesRequest.RegionUpdateInfo> openInfoList = request.getUpdateInfoList();
1621     UpdateFavoredNodesResponse.Builder respBuilder = UpdateFavoredNodesResponse.newBuilder();
1622     for (UpdateFavoredNodesRequest.RegionUpdateInfo regionUpdateInfo : openInfoList) {
1623       HRegionInfo hri = HRegionInfo.convert(regionUpdateInfo.getRegion());
1624       regionServer.updateRegionFavoredNodesMapping(hri.getEncodedName(),
1625         regionUpdateInfo.getFavoredNodesList());
1626     }
1627     respBuilder.setResponse(openInfoList.size());
1628     return respBuilder.build();
1629   }
1630 
1631   /**
1632    * Atomically bulk load several HFiles into an open region
1633    * @return true if successful, false is failed but recoverably (no action)
1634    * @throws IOException if failed unrecoverably
1635    */
1636   @Override
1637   public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller,
1638       final BulkLoadHFileRequest request) throws ServiceException {
1639     try {
1640       checkOpen();
1641       requestCount.increment();
1642       HRegion region = getRegion(request.getRegion());
1643       List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
1644       for (FamilyPath familyPath: request.getFamilyPathList()) {
1645         familyPaths.add(new Pair<byte[], String>(familyPath.getFamily().toByteArray(),
1646           familyPath.getPath()));
1647       }
1648       boolean bypass = false;
1649       if (region.getCoprocessorHost() != null) {
1650         bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
1651       }
1652       boolean loaded = false;
1653       if (!bypass) {
1654         loaded = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum());
1655       }
1656       if (region.getCoprocessorHost() != null) {
1657         loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
1658       }
1659       BulkLoadHFileResponse.Builder builder = BulkLoadHFileResponse.newBuilder();
1660       builder.setLoaded(loaded);
1661       return builder.build();
1662     } catch (IOException ie) {
1663       throw new ServiceException(ie);
1664     }
1665   }
1666 
1667   @Override
1668   public CoprocessorServiceResponse execService(final RpcController controller,
1669       final CoprocessorServiceRequest request) throws ServiceException {
1670     try {
1671       checkOpen();
1672       requestCount.increment();
1673       HRegion region = getRegion(request.getRegion());
1674       Message result = execServiceOnRegion(region, request.getCall());
1675       CoprocessorServiceResponse.Builder builder =
1676         CoprocessorServiceResponse.newBuilder();
1677       builder.setRegion(RequestConverter.buildRegionSpecifier(
1678         RegionSpecifierType.REGION_NAME, region.getRegionName()));
1679       builder.setValue(
1680         builder.getValueBuilder().setName(result.getClass().getName())
1681           .setValue(result.toByteString()));
1682       return builder.build();
1683     } catch (IOException ie) {
1684       throw new ServiceException(ie);
1685     }
1686   }
1687 
1688   private Message execServiceOnRegion(HRegion region,
1689       final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException {
1690     // ignore the passed in controller (from the serialized call)
1691     ServerRpcController execController = new ServerRpcController();
1692     Message result = region.execService(execController, serviceCall);
1693     if (execController.getFailedOn() != null) {
1694       throw execController.getFailedOn();
1695     }
1696     return result;
1697   }
1698 
1699   /**
1700    * Get data from a table.
1701    *
1702    * @param controller the RPC controller
1703    * @param request the get request
1704    * @throws ServiceException
1705    */
1706   @Override
1707   public GetResponse get(final RpcController controller,
1708       final GetRequest request) throws ServiceException {
1709     long before = EnvironmentEdgeManager.currentTime();
1710     try {
1711       checkOpen();
1712       requestCount.increment();
1713       HRegion region = getRegion(request.getRegion());
1714 
1715       GetResponse.Builder builder = GetResponse.newBuilder();
1716       ClientProtos.Get get = request.getGet();
1717       Boolean existence = null;
1718       Result r = null;
1719 
1720       if (get.hasClosestRowBefore() && get.getClosestRowBefore()) {
1721         if (get.getColumnCount() != 1) {
1722           throw new DoNotRetryIOException(
1723             "get ClosestRowBefore supports one and only one family now, not "
1724               + get.getColumnCount() + " families");
1725         }
1726         byte[] row = get.getRow().toByteArray();
1727         byte[] family = get.getColumn(0).getFamily().toByteArray();
1728         r = region.getClosestRowBefore(row, family);
1729       } else {
1730         Get clientGet = ProtobufUtil.toGet(get);
1731         if (get.getExistenceOnly() && region.getCoprocessorHost() != null) {
1732           existence = region.getCoprocessorHost().preExists(clientGet);
1733         }
1734         if (existence == null) {
1735           r = region.get(clientGet);
1736           if (get.getExistenceOnly()) {
1737             boolean exists = r.getExists();
1738             if (region.getCoprocessorHost() != null) {
1739               exists = region.getCoprocessorHost().postExists(clientGet, exists);
1740             }
1741             existence = exists;
1742           }
1743         }
1744       }
1745       if (existence != null){
1746         ClientProtos.Result pbr =
1747             ProtobufUtil.toResult(existence, region.getRegionInfo().getReplicaId() != 0);
1748         builder.setResult(pbr);
1749       } else  if (r != null) {
1750         ClientProtos.Result pbr = ProtobufUtil.toResult(r);
1751         builder.setResult(pbr);
1752       }
1753       return builder.build();
1754     } catch (IOException ie) {
1755       throw new ServiceException(ie);
1756     } finally {
1757       if (regionServer.metricsRegionServer != null) {
1758         regionServer.metricsRegionServer.updateGet(
1759           EnvironmentEdgeManager.currentTime() - before);
1760       }
1761     }
1762   }
1763 
1764   /**
1765    * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
1766    *
1767    * @param rpcc the RPC controller
1768    * @param request the multi request
1769    * @throws ServiceException
1770    */
1771   @Override
1772   public MultiResponse multi(final RpcController rpcc, final MultiRequest request)
1773   throws ServiceException {
1774     try {
1775       checkOpen();
1776     } catch (IOException ie) {
1777       throw new ServiceException(ie);
1778     }
1779 
1780     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
1781     // It is also the conduit via which we pass back data.
1782     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
1783     CellScanner cellScanner = controller != null ? controller.cellScanner(): null;
1784     if (controller != null) controller.setCellScanner(null);
1785 
1786     long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
1787 
1788     // this will contain all the cells that we need to return. It's created later, if needed.
1789     List<CellScannable> cellsToReturn = null;
1790     MultiResponse.Builder responseBuilder = MultiResponse.newBuilder();
1791     RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.newBuilder();
1792     Boolean processed = null;
1793 
1794     for (RegionAction regionAction : request.getRegionActionList()) {
1795       this.requestCount.add(regionAction.getActionCount());
1796       HRegion region;
1797       regionActionResultBuilder.clear();
1798       try {
1799         region = getRegion(regionAction.getRegion());
1800       } catch (IOException e) {
1801         regionActionResultBuilder.setException(ResponseConverter.buildException(e));
1802         responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
1803         continue;  // For this region it's a failure.
1804       }
1805 
1806       if (regionAction.hasAtomic() && regionAction.getAtomic()) {
1807         // How does this call happen?  It may need some work to play well w/ the surroundings.
1808         // Need to return an item per Action along w/ Action index.  TODO.
1809         try {
1810           if (request.hasCondition()) {
1811             Condition condition = request.getCondition();
1812             byte[] row = condition.getRow().toByteArray();
1813             byte[] family = condition.getFamily().toByteArray();
1814             byte[] qualifier = condition.getQualifier().toByteArray();
1815             CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
1816             ByteArrayComparable comparator =
1817                 ProtobufUtil.toComparator(condition.getComparator());
1818             processed = checkAndRowMutate(region, regionAction.getActionList(),
1819                   cellScanner, row, family, qualifier, compareOp, comparator);
1820           } else {
1821             ClientProtos.RegionLoadStats stats = mutateRows(region, regionAction.getActionList(),
1822                 cellScanner);
1823             // add the stats to the request
1824             if(stats != null) {
1825               responseBuilder.addRegionActionResult(RegionActionResult.newBuilder()
1826                   .addResultOrException(ResultOrException.newBuilder().setLoadStats(stats)));
1827             }
1828             processed = Boolean.TRUE;
1829           }
1830         } catch (IOException e) {
1831           // As it's atomic, we may expect it's a global failure.
1832           regionActionResultBuilder.setException(ResponseConverter.buildException(e));
1833         }
1834       } else {
1835         // doNonAtomicRegionMutation manages the exception internally
1836         cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner,
1837             regionActionResultBuilder, cellsToReturn, nonceGroup);
1838       }
1839       responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
1840     }
1841     // Load the controller with the Cells to return.
1842     if (cellsToReturn != null && !cellsToReturn.isEmpty() && controller != null) {
1843       controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn));
1844     }
1845     if (processed != null) responseBuilder.setProcessed(processed);
1846     return responseBuilder.build();
1847   }
1848 
1849   /**
1850    * Mutate data in a table.
1851    *
1852    * @param rpcc the RPC controller
1853    * @param request the mutate request
1854    * @throws ServiceException
1855    */
1856   @Override
1857   public MutateResponse mutate(final RpcController rpcc,
1858       final MutateRequest request) throws ServiceException {
1859     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
1860     // It is also the conduit via which we pass back data.
1861     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
1862     CellScanner cellScanner = controller != null? controller.cellScanner(): null;
1863     // Clear scanner so we are not holding on to reference across call.
1864     if (controller != null) controller.setCellScanner(null);
1865     try {
1866       checkOpen();
1867       requestCount.increment();
1868       HRegion region = getRegion(request.getRegion());
1869       MutateResponse.Builder builder = MutateResponse.newBuilder();
1870       MutationProto mutation = request.getMutation();
1871       if (!region.getRegionInfo().isMetaTable()) {
1872         regionServer.cacheFlusher.reclaimMemStoreMemory();
1873       }
1874       long nonceGroup = request.hasNonceGroup()
1875           ? request.getNonceGroup() : HConstants.NO_NONCE;
1876       Result r = null;
1877       Boolean processed = null;
1878       MutationType type = mutation.getMutateType();
1879       switch (type) {
1880       case APPEND:
1881         // TODO: this doesn't actually check anything.
1882         r = append(region, mutation, cellScanner, nonceGroup);
1883         break;
1884       case INCREMENT:
1885         // TODO: this doesn't actually check anything.
1886         r = increment(region, mutation, cellScanner, nonceGroup);
1887         break;
1888       case PUT:
1889         Put put = ProtobufUtil.toPut(mutation, cellScanner);
1890         if (request.hasCondition()) {
1891           Condition condition = request.getCondition();
1892           byte[] row = condition.getRow().toByteArray();
1893           byte[] family = condition.getFamily().toByteArray();
1894           byte[] qualifier = condition.getQualifier().toByteArray();
1895           CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
1896           ByteArrayComparable comparator =
1897             ProtobufUtil.toComparator(condition.getComparator());
1898           if (region.getCoprocessorHost() != null) {
1899             processed = region.getCoprocessorHost().preCheckAndPut(
1900               row, family, qualifier, compareOp, comparator, put);
1901           }
1902           if (processed == null) {
1903             boolean result = region.checkAndMutate(row, family,
1904               qualifier, compareOp, comparator, put, true);
1905             if (region.getCoprocessorHost() != null) {
1906               result = region.getCoprocessorHost().postCheckAndPut(row, family,
1907                 qualifier, compareOp, comparator, put, result);
1908             }
1909             processed = result;
1910           }
1911         } else {
1912           region.put(put);
1913           processed = Boolean.TRUE;
1914         }
1915         break;
1916       case DELETE:
1917         Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
1918         if (request.hasCondition()) {
1919           Condition condition = request.getCondition();
1920           byte[] row = condition.getRow().toByteArray();
1921           byte[] family = condition.getFamily().toByteArray();
1922           byte[] qualifier = condition.getQualifier().toByteArray();
1923           CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
1924           ByteArrayComparable comparator =
1925             ProtobufUtil.toComparator(condition.getComparator());
1926           if (region.getCoprocessorHost() != null) {
1927             processed = region.getCoprocessorHost().preCheckAndDelete(
1928               row, family, qualifier, compareOp, comparator, delete);
1929           }
1930           if (processed == null) {
1931             boolean result = region.checkAndMutate(row, family,
1932               qualifier, compareOp, comparator, delete, true);
1933             if (region.getCoprocessorHost() != null) {
1934               result = region.getCoprocessorHost().postCheckAndDelete(row, family,
1935                 qualifier, compareOp, comparator, delete, result);
1936             }
1937             processed = result;
1938           }
1939         } else {
1940           region.delete(delete);
1941           processed = Boolean.TRUE;
1942         }
1943         break;
1944       default:
1945           throw new DoNotRetryIOException(
1946             "Unsupported mutate type: " + type.name());
1947       }
1948       if (processed != null) builder.setProcessed(processed.booleanValue());
1949       addResult(builder, r, controller);
1950       return builder.build();
1951     } catch (IOException ie) {
1952       regionServer.checkFileSystem();
1953       throw new ServiceException(ie);
1954     }
1955   }
1956 
1957   /**
1958    * Scan data in a table.
1959    *
1960    * @param controller the RPC controller
1961    * @param request the scan request
1962    * @throws ServiceException
1963    */
1964   @Override
1965   public ScanResponse scan(final RpcController controller, final ScanRequest request)
1966   throws ServiceException {
1967     Leases.Lease lease = null;
1968     String scannerName = null;
1969     try {
1970       if (!request.hasScannerId() && !request.hasScan()) {
1971         throw new DoNotRetryIOException(
1972           "Missing required input: scannerId or scan");
1973       }
1974       long scannerId = -1;
1975       if (request.hasScannerId()) {
1976         scannerId = request.getScannerId();
1977         scannerName = String.valueOf(scannerId);
1978       }
1979       try {
1980         checkOpen();
1981       } catch (IOException e) {
1982         // If checkOpen failed, server not running or filesystem gone,
1983         // cancel this lease; filesystem is gone or we're closing or something.
1984         if (scannerName != null) {
1985           LOG.debug("Server shutting down and client tried to access missing scanner "
1986             + scannerName);
1987           if (regionServer.leases != null) {
1988             try {
1989               regionServer.leases.cancelLease(scannerName);
1990             } catch (LeaseException le) {
1991               // No problem, ignore
1992             }
1993           }
1994         }
1995         throw e;
1996       }
1997       requestCount.increment();
1998 
1999       int ttl = 0;
2000       HRegion region = null;
2001       RegionScanner scanner = null;
2002       RegionScannerHolder rsh = null;
2003       boolean moreResults = true;
2004       boolean closeScanner = false;
2005       ScanResponse.Builder builder = ScanResponse.newBuilder();
2006       if (request.hasCloseScanner()) {
2007         closeScanner = request.getCloseScanner();
2008       }
2009       int rows = closeScanner ? 0 : 1;
2010       if (request.hasNumberOfRows()) {
2011         rows = request.getNumberOfRows();
2012       }
2013       if (request.hasScannerId()) {
2014         rsh = scanners.get(scannerName);
2015         if (rsh == null) {
2016           LOG.info("Client tried to access missing scanner " + scannerName);
2017           throw new UnknownScannerException(
2018             "Name: " + scannerName + ", already closed?");
2019         }
2020         scanner = rsh.s;
2021         HRegionInfo hri = scanner.getRegionInfo();
2022         region = regionServer.getRegion(hri.getRegionName());
2023         if (region != rsh.r) { // Yes, should be the same instance
2024           throw new NotServingRegionException("Region was re-opened after the scanner"
2025             + scannerName + " was created: " + hri.getRegionNameAsString());
2026         }
2027       } else {
2028         region = getRegion(request.getRegion());
2029         ClientProtos.Scan protoScan = request.getScan();
2030         boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
2031         Scan scan = ProtobufUtil.toScan(protoScan);
2032         // if the request doesn't set this, get the default region setting.
2033         if (!isLoadingCfsOnDemandSet) {
2034           scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());
2035         }
2036         region.prepareScanner(scan);
2037         if (region.getCoprocessorHost() != null) {
2038           scanner = region.getCoprocessorHost().preScannerOpen(scan);
2039         }
2040         if (scanner == null) {
2041           scanner = region.getScanner(scan);
2042         }
2043         if (region.getCoprocessorHost() != null) {
2044           scanner = region.getCoprocessorHost().postScannerOpen(scan, scanner);
2045         }
2046         scannerId = addScanner(scanner, region);
2047         scannerName = String.valueOf(scannerId);
2048         ttl = this.scannerLeaseTimeoutPeriod;
2049       }
2050 
2051       if (rows > 0) {
2052         // if nextCallSeq does not match throw Exception straight away. This needs to be
2053         // performed even before checking of Lease.
2054         // See HBASE-5974
2055         if (request.hasNextCallSeq()) {
2056           if (rsh == null) {
2057             rsh = scanners.get(scannerName);
2058           }
2059           if (rsh != null) {
2060             if (request.getNextCallSeq() != rsh.nextCallSeq) {
2061               throw new OutOfOrderScannerNextException("Expected nextCallSeq: " + rsh.nextCallSeq
2062                 + " But the nextCallSeq got from client: " + request.getNextCallSeq() +
2063                 "; request=" + TextFormat.shortDebugString(request));
2064             }
2065             // Increment the nextCallSeq value which is the next expected from client.
2066             rsh.nextCallSeq++;
2067           }
2068         }
2069         try {
2070           // Remove lease while its being processed in server; protects against case
2071           // where processing of request takes > lease expiration time.
2072           lease = regionServer.leases.removeLease(scannerName);
2073           // Limit the initial allocation of the result array to the minimum
2074           // of 'rows' or 100
2075           List<Result> results = new ArrayList<Result>(Math.min(rows, 100));
2076           long currentScanResultSize = 0;
2077           long totalCellSize = 0;
2078 
2079           boolean done = false;
2080           // Call coprocessor. Get region info from scanner.
2081           if (region != null && region.getCoprocessorHost() != null) {
2082             Boolean bypass = region.getCoprocessorHost().preScannerNext(
2083               scanner, results, rows);
2084             if (!results.isEmpty()) {
2085               for (Result r : results) {
2086                 for (Cell cell : r.rawCells()) {
2087                   currentScanResultSize += CellUtil.estimatedHeapSizeOfWithoutTags(cell);
2088                   totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
2089                 }
2090               }
2091             }
2092             if (bypass != null && bypass.booleanValue()) {
2093               done = true;
2094             }
2095           }
2096 
2097           if (!done) {
2098             long maxResultSize = scanner.getMaxResultSize();
2099             if (maxResultSize <= 0) {
2100               maxResultSize = maxScannerResultSize;
2101             }
2102             List<Cell> values = new ArrayList<Cell>();
2103             region.startRegionOperation(Operation.SCAN);
2104             try {
2105               int i = 0;
2106               synchronized(scanner) {
2107                 boolean stale = (region.getRegionInfo().getReplicaId() != 0);
2108                 boolean moreRows = false;
2109                 while (i < rows) {
2110                   // Stop collecting results if maxScannerResultSize is set and we have exceeded it
2111                   if ((maxResultSize < Long.MAX_VALUE) &&
2112                       (currentScanResultSize >= maxResultSize)) {
2113                     builder.setMoreResultsInRegion(true);
2114                     break;
2115                   }
2116                   // Collect values to be returned here
2117                   moreRows = scanner.nextRaw(values);
2118                   if (!values.isEmpty()) {
2119                     for (Cell cell : values) {
2120                       currentScanResultSize += CellUtil.estimatedHeapSizeOfWithoutTags(cell);
2121                       totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
2122                     }
2123                     results.add(Result.create(values, null, stale));
2124                     i++;
2125                   }
2126                   if (!moreRows) {
2127                     break;
2128                   }
2129                   values.clear();
2130                 }
2131                 // currentScanResultSize >= maxResultSize should be functionally equivalent to
2132                 // state.sizeLimitReached()
2133                 if (currentScanResultSize >= maxResultSize || i >= rows || moreRows) {
2134                   // We stopped prematurely
2135                   builder.setMoreResultsInRegion(true);
2136                 } else {
2137                   // We didn't get a single batch
2138                   builder.setMoreResultsInRegion(false);
2139                 }
2140               }
2141               region.readRequestsCount.add(i);
2142               region.getMetrics().updateScanNext(totalCellSize);
2143             } finally {
2144               region.closeRegionOperation();
2145             }
2146 
2147             // coprocessor postNext hook
2148             if (region != null && region.getCoprocessorHost() != null) {
2149               region.getCoprocessorHost().postScannerNext(scanner, results, rows, true);
2150             }
2151           }
2152 
2153           // If the scanner's filter - if any - is done with the scan
2154           // and wants to tell the client to stop the scan. This is done by passing
2155           // a null result, and setting moreResults to false.
2156           if (scanner.isFilterDone() && results.isEmpty()) {
2157             moreResults = false;
2158             results = null;
2159           } else {
2160             addResults(builder, results, controller, RegionReplicaUtil.isDefaultReplica(region.getRegionInfo()));
2161           }
2162         } finally { 
2163           // We're done. On way out re-add the above removed lease.
2164           // Adding resets expiration time on lease.
2165           if (scanners.containsKey(scannerName)) {
2166             if (lease != null) regionServer.leases.addLease(lease);
2167             ttl = this.scannerLeaseTimeoutPeriod;
2168           }
2169         }
2170       }
2171 
2172       if (!moreResults || closeScanner) {
2173         ttl = 0;
2174         moreResults = false;
2175         if (region != null && region.getCoprocessorHost() != null) {
2176           if (region.getCoprocessorHost().preScannerClose(scanner)) {
2177             return builder.build(); // bypass
2178           }
2179         }
2180         rsh = scanners.remove(scannerName);
2181         if (rsh != null) {
2182           scanner = rsh.s;
2183           scanner.close();
2184           regionServer.leases.cancelLease(scannerName);
2185           if (region != null && region.getCoprocessorHost() != null) {
2186             region.getCoprocessorHost().postScannerClose(scanner);
2187           }
2188         }
2189       }
2190 
2191       if (ttl > 0) {
2192         builder.setTtl(ttl);
2193       }
2194       builder.setScannerId(scannerId);
2195       builder.setMoreResults(moreResults);
2196       return builder.build();
2197     } catch (IOException ie) {
2198       if (scannerName != null && ie instanceof NotServingRegionException) {
2199         RegionScannerHolder rsh = scanners.remove(scannerName);
2200         if (rsh != null) {
2201           try {
2202             RegionScanner scanner = rsh.s;
2203             LOG.warn(scannerName + " encountered " + ie.getMessage() + ", closing ...");
2204             scanner.close();
2205             regionServer.leases.cancelLease(scannerName);
2206           } catch (IOException e) {
2207            LOG.warn("Getting exception closing " + scannerName, e);
2208           }
2209         }
2210       }
2211       throw new ServiceException(ie);
2212     }
2213   }
2214 
2215   @Override
2216   public CoprocessorServiceResponse execRegionServerService(RpcController controller,
2217       CoprocessorServiceRequest request) throws ServiceException {
2218     return regionServer.execRegionServerService(controller, request);
2219   }
2220 
2221   @Override
2222   public UpdateConfigurationResponse updateConfiguration(
2223       RpcController controller, UpdateConfigurationRequest request)
2224       throws ServiceException {
2225     try {
2226       this.regionServer.updateConfiguration();
2227     } catch (Exception e) {
2228       throw new ServiceException(e);
2229     }
2230     return UpdateConfigurationResponse.getDefaultInstance();
2231   }
2232 }