View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.protobuf;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.List;
23  
24  import edu.umd.cs.findbugs.annotations.Nullable;
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.Cell;
29  import org.apache.hadoop.hbase.CellScanner;
30  import org.apache.hadoop.hbase.DoNotRetryIOException;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.ServerName;
33  import org.apache.hadoop.hbase.client.Result;
34  import org.apache.hadoop.hbase.ipc.ServerRpcController;
35  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
36  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
37  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
38  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
39  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
40  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
41  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
42  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
43  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
44  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
45  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
46  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
47  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
48  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
49  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
50  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
51  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
52  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
53  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
54  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
55  import org.apache.hadoop.hbase.security.access.UserPermission;
56  import org.apache.hadoop.util.StringUtils;
57  
58  import com.google.protobuf.ByteString;
59  import com.google.protobuf.RpcController;
60  
61  /**
62   * Helper utility to build protocol buffer responses,
63   * or retrieve data from protocol buffer responses.
64   */
65  @InterfaceAudience.Private
66  public final class ResponseConverter {
67    public static final Log LOG = LogFactory.getLog(ResponseConverter.class);
68  
69    private ResponseConverter() {
70    }
71  
72  // Start utilities for Client
73  
74    /**
75     * Get the results from a protocol buffer MultiResponse
76     *
77     * @param request the protocol buffer MultiResponse to convert
78     * @param cells Cells to go with the passed in <code>proto</code>.  Can be null.
79     * @return the results that were in the MultiResponse (a Result or an Exception).
80     * @throws IOException
81     */
82    public static org.apache.hadoop.hbase.client.MultiResponse getResults(final MultiRequest request,
83        final MultiResponse response, final CellScanner cells)
84    throws IOException {
85      int requestRegionActionCount = request.getRegionActionCount();
86      int responseRegionActionResultCount = response.getRegionActionResultCount();
87      if (requestRegionActionCount != responseRegionActionResultCount) {
88        throw new IllegalStateException("Request mutation count=" + responseRegionActionResultCount +
89            " does not match response mutation result count=" + responseRegionActionResultCount);
90      }
91  
92      org.apache.hadoop.hbase.client.MultiResponse results =
93        new org.apache.hadoop.hbase.client.MultiResponse();
94  
95      for (int i = 0; i < responseRegionActionResultCount; i++) {
96        RegionAction actions = request.getRegionAction(i);
97        RegionActionResult actionResult = response.getRegionActionResult(i);
98        HBaseProtos.RegionSpecifier rs = actions.getRegion();
99        if (rs.hasType() &&
100           (rs.getType() != HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)){
101         throw new IllegalArgumentException(
102             "We support only encoded types for protobuf multi response.");
103       }
104       byte[] regionName = rs.getValue().toByteArray();
105 
106       if (actionResult.hasException()) {
107         Throwable regionException =  ProtobufUtil.toException(actionResult.getException());
108         results.addException(regionName, regionException);
109         continue;
110       }
111 
112       if (actions.getActionCount() != actionResult.getResultOrExceptionCount()) {
113         throw new IllegalStateException("actions.getActionCount=" + actions.getActionCount() +
114             ", actionResult.getResultOrExceptionCount=" +
115             actionResult.getResultOrExceptionCount() + " for region " + actions.getRegion());
116       }
117 
118       for (ResultOrException roe : actionResult.getResultOrExceptionList()) {
119         Object responseValue;
120         if (roe.hasException()) {
121           responseValue = ProtobufUtil.toException(roe.getException());
122         } else if (roe.hasResult()) {
123           responseValue = ProtobufUtil.toResult(roe.getResult(), cells);
124           // add the load stats, if we got any
125           if (roe.hasLoadStats()) {
126             ((Result) responseValue).addResults(roe.getLoadStats());
127           }
128         } else if (roe.hasServiceResult()) {
129           responseValue = roe.getServiceResult();
130         } else {
131           // no result & no exception. Unexpected.
132           throw new IllegalStateException("No result & no exception roe=" + roe +
133               " for region " + actions.getRegion());
134         }
135         results.add(regionName, roe.getIndex(), responseValue);
136       }
137     }
138 
139     return results;
140   }
141 
142   /**
143    * Wrap a throwable to an action result.
144    *
145    * @param t
146    * @return an action result builder
147    */
148   public static ResultOrException.Builder buildActionResult(final Throwable t) {
149     ResultOrException.Builder builder = ResultOrException.newBuilder();
150     if (t != null) builder.setException(buildException(t));
151     return builder;
152   }
153 
154   /**
155    * Wrap a throwable to an action result.
156    *
157    * @param r
158    * @return an action result builder
159    */
160   public static ResultOrException.Builder buildActionResult(final ClientProtos.Result r,
161       ClientProtos.RegionLoadStats stats) {
162     ResultOrException.Builder builder = ResultOrException.newBuilder();
163     if (r != null) builder.setResult(r);
164     if(stats != null) builder.setLoadStats(stats);
165     return builder;
166   }
167 
168   /**
169    * @param t
170    * @return NameValuePair of the exception name to stringified version os exception.
171    */
172   public static NameBytesPair buildException(final Throwable t) {
173     NameBytesPair.Builder parameterBuilder = NameBytesPair.newBuilder();
174     parameterBuilder.setName(t.getClass().getName());
175     parameterBuilder.setValue(
176       ByteString.copyFromUtf8(StringUtils.stringifyException(t)));
177     return parameterBuilder.build();
178   }
179 
180   /**
181    * Converts the permissions list into a protocol buffer GetUserPermissionsResponse
182    */
183   public static GetUserPermissionsResponse buildGetUserPermissionsResponse(
184       final List<UserPermission> permissions) {
185     GetUserPermissionsResponse.Builder builder = GetUserPermissionsResponse.newBuilder();
186     for (UserPermission perm : permissions) {
187       builder.addUserPermission(ProtobufUtil.toUserPermission(perm));
188     }
189     return builder.build();
190   }
191 
192 // End utilities for Client
193 // Start utilities for Admin
194 
195   /**
196    * Get the list of region info from a GetOnlineRegionResponse
197    *
198    * @param proto the GetOnlineRegionResponse
199    * @return the list of region info
200    */
201   public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
202     if (proto == null || proto.getRegionInfoCount() == 0) return null;
203     return ProtobufUtil.getRegionInfos(proto);
204   }
205 
206   /**
207    * Get the region opening state from a OpenRegionResponse
208    *
209    * @param proto the OpenRegionResponse
210    * @return the region opening state
211    */
212   public static RegionOpeningState getRegionOpeningState
213       (final OpenRegionResponse proto) {
214     if (proto == null || proto.getOpeningStateCount() != 1) return null;
215     return RegionOpeningState.valueOf(
216       proto.getOpeningState(0).name());
217   }
218 
219   /**
220    * Get a list of region opening state from a OpenRegionResponse
221    * 
222    * @param proto the OpenRegionResponse
223    * @return the list of region opening state
224    */
225   public static List<RegionOpeningState> getRegionOpeningStateList(
226       final OpenRegionResponse proto) {
227     if (proto == null) return null;
228     List<RegionOpeningState> regionOpeningStates = new ArrayList<RegionOpeningState>();
229     for (int i = 0; i < proto.getOpeningStateCount(); i++) {
230       regionOpeningStates.add(RegionOpeningState.valueOf(
231           proto.getOpeningState(i).name()));
232     }
233     return regionOpeningStates;
234   }
235 
236   /**
237    * Check if the region is closed from a CloseRegionResponse
238    *
239    * @param proto the CloseRegionResponse
240    * @return the region close state
241    */
242   public static boolean isClosed
243       (final CloseRegionResponse proto) {
244     if (proto == null || !proto.hasClosed()) return false;
245     return proto.getClosed();
246   }
247 
248   /**
249    * A utility to build a GetServerInfoResponse.
250    *
251    * @param serverName
252    * @param webuiPort
253    * @return the response
254    */
255   public static GetServerInfoResponse buildGetServerInfoResponse(
256       final ServerName serverName, final int webuiPort) {
257     GetServerInfoResponse.Builder builder = GetServerInfoResponse.newBuilder();
258     ServerInfo.Builder serverInfoBuilder = ServerInfo.newBuilder();
259     serverInfoBuilder.setServerName(ProtobufUtil.toServerName(serverName));
260     if (webuiPort >= 0) {
261       serverInfoBuilder.setWebuiPort(webuiPort);
262     }
263     builder.setServerInfo(serverInfoBuilder.build());
264     return builder.build();
265   }
266 
267   /**
268    * A utility to build a GetOnlineRegionResponse.
269    *
270    * @param regions
271    * @return the response
272    */
273   public static GetOnlineRegionResponse buildGetOnlineRegionResponse(
274       final List<HRegionInfo> regions) {
275     GetOnlineRegionResponse.Builder builder = GetOnlineRegionResponse.newBuilder();
276     for (HRegionInfo region: regions) {
277       builder.addRegionInfo(HRegionInfo.convert(region));
278     }
279     return builder.build();
280   }
281 
282   /**
283    * Creates a response for the catalog scan request
284    * @return A RunCatalogScanResponse
285    */
286   public static RunCatalogScanResponse buildRunCatalogScanResponse(int numCleaned) {
287     return RunCatalogScanResponse.newBuilder().setScanResult(numCleaned).build();
288   }
289 
290   /**
291    * Creates a response for the catalog scan request
292    * @return A EnableCatalogJanitorResponse
293    */
294   public static EnableCatalogJanitorResponse buildEnableCatalogJanitorResponse(boolean prevValue) {
295     return EnableCatalogJanitorResponse.newBuilder().setPrevValue(prevValue).build();
296   }
297 
298 // End utilities for Admin
299 
300   /**
301    * Creates a response for the last flushed sequence Id request
302    * @return A GetLastFlushedSequenceIdResponse
303    */
304   public static GetLastFlushedSequenceIdResponse buildGetLastFlushedSequenceIdResponse(
305       RegionStoreSequenceIds ids) {
306     return GetLastFlushedSequenceIdResponse.newBuilder()
307         .setLastFlushedSequenceId(ids.getLastFlushedSequenceId())
308         .addAllStoreLastFlushedSequenceId(ids.getStoreSequenceIdList()).build();
309   }
310 
311   /**
312    * Stores an exception encountered during RPC invocation so it can be passed back
313    * through to the client.
314    * @param controller the controller instance provided by the client when calling the service
315    * @param ioe the exception encountered
316    */
317   public static void setControllerException(RpcController controller, IOException ioe) {
318     if (controller != null) {
319       if (controller instanceof ServerRpcController) {
320         ((ServerRpcController)controller).setFailedOn(ioe);
321       } else {
322         controller.setFailed(StringUtils.stringifyException(ioe));
323       }
324     }
325   }
326 
327   /**
328    * Create Results from the cells using the cells meta data. 
329    * @param cellScanner
330    * @param response
331    * @return results
332    */
333   public static Result[] getResults(CellScanner cellScanner, ScanResponse response)
334       throws IOException {
335     if (response == null) return null;
336     // If cellscanner, then the number of Results to return is the count of elements in the
337     // cellsPerResult list.  Otherwise, it is how many results are embedded inside the response.
338     int noOfResults = cellScanner != null?
339       response.getCellsPerResultCount(): response.getResultsCount();
340     Result[] results = new Result[noOfResults];
341     for (int i = 0; i < noOfResults; i++) {
342       if (cellScanner != null) {
343         // Cells are out in cellblocks.  Group them up again as Results.  How many to read at a
344         // time will be found in getCellsLength -- length here is how many Cells in the i'th Result
345         int noOfCells = response.getCellsPerResult(i);
346         boolean isPartial =
347             response.getPartialFlagPerResultCount() > i ?
348                 response.getPartialFlagPerResult(i) : false;
349         List<Cell> cells = new ArrayList<Cell>(noOfCells);
350         for (int j = 0; j < noOfCells; j++) {
351           try {
352             if (cellScanner.advance() == false) {
353               // We are not able to retrieve the exact number of cells which ResultCellMeta says us.
354               // We have to scan for the same results again. Throwing DNRIOE as a client retry on the
355               // same scanner will result in OutOfOrderScannerNextException
356               String msg = "Results sent from server=" + noOfResults + ". But only got " + i
357                 + " results completely at client. Resetting the scanner to scan again.";
358               LOG.error(msg);
359               throw new DoNotRetryIOException(msg);
360             }
361           } catch (IOException ioe) {
362             // We are getting IOE while retrieving the cells for Results.
363             // We have to scan for the same results again. Throwing DNRIOE as a client retry on the
364             // same scanner will result in OutOfOrderScannerNextException
365             LOG.error("Exception while reading cells from result."
366               + "Resetting the scanner to scan again.", ioe);
367             throw new DoNotRetryIOException("Resetting the scanner.", ioe);
368           }
369           cells.add(cellScanner.current());
370         }
371         results[i] = Result.create(cells, null, response.getStale(), isPartial);
372       } else {
373         // Result is pure pb.
374         results[i] = ProtobufUtil.toResult(response.getResults(i));
375       }
376     }
377     return results;
378   }
379 }