1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.IOException;
22
23 import org.apache.hadoop.hbase.HRegionInfo;
24 import org.apache.hadoop.hbase.TableName;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
27 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
28 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
29 import org.apache.hadoop.hbase.protobuf.RequestConverter;
30 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
31 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
32 import org.apache.hadoop.hbase.util.Bytes;
33 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
34 import org.mortbay.log.Log;
35
36 import com.google.protobuf.ServiceException;
37
38
39
40
41 @InterfaceAudience.Private
42 public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
43
44 private final byte[] regionName;
45 private final boolean writeFlushWalMarker;
46 private boolean reload;
47
48 public FlushRegionCallable(ClusterConnection connection,
49 RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] regionName,
50 byte[] regionStartKey, boolean writeFlushWalMarker) {
51 super(connection, rpcControllerFactory, tableName, regionStartKey);
52 this.regionName = regionName;
53 this.writeFlushWalMarker = writeFlushWalMarker;
54 }
55
56 public FlushRegionCallable(ClusterConnection connection,
57 RpcControllerFactory rpcControllerFactory, HRegionInfo regionInfo,
58 boolean writeFlushWalMarker) {
59 this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
60 regionInfo.getStartKey(), writeFlushWalMarker);
61 }
62
63 @Override
64 public FlushRegionResponse call(int callTimeout) throws Exception {
65 return flushRegion();
66 }
67
68 @Override
69 public void prepare(boolean reload) throws IOException {
70 super.prepare(reload);
71 this.reload = reload;
72 }
73
74 private FlushRegionResponse flushRegion() throws IOException {
75
76
77 if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) {
78 if (!reload) {
79 throw new IOException("Cached location seems to be different than requested region.");
80 }
81 Log.info("Skipping flush region, because the located region "
82 + Bytes.toStringBinary(location.getRegionInfo().getRegionName()) + " is different than "
83 + " requested region " + Bytes.toStringBinary(regionName));
84 return FlushRegionResponse.newBuilder()
85 .setLastFlushTime(EnvironmentEdgeManager.currentTime())
86 .setFlushed(false)
87 .setWroteFlushWalMarker(false)
88 .build();
89 }
90
91 FlushRegionRequest request =
92 RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker);
93
94 try {
95 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
96 controller.setPriority(tableName);
97 return stub.flushRegion(controller, request);
98 } catch (ServiceException se) {
99 throw ProtobufUtil.getRemoteException(se);
100 }
101 }
102 }