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