diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java index 73bdb740ca1..1460c1bc4d5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.client; +import com.google.protobuf.ServiceException; + import java.io.IOException; import org.apache.commons.logging.Log; @@ -25,7 +27,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; @@ -34,8 +36,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRespons import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import com.google.protobuf.ServiceException; - /** * A Callable for flushRegion() RPC. */ @@ -95,7 +95,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable(getConnection()) { @Override public AbortProcedureResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); AbortProcedureRequest abortProcRequest = AbortProcedureRequest.newBuilder().setProcId(procId).build(); @@ -441,7 +438,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public HTableDescriptor[] call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); GetTableDescriptorsRequest req = RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables); @@ -522,7 +519,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public TableName[] call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); GetTableNamesRequest req = RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables); @@ -560,7 +557,7 @@ public class HBaseAdmin implements Admin { HTableDescriptor htd = executeCallable(new MasterCallable(connection) { @Override public HTableDescriptor call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); GetTableDescriptorsResponse htds; GetTableDescriptorsRequest req = @@ -758,7 +755,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public CreateTableResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(desc.getTableName()); CreateTableRequest request = RequestConverter.buildCreateTableRequest( @@ -932,7 +929,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public DeleteTableResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); DeleteTableRequest req = @@ -1183,7 +1180,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public EnableTableResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); @@ -1375,7 +1372,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public DisableTableResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); @@ -1592,7 +1589,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable>(getConnection()) { @Override public Pair call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); @@ -1664,7 +1661,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); AddColumnRequest req = RequestConverter.buildAddColumnRequest( @@ -1715,7 +1712,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest( @@ -1766,7 +1763,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest( @@ -1857,7 +1854,7 @@ public class HBaseAdmin implements Admin { CloseRegionRequest request = RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false); try { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); // TODO: this does not do retries, it should. Set priority and timeout in controller CloseRegionResponse response = admin.closeRegion(controller, request); @@ -1882,7 +1879,7 @@ public class HBaseAdmin implements Admin { public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException { AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); // Close the region without updating zk state. ProtobufUtil.closeRegion(controller, admin, sn, hri.getRegionName(), false); @@ -1894,7 +1891,7 @@ public class HBaseAdmin implements Admin { @Override public List getOnlineRegions(final ServerName sn) throws IOException { AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); return ProtobufUtil.getOnlineRegions(controller, admin); } @@ -1954,7 +1951,7 @@ public class HBaseAdmin implements Admin { private void flush(final ServerName sn, final HRegionInfo hri) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); AdminService.BlockingInterface admin = this.connection.getAdmin(sn); FlushRegionRequest request = RequestConverter.buildFlushRegionRequest(hri.getRegionName()); @@ -2211,7 +2208,7 @@ public class HBaseAdmin implements Admin { private void compact(final ServerName sn, final HRegionInfo hri, final boolean major, final byte [] family) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); AdminService.BlockingInterface admin = this.connection.getAdmin(sn); CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family); @@ -2243,7 +2240,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); // Hard to know the table name, at least check if meta if (isMetaRegion(encodedRegionName)) { @@ -2282,7 +2279,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); // Hard to know the table name, at least check if meta if (isMetaRegion(regionName)) { @@ -2318,7 +2315,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); // Hard to know the table name, at least check if meta if (isMetaRegion(regionName)) { @@ -2350,7 +2347,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); // Hard to know the table name, at least check if meta if (isMetaRegion(regionName)) { @@ -2374,7 +2371,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); SetBalancerRunningRequest req = @@ -2395,7 +2392,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.balance(controller, @@ -2409,7 +2406,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.balance(controller, @@ -2429,7 +2426,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isBalancerEnabled(controller, @@ -2448,7 +2445,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.normalize(controller, @@ -2466,7 +2463,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isNormalizerEnabled(controller, @@ -2484,7 +2481,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); SetNormalizerRunningRequest req = @@ -2506,7 +2503,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.enableCatalogJanitor(controller, @@ -2525,7 +2522,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Integer call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.runCatalogScan(controller, @@ -2543,7 +2540,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isCatalogJanitorEnabled(controller, @@ -2624,7 +2621,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); try { @@ -2763,7 +2760,7 @@ public class HBaseAdmin implements Admin { Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) { throw new IOException("should not give a splitkey which equals to startkey!"); } - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(hri.getTable()); // TODO: this does not do retries, it should. Set priority and timeout in controller @@ -2791,7 +2788,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(tableName); ModifyTableRequest request = RequestConverter.buildModifyTableRequest( @@ -2909,7 +2906,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(HConstants.HIGH_QOS); master.shutdown(controller, ShutdownRequest.newBuilder().build()); @@ -2929,7 +2926,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); controller.setPriority(HConstants.HIGH_QOS); master.stopMaster(controller, StopMasterRequest.newBuilder().build()); @@ -2953,7 +2950,7 @@ public class HBaseAdmin implements Admin { this.connection.getAdmin(ServerName.valueOf(hostname, port, 0)); StopServerRequest request = RequestConverter.buildStopServerRequest( "Called by admin client " + this.connection.toString()); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(HConstants.HIGH_QOS); try { @@ -2974,7 +2971,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public IsInMaintenanceModeResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isMasterInMaintenanceMode( controller, IsInMaintenanceModeRequest.newBuilder().build()); @@ -2987,7 +2984,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public ClusterStatus call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(); return ClusterStatus.convert(master.getClusterStatus(controller, req).getClusterStatus()); @@ -3013,7 +3010,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); // TODO: set priority based on NS? master.createNamespace(controller, @@ -3036,7 +3033,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder(). setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build()); @@ -3055,7 +3052,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder(). setNamespaceName(name).build()); @@ -3077,7 +3074,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public NamespaceDescriptor call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return ProtobufUtil.toNamespaceDescriptor( master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder(). @@ -3097,7 +3094,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public NamespaceDescriptor[] call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); List list = master.listNamespaceDescriptors(controller, @@ -3123,7 +3120,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public ProcedureInfo[] call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); List procList = master.listProcedures( controller, ListProceduresRequest.newBuilder().build()).getProcedureList(); @@ -3148,7 +3145,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public HTableDescriptor[] call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); List list = master.listTableDescriptorsByNamespace(controller, @@ -3176,7 +3173,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public TableName[] call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); List tableNames = master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest. @@ -3277,7 +3274,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public HTableDescriptor[] call(int callTimeout) throws Exception { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); GetTableDescriptorsRequest req = RequestConverter.buildGetTableDescriptorsRequest(tableNames); @@ -3327,7 +3324,7 @@ public class HBaseAdmin implements Admin { FailedLogCloseException { AdminService.BlockingInterface admin = this.connection.getAdmin(sn); RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest(); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); try { // TODO: this does not do retries, it should. Set priority and timeout in controller @@ -3476,7 +3473,7 @@ public class HBaseAdmin implements Admin { AdminService.BlockingInterface admin = this.connection.getAdmin(sn); GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest( regionServerPair.getFirst().getRegionName(), true); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); // TODO: this does not do retries, it should. Set priority and timeout in controller GetRegionInfoResponse response = admin.getRegionInfo(controller, request); return response.getCompactionState(); @@ -3686,7 +3683,7 @@ public class HBaseAdmin implements Admin { done = executeCallable(new MasterCallable(getConnection()) { @Override public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isSnapshotDone(controller, request); } @@ -3718,7 +3715,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public SnapshotResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.snapshot(controller, request); } @@ -3752,7 +3749,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isSnapshotDone(controller, IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build()); @@ -4009,7 +4006,7 @@ public class HBaseAdmin implements Admin { getConnection()) { @Override public ExecProcedureResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.execProcedureWithRet(controller, request); } @@ -4045,7 +4042,7 @@ public class HBaseAdmin implements Admin { getConnection()) { @Override public ExecProcedureResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.execProcedure(controller, request); } @@ -4111,7 +4108,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public IsProcedureDoneResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isProcedureDone(controller, IsProcedureDoneRequest .newBuilder().setProcedure(desc).build()); @@ -4159,7 +4156,7 @@ public class HBaseAdmin implements Admin { getConnection()) { @Override public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.isRestoreSnapshotDone(controller, request); } @@ -4191,7 +4188,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public RestoreSnapshotResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.restoreSnapshot(controller, request); } @@ -4208,7 +4205,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable>(getConnection()) { @Override public List call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); return master.getCompletedSnapshots(controller, GetCompletedSnapshotsRequest.newBuilder().build()).getSnapshotsList(); @@ -4309,7 +4306,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder(). @@ -4353,7 +4350,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder() .setSnapshot(snapshot).build()); @@ -4406,7 +4403,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota)); return null; @@ -4550,7 +4547,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Long call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); MajorCompactionTimestampRequest req = MajorCompactionTimestampRequest.newBuilder() @@ -4565,7 +4562,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable(getConnection()) { @Override public Long call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); MajorCompactionTimestampForRegionRequest req = MajorCompactionTimestampForRegionRequest @@ -4633,7 +4630,7 @@ public class HBaseAdmin implements Admin { admin.getConnection()) { @Override public AbortProcedureResponse call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newController(); + HBaseRpcController controller = admin.getRpcControllerFactory().newController(); controller.setCallTimeout(callTimeout); return master.abortProcedure(controller, request); } @@ -4847,7 +4844,7 @@ public class HBaseAdmin implements Admin { return executeCallable(new MasterCallable>(getConnection()) { @Override public List call(int callTimeout) throws ServiceException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setCallTimeout(callTimeout); SecurityCapabilitiesRequest req = SecurityCapabilitiesRequest.newBuilder().build(); return ProtobufUtil.toSecurityCapabilityList( diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 527dc7242fd..d4fa2e32ea2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -755,7 +755,7 @@ public class HTable implements HTableInterface, RegionLocator { tableName, row) { @Override public Result call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); ClientProtos.GetRequest request = RequestConverter.buildGetRowOrBeforeRequest( @@ -851,7 +851,7 @@ public class HTable implements HTableInterface, RegionLocator { public Result call(int callTimeout) throws IOException { ClientProtos.GetRequest request = RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); try { @@ -976,7 +976,7 @@ public class HTable implements HTableInterface, RegionLocator { tableName, delete.getRow()) { @Override public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); @@ -1053,6 +1053,7 @@ public class HTable implements HTableInterface, RegionLocator { rpcControllerFactory) { @Override public MultiResponse call(int callTimeout) throws IOException { + controller.reset(); controller.setPriority(tableName); int remainingTime = tracker.getRemainingTime(callTimeout); if (remainingTime == 0) { @@ -1105,7 +1106,7 @@ public class HTable implements HTableInterface, RegionLocator { new RegionServerCallable(this.connection, getName(), append.getRow()) { @Override public Result call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(getTableName()); controller.setCallTimeout(callTimeout); try { @@ -1138,7 +1139,7 @@ public class HTable implements HTableInterface, RegionLocator { getName(), increment.getRow()) { @Override public Result call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(getTableName()); controller.setCallTimeout(callTimeout); try { @@ -1206,7 +1207,7 @@ public class HTable implements HTableInterface, RegionLocator { new RegionServerCallable(connection, getName(), row) { @Override public Long call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(getTableName()); controller.setCallTimeout(callTimeout); try { @@ -1238,7 +1239,7 @@ public class HTable implements HTableInterface, RegionLocator { new RegionServerCallable(connection, getName(), row) { @Override public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); try { @@ -1268,7 +1269,7 @@ public class HTable implements HTableInterface, RegionLocator { new RegionServerCallable(connection, getName(), row) { @Override public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = new PayloadCarryingRpcController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); try { @@ -1299,7 +1300,7 @@ public class HTable implements HTableInterface, RegionLocator { new RegionServerCallable(connection, getName(), row) { @Override public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); try { @@ -1329,7 +1330,7 @@ public class HTable implements HTableInterface, RegionLocator { new RegionServerCallable(connection, getName(), row) { @Override public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + HBaseRpcController controller = rpcControllerFactory.newController(); controller.setPriority(tableName); controller.setCallTimeout(callTimeout); try { @@ -1361,6 +1362,7 @@ public class HTable implements HTableInterface, RegionLocator { rpcControllerFactory) { @Override public MultiResponse call(int callTimeout) throws IOException { + controller.reset(); controller.setPriority(tableName); int remainingTime = tracker.getRemainingTime(callTimeout); if (remainingTime == 0) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java index 115ba33567b..738ff6e1428 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java @@ -127,6 +127,7 @@ class MultiServerCallable extends PayloadCarryingServerCallable extends RegionServerCallable implements Cancellable { - protected PayloadCarryingRpcController controller; + protected HBaseRpcController controller; public PayloadCarryingServerCallable(Connection connection, TableName tableName, byte[] row, RpcControllerFactory rpcControllerFactory) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java index d1c40abb6a4..66304570090 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java @@ -21,26 +21,8 @@ package org.apache.hadoop.hbase.client; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseIOException; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; - import com.google.protobuf.ServiceException; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.Collections; @@ -50,7 +32,23 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; /** * Caller that goes to replica if the primary region does no answer within a configurable @@ -98,7 +96,7 @@ public class RpcRetryingCallerWithReadReplicas { */ class ReplicaRegionServerCallable extends RegionServerCallable implements Cancellable { final int id; - private final PayloadCarryingRpcController controller; + private final HBaseRpcController controller; public ReplicaRegionServerCallable(int id, HRegionLocation location) { super(RpcRetryingCallerWithReadReplicas.this.cConnection, @@ -155,6 +153,7 @@ public class RpcRetryingCallerWithReadReplicas { ClientProtos.GetRequest request = RequestConverter.buildGetRequest(reg, get); + controller.reset(); controller.setCallTimeout(callTimeout); try { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java index fd884e33f95..ebac3618050 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java @@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.exceptions.ScannerResetException; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; @@ -104,7 +104,7 @@ public class ScannerCallable extends RegionServerCallable { protected boolean isRegionServerRemote = true; private long nextCallSeq = 0; protected RpcControllerFactory controllerFactory; - protected PayloadCarryingRpcController controller; + protected HBaseRpcController controller; /** * @param connection which connection @@ -141,7 +141,7 @@ public class ScannerCallable extends RegionServerCallable { this.controller = rpcControllerFactory.newController(); } - PayloadCarryingRpcController getController() { + HBaseRpcController getController() { return controller; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index 3740b7f9ff1..634c101e549 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -18,62 +18,108 @@ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE; +import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException; + import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.protobuf.BlockingRpcChannel; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcChannel; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; +import io.netty.util.HashedWheelTimer; + import java.io.IOException; -import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.SocketAddress; -import java.net.SocketTimeoutException; import java.net.UnknownHostException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.codec.KeyValueCodec; -import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; +import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.PoolMap; +import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.security.token.TokenSelector; /** * Provides the basics for a RpcClient implementation like configuration and Logging. + *

+ * Locking schema of the current IPC implementation + *

    + *
  • There is a lock in {@link AbstractRpcClient} to protect the fetching or creating + * connection.
  • + *
  • There is a lock in {@link Call} to make sure that we can only finish the call once.
  • + *
  • The same for {@link HBaseRpcController} as {@link Call}. And see the comment of + * {@link HBaseRpcController#notifyOnCancel(RpcCallback, HBaseRpcController.CancellationCallback)} + * of how to deal with cancel.
  • + *
  • For connection implementation, the construction of a connection should be as fast as possible + * because the creation is protected under a lock. Connect to remote side when needed. There is no + * forced locking schema for a connection implementation.
  • + *
  • For the locking order, the {@link Call} and {@link HBaseRpcController}'s lock should be held + * at last. So the callbacks in {@link Call} and {@link HBaseRpcController} should be execute + * outside the lock in {@link Call} and {@link HBaseRpcController} which means the implementations + * of the callbacks are free to hold any lock.
  • + *
*/ @InterfaceAudience.Private -public abstract class AbstractRpcClient implements RpcClient { +public abstract class AbstractRpcClient implements RpcClient { // Log level is being changed in tests public static final Log LOG = LogFactory.getLog(AbstractRpcClient.class); + protected static final HashedWheelTimer WHEEL_TIMER = new HashedWheelTimer( + Threads.newDaemonThreadFactory("RpcClient-timer"), 10, TimeUnit.MILLISECONDS); + + private static final ScheduledExecutorService IDLE_CONN_SWEEPER = Executors + .newScheduledThreadPool(1, Threads.newDaemonThreadFactory("Idle-Rpc-Conn-Sweeper")); + + protected final static Map> TOKEN_HANDLERS = new HashMap<>(); + + static { + TOKEN_HANDLERS.put(Kind.HBASE_AUTH_TOKEN, new AuthenticationTokenSelector()); + } + + protected boolean running = true; // if client runs + protected final Configuration conf; - protected String clusterId; + protected final String clusterId; protected final SocketAddress localAddr; protected final MetricsConnection metrics; - protected UserProvider userProvider; - protected final IPCUtil ipcUtil; + protected final UserProvider userProvider; + protected final CellBlockBuilder cellBlockBuilder; protected final int minIdleTimeBeforeClose; // if the connection is idle for more than this // time (in ms), it will be closed at any moment. - protected final int maxRetries; //the max. no. of retries for socket connections + protected final int maxRetries; // the max. no. of retries for socket connections protected final long failureSleep; // Time to sleep before retry on failure. protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm protected final boolean tcpKeepAlive; // if T then use keepalives @@ -81,10 +127,18 @@ public abstract class AbstractRpcClient implements RpcClient { protected final CompressionCodec compressor; protected final boolean fallbackAllowed; + protected final FailedServers failedServers; + protected final int connectTO; protected final int readTO; protected final int writeTO; + protected final PoolMap connections; + + private final AtomicInteger callIdCnt = new AtomicInteger(0); + + private final ScheduledFuture cleanupIdleConnectionTask; + private int maxConcurrentCallsPerServer; private static final LoadingCache concurrentCounterCache = @@ -97,7 +151,6 @@ public abstract class AbstractRpcClient implements RpcClient { /** * Construct an IPC client for the cluster clusterId - * * @param conf configuration * @param clusterId the cluster id * @param localAddr client socket bind address. @@ -110,17 +163,18 @@ public abstract class AbstractRpcClient implements RpcClient { this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true); this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT; this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); + HConstants.DEFAULT_HBASE_CLIENT_PAUSE); this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0); this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true); - this.ipcUtil = new IPCUtil(conf); + this.cellBlockBuilder = new CellBlockBuilder(conf); this.minIdleTimeBeforeClose = conf.getInt(IDLE_TIME, 120000); // 2 minutes this.conf = conf; this.codec = getCodec(); this.compressor = getCompressor(conf); this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY, - IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT); + IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT); + this.failedServers = new FailedServers(conf); this.connectTO = conf.getInt(SOCKET_TIMEOUT_CONNECT, DEFAULT_SOCKET_TIMEOUT_CONNECT); this.readTO = conf.getInt(SOCKET_TIMEOUT_READ, DEFAULT_SOCKET_TIMEOUT_READ); this.writeTO = conf.getInt(SOCKET_TIMEOUT_WRITE, DEFAULT_SOCKET_TIMEOUT_WRITE); @@ -129,25 +183,47 @@ public abstract class AbstractRpcClient implements RpcClient { HConstants.HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD, HConstants.DEFAULT_HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD); - // login the server principal (if using secure Hadoop) + this.connections = new PoolMap<>(getPoolType(conf), getPoolSize(conf)); + + this.cleanupIdleConnectionTask = IDLE_CONN_SWEEPER.scheduleAtFixedRate(new Runnable() { + + @Override + public void run() { + cleanupIdleConnections(); + } + }, minIdleTimeBeforeClose, minIdleTimeBeforeClose, TimeUnit.MILLISECONDS); + if (LOG.isDebugEnabled()) { - LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor + - ", tcpKeepAlive=" + this.tcpKeepAlive + - ", tcpNoDelay=" + this.tcpNoDelay + - ", connectTO=" + this.connectTO + - ", readTO=" + this.readTO + - ", writeTO=" + this.writeTO + - ", minIdleTimeBeforeClose=" + this.minIdleTimeBeforeClose + - ", maxRetries=" + this.maxRetries + - ", fallbackAllowed=" + this.fallbackAllowed + - ", bind address=" + (this.localAddr != null ? this.localAddr : "null")); + LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor + ", tcpKeepAlive=" + + this.tcpKeepAlive + ", tcpNoDelay=" + this.tcpNoDelay + ", connectTO=" + this.connectTO + + ", readTO=" + this.readTO + ", writeTO=" + this.writeTO + ", minIdleTimeBeforeClose=" + + this.minIdleTimeBeforeClose + ", maxRetries=" + this.maxRetries + ", fallbackAllowed=" + + this.fallbackAllowed + ", bind address=" + + (this.localAddr != null ? this.localAddr : "null")); + } + } + + private void cleanupIdleConnections() { + long closeBeforeTime = EnvironmentEdgeManager.currentTime() - minIdleTimeBeforeClose; + synchronized (connections) { + for (T conn : connections.values()) { + // remove connection if it has not been chosen by anyone for more than maxIdleTime, and the + // connection itself has already shutdown. The latter check is because that we may still + // have some pending calls on connection so we should not shutdown the connection outside. + // The connection itself will disconnect if there is no pending call for maxIdleTime. + if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) { + LOG.info("Cleanup idle connection to " + conn.remoteId().address); + connections.removeValue(conn.remoteId(), conn); + conn.cleanupConnection(); + } + } } } @VisibleForTesting public static String getDefaultCodec(final Configuration c) { // If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because - // Configuration will complain -- then no default codec (and we'll pb everything). Else + // Configuration will complain -- then no default codec (and we'll pb everything). Else // default is KeyValueCodec return c.get(DEFAULT_CODEC_CLASS, KeyValueCodec.class.getCanonicalName()); } @@ -160,9 +236,11 @@ public abstract class AbstractRpcClient implements RpcClient { // For NO CODEC, "hbase.client.rpc.codec" must be configured with empty string AND // "hbase.client.default.rpc.codec" also -- because default is to do cell block encoding. String className = conf.get(HConstants.RPC_CODEC_CONF_KEY, getDefaultCodec(this.conf)); - if (className == null || className.length() == 0) return null; + if (className == null || className.length() == 0) { + return null; + } try { - return (Codec)Class.forName(className).newInstance(); + return (Codec) Class.forName(className).newInstance(); } catch (Exception e) { throw new RuntimeException("Failed getting codec " + className, e); } @@ -173,6 +251,12 @@ public abstract class AbstractRpcClient implements RpcClient { return this.codec != null; } + // for writing tests that want to throw exception when connecting. + @VisibleForTesting + boolean isTcpNoDelay() { + return tcpNoDelay; + } + /** * Encapsulate the ugly casting and RuntimeException conversion in private method. * @param conf configuration @@ -180,142 +264,289 @@ public abstract class AbstractRpcClient implements RpcClient { */ private static CompressionCodec getCompressor(final Configuration conf) { String className = conf.get("hbase.client.rpc.compressor", null); - if (className == null || className.isEmpty()) return null; + if (className == null || className.isEmpty()) { + return null; + } try { - return (CompressionCodec)Class.forName(className).newInstance(); + return (CompressionCodec) Class.forName(className).newInstance(); } catch (Exception e) { throw new RuntimeException("Failed getting compressor " + className, e); } } /** - * Return the pool type specified in the configuration, which must be set to - * either {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or - * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal}, - * otherwise default to the former. - * - * For applications with many user threads, use a small round-robin pool. For - * applications with few user threads, you may want to try using a - * thread-local pool. In any case, the number of {@link org.apache.hadoop.hbase.ipc.RpcClient} - * instances should not exceed the operating system's hard limit on the number of - * connections. - * + * Return the pool type specified in the configuration, which must be set to either + * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or + * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal}, otherwise default to the + * former. For applications with many user threads, use a small round-robin pool. For applications + * with few user threads, you may want to try using a thread-local pool. In any case, the number + * of {@link org.apache.hadoop.hbase.ipc.RpcClient} instances should not exceed the operating + * system's hard limit on the number of connections. * @param config configuration * @return either a {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal} */ - protected static PoolMap.PoolType getPoolType(Configuration config) { - return PoolMap.PoolType - .valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), PoolMap.PoolType.RoundRobin, - PoolMap.PoolType.ThreadLocal); + private static PoolMap.PoolType getPoolType(Configuration config) { + return PoolMap.PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), + PoolMap.PoolType.RoundRobin, PoolMap.PoolType.ThreadLocal); } /** - * Return the pool size specified in the configuration, which is applicable only if - * the pool type is {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin}. - * + * Return the pool size specified in the configuration, which is applicable only if the pool type + * is {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin}. * @param config configuration * @return the maximum pool size */ - protected static int getPoolSize(Configuration config) { + private static int getPoolSize(Configuration config) { return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1); } + private int nextCallId() { + int id, next; + do { + id = callIdCnt.get(); + next = id < Integer.MAX_VALUE ? id + 1 : 0; + } while (!callIdCnt.compareAndSet(id, next)); + return id; + } + /** * Make a blocking call. Throws exceptions if there are network problems or if the remote code * threw an exception. - * * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. - * {@link UserProvider#getCurrent()} makes a new instance of User each time so - * will be a - * new Connection each time. + * {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a + * new Connection each time. * @return A pair with the Message response and the Cell data (if any). */ - Message callBlockingMethod(Descriptors.MethodDescriptor md, PayloadCarryingRpcController pcrc, + private Message callBlockingMethod(Descriptors.MethodDescriptor md, HBaseRpcController hrc, Message param, Message returnType, final User ticket, final InetSocketAddress isa) throws ServiceException { - if (pcrc == null) { - pcrc = new PayloadCarryingRpcController(); + BlockingRpcCallback done = new BlockingRpcCallback<>(); + callMethod(md, hrc, param, returnType, ticket, isa, done); + Message val; + try { + val = done.get(); + } catch (IOException e) { + throw new ServiceException(e); } + if (hrc.failed()) { + throw new ServiceException(hrc.getFailed()); + } else { + return val; + } + } - Pair val; - AtomicInteger counter = concurrentCounterCache.getUnchecked(isa); + /** + * Get a connection from the pool, or create a new one and add it to the pool. Connections to a + * given host/port are reused. + */ + private T getConnection(ConnectionId remoteId) throws IOException { + if (failedServers.isFailedServer(remoteId.getAddress())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Not trying to connect to " + remoteId.address + + " this server is in the failed servers list"); + } + throw new FailedServerException( + "This server is in the failed servers list: " + remoteId.address); + } + T conn; + synchronized (connections) { + if (!running) { + throw new StoppedRpcClientException(); + } + conn = connections.get(remoteId); + if (conn == null) { + conn = createConnection(remoteId); + connections.put(remoteId, conn); + } + conn.setLastTouched(EnvironmentEdgeManager.currentTime()); + } + return conn; + } + + /** + * Not connected. + */ + protected abstract T createConnection(ConnectionId remoteId) throws IOException; + + private void onCallFinished(Call call, HBaseRpcController hrc, InetSocketAddress addr, + RpcCallback callback) { + call.callStats.setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.getStartTime()); + if (metrics != null) { + metrics.updateRpc(call.md, call.param, call.callStats); + } + if (LOG.isTraceEnabled()) { + LOG.trace( + "Call: " + call.md.getName() + ", callTime: " + call.callStats.getCallTimeMs() + "ms"); + } + if (call.error != null) { + if (call.error instanceof RemoteException) { + call.error.fillInStackTrace(); + hrc.setFailed(call.error); + } else { + hrc.setFailed(wrapException(addr, call.error)); + } + callback.run(null); + } else { + hrc.setDone(call.cells); + callback.run(call.response); + } + } + + private void callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc, + final Message param, Message returnType, final User ticket, final InetSocketAddress addr, + final RpcCallback callback) { + final MetricsConnection.CallStats cs = MetricsConnection.newCallStats(); + cs.setStartTime(EnvironmentEdgeManager.currentTime()); + final AtomicInteger counter = concurrentCounterCache.getUnchecked(addr); + Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType, + hrc.getCallTimeout(), hrc.getPriority(), new RpcCallback() { + @Override + public void run(Call call) { + counter.decrementAndGet(); + onCallFinished(call, hrc, addr, callback); + } + }, cs); + ConnectionId remoteId = new ConnectionId(ticket, md.getService().getName(), addr); int count = counter.incrementAndGet(); try { if (count > maxConcurrentCallsPerServer) { - throw new ServerTooBusyException(isa, count); + throw new ServerTooBusyException(addr, count); } - final MetricsConnection.CallStats cs = MetricsConnection.newCallStats(); - cs.setStartTime(EnvironmentEdgeManager.currentTime()); - val = call(pcrc, md, param, returnType, ticket, isa, cs); - // Shove the results into controller so can be carried across the proxy/pb service void. - pcrc.setCellScanner(val.getSecond()); + T connection = getConnection(remoteId); + connection.sendRequest(call, hrc); + } catch (Exception e) { + call.setException(toIOE(e)); + } + } - cs.setCallTimeMs(EnvironmentEdgeManager.currentTime() - cs.getStartTime()); - if (metrics != null) { - metrics.updateRpc(md, param, cs); + private InetSocketAddress createAddr(ServerName sn) throws UnknownHostException { + InetSocketAddress addr = new InetSocketAddress(sn.getHostname(), sn.getPort()); + if (addr.isUnresolved()) { + throw new UnknownHostException("can not resolve " + sn.getServerName()); + } + return addr; + } + + /** + * Interrupt the connections to the given ip:port server. This should be called if the server is + * known as actually dead. This will not prevent current operation to be retried, and, depending + * on their own behavior, they may retry on the same server. This can be a feature, for example at + * startup. In any case, they're likely to get connection refused (if the process died) or no + * route to host: i.e. their next retries should be faster and with a safe exception. + */ + @Override + public void cancelConnections(ServerName sn) { + synchronized (connections) { + for (T connection : connections.values()) { + ConnectionId remoteId = connection.remoteId(); + if (remoteId.address.getPort() == sn.getPort() && + remoteId.address.getHostName().equals(sn.getHostname())) { + LOG.info("The server on " + sn.toString() + " is dead - stopping the connection " + + connection.remoteId); + connections.removeValue(remoteId, connection); + connection.shutdown(); + } } - if (LOG.isTraceEnabled()) { - LOG.trace("Call: " + md.getName() + ", callTime: " + cs.getCallTimeMs() + "ms"); - } - return val.getFirst(); - } catch (Throwable e) { - throw new ServiceException(e); - } finally { - counter.decrementAndGet(); } } /** - * Make a call, passing param, to the IPC server running at - * address which is servicing the protocol protocol, - * with the ticket credentials, returning the value. - * Throws exceptions if there are network problems or if the remote code - * threw an exception. - * - * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. - * {@link UserProvider#getCurrent()} makes a new instance of User each time so - * will be a - * new Connection each time. - * @return A pair with the Message response and the Cell data (if any). - * @throws InterruptedException - * @throws java.io.IOException + * Configure an hbase rpccontroller + * @param controller to configure + * @param channelOperationTimeout timeout for operation + * @return configured controller */ - protected abstract Pair call(PayloadCarryingRpcController pcrc, - Descriptors.MethodDescriptor md, Message param, Message returnType, User ticket, - InetSocketAddress isa, MetricsConnection.CallStats callStats) - throws IOException, InterruptedException; + static HBaseRpcController configureHBaseRpcController( + RpcController controller, int channelOperationTimeout) { + HBaseRpcController hrc; + if (controller != null && controller instanceof HBaseRpcController) { + hrc = (HBaseRpcController) controller; + if (!hrc.hasCallTimeout()) { + hrc.setCallTimeout(channelOperationTimeout); + } + } else { + hrc = new HBaseRpcControllerImpl(); + hrc.setCallTimeout(channelOperationTimeout); + } + return hrc; + } + + protected abstract void closeInternal(); + + @Override + public void close() { + if (LOG.isDebugEnabled()) { + LOG.debug("Stopping rpc client"); + } + Collection connToClose; + synchronized (connections) { + if (!running) { + return; + } + running = false; + connToClose = connections.values(); + connections.clear(); + } + cleanupIdleConnectionTask.cancel(true); + for (T conn : connToClose) { + conn.shutdown(); + } + closeInternal(); + for (T conn : connToClose) { + conn.cleanupConnection(); + } + } @Override public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn, final User ticket, - int defaultOperationTimeout) throws UnknownHostException { - return new BlockingRpcChannelImplementation(this, sn, ticket, defaultOperationTimeout); + int rpcTimeout) throws UnknownHostException { + return new BlockingRpcChannelImplementation(this, createAddr(sn), ticket, rpcTimeout); } - /** - * Takes an Exception and the address we were trying to connect to and return an IOException with - * the input exception as the cause. The new exception provides the stack trace of the place where - * the exception is thrown and some extra diagnostics information. If the exception is - * ConnectException or SocketTimeoutException, return a new one of the same type; Otherwise return - * an IOException. - * @param addr target address - * @param exception the relevant exception - * @return an exception to throw - */ - protected IOException wrapException(InetSocketAddress addr, Exception exception) { - if (exception instanceof ConnectException) { - // connection refused; include the host:port in the error - return (ConnectException) new ConnectException("Call to " + addr - + " failed on connection exception: " + exception).initCause(exception); - } else if (exception instanceof SocketTimeoutException) { - return (SocketTimeoutException) new SocketTimeoutException("Call to " + addr - + " failed because " + exception).initCause(exception); - } else if (exception instanceof ConnectionClosingException) { - return (ConnectionClosingException) new ConnectionClosingException("Call to " + addr - + " failed on local exception: " + exception).initCause(exception); - } else { - return (IOException) new IOException("Call to " + addr + " failed on local exception: " - + exception).initCause(exception); + @Override + public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout) + throws UnknownHostException { + return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout); + } + + private static class AbstractRpcChannel { + + protected final InetSocketAddress addr; + + protected final AbstractRpcClient rpcClient; + + protected final User ticket; + + protected final int rpcTimeout; + + protected AbstractRpcChannel(AbstractRpcClient rpcClient, InetSocketAddress addr, + User ticket, int rpcTimeout) { + this.addr = addr; + this.rpcClient = rpcClient; + this.ticket = ticket; + this.rpcTimeout = rpcTimeout; + } + + /** + * Configure an rpc controller + * @param controller to configure + * @return configured rpc controller + */ + protected HBaseRpcController configureRpcController(RpcController controller) { + HBaseRpcController hrc; + // TODO: Ideally we should not use an RpcController other than HBaseRpcController at client + // side. And now we may use ServerRpcController. + if (controller != null && controller instanceof HBaseRpcController) { + hrc = (HBaseRpcController) controller; + if (!hrc.hasCallTimeout()) { + hrc.setCallTimeout(rpcTimeout); + } + } else { + hrc = new HBaseRpcControllerImpl(); + hrc.setCallTimeout(rpcTimeout); + } + return hrc; } } @@ -323,42 +554,42 @@ public abstract class AbstractRpcClient implements RpcClient { * Blocking rpc channel that goes via hbase rpc. */ @VisibleForTesting - public static class BlockingRpcChannelImplementation implements BlockingRpcChannel { - private final InetSocketAddress isa; - private final AbstractRpcClient rpcClient; - private final User ticket; - private final int channelOperationTimeout; + public static class BlockingRpcChannelImplementation extends AbstractRpcChannel + implements BlockingRpcChannel { - /** - * @param channelOperationTimeout - the default timeout when no timeout is given - */ - protected BlockingRpcChannelImplementation(final AbstractRpcClient rpcClient, - final ServerName sn, final User ticket, int channelOperationTimeout) - throws UnknownHostException { - this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort()); - if (this.isa.isUnresolved()) { - throw new UnknownHostException(sn.getHostname()); - } - this.rpcClient = rpcClient; - this.ticket = ticket; - this.channelOperationTimeout = channelOperationTimeout; + protected BlockingRpcChannelImplementation(AbstractRpcClient rpcClient, + InetSocketAddress addr, User ticket, int rpcTimeout) { + super(rpcClient, addr, ticket, rpcTimeout); } @Override public Message callBlockingMethod(Descriptors.MethodDescriptor md, RpcController controller, Message param, Message returnType) throws ServiceException { - PayloadCarryingRpcController pcrc; - if (controller != null && controller instanceof PayloadCarryingRpcController) { - pcrc = (PayloadCarryingRpcController) controller; - if (!pcrc.hasCallTimeout()) { - pcrc.setCallTimeout(channelOperationTimeout); - } - } else { - pcrc = new PayloadCarryingRpcController(); - pcrc.setCallTimeout(channelOperationTimeout); - } - - return this.rpcClient.callBlockingMethod(md, pcrc, param, returnType, this.ticket, this.isa); + return rpcClient.callBlockingMethod(md, configureRpcController(controller), + param, returnType, ticket, addr); } } -} + + /** + * Async rpc channel that goes via hbase rpc. + */ + public static class RpcChannelImplementation extends AbstractRpcChannel implements + RpcChannel { + + protected RpcChannelImplementation(AbstractRpcClient rpcClient, InetSocketAddress addr, + User ticket, int rpcTimeout) throws UnknownHostException { + super(rpcClient, addr, ticket, rpcTimeout); + } + + @Override + public void callMethod(Descriptors.MethodDescriptor md, RpcController controller, + Message param, Message returnType, RpcCallback done) { + // This method does not throw any exceptions, so the caller must provide a + // HBaseRpcController which is used to pass the exceptions. + this.rpcClient.callMethod(md, + configureRpcController(Preconditions.checkNotNull(controller, + "RpcController can not be null for async rpc call")), + param, returnType, ticket, addr, done); + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java deleted file mode 100644 index a5da0dcdeaa..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java +++ /dev/null @@ -1,141 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import io.netty.channel.EventLoop; -import io.netty.util.concurrent.DefaultPromise; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ExceptionUtil; -import org.apache.hadoop.ipc.RemoteException; - -import java.io.IOException; - -/** - * Represents an Async Hbase call and its response. - * - * Responses are passed on to its given doneHandler and failures to the rpcController - */ -@InterfaceAudience.Private -public class AsyncCall extends DefaultPromise { - private static final Log LOG = LogFactory.getLog(AsyncCall.class.getName()); - - final int id; - - final Descriptors.MethodDescriptor method; - final Message param; - final PayloadCarryingRpcController controller; - final Message responseDefaultType; - final long startTime; - final long rpcTimeout; - final MetricsConnection.CallStats callStats; - - /** - * Constructor - * - * @param eventLoop for call - * @param connectId connection id - * @param md the method descriptor - * @param param parameters to send to Server - * @param controller controller for response - * @param responseDefaultType the default response type - */ - public AsyncCall(EventLoop eventLoop, int connectId, Descriptors.MethodDescriptor md, Message - param, PayloadCarryingRpcController controller, Message responseDefaultType, - MetricsConnection.CallStats callStats) { - super(eventLoop); - - this.id = connectId; - - this.method = md; - this.param = param; - this.controller = controller; - this.responseDefaultType = responseDefaultType; - - this.startTime = EnvironmentEdgeManager.currentTime(); - this.rpcTimeout = controller.hasCallTimeout() ? controller.getCallTimeout() : 0; - this.callStats = callStats; - } - - /** - * Get the start time - * - * @return start time for the call - */ - public long getStartTime() { - return this.startTime; - } - - @Override - public String toString() { - return "callId=" + this.id + ", method=" + this.method.getName() + - ", rpcTimeout=" + this.rpcTimeout + ", param {" + - (this.param != null ? ProtobufUtil.getShortTextFormat(this.param) : "") + "}"; - } - - /** - * Set success with a cellBlockScanner - * - * @param value to set - * @param cellBlockScanner to set - */ - public void setSuccess(Message value, CellScanner cellBlockScanner) { - if (cellBlockScanner != null) { - controller.setCellScanner(cellBlockScanner); - } - - if (LOG.isTraceEnabled()) { - long callTime = EnvironmentEdgeManager.currentTime() - startTime; - LOG.trace("Call: " + method.getName() + ", callTime: " + callTime + "ms"); - } - - this.setSuccess(value); - } - - /** - * Set failed - * - * @param exception to set - */ - public void setFailed(IOException exception) { - if (ExceptionUtil.isInterrupt(exception)) { - exception = ExceptionUtil.asInterrupt(exception); - } - if (exception instanceof RemoteException) { - exception = ((RemoteException) exception).unwrapRemoteException(); - } - - this.setFailure(exception); - } - - /** - * Get the rpc timeout - * - * @return current timeout for this call - */ - public long getRpcTimeout() { - return rpcTimeout; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java deleted file mode 100644 index 878d8b87c1a..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java +++ /dev/null @@ -1,785 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import java.io.IOException; -import java.net.ConnectException; -import java.net.InetSocketAddress; -import java.net.SocketException; -import java.nio.ByteBuffer; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.TimeUnit; - -import javax.security.sasl.SaslException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; -import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; -import org.apache.hadoop.hbase.protobuf.generated.TracingProtos; -import org.apache.hadoop.hbase.security.AuthMethod; -import org.apache.hadoop.hbase.security.SaslClientHandler; -import org.apache.hadoop.hbase.security.SaslUtil; -import org.apache.hadoop.hbase.security.SecurityInfo; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.security.token.TokenSelector; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; - -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcCallback; - -import io.netty.bootstrap.Bootstrap; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufOutputStream; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import io.netty.handler.codec.LengthFieldBasedFrameDecoder; -import io.netty.util.Timeout; -import io.netty.util.TimerTask; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; - -/** - * Netty RPC channel - */ -@InterfaceAudience.Private -public class AsyncRpcChannel { - private static final Log LOG = LogFactory.getLog(AsyncRpcChannel.class.getName()); - - private static final int MAX_SASL_RETRIES = 5; - - protected final static Map> TOKEN_HANDDLERS - = new HashMap<>(); - - static { - TOKEN_HANDDLERS.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN, - new AuthenticationTokenSelector()); - } - - final AsyncRpcClient client; - - // Contains the channel to work with. - // Only exists when connected - private Channel channel; - - String name; - final User ticket; - final String serviceName; - final InetSocketAddress address; - - private int ioFailureCounter = 0; - private int connectFailureCounter = 0; - - boolean useSasl; - AuthMethod authMethod; - private int reloginMaxBackoff; - private Token token; - private String serverPrincipal; - - // NOTE: closed and connected flags below are only changed when a lock on pendingCalls - private final Map pendingCalls = new HashMap(); - private boolean connected = false; - private boolean closed = false; - - private Timeout cleanupTimer; - - private final TimerTask timeoutTask = new TimerTask() { - @Override - public void run(Timeout timeout) throws Exception { - cleanupCalls(); - } - }; - - /** - * Constructor for netty RPC channel - * @param bootstrap to construct channel on - * @param client to connect with - * @param ticket of user which uses connection - * @param serviceName name of service to connect to - * @param address to connect to - */ - public AsyncRpcChannel(Bootstrap bootstrap, final AsyncRpcClient client, User ticket, - String serviceName, InetSocketAddress address) { - this.client = client; - - this.ticket = ticket; - this.serviceName = serviceName; - this.address = address; - - this.channel = connect(bootstrap).channel(); - - name = ("IPC Client (" + channel.hashCode() + ") to " + address.toString() - + ((ticket == null) ? " from unknown user" : (" from " + ticket.getName()))); - } - - /** - * Connect to channel - * @param bootstrap to connect to - * @return future of connection - */ - private ChannelFuture connect(final Bootstrap bootstrap) { - return bootstrap.remoteAddress(address).connect() - .addListener(new GenericFutureListener() { - @Override - public void operationComplete(final ChannelFuture f) throws Exception { - if (!f.isSuccess()) { - if (f.cause() instanceof SocketException) { - retryOrClose(bootstrap, connectFailureCounter++, f.cause()); - } else { - retryOrClose(bootstrap, ioFailureCounter++, f.cause()); - } - return; - } - channel = f.channel(); - - setupAuthorization(); - - ByteBuf b = channel.alloc().directBuffer(6); - createPreamble(b, authMethod); - channel.writeAndFlush(b).addListener(ChannelFutureListener.CLOSE_ON_FAILURE); - if (useSasl) { - UserGroupInformation ticket = AsyncRpcChannel.this.ticket.getUGI(); - if (authMethod == AuthMethod.KERBEROS) { - if (ticket != null && ticket.getRealUser() != null) { - ticket = ticket.getRealUser(); - } - } - SaslClientHandler saslHandler; - if (ticket == null) { - throw new FatalConnectionException("ticket/user is null"); - } - final UserGroupInformation realTicket = ticket; - saslHandler = ticket.doAs(new PrivilegedExceptionAction() { - @Override - public SaslClientHandler run() throws IOException { - return getSaslHandler(realTicket, bootstrap); - } - }); - if (saslHandler != null) { - // Sasl connect is successful. Let's set up Sasl channel handler - channel.pipeline().addFirst(saslHandler); - } else { - // fall back to simple auth because server told us so. - authMethod = AuthMethod.SIMPLE; - useSasl = false; - } - } else { - startHBaseConnection(f.channel()); - } - } - }); - } - - /** - * Start HBase connection - * @param ch channel to start connection on - */ - private void startHBaseConnection(Channel ch) { - ch.pipeline().addLast("frameDecoder", - new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)); - ch.pipeline().addLast(new AsyncServerResponseHandler(this)); - try { - writeChannelHeader(ch).addListener(new GenericFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (!future.isSuccess()) { - close(future.cause()); - return; - } - List callsToWrite; - synchronized (pendingCalls) { - connected = true; - callsToWrite = new ArrayList(pendingCalls.values()); - } - for (AsyncCall call : callsToWrite) { - writeRequest(call); - } - } - }); - } catch (IOException e) { - close(e); - } - } - - /** - * Start HBase connection with sasl encryption - * @param ch channel to start connection on - */ - private void startConnectionWithEncryption(Channel ch) { - // for rpc encryption, the order of ChannelInboundHandler should be: - // LengthFieldBasedFrameDecoder->SaslClientHandler->LengthFieldBasedFrameDecoder - // Don't skip the first 4 bytes for length in beforeUnwrapDecoder, - // SaslClientHandler will handler this - ch.pipeline().addFirst("beforeUnwrapDecoder", - new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 0)); - ch.pipeline().addLast("afterUnwrapDecoder", - new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)); - ch.pipeline().addLast(new AsyncServerResponseHandler(this)); - List callsToWrite; - synchronized (pendingCalls) { - connected = true; - callsToWrite = new ArrayList(pendingCalls.values()); - } - for (AsyncCall call : callsToWrite) { - writeRequest(call); - } - } - - /** - * Get SASL handler - * @param bootstrap to reconnect to - * @return new SASL handler - * @throws java.io.IOException if handler failed to create - */ - private SaslClientHandler getSaslHandler(final UserGroupInformation realTicket, - final Bootstrap bootstrap) throws IOException { - return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal, - client.fallbackAllowed, - client.conf.get("hbase.rpc.protection", - SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)), - getChannelHeaderBytes(authMethod), - new SaslClientHandler.SaslExceptionHandler() { - @Override - public void handle(int retryCount, Random random, Throwable cause) { - try { - // Handle Sasl failure. Try to potentially get new credentials - handleSaslConnectionFailure(retryCount, cause, realTicket); - - // Try to reconnect - client.newTimeout(new TimerTask() { - @Override - public void run(Timeout timeout) throws Exception { - connect(bootstrap); - } - }, random.nextInt(reloginMaxBackoff) + 1, TimeUnit.MILLISECONDS); - } catch (IOException | InterruptedException e) { - close(e); - } - } - }, new SaslClientHandler.SaslSuccessfulConnectHandler() { - @Override - public void onSuccess(Channel channel) { - startHBaseConnection(channel); - } - - @Override - public void onSaslProtectionSucess(Channel channel) { - startConnectionWithEncryption(channel); - } - }); - } - - /** - * Retry to connect or close - * - * @param bootstrap to connect with - * @param connectCounter amount of tries - * @param e exception of fail - */ - private void retryOrClose(final Bootstrap bootstrap, int connectCounter, Throwable e) { - if (connectCounter < client.maxRetries) { - client.newTimeout(new TimerTask() { - @Override public void run(Timeout timeout) throws Exception { - connect(bootstrap); - } - }, client.failureSleep, TimeUnit.MILLISECONDS); - } else { - client.failedServers.addToFailedServers(address); - close(e); - } - } - - /** - * Calls method on channel - * @param method to call - * @param controller to run call with - * @param request to send - * @param responsePrototype to construct response with - */ - public Promise callMethod(final Descriptors.MethodDescriptor method, - final PayloadCarryingRpcController controller, final Message request, - final Message responsePrototype, MetricsConnection.CallStats callStats) { - final AsyncCall call = new AsyncCall(channel.eventLoop(), client.callIdCnt.getAndIncrement(), - method, request, controller, responsePrototype, callStats); - controller.notifyOnCancel(new RpcCallback() { - @Override - public void run(Object parameter) { - // TODO: do not need to call AsyncCall.setFailed? - synchronized (pendingCalls) { - pendingCalls.remove(call.id); - } - } - }); - // TODO: this should be handled by PayloadCarryingRpcController. - if (controller.isCanceled()) { - // To finish if the call was cancelled before we set the notification (race condition) - call.cancel(true); - return call; - } - - synchronized (pendingCalls) { - if (closed) { - Promise promise = channel.eventLoop().newPromise(); - promise.setFailure(new ConnectException()); - return promise; - } - pendingCalls.put(call.id, call); - // Add timeout for cleanup if none is present - if (cleanupTimer == null && call.getRpcTimeout() > 0) { - cleanupTimer = client.newTimeout(timeoutTask, call.getRpcTimeout(), TimeUnit.MILLISECONDS); - } - if (!connected) { - return call; - } - } - writeRequest(call); - return call; - } - - AsyncCall removePendingCall(int id) { - synchronized (pendingCalls) { - return pendingCalls.remove(id); - } - } - - /** - * Write the channel header - * @param channel to write to - * @return future of write - * @throws java.io.IOException on failure to write - */ - private ChannelFuture writeChannelHeader(Channel channel) throws IOException { - RPCProtos.ConnectionHeader header = getChannelHeader(authMethod); - int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header); - - ByteBuf b = channel.alloc().directBuffer(totalSize); - - b.writeInt(header.getSerializedSize()); - b.writeBytes(header.toByteArray()); - - return channel.writeAndFlush(b); - } - - private byte[] getChannelHeaderBytes(AuthMethod authMethod) { - RPCProtos.ConnectionHeader header = getChannelHeader(authMethod); - ByteBuffer b = ByteBuffer.allocate(header.getSerializedSize() + 4); - b.putInt(header.getSerializedSize()); - b.put(header.toByteArray()); - return b.array(); - } - - private RPCProtos.ConnectionHeader getChannelHeader(AuthMethod authMethod) { - RPCProtos.ConnectionHeader.Builder headerBuilder = RPCProtos.ConnectionHeader.newBuilder() - .setServiceName(serviceName); - - RPCProtos.UserInformation userInfoPB = buildUserInfo(ticket.getUGI(), authMethod); - if (userInfoPB != null) { - headerBuilder.setUserInfo(userInfoPB); - } - - if (client.codec != null) { - headerBuilder.setCellBlockCodecClass(client.codec.getClass().getCanonicalName()); - } - if (client.compressor != null) { - headerBuilder.setCellBlockCompressorClass(client.compressor.getClass().getCanonicalName()); - } - - headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo()); - return headerBuilder.build(); - } - - /** - * Write request to channel - * @param call to write - */ - private void writeRequest(final AsyncCall call) { - try { - final RPCProtos.RequestHeader.Builder requestHeaderBuilder = RPCProtos.RequestHeader - .newBuilder(); - requestHeaderBuilder.setCallId(call.id).setMethodName(call.method.getName()) - .setRequestParam(call.param != null); - - if (Trace.isTracing()) { - Span s = Trace.currentSpan(); - requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder() - .setParentId(s.getSpanId()).setTraceId(s.getTraceId())); - } - - ByteBuffer cellBlock = client.buildCellBlock(call.controller.cellScanner()); - if (cellBlock != null) { - final RPCProtos.CellBlockMeta.Builder cellBlockBuilder = RPCProtos.CellBlockMeta - .newBuilder(); - cellBlockBuilder.setLength(cellBlock.limit()); - requestHeaderBuilder.setCellBlockMeta(cellBlockBuilder.build()); - } - // Only pass priority if there one. Let zero be same as no priority. - if (call.controller.getPriority() != PayloadCarryingRpcController.PRIORITY_UNSET) { - requestHeaderBuilder.setPriority(call.controller.getPriority()); - } - requestHeaderBuilder.setTimeout(call.rpcTimeout > Integer.MAX_VALUE ? - Integer.MAX_VALUE : (int)call.rpcTimeout); - RPCProtos.RequestHeader rh = requestHeaderBuilder.build(); - - int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(rh, call.param); - if (cellBlock != null) { - totalSize += cellBlock.remaining(); - } - - ByteBuf b = channel.alloc().directBuffer(4 + totalSize); - try (ByteBufOutputStream out = new ByteBufOutputStream(b)) { - call.callStats.setRequestSizeBytes(IPCUtil.write(out, rh, call.param, cellBlock)); - } - - channel.writeAndFlush(b).addListener(new CallWriteListener(this, call.id)); - } catch (IOException e) { - close(e); - } - } - - /** - * Set up server authorization - * @throws java.io.IOException if auth setup failed - */ - private void setupAuthorization() throws IOException { - SecurityInfo securityInfo = SecurityInfo.getInfo(serviceName); - this.useSasl = client.userProvider.isHBaseSecurityEnabled(); - - this.token = null; - if (useSasl && securityInfo != null) { - AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind(); - if (tokenKind != null) { - TokenSelector tokenSelector = TOKEN_HANDDLERS.get(tokenKind); - if (tokenSelector != null) { - token = tokenSelector.selectToken(new Text(client.clusterId), - ticket.getUGI().getTokens()); - } else if (LOG.isDebugEnabled()) { - LOG.debug("No token selector found for type " + tokenKind); - } - } - String serverKey = securityInfo.getServerPrincipal(); - if (serverKey == null) { - throw new IOException("Can't obtain server Kerberos config key from SecurityInfo"); - } - this.serverPrincipal = SecurityUtil.getServerPrincipal(client.conf.get(serverKey), - address.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT)); - if (LOG.isDebugEnabled()) { - LOG.debug("RPC Server Kerberos principal name for service=" + serviceName + " is " - + serverPrincipal); - } - } - - if (!useSasl) { - authMethod = AuthMethod.SIMPLE; - } else if (token != null) { - authMethod = AuthMethod.DIGEST; - } else { - authMethod = AuthMethod.KERBEROS; - } - - if (LOG.isDebugEnabled()) { - LOG.debug( - "Use " + authMethod + " authentication for service " + serviceName + ", sasl=" + useSasl); - } - reloginMaxBackoff = client.conf.getInt("hbase.security.relogin.maxbackoff", 5000); - } - - /** - * Build the user information - * @param ugi User Group Information - * @param authMethod Authorization method - * @return UserInformation protobuf - */ - private RPCProtos.UserInformation buildUserInfo(UserGroupInformation ugi, AuthMethod authMethod) { - if (ugi == null || authMethod == AuthMethod.DIGEST) { - // Don't send user for token auth - return null; - } - RPCProtos.UserInformation.Builder userInfoPB = RPCProtos.UserInformation.newBuilder(); - if (authMethod == AuthMethod.KERBEROS) { - // Send effective user for Kerberos auth - userInfoPB.setEffectiveUser(ugi.getUserName()); - } else if (authMethod == AuthMethod.SIMPLE) { - // Send both effective user and real user for simple auth - userInfoPB.setEffectiveUser(ugi.getUserName()); - if (ugi.getRealUser() != null) { - userInfoPB.setRealUser(ugi.getRealUser().getUserName()); - } - } - return userInfoPB.build(); - } - - /** - * Create connection preamble - * @param byteBuf to write to - * @param authMethod to write - */ - private void createPreamble(ByteBuf byteBuf, AuthMethod authMethod) { - byteBuf.writeBytes(HConstants.RPC_HEADER); - byteBuf.writeByte(HConstants.RPC_CURRENT_VERSION); - byteBuf.writeByte(authMethod.code); - } - - private void close0(Throwable e) { - List toCleanup; - synchronized (pendingCalls) { - if (closed) { - return; - } - closed = true; - toCleanup = new ArrayList(pendingCalls.values()); - pendingCalls.clear(); - } - IOException closeException = null; - if (e != null) { - if (e instanceof IOException) { - closeException = (IOException) e; - } else { - closeException = new IOException(e); - } - } - // log the info - if (LOG.isDebugEnabled() && closeException != null) { - LOG.debug(name + ": closing ipc connection to " + address, closeException); - } - if (cleanupTimer != null) { - cleanupTimer.cancel(); - cleanupTimer = null; - } - for (AsyncCall call : toCleanup) { - call.setFailed(closeException != null ? closeException - : new ConnectionClosingException( - "Call id=" + call.id + " on server " + address + " aborted: connection is closing")); - } - channel.disconnect().addListener(ChannelFutureListener.CLOSE); - if (LOG.isDebugEnabled()) { - LOG.debug(name + ": closed"); - } - } - - /** - * Close connection - * @param e exception on close - */ - public void close(final Throwable e) { - client.removeConnection(this); - - // Move closing from the requesting thread to the channel thread - if (channel.eventLoop().inEventLoop()) { - close0(e); - } else { - channel.eventLoop().execute(new Runnable() { - @Override - public void run() { - close0(e); - } - }); - } - } - - /** - * Clean up calls. - */ - private void cleanupCalls() { - List toCleanup = new ArrayList(); - long currentTime = EnvironmentEdgeManager.currentTime(); - long nextCleanupTaskDelay = -1L; - synchronized (pendingCalls) { - for (Iterator iter = pendingCalls.values().iterator(); iter.hasNext();) { - AsyncCall call = iter.next(); - long timeout = call.getRpcTimeout(); - if (timeout > 0) { - if (currentTime - call.getStartTime() >= timeout) { - iter.remove(); - toCleanup.add(call); - } else { - if (nextCleanupTaskDelay < 0 || timeout < nextCleanupTaskDelay) { - nextCleanupTaskDelay = timeout; - } - } - } - } - if (nextCleanupTaskDelay > 0) { - cleanupTimer = client.newTimeout(timeoutTask, nextCleanupTaskDelay, TimeUnit.MILLISECONDS); - } else { - cleanupTimer = null; - } - } - for (AsyncCall call : toCleanup) { - call.setFailed(new CallTimeoutException("Call id=" + call.id + ", waitTime=" - + (currentTime - call.getStartTime()) + ", rpcTimeout=" + call.getRpcTimeout())); - } - } - - /** - * Check if the connection is alive - * @return true if alive - */ - public boolean isAlive() { - return channel.isOpen(); - } - - /** - * Check if user should authenticate over Kerberos - * @return true if should be authenticated over Kerberos - * @throws java.io.IOException on failure of check - */ - private synchronized boolean shouldAuthenticateOverKrb() throws IOException { - UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); - UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); - UserGroupInformation realUser = currentUser.getRealUser(); - return authMethod == AuthMethod.KERBEROS && loginUser != null && - // Make sure user logged in using Kerberos either keytab or TGT - loginUser.hasKerberosCredentials() && - // relogin only in case it is the login user (e.g. JT) - // or superuser (like oozie). - (loginUser.equals(currentUser) || loginUser.equals(realUser)); - } - - /** - * If multiple clients with the same principal try to connect to the same server at the same time, - * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to - * work around this, what is done is that the client backs off randomly and tries to initiate the - * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is - * attempted. - *

- * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the - * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such - * cases, it is prudent to throw a runtime exception when we receive a SaslException from the - * underlying authentication implementation, so there is no retry from other high level (for eg, - * HCM or HBaseAdmin). - *

- * @param currRetries retry count - * @param ex exception describing fail - * @param user which is trying to connect - * @throws java.io.IOException if IO fail - * @throws InterruptedException if thread is interrupted - */ - private void handleSaslConnectionFailure(final int currRetries, final Throwable ex, - final UserGroupInformation user) throws IOException, InterruptedException { - user.doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws IOException, InterruptedException { - if (shouldAuthenticateOverKrb()) { - if (currRetries < MAX_SASL_RETRIES) { - LOG.debug("Exception encountered while connecting to the server : " + ex); - // try re-login - if (UserGroupInformation.isLoginKeytabBased()) { - UserGroupInformation.getLoginUser().reloginFromKeytab(); - } else { - UserGroupInformation.getLoginUser().reloginFromTicketCache(); - } - - // Should reconnect - return null; - } else { - String msg = "Couldn't setup connection for " - + UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal; - LOG.warn(msg); - throw (IOException) new IOException(msg).initCause(ex); - } - } else { - LOG.warn("Exception encountered while connecting to " + "the server : " + ex); - } - if (ex instanceof RemoteException) { - throw (RemoteException) ex; - } - if (ex instanceof SaslException) { - String msg = "SASL authentication failed." - + " The most likely cause is missing or invalid credentials." + " Consider 'kinit'."; - LOG.fatal(msg, ex); - throw new RuntimeException(msg, ex); - } - throw new IOException(ex); - } - }); - } - - public int getConnectionHashCode() { - return ConnectionId.hashCode(ticket, serviceName, address); - } - - @Override - public int hashCode() { - return getConnectionHashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof AsyncRpcChannel) { - AsyncRpcChannel channel = (AsyncRpcChannel) obj; - return channel.hashCode() == obj.hashCode(); - } - return false; - } - - @Override - public String toString() { - return this.address.toString() + "/" + this.serviceName + "/" + this.ticket; - } - - /** - * Listens to call writes and fails if write failed - */ - private static final class CallWriteListener implements ChannelFutureListener { - private final AsyncRpcChannel rpcChannel; - private final int id; - - public CallWriteListener(AsyncRpcChannel asyncRpcChannel, int id) { - this.rpcChannel = asyncRpcChannel; - this.id = id; - } - - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (!future.isSuccess()) { - AsyncCall call = rpcChannel.removePendingCall(id); - if (call != null) { - if (future.cause() instanceof IOException) { - call.setFailed((IOException) future.cause()); - } else { - call.setFailed(new IOException(future.cause())); - } - } - } - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java deleted file mode 100644 index e12e298f682..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java +++ /dev/null @@ -1,499 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelInitializer; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.epoll.EpollSocketChannel; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; -import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.util.HashedWheelTimer; -import io.netty.util.Timeout; -import io.netty.util.TimerTask; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.nio.ByteBuffer; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.JVM; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.PoolMap; -import org.apache.hadoop.hbase.util.Threads; - -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcChannel; -import com.google.protobuf.RpcController; - -/** - * Netty client for the requests and responses - */ -@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) -public class AsyncRpcClient extends AbstractRpcClient { - - private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class); - - public static final String CLIENT_MAX_THREADS = "hbase.rpc.client.threads.max"; - public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.nativetransport"; - public static final String USE_GLOBAL_EVENT_LOOP_GROUP = "hbase.rpc.client.globaleventloopgroup"; - - private static final HashedWheelTimer WHEEL_TIMER = - new HashedWheelTimer(Threads.newDaemonThreadFactory("AsyncRpcChannel-timer"), - 100, TimeUnit.MILLISECONDS); - - private static final ChannelInitializer DEFAULT_CHANNEL_INITIALIZER = - new ChannelInitializer() { - @Override - protected void initChannel(SocketChannel ch) throws Exception { - //empty initializer - } - }; - - protected final AtomicInteger callIdCnt = new AtomicInteger(); - - private final PoolMap connections; - - final FailedServers failedServers; - - @VisibleForTesting - final Bootstrap bootstrap; - - private final boolean useGlobalEventLoopGroup; - - @VisibleForTesting - static Pair> GLOBAL_EVENT_LOOP_GROUP; - - private synchronized static Pair> - getGlobalEventLoopGroup(Configuration conf) { - if (GLOBAL_EVENT_LOOP_GROUP == null) { - GLOBAL_EVENT_LOOP_GROUP = createEventLoopGroup(conf); - if (LOG.isDebugEnabled()) { - LOG.debug("Create global event loop group " - + GLOBAL_EVENT_LOOP_GROUP.getFirst().getClass().getSimpleName()); - } - } - return GLOBAL_EVENT_LOOP_GROUP; - } - - private static Pair> createEventLoopGroup( - Configuration conf) { - // Max amount of threads to use. 0 lets Netty decide based on amount of cores - int maxThreads = conf.getInt(CLIENT_MAX_THREADS, 0); - - // Config to enable native transport. Does not seem to be stable at time of implementation - // although it is not extensively tested. - boolean epollEnabled = conf.getBoolean(USE_NATIVE_TRANSPORT, false); - - // Use the faster native epoll transport mechanism on linux if enabled - if (epollEnabled && JVM.isLinux() && JVM.isAmd64()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Create EpollEventLoopGroup with maxThreads = " + maxThreads); - } - return new Pair>(new EpollEventLoopGroup(maxThreads, - Threads.newDaemonThreadFactory("AsyncRpcChannel")), EpollSocketChannel.class); - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Create NioEventLoopGroup with maxThreads = " + maxThreads); - } - return new Pair>(new NioEventLoopGroup(maxThreads, - Threads.newDaemonThreadFactory("AsyncRpcChannel")), NioSocketChannel.class); - } - } - - /** - * Constructor for tests - * - * @param configuration to HBase - * @param clusterId for the cluster - * @param localAddress local address to connect to - * @param metrics the connection metrics - * @param channelInitializer for custom channel handlers - */ - protected AsyncRpcClient(Configuration configuration, String clusterId, - SocketAddress localAddress, MetricsConnection metrics, - ChannelInitializer channelInitializer) { - super(configuration, clusterId, localAddress, metrics); - - if (LOG.isDebugEnabled()) { - LOG.debug("Starting async Hbase RPC client"); - } - - Pair> eventLoopGroupAndChannelClass; - this.useGlobalEventLoopGroup = conf.getBoolean(USE_GLOBAL_EVENT_LOOP_GROUP, true); - if (useGlobalEventLoopGroup) { - eventLoopGroupAndChannelClass = getGlobalEventLoopGroup(configuration); - } else { - eventLoopGroupAndChannelClass = createEventLoopGroup(configuration); - } - if (LOG.isDebugEnabled()) { - LOG.debug("Use " + (useGlobalEventLoopGroup ? "global" : "individual") + " event loop group " - + eventLoopGroupAndChannelClass.getFirst().getClass().getSimpleName()); - } - - this.connections = new PoolMap<>(getPoolType(configuration), getPoolSize(configuration)); - this.failedServers = new FailedServers(configuration); - - int operationTimeout = configuration.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - - // Configure the default bootstrap. - this.bootstrap = new Bootstrap(); - bootstrap.group(eventLoopGroupAndChannelClass.getFirst()) - .channel(eventLoopGroupAndChannelClass.getSecond()) - .option(ChannelOption.TCP_NODELAY, tcpNoDelay) - .option(ChannelOption.SO_KEEPALIVE, tcpKeepAlive) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, operationTimeout); - if (channelInitializer == null) { - channelInitializer = DEFAULT_CHANNEL_INITIALIZER; - } - bootstrap.handler(channelInitializer); - if (localAddress != null) { - bootstrap.localAddress(localAddress); - } - } - - /** Used in test only. */ - AsyncRpcClient(Configuration configuration) { - this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null); - } - - /** Used in test only. */ - AsyncRpcClient(Configuration configuration, - ChannelInitializer channelInitializer) { - this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null, channelInitializer); - } - - /** - * Constructor - * - * @param configuration to HBase - * @param clusterId for the cluster - * @param localAddress local address to connect to - * @param metrics the connection metrics - */ - public AsyncRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress, - MetricsConnection metrics) { - this(configuration, clusterId, localAddress, metrics, null); - } - - /** - * Make a call, passing param, to the IPC server running at - * address which is servicing the protocol protocol, - * with the ticket credentials, returning the value. - * Throws exceptions if there are network problems or if the remote code - * threw an exception. - * - * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. - * {@link org.apache.hadoop.hbase.security.UserProvider#getCurrent()} makes a new - * instance of User each time so will be a new Connection each time. - * @return A pair with the Message response and the Cell data (if any). - * @throws InterruptedException if call is interrupted - * @throws java.io.IOException if a connection failure is encountered - */ - @Override - protected Pair call(PayloadCarryingRpcController pcrc, - Descriptors.MethodDescriptor md, Message param, Message returnType, User ticket, - InetSocketAddress addr, MetricsConnection.CallStats callStats) - throws IOException, InterruptedException { - if (pcrc == null) { - pcrc = new PayloadCarryingRpcController(); - } - final AsyncRpcChannel connection = createRpcChannel(md.getService().getName(), addr, ticket); - - Promise promise = connection.callMethod(md, pcrc, param, returnType, callStats); - long timeout = pcrc.hasCallTimeout() ? pcrc.getCallTimeout() : 0; - try { - Message response = timeout > 0 ? promise.get(timeout, TimeUnit.MILLISECONDS) : promise.get(); - return new Pair<>(response, pcrc.cellScanner()); - } catch (ExecutionException e) { - if (e.getCause() instanceof IOException) { - throw (IOException) e.getCause(); - } else { - throw wrapException(addr, (Exception) e.getCause()); - } - } catch (TimeoutException e) { - CallTimeoutException cte = new CallTimeoutException(promise.toString()); - throw wrapException(addr, cte); - } - } - - /** - * Call method async - */ - private void callMethod(final Descriptors.MethodDescriptor md, - final PayloadCarryingRpcController pcrc, final Message param, Message returnType, User ticket, - InetSocketAddress addr, final RpcCallback done) { - final AsyncRpcChannel connection; - try { - connection = createRpcChannel(md.getService().getName(), addr, ticket); - final MetricsConnection.CallStats cs = MetricsConnection.newCallStats(); - GenericFutureListener> listener = - new GenericFutureListener>() { - @Override - public void operationComplete(Future future) throws Exception { - cs.setCallTimeMs(EnvironmentEdgeManager.currentTime() - cs.getStartTime()); - if (metrics != null) { - metrics.updateRpc(md, param, cs); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Call: " + md.getName() + ", callTime: " + cs.getCallTimeMs() + "ms"); - } - if (!future.isSuccess()) { - Throwable cause = future.cause(); - if (cause instanceof IOException) { - pcrc.setFailed((IOException) cause); - } else { - pcrc.setFailed(new IOException(cause)); - } - } else { - try { - done.run(future.get()); - } catch (ExecutionException e) { - Throwable cause = e.getCause(); - if (cause instanceof IOException) { - pcrc.setFailed((IOException) cause); - } else { - pcrc.setFailed(new IOException(cause)); - } - } catch (InterruptedException e) { - pcrc.setFailed(new IOException(e)); - } - } - } - }; - cs.setStartTime(EnvironmentEdgeManager.currentTime()); - connection.callMethod(md, pcrc, param, returnType, cs).addListener(listener); - } catch (StoppedRpcClientException|FailedServerException e) { - pcrc.setFailed(e); - } - } - - private boolean closed = false; - - /** - * Close netty - */ - public void close() { - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping async HBase RPC client"); - } - - synchronized (connections) { - if (closed) { - return; - } - closed = true; - for (AsyncRpcChannel conn : connections.values()) { - conn.close(null); - } - } - // do not close global EventLoopGroup. - if (!useGlobalEventLoopGroup) { - bootstrap.group().shutdownGracefully(); - } - } - - /** - * Create a cell scanner - * - * @param cellBlock to create scanner for - * @return CellScanner - * @throws java.io.IOException on error on creation cell scanner - */ - public CellScanner createCellScanner(byte[] cellBlock) throws IOException { - return ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock); - } - - /** - * Build cell block - * - * @param cells to create block with - * @return ByteBuffer with cells - * @throws java.io.IOException if block creation fails - */ - public ByteBuffer buildCellBlock(CellScanner cells) throws IOException { - return ipcUtil.buildCellBlock(this.codec, this.compressor, cells); - } - - /** - * Creates an RPC client - * - * @param serviceName name of servicce - * @param location to connect to - * @param ticket for current user - * @return new RpcChannel - * @throws StoppedRpcClientException when Rpc client is stopped - * @throws FailedServerException if server failed - */ - private AsyncRpcChannel createRpcChannel(String serviceName, InetSocketAddress location, - User ticket) throws StoppedRpcClientException, FailedServerException { - // Check if server is failed - if (this.failedServers.isFailedServer(location)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not trying to connect to " + location + - " this server is in the failed servers list"); - } - throw new FailedServerException( - "This server is in the failed servers list: " + location); - } - - int hashCode = ConnectionId.hashCode(ticket,serviceName,location); - - AsyncRpcChannel rpcChannel; - synchronized (connections) { - if (closed) { - throw new StoppedRpcClientException(); - } - rpcChannel = connections.get(hashCode); - if (rpcChannel == null || !rpcChannel.isAlive()) { - rpcChannel = new AsyncRpcChannel(this.bootstrap, this, ticket, serviceName, location); - connections.put(hashCode, rpcChannel); - } - } - - return rpcChannel; - } - - /** - * Interrupt the connections to the given ip:port server. This should be called if the server - * is known as actually dead. This will not prevent current operation to be retried, and, - * depending on their own behavior, they may retry on the same server. This can be a feature, - * for example at startup. In any case, they're likely to get connection refused (if the - * process died) or no route to host: i.e. there next retries should be faster and with a - * safe exception. - * - * @param sn server to cancel connections for - */ - @Override - public void cancelConnections(ServerName sn) { - synchronized (connections) { - for (AsyncRpcChannel rpcChannel : connections.values()) { - if (rpcChannel.isAlive() && - rpcChannel.address.getPort() == sn.getPort() && - rpcChannel.address.getHostName().contentEquals(sn.getHostname())) { - LOG.info("The server on " + sn.toString() + - " is dead - stopping the connection " + rpcChannel.toString()); - rpcChannel.close(null); - } - } - } - } - - /** - * Remove connection from pool - */ - public void removeConnection(AsyncRpcChannel connection) { - int connectionHashCode = connection.hashCode(); - synchronized (connections) { - // we use address as cache key, so we should check here to prevent removing the - // wrong connection - AsyncRpcChannel connectionInPool = this.connections.get(connectionHashCode); - if (connectionInPool != null && connectionInPool.equals(connection)) { - this.connections.remove(connectionHashCode); - } else if (LOG.isDebugEnabled()) { - LOG.debug(String.format("%s already removed, expected instance %08x, actual %08x", - connection.toString(), System.identityHashCode(connection), - System.identityHashCode(connectionInPool))); - } - } - } - - /** - * Creates a "channel" that can be used by a protobuf service. Useful setting up - * protobuf stubs. - * - * @param sn server name describing location of server - * @param user which is to use the connection - * @param rpcTimeout default rpc operation timeout - * - * @return A rpc channel that goes via this rpc client instance. - */ - public RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout) { - return new RpcChannelImplementation(this, sn, user, rpcTimeout); - } - - /** - * Blocking rpc channel that goes via hbase rpc. - */ - @VisibleForTesting - public static class RpcChannelImplementation implements RpcChannel { - private final InetSocketAddress isa; - private final AsyncRpcClient rpcClient; - private final User ticket; - private final int channelOperationTimeout; - - /** - * @param channelOperationTimeout - the default timeout when no timeout is given - */ - protected RpcChannelImplementation(final AsyncRpcClient rpcClient, - final ServerName sn, final User ticket, int channelOperationTimeout) { - this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort()); - this.rpcClient = rpcClient; - this.ticket = ticket; - this.channelOperationTimeout = channelOperationTimeout; - } - - @Override - public void callMethod(Descriptors.MethodDescriptor md, RpcController controller, - Message param, Message returnType, RpcCallback done) { - PayloadCarryingRpcController pcrc; - if (controller != null) { - pcrc = (PayloadCarryingRpcController) controller; - if (!pcrc.hasCallTimeout()) { - pcrc.setCallTimeout(channelOperationTimeout); - } - } else { - pcrc = new PayloadCarryingRpcController(); - pcrc.setCallTimeout(channelOperationTimeout); - } - - this.rpcClient.callMethod(md, pcrc, param, returnType, this.ticket, this.isa, done); - } - } - - Timeout newTimeout(TimerTask task, long delay, TimeUnit unit) { - return WHEEL_TIMER.newTimeout(task, delay, unit); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java deleted file mode 100644 index e0c7586477c..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java +++ /dev/null @@ -1,126 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import java.io.IOException; - -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; -import org.apache.hadoop.ipc.RemoteException; - -import com.google.protobuf.Message; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufInputStream; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; - -/** - * Handles Hbase responses - */ -@InterfaceAudience.Private -public class AsyncServerResponseHandler extends SimpleChannelInboundHandler { - private final AsyncRpcChannel channel; - - /** - * Constructor - * @param channel on which this response handler operates - */ - public AsyncServerResponseHandler(AsyncRpcChannel channel) { - this.channel = channel; - } - - @Override - protected void channelRead0(ChannelHandlerContext ctx, ByteBuf inBuffer) throws Exception { - ByteBufInputStream in = new ByteBufInputStream(inBuffer); - int totalSize = inBuffer.readableBytes(); - // Read the header - RPCProtos.ResponseHeader responseHeader = RPCProtos.ResponseHeader.parseDelimitedFrom(in); - int id = responseHeader.getCallId(); - AsyncCall call = channel.removePendingCall(id); - if (call == null) { - // So we got a response for which we have no corresponding 'call' here on the client-side. - // We probably timed out waiting, cleaned up all references, and now the server decides - // to return a response. There is nothing we can do w/ the response at this stage. Clean - // out the wire of the response so its out of the way and we can get other responses on - // this connection. - int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader); - int whatIsLeftToRead = totalSize - readSoFar; - - // This is done through a Netty ByteBuf which has different behavior than InputStream. - // It does not return number of bytes read but will update pointer internally and throws an - // exception when too many bytes are to be skipped. - inBuffer.skipBytes(whatIsLeftToRead); - return; - } - - if (responseHeader.hasException()) { - RPCProtos.ExceptionResponse exceptionResponse = responseHeader.getException(); - RemoteException re = createRemoteException(exceptionResponse); - if (exceptionResponse.getExceptionClassName() - .equals(FatalConnectionException.class.getName())) { - channel.close(re); - } else { - call.setFailed(re); - } - } else { - Message value = null; - // Call may be null because it may have timedout and been cleaned up on this side already - if (call.responseDefaultType != null) { - Message.Builder builder = call.responseDefaultType.newBuilderForType(); - ProtobufUtil.mergeDelimitedFrom(builder, in); - value = builder.build(); - } - CellScanner cellBlockScanner = null; - if (responseHeader.hasCellBlockMeta()) { - int size = responseHeader.getCellBlockMeta().getLength(); - byte[] cellBlock = new byte[size]; - inBuffer.readBytes(cellBlock, 0, cellBlock.length); - cellBlockScanner = channel.client.createCellScanner(cellBlock); - } - call.setSuccess(value, cellBlockScanner); - call.callStats.setResponseSizeBytes(totalSize); - } - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - channel.close(cause); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - channel.close(new IOException("connection closed")); - } - - /** - * @param e Proto exception - * @return RemoteException made from passed e - */ - private RemoteException createRemoteException(final RPCProtos.ExceptionResponse e) { - String innerExceptionClassName = e.getExceptionClassName(); - boolean doNotRetry = e.getDoNotRetry(); - return e.hasHostname() ? - // If a hostname then add it to the RemoteWithExtrasException - new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), e.getHostname(), - e.getPort(), doNotRetry) - : new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java new file mode 100644 index 00000000000..d27602e7de6 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import com.google.common.annotations.VisibleForTesting; + +import java.io.IOException; +import java.net.SocketAddress; + +import javax.net.SocketFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.MetricsConnection; +import org.apache.hadoop.net.NetUtils; + +/** + * Does RPC against a cluster. Manages connections per regionserver in the cluster. + *

+ * See HBaseServer + */ +@InterfaceAudience.Private +public class BlockingRpcClient extends AbstractRpcClient { + + protected final SocketFactory socketFactory; // how to create sockets + + /** + * Used in test only. Construct an IPC client for the cluster {@code clusterId} with the default + * SocketFactory + */ + @VisibleForTesting + BlockingRpcClient(Configuration conf) { + this(conf, HConstants.CLUSTER_ID_DEFAULT, null, null); + } + + /** + * Construct an IPC client for the cluster {@code clusterId} with the default SocketFactory This + * method is called with reflection by the RpcClientFactory to create an instance + * @param conf configuration + * @param clusterId the cluster id + * @param localAddr client socket bind address. + * @param metrics the connection metrics + */ + public BlockingRpcClient(Configuration conf, String clusterId, SocketAddress localAddr, + MetricsConnection metrics) { + super(conf, clusterId, localAddr, metrics); + this.socketFactory = NetUtils.getDefaultSocketFactory(conf); + } + + /** + * Creates a connection. Can be overridden by a subclass for testing. + * @param remoteId - the ConnectionId to use for the connection creation. + */ + protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException { + return new BlockingRpcConnection(this, remoteId); + } + + @Override + protected void closeInternal() { + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java new file mode 100644 index 00000000000..4dc121cd0c8 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java @@ -0,0 +1,730 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader; +import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException; +import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited; +import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException; +import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled; +import static org.apache.hadoop.hbase.ipc.IPCUtil.write; + +import com.google.protobuf.Message; +import com.google.protobuf.Message.Builder; +import com.google.protobuf.RpcCallback; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.io.OutputStream; +import java.net.Socket; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayDeque; +import java.util.Locale; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ThreadLocalRandom; + +import javax.security.sasl.SaslException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; +import org.apache.hadoop.hbase.io.ByteArrayOutputStream; +import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader; +import org.apache.hadoop.hbase.security.HBaseSaslRpcClient; +import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ExceptionUtil; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.htrace.Trace; +import org.apache.htrace.TraceScope; + +/** + * Thread that reads responses and notifies callers. Each connection owns a socket connected to a + * remote address. Calls are multiplexed through this socket: responses may be delivered out of + * order. + */ +@InterfaceAudience.Private +class BlockingRpcConnection extends RpcConnection implements Runnable { + + private static final Log LOG = LogFactory.getLog(BlockingRpcConnection.class); + + private final BlockingRpcClient rpcClient; + + private final String threadName; + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", + justification = "We are always under lock actually") + private Thread thread; + + // connected socket. protected for writing UT. + protected Socket socket = null; + private DataInputStream in; + private DataOutputStream out; + + private HBaseSaslRpcClient saslRpcClient; + + // currently active calls + private final ConcurrentMap calls = new ConcurrentHashMap<>(); + + private final CallSender callSender; + + private boolean closed = false; + + private byte[] connectionHeaderPreamble; + + private byte[] connectionHeaderWithLength; + + /** + * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), it gets into a + * java issue: an interruption during a write closes the socket/channel. A way to avoid this is to + * use a different thread for writing. This way, on interruptions, we either cancel the writes or + * ignore the answer if the write is already done, but we don't stop the write in the middle. This + * adds a thread per region server in the client, so it's kept as an option. + *

+ * The implementation is simple: the client threads adds their call to the queue, and then wait + * for an answer. The CallSender blocks on the queue, and writes the calls one after the other. On + * interruption, the client cancels its call. The CallSender checks that the call has not been + * canceled before writing it. + *

+ * When the connection closes, all the calls not yet sent are dismissed. The client thread is + * notified with an appropriate exception, as if the call was already sent but the answer not yet + * received. + *

+ */ + private class CallSender extends Thread { + + private final Queue callsToWrite; + + private final int maxQueueSize; + + public CallSender(String name, Configuration conf) { + int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000); + callsToWrite = new ArrayDeque<>(queueSize); + this.maxQueueSize = queueSize; + setDaemon(true); + setName(name + " - writer"); + } + + public void sendCall(final Call call) throws IOException { + if (callsToWrite.size() >= maxQueueSize) { + throw new IOException("Can't add the call " + call.id + + " to the write queue. callsToWrite.size()=" + callsToWrite.size()); + } + callsToWrite.offer(call); + BlockingRpcConnection.this.notifyAll(); + } + + public void remove(Call call) { + callsToWrite.remove(); + // By removing the call from the expected call list, we make the list smaller, but + // it means as well that we don't know how many calls we cancelled. + calls.remove(call.id); + call.setException(new CallCancelledException("Call id=" + call.id + ", waitTime=" + + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout=" + + call.timeout)); + } + + /** + * Reads the call from the queue, write them on the socket. + */ + @Override + public void run() { + synchronized (BlockingRpcConnection.this) { + while (!closed) { + if (callsToWrite.isEmpty()) { + // We should use another monitor object here for better performance since the read + // thread also uses ConnectionImpl.this. But this makes the locking schema more + // complicated, can do it later as an optimization. + try { + BlockingRpcConnection.this.wait(); + } catch (InterruptedException e) { + } + // check if we need to quit, so continue the main loop instead of fallback. + continue; + } + Call call = callsToWrite.poll(); + if (call.isDone()) { + continue; + } + try { + tracedWriteRequest(call); + } catch (IOException e) { + // exception here means the call has not been added to the pendingCalls yet, so we need + // to fail it by our own. + if (LOG.isDebugEnabled()) { + LOG.debug("call write error for call #" + call.id, e); + } + call.setException(e); + closeConn(e); + } + } + } + } + + /** + * Cleans the call not yet sent when we finish. + */ + public void cleanup(IOException e) { + IOException ie = + new ConnectionClosingException("Connection to " + remoteId.address + " is closing."); + for (Call call : callsToWrite) { + call.setException(ie); + } + callsToWrite.clear(); + } + } + + BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException { + super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, + rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor); + this.rpcClient = rpcClient; + if (remoteId.getAddress().isUnresolved()) { + throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName()); + } + + this.connectionHeaderPreamble = getConnectionHeaderPreamble(); + ConnectionHeader header = getConnectionHeader(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(4 + header.getSerializedSize()); + DataOutputStream dos = new DataOutputStream(baos); + dos.writeInt(header.getSerializedSize()); + header.writeTo(dos); + assert baos.size() == 4 + header.getSerializedSize(); + this.connectionHeaderWithLength = baos.getBuffer(); + + UserGroupInformation ticket = remoteId.ticket.getUGI(); + this.threadName = "IPC Client (" + this.rpcClient.socketFactory.hashCode() + + ") connection to " + remoteId.getAddress().toString() + + ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName())); + + if (this.rpcClient.conf.getBoolean(BlockingRpcClient.SPECIFIC_WRITE_THREAD, false)) { + callSender = new CallSender(threadName, this.rpcClient.conf); + callSender.start(); + } else { + callSender = null; + } + } + + // protected for write UT. + protected void setupConnection() throws IOException { + short ioFailures = 0; + short timeoutFailures = 0; + while (true) { + try { + this.socket = this.rpcClient.socketFactory.createSocket(); + this.socket.setTcpNoDelay(this.rpcClient.isTcpNoDelay()); + this.socket.setKeepAlive(this.rpcClient.tcpKeepAlive); + if (this.rpcClient.localAddr != null) { + this.socket.bind(this.rpcClient.localAddr); + } + NetUtils.connect(this.socket, remoteId.getAddress(), this.rpcClient.connectTO); + this.socket.setSoTimeout(this.rpcClient.readTO); + return; + } catch (SocketTimeoutException toe) { + /* + * The max number of retries is 45, which amounts to 20s*45 = 15 minutes retries. + */ + handleConnectionFailure(timeoutFailures++, this.rpcClient.maxRetries, toe); + } catch (IOException ie) { + handleConnectionFailure(ioFailures++, this.rpcClient.maxRetries, ie); + } + } + } + + /** + * Handle connection failures If the current number of retries is equal to the max number of + * retries, stop retrying and throw the exception; Otherwise backoff N seconds and try connecting + * again. This Method is only called from inside setupIOstreams(), which is synchronized. Hence + * the sleep is synchronized; the locks will be retained. + * @param curRetries current number of retries + * @param maxRetries max number of retries allowed + * @param ioe failure reason + * @throws IOException if max number of retries is reached + */ + private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe) + throws IOException { + closeSocket(); + + // throw the exception if the maximum number of retries is reached + if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) { + throw ioe; + } + + // otherwise back off and retry + try { + Thread.sleep(this.rpcClient.failureSleep); + } catch (InterruptedException ie) { + ExceptionUtil.rethrowIfInterrupt(ie); + } + + LOG.info("Retrying connect to server: " + remoteId.getAddress() + " after sleeping " + + this.rpcClient.failureSleep + "ms. Already tried " + curRetries + " time(s)."); + } + + /* + * wait till someone signals us to start reading RPC response or it is idle too long, it is marked + * as to be closed, or the client is marked as not running. + * @return true if it is time to read a response; false otherwise. + */ + private synchronized boolean waitForWork() { + // beware of the concurrent access to the calls list: we can add calls, but as well + // remove them. + long waitUntil = EnvironmentEdgeManager.currentTime() + this.rpcClient.minIdleTimeBeforeClose; + for (;;) { + if (thread == null) { + return false; + } + if (!calls.isEmpty()) { + return true; + } + if (EnvironmentEdgeManager.currentTime() >= waitUntil) { + closeConn( + new IOException("idle connection closed with " + calls.size() + " pending request(s)")); + return false; + } + try { + wait(Math.min(this.rpcClient.minIdleTimeBeforeClose, 1000)); + } catch (InterruptedException e) { + } + } + } + + @Override + public void run() { + if (LOG.isTraceEnabled()) { + LOG.trace(threadName + ": starting, connections " + this.rpcClient.connections.size()); + } + while (waitForWork()) { + readResponse(); + } + if (LOG.isTraceEnabled()) { + LOG.trace(threadName + ": stopped, connections " + this.rpcClient.connections.size()); + } + } + + private void disposeSasl() { + if (saslRpcClient != null) { + saslRpcClient.dispose(); + saslRpcClient = null; + } + } + + private boolean setupSaslConnection(final InputStream in2, final OutputStream out2) + throws IOException { + saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, + this.rpcClient.fallbackAllowed, this.rpcClient.conf.get("hbase.rpc.protection", + QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT))); + return saslRpcClient.saslConnect(in2, out2); + } + + /** + * If multiple clients with the same principal try to connect to the same server at the same time, + * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to + * work around this, what is done is that the client backs off randomly and tries to initiate the + * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is + * attempted. + *

+ * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the + * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such + * cases, it is prudent to throw a runtime exception when we receive a SaslException from the + * underlying authentication implementation, so there is no retry from other high level (for eg, + * HCM or HBaseAdmin). + *

+ */ + private void handleSaslConnectionFailure(final int currRetries, final int maxRetries, + final Exception ex, final UserGroupInformation user) + throws IOException, InterruptedException { + closeSocket(); + user.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws IOException, InterruptedException { + if (shouldAuthenticateOverKrb()) { + if (currRetries < maxRetries) { + if (LOG.isDebugEnabled()) { + LOG.debug("Exception encountered while connecting to " + "the server : " + ex); + } + // try re-login + relogin(); + disposeSasl(); + // have granularity of milliseconds + // we are sleeping with the Connection lock held but since this + // connection instance is being used for connecting to the server + // in question, it is okay + Thread.sleep(ThreadLocalRandom.current().nextInt(reloginMaxBackoff) + 1); + return null; + } else { + String msg = "Couldn't setup connection for " + + UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal; + LOG.warn(msg, ex); + throw (IOException) new IOException(msg).initCause(ex); + } + } else { + LOG.warn("Exception encountered while connecting to " + "the server : " + ex); + } + if (ex instanceof RemoteException) { + throw (RemoteException) ex; + } + if (ex instanceof SaslException) { + String msg = "SASL authentication failed." + + " The most likely cause is missing or invalid credentials." + " Consider 'kinit'."; + LOG.fatal(msg, ex); + throw new RuntimeException(msg, ex); + } + throw new IOException(ex); + } + }); + } + + private void setupIOstreams() throws IOException { + if (socket != null) { + // The connection is already available. Perfect. + return; + } + + if (this.rpcClient.failedServers.isFailedServer(remoteId.getAddress())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Not trying to connect to " + remoteId.address + + " this server is in the failed servers list"); + } + throw new FailedServerException( + "This server is in the failed servers list: " + remoteId.address); + } + + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Connecting to " + remoteId.address); + } + + short numRetries = 0; + final short MAX_RETRIES = 5; + while (true) { + setupConnection(); + InputStream inStream = NetUtils.getInputStream(socket); + // This creates a socket with a write timeout. This timeout cannot be changed. + OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO); + // Write out the preamble -- MAGIC, version, and auth to use. + writeConnectionHeaderPreamble(outStream); + if (useSasl) { + final InputStream in2 = inStream; + final OutputStream out2 = outStream; + UserGroupInformation ticket = getUGI(); + boolean continueSasl; + if (ticket == null) { + throw new FatalConnectionException("ticket/user is null"); + } + try { + continueSasl = ticket.doAs(new PrivilegedExceptionAction() { + @Override + public Boolean run() throws IOException { + return setupSaslConnection(in2, out2); + } + }); + } catch (Exception ex) { + ExceptionUtil.rethrowIfInterrupt(ex); + handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, ticket); + continue; + } + if (continueSasl) { + // Sasl connect is successful. Let's set up Sasl i/o streams. + inStream = saslRpcClient.getInputStream(inStream); + outStream = saslRpcClient.getOutputStream(outStream); + } else { + // fall back to simple auth because server told us so. + // do not change authMethod and useSasl here, we should start from secure when + // reconnecting because regionserver may change its sasl config after restart. + } + } + this.in = new DataInputStream(new BufferedInputStream(inStream)); + this.out = new DataOutputStream(new BufferedOutputStream(outStream)); + // Now write out the connection header + writeConnectionHeader(); + break; + } + } catch (Throwable t) { + closeSocket(); + IOException e = ExceptionUtil.asInterrupt(t); + if (e == null) { + this.rpcClient.failedServers.addToFailedServers(remoteId.address); + if (t instanceof LinkageError) { + // probably the hbase hadoop version does not match the running hadoop version + e = new DoNotRetryIOException(t); + } else if (t instanceof IOException) { + e = (IOException) t; + } else { + e = new IOException("Could not set up IO Streams to " + remoteId.address, t); + } + } + throw e; + } + + // start the receiver thread after the socket connection has been set up + thread = new Thread(this, threadName); + thread.setDaemon(true); + thread.start(); + } + + /** + * Write the RPC header: {@code } + */ + private void writeConnectionHeaderPreamble(OutputStream out) throws IOException { + out.write(connectionHeaderPreamble); + out.flush(); + } + + /** + * Write the connection header. + */ + private void writeConnectionHeader() throws IOException { + this.out.write(connectionHeaderWithLength); + this.out.flush(); + } + + private void tracedWriteRequest(Call call) throws IOException { + try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", call.span)) { + writeRequest(call); + } + } + + /** + * Initiates a call by sending the parameter to the remote server. Note: this is not called from + * the Connection thread, but by other threads. + * @see #readResponse() + */ + private void writeRequest(Call call) throws IOException { + ByteBuffer cellBlock = + this.rpcClient.cellBlockBuilder.buildCellBlock(this.codec, this.compressor, call.cells); + CellBlockMeta cellBlockMeta; + if (cellBlock != null) { + cellBlockMeta = CellBlockMeta.newBuilder().setLength(cellBlock.limit()).build(); + } else { + cellBlockMeta = null; + } + RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta); + + setupIOstreams(); + + // Now we're going to write the call. We take the lock, then check that the connection + // is still valid, and, if so we do the write to the socket. If the write fails, we don't + // know where we stand, we have to close the connection. + if (Thread.interrupted()) { + throw new InterruptedIOException(); + } + + calls.put(call.id, call); // We put first as we don't want the connection to become idle. + // from here, we do not throw any exception to upper layer as the call has been tracked in the + // pending calls map. + try { + call.callStats.setRequestSizeBytes(write(this.out, requestHeader, call.param, cellBlock)); + } catch (IOException e) { + closeConn(e); + return; + } + notifyAll(); + } + + /* + * Receive a response. Because only one receiver, so no synchronization on in. + */ + private void readResponse() { + Call call = null; + boolean expectedCall = false; + try { + // See HBaseServer.Call.setResponse for where we write out the response. + // Total size of the response. Unused. But have to read it in anyways. + int totalSize = in.readInt(); + + // Read the header + ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); + int id = responseHeader.getCallId(); + call = calls.remove(id); // call.done have to be set before leaving this method + expectedCall = (call != null && !call.isDone()); + if (!expectedCall) { + // So we got a response for which we have no corresponding 'call' here on the client-side. + // We probably timed out waiting, cleaned up all references, and now the server decides + // to return a response. There is nothing we can do w/ the response at this stage. Clean + // out the wire of the response so its out of the way and we can get other responses on + // this connection. + int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader); + int whatIsLeftToRead = totalSize - readSoFar; + IOUtils.skipFully(in, whatIsLeftToRead); + if (call != null) { + call.callStats.setResponseSizeBytes(totalSize); + call.callStats + .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); + } + return; + } + if (responseHeader.hasException()) { + ExceptionResponse exceptionResponse = responseHeader.getException(); + RemoteException re = createRemoteException(exceptionResponse); + call.setException(re); + call.callStats.setResponseSizeBytes(totalSize); + call.callStats + .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); + if (isFatalConnectionException(exceptionResponse)) { + synchronized (this) { + closeConn(re); + } + } + } else { + Message value = null; + if (call.responseDefaultType != null) { + Builder builder = call.responseDefaultType.newBuilderForType(); + ProtobufUtil.mergeDelimitedFrom(builder, in); + value = builder.build(); + } + CellScanner cellBlockScanner = null; + if (responseHeader.hasCellBlockMeta()) { + int size = responseHeader.getCellBlockMeta().getLength(); + byte[] cellBlock = new byte[size]; + IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); + cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec, + this.compressor, cellBlock); + } + call.setResponse(value, cellBlockScanner); + call.callStats.setResponseSizeBytes(totalSize); + call.callStats + .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); + } + } catch (IOException e) { + if (expectedCall) { + call.setException(e); + } + if (e instanceof SocketTimeoutException) { + // Clean up open calls but don't treat this as a fatal condition, + // since we expect certain responses to not make it by the specified + // {@link ConnectionId#rpcTimeout}. + if (LOG.isTraceEnabled()) { + LOG.trace("ignored", e); + } + } else { + synchronized (this) { + closeConn(e); + } + } + } + } + + @Override + protected synchronized void callTimeout(Call call) { + // call sender + calls.remove(call.id); + } + + // just close socket input and output. + private void closeSocket() { + IOUtils.closeStream(out); + IOUtils.closeStream(in); + IOUtils.closeSocket(socket); + out = null; + in = null; + socket = null; + } + + // close socket, reader, and clean up all pending calls. + private void closeConn(IOException e) { + if (thread == null) { + return; + } + thread.interrupt(); + thread = null; + closeSocket(); + if (callSender != null) { + callSender.cleanup(e); + } + for (Call call : calls.values()) { + call.setException(e); + } + calls.clear(); + } + + // release all resources, the connection will not be used any more. + @Override + public synchronized void shutdown() { + closed = true; + if (callSender != null) { + callSender.interrupt(); + } + closeConn(new IOException("connection to " + remoteId.address + " closed")); + } + + @Override + public void cleanupConnection() { + // do nothing + } + + @Override + public synchronized void sendRequest(final Call call, HBaseRpcController pcrc) + throws IOException { + pcrc.notifyOnCancel(new RpcCallback() { + + @Override + public void run(Object parameter) { + setCancelled(call); + synchronized (BlockingRpcConnection.this) { + if (callSender != null) { + callSender.remove(call); + } else { + calls.remove(call.id); + } + } + } + }, new CancellationCallback() { + + @Override + public void run(boolean cancelled) throws IOException { + if (cancelled) { + setCancelled(call); + return; + } + scheduleTimeoutTask(call); + if (callSender != null) { + callSender.sendCall(call); + } else { + tracedWriteRequest(call); + } + } + }); + } + + @Override + public synchronized boolean isActive() { + return thread != null; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java new file mode 100644 index 00000000000..c628c3172c9 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPromise; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * We will expose the connection to upper layer before initialized, so we need to buffer the calls + * passed in and write them out once the connection is established. + */ +@InterfaceAudience.Private +class BufferCallBeforeInitHandler extends ChannelDuplexHandler { + + private enum BufferCallAction { + FLUSH, FAIL + } + + public static final class BufferCallEvent { + + public final BufferCallAction action; + + public final IOException error; + + private BufferCallEvent(BufferCallBeforeInitHandler.BufferCallAction action, + IOException error) { + this.action = action; + this.error = error; + } + + public static BufferCallBeforeInitHandler.BufferCallEvent success() { + return SUCCESS_EVENT; + } + + public static BufferCallBeforeInitHandler.BufferCallEvent fail(IOException error) { + return new BufferCallEvent(BufferCallAction.FAIL, error); + } + } + + private static final BufferCallEvent SUCCESS_EVENT = new BufferCallEvent(BufferCallAction.FLUSH, + null); + + private final Map id2Call = new HashMap<>(); + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) { + if (msg instanceof Call) { + Call call = (Call) msg; + id2Call.put(call.id, call); + // The call is already in track so here we set the write operation as success. + // We will fail the call directly if we can not write it out. + promise.trySuccess(); + } else { + ctx.write(msg, promise); + } + } + + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof BufferCallEvent) { + BufferCallEvent bcEvt = (BufferCallBeforeInitHandler.BufferCallEvent) evt; + switch (bcEvt.action) { + case FLUSH: + for (Call call : id2Call.values()) { + ctx.write(call); + } + break; + case FAIL: + for (Call call : id2Call.values()) { + call.setException(bcEvt.error); + } + break; + } + ctx.flush(); + ctx.pipeline().remove(this); + } else if (evt instanceof CallEvent) { + // just remove the call for now until we add other call event other than timeout and cancel. + id2Call.remove(((CallEvent) evt).call.id); + } else { + ctx.fireUserEventTriggered(evt); + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java index 5f90837e12b..a6203d54e6c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java @@ -19,36 +19,50 @@ package org.apache.hadoop.hbase.ipc; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; + +import io.netty.util.Timeout; + +import java.io.IOException; + import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; - -import java.io.IOException; +import org.apache.htrace.Span; +import org.apache.htrace.Trace; /** A call waiting for a value. */ @InterfaceAudience.Private -public class Call { - final int id; // call id - final Message param; // rpc request method param object +class Call { + final int id; // call id + final Message param; // rpc request method param object /** - * Optionally has cells when making call. Optionally has cells set on response. Used - * passing cells to the rpc and receiving the response. + * Optionally has cells when making call. Optionally has cells set on response. Used passing cells + * to the rpc and receiving the response. */ CellScanner cells; - Message response; // value, null if error - // The return type. Used to create shell into which we deserialize the response if any. + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", + justification = "Direct access is only allowed after done") + Message response; // value, null if error + // The return type. Used to create shell into which we deserialize the response if any. Message responseDefaultType; - IOException error; // exception, null if value - volatile boolean done; // true when call is done + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", + justification = "Direct access is only allowed after done") + IOException error; // exception, null if value + private boolean done; // true when call is done final Descriptors.MethodDescriptor md; final int timeout; // timeout in millisecond for this call; 0 means infinite. + final int priority; final MetricsConnection.CallStats callStats; + final RpcCallback callback; + final Span span; + Timeout timeoutTask; protected Call(int id, final Descriptors.MethodDescriptor md, Message param, - final CellScanner cells, final Message responseDefaultType, int timeout, - MetricsConnection.CallStats callStats) { + final CellScanner cells, final Message responseDefaultType, int timeout, int priority, + RpcCallback callback, MetricsConnection.CallStats callStats) { this.param = param; this.md = md; this.cells = cells; @@ -57,73 +71,74 @@ public class Call { this.responseDefaultType = responseDefaultType; this.id = id; this.timeout = timeout; - } - - /** - * Check if the call did timeout. Set an exception (includes a notify) if it's the case. - * @return true if the call is on timeout, false otherwise. - */ - public boolean checkAndSetTimeout() { - if (timeout == 0){ - return false; - } - - long waitTime = EnvironmentEdgeManager.currentTime() - getStartTime(); - if (waitTime >= timeout) { - IOException ie = new CallTimeoutException("Call id=" + id + - ", waitTime=" + waitTime + ", operationTimeout=" + timeout + " expired."); - setException(ie); // includes a notify - return true; - } else { - return false; - } - } - - public int remainingTime() { - if (timeout == 0) { - return Integer.MAX_VALUE; - } - - int remaining = timeout - (int) (EnvironmentEdgeManager.currentTime() - getStartTime()); - return remaining > 0 ? remaining : 0; + this.priority = priority; + this.callback = callback; + this.span = Trace.currentSpan(); } @Override public String toString() { - return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" + - (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") + "}"; + return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" + + (this.param != null ? ProtobufUtil.getShortTextFormat(this.param) : "") + "}"; } - /** Indicate when the call is complete and the - * value or error are available. Notifies by default. */ - protected synchronized void callComplete() { - this.done = true; - notify(); // notify caller + /** + * called from timeoutTask, prevent self cancel + */ + public void setTimeout(IOException error) { + synchronized (this) { + if (done) { + return; + } + this.done = true; + this.error = error; + } + callback.run(this); } - /** Set the exception when there is an error. - * Notify the caller the call is done. - * + private void callComplete() { + if (timeoutTask != null) { + timeoutTask.cancel(); + } + callback.run(this); + } + + /** + * Set the exception when there is an error. Notify the caller the call is done. * @param error exception thrown by the call; either local or remote */ public void setException(IOException error) { - this.error = error; + synchronized (this) { + if (done) { + return; + } + this.done = true; + this.error = error; + } callComplete(); } /** - * Set the return value when there is no error. - * Notify the caller the call is done. - * + * Set the return value when there is no error. Notify the caller the call is done. * @param response return value of the call. * @param cells Can be null */ public void setResponse(Message response, final CellScanner cells) { - this.response = response; - this.cells = cells; + synchronized (this) { + if (done) { + return; + } + this.done = true; + this.response = response; + this.cells = cells; + } callComplete(); } + public synchronized boolean isDone() { + return done; + } + public long getStartTime() { return this.callStats.getStartTime(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java similarity index 61% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java rename to hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java index 2fc270dd0f4..a6777c0e0c3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java @@ -15,19 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.hadoop.hbase.ipc; -package org.apache.hadoop.hbase.security; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.ipc.AsyncRpcClient; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.testclassification.SecurityTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.experimental.categories.Category; +/** + * Client side call cancelled. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class CallCancelledException extends HBaseIOException { -@Category({ SecurityTests.class, SmallTests.class }) -public class TestAsyncSecureIPC extends AbstractTestSecureIPC { + private static final long serialVersionUID = 309775809470318208L; - Class getRpcClientClass() { - return AsyncRpcClient.class; + public CallCancelledException(String message) { + super(message); } -} \ No newline at end of file + +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java new file mode 100644 index 00000000000..1c2ea324e2d --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Used to tell netty handler the call is cancelled, timeout... + */ +@InterfaceAudience.Private +class CallEvent { + + public enum Type { + TIMEOUT, CANCELLED + } + + final Type type; + + final Call call; + + CallEvent(Type type, Call call) { + this.type = type; + this.call = call; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java new file mode 100644 index 00000000000..0dac2d19fce --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java @@ -0,0 +1,293 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.ByteBufOutputStream; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.BufferOverflowException; +import java.nio.ByteBuffer; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.io.BoundedByteBufferPool; +import org.apache.hadoop.hbase.io.ByteBufferInputStream; +import org.apache.hadoop.hbase.io.ByteBufferOutputStream; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.io.compress.CodecPool; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; + +/** + * Helper class for building cell block. + */ +@InterfaceAudience.Private +class CellBlockBuilder { + + // LOG is being used in TestCellBlockBuilder + static final Log LOG = LogFactory.getLog(CellBlockBuilder.class); + + private final Configuration conf; + + /** + * How much we think the decompressor will expand the original compressed content. + */ + private final int cellBlockDecompressionMultiplier; + + private final int cellBlockBuildingInitialBufferSize; + + public CellBlockBuilder(Configuration conf) { + this.conf = conf; + this.cellBlockDecompressionMultiplier = + conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3); + + // Guess that 16k is a good size for rpc buffer. Could go bigger. See the TODO below in + // #buildCellBlock. + this.cellBlockBuildingInitialBufferSize = + ClassSize.align(conf.getInt("hbase.ipc.cellblock.building.initial.buffersize", 16 * 1024)); + } + + private interface OutputStreamSupplier { + + OutputStream get(int expectedSize); + + int size(); + } + + private static final class ByteBufferOutputStreamSupplier implements OutputStreamSupplier { + + private ByteBufferOutputStream baos; + + @Override + public OutputStream get(int expectedSize) { + baos = new ByteBufferOutputStream(expectedSize); + return baos; + } + + @Override + public int size() { + return baos.size(); + } + } + + /** + * Puts CellScanner Cells into a cell block using passed in codec and/or + * compressor. + * @param codec + * @param compressor + * @param cellScanner + * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using + * passed in codec and/or compressor; the returned buffer has + * been flipped and is ready for reading. Use limit to find total size. + * @throws IOException + */ + public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor, + final CellScanner cellScanner) throws IOException { + ByteBufferOutputStreamSupplier supplier = new ByteBufferOutputStreamSupplier(); + if (buildCellBlock(codec, compressor, cellScanner, supplier)) { + ByteBuffer bb = supplier.baos.getByteBuffer(); + // If no cells, don't mess around. Just return null (could be a bunch of existence checking + // gets or something -- stuff that does not return a cell). + return bb.hasRemaining() ? bb : null; + } else { + return null; + } + } + + private static final class ByteBufOutputStreamSupplier implements OutputStreamSupplier { + + private final ByteBufAllocator alloc; + + private ByteBuf buf; + + public ByteBufOutputStreamSupplier(ByteBufAllocator alloc) { + this.alloc = alloc; + } + + @Override + public OutputStream get(int expectedSize) { + buf = alloc.buffer(expectedSize); + return new ByteBufOutputStream(buf); + } + + @Override + public int size() { + return buf.writerIndex(); + } + } + + public ByteBuf buildCellBlock(Codec codec, CompressionCodec compressor, CellScanner cellScanner, + ByteBufAllocator alloc) throws IOException { + ByteBufOutputStreamSupplier supplier = new ByteBufOutputStreamSupplier(alloc); + if (buildCellBlock(codec, compressor, cellScanner, supplier)) { + return supplier.buf; + } else { + return null; + } + } + + private boolean buildCellBlock(final Codec codec, final CompressionCodec compressor, + final CellScanner cellScanner, OutputStreamSupplier supplier) throws IOException { + if (cellScanner == null) { + return false; + } + if (codec == null) { + throw new CellScannerButNoCodecException(); + } + int bufferSize = cellBlockBuildingInitialBufferSize; + encodeCellsTo(supplier.get(bufferSize), cellScanner, codec, compressor); + if (LOG.isTraceEnabled() && bufferSize < supplier.size()) { + LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + supplier.size() + + "; up hbase.ipc.cellblock.building.initial.buffersize?"); + } + return true; + } + + private void encodeCellsTo(OutputStream os, CellScanner cellScanner, Codec codec, + CompressionCodec compressor) throws IOException { + Compressor poolCompressor = null; + try { + if (compressor != null) { + if (compressor instanceof Configurable) { + ((Configurable) compressor).setConf(this.conf); + } + poolCompressor = CodecPool.getCompressor(compressor); + os = compressor.createOutputStream(os, poolCompressor); + } + Codec.Encoder encoder = codec.getEncoder(os); + while (cellScanner.advance()) { + encoder.write(cellScanner.current()); + } + encoder.flush(); + } catch (BufferOverflowException | IndexOutOfBoundsException e) { + throw new DoNotRetryIOException(e); + } finally { + os.close(); + if (poolCompressor != null) { + CodecPool.returnCompressor(poolCompressor); + } + } + } + + /** + * Puts CellScanner Cells into a cell block using passed in codec and/or + * compressor. + * @param codec + * @param compressor + * @param cellScanner + * @param pool Pool of ByteBuffers to make use of. Can be null and then we'll allocate our own + * ByteBuffer. + * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using + * passed in codec and/or compressor; the returned buffer has + * been flipped and is ready for reading. Use limit to find total size. If + * pool was not null, then this returned ByteBuffer came from there and + * should be returned to the pool when done. + * @throws IOException + */ + public ByteBuffer buildCellBlock(Codec codec, CompressionCodec compressor, + CellScanner cellScanner, BoundedByteBufferPool pool) throws IOException { + if (cellScanner == null) { + return null; + } + if (codec == null) { + throw new CellScannerButNoCodecException(); + } + ByteBufferOutputStream bbos; + ByteBuffer bb = null; + if (pool != null) { + bb = pool.getBuffer(); + bbos = new ByteBufferOutputStream(bb); + } else { + bbos = new ByteBufferOutputStream(cellBlockBuildingInitialBufferSize); + } + encodeCellsTo(bbos, cellScanner, codec, compressor); + if (bbos.size() == 0) { + if (pool != null) { + pool.putBuffer(bb); + } + return null; + } + return bbos.getByteBuffer(); + } + + /** + * @param codec to use for cellblock + * @param cellBlock to encode + * @return CellScanner to work against the content of cellBlock + * @throws IOException if encoding fails + */ + public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor, + final byte[] cellBlock) throws IOException { + return createCellScanner(codec, compressor, ByteBuffer.wrap(cellBlock)); + } + + /** + * @param codec + * @param cellBlock ByteBuffer containing the cells written by the Codec. The buffer should be + * position()'ed at the start of the cell block and limit()'ed at the end. + * @return CellScanner to work against the content of cellBlock + * @throws IOException + */ + public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor, + ByteBuffer cellBlock) throws IOException { + if (compressor != null) { + cellBlock = decompress(compressor, cellBlock); + } + // Not making the Decoder over the ByteBuffer purposefully. The Decoder over the BB will + // make Cells directly over the passed BB. This method is called at client side and we don't + // want the Cells to share the same byte[] where the RPC response is being read. Caching of any + // of the Cells at user's app level will make it not possible to GC the response byte[] + return codec.getDecoder(new ByteBufferInputStream(cellBlock)); + } + + private ByteBuffer decompress(CompressionCodec compressor, ByteBuffer cellBlock) + throws IOException { + // GZIPCodec fails w/ NPE if no configuration. + if (compressor instanceof Configurable) { + ((Configurable) compressor).setConf(this.conf); + } + Decompressor poolDecompressor = CodecPool.getDecompressor(compressor); + CompressionInputStream cis = + compressor.createInputStream(new ByteBufferInputStream(cellBlock), poolDecompressor); + ByteBufferOutputStream bbos; + try { + // TODO: This is ugly. The buffer will be resized on us if we guess wrong. + // TODO: Reuse buffers. + bbos = + new ByteBufferOutputStream(cellBlock.remaining() * this.cellBlockDecompressionMultiplier); + IOUtils.copy(cis, bbos); + bbos.close(); + cellBlock = bbos.getByteBuffer(); + } finally { + CodecPool.returnDecompressor(poolDecompressor); + } + return cellBlock; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java new file mode 100644 index 00000000000..ffd27b3f9b5 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * Thrown if a cellscanner but no codec to encode it with. + */ +@SuppressWarnings("serial") +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class CellScannerButNoCodecException extends HBaseIOException { +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java new file mode 100644 index 00000000000..f710d54e0b0 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.util.concurrent.DefaultThreadFactory; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.Pair; + +/** + * The default netty event loop config + */ +@InterfaceAudience.Private +class DefaultNettyEventLoopConfig { + + public static final Pair> GROUP_AND_CHANNEL_CLASS = Pair + .> newPair( + new NioEventLoopGroup(0, + new DefaultThreadFactory("Default-IPC-NioEventLoopGroup", true, Thread.MAX_PRIORITY)), + NioSocketChannel.class); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java new file mode 100644 index 00000000000..aaaea1f0116 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import com.google.protobuf.RpcCallback; + +import java.io.IOException; + +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Simple delegating controller for use with the {@link RpcControllerFactory} to help override + * standard behavior of a {@link HBaseRpcController}. Used testing. + */ +@InterfaceAudience.Private +public class DelegatingHBaseRpcController implements HBaseRpcController { + + private final HBaseRpcController delegate; + + public DelegatingHBaseRpcController(HBaseRpcController delegate) { + this.delegate = delegate; + } + + @Override + public void reset() { + delegate.reset(); + } + + @Override + public boolean failed() { + return delegate.failed(); + } + + @Override + public String errorText() { + return delegate.errorText(); + } + + @Override + public void startCancel() { + delegate.startCancel(); + } + + @Override + public void setFailed(String reason) { + delegate.setFailed(reason); + } + + @Override + public boolean isCanceled() { + return delegate.isCanceled(); + } + + @Override + public void notifyOnCancel(RpcCallback callback) { + delegate.notifyOnCancel(callback); + } + + @Override + public CellScanner cellScanner() { + return delegate.cellScanner(); + } + + @Override + public void setCellScanner(CellScanner cellScanner) { + delegate.setCellScanner(cellScanner); + } + + @Override + public void setPriority(int priority) { + delegate.setPriority(priority); + } + + @Override + public void setPriority(TableName tn) { + delegate.setPriority(tn); + } + + @Override + public int getPriority() { + return delegate.getPriority(); + } + + @Override + public int getCallTimeout() { + return delegate.getCallTimeout(); + } + + @Override + public void setCallTimeout(int callTimeout) { + delegate.setCallTimeout(callTimeout); + } + + @Override + public boolean hasCallTimeout() { + return delegate.hasCallTimeout(); + } + + @Override + public void setFailed(IOException e) { + delegate.setFailed(e); + } + + @Override + public IOException getFailed() { + return delegate.getFailed(); + } + + @Override + public void setDone(CellScanner cellScanner) { + delegate.setDone(cellScanner); + } + + @Override + public void notifyOnCancel(RpcCallback callback, CancellationCallback action) + throws IOException { + delegate.notifyOnCancel(callback, action); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingPayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingPayloadCarryingRpcController.java deleted file mode 100644 index ad4224b7c58..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingPayloadCarryingRpcController.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.classification.InterfaceAudience; - -/** - * Simple delegating controller for use with the {@link RpcControllerFactory} to help override - * standard behavior of a {@link PayloadCarryingRpcController}. - */ -@InterfaceAudience.Private -public class DelegatingPayloadCarryingRpcController extends PayloadCarryingRpcController { - private PayloadCarryingRpcController delegate; - - public DelegatingPayloadCarryingRpcController(PayloadCarryingRpcController delegate) { - this.delegate = delegate; - } - - @Override - public CellScanner cellScanner() { - return delegate.cellScanner(); - } - - @Override - public void setCellScanner(final CellScanner cellScanner) { - delegate.setCellScanner(cellScanner); - } - - @Override - public void setPriority(int priority) { - delegate.setPriority(priority); - } - - @Override - public void setPriority(final TableName tn) { - delegate.setPriority(tn); - } - - @Override - public int getPriority() { - return delegate.getPriority(); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java new file mode 100644 index 00000000000..721148b08b4 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * Indicate that the rpc server tells client to fallback to simple auth but client is disabled to do + * so. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class FallbackDisallowedException extends HBaseIOException { + + private static final long serialVersionUID = -6942845066279358253L; + + public FallbackDisallowedException() { + super("Server asks us to fall back to SIMPLE auth, " + + "but this client is configured to only allow secure connections."); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java new file mode 100644 index 00000000000..2c4b335d7dc --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; + +import java.io.IOException; + +import org.apache.hadoop.hbase.CellScannable; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Optionally carries Cells across the proxy/service interface down into ipc. On its way out it + * optionally carries a set of result Cell data. We stick the Cells here when we want to avoid + * having to protobuf them (for performance reasons). This class is used ferrying data across the + * proxy/protobuf service chasm. Also does call timeout. Used by client and server ipc'ing. + */ +@InterfaceAudience.Private +public interface HBaseRpcController extends RpcController, CellScannable { + + static final int PRIORITY_UNSET = -1; + + /** + * Only used to send cells to rpc server, the returned cells should be set by + * {@link #setDone(CellScanner)}. + */ + void setCellScanner(CellScanner cellScanner); + + /** + * @param priority Priority for this request; should fall roughly in the range + * {@link HConstants#NORMAL_QOS} to {@link HConstants#HIGH_QOS} + */ + void setPriority(int priority); + + /** + * @param tn Set priority based off the table we are going against. + */ + void setPriority(final TableName tn); + + /** + * @return The priority of this request + */ + int getPriority(); + + int getCallTimeout(); + + void setCallTimeout(int callTimeout); + + boolean hasCallTimeout(); + + /** + * Set failed with an exception to pass on. For use in async rpc clients + * @param e exception to set with + */ + void setFailed(IOException e); + + /** + * Return the failed exception, null if not failed. + */ + IOException getFailed(); + + /** + * IMPORTANT: always call this method if the call finished without any exception to tell + * the {@code HBaseRpcController} that we are done. + */ + void setDone(CellScanner cellScanner); + + /** + * A little different from the basic RpcController: + *
    + *
  1. You can register multiple callbacks to an {@code HBaseRpcController}.
  2. + *
  3. The callback will not be called if the rpc call is finished without any cancellation.
  4. + *
  5. You can call me at client side also.
  6. + *
+ */ + @Override + void notifyOnCancel(RpcCallback callback); + + interface CancellationCallback { + void run(boolean cancelled) throws IOException; + } + + /** + * If not cancelled, add the callback to cancellation callback list. And then execute the action + * with the cancellation state as a parameter. The implementation should guarantee that the + * cancellation state does not change during this call. + */ + void notifyOnCancel(RpcCallback callback, CancellationCallback action) throws IOException; +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java new file mode 100644 index 00000000000..a97647399ff --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java @@ -0,0 +1,244 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import com.google.protobuf.RpcCallback; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.CellScannable; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Optionally carries Cells across the proxy/service interface down into ipc. On its way out it + * optionally carries a set of result Cell data. We stick the Cells here when we want to avoid + * having to protobuf them (for performance reasons). This class is used ferrying data across the + * proxy/protobuf service chasm. Also does call timeout. Used by client and server ipc'ing. + */ +@InterfaceAudience.Private +public class HBaseRpcControllerImpl implements HBaseRpcController { + /** + * The time, in ms before the call should expire. + */ + private Integer callTimeout; + + private boolean done = false; + + private boolean cancelled = false; + + private final List> cancellationCbs = new ArrayList<>(); + + private IOException exception; + + /** + * Priority to set on this request. Set it here in controller so available composing the request. + * This is the ordained way of setting priorities going forward. We will be undoing the old + * annotation-based mechanism. + */ + private int priority = PRIORITY_UNSET; + + /** + * They are optionally set on construction, cleared after we make the call, and then optionally + * set on response with the result. We use this lowest common denominator access to Cells because + * sometimes the scanner is backed by a List of Cells and other times, it is backed by an encoded + * block that implements CellScanner. + */ + private CellScanner cellScanner; + + public HBaseRpcControllerImpl() { + this((CellScanner) null); + } + + public HBaseRpcControllerImpl(final CellScanner cellScanner) { + this.cellScanner = cellScanner; + } + + public HBaseRpcControllerImpl(final List cellIterables) { + this.cellScanner = cellIterables == null ? null : CellUtil.createCellScanner(cellIterables); + } + + /** + * @return One-shot cell scanner (you cannot back it up and restart) + */ + @Override + public CellScanner cellScanner() { + return cellScanner; + } + + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", + justification = "The only possible race method is startCancel") + @Override + public void setCellScanner(final CellScanner cellScanner) { + this.cellScanner = cellScanner; + } + + @Override + public void setPriority(int priority) { + this.priority = priority; + } + + @Override + public void setPriority(final TableName tn) { + setPriority( + tn != null && tn.isSystemTable() ? HConstants.SYSTEMTABLE_QOS : HConstants.NORMAL_QOS); + } + + @Override + public int getPriority() { + return priority; + } + + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", + justification = "The only possible race method is startCancel") + @Override + public void reset() { + priority = 0; + cellScanner = null; + exception = null; + callTimeout = null; + // In the implementations of some callable with replicas, rpc calls are executed in a executor + // and we could cancel the operation from outside which means there could be a race between + // reset and startCancel. Although I think the race should be handled by the callable since the + // reset may clear the cancel state... + synchronized (this) { + done = false; + cancelled = false; + cancellationCbs.clear(); + } + } + + @Override + public int getCallTimeout() { + if (callTimeout != null) { + return callTimeout.intValue(); + } else { + return 0; + } + } + + @Override + public void setCallTimeout(int callTimeout) { + this.callTimeout = callTimeout; + } + + @Override + public boolean hasCallTimeout() { + return callTimeout != null; + } + + @Override + public synchronized String errorText() { + if (!done || exception == null) { + return null; + } + return exception.getMessage(); + } + + @Override + public synchronized boolean failed() { + return done && this.exception != null; + } + + @Override + public synchronized boolean isCanceled() { + return cancelled; + } + + @Override + public void notifyOnCancel(RpcCallback callback) { + synchronized (this) { + if (done) { + return; + } + if (!cancelled) { + cancellationCbs.add(callback); + return; + } + } + // run it directly as we have already been cancelled. + callback.run(null); + } + + @Override + public synchronized void setFailed(String reason) { + if (done) { + return; + } + done = true; + exception = new IOException(reason); + } + + @Override + public synchronized void setFailed(IOException e) { + if (done) { + return; + } + done = true; + exception = e; + } + + @Override + public synchronized IOException getFailed() { + return done ? exception : null; + } + + @Override + public synchronized void setDone(CellScanner cellScanner) { + if (done) { + return; + } + done = true; + this.cellScanner = cellScanner; + } + + @Override + public void startCancel() { + // As said above in the comment of reset, the cancellationCbs maybe cleared by reset, so we need + // to copy it. + List> cbs; + synchronized (this) { + if (done) { + return; + } + done = true; + cancelled = true; + cbs = new ArrayList<>(cancellationCbs); + } + for (RpcCallback cb : cbs) { + cb.run(null); + } + } + + @Override + public synchronized void notifyOnCancel(RpcCallback callback, CancellationCallback action) + throws IOException { + if (cancelled) { + action.run(true); + } else { + cancellationCbs.add(callback); + action.run(false); + } + } + +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java index c238adbd56e..4fa58adb082 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java @@ -17,300 +17,173 @@ */ package org.apache.hadoop.hbase.ipc; -import java.io.DataInput; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.BufferOverflowException; -import java.nio.ByteBuffer; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseIOException; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.io.BoundedByteBufferPool; -import org.apache.hadoop.hbase.io.ByteBufferInputStream; -import org.apache.hadoop.hbase.io.ByteBufferOutputStream; -import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.io.compress.CodecPool; -import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.CompressionInputStream; -import org.apache.hadoop.io.compress.Compressor; -import org.apache.hadoop.io.compress.Decompressor; - import com.google.common.base.Preconditions; import com.google.protobuf.CodedOutputStream; import com.google.protobuf.Message; +import java.io.IOException; +import java.io.OutputStream; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; +import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.ipc.RemoteException; + /** * Utility to help ipc'ing. */ @InterfaceAudience.Private -public class IPCUtil { - // LOG is being used in TestIPCUtil - public static final Log LOG = LogFactory.getLog(IPCUtil.class); - /** - * How much we think the decompressor will expand the original compressed content. - */ - private final int cellBlockDecompressionMultiplier; - private final int cellBlockBuildingInitialBufferSize; - private final Configuration conf; - - public IPCUtil(final Configuration conf) { - super(); - this.conf = conf; - this.cellBlockDecompressionMultiplier = - conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3); - - // Guess that 16k is a good size for rpc buffer. Could go bigger. See the TODO below in - // #buildCellBlock. - this.cellBlockBuildingInitialBufferSize = - ClassSize.align(conf.getInt("hbase.ipc.cellblock.building.initial.buffersize", 16 * 1024)); - } - - /** - * Thrown if a cellscanner but no codec to encode it with. - */ - public static class CellScannerButNoCodecException extends HBaseIOException {}; - - /** - * Puts CellScanner Cells into a cell block using passed in codec and/or - * compressor. - * @param codec - * @param compressor - * @param cellScanner - * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using - * passed in codec and/or compressor; the returned buffer has been - * flipped and is ready for reading. Use limit to find total size. - * @throws IOException - */ - @SuppressWarnings("resource") - public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor, - final CellScanner cellScanner) - throws IOException { - return buildCellBlock(codec, compressor, cellScanner, null); - } - - /** - * Puts CellScanner Cells into a cell block using passed in codec and/or - * compressor. - * @param codec - * @param compressor - * @param cellScanner - * @param pool Pool of ByteBuffers to make use of. Can be null and then we'll allocate - * our own ByteBuffer. - * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using - * passed in codec and/or compressor; the returned buffer has been - * flipped and is ready for reading. Use limit to find total size. If pool was not - * null, then this returned ByteBuffer came from there and should be returned to the pool when - * done. - * @throws IOException - */ - @SuppressWarnings("resource") - public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor, - final CellScanner cellScanner, final BoundedByteBufferPool pool) - throws IOException { - if (cellScanner == null) return null; - if (codec == null) throw new CellScannerButNoCodecException(); - int bufferSize = this.cellBlockBuildingInitialBufferSize; - ByteBufferOutputStream baos = null; - ByteBuffer bb = null; - if (pool != null) { - bb = pool.getBuffer(); - bufferSize = bb.capacity(); - baos = new ByteBufferOutputStream(bb); - } else { - // Then we need to make our own to return. - if (cellScanner instanceof HeapSize) { - long longSize = ((HeapSize)cellScanner).heapSize(); - // Just make sure we don't have a size bigger than an int. - if (longSize > Integer.MAX_VALUE) { - throw new IOException("Size " + longSize + " > " + Integer.MAX_VALUE); - } - bufferSize = ClassSize.align((int)longSize); - } - baos = new ByteBufferOutputStream(bufferSize); - } - OutputStream os = baos; - Compressor poolCompressor = null; - try { - if (compressor != null) { - if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf); - poolCompressor = CodecPool.getCompressor(compressor); - os = compressor.createOutputStream(os, poolCompressor); - } - Codec.Encoder encoder = codec.getEncoder(os); - int count = 0; - while (cellScanner.advance()) { - encoder.write(cellScanner.current()); - count++; - } - encoder.flush(); - // If no cells, don't mess around. Just return null (could be a bunch of existence checking - // gets or something -- stuff that does not return a cell). - if (count == 0) { - if (pool != null && bb != null) { - pool.putBuffer(bb); - } - return null; - } - } catch (BufferOverflowException e) { - throw new DoNotRetryIOException(e); - } finally { - os.close(); - if (poolCompressor != null) CodecPool.returnCompressor(poolCompressor); - } - if (LOG.isTraceEnabled()) { - if (bufferSize < baos.size()) { - LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size() + - "; up hbase.ipc.cellblock.building.initial.buffersize?"); - } - } - return baos.getByteBuffer(); - } - - /** - * @param codec - * @param cellBlock - * @return CellScanner to work against the content of cellBlock - * @throws IOException - */ - public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor, - final byte [] cellBlock) - throws IOException { - return createCellScanner(codec, compressor, ByteBuffer.wrap(cellBlock)); - } - - /** - * @param codec - * @param cellBlock ByteBuffer containing the cells written by the Codec. The buffer should be - * position()'ed at the start of the cell block and limit()'ed at the end. - * @return CellScanner to work against the content of cellBlock - * @throws IOException - */ - public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor, - final ByteBuffer cellBlock) - throws IOException { - // If compressed, decompress it first before passing it on else we will leak compression - // resources if the stream is not closed properly after we let it out. - InputStream is = null; - if (compressor != null) { - // GZIPCodec fails w/ NPE if no configuration. - if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf); - Decompressor poolDecompressor = CodecPool.getDecompressor(compressor); - CompressionInputStream cis = - compressor.createInputStream(new ByteBufferInputStream(cellBlock), poolDecompressor); - ByteBufferOutputStream bbos = null; - try { - // TODO: This is ugly. The buffer will be resized on us if we guess wrong. - // TODO: Reuse buffers. - bbos = new ByteBufferOutputStream(cellBlock.remaining() * - this.cellBlockDecompressionMultiplier); - IOUtils.copy(cis, bbos); - bbos.close(); - ByteBuffer bb = bbos.getByteBuffer(); - is = new ByteBufferInputStream(bb); - } finally { - if (is != null) is.close(); - if (bbos != null) bbos.close(); - - CodecPool.returnDecompressor(poolDecompressor); - } - } else { - is = new ByteBufferInputStream(cellBlock); - } - return codec.getDecoder(is); - } - - /** - * @param m Message to serialize delimited; i.e. w/ a vint of its size preceeding its - * serialization. - * @return The passed in Message serialized with delimiter. Return null if m is null - * @throws IOException - */ - public static ByteBuffer getDelimitedMessageAsByteBuffer(final Message m) throws IOException { - if (m == null) return null; - int serializedSize = m.getSerializedSize(); - int vintSize = CodedOutputStream.computeRawVarint32Size(serializedSize); - byte [] buffer = new byte[serializedSize + vintSize]; - // Passing in a byte array saves COS creating a buffer which it does when using streams. - CodedOutputStream cos = CodedOutputStream.newInstance(buffer); - // This will write out the vint preamble and the message serialized. - cos.writeMessageNoTag(m); - cos.flush(); - cos.checkNoSpaceLeft(); - return ByteBuffer.wrap(buffer); - } +class IPCUtil { /** * Write out header, param, and cell block if there is one. - * @param dos - * @param header - * @param param - * @param cellBlock + * @param dos Stream to write into + * @param header to write + * @param param to write + * @param cellBlock to write * @return Total number of bytes written. - * @throws IOException + * @throws IOException if write action fails */ public static int write(final OutputStream dos, final Message header, final Message param, - final ByteBuffer cellBlock) - throws IOException { + final ByteBuffer cellBlock) throws IOException { // Must calculate total size and write that first so other side can read it all in in one - // swoop. This is dictated by how the server is currently written. Server needs to change + // swoop. This is dictated by how the server is currently written. Server needs to change // if we are to be able to write without the length prefixing. int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header, param); - if (cellBlock != null) totalSize += cellBlock.remaining(); + if (cellBlock != null) { + totalSize += cellBlock.remaining(); + } return write(dos, header, param, cellBlock, totalSize); } private static int write(final OutputStream dos, final Message header, final Message param, - final ByteBuffer cellBlock, final int totalSize) - throws IOException { + final ByteBuffer cellBlock, final int totalSize) throws IOException { // I confirmed toBytes does same as DataOutputStream#writeInt. dos.write(Bytes.toBytes(totalSize)); // This allocates a buffer that is the size of the message internally. header.writeDelimitedTo(dos); - if (param != null) param.writeDelimitedTo(dos); - if (cellBlock != null) dos.write(cellBlock.array(), 0, cellBlock.remaining()); + if (param != null) { + param.writeDelimitedTo(dos); + } + if (cellBlock != null) { + dos.write(cellBlock.array(), 0, cellBlock.remaining()); + } dos.flush(); return totalSize; } - /** - * Read in chunks of 8K (HBASE-7239) - * @param in - * @param dest - * @param offset - * @param len - * @throws IOException - */ - public static void readChunked(final DataInput in, byte[] dest, int offset, int len) - throws IOException { - int maxRead = 8192; - - for (; offset < len; offset += maxRead) { - in.readFully(dest, offset, Math.min(len - offset, maxRead)); - } - } - /** * @return Size on the wire when the two messages are written with writeDelimitedTo */ - public static int getTotalSizeWhenWrittenDelimited(Message ... messages) { + public static int getTotalSizeWhenWrittenDelimited(Message... messages) { int totalSize = 0; - for (Message m: messages) { - if (m == null) continue; + for (Message m : messages) { + if (m == null) { + continue; + } totalSize += m.getSerializedSize(); totalSize += CodedOutputStream.computeRawVarint32Size(m.getSerializedSize()); } Preconditions.checkArgument(totalSize < Integer.MAX_VALUE); return totalSize; } + + static RequestHeader buildRequestHeader(Call call, CellBlockMeta cellBlockMeta) { + RequestHeader.Builder builder = RequestHeader.newBuilder(); + builder.setCallId(call.id); + if (call.span != null) { + builder.setTraceInfo(RPCTInfo.newBuilder().setParentId(call.span.getSpanId()) + .setTraceId(call.span.getTraceId())); + } + builder.setMethodName(call.md.getName()); + builder.setRequestParam(call.param != null); + if (cellBlockMeta != null) { + builder.setCellBlockMeta(cellBlockMeta); + } + // Only pass priority if there is one set. + if (call.priority != HBaseRpcController.PRIORITY_UNSET) { + builder.setPriority(call.priority); + } + builder.setTimeout(call.timeout); + + return builder.build(); + } + + /** + * @param e exception to be wrapped + * @return RemoteException made from passed e + */ + static RemoteException createRemoteException(final ExceptionResponse e) { + String innerExceptionClassName = e.getExceptionClassName(); + boolean doNotRetry = e.getDoNotRetry(); + return e.hasHostname() ? + // If a hostname then add it to the RemoteWithExtrasException + new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), e.getHostname(), + e.getPort(), doNotRetry) + : new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry); + } + + /** + * @return True if the exception is a fatal connection exception. + */ + static boolean isFatalConnectionException(final ExceptionResponse e) { + return e.getExceptionClassName().equals(FatalConnectionException.class.getName()); + } + + static IOException toIOE(Throwable t) { + if (t instanceof IOException) { + return (IOException) t; + } else { + return new IOException(t); + } + } + + /** + * Takes an Exception and the address we were trying to connect to and return an IOException with + * the input exception as the cause. The new exception provides the stack trace of the place where + * the exception is thrown and some extra diagnostics information. If the exception is + * ConnectException or SocketTimeoutException, return a new one of the same type; Otherwise return + * an IOException. + * @param addr target address + * @param exception the relevant exception + * @return an exception to throw + */ + static IOException wrapException(InetSocketAddress addr, Exception exception) { + if (exception instanceof ConnectException) { + // connection refused; include the host:port in the error + return (ConnectException) new ConnectException( + "Call to " + addr + " failed on connection exception: " + exception).initCause(exception); + } else if (exception instanceof SocketTimeoutException) { + return (SocketTimeoutException) new SocketTimeoutException( + "Call to " + addr + " failed because " + exception).initCause(exception); + } else if (exception instanceof ConnectionClosingException) { + return (ConnectionClosingException) new ConnectionClosingException( + "Call to " + addr + " failed on local exception: " + exception).initCause(exception); + } else if (exception instanceof ServerTooBusyException) { + // we already have address in the exception message + return (IOException) exception; + } else if (exception instanceof DoNotRetryIOException) { + return (IOException) new DoNotRetryIOException( + "Call to " + addr + " failed on local exception: " + exception).initCause(exception); + } else { + return (IOException) new IOException( + "Call to " + addr + " failed on local exception: " + exception).initCause(exception); + } + } + + static void setCancelled(Call call) { + call.setException(new CallCancelledException("Call id=" + call.id + ", waitTime=" + + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout=" + + call.timeout)); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java index 7bb27e9eeba..de2c8de4bf4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java @@ -18,23 +18,21 @@ package org.apache.hadoop.hbase.ipc; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcController; + import java.io.IOException; -import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; - /** * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s * against the active master. An instance of this class may be obtained diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java new file mode 100644 index 00000000000..cde453fd3e5 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.util.concurrent.DefaultThreadFactory; + +import java.io.IOException; +import java.net.SocketAddress; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.MetricsConnection; +import org.apache.hadoop.hbase.util.Pair; + +/** + * Netty client for the requests and responses. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class NettyRpcClient extends AbstractRpcClient { + + final EventLoopGroup group; + + final Class channelClass; + + private final boolean shutdownGroupWhenClose; + + public NettyRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress, + MetricsConnection metrics) { + super(configuration, clusterId, localAddress, metrics); + Pair> groupAndChannelClass = NettyRpcClientConfigHelper + .getEventLoopConfig(conf); + if (groupAndChannelClass == null) { + // Use our own EventLoopGroup. + this.group = new NioEventLoopGroup(0, + new DefaultThreadFactory("IPC-NioEventLoopGroup", true, Thread.MAX_PRIORITY)); + this.channelClass = NioSocketChannel.class; + this.shutdownGroupWhenClose = true; + } else { + this.group = groupAndChannelClass.getFirst(); + this.channelClass = groupAndChannelClass.getSecond(); + this.shutdownGroupWhenClose = false; + } + } + + /** Used in test only. */ + NettyRpcClient(Configuration configuration) { + this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null); + } + + @Override + protected NettyRpcConnection createConnection(ConnectionId remoteId) throws IOException { + return new NettyRpcConnection(this, remoteId); + } + + @Override + protected void closeInternal() { + if (shutdownGroupWhenClose) { + group.shutdownGracefully(); + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java new file mode 100644 index 00000000000..a8af69c905e --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import com.google.common.base.Preconditions; + +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Pair; + +/** + * Helper class for passing config to {@link NettyRpcClient}. + *

+ * As hadoop Configuration can not pass an Object directly, we need to find a way to pass the + * EventLoopGroup to {@code AsyncRpcClient} if we want to use a single {@code EventLoopGroup} for + * the whole process. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class NettyRpcClientConfigHelper { + + public static final String EVENT_LOOP_CONFIG = "hbase.rpc.client.event-loop.config"; + + private static final String CONFIG_NAME = "global-event-loop"; + + private static final Map>> + EVENT_LOOP_CONFIG_MAP = new HashMap<>(); + + /** + * Set the EventLoopGroup and channel class for {@code AsyncRpcClient}. + */ + public static void setEventLoopConfig(Configuration conf, EventLoopGroup group, + Class channelClass) { + Preconditions.checkNotNull(group, "group is null"); + Preconditions.checkNotNull(channelClass, "channel class is null"); + conf.set(EVENT_LOOP_CONFIG, CONFIG_NAME); + EVENT_LOOP_CONFIG_MAP.put(CONFIG_NAME, + Pair.> newPair(group, channelClass)); + } + + /** + * The {@code AsyncRpcClient} will create its own {@code NioEventLoopGroup}. + */ + public static void createEventLoopPerClient(Configuration conf) { + conf.set(EVENT_LOOP_CONFIG, ""); + EVENT_LOOP_CONFIG_MAP.clear(); + } + + static Pair> getEventLoopConfig(Configuration conf) { + String name = conf.get(EVENT_LOOP_CONFIG); + if (name == null) { + return DefaultNettyEventLoopConfig.GROUP_AND_CHANNEL_CLASS; + } + if (StringUtils.isBlank(name)) { + return null; + } + return EVENT_LOOP_CONFIG_MAP.get(name); + } + +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java new file mode 100644 index 00000000000..9a90b09da3c --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java @@ -0,0 +1,293 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import static org.apache.hadoop.hbase.ipc.CallEvent.Type.CANCELLED; +import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT; +import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled; +import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE; + +import com.google.protobuf.RpcCallback; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufOutputStream; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.handler.timeout.IdleStateHandler; +import io.netty.util.ReferenceCountUtil; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.FutureListener; +import io.netty.util.concurrent.Promise; + +import java.io.IOException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler.BufferCallEvent; +import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; +import org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler; +import org.apache.hadoop.hbase.security.SaslChallengeDecoder; +import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.security.UserGroupInformation; + +/** + * RPC connection implementation based on netty. + *

+ * Most operations are executed in handlers. Netty handler is always executed in the same + * thread(EventLoop) so no lock is needed. + */ +@InterfaceAudience.Private +class NettyRpcConnection extends RpcConnection { + + private static final Log LOG = LogFactory.getLog(NettyRpcConnection.class); + + private static final ScheduledExecutorService RELOGIN_EXECUTOR = Executors + .newSingleThreadScheduledExecutor(Threads.newDaemonThreadFactory("Relogin")); + + private final NettyRpcClient rpcClient; + + private ByteBuf connectionHeaderPreamble; + + private ByteBuf connectionHeaderWithLength; + + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", + justification = "connect is also under lock as notifyOnCancel will call our action directly") + private Channel channel; + + NettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) throws IOException { + super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, + rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor); + this.rpcClient = rpcClient; + byte[] connectionHeaderPreamble = getConnectionHeaderPreamble(); + this.connectionHeaderPreamble = Unpooled.directBuffer(connectionHeaderPreamble.length) + .writeBytes(connectionHeaderPreamble); + ConnectionHeader header = getConnectionHeader(); + this.connectionHeaderWithLength = Unpooled.directBuffer(4 + header.getSerializedSize()); + this.connectionHeaderWithLength.writeInt(header.getSerializedSize()); + header.writeTo(new ByteBufOutputStream(this.connectionHeaderWithLength)); + } + + @Override + protected synchronized void callTimeout(Call call) { + if (channel != null) { + channel.pipeline().fireUserEventTriggered(new CallEvent(TIMEOUT, call)); + } + } + + @Override + public synchronized boolean isActive() { + return channel != null; + } + + private void shutdown0() { + if (channel != null) { + channel.close(); + channel = null; + } + } + + @Override + public synchronized void shutdown() { + shutdown0(); + } + + @Override + public synchronized void cleanupConnection() { + if (connectionHeaderPreamble != null) { + ReferenceCountUtil.safeRelease(connectionHeaderPreamble); + } + if (connectionHeaderWithLength != null) { + ReferenceCountUtil.safeRelease(connectionHeaderWithLength); + } + } + + private void established(Channel ch) { + ch.write(connectionHeaderWithLength.retainedDuplicate()); + ChannelPipeline p = ch.pipeline(); + String addBeforeHandler = p.context(BufferCallBeforeInitHandler.class).name(); + p.addBefore(addBeforeHandler, null, + new IdleStateHandler(0, rpcClient.minIdleTimeBeforeClose, 0, TimeUnit.MILLISECONDS)); + p.addBefore(addBeforeHandler, null, new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4)); + p.addBefore(addBeforeHandler, null, + new NettyRpcDuplexHandler(this, rpcClient.cellBlockBuilder, codec, compressor)); + p.fireUserEventTriggered(BufferCallEvent.success()); + } + + private boolean reloginInProgress; + + private void scheduleRelogin(Throwable error) { + if (error instanceof FallbackDisallowedException) { + return; + } + synchronized (this) { + if (reloginInProgress) { + return; + } + reloginInProgress = true; + RELOGIN_EXECUTOR.schedule(new Runnable() { + + @Override + public void run() { + try { + if (shouldAuthenticateOverKrb()) { + relogin(); + } + } catch (IOException e) { + LOG.warn("relogin failed", e); + } + synchronized (this) { + reloginInProgress = false; + } + } + }, ThreadLocalRandom.current().nextInt(reloginMaxBackoff), TimeUnit.MILLISECONDS); + } + } + + private void failInit(Channel ch, IOException e) { + synchronized (this) { + // fail all pending calls + ch.pipeline().fireUserEventTriggered(BufferCallEvent.fail(e)); + shutdown0(); + return; + } + } + + private void saslNegotiate(final Channel ch) { + UserGroupInformation ticket = getUGI(); + if (ticket == null) { + failInit(ch, new FatalConnectionException("ticket/user is null")); + return; + } + Promise saslPromise = ch.eventLoop().newPromise(); + ChannelHandler saslHandler; + try { + saslHandler = new NettyHBaseSaslRpcClientHandler(saslPromise, ticket, authMethod, token, + serverPrincipal, rpcClient.fallbackAllowed, this.rpcClient.conf.get( + "hbase.rpc.protection", QualityOfProtection.AUTHENTICATION.name().toLowerCase())); + } catch (IOException e) { + failInit(ch, e); + return; + } + ch.pipeline().addFirst(new SaslChallengeDecoder(), saslHandler); + saslPromise.addListener(new FutureListener() { + + @Override + public void operationComplete(Future future) throws Exception { + if (future.isSuccess()) { + ChannelPipeline p = ch.pipeline(); + p.remove(SaslChallengeDecoder.class); + p.remove(NettyHBaseSaslRpcClientHandler.class); + established(ch); + } else { + final Throwable error = future.cause(); + scheduleRelogin(error); + failInit(ch, toIOE(error)); + } + } + }); + } + + private void connect() { + if (LOG.isDebugEnabled()) { + LOG.debug("Connecting to " + remoteId.address); + } + + this.channel = new Bootstrap().group(rpcClient.group).channel(rpcClient.channelClass) + .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay()) + .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO) + .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr) + .remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() { + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + Channel ch = future.channel(); + if (!future.isSuccess()) { + failInit(ch, toIOE(future.cause())); + rpcClient.failedServers.addToFailedServers(remoteId.address); + return; + } + ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate()); + if (useSasl) { + saslNegotiate(ch); + } else { + established(ch); + } + } + }).channel(); + } + + @Override + public synchronized void sendRequest(final Call call, HBaseRpcController hrc) + throws IOException { + if (reloginInProgress) { + throw new IOException("Can not send request because relogin is in progress."); + } + hrc.notifyOnCancel(new RpcCallback() { + + @Override + public void run(Object parameter) { + setCancelled(call); + synchronized (this) { + if (channel != null) { + channel.pipeline().fireUserEventTriggered(new CallEvent(CANCELLED, call)); + } + } + } + }, new CancellationCallback() { + + @Override + public void run(boolean cancelled) throws IOException { + if (cancelled) { + setCancelled(call); + } else { + if (channel == null) { + connect(); + } + scheduleTimeoutTask(call); + channel.writeAndFlush(call).addListener(new ChannelFutureListener() { + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + // Fail the call if we failed to write it out. This usually because the channel is + // closed. This is needed because we may shutdown the channel inside event loop and + // there may still be some pending calls in the event loop queue after us. + if (!future.isSuccess()) { + call.setException(toIOE(future.cause())); + } + } + }); + } + } + }); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java new file mode 100644 index 00000000000..5faaede082a --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java @@ -0,0 +1,250 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import com.google.protobuf.Message; +import com.google.protobuf.Message.Builder; +import com.google.protobuf.TextFormat; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufInputStream; +import io.netty.buffer.ByteBufOutputStream; +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPromise; +import io.netty.handler.timeout.IdleStateEvent; +import io.netty.util.concurrent.PromiseCombiner; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.ipc.RemoteException; + +/** + * The netty rpc handler. + */ +@InterfaceAudience.Private +class NettyRpcDuplexHandler extends ChannelDuplexHandler { + + private static final Log LOG = LogFactory.getLog(NettyRpcDuplexHandler.class); + + private final NettyRpcConnection conn; + + private final CellBlockBuilder cellBlockBuilder; + + private final Codec codec; + + private final CompressionCodec compressor; + + private final Map id2Call = new HashMap(); + + public NettyRpcDuplexHandler(NettyRpcConnection conn, CellBlockBuilder cellBlockBuilder, + Codec codec, CompressionCodec compressor) { + this.conn = conn; + this.cellBlockBuilder = cellBlockBuilder; + this.codec = codec; + this.compressor = compressor; + + } + + private void writeRequest(ChannelHandlerContext ctx, Call call, ChannelPromise promise) + throws IOException { + id2Call.put(call.id, call); + ByteBuf cellBlock = cellBlockBuilder.buildCellBlock(codec, compressor, call.cells, ctx.alloc()); + CellBlockMeta cellBlockMeta; + if (cellBlock != null) { + CellBlockMeta.Builder cellBlockMetaBuilder = CellBlockMeta.newBuilder(); + cellBlockMetaBuilder.setLength(cellBlock.writerIndex()); + cellBlockMeta = cellBlockMetaBuilder.build(); + } else { + cellBlockMeta = null; + } + RequestHeader requestHeader = IPCUtil.buildRequestHeader(call, cellBlockMeta); + int sizeWithoutCellBlock = IPCUtil.getTotalSizeWhenWrittenDelimited(requestHeader, call.param); + int totalSize = cellBlock != null ? sizeWithoutCellBlock + cellBlock.writerIndex() + : sizeWithoutCellBlock; + ByteBuf buf = ctx.alloc().buffer(sizeWithoutCellBlock + 4); + buf.writeInt(totalSize); + ByteBufOutputStream bbos = new ByteBufOutputStream(buf); + requestHeader.writeDelimitedTo(bbos); + if (call.param != null) { + call.param.writeDelimitedTo(bbos); + } + if (cellBlock != null) { + ChannelPromise withoutCellBlockPromise = ctx.newPromise(); + ctx.write(buf, withoutCellBlockPromise); + ChannelPromise cellBlockPromise = ctx.newPromise(); + ctx.write(cellBlock, cellBlockPromise); + PromiseCombiner combiner = new PromiseCombiner(); + combiner.addAll(withoutCellBlockPromise, cellBlockPromise); + combiner.finish(promise); + } else { + ctx.write(buf, promise); + } + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + if (msg instanceof Call) { + writeRequest(ctx, (Call) msg, promise); + } else { + ctx.write(msg, promise); + } + } + + private void readResponse(ChannelHandlerContext ctx, ByteBuf buf) throws IOException { + int totalSize = buf.readInt(); + ByteBufInputStream in = new ByteBufInputStream(buf); + ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); + int id = responseHeader.getCallId(); + if (LOG.isTraceEnabled()) { + LOG.trace("got response header " + TextFormat.shortDebugString(responseHeader) + + ", totalSize: " + totalSize + " bytes"); + } + RemoteException remoteExc; + if (responseHeader.hasException()) { + ExceptionResponse exceptionResponse = responseHeader.getException(); + remoteExc = IPCUtil.createRemoteException(exceptionResponse); + if (IPCUtil.isFatalConnectionException(exceptionResponse)) { + // Here we will cleanup all calls so do not need to fall back, just return. + exceptionCaught(ctx, remoteExc); + return; + } + } else { + remoteExc = null; + } + Call call = id2Call.remove(id); + if (call == null) { + // So we got a response for which we have no corresponding 'call' here on the client-side. + // We probably timed out waiting, cleaned up all references, and now the server decides + // to return a response. There is nothing we can do w/ the response at this stage. Clean + // out the wire of the response so its out of the way and we can get other responses on + // this connection. + int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader); + int whatIsLeftToRead = totalSize - readSoFar; + if (LOG.isDebugEnabled()) { + LOG.debug("Unknown callId: " + id + ", skipping over this response of " + whatIsLeftToRead + + " bytes"); + } + return; + } + if (remoteExc != null) { + call.setException(remoteExc); + return; + } + Message value; + if (call.responseDefaultType != null) { + Builder builder = call.responseDefaultType.newBuilderForType(); + builder.mergeDelimitedFrom(in); + value = builder.build(); + } else { + value = null; + } + CellScanner cellBlockScanner; + if (responseHeader.hasCellBlockMeta()) { + int size = responseHeader.getCellBlockMeta().getLength(); + // Maybe we could read directly from the ByteBuf. + // The problem here is that we do not know when to release it. + byte[] cellBlock = new byte[size]; + buf.readBytes(cellBlock); + cellBlockScanner = cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock); + } else { + cellBlockScanner = null; + } + call.setResponse(value, cellBlockScanner); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof ByteBuf) { + ByteBuf buf = (ByteBuf) msg; + try { + readResponse(ctx, buf); + } finally { + buf.release(); + } + } else { + super.channelRead(ctx, msg); + } + } + + private void cleanupCalls(ChannelHandlerContext ctx, IOException error) { + for (Call call : id2Call.values()) { + call.setException(error); + } + id2Call.clear(); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + if (!id2Call.isEmpty()) { + cleanupCalls(ctx, new IOException("Connection closed")); + } + conn.shutdown(); + ctx.fireChannelInactive(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + if (!id2Call.isEmpty()) { + cleanupCalls(ctx, IPCUtil.toIOE(cause)); + } + conn.shutdown(); + } + + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + IdleStateEvent idleEvt = (IdleStateEvent) evt; + switch (idleEvt.state()) { + case WRITER_IDLE: + if (id2Call.isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("shutdown connection to " + conn.remoteId().address + + " because idle for a long time"); + } + // It may happen that there are still some pending calls in the event loop queue and + // they will get a closed channel exception. But this is not a big deal as it rarely + // rarely happens and the upper layer could retry immediately. + conn.shutdown(); + } + break; + default: + LOG.warn("Unrecognized idle state " + idleEvt.state()); + break; + } + } else if (evt instanceof CallEvent) { + // just remove the call for now until we add other call event other than timeout and cancel. + id2Call.remove(((CallEvent) evt).call.id); + } else { + ctx.fireUserEventTriggered(evt); + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java deleted file mode 100644 index 82634e50635..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java +++ /dev/null @@ -1,107 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import java.util.List; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.CellScannable; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; - -/** - * Optionally carries Cells across the proxy/service interface down into ipc. On its - * way out it optionally carries a set of result Cell data. We stick the Cells here when we want - * to avoid having to protobuf them. This class is used ferrying data across the proxy/protobuf - * service chasm. Used by client and server ipc'ing. - */ -@InterfaceAudience.Private -public class PayloadCarryingRpcController - extends TimeLimitedRpcController implements CellScannable { - - public static final int PRIORITY_UNSET = -1; - /** - * Priority to set on this request. Set it here in controller so available composing the - * request. This is the ordained way of setting priorities going forward. We will be - * undoing the old annotation-based mechanism. - */ - private int priority = PRIORITY_UNSET; - - /** - * They are optionally set on construction, cleared after we make the call, and then optionally - * set on response with the result. We use this lowest common denominator access to Cells because - * sometimes the scanner is backed by a List of Cells and other times, it is backed by an - * encoded block that implements CellScanner. - */ - private CellScanner cellScanner; - - public PayloadCarryingRpcController() { - this((CellScanner)null); - } - - public PayloadCarryingRpcController(final CellScanner cellScanner) { - this.cellScanner = cellScanner; - } - - public PayloadCarryingRpcController(final List cellIterables) { - this.cellScanner = cellIterables == null? null: CellUtil.createCellScanner(cellIterables); - } - - /** - * @return One-shot cell scanner (you cannot back it up and restart) - */ - @Override - public CellScanner cellScanner() { - return cellScanner; - } - - public void setCellScanner(final CellScanner cellScanner) { - this.cellScanner = cellScanner; - } - - /** - * @param priority Priority for this request; should fall roughly in the range - * {@link HConstants#NORMAL_QOS} to {@link HConstants#HIGH_QOS} - */ - public void setPriority(int priority) { - this.priority = priority; - } - - /** - * @param tn Set priority based off the table we are going against. - */ - public void setPriority(final TableName tn) { - this.priority = - (tn != null && tn.isSystemTable())? HConstants.SYSTEMTABLE_QOS: HConstants.NORMAL_QOS; - } - - /** - * @return The priority of this request - */ - public int getPriority() { - return priority; - } - - @Override public void reset() { - super.reset(); - priority = 0; - cellScanner = null; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java index 3ceecfc400a..0052423ff9b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java @@ -18,15 +18,16 @@ package org.apache.hadoop.hbase.ipc; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcController; + import java.io.IOException; -import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.RegionServerCallable; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; @@ -35,10 +36,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; import org.apache.hadoop.hbase.util.Bytes; -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; - /** * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s * against a given table region. An instance of this class may be obtained @@ -90,11 +87,10 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{ new RegionServerCallable(connection, table, row) { @Override public CoprocessorServiceResponse call(int callTimeout) throws Exception { - if (rpcController instanceof PayloadCarryingRpcController) { - ((PayloadCarryingRpcController) rpcController).setPriority(tableName); - } - if (rpcController instanceof TimeLimitedRpcController) { - ((TimeLimitedRpcController) rpcController).setCallTimeout(callTimeout); + if (rpcController instanceof HBaseRpcController) { + HBaseRpcController hrc = (HBaseRpcController) rpcController; + hrc.setPriority(tableName); + hrc.setCallTimeout(callTimeout); } byte[] regionName = getLocation().getRegionInfo().getRegionName(); return ProtobufUtil.execService(rpcController, getStub(), call, regionName); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java index 540e224bdfe..26a5739f200 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java @@ -18,40 +18,43 @@ package org.apache.hadoop.hbase.ipc; import com.google.protobuf.BlockingRpcChannel; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.security.User; +import com.google.protobuf.RpcChannel; import java.io.Closeable; import java.io.IOException; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.security.User; + /** * Interface for RpcClient implementations so ConnectionManager can handle it. */ -@InterfaceAudience.Private public interface RpcClient extends Closeable { - public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry"; - public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000; - public final static String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose"; - public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = +@InterfaceAudience.Private +public interface RpcClient extends Closeable { + String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry"; + int FAILED_SERVER_EXPIRY_DEFAULT = 2000; + String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose"; + String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "hbase.ipc.client.fallback-to-simple-auth-allowed"; - public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false; - public static final String SPECIFIC_WRITE_THREAD = "hbase.ipc.client.specificThreadForWriting"; - public static final String DEFAULT_CODEC_CLASS = "hbase.client.default.rpc.codec"; + boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false; + String SPECIFIC_WRITE_THREAD = "hbase.ipc.client.specificThreadForWriting"; + String DEFAULT_CODEC_CLASS = "hbase.client.default.rpc.codec"; - public final static String SOCKET_TIMEOUT_CONNECT = "hbase.ipc.client.socket.timeout.connect"; + String SOCKET_TIMEOUT_CONNECT = "hbase.ipc.client.socket.timeout.connect"; /** * How long we wait when we wait for an answer. It's not the operation time, it's the time * we wait when we start to receive an answer, when the remote write starts to send the data. */ - public final static String SOCKET_TIMEOUT_READ = "hbase.ipc.client.socket.timeout.read"; - public final static String SOCKET_TIMEOUT_WRITE = "hbase.ipc.client.socket.timeout.write"; - public final static int DEFAULT_SOCKET_TIMEOUT_CONNECT = 10000; // 10 seconds - public final static int DEFAULT_SOCKET_TIMEOUT_READ = 20000; // 20 seconds - public final static int DEFAULT_SOCKET_TIMEOUT_WRITE = 60000; // 60 seconds + String SOCKET_TIMEOUT_READ = "hbase.ipc.client.socket.timeout.read"; + String SOCKET_TIMEOUT_WRITE = "hbase.ipc.client.socket.timeout.write"; + int DEFAULT_SOCKET_TIMEOUT_CONNECT = 10000; // 10 seconds + int DEFAULT_SOCKET_TIMEOUT_READ = 20000; // 20 seconds + int DEFAULT_SOCKET_TIMEOUT_WRITE = 60000; // 60 seconds // Used by the server, for compatibility with old clients. // The client in 0.99+ does not ping the server. - final static int PING_CALL_ID = -1; + int PING_CALL_ID = -1; /** * Creates a "channel" that can be used by a blocking protobuf service. Useful setting up @@ -64,8 +67,21 @@ import java.io.IOException; * @return A blocking rpc channel that goes via this rpc client instance. * @throws IOException when channel could not be created */ - public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User user, - int rpcTimeout) throws IOException; + BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User user, int rpcTimeout) + throws IOException; + + /** + * Creates a "channel" that can be used by a protobuf service. Useful setting up + * protobuf stubs. + * + * @param sn server name describing location of server + * @param user which is to use the connection + * @param rpcTimeout default rpc operation timeout + * + * @return A rpc channel that goes via this rpc client instance. + */ + RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout) + throws IOException; /** * Interrupt the connections to the given server. This should be called if the server @@ -76,13 +92,14 @@ import java.io.IOException; * safe exception. * @param sn server location to cancel connections of */ - public void cancelConnections(ServerName sn); + void cancelConnections(ServerName sn); /** * Stop all threads related to this client. No further calls may be made * using this client. */ - @Override public void close(); + @Override + void close(); /** * @return true when this client uses a {@link org.apache.hadoop.hbase.codec.Codec} and so diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java index 8f45eb75da7..1f7a5a2cb1c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java @@ -18,13 +18,15 @@ package org.apache.hadoop.hbase.ipc; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; + +import java.net.SocketAddress; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.util.ReflectionUtils; -import java.net.SocketAddress; - /** * Factory to create a {@link org.apache.hadoop.hbase.ipc.RpcClient} */ @@ -33,6 +35,10 @@ public final class RpcClientFactory { public static final String CUSTOM_RPC_CLIENT_IMPL_CONF_KEY = "hbase.rpc.client.impl"; + private static final ImmutableMap DEPRECATED_NAME_MAPPING = ImmutableMap.of( + "org.apache.hadoop.hbase.ipc.RpcClientImpl", BlockingRpcClient.class.getName(), + "org.apache.hadoop.hbase.ipc.AsyncRpcClient", NettyRpcClient.class.getName()); + /** * Private Constructor */ @@ -58,6 +64,15 @@ public final class RpcClientFactory { return createClient(conf, clusterId, null, metrics); } + private static String getRpcClientClass(Configuration conf) { + String rpcClientClass = conf.get(CUSTOM_RPC_CLIENT_IMPL_CONF_KEY); + if (rpcClientClass == null) { + return BlockingRpcClient.class.getName(); + } + String mappedName = DEPRECATED_NAME_MAPPING.get(rpcClientClass); + return mappedName == null ? rpcClientClass : mappedName; + } + /** * Creates a new RpcClient by the class defined in the configuration or falls back to * RpcClientImpl @@ -69,14 +84,9 @@ public final class RpcClientFactory { */ public static RpcClient createClient(Configuration conf, String clusterId, SocketAddress localAddr, MetricsConnection metrics) { - String rpcClientClass = - conf.get(CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, - RpcClientImpl.class.getName()); - return ReflectionUtils.instantiateWithCustomCtor( - rpcClientClass, - new Class[] { Configuration.class, String.class, SocketAddress.class, - MetricsConnection.class }, - new Object[] { conf, clusterId, localAddr, metrics } - ); + String rpcClientClass = getRpcClientClass(conf); + return ReflectionUtils.instantiateWithCustomCtor(rpcClientClass, new Class[] { + Configuration.class, String.class, SocketAddress.class, MetricsConnection.class }, + new Object[] { conf, clusterId, localAddr, metrics }); } } \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java deleted file mode 100644 index 58e577f7219..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java +++ /dev/null @@ -1,1326 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.ipc; - -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; -import java.io.Closeable; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.InterruptedIOException; -import java.io.OutputStream; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.net.SocketAddress; -import java.net.SocketTimeoutException; -import java.net.UnknownHostException; -import java.nio.ByteBuffer; -import java.security.PrivilegedExceptionAction; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Locale; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import javax.net.SocketFactory; -import javax.security.sasl.SaslException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; -import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo; -import org.apache.hadoop.hbase.security.AuthMethod; -import org.apache.hadoop.hbase.security.HBaseSaslRpcClient; -import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; -import org.apache.hadoop.hbase.security.SecurityInfo; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.PoolMap; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.security.token.TokenSelector; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; - -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.Descriptors.MethodDescriptor; -import com.google.protobuf.Message; -import com.google.protobuf.Message.Builder; -import com.google.protobuf.RpcCallback; - -/** - * Does RPC against a cluster. Manages connections per regionserver in the cluster. - *

See HBaseServer - */ -@InterfaceAudience.Private -public class RpcClientImpl extends AbstractRpcClient { - private static final Log LOG = LogFactory.getLog(RpcClientImpl.class); - protected final AtomicInteger callIdCnt = new AtomicInteger(); - - protected final PoolMap connections; - - protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs - - protected final FailedServers failedServers; - - protected final SocketFactory socketFactory; // how to create sockets - - protected final static Map> tokenHandlers = - new HashMap>(); - static { - tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN, - new AuthenticationTokenSelector()); - } - - /** - * Creates a connection. Can be overridden by a subclass for testing. - * @param remoteId - the ConnectionId to use for the connection creation. - */ - protected Connection createConnection(ConnectionId remoteId, final Codec codec, - final CompressionCodec compressor) - throws IOException { - return new Connection(remoteId, codec, compressor); - } - - /** - * see {@link RpcClientImpl.Connection.CallSender} - */ - private static class CallFuture { - final Call call; - final int priority; - final Span span; - - // We will use this to stop the writer - final static CallFuture DEATH_PILL = new CallFuture(null, -1, null); - - CallFuture(Call call, int priority, Span span) { - this.call = call; - this.priority = priority; - this.span = span; - } - } - - /** Thread that reads responses and notifies callers. Each connection owns a - * socket connected to a remote address. Calls are multiplexed through this - * socket: responses may be delivered out of order. */ - protected class Connection extends Thread { - private ConnectionHeader header; // connection header - protected ConnectionId remoteId; - protected Socket socket = null; // connected socket - protected DataInputStream in; - protected DataOutputStream out; - private Object outLock = new Object(); - private InetSocketAddress server; // server ip:port - private String serverPrincipal; // server's krb5 principal name - private AuthMethod authMethod; // authentication method - private boolean useSasl; - private Token token; - private HBaseSaslRpcClient saslRpcClient; - private int reloginMaxBackoff; // max pause before relogin on sasl failure - private final Codec codec; - private final CompressionCodec compressor; - - // currently active calls - protected final ConcurrentSkipListMap calls = - new ConcurrentSkipListMap(); - - protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean(); - protected final CallSender callSender; - - - /** - * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), - * it gets into a java issue: an interruption during a write closes the socket/channel. - * A way to avoid this is to use a different thread for writing. This way, on interruptions, - * we either cancel the writes or ignore the answer if the write is already done, but we - * don't stop the write in the middle. - * This adds a thread per region server in the client, so it's kept as an option. - *

- * The implementation is simple: the client threads adds their call to the queue, and then - * wait for an answer. The CallSender blocks on the queue, and writes the calls one - * after the other. On interruption, the client cancels its call. The CallSender checks that - * the call has not been canceled before writing it. - *

- * When the connection closes, all the calls not yet sent are dismissed. The client thread - * is notified with an appropriate exception, as if the call was already sent but the answer - * not yet received. - *

- */ - private class CallSender extends Thread implements Closeable { - protected final BlockingQueue callsToWrite; - - - public CallFuture sendCall(Call call, int priority, Span span) - throws InterruptedException, IOException { - CallFuture cts = new CallFuture(call, priority, span); - if (!callsToWrite.offer(cts)) { - throw new IOException("Can't add the call " + call.id + - " to the write queue. callsToWrite.size()=" + callsToWrite.size()); - } - checkIsOpen(); // We check after the put, to be sure that the call we added won't stay - // in the list while the cleanup was already done. - return cts; - } - - @Override - public void close(){ - assert shouldCloseConnection.get(); - callsToWrite.offer(CallFuture.DEATH_PILL); - // We don't care if we can't add the death pill to the queue: the writer - // won't be blocked in the 'take', as its queue is full. - } - - CallSender(String name, Configuration conf) { - int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000); - callsToWrite = new ArrayBlockingQueue(queueSize); - setDaemon(true); - setName(name + " - writer"); - } - - public void remove(CallFuture cts){ - callsToWrite.remove(cts); - - // By removing the call from the expected call list, we make the list smaller, but - // it means as well that we don't know how many calls we cancelled. - calls.remove(cts.call.id); - cts.call.callComplete(); - } - - /** - * Reads the call from the queue, write them on the socket. - */ - @Override - public void run() { - while (!shouldCloseConnection.get()) { - CallFuture cts = null; - try { - cts = callsToWrite.take(); - } catch (InterruptedException e) { - markClosed(new InterruptedIOException()); - } - - if (cts == null || cts == CallFuture.DEATH_PILL) { - assert shouldCloseConnection.get(); - break; - } - - if (cts.call.done) { - continue; - } - - if (cts.call.checkAndSetTimeout()) { - continue; - } - - try { - Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span); - } catch (IOException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("call write error for call #" + cts.call.id - + ", message =" + e.getMessage()); - } - cts.call.setException(e); - markClosed(e); - } - } - - cleanup(); - } - - /** - * Cleans the call not yet sent when we finish. - */ - private void cleanup() { - assert shouldCloseConnection.get(); - - IOException ie = new ConnectionClosingException("Connection to " + server + " is closing."); - while (true) { - CallFuture cts = callsToWrite.poll(); - if (cts == null) { - break; - } - if (cts.call != null && !cts.call.done) { - cts.call.setException(ie); - } - } - } - } - - Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor) - throws IOException { - if (remoteId.getAddress().isUnresolved()) { - throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName()); - } - this.server = remoteId.getAddress(); - this.codec = codec; - this.compressor = compressor; - - UserGroupInformation ticket = remoteId.getTicket().getUGI(); - SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName()); - this.useSasl = userProvider.isHBaseSecurityEnabled(); - if (useSasl && securityInfo != null) { - AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind(); - if (tokenKind != null) { - TokenSelector tokenSelector = - tokenHandlers.get(tokenKind); - if (tokenSelector != null) { - token = tokenSelector.selectToken(new Text(clusterId), - ticket.getTokens()); - } else if (LOG.isDebugEnabled()) { - LOG.debug("No token selector found for type "+tokenKind); - } - } - String serverKey = securityInfo.getServerPrincipal(); - if (serverKey == null) { - throw new IOException( - "Can't obtain server Kerberos config key from SecurityInfo"); - } - serverPrincipal = SecurityUtil.getServerPrincipal( - conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT)); - if (LOG.isDebugEnabled()) { - LOG.debug("RPC Server Kerberos principal name for service=" - + remoteId.getServiceName() + " is " + serverPrincipal); - } - } - - if (!useSasl) { - authMethod = AuthMethod.SIMPLE; - } else if (token != null) { - authMethod = AuthMethod.DIGEST; - } else { - authMethod = AuthMethod.KERBEROS; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName + - ", sasl=" + useSasl); - } - reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000); - this.remoteId = remoteId; - - ConnectionHeader.Builder builder = ConnectionHeader.newBuilder(); - builder.setServiceName(remoteId.getServiceName()); - UserInformation userInfoPB = getUserInfo(ticket); - if (userInfoPB != null) { - builder.setUserInfo(userInfoPB); - } - if (this.codec != null) { - builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName()); - } - if (this.compressor != null) { - builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName()); - } - builder.setVersionInfo(ProtobufUtil.getVersionInfo()); - this.header = builder.build(); - - this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " + - remoteId.getAddress().toString() + - ((ticket==null)?" from an unknown user": (" from " - + ticket.getUserName()))); - this.setDaemon(true); - - if (conf.getBoolean(SPECIFIC_WRITE_THREAD, false)) { - callSender = new CallSender(getName(), conf); - callSender.start(); - } else { - callSender = null; - } - } - - private synchronized UserInformation getUserInfo(UserGroupInformation ugi) { - if (ugi == null || authMethod == AuthMethod.DIGEST) { - // Don't send user for token auth - return null; - } - UserInformation.Builder userInfoPB = UserInformation.newBuilder(); - if (authMethod == AuthMethod.KERBEROS) { - // Send effective user for Kerberos auth - userInfoPB.setEffectiveUser(ugi.getUserName()); - } else if (authMethod == AuthMethod.SIMPLE) { - //Send both effective user and real user for simple auth - userInfoPB.setEffectiveUser(ugi.getUserName()); - if (ugi.getRealUser() != null) { - userInfoPB.setRealUser(ugi.getRealUser().getUserName()); - } - } - return userInfoPB.build(); - } - - protected synchronized void setupConnection() throws IOException { - short ioFailures = 0; - short timeoutFailures = 0; - while (true) { - try { - this.socket = socketFactory.createSocket(); - this.socket.setTcpNoDelay(tcpNoDelay); - this.socket.setKeepAlive(tcpKeepAlive); - if (localAddr != null) { - this.socket.bind(localAddr); - } - NetUtils.connect(this.socket, remoteId.getAddress(), connectTO); - this.socket.setSoTimeout(readTO); - return; - } catch (SocketTimeoutException toe) { - /* The max number of retries is 45, - * which amounts to 20s*45 = 15 minutes retries. - */ - handleConnectionFailure(timeoutFailures++, maxRetries, toe); - } catch (IOException ie) { - handleConnectionFailure(ioFailures++, maxRetries, ie); - } - } - } - - protected synchronized void closeConnection() { - if (socket == null) { - return; - } - - // close the current connection - try { - if (socket.getOutputStream() != null) { - socket.getOutputStream().close(); - } - } catch (IOException ignored) { // Can happen if the socket is already closed - if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored); - } - try { - if (socket.getInputStream() != null) { - socket.getInputStream().close(); - } - } catch (IOException ignored) { // Can happen if the socket is already closed - if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored); - } - try { - if (socket.getChannel() != null) { - socket.getChannel().close(); - } - } catch (IOException ignored) { // Can happen if the socket is already closed - if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored); - } - try { - socket.close(); - } catch (IOException e) { - LOG.warn("Not able to close a socket", e); - } - - // set socket to null so that the next call to setupIOstreams - // can start the process of connect all over again. - socket = null; - } - - /** - * Handle connection failures - * - * If the current number of retries is equal to the max number of retries, - * stop retrying and throw the exception; Otherwise backoff N seconds and - * try connecting again. - * - * This Method is only called from inside setupIOstreams(), which is - * synchronized. Hence the sleep is synchronized; the locks will be retained. - * - * @param curRetries current number of retries - * @param maxRetries max number of retries allowed - * @param ioe failure reason - * @throws IOException if max number of retries is reached - */ - private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe) - throws IOException { - closeConnection(); - - // throw the exception if the maximum number of retries is reached - if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) { - throw ioe; - } - - // otherwise back off and retry - try { - Thread.sleep(failureSleep); - } catch (InterruptedException ie) { - ExceptionUtil.rethrowIfInterrupt(ie); - } - - LOG.info("Retrying connect to server: " + remoteId.getAddress() + - " after sleeping " + failureSleep + "ms. Already tried " + curRetries + - " time(s)."); - } - - /** - * @throws IOException if the connection is not open. - */ - private void checkIsOpen() throws IOException { - if (shouldCloseConnection.get()) { - throw new ConnectionClosingException(getName() + " is closing"); - } - } - - /* wait till someone signals us to start reading RPC response or - * it is idle too long, it is marked as to be closed, - * or the client is marked as not running. - * - * @return true if it is time to read a response; false otherwise. - */ - protected synchronized boolean waitForWork() throws InterruptedException { - // beware of the concurrent access to the calls list: we can add calls, but as well - // remove them. - long waitUntil = EnvironmentEdgeManager.currentTime() + minIdleTimeBeforeClose; - - while (true) { - if (shouldCloseConnection.get()) { - return false; - } - - if (!running.get()) { - markClosed(new IOException("stopped with " + calls.size() + " pending request(s)")); - return false; - } - - if (!calls.isEmpty()) { - // shouldCloseConnection can be set to true by a parallel thread here. The caller - // will need to check anyway. - return true; - } - - if (EnvironmentEdgeManager.currentTime() >= waitUntil) { - // Connection is idle. - // We expect the number of calls to be zero here, but actually someone can - // adds a call at the any moment, as there is no synchronization between this task - // and adding new calls. It's not a big issue, but it will get an exception. - markClosed(new IOException( - "idle connection closed with " + calls.size() + " pending request(s)")); - return false; - } - - wait(Math.min(minIdleTimeBeforeClose, 1000)); - } - } - - public InetSocketAddress getRemoteAddress() { - return remoteId.getAddress(); - } - - @Override - public void run() { - if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": starting, connections " + connections.size()); - } - - try { - while (waitForWork()) { // Wait here for work - read or close connection - readResponse(); - } - } catch (InterruptedException t) { - if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": interrupted while waiting for call responses"); - } - markClosed(ExceptionUtil.asInterrupt(t)); - } catch (Throwable t) { - if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": unexpected throwable while waiting for call responses", t); - } - markClosed(new IOException("Unexpected throwable while waiting call responses", t)); - } - - close(); - - if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": stopped, connections " + connections.size()); - } - } - - private synchronized void disposeSasl() { - if (saslRpcClient != null) { - try { - saslRpcClient.dispose(); - saslRpcClient = null; - } catch (IOException ioe) { - LOG.error("Error disposing of SASL client", ioe); - } - } - } - - private synchronized boolean shouldAuthenticateOverKrb() throws IOException { - UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); - UserGroupInformation currentUser = - UserGroupInformation.getCurrentUser(); - UserGroupInformation realUser = currentUser.getRealUser(); - return authMethod == AuthMethod.KERBEROS && - loginUser != null && - //Make sure user logged in using Kerberos either keytab or TGT - loginUser.hasKerberosCredentials() && - // relogin only in case it is the login user (e.g. JT) - // or superuser (like oozie). - (loginUser.equals(currentUser) || loginUser.equals(realUser)); - } - - private synchronized boolean setupSaslConnection(final InputStream in2, - final OutputStream out2) throws IOException { - saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed, - conf.get("hbase.rpc.protection", - QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT))); - return saslRpcClient.saslConnect(in2, out2); - } - - /** - * If multiple clients with the same principal try to connect - * to the same server at the same time, the server assumes a - * replay attack is in progress. This is a feature of kerberos. - * In order to work around this, what is done is that the client - * backs off randomly and tries to initiate the connection - * again. - * The other problem is to do with ticket expiry. To handle that, - * a relogin is attempted. - *

- * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} - * method. In case when the user doesn't have valid credentials, we don't - * need to retry (from cache or ticket). In such cases, it is prudent to - * throw a runtime exception when we receive a SaslException from the - * underlying authentication implementation, so there is no retry from - * other high level (for eg, HCM or HBaseAdmin). - *

- */ - private synchronized void handleSaslConnectionFailure( - final int currRetries, - final int maxRetries, final Exception ex, final Random rand, - final UserGroupInformation user) - throws IOException, InterruptedException{ - user.doAs(new PrivilegedExceptionAction() { - @Override - public Object run() throws IOException, InterruptedException { - closeConnection(); - if (shouldAuthenticateOverKrb()) { - if (currRetries < maxRetries) { - if (LOG.isDebugEnabled()) { - LOG.debug("Exception encountered while connecting to " + - "the server : " + ex); - } - //try re-login - if (UserGroupInformation.isLoginKeytabBased()) { - UserGroupInformation.getLoginUser().reloginFromKeytab(); - } else { - UserGroupInformation.getLoginUser().reloginFromTicketCache(); - } - disposeSasl(); - //have granularity of milliseconds - //we are sleeping with the Connection lock held but since this - //connection instance is being used for connecting to the server - //in question, it is okay - Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1)); - return null; - } else { - String msg = "Couldn't setup connection for " + - UserGroupInformation.getLoginUser().getUserName() + - " to " + serverPrincipal; - LOG.warn(msg, ex); - throw (IOException) new IOException(msg).initCause(ex); - } - } else { - LOG.warn("Exception encountered while connecting to " + - "the server : " + ex); - } - if (ex instanceof RemoteException) { - throw (RemoteException)ex; - } - if (ex instanceof SaslException) { - String msg = "SASL authentication failed." + - " The most likely cause is missing or invalid credentials." + - " Consider 'kinit'."; - LOG.fatal(msg, ex); - throw new RuntimeException(msg, ex); - } - throw new IOException(ex); - } - }); - } - - protected synchronized void setupIOstreams() throws IOException { - if (socket != null) { - // The connection is already available. Perfect. - return; - } - - if (shouldCloseConnection.get()){ - throw new ConnectionClosingException("This connection is closing"); - } - - if (failedServers.isFailedServer(remoteId.getAddress())) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not trying to connect to " + server + - " this server is in the failed servers list"); - } - IOException e = new FailedServerException( - "This server is in the failed servers list: " + server); - markClosed(e); - close(); - throw e; - } - - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Connecting to " + server); - } - short numRetries = 0; - final short MAX_RETRIES = 5; - Random rand = null; - while (true) { - setupConnection(); - InputStream inStream = NetUtils.getInputStream(socket); - // This creates a socket with a write timeout. This timeout cannot be changed. - OutputStream outStream = NetUtils.getOutputStream(socket, writeTO); - // Write out the preamble -- MAGIC, version, and auth to use. - writeConnectionHeaderPreamble(outStream); - if (useSasl) { - final InputStream in2 = inStream; - final OutputStream out2 = outStream; - UserGroupInformation ticket = remoteId.getTicket().getUGI(); - if (authMethod == AuthMethod.KERBEROS) { - if (ticket != null && ticket.getRealUser() != null) { - ticket = ticket.getRealUser(); - } - } - boolean continueSasl; - if (ticket == null) throw new FatalConnectionException("ticket/user is null"); - try { - continueSasl = ticket.doAs(new PrivilegedExceptionAction() { - @Override - public Boolean run() throws IOException { - return setupSaslConnection(in2, out2); - } - }); - } catch (Exception ex) { - ExceptionUtil.rethrowIfInterrupt(ex); - if (rand == null) { - rand = new Random(); - } - handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket); - continue; - } - if (continueSasl) { - // Sasl connect is successful. Let's set up Sasl i/o streams. - inStream = saslRpcClient.getInputStream(inStream); - outStream = saslRpcClient.getOutputStream(outStream); - } else { - // fall back to simple auth because server told us so. - authMethod = AuthMethod.SIMPLE; - useSasl = false; - } - } - this.in = new DataInputStream(new BufferedInputStream(inStream)); - synchronized (this.outLock) { - this.out = new DataOutputStream(new BufferedOutputStream(outStream)); - } - // Now write out the connection header - writeConnectionHeader(); - - // start the receiver thread after the socket connection has been set up - start(); - return; - } - } catch (Throwable t) { - IOException e = ExceptionUtil.asInterrupt(t); - if (e == null) { - failedServers.addToFailedServers(remoteId.address); - if (t instanceof LinkageError) { - // probably the hbase hadoop version does not match the running hadoop version - e = new DoNotRetryIOException(t); - } else if (t instanceof IOException) { - e = (IOException) t; - } else { - e = new IOException("Could not set up IO Streams to " + server, t); - } - } - markClosed(e); - close(); - throw e; - } - } - - /** - * Write the RPC header: - */ - private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException { - // Assemble the preamble up in a buffer first and then send it. Writing individual elements, - // they are getting sent across piecemeal according to wireshark and then server is messing - // up the reading on occasion (the passed in stream is not buffered yet). - - // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE - int rpcHeaderLen = HConstants.RPC_HEADER.length; - byte [] preamble = new byte [rpcHeaderLen + 2]; - System.arraycopy(HConstants.RPC_HEADER, 0, preamble, 0, rpcHeaderLen); - preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION; - synchronized (this) { - preamble[rpcHeaderLen + 1] = authMethod.code; - } - outStream.write(preamble); - outStream.flush(); - } - - /** - * Write the connection header. - */ - private synchronized void writeConnectionHeader() throws IOException { - synchronized (this.outLock) { - this.out.writeInt(this.header.getSerializedSize()); - this.header.writeTo(this.out); - this.out.flush(); - } - } - - /** Close the connection. */ - protected synchronized void close() { - if (!shouldCloseConnection.get()) { - LOG.error(getName() + ": the connection is not in the closed state"); - return; - } - - // release the resources - // first thing to do;take the connection out of the connection list - synchronized (connections) { - connections.removeValue(remoteId, this); - } - - // close the streams and therefore the socket - synchronized(this.outLock) { - if (this.out != null) { - IOUtils.closeStream(out); - this.out = null; - } - } - IOUtils.closeStream(in); - this.in = null; - if (this.socket != null) { - try { - this.socket.close(); - this.socket = null; - } catch (IOException e) { - LOG.error("Error while closing socket", e); - } - } - - disposeSasl(); - - // log the info - if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": closing ipc connection to " + server); - } - - cleanupCalls(true); - - if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": ipc connection to " + server + " closed"); - } - } - - protected void tracedWriteRequest(Call call, int priority, Span span) throws IOException { - TraceScope ts = Trace.startSpan("RpcClientImpl.tracedWriteRequest", span); - try { - writeRequest(call, priority, span); - } finally { - ts.close(); - } - } - - /** - * Initiates a call by sending the parameter to the remote server. - * Note: this is not called from the Connection thread, but by other - * threads. - * @see #readResponse() - */ - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC", - justification="Findbugs is misinterpreting locking missing fact that this.outLock is held") - private void writeRequest(Call call, final int priority, Span span) throws IOException { - RequestHeader.Builder builder = RequestHeader.newBuilder(); - builder.setCallId(call.id); - if (span != null) { - builder.setTraceInfo( - RPCTInfo.newBuilder().setParentId(span.getSpanId()).setTraceId(span.getTraceId())); - } - builder.setMethodName(call.md.getName()); - builder.setRequestParam(call.param != null); - ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells); - if (cellBlock != null) { - CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder(); - cellBlockBuilder.setLength(cellBlock.limit()); - builder.setCellBlockMeta(cellBlockBuilder.build()); - } - // Only pass priority if there is one set. - if (priority != PayloadCarryingRpcController.PRIORITY_UNSET) { - builder.setPriority(priority); - } - builder.setTimeout(call.timeout); - RequestHeader header = builder.build(); - - setupIOstreams(); - - // Now we're going to write the call. We take the lock, then check that the connection - // is still valid, and, if so we do the write to the socket. If the write fails, we don't - // know where we stand, we have to close the connection. - checkIsOpen(); - IOException writeException = null; - synchronized (this.outLock) { - if (Thread.interrupted()) throw new InterruptedIOException(); - - calls.put(call.id, call); // We put first as we don't want the connection to become idle. - checkIsOpen(); // Now we're checking that it didn't became idle in between. - - try { - call.callStats.setRequestSizeBytes(IPCUtil.write(this.out, header, call.param, - cellBlock)); - } catch (IOException e) { - // We set the value inside the synchronized block, this way the next in line - // won't even try to write. Otherwise we might miss a call in the calls map? - shouldCloseConnection.set(true); - writeException = e; - interrupt(); - } - } - - // call close outside of the synchronized (outLock) to prevent deadlock - HBASE-14474 - if (writeException != null) { - markClosed(writeException); - close(); - } - - // We added a call, and may be started the connection close. In both cases, we - // need to notify the reader. - doNotify(); - - // Now that we notified, we can rethrow the exception if any. Otherwise we're good. - if (writeException != null) throw writeException; - } - - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY", - justification="Presume notifyAll is because we are closing/shutting down") - private synchronized void doNotify() { - // Make a separate method so can do synchronize and add findbugs annotation; only one - // annotation at at time in source 1.7. - notifyAll(); // Findbugs: NN_NAKED_NOTIFY - } - - /* Receive a response. - * Because only one receiver, so no synchronization on in. - */ - protected void readResponse() { - if (shouldCloseConnection.get()) return; - Call call = null; - boolean expectedCall = false; - try { - // See HBaseServer.Call.setResponse for where we write out the response. - // Total size of the response. Unused. But have to read it in anyways. - int totalSize = in.readInt(); - - // Read the header - ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); - int id = responseHeader.getCallId(); - call = calls.remove(id); // call.done have to be set before leaving this method - expectedCall = (call != null && !call.done); - if (!expectedCall) { - // So we got a response for which we have no corresponding 'call' here on the client-side. - // We probably timed out waiting, cleaned up all references, and now the server decides - // to return a response. There is nothing we can do w/ the response at this stage. Clean - // out the wire of the response so its out of the way and we can get other responses on - // this connection. - int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader); - int whatIsLeftToRead = totalSize - readSoFar; - IOUtils.skipFully(in, whatIsLeftToRead); - if (call != null) { - call.callStats.setResponseSizeBytes(totalSize); - call.callStats.setCallTimeMs( - EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); - } - return; - } - if (responseHeader.hasException()) { - ExceptionResponse exceptionResponse = responseHeader.getException(); - RemoteException re = createRemoteException(exceptionResponse); - call.setException(re); - call.callStats.setResponseSizeBytes(totalSize); - call.callStats.setCallTimeMs( - EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); - if (isFatalConnectionException(exceptionResponse)) { - markClosed(re); - } - } else { - Message value = null; - if (call.responseDefaultType != null) { - Builder builder = call.responseDefaultType.newBuilderForType(); - ProtobufUtil.mergeDelimitedFrom(builder, in); - value = builder.build(); - } - CellScanner cellBlockScanner = null; - if (responseHeader.hasCellBlockMeta()) { - int size = responseHeader.getCellBlockMeta().getLength(); - byte [] cellBlock = new byte[size]; - IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); - cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock); - } - call.setResponse(value, cellBlockScanner); - call.callStats.setResponseSizeBytes(totalSize); - call.callStats.setCallTimeMs( - EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); - } - } catch (IOException e) { - if (expectedCall) call.setException(e); - if (e instanceof SocketTimeoutException) { - // Clean up open calls but don't treat this as a fatal condition, - // since we expect certain responses to not make it by the specified - // {@link ConnectionId#rpcTimeout}. - if (LOG.isTraceEnabled()) LOG.trace("ignored", e); - } else { - // Treat this as a fatal condition and close this connection - markClosed(e); - } - } finally { - cleanupCalls(false); - } - } - - /** - * @return True if the exception is a fatal connection exception. - */ - private boolean isFatalConnectionException(final ExceptionResponse e) { - return e.getExceptionClassName(). - equals(FatalConnectionException.class.getName()); - } - - /** - * @param e exception to be wrapped - * @return RemoteException made from passed e - */ - private RemoteException createRemoteException(final ExceptionResponse e) { - String innerExceptionClassName = e.getExceptionClassName(); - boolean doNotRetry = e.getDoNotRetry(); - return e.hasHostname()? - // If a hostname then add it to the RemoteWithExtrasException - new RemoteWithExtrasException(innerExceptionClassName, - e.getStackTrace(), e.getHostname(), e.getPort(), doNotRetry): - new RemoteWithExtrasException(innerExceptionClassName, - e.getStackTrace(), doNotRetry); - } - - protected synchronized boolean markClosed(IOException e) { - if (e == null) throw new NullPointerException(); - - boolean ret = shouldCloseConnection.compareAndSet(false, true); - if (ret) { - if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": marking at should close, reason: " + e.getMessage()); - } - if (callSender != null) { - callSender.close(); - } - notifyAll(); - } - return ret; - } - - - /** - * Cleanup the calls older than a given timeout, in milli seconds. - * @param allCalls true for all calls, false for only the calls in timeout - */ - protected synchronized void cleanupCalls(boolean allCalls) { - Iterator> itor = calls.entrySet().iterator(); - while (itor.hasNext()) { - Call c = itor.next().getValue(); - if (c.done) { - // To catch the calls without timeout that were cancelled. - itor.remove(); - } else if (allCalls) { - long waitTime = EnvironmentEdgeManager.currentTime() - c.getStartTime(); - IOException ie = new ConnectionClosingException("Connection to " + getRemoteAddress() - + " is closing. Call id=" + c.id + ", waitTime=" + waitTime); - c.setException(ie); - itor.remove(); - } else if (c.checkAndSetTimeout()) { - itor.remove(); - } else { - // We expect the call to be ordered by timeout. It may not be the case, but stopping - // at the first valid call allows to be sure that we still have something to do without - // spending too much time by reading the full list. - break; - } - } - } - } - - /** - * Used in test only. Construct an IPC cluster client whose values are of the - * {@link Message} class. - * @param conf configuration - * @param clusterId the cluster id - * @param factory socket factory - */ - @VisibleForTesting - RpcClientImpl(Configuration conf, String clusterId, SocketFactory factory) { - this(conf, clusterId, factory, null, null); - } - - /** - * Construct an IPC cluster client whose values are of the {@link Message} class. - * @param conf configuration - * @param clusterId the cluster id - * @param factory socket factory - * @param localAddr client socket bind address - * @param metrics the connection metrics - */ - RpcClientImpl(Configuration conf, String clusterId, SocketFactory factory, - SocketAddress localAddr, MetricsConnection metrics) { - super(conf, clusterId, localAddr, metrics); - - this.socketFactory = factory; - this.connections = new PoolMap(getPoolType(conf), getPoolSize(conf)); - this.failedServers = new FailedServers(conf); - } - - /** - * Used in test only. Construct an IPC client for the cluster {@code clusterId} with - * the default SocketFactory - */ - @VisibleForTesting - RpcClientImpl(Configuration conf, String clusterId) { - this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null, null); - } - - /** - * Construct an IPC client for the cluster {@code clusterId} with the default SocketFactory - * - * This method is called with reflection by the RpcClientFactory to create an instance - * - * @param conf configuration - * @param clusterId the cluster id - * @param localAddr client socket bind address. - * @param metrics the connection metrics - */ - public RpcClientImpl(Configuration conf, String clusterId, SocketAddress localAddr, - MetricsConnection metrics) { - this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr, metrics); - } - - /** Stop all threads related to this client. No further calls may be made - * using this client. */ - @Override - public void close() { - if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client"); - if (!running.compareAndSet(true, false)) return; - - Set connsToClose = null; - // wake up all connections - synchronized (connections) { - for (Connection conn : connections.values()) { - conn.interrupt(); - if (conn.callSender != null) { - conn.callSender.interrupt(); - } - - // In case the CallSender did not setupIOStreams() yet, the Connection may not be started - // at all (if CallSender has a cancelled Call it can happen). See HBASE-13851 - if (!conn.isAlive()) { - if (connsToClose == null) { - connsToClose = new HashSet(); - } - connsToClose.add(conn); - } - } - } - if (connsToClose != null) { - for (Connection conn : connsToClose) { - conn.markClosed(new InterruptedIOException("RpcClient is closing")); - conn.close(); - } - } - // wait until all connections are closed - while (!connections.isEmpty()) { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - LOG.info("Interrupted while stopping the client. We still have " + connections.size() + - " connections."); - Thread.currentThread().interrupt(); - return; - } - } - } - - /** Make a call, passing param, to the IPC server running at - * address which is servicing the protocol protocol, - * with the ticket credentials, returning the value. - * Throws exceptions if there are network problems or if the remote code - * threw an exception. - * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. - * {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a - * new Connection each time. - * @return A pair with the Message response and the Cell data (if any). - * @throws InterruptedException - * @throws IOException - */ - @Override - protected Pair call(PayloadCarryingRpcController pcrc, MethodDescriptor md, - Message param, Message returnType, User ticket, InetSocketAddress addr, - MetricsConnection.CallStats callStats) - throws IOException, InterruptedException { - if (pcrc == null) { - pcrc = new PayloadCarryingRpcController(); - } - CellScanner cells = pcrc.cellScanner(); - - final Call call = new Call(this.callIdCnt.getAndIncrement(), md, param, cells, returnType, - pcrc.getCallTimeout(), MetricsConnection.newCallStats()); - - final Connection connection = getConnection(ticket, call, addr); - - final CallFuture cts; - if (connection.callSender != null) { - cts = connection.callSender.sendCall(call, pcrc.getPriority(), Trace.currentSpan()); - pcrc.notifyOnCancel(new RpcCallback() { - @Override - public void run(Object parameter) { - connection.callSender.remove(cts); - } - }); - if (pcrc.isCanceled()) { - // To finish if the call was cancelled before we set the notification (race condition) - call.callComplete(); - return new Pair(call.response, call.cells); - } - } else { - cts = null; - connection.tracedWriteRequest(call, pcrc.getPriority(), Trace.currentSpan()); - } - - while (!call.done) { - if (call.checkAndSetTimeout()) { - if (cts != null) connection.callSender.remove(cts); - break; - } - if (connection.shouldCloseConnection.get()) { - throw new ConnectionClosingException("Call id=" + call.id + - " on server " + addr + " aborted: connection is closing"); - } - try { - synchronized (call) { - if (call.done) break; - call.wait(Math.min(call.remainingTime(), 1000) + 1); - } - } catch (InterruptedException e) { - call.setException(new InterruptedIOException()); - if (cts != null) connection.callSender.remove(cts); - throw e; - } - } - - if (call.error != null) { - if (call.error instanceof RemoteException) { - call.error.fillInStackTrace(); - throw call.error; - } - // local exception - throw wrapException(addr, call.error); - } - - return new Pair(call.response, call.cells); - } - - - /** - * Interrupt the connections to the given ip:port server. This should be called if the server - * is known as actually dead. This will not prevent current operation to be retried, and, - * depending on their own behavior, they may retry on the same server. This can be a feature, - * for example at startup. In any case, they're likely to get connection refused (if the - * process died) or no route to host: i.e. their next retries should be faster and with a - * safe exception. - */ - @Override - public void cancelConnections(ServerName sn) { - synchronized (connections) { - for (Connection connection : connections.values()) { - if (connection.isAlive() && - connection.getRemoteAddress().getPort() == sn.getPort() && - connection.getRemoteAddress().getHostName().equals(sn.getHostname())) { - LOG.info("The server on " + sn.toString() + - " is dead - stopping the connection " + connection.remoteId); - connection.interrupt(); // We're interrupting a Reader. It means we want it to finish. - // This will close the connection as well. - } - } - } - } - - /** - * Get a connection from the pool, or create a new one and add it to the - * pool. Connections to a given host/port are reused. - */ - protected Connection getConnection(User ticket, Call call, InetSocketAddress addr) - throws IOException { - if (!running.get()) throw new StoppedRpcClientException(); - Connection connection; - ConnectionId remoteId = - new ConnectionId(ticket, call.md.getService().getName(), addr); - synchronized (connections) { - connection = connections.get(remoteId); - if (connection == null) { - connection = createConnection(remoteId, this.codec, this.compressor); - connections.put(remoteId, connection); - } - } - - return connection; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java new file mode 100644 index 00000000000..5e9e97e219d --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java @@ -0,0 +1,260 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import io.netty.util.HashedWheelTimer; +import io.netty.util.Timeout; +import io.netty.util.TimerTask; + +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; +import org.apache.hadoop.hbase.security.AuthMethod; +import org.apache.hadoop.hbase.security.SecurityInfo; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.security.token.TokenSelector; + +/** + * Base class for ipc connection. + */ +@InterfaceAudience.Private +abstract class RpcConnection { + + private static final Log LOG = LogFactory.getLog(RpcConnection.class); + + protected final ConnectionId remoteId; + + protected final AuthMethod authMethod; + + protected final boolean useSasl; + + protected final Token token; + + protected final String serverPrincipal; // server's krb5 principal name + + protected final int reloginMaxBackoff; // max pause before relogin on sasl failure + + protected final Codec codec; + + protected final CompressionCodec compressor; + + protected final HashedWheelTimer timeoutTimer; + + // the last time we were picked up from connection pool. + protected long lastTouched; + + protected RpcConnection(Configuration conf, HashedWheelTimer timeoutTimer, ConnectionId remoteId, + String clusterId, boolean isSecurityEnabled, Codec codec, CompressionCodec compressor) + throws IOException { + if (remoteId.getAddress().isUnresolved()) { + throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName()); + } + this.timeoutTimer = timeoutTimer; + this.codec = codec; + this.compressor = compressor; + + UserGroupInformation ticket = remoteId.getTicket().getUGI(); + SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName()); + this.useSasl = isSecurityEnabled; + Token token = null; + String serverPrincipal = null; + if (useSasl && securityInfo != null) { + AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind(); + if (tokenKind != null) { + TokenSelector tokenSelector = AbstractRpcClient.TOKEN_HANDLERS + .get(tokenKind); + if (tokenSelector != null) { + token = tokenSelector.selectToken(new Text(clusterId), ticket.getTokens()); + } else if (LOG.isDebugEnabled()) { + LOG.debug("No token selector found for type " + tokenKind); + } + } + String serverKey = securityInfo.getServerPrincipal(); + if (serverKey == null) { + throw new IOException("Can't obtain server Kerberos config key from SecurityInfo"); + } + serverPrincipal = SecurityUtil.getServerPrincipal(conf.get(serverKey), + remoteId.address.getAddress().getCanonicalHostName().toLowerCase()); + if (LOG.isDebugEnabled()) { + LOG.debug("RPC Server Kerberos principal name for service=" + remoteId.getServiceName() + + " is " + serverPrincipal); + } + } + this.token = token; + this.serverPrincipal = serverPrincipal; + if (!useSasl) { + authMethod = AuthMethod.SIMPLE; + } else if (token != null) { + authMethod = AuthMethod.DIGEST; + } else { + authMethod = AuthMethod.KERBEROS; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName + + ", sasl=" + useSasl); + } + reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000); + this.remoteId = remoteId; + } + + private UserInformation getUserInfo(UserGroupInformation ugi) { + if (ugi == null || authMethod == AuthMethod.DIGEST) { + // Don't send user for token auth + return null; + } + UserInformation.Builder userInfoPB = UserInformation.newBuilder(); + if (authMethod == AuthMethod.KERBEROS) { + // Send effective user for Kerberos auth + userInfoPB.setEffectiveUser(ugi.getUserName()); + } else if (authMethod == AuthMethod.SIMPLE) { + // Send both effective user and real user for simple auth + userInfoPB.setEffectiveUser(ugi.getUserName()); + if (ugi.getRealUser() != null) { + userInfoPB.setRealUser(ugi.getRealUser().getUserName()); + } + } + return userInfoPB.build(); + } + + protected UserGroupInformation getUGI() { + UserGroupInformation ticket = remoteId.getTicket().getUGI(); + if (authMethod == AuthMethod.KERBEROS) { + if (ticket != null && ticket.getRealUser() != null) { + ticket = ticket.getRealUser(); + } + } + return ticket; + } + + protected boolean shouldAuthenticateOverKrb() throws IOException { + UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); + UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + UserGroupInformation realUser = currentUser.getRealUser(); + return authMethod == AuthMethod.KERBEROS && loginUser != null && + // Make sure user logged in using Kerberos either keytab or TGT + loginUser.hasKerberosCredentials() && + // relogin only in case it is the login user (e.g. JT) + // or superuser (like oozie). + (loginUser.equals(currentUser) || loginUser.equals(realUser)); + } + + protected void relogin() throws IOException { + if (UserGroupInformation.isLoginKeytabBased()) { + UserGroupInformation.getLoginUser().reloginFromKeytab(); + } else { + UserGroupInformation.getLoginUser().reloginFromTicketCache(); + } + } + + protected void scheduleTimeoutTask(final Call call) { + if (call.timeout > 0) { + call.timeoutTask = timeoutTimer.newTimeout(new TimerTask() { + + @Override + public void run(Timeout timeout) throws Exception { + call.setTimeout(new CallTimeoutException("Call id=" + call.id + ", waitTime=" + + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimetout=" + + call.timeout)); + callTimeout(call); + } + }, call.timeout, TimeUnit.MILLISECONDS); + } + } + + protected byte[] getConnectionHeaderPreamble() { + // Assemble the preamble up in a buffer first and then send it. Writing individual elements, + // they are getting sent across piecemeal according to wireshark and then server is messing + // up the reading on occasion (the passed in stream is not buffered yet). + + // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE + int rpcHeaderLen = HConstants.RPC_HEADER.length; + byte[] preamble = new byte[rpcHeaderLen + 2]; + System.arraycopy(HConstants.RPC_HEADER, 0, preamble, 0, rpcHeaderLen); + preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION; + synchronized (this) { + preamble[rpcHeaderLen + 1] = authMethod.code; + } + return preamble; + } + + protected ConnectionHeader getConnectionHeader() { + ConnectionHeader.Builder builder = ConnectionHeader.newBuilder(); + builder.setServiceName(remoteId.getServiceName()); + UserInformation userInfoPB; + if ((userInfoPB = getUserInfo(remoteId.ticket.getUGI())) != null) { + builder.setUserInfo(userInfoPB); + } + if (this.codec != null) { + builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName()); + } + if (this.compressor != null) { + builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName()); + } + builder.setVersionInfo(ProtobufUtil.getVersionInfo()); + return builder.build(); + } + + protected abstract void callTimeout(Call call); + + public ConnectionId remoteId() { + return remoteId; + } + + public long getLastTouched() { + return lastTouched; + } + + public void setLastTouched(long lastTouched) { + this.lastTouched = lastTouched; + } + + /** + * Tell the idle connection sweeper whether we could be swept. + */ + public abstract boolean isActive(); + + /** + * Just close connection. Do not need to remove from connection pool. + */ + public abstract void shutdown(); + + public abstract void sendRequest(Call call, HBaseRpcController hrc) throws IOException; + + /** + * Does the clean up work after the connection is removed from the connection pool + */ + public abstract void cleanupConnection(); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java index faeca8de45d..ca4073d3baa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ReflectionUtils; /** - * Factory to create a {@link PayloadCarryingRpcController} + * Factory to create a {@link HBaseRpcController}. */ @InterfaceAudience.Private public class RpcControllerFactory { @@ -46,16 +46,16 @@ public class RpcControllerFactory { this.conf = conf; } - public PayloadCarryingRpcController newController() { - return new PayloadCarryingRpcController(); + public HBaseRpcController newController() { + return new HBaseRpcControllerImpl(); } - public PayloadCarryingRpcController newController(final CellScanner cellScanner) { - return new PayloadCarryingRpcController(cellScanner); + public HBaseRpcController newController(final CellScanner cellScanner) { + return new HBaseRpcControllerImpl(cellScanner); } - public PayloadCarryingRpcController newController(final List cellIterables) { - return new PayloadCarryingRpcController(cellIterables); + public HBaseRpcController newController(final List cellIterables) { + return new HBaseRpcControllerImpl(cellIterables); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java deleted file mode 100644 index de502cb4332..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java +++ /dev/null @@ -1,142 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.ipc; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicReference; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; - -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; - -@InterfaceAudience.Private -public class TimeLimitedRpcController implements RpcController { - - /** - * The time, in ms before the call should expire. - */ - protected volatile Integer callTimeout; - protected volatile boolean cancelled = false; - protected final AtomicReference> cancellationCb = - new AtomicReference>(null); - - protected final AtomicReference> failureCb = - new AtomicReference>(null); - - private IOException exception; - - public int getCallTimeout() { - if (callTimeout != null) { - return callTimeout; - } else { - return 0; - } - } - - public void setCallTimeout(int callTimeout) { - this.callTimeout = callTimeout; - } - - public boolean hasCallTimeout(){ - return callTimeout != null; - } - - @Override - public String errorText() { - if (exception != null) { - return exception.getMessage(); - } else { - return null; - } - } - - /** - * For use in async rpc clients - * @return true if failed - */ - @Override - public boolean failed() { - return this.exception != null; - } - - @Override - public boolean isCanceled() { - return cancelled; - } - - @Override - public void notifyOnCancel(RpcCallback cancellationCb) { - this.cancellationCb.set(cancellationCb); - if (this.cancelled) { - cancellationCb.run(null); - } - } - - /** - * Notify a callback on error. - * For use in async rpc clients - * - * @param failureCb the callback to call on error - */ - public void notifyOnFail(RpcCallback failureCb) { - this.failureCb.set(failureCb); - if (this.exception != null) { - failureCb.run(this.exception); - } - } - - @Override - public void reset() { - exception = null; - cancelled = false; - failureCb.set(null); - cancellationCb.set(null); - callTimeout = null; - } - - @Override - public void setFailed(String reason) { - this.exception = new IOException(reason); - if (this.failureCb.get() != null) { - this.failureCb.get().run(this.exception); - } - } - - /** - * Set failed with an exception to pass on. - * For use in async rpc clients - * - * @param e exception to set with - */ - public void setFailed(IOException e) { - this.exception = e; - if (this.failureCb.get() != null) { - this.failureCb.get().run(this.exception); - } - } - - @Override - public void startCancel() { - cancelled = true; - if (cancellationCb.get() != null) { - cancellationCb.get().run(null); - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java new file mode 100644 index 00000000000..cc0114ba654 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java @@ -0,0 +1,197 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security; + +import com.google.common.annotations.VisibleForTesting; + +import java.io.IOException; +import java.util.Map; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.RealmChoiceCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; + +/** + * A utility class that encapsulates SASL logic for RPC client. Copied from + * org.apache.hadoop.security + */ +@InterfaceAudience.Private +public abstract class AbstractHBaseSaslRpcClient { + + private static final Log LOG = LogFactory.getLog(AbstractHBaseSaslRpcClient.class); + + private static final byte[] EMPTY_TOKEN = new byte[0]; + + protected final SaslClient saslClient; + + protected final boolean fallbackAllowed; + + protected final Map saslProps; + + /** + * Create a HBaseSaslRpcClient for an authentication method + * @param method the requested authentication method + * @param token token to use if needed by the authentication method + * @param serverPrincipal the server principal that we are trying to set the connection up to + * @param fallbackAllowed does the client allow fallback to simple authentication + * @throws IOException + */ + protected AbstractHBaseSaslRpcClient(AuthMethod method, Token token, + String serverPrincipal, boolean fallbackAllowed) throws IOException { + this(method, token, serverPrincipal, fallbackAllowed, "authentication"); + } + + /** + * Create a HBaseSaslRpcClient for an authentication method + * @param method the requested authentication method + * @param token token to use if needed by the authentication method + * @param serverPrincipal the server principal that we are trying to set the connection up to + * @param fallbackAllowed does the client allow fallback to simple authentication + * @param rpcProtection the protection level ("authentication", "integrity" or "privacy") + * @throws IOException + */ + protected AbstractHBaseSaslRpcClient(AuthMethod method, Token token, + String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException { + this.fallbackAllowed = fallbackAllowed; + saslProps = SaslUtil.initSaslProperties(rpcProtection); + switch (method) { + case DIGEST: + if (LOG.isDebugEnabled()) LOG.debug("Creating SASL " + AuthMethod.DIGEST.getMechanismName() + + " client to authenticate to service at " + token.getService()); + saslClient = createDigestSaslClient(new String[] { AuthMethod.DIGEST.getMechanismName() }, + SaslUtil.SASL_DEFAULT_REALM, new SaslClientCallbackHandler(token)); + break; + case KERBEROS: + if (LOG.isDebugEnabled()) { + LOG.debug("Creating SASL " + AuthMethod.KERBEROS.getMechanismName() + + " client. Server's Kerberos principal name is " + serverPrincipal); + } + if (serverPrincipal == null || serverPrincipal.length() == 0) { + throw new IOException("Failed to specify server's Kerberos principal name"); + } + String[] names = SaslUtil.splitKerberosName(serverPrincipal); + if (names.length != 3) { + throw new IOException( + "Kerberos principal does not have the expected format: " + serverPrincipal); + } + saslClient = createKerberosSaslClient( + new String[] { AuthMethod.KERBEROS.getMechanismName() }, names[0], names[1]); + break; + default: + throw new IOException("Unknown authentication method " + method); + } + if (saslClient == null) { + throw new IOException("Unable to find SASL client implementation"); + } + } + + protected SaslClient createDigestSaslClient(String[] mechanismNames, String saslDefaultRealm, + CallbackHandler saslClientCallbackHandler) throws IOException { + return Sasl.createSaslClient(mechanismNames, null, null, saslDefaultRealm, saslProps, + saslClientCallbackHandler); + } + + protected SaslClient createKerberosSaslClient(String[] mechanismNames, String userFirstPart, + String userSecondPart) throws IOException { + return Sasl.createSaslClient(mechanismNames, null, userFirstPart, userSecondPart, saslProps, + null); + } + + public byte[] getInitialResponse() throws SaslException { + if (saslClient.hasInitialResponse()) { + return saslClient.evaluateChallenge(EMPTY_TOKEN); + } else { + return EMPTY_TOKEN; + } + } + + public boolean isComplete() { + return saslClient.isComplete(); + } + + public byte[] evaluateChallenge(byte[] challenge) throws SaslException { + return saslClient.evaluateChallenge(challenge); + } + + /** Release resources used by wrapped saslClient */ + public void dispose() { + SaslUtil.safeDispose(saslClient); + } + + @VisibleForTesting + static class SaslClientCallbackHandler implements CallbackHandler { + private final String userName; + private final char[] userPassword; + + public SaslClientCallbackHandler(Token token) { + this.userName = SaslUtil.encodeIdentifier(token.getIdentifier()); + this.userPassword = SaslUtil.encodePassword(token.getPassword()); + } + + @Override + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + NameCallback nc = null; + PasswordCallback pc = null; + RealmCallback rc = null; + for (Callback callback : callbacks) { + if (callback instanceof RealmChoiceCallback) { + continue; + } else if (callback instanceof NameCallback) { + nc = (NameCallback) callback; + } else if (callback instanceof PasswordCallback) { + pc = (PasswordCallback) callback; + } else if (callback instanceof RealmCallback) { + rc = (RealmCallback) callback; + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL client callback"); + } + } + if (nc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("SASL client callback: setting username: " + userName); + } + nc.setName(userName); + } + if (pc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("SASL client callback: setting userPassword"); + } + pc.setPassword(userPassword); + } + if (rc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("SASL client callback: setting realm: " + rc.getDefaultText()); + } + rc.setText(rc.getDefaultText()); + } + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java index bb6763ff120..3d88115cb01 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java @@ -18,6 +18,17 @@ package org.apache.hadoop.hbase.security; +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -28,123 +39,23 @@ import org.apache.hadoop.security.SaslOutputStream; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.sasl.RealmCallback; -import javax.security.sasl.RealmChoiceCallback; -import javax.security.sasl.Sasl; -import javax.security.sasl.SaslClient; -import javax.security.sasl.SaslException; - -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - -import com.google.common.annotations.VisibleForTesting; - /** - * A utility class that encapsulates SASL logic for RPC client. - * Copied from org.apache.hadoop.security + * A utility class that encapsulates SASL logic for RPC client. Copied from + * org.apache.hadoop.security */ @InterfaceAudience.Private -public class HBaseSaslRpcClient { +public class HBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { + private static final Log LOG = LogFactory.getLog(HBaseSaslRpcClient.class); - private final SaslClient saslClient; - private final boolean fallbackAllowed; - /** - * Create a HBaseSaslRpcClient for an authentication method - * - * @param method - * the requested authentication method - * @param token - * token to use if needed by the authentication method - * @param serverPrincipal - * the server principal that we are trying to set the connection up to - * @param fallbackAllowed - * does the client allow fallback to simple authentication - * @throws IOException - */ - public HBaseSaslRpcClient(AuthMethod method, - Token token, String serverPrincipal, boolean fallbackAllowed) - throws IOException { - this(method, token, serverPrincipal, fallbackAllowed, "authentication"); - } - /** - * Create a HBaseSaslRpcClient for an authentication method - * - * @param method - * the requested authentication method - * @param token - * token to use if needed by the authentication method - * @param serverPrincipal - * the server principal that we are trying to set the connection up to - * @param fallbackAllowed - * does the client allow fallback to simple authentication - * @param rpcProtection - * the protection level ("authentication", "integrity" or "privacy") - * @throws IOException - */ - public HBaseSaslRpcClient(AuthMethod method, - Token token, String serverPrincipal, boolean fallbackAllowed, - String rpcProtection) throws IOException { - this.fallbackAllowed = fallbackAllowed; - SaslUtil.initSaslProperties(rpcProtection); - switch (method) { - case DIGEST: - if (LOG.isDebugEnabled()) - LOG.debug("Creating SASL " + AuthMethod.DIGEST.getMechanismName() - + " client to authenticate to service at " + token.getService()); - saslClient = createDigestSaslClient( - new String[] { AuthMethod.DIGEST.getMechanismName() }, - SaslUtil.SASL_DEFAULT_REALM, new SaslClientCallbackHandler(token)); - break; - case KERBEROS: - if (LOG.isDebugEnabled()) { - LOG - .debug("Creating SASL " + AuthMethod.KERBEROS.getMechanismName() - + " client. Server's Kerberos principal name is " - + serverPrincipal); - } - if (serverPrincipal == null || serverPrincipal.length() == 0) { - throw new IOException( - "Failed to specify server's Kerberos principal name"); - } - String[] names = SaslUtil.splitKerberosName(serverPrincipal); - if (names.length != 3) { - throw new IOException( - "Kerberos principal does not have the expected format: " - + serverPrincipal); - } - saslClient = createKerberosSaslClient( - new String[] { AuthMethod.KERBEROS.getMechanismName() }, - names[0], names[1]); - break; - default: - throw new IOException("Unknown authentication method " + method); - } - if (saslClient == null) - throw new IOException("Unable to find SASL client implementation"); + public HBaseSaslRpcClient(AuthMethod method, Token token, + String serverPrincipal, boolean fallbackAllowed) throws IOException { + super(method, token, serverPrincipal, fallbackAllowed); } - protected SaslClient createDigestSaslClient(String[] mechanismNames, - String saslDefaultRealm, CallbackHandler saslClientCallbackHandler) - throws IOException { - return Sasl.createSaslClient(mechanismNames, null, null, saslDefaultRealm, - SaslUtil.SASL_PROPS, saslClientCallbackHandler); - } - - protected SaslClient createKerberosSaslClient(String[] mechanismNames, - String userFirstPart, String userSecondPart) throws IOException { - return Sasl.createSaslClient(mechanismNames, null, userFirstPart, - userSecondPart, SaslUtil.SASL_PROPS, null); + public HBaseSaslRpcClient(AuthMethod method, Token token, + String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException { + super(method, token, serverPrincipal, fallbackAllowed, rpcProtection); } private static void readStatus(DataInputStream inStream) throws IOException { @@ -156,72 +67,65 @@ public class HBaseSaslRpcClient { } /** - * Do client side SASL authentication with server via the given InputStream - * and OutputStream - * - * @param inS - * InputStream to use - * @param outS - * OutputStream to use - * @return true if connection is set up, or false if needs to switch - * to simple Auth. + * Do client side SASL authentication with server via the given InputStream and OutputStream + * @param inS InputStream to use + * @param outS OutputStream to use + * @return true if connection is set up, or false if needs to switch to simple Auth. * @throws IOException */ - public boolean saslConnect(InputStream inS, OutputStream outS) - throws IOException { + public boolean saslConnect(InputStream inS, OutputStream outS) throws IOException { DataInputStream inStream = new DataInputStream(new BufferedInputStream(inS)); - DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream( - outS)); + DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(outS)); try { - byte[] saslToken = new byte[0]; - if (saslClient.hasInitialResponse()) - saslToken = saslClient.evaluateChallenge(saslToken); + byte[] saslToken = getInitialResponse(); if (saslToken != null) { outStream.writeInt(saslToken.length); outStream.write(saslToken, 0, saslToken.length); outStream.flush(); - if (LOG.isDebugEnabled()) - LOG.debug("Have sent token of size " + saslToken.length - + " from initSASLContext."); + if (LOG.isDebugEnabled()) { + LOG.debug("Have sent token of size " + saslToken.length + " from initSASLContext."); + } } - if (!saslClient.isComplete()) { + if (!isComplete()) { readStatus(inStream); int len = inStream.readInt(); if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) { if (!fallbackAllowed) { - throw new IOException("Server asks us to fall back to SIMPLE auth, " + - "but this client is configured to only allow secure connections."); + throw new IOException("Server asks us to fall back to SIMPLE auth, " + + "but this client is configured to only allow secure connections."); } if (LOG.isDebugEnabled()) { LOG.debug("Server asks us to fall back to simple auth."); } - saslClient.dispose(); + dispose(); return false; } saslToken = new byte[len]; - if (LOG.isDebugEnabled()) + if (LOG.isDebugEnabled()) { LOG.debug("Will read input token of size " + saslToken.length + " for processing by initSASLContext"); + } inStream.readFully(saslToken); } - while (!saslClient.isComplete()) { - saslToken = saslClient.evaluateChallenge(saslToken); + while (!isComplete()) { + saslToken = evaluateChallenge(saslToken); if (saslToken != null) { - if (LOG.isDebugEnabled()) - LOG.debug("Will send token of size " + saslToken.length - + " from initSASLContext."); + if (LOG.isDebugEnabled()) { + LOG.debug("Will send token of size " + saslToken.length + " from initSASLContext."); + } outStream.writeInt(saslToken.length); outStream.write(saslToken, 0, saslToken.length); outStream.flush(); } - if (!saslClient.isComplete()) { + if (!isComplete()) { readStatus(inStream); saslToken = new byte[inStream.readInt()]; - if (LOG.isDebugEnabled()) + if (LOG.isDebugEnabled()) { LOG.debug("Will read input token of size " + saslToken.length + " for processing by initSASLContext"); + } inStream.readFully(saslToken); } } @@ -241,11 +145,8 @@ public class HBaseSaslRpcClient { } /** - * Get a SASL wrapped InputStream. Can be called only after saslConnect() has - * been called. - * - * @param in - * the InputStream to wrap + * Get a SASL wrapped InputStream. Can be called only after saslConnect() has been called. + * @param in the InputStream to wrap * @return a SASL wrapped InputStream * @throws IOException */ @@ -257,11 +158,8 @@ public class HBaseSaslRpcClient { } /** - * Get a SASL wrapped OutputStream. Can be called only after saslConnect() has - * been called. - * - * @param out - * the OutputStream to wrap + * Get a SASL wrapped OutputStream. Can be called only after saslConnect() has been called. + * @param out the OutputStream to wrap * @return a SASL wrapped OutputStream * @throws IOException */ @@ -271,58 +169,4 @@ public class HBaseSaslRpcClient { } return new SaslOutputStream(out, saslClient); } - - /** Release resources used by wrapped saslClient */ - public void dispose() throws SaslException { - saslClient.dispose(); - } - - @VisibleForTesting - static class SaslClientCallbackHandler implements CallbackHandler { - private final String userName; - private final char[] userPassword; - - public SaslClientCallbackHandler(Token token) { - this.userName = SaslUtil.encodeIdentifier(token.getIdentifier()); - this.userPassword = SaslUtil.encodePassword(token.getPassword()); - } - - @Override - public void handle(Callback[] callbacks) - throws UnsupportedCallbackException { - NameCallback nc = null; - PasswordCallback pc = null; - RealmCallback rc = null; - for (Callback callback : callbacks) { - if (callback instanceof RealmChoiceCallback) { - continue; - } else if (callback instanceof NameCallback) { - nc = (NameCallback) callback; - } else if (callback instanceof PasswordCallback) { - pc = (PasswordCallback) callback; - } else if (callback instanceof RealmCallback) { - rc = (RealmCallback) callback; - } else { - throw new UnsupportedCallbackException(callback, - "Unrecognized SASL client callback"); - } - } - if (nc != null) { - if (LOG.isDebugEnabled()) - LOG.debug("SASL client callback: setting username: " + userName); - nc.setName(userName); - } - if (pc != null) { - if (LOG.isDebugEnabled()) - LOG.debug("SASL client callback: setting userPassword"); - pc.setPassword(userPassword); - } - if (rc != null) { - if (LOG.isDebugEnabled()) - LOG.debug("SASL client callback: setting realm: " - + rc.getDefaultText()); - rc.setText(rc.getDefaultText()); - } - } - } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java new file mode 100644 index 00000000000..f624608e0d1 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClient.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security; + +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; + +import java.io.IOException; + +import javax.security.sasl.Sasl; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; + +/** + * Implement SASL logic for netty rpc client. + */ +@InterfaceAudience.Private +public class NettyHBaseSaslRpcClient extends AbstractHBaseSaslRpcClient { + private static final Log LOG = LogFactory.getLog(NettyHBaseSaslRpcClient.class); + + public NettyHBaseSaslRpcClient(AuthMethod method, Token token, + String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException { + super(method, token, serverPrincipal, fallbackAllowed, rpcProtection); + } + + public void setupSaslHandler(ChannelPipeline p) { + String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP); + if (LOG.isDebugEnabled()) { + LOG.debug("SASL client context established. Negotiated QoP: " + qop); + } + if (qop == null || "auth".equalsIgnoreCase(qop)) { + return; + } + // add wrap and unwrap handlers to pipeline. + p.addFirst(new SaslWrapHandler(saslClient), + new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4), + new SaslUnwrapHandler(saslClient)); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java new file mode 100644 index 00000000000..50609b4b3e1 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java @@ -0,0 +1,142 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.util.concurrent.Promise; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.FallbackDisallowedException; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; + +/** + * Implement SASL logic for netty rpc client. + */ +@InterfaceAudience.Private +public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler { + + private static final Log LOG = LogFactory.getLog(NettyHBaseSaslRpcClientHandler.class); + + private final Promise saslPromise; + + private final UserGroupInformation ugi; + + private final NettyHBaseSaslRpcClient saslRpcClient; + + /** + * @param saslPromise {@code true} if success, {@code false} if server tells us to fallback to + * simple. + */ + public NettyHBaseSaslRpcClientHandler(Promise saslPromise, UserGroupInformation ugi, + AuthMethod method, Token token, String serverPrincipal, + boolean fallbackAllowed, String rpcProtection) throws IOException { + this.saslPromise = saslPromise; + this.ugi = ugi; + this.saslRpcClient = new NettyHBaseSaslRpcClient(method, token, serverPrincipal, + fallbackAllowed, rpcProtection); + } + + private void writeResponse(ChannelHandlerContext ctx, byte[] response) { + if (LOG.isDebugEnabled()) { + LOG.debug("Will send token of size " + response.length + " from initSASLContext."); + } + ctx.writeAndFlush( + ctx.alloc().buffer(4 + response.length).writeInt(response.length).writeBytes(response)); + } + + private void tryComplete(ChannelHandlerContext ctx) { + if (!saslRpcClient.isComplete()) { + return; + } + saslRpcClient.setupSaslHandler(ctx.pipeline()); + saslPromise.setSuccess(true); + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + try { + byte[] initialResponse = ugi.doAs(new PrivilegedExceptionAction() { + + @Override + public byte[] run() throws Exception { + return saslRpcClient.getInitialResponse(); + } + }); + if (initialResponse != null) { + writeResponse(ctx, initialResponse); + } + tryComplete(ctx); + } catch (Exception e) { + // the exception thrown by handlerAdded will not be passed to the exceptionCaught below + // because netty will remove a handler if handlerAdded throws an exception. + exceptionCaught(ctx, e); + } + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { + int len = msg.readInt(); + if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) { + saslRpcClient.dispose(); + if (saslRpcClient.fallbackAllowed) { + saslPromise.trySuccess(false); + } else { + saslPromise.tryFailure(new FallbackDisallowedException()); + } + return; + } + if (LOG.isDebugEnabled()) { + LOG.debug("Will read input token of size " + len + " for processing by initSASLContext"); + } + final byte[] challenge = new byte[len]; + msg.readBytes(challenge); + byte[] response = ugi.doAs(new PrivilegedExceptionAction() { + + @Override + public byte[] run() throws Exception { + return saslRpcClient.evaluateChallenge(challenge); + } + }); + if (response != null) { + writeResponse(ctx, response); + } + tryComplete(ctx); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + saslRpcClient.dispose(); + saslPromise.tryFailure(new IOException("Connection closed")); + ctx.fireChannelInactive(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + saslRpcClient.dispose(); + saslPromise.tryFailure(cause); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java new file mode 100644 index 00000000000..57bb36cf750 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.ByteToMessageDecoder; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.ipc.RemoteException; + +/** + * Decode the sasl challenge sent by RpcServer. + */ +@InterfaceAudience.Private +public class SaslChallengeDecoder extends ByteToMessageDecoder { + + private static final int MAX_CHALLENGE_SIZE = 1024 * 1024; // 1M + + private ByteBuf tryDecodeChallenge(ByteBuf in, int offset, int readableBytes) throws IOException { + if (readableBytes < 4) { + return null; + } + int len = in.getInt(offset); + if (len <= 0) { + // fall back to simple + in.readerIndex(offset + 4); + return in.retainedSlice(offset, 4); + } + if (len > MAX_CHALLENGE_SIZE) { + throw new IOException( + "Sasl challenge too large(" + len + "), max allowed is " + MAX_CHALLENGE_SIZE); + } + int totalLen = 4 + len; + if (readableBytes < totalLen) { + return null; + } + in.readerIndex(offset + totalLen); + return in.retainedSlice(offset, totalLen); + } + + // will throw a RemoteException out if data is enough, so do not need to return anything. + private void tryDecodeError(ByteBuf in, int offset, int readableBytes) throws IOException { + if (readableBytes < 4) { + return; + } + int classLen = in.getInt(offset); + if (classLen <= 0) { + throw new IOException("Invalid exception class name length " + classLen); + } + if (classLen > MAX_CHALLENGE_SIZE) { + throw new IOException("Exception class name length too large(" + classLen + + "), max allowed is " + MAX_CHALLENGE_SIZE); + } + if (readableBytes < 4 + classLen + 4) { + return; + } + int msgLen = in.getInt(offset + 4 + classLen); + if (msgLen <= 0) { + throw new IOException("Invalid exception message length " + msgLen); + } + if (msgLen > MAX_CHALLENGE_SIZE) { + throw new IOException("Exception message length too large(" + msgLen + "), max allowed is " + + MAX_CHALLENGE_SIZE); + } + int totalLen = classLen + msgLen + 8; + if (readableBytes < totalLen) { + return; + } + String className = in.toString(offset + 4, classLen, HConstants.UTF8_CHARSET); + String msg = in.toString(offset + classLen + 8, msgLen, HConstants.UTF8_CHARSET); + in.readerIndex(offset + totalLen); + throw new RemoteException(className, msg); + } + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { + int readableBytes = in.readableBytes(); + if (readableBytes < 4) { + return; + } + int offset = in.readerIndex(); + int status = in.getInt(offset); + if (status == SaslStatus.SUCCESS.state) { + ByteBuf challenge = tryDecodeChallenge(in, offset + 4, readableBytes - 4); + if (challenge != null) { + out.add(challenge); + } + } else { + tryDecodeError(in, offset + 4, readableBytes - 4); + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java deleted file mode 100644 index 0f11083da97..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java +++ /dev/null @@ -1,401 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.security; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.Channel; -import io.netty.channel.ChannelDuplexHandler; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelPromise; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; - -import javax.security.auth.callback.CallbackHandler; -import javax.security.sasl.Sasl; -import javax.security.sasl.SaslClient; -import javax.security.sasl.SaslException; - -import java.io.IOException; -import java.nio.charset.Charset; -import java.security.PrivilegedExceptionAction; -import java.util.Random; - -/** - * Handles Sasl connections - */ -@InterfaceAudience.Private -public class SaslClientHandler extends ChannelDuplexHandler { - private static final Log LOG = LogFactory.getLog(SaslClientHandler.class); - - private final boolean fallbackAllowed; - - private final UserGroupInformation ticket; - - /** - * Used for client or server's token to send or receive from each other. - */ - private final SaslClient saslClient; - private final SaslExceptionHandler exceptionHandler; - private final SaslSuccessfulConnectHandler successfulConnectHandler; - private byte[] saslToken; - private byte[] connectionHeader; - private boolean firstRead = true; - - private int retryCount = 0; - private Random random; - - /** - * Constructor - * - * @param ticket the ugi - * @param method auth method - * @param token for Sasl - * @param serverPrincipal Server's Kerberos principal name - * @param fallbackAllowed True if server may also fall back to less secure connection - * @param rpcProtection Quality of protection. Can be 'authentication', 'integrity' or - * 'privacy'. - * @param exceptionHandler handler for exceptions - * @param successfulConnectHandler handler for succesful connects - * @throws java.io.IOException if handler could not be created - */ - public SaslClientHandler(UserGroupInformation ticket, AuthMethod method, - Token token, String serverPrincipal, boolean fallbackAllowed, - String rpcProtection, byte[] connectionHeader, SaslExceptionHandler exceptionHandler, - SaslSuccessfulConnectHandler successfulConnectHandler) throws IOException { - this.ticket = ticket; - this.fallbackAllowed = fallbackAllowed; - this.connectionHeader = connectionHeader; - - this.exceptionHandler = exceptionHandler; - this.successfulConnectHandler = successfulConnectHandler; - - SaslUtil.initSaslProperties(rpcProtection); - switch (method) { - case DIGEST: - if (LOG.isDebugEnabled()) - LOG.debug("Creating SASL " + AuthMethod.DIGEST.getMechanismName() - + " client to authenticate to service at " + token.getService()); - saslClient = createDigestSaslClient(new String[] { AuthMethod.DIGEST.getMechanismName() }, - SaslUtil.SASL_DEFAULT_REALM, new HBaseSaslRpcClient.SaslClientCallbackHandler(token)); - break; - case KERBEROS: - if (LOG.isDebugEnabled()) { - LOG.debug("Creating SASL " + AuthMethod.KERBEROS.getMechanismName() - + " client. Server's Kerberos principal name is " + serverPrincipal); - } - if (serverPrincipal == null || serverPrincipal.isEmpty()) { - throw new IOException("Failed to specify server's Kerberos principal name"); - } - String[] names = SaslUtil.splitKerberosName(serverPrincipal); - if (names.length != 3) { - throw new IOException( - "Kerberos principal does not have the expected format: " + serverPrincipal); - } - saslClient = createKerberosSaslClient(new String[] { AuthMethod.KERBEROS.getMechanismName() }, - names[0], names[1]); - break; - default: - throw new IOException("Unknown authentication method " + method); - } - if (saslClient == null) { - throw new IOException("Unable to find SASL client implementation"); - } - } - - /** - * Create a Digest Sasl client - * - * @param mechanismNames names of mechanisms - * @param saslDefaultRealm default realm for sasl - * @param saslClientCallbackHandler handler for the client - * @return new SaslClient - * @throws java.io.IOException if creation went wrong - */ - protected SaslClient createDigestSaslClient(String[] mechanismNames, String saslDefaultRealm, - CallbackHandler saslClientCallbackHandler) throws IOException { - return Sasl.createSaslClient(mechanismNames, null, null, saslDefaultRealm, SaslUtil.SASL_PROPS, - saslClientCallbackHandler); - } - - /** - * Create Kerberos client - * - * @param mechanismNames names of mechanisms - * @param userFirstPart first part of username - * @param userSecondPart second part of username - * @return new SaslClient - * @throws java.io.IOException if fails - */ - protected SaslClient createKerberosSaslClient(String[] mechanismNames, String userFirstPart, - String userSecondPart) throws IOException { - return Sasl - .createSaslClient(mechanismNames, null, userFirstPart, userSecondPart, SaslUtil.SASL_PROPS, - null); - } - - @Override - public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { - saslClient.dispose(); - } - - private byte[] evaluateChallenge(final byte[] challenge) throws Exception { - return ticket.doAs(new PrivilegedExceptionAction() { - - @Override - public byte[] run() throws Exception { - return saslClient.evaluateChallenge(challenge); - } - }); - } - - @Override - public void handlerAdded(final ChannelHandlerContext ctx) throws Exception { - saslToken = new byte[0]; - if (saslClient.hasInitialResponse()) { - saslToken = evaluateChallenge(saslToken); - } - if (saslToken != null) { - writeSaslToken(ctx, saslToken); - if (LOG.isDebugEnabled()) { - LOG.debug("Have sent token of size " + saslToken.length + " from initSASLContext."); - } - } - } - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - ByteBuf in = (ByteBuf) msg; - - // If not complete, try to negotiate - if (!saslClient.isComplete()) { - while (!saslClient.isComplete() && in.isReadable()) { - readStatus(in); - int len = in.readInt(); - if (firstRead) { - firstRead = false; - if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) { - if (!fallbackAllowed) { - throw new IOException("Server asks us to fall back to SIMPLE auth, " + "but this " - + "client is configured to only allow secure connections."); - } - if (LOG.isDebugEnabled()) { - LOG.debug("Server asks us to fall back to simple auth."); - } - saslClient.dispose(); - - ctx.pipeline().remove(this); - successfulConnectHandler.onSuccess(ctx.channel()); - return; - } - } - saslToken = new byte[len]; - if (LOG.isDebugEnabled()) { - LOG.debug("Will read input token of size " + saslToken.length - + " for processing by initSASLContext"); - } - in.readBytes(saslToken); - - saslToken = evaluateChallenge(saslToken); - if (saslToken != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Will send token of size " + saslToken.length + " from initSASLContext."); - } - writeSaslToken(ctx, saslToken); - } - } - // release the memory - in.release(); - - if (saslClient.isComplete()) { - String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP); - - if (LOG.isDebugEnabled()) { - LOG.debug("SASL client context established. Negotiated QoP: " + qop); - } - - boolean useWrap = qop != null && !"auth".equalsIgnoreCase(qop); - - if (!useWrap) { - ctx.pipeline().remove(this); - successfulConnectHandler.onSuccess(ctx.channel()); - } else { - byte[] wrappedCH = saslClient.wrap(connectionHeader, 0, connectionHeader.length); - // write connection header - writeSaslToken(ctx, wrappedCH); - successfulConnectHandler.onSaslProtectionSucess(ctx.channel()); - } - } - } - // Normal wrapped reading - else { - try { - int length = in.readInt(); - if (LOG.isDebugEnabled()) { - LOG.debug("Actual length is " + length); - } - saslToken = new byte[length]; - in.readBytes(saslToken); - // release the memory - in.release(); - } catch (IndexOutOfBoundsException e) { - return; - } - try { - ByteBuf b = ctx.channel().alloc().buffer(saslToken.length); - - b.writeBytes(saslClient.unwrap(saslToken, 0, saslToken.length)); - ctx.fireChannelRead(b); - - } catch (SaslException se) { - try { - saslClient.dispose(); - } catch (SaslException ignored) { - LOG.debug("Ignoring SASL exception", ignored); - } - throw se; - } - } - } - - /** - * Write SASL token - * @param ctx to write to - * @param saslToken to write - */ - private void writeSaslToken(final ChannelHandlerContext ctx, byte[] saslToken) { - ByteBuf b = ctx.alloc().buffer(4 + saslToken.length); - b.writeInt(saslToken.length); - b.writeBytes(saslToken, 0, saslToken.length); - ctx.writeAndFlush(b).addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (!future.isSuccess()) { - exceptionCaught(ctx, future.cause()); - } - } - }); - } - - /** - * Get the read status - * - * @param inStream to read - * @throws org.apache.hadoop.ipc.RemoteException if status was not success - */ - private static void readStatus(ByteBuf inStream) throws RemoteException { - int status = inStream.readInt(); // read status - if (status != SaslStatus.SUCCESS.state) { - throw new RemoteException(inStream.toString(Charset.forName("UTF-8")), - inStream.toString(Charset.forName("UTF-8"))); - } - } - - @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) - throws Exception { - saslClient.dispose(); - - ctx.close(); - - if (this.random == null) { - this.random = new Random(); - } - exceptionHandler.handle(this.retryCount++, this.random, cause); - } - - @Override - public void write(final ChannelHandlerContext ctx, Object msg, ChannelPromise promise) - throws Exception { - // If not complete, try to negotiate - if (!saslClient.isComplete()) { - super.write(ctx, msg, promise); - } else { - ByteBuf in = (ByteBuf) msg; - byte[] unwrapped = new byte[in.readableBytes()]; - in.readBytes(unwrapped); - // release the memory - in.release(); - - try { - saslToken = saslClient.wrap(unwrapped, 0, unwrapped.length); - } catch (SaslException se) { - try { - saslClient.dispose(); - } catch (SaslException ignored) { - LOG.debug("Ignoring SASL exception", ignored); - } - promise.setFailure(se); - } - if (saslToken != null) { - ByteBuf out = ctx.channel().alloc().buffer(4 + saslToken.length); - out.writeInt(saslToken.length); - out.writeBytes(saslToken, 0, saslToken.length); - - ctx.write(out).addListener(new ChannelFutureListener() { - @Override public void operationComplete(ChannelFuture future) throws Exception { - if (!future.isSuccess()) { - exceptionCaught(ctx, future.cause()); - } - } - }); - - saslToken = null; - } - } - } - - /** - * Handler for exceptions during Sasl connection - */ - public interface SaslExceptionHandler { - /** - * Handle the exception - * - * @param retryCount current retry count - * @param random to create new backoff with - * @param cause of fail - */ - public void handle(int retryCount, Random random, Throwable cause); - } - - /** - * Handler for successful connects - */ - public interface SaslSuccessfulConnectHandler { - /** - * Runs on success - * - * @param channel which is successfully authenticated - */ - public void onSuccess(Channel channel); - - /** - * Runs on success if data protection used in Sasl - * - * @param channel which is successfully authenticated - */ - public void onSaslProtectionSucess(Channel channel); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java new file mode 100644 index 00000000000..e631478940f --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; + +import javax.security.sasl.SaslClient; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Unwrap sasl messages. Should be placed after a + * {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder} + */ +@InterfaceAudience.Private +public class SaslUnwrapHandler extends SimpleChannelInboundHandler { + + private final SaslClient saslClient; + + public SaslUnwrapHandler(SaslClient saslClient) { + this.saslClient = saslClient; + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + SaslUtil.safeDispose(saslClient); + ctx.fireChannelInactive(); + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { + byte[] bytes = new byte[msg.readableBytes()]; + msg.readBytes(bytes); + ctx.fireChannelRead(Unpooled.wrappedBuffer(saslClient.unwrap(bytes, 0, bytes.length))); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java index cfc40886820..aaa9d7abfaf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java @@ -18,14 +18,12 @@ */ package org.apache.hadoop.hbase.security; -import org.apache.commons.codec.binary.Base64; -import org.apache.hadoop.hbase.classification.InterfaceAudience; - -import java.util.Locale; import java.util.Map; import java.util.TreeMap; import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; import org.apache.commons.codec.binary.Base64; import org.apache.commons.logging.Log; @@ -34,26 +32,33 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; @InterfaceAudience.Private public class SaslUtil { - private static final Log log = LogFactory.getLog(SaslUtil.class); + private static final Log LOG = LogFactory.getLog(SaslUtil.class); public static final String SASL_DEFAULT_REALM = "default"; - public static final Map SASL_PROPS = - new TreeMap(); public static final int SWITCH_TO_SIMPLE_AUTH = -88; - public static enum QualityOfProtection { + public enum QualityOfProtection { AUTHENTICATION("auth"), INTEGRITY("auth-int"), PRIVACY("auth-conf"); - public final String saslQop; + private final String saslQop; - private QualityOfProtection(String saslQop) { + QualityOfProtection(String saslQop) { this.saslQop = saslQop; } public String getSaslQop() { return saslQop; } + + public boolean matches(String stringQop) { + if (saslQop.equals(stringQop)) { + LOG.warn("Use authentication/integrity/privacy as value for rpc protection " + + "configurations instead of auth/auth-int/auth-conf."); + return true; + } + return name().equalsIgnoreCase(stringQop); + } } /** Splitting fully qualified Kerberos name into parts */ @@ -75,40 +80,47 @@ public class SaslUtil { /** * Returns {@link org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection} - * corresponding to the given {@code stringQop} value. Returns null if value is - * invalid. + * corresponding to the given {@code stringQop} value. + * @throws IllegalArgumentException If stringQop doesn't match any QOP. */ public static QualityOfProtection getQop(String stringQop) { - QualityOfProtection qop = null; - if (QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT).equals(stringQop) - || QualityOfProtection.AUTHENTICATION.saslQop.equals(stringQop)) { - qop = QualityOfProtection.AUTHENTICATION; - } else if (QualityOfProtection.INTEGRITY.name().toLowerCase(Locale.ROOT).equals(stringQop) - || QualityOfProtection.INTEGRITY.saslQop.equals(stringQop)) { - qop = QualityOfProtection.INTEGRITY; - } else if (QualityOfProtection.PRIVACY.name().toLowerCase(Locale.ROOT).equals(stringQop) - || QualityOfProtection.PRIVACY.saslQop.equals(stringQop)) { - qop = QualityOfProtection.PRIVACY; + for (QualityOfProtection qop : QualityOfProtection.values()) { + if (qop.matches(stringQop)) { + return qop; + } } - if (qop == null) { - throw new IllegalArgumentException("Invalid qop: " + stringQop - + ". It must be one of 'authentication', 'integrity', 'privacy'."); - } - if (QualityOfProtection.AUTHENTICATION.saslQop.equals(stringQop) - || QualityOfProtection.INTEGRITY.saslQop.equals(stringQop) - || QualityOfProtection.PRIVACY.saslQop.equals(stringQop)) { - log.warn("Use authentication/integrity/privacy as value for rpc protection " - + "configurations instead of auth/auth-int/auth-conf."); - } - return qop; + throw new IllegalArgumentException("Invalid qop: " + stringQop + + ". It must be one of 'authentication', 'integrity', 'privacy'."); } - static void initSaslProperties(String rpcProtection) { - QualityOfProtection saslQOP = getQop(rpcProtection); - if (saslQOP == null) { - saslQOP = QualityOfProtection.AUTHENTICATION; + /** + * @param rpcProtection Value of 'hbase.rpc.protection' configuration. + * @return Map with values for SASL properties. + */ + static Map initSaslProperties(String rpcProtection) { + String saslQop; + if (rpcProtection.isEmpty()) { + saslQop = QualityOfProtection.AUTHENTICATION.getSaslQop(); + } else { + String[] qops = rpcProtection.split(","); + StringBuilder saslQopBuilder = new StringBuilder(); + for (int i = 0; i < qops.length; ++i) { + QualityOfProtection qop = getQop(qops[i]); + saslQopBuilder.append(",").append(qop.getSaslQop()); + } + saslQop = saslQopBuilder.substring(1); // remove first ',' + } + Map saslProps = new TreeMap<>(); + saslProps.put(Sasl.QOP, saslQop); + saslProps.put(Sasl.SERVER_AUTH, "true"); + return saslProps; + } + + static void safeDispose(SaslClient saslClient) { + try { + saslClient.dispose(); + } catch (SaslException e) { + LOG.error("Error disposing of SASL client", e); } - SaslUtil.SASL_PROPS.put(Sasl.QOP, saslQOP.getSaslQop()); - SaslUtil.SASL_PROPS.put(Sasl.SERVER_AUTH, "true"); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java new file mode 100644 index 00000000000..14ecf2e5e3c --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.CoalescingBufferQueue; +import io.netty.util.ReferenceCountUtil; +import io.netty.util.concurrent.PromiseCombiner; + +import java.io.IOException; + +import javax.security.sasl.SaslClient; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * wrap sasl messages. + */ +@InterfaceAudience.Private +public class SaslWrapHandler extends ChannelOutboundHandlerAdapter { + + private final SaslClient saslClient; + + private CoalescingBufferQueue queue; + + public SaslWrapHandler(SaslClient saslClient) { + this.saslClient = saslClient; + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + queue = new CoalescingBufferQueue(ctx.channel()); + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + if (msg instanceof ByteBuf) { + queue.add((ByteBuf) msg, promise); + } else { + ctx.write(msg, promise); + } + } + + @Override + public void flush(ChannelHandlerContext ctx) throws Exception { + if (queue.isEmpty()) { + return; + } + ByteBuf buf = null; + try { + ChannelPromise promise = ctx.newPromise(); + int readableBytes = queue.readableBytes(); + buf = queue.remove(readableBytes, promise); + byte[] bytes = new byte[readableBytes]; + buf.readBytes(bytes); + byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length); + ChannelPromise lenPromise = ctx.newPromise(); + ctx.write(ctx.alloc().buffer(4).writeInt(wrapperBytes.length), lenPromise); + ChannelPromise contentPromise = ctx.newPromise(); + ctx.write(Unpooled.wrappedBuffer(wrapperBytes), contentPromise); + PromiseCombiner combiner = new PromiseCombiner(); + combiner.addAll(lenPromise, contentPromise); + combiner.finish(promise); + ctx.flush(); + } finally { + if (buf != null) { + ReferenceCountUtil.safeRelease(buf); + } + } + } + + @Override + public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { + if (!queue.isEmpty()) { + queue.releaseAndFailAll(new IOException("Connection closed")); + } + ctx.close(promise); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java index 5799aaf6f16..2f9d921f582 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.security.SecurityCapability; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface; @@ -99,7 +99,7 @@ public class AccessControlClient { public static void grant(Connection connection, final TableName tableName, final String userName, final byte[] family, final byte[] qual, boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable { - PayloadCarryingRpcController controller = + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); controller.setPriority(tableName); try (Table table = connection.getTable(ACL_TABLE_NAME)) { @@ -139,7 +139,7 @@ public class AccessControlClient { public static void grant(final Connection connection, final String namespace, final String userName, boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable { - PayloadCarryingRpcController controller = + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); try (Table table = connection.getTable(ACL_TABLE_NAME)) { @@ -174,7 +174,7 @@ public class AccessControlClient { */ public static void grant(final Connection connection, final String userName, boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable { - PayloadCarryingRpcController controller = + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); try (Table table = connection.getTable(ACL_TABLE_NAME)) { ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, @@ -214,7 +214,7 @@ public class AccessControlClient { public static void revoke(final Connection connection, final TableName tableName, final String username, final byte[] family, final byte[] qualifier, final Permission.Action... actions) throws Throwable { - PayloadCarryingRpcController controller + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); controller.setPriority(tableName); try (Table table = connection.getTable(ACL_TABLE_NAME)) { @@ -233,7 +233,7 @@ public class AccessControlClient { */ public static void revoke(final Connection connection, final String namespace, final String userName, final Permission.Action... actions) throws Throwable { - PayloadCarryingRpcController controller + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); try (Table table = connection.getTable(ACL_TABLE_NAME)) { ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), userName, namespace, @@ -247,7 +247,7 @@ public class AccessControlClient { */ public static void revoke(final Connection connection, final String userName, final Permission.Action... actions) throws Throwable { - PayloadCarryingRpcController controller + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); try (Table table = connection.getTable(ACL_TABLE_NAME)) { ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), userName, actions); @@ -263,7 +263,7 @@ public class AccessControlClient { */ public static List getUserPermissions(Connection connection, String tableRegex) throws Throwable { - PayloadCarryingRpcController controller + HBaseRpcController controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); List permList = new ArrayList(); try (Table table = connection.getTable(ACL_TABLE_NAME)) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index 1630d83c611..f70e0de89d6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hbase.zookeeper; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.RpcController; + import java.io.EOFException; import java.io.IOException; import java.net.ConnectException; @@ -43,7 +46,6 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.ipc.FailedServerException; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -58,8 +60,6 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.ipc.RemoteException; import org.apache.zookeeper.KeeperException; -import com.google.protobuf.InvalidProtocolBufferException; - /** * Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper * which keeps hbase:meta region server location. @@ -319,7 +319,7 @@ public class MetaTableLocator { return false; } Throwable t; - PayloadCarryingRpcController controller = null; + RpcController controller = null; if (connection instanceof ClusterConnection) { controller = ((ClusterConnection) connection).getRpcControllerFactory().newController(); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java index 8aa80076059..6385c27dfa4 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.RpcController; + import java.io.IOException; import org.apache.commons.logging.Log; @@ -27,21 +29,19 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; -import com.google.protobuf.RpcController; - /** * Test snapshot logic from the client */ @@ -88,7 +88,7 @@ public class TestSnapshotFromAdmin { RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); Mockito.when(controllerFactory.newController()).thenReturn( - Mockito.mock(PayloadCarryingRpcController.class)); + Mockito.mock(HBaseRpcController.class)); Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); // set the max wait time for the snapshot to complete @@ -136,7 +136,7 @@ public class TestSnapshotFromAdmin { RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); Mockito.when(controllerFactory.newController()).thenReturn( - Mockito.mock(PayloadCarryingRpcController.class)); + Mockito.mock(HBaseRpcController.class)); Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); Admin admin = new HBaseAdmin(mockConnection); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java new file mode 100644 index 00000000000..650634706b7 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java @@ -0,0 +1,195 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; + +import org.apache.commons.lang.time.StopWatch; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.codec.KeyValueCodec; +import org.apache.hadoop.hbase.io.SizedCellScanner; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.log4j.Level; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ClientTests.class, SmallTests.class }) +public class TestCellBlockBuilder { + + private static final Log LOG = LogFactory.getLog(TestCellBlockBuilder.class); + + private CellBlockBuilder builder; + + @Before + public void before() { + this.builder = new CellBlockBuilder(HBaseConfiguration.create()); + } + + @Test + public void testBuildCellBlock() throws IOException { + doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), null); + doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), new DefaultCodec()); + doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), new GzipCodec()); + } + + static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder builder, final Codec codec, + final CompressionCodec compressor) throws IOException { + doBuildCellBlockUndoCellBlock(builder, codec, compressor, 10, 1, false); + } + + static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder builder, final Codec codec, + final CompressionCodec compressor, final int count, final int size, final boolean sized) + throws IOException { + Cell[] cells = getCells(count, size); + CellScanner cellScanner = sized ? getSizedCellScanner(cells) + : CellUtil.createCellScanner(Arrays.asList(cells).iterator()); + ByteBuffer bb = builder.buildCellBlock(codec, compressor, cellScanner); + cellScanner = builder.createCellScanner(codec, compressor, bb); + int i = 0; + while (cellScanner.advance()) { + i++; + } + assertEquals(count, i); + } + + static CellScanner getSizedCellScanner(final Cell[] cells) { + int size = -1; + for (Cell cell : cells) { + size += CellUtil.estimatedSerializedSizeOf(cell); + } + final int totalSize = ClassSize.align(size); + final CellScanner cellScanner = CellUtil.createCellScanner(cells); + return new SizedCellScanner() { + @Override + public long heapSize() { + return totalSize; + } + + @Override + public Cell current() { + return cellScanner.current(); + } + + @Override + public boolean advance() throws IOException { + return cellScanner.advance(); + } + }; + } + + static Cell[] getCells(final int howMany) { + return getCells(howMany, 1024); + } + + static Cell[] getCells(final int howMany, final int valueSize) { + Cell[] cells = new Cell[howMany]; + byte[] value = new byte[valueSize]; + for (int i = 0; i < howMany; i++) { + byte[] index = Bytes.toBytes(i); + KeyValue kv = new KeyValue(index, Bytes.toBytes("f"), index, value); + cells[i] = kv; + } + return cells; + } + + private static final String COUNT = "--count="; + private static final String SIZE = "--size="; + + /** + * Prints usage and then exits w/ passed errCode + * @param errCode + */ + private static void usage(final int errCode) { + System.out.println("Usage: IPCUtil [options]"); + System.out.println("Micro-benchmarking how changed sizes and counts work with buffer resizing"); + System.out.println(" --count Count of Cells"); + System.out.println(" --size Size of Cell values"); + System.out.println("Example: IPCUtil --count=1024 --size=1024"); + System.exit(errCode); + } + + private static void timerTests(final CellBlockBuilder builder, final int count, final int size, + final Codec codec, final CompressionCodec compressor) throws IOException { + final int cycles = 1000; + StopWatch timer = new StopWatch(); + timer.start(); + for (int i = 0; i < cycles; i++) { + timerTest(builder, timer, count, size, codec, compressor, false); + } + timer.stop(); + LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + false + ", count=" + + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); + timer.reset(); + timer.start(); + for (int i = 0; i < cycles; i++) { + timerTest(builder, timer, count, size, codec, compressor, true); + } + timer.stop(); + LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + true + ", count=" + + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); + } + + private static void timerTest(final CellBlockBuilder builder, final StopWatch timer, + final int count, final int size, final Codec codec, final CompressionCodec compressor, + final boolean sized) throws IOException { + doBuildCellBlockUndoCellBlock(builder, codec, compressor, count, size, sized); + } + + /** + * For running a few tests of methods herein. + * @param args + * @throws IOException + */ + public static void main(String[] args) throws IOException { + int count = 1024; + int size = 10240; + for (String arg : args) { + if (arg.startsWith(COUNT)) { + count = Integer.parseInt(arg.replace(COUNT, "")); + } else if (arg.startsWith(SIZE)) { + size = Integer.parseInt(arg.replace(SIZE, "")); + } else { + usage(1); + } + } + CellBlockBuilder builder = new CellBlockBuilder(HBaseConfiguration.create()); + ((Log4JLogger) CellBlockBuilder.LOG).getLogger().setLevel(Level.ALL); + timerTests(builder, count, size, new KeyValueCodec(), null); + timerTests(builder, count, size, new KeyValueCodec(), new DefaultCodec()); + timerTests(builder, count, size, new KeyValueCodec(), new GzipCodec()); + } +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java similarity index 92% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java rename to hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java index b506b88ad03..3b0a6b2ee12 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseRpcControllerImpl.java @@ -28,14 +28,15 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.experimental.categories.Category; +@Category({ ClientTests.class, SmallTests.class }) +public class TestHBaseRpcControllerImpl { -@Category(SmallTests.class) -public class TestPayloadCarryingRpcController { @Test public void testListOfCellScannerables() throws IOException { List cells = new ArrayList(); @@ -43,12 +44,12 @@ public class TestPayloadCarryingRpcController { for (int i = 0; i < count; i++) { cells.add(createCell(i)); } - PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells); + HBaseRpcController controller = new HBaseRpcControllerImpl(cells); CellScanner cellScanner = controller.cellScanner(); int index = 0; for (; cellScanner.advance(); index++) { Cell cell = cellScanner.current(); - byte [] indexBytes = Bytes.toBytes(index); + byte[] indexBytes = Bytes.toBytes(index); assertTrue("" + index, Bytes.equals(indexBytes, 0, indexBytes.length, cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); } @@ -66,7 +67,7 @@ public class TestPayloadCarryingRpcController { return new CellScanner() { @Override public Cell current() { - // Fake out a Cell. All this Cell has is a value that is an int in size and equal + // Fake out a Cell. All this Cell has is a value that is an int in size and equal // to the above 'index' param serialized as an int. return new Cell() { private final int i = index; @@ -137,12 +138,6 @@ public class TestPayloadCarryingRpcController { return 0; } - @Override - public long getMvccVersion() { - // unused - return 0; - } - @Override public long getSequenceId() { // unused @@ -182,33 +177,35 @@ public class TestPayloadCarryingRpcController { return null; } + @Override + public long getMvccVersion() { + return 0; + } + @Override public byte[] getValue() { - // unused - return null; + return Bytes.toBytes(this.i); } @Override public byte[] getFamily() { - // unused return null; } @Override public byte[] getQualifier() { - // unused return null; } @Override public byte[] getRow() { - // unused return null; } }; } private boolean hasCell = true; + @Override public boolean advance() { // We have one Cell only so return true first time then false ever after. diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java index bb580c8561a..7c4ac029acd 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java @@ -17,180 +17,32 @@ */ package org.apache.hadoop.hbase.ipc; -import static org.junit.Assert.assertEquals; +import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException; +import static org.junit.Assert.assertTrue; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.SocketTimeoutException; -import org.apache.commons.lang.time.StopWatch; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; +import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.codec.KeyValueCodec; -import org.apache.hadoop.hbase.io.SizedCellScanner; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.io.compress.GzipCodec; -import org.apache.log4j.Level; -import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; -@Category(SmallTests.class) +@Category({ ClientTests.class, SmallTests.class }) public class TestIPCUtil { - private static final Log LOG = LogFactory.getLog(TestIPCUtil.class); - - IPCUtil util; - @Before - public void before() { - this.util = new IPCUtil(new Configuration()); - } - @Test - public void testBuildCellBlock() throws IOException { - doBuildCellBlockUndoCellBlock(this.util, new KeyValueCodec(), null); - doBuildCellBlockUndoCellBlock(this.util, new KeyValueCodec(), new DefaultCodec()); - doBuildCellBlockUndoCellBlock(this.util, new KeyValueCodec(), new GzipCodec()); - } - - static void doBuildCellBlockUndoCellBlock(final IPCUtil util, - final Codec codec, final CompressionCodec compressor) - throws IOException { - doBuildCellBlockUndoCellBlock(util, codec, compressor, 10, 1, false); - } - - static void doBuildCellBlockUndoCellBlock(final IPCUtil util, final Codec codec, - final CompressionCodec compressor, final int count, final int size, final boolean sized) - throws IOException { - Cell [] cells = getCells(count, size); - CellScanner cellScanner = sized? getSizedCellScanner(cells): - CellUtil.createCellScanner(Arrays.asList(cells).iterator()); - ByteBuffer bb = util.buildCellBlock(codec, compressor, cellScanner); - cellScanner = util.createCellScanner(codec, compressor, bb); - int i = 0; - while (cellScanner.advance()) { - i++; - } - assertEquals(count, i); - } - - static CellScanner getSizedCellScanner(final Cell [] cells) { - int size = -1; - for (Cell cell: cells) { - size += CellUtil.estimatedSerializedSizeOf(cell); - } - final int totalSize = ClassSize.align(size); - final CellScanner cellScanner = CellUtil.createCellScanner(cells); - return new SizedCellScanner() { - @Override - public long heapSize() { - return totalSize; - } - - @Override - public Cell current() { - return cellScanner.current(); - } - - @Override - public boolean advance() throws IOException { - return cellScanner.advance(); - } - }; - } - - static Cell [] getCells(final int howMany) { - return getCells(howMany, 1024); - } - - static Cell [] getCells(final int howMany, final int valueSize) { - Cell [] cells = new Cell[howMany]; - byte [] value = new byte[valueSize]; - for (int i = 0; i < howMany; i++) { - byte [] index = Bytes.toBytes(i); - KeyValue kv = new KeyValue(index, Bytes.toBytes("f"), index, value); - cells[i] = kv; - } - return cells; - } - - private static final String COUNT = "--count="; - private static final String SIZE = "--size="; - - /** - * Prints usage and then exits w/ passed errCode - * @param errCode - */ - private static void usage(final int errCode) { - System.out.println("Usage: IPCUtil [options]"); - System.out.println("Micro-benchmarking how changed sizes and counts work with buffer resizing"); - System.out.println(" --count Count of Cells"); - System.out.println(" --size Size of Cell values"); - System.out.println("Example: IPCUtil --count=1024 --size=1024"); - System.exit(errCode); - } - - private static void timerTests(final IPCUtil util, final int count, final int size, - final Codec codec, final CompressionCodec compressor) - throws IOException { - final int cycles = 1000; - StopWatch timer = new StopWatch(); - timer.start(); - for (int i = 0; i < cycles; i++) { - timerTest(util, timer, count, size, codec, compressor, false); - } - timer.stop(); - LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + false + - ", count=" + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); - timer.reset(); - timer.start(); - for (int i = 0; i < cycles; i++) { - timerTest(util, timer, count, size, codec, compressor, true); - } - timer.stop(); - LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + true + - ", count=" + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); - } - - private static void timerTest(final IPCUtil util, final StopWatch timer, final int count, - final int size, final Codec codec, final CompressionCodec compressor, final boolean sized) - throws IOException { - doBuildCellBlockUndoCellBlock(util, codec, compressor, count, size, sized); - } - - /** - * For running a few tests of methods herein. - * @param args - * @throws IOException - */ - public static void main(String[] args) throws IOException { - int count = 1024; - int size = 10240; - for (String arg: args) { - if (arg.startsWith(COUNT)) { - count = Integer.parseInt(arg.replace(COUNT, "")); - } else if (arg.startsWith(SIZE)) { - size = Integer.parseInt(arg.replace(SIZE, "")); - } else { - usage(1); - } - } - IPCUtil util = new IPCUtil(HBaseConfiguration.create()); - ((Log4JLogger)IPCUtil.LOG).getLogger().setLevel(Level.ALL); - timerTests(util, count, size, new KeyValueCodec(), null); - timerTests(util, count, size, new KeyValueCodec(), new DefaultCodec()); - timerTests(util, count, size, new KeyValueCodec(), new GzipCodec()); + public void testWrapException() throws Exception { + final InetSocketAddress address = InetSocketAddress.createUnresolved("localhost", 0); + assertTrue(wrapException(address, new ConnectException()) instanceof ConnectException); + assertTrue( + wrapException(address, new SocketTimeoutException()) instanceof SocketTimeoutException); + assertTrue(wrapException(address, new ConnectionClosingException( + "Test AbstractRpcClient#wrapException")) instanceof ConnectionClosingException); + assertTrue( + wrapException(address, new CallTimeoutException("Test AbstractRpcClient#wrapException")) + .getCause() instanceof CallTimeoutException); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java similarity index 88% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java rename to hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java index db4a8eef74f..12b36616db8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,6 +27,8 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.common.base.Strings; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -38,13 +39,14 @@ import javax.security.auth.callback.NameCallback; import javax.security.auth.callback.PasswordCallback; import javax.security.auth.callback.TextOutputCallback; import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.sasl.Sasl; import javax.security.sasl.RealmCallback; import javax.security.sasl.RealmChoiceCallback; +import javax.security.sasl.Sasl; import javax.security.sasl.SaslClient; +import org.apache.hadoop.hbase.security.AbstractHBaseSaslRpcClient.SaslClientCallbackHandler; +import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.security.HBaseSaslRpcClient.SaslClientCallbackHandler; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.security.token.Token; @@ -58,16 +60,14 @@ import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.mockito.Mockito; -import com.google.common.base.Strings; - -@Category(SmallTests.class) +@Category({SecurityTests.class, SmallTests.class}) public class TestHBaseSaslRpcClient { - + static { System.setProperty("java.security.krb5.realm", "DOMAIN.COM"); System.setProperty("java.security.krb5.kdc", "DOMAIN.COM"); } - + static final String DEFAULT_USER_NAME = "principal"; static final String DEFAULT_USER_PASSWORD = "password"; @@ -83,33 +83,18 @@ public class TestHBaseSaslRpcClient { } @Test - public void testSaslQOPNotEmpty() throws Exception { + public void testSaslClientUsesGivenRpcProtection() throws Exception { Token token = createTokenMockWithCredentials(DEFAULT_USER_NAME, DEFAULT_USER_PASSWORD); - // default QOP is authentication - new HBaseSaslRpcClient(AuthMethod.DIGEST, token, "principal/host@DOMAIN.COM", false); - assertTrue(SaslUtil.SASL_PROPS.get(Sasl.QOP).equals(SaslUtil.QualityOfProtection. - AUTHENTICATION.getSaslQop())); - - // check with specific QOPs - new HBaseSaslRpcClient(AuthMethod.DIGEST, token, "principal/host@DOMAIN.COM", false, - "authentication"); - assertTrue(SaslUtil.SASL_PROPS.get(Sasl.QOP).equals(SaslUtil.QualityOfProtection. - AUTHENTICATION.getSaslQop())); - - new HBaseSaslRpcClient(AuthMethod.DIGEST, token, "principal/host@DOMAIN.COM", false, - "privacy"); - assertTrue(SaslUtil.SASL_PROPS.get(Sasl.QOP).equals(SaslUtil.QualityOfProtection. - PRIVACY.getSaslQop())); - - new HBaseSaslRpcClient(AuthMethod.DIGEST, token, "principal/host@DOMAIN.COM", false, - "integrity"); - assertTrue(SaslUtil.SASL_PROPS.get(Sasl.QOP).equals(SaslUtil.QualityOfProtection. - INTEGRITY.getSaslQop())); - - exception.expect(IllegalArgumentException.class); - new HBaseSaslRpcClient(AuthMethod.DIGEST, token, "principal/host@DOMAIN.COM", false, - "wrongvalue"); + for (SaslUtil.QualityOfProtection qop : SaslUtil.QualityOfProtection.values()) { + String negotiatedQop = new HBaseSaslRpcClient(AuthMethod.DIGEST, token, + "principal/host@DOMAIN.COM", false, qop.name()) { + public String getQop() { + return saslProps.get(Sasl.QOP); + } + }.getQop(); + assertEquals(negotiatedQop, qop.getSaslQop()); + } } @Test @@ -209,7 +194,7 @@ public class TestHBaseSaslRpcClient { boolean inState = false; boolean outState = false; - HBaseSaslRpcClient rpcClient = new HBaseSaslRpcClient(AuthMethod.DIGEST, + HBaseSaslRpcClient rpcClient = new HBaseSaslRpcClient(AuthMethod.DIGEST, createTokenMockWithCredentials(principal, password), principal, false) { @Override public SaslClient createDigestSaslClient(String[] mechanismNames, @@ -224,7 +209,7 @@ public class TestHBaseSaslRpcClient { return Mockito.mock(SaslClient.class); } }; - + try { rpcClient.getInputStream(Mockito.mock(InputStream.class)); } catch(IOException ex) { @@ -244,7 +229,7 @@ public class TestHBaseSaslRpcClient { private boolean assertIOExceptionThenSaslClientIsNull(String principal, String password) { try { - new HBaseSaslRpcClient(AuthMethod.DIGEST, + new HBaseSaslRpcClient(AuthMethod.DIGEST, createTokenMockWithCredentials(principal, password), principal, false) { @Override public SaslClient createDigestSaslClient(String[] mechanismNames, @@ -252,7 +237,7 @@ public class TestHBaseSaslRpcClient { throws IOException { return null; } - + @Override public SaslClient createKerberosSaslClient(String[] mechanismNames, String userFirstPart, String userSecondPart) throws IOException { @@ -278,7 +263,7 @@ public class TestHBaseSaslRpcClient { private boolean assertSuccessCreationDigestPrincipal(String principal, String password) { HBaseSaslRpcClient rpcClient = null; try { - rpcClient = new HBaseSaslRpcClient(AuthMethod.DIGEST, + rpcClient = new HBaseSaslRpcClient(AuthMethod.DIGEST, createTokenMockWithCredentials(principal, password), principal, false); } catch(Exception ex) { LOG.error(ex.getMessage(), ex); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java index 0a7ac8a042f..f41efc7e346 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java @@ -19,10 +19,17 @@ package org.apache.hadoop.hbase.ipc; import static org.apache.hadoop.hbase.ipc.RpcClient.SPECIFIC_WRITE_THREAD; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; +import com.google.protobuf.BlockingService; +import com.google.protobuf.Descriptors.MethodDescriptor; +import com.google.protobuf.Message; + import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; @@ -40,27 +47,17 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; -import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Lists; -import com.google.protobuf.BlockingService; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import com.google.protobuf.Descriptors.MethodDescriptor; @Category(IntegrationTests.class) public class IntegrationTestRpcClient { @@ -95,38 +92,13 @@ public class IntegrationTestRpcClient { } } - static final BlockingService SERVICE = - TestRpcServiceProtos.TestProtobufRpcProto - .newReflectiveBlockingService(new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() { - - @Override - public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public EmptyResponseProto error(RpcController controller, EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public EchoResponseProto echo(RpcController controller, EchoRequestProto request) - throws ServiceException { - return EchoResponseProto.newBuilder().setMessage(request.getMessage()).build(); - } - }); - - protected AbstractRpcClient createRpcClient(Configuration conf, boolean isSyncClient) { - return isSyncClient ? - new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT) : - new AsyncRpcClient(conf) { - @Override - Codec getCodec() { - return null; - } - }; + protected AbstractRpcClient createRpcClient(Configuration conf, boolean isSyncClient) { + return isSyncClient ? new BlockingRpcClient(conf) : new NettyRpcClient(conf) { + @Override + Codec getCodec() { + return null; + } + }; } static String BIG_PAYLOAD; @@ -283,7 +255,7 @@ public class IntegrationTestRpcClient { } static class SimpleClient extends Thread { - AbstractRpcClient rpcClient; + AbstractRpcClient rpcClient; AtomicBoolean running = new AtomicBoolean(true); AtomicBoolean sending = new AtomicBoolean(false); AtomicReference exception = new AtomicReference<>(null); @@ -292,7 +264,7 @@ public class IntegrationTestRpcClient { long numCalls = 0; Random random = new Random(); - public SimpleClient(Cluster cluster, AbstractRpcClient rpcClient, String id) { + public SimpleClient(Cluster cluster, AbstractRpcClient rpcClient, String id) { this.cluster = cluster; this.rpcClient = rpcClient; this.id = id; @@ -301,24 +273,16 @@ public class IntegrationTestRpcClient { @Override public void run() { - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - while (running.get()) { boolean isBigPayload = random.nextBoolean(); String message = isBigPayload ? BIG_PAYLOAD : id + numCalls; EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message).build(); - EchoResponseProto ret = EchoResponseProto.newBuilder().setMessage("foo").build(); - + EchoResponseProto ret; TestRpcServer server = cluster.getRandomServer(); try { - User user = User.getCurrent(); - InetSocketAddress address = server.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } sending.set(true); - ret = (EchoResponseProto) - rpcClient.callBlockingMethod(md, null, param, ret, user, address); + BlockingInterface stub = newBlockingStub(rpcClient, server.getListenerAddress()); + ret = stub.echo(null, param); } catch (Exception e) { LOG.warn(e); continue; // expected in case connection is closing or closed @@ -360,7 +324,7 @@ public class IntegrationTestRpcClient { cluster.startServer(); conf.setBoolean(SPECIFIC_WRITE_THREAD, true); for(int i = 0; i <1000; i++) { - AbstractRpcClient rpcClient = createRpcClient(conf, true); + AbstractRpcClient rpcClient = createRpcClient(conf, true); SimpleClient client = new SimpleClient(cluster, rpcClient, "Client1"); client.start(); while(!client.isSending()) { @@ -452,7 +416,7 @@ public class IntegrationTestRpcClient { ArrayList clients = new ArrayList<>(); // all threads should share the same rpc client - AbstractRpcClient rpcClient = createRpcClient(conf, isSyncClient); + AbstractRpcClient rpcClient = createRpcClient(conf, isSyncClient); for (int i = 0; i < 30; i++) { String clientId = "client_" + i + "_"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index db73dfef35f..871ea65de62 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -20,6 +20,15 @@ package org.apache.hadoop.hbase.ipc; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.protobuf.BlockingService; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.Descriptors.MethodDescriptor; +import com.google.protobuf.Message; +import com.google.protobuf.ServiceException; +import com.google.protobuf.TextFormat; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; @@ -58,7 +67,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -68,20 +76,17 @@ import javax.security.sasl.SaslServer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.CallQueueTooBigException; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CallQueueTooBigException; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException; -import org.apache.hadoop.hbase.client.Operation; import org.apache.hadoop.hbase.client.VersionInfoUtil; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.conf.ConfigurationObserver; @@ -90,7 +95,6 @@ import org.apache.hadoop.hbase.exceptions.RequestTooBigException; import org.apache.hadoop.hbase.io.BoundedByteBufferPool; import org.apache.hadoop.hbase.io.ByteBufferInputStream; import org.apache.hadoop.hbase.io.ByteBufferOutputStream; -import org.apache.hadoop.hbase.io.BoundedByteBufferPool; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -102,17 +106,16 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; -import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.AuthMethod; import org.apache.hadoop.hbase.security.HBasePolicyProvider; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; -import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler; import org.apache.hadoop.hbase.security.SaslStatus; import org.apache.hadoop.hbase.security.SaslUtil; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager; import org.apache.hadoop.hbase.util.Bytes; @@ -134,17 +137,8 @@ import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.StringUtils; -import org.codehaus.jackson.map.ObjectMapper; import org.apache.htrace.TraceInfo; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.google.protobuf.BlockingService; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; -import com.google.protobuf.Descriptors.MethodDescriptor; -import com.google.protobuf.Message; -import com.google.protobuf.ServiceException; -import com.google.protobuf.TextFormat; +import org.codehaus.jackson.map.ObjectMapper; /** * An RPC server that hosts protobuf described Services. @@ -163,8 +157,6 @@ import com.google.protobuf.TextFormat; * * CallRunner#run executes the call. When done, asks the included Call to put itself on new * queue for Responder to pull from and return result to client. - * - * @see RpcClientImpl */ @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX}) @InterfaceStability.Evolving @@ -195,7 +187,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { */ private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024; - private final IPCUtil ipcUtil; + private final CellBlockBuilder cellBlockBuilder; private static final String AUTH_FAILED_FOR = "Auth failed for "; private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for "; @@ -468,7 +460,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { // Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the // reservoir when finished. This is hacky and the hack is not contained but benefits are // high when we can avoid a big buffer allocation on each rpc. - this.cellBlock = ipcUtil.buildCellBlock(this.connection.codec, + this.cellBlock = cellBlockBuilder.buildCellBlock(this.connection.codec, this.connection.compressionCodec, cells, reservoir); if (this.cellBlock != null) { CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder(); @@ -1175,6 +1167,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { * @throws IOException */ private boolean processResponse(final Call call) throws IOException { + LOG.info("processing " + call); boolean error = true; try { // Send as much data as we can in the non-blocking fashion @@ -1442,7 +1435,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } saslServer = Sasl.createSaslServer(AuthMethod.DIGEST .getMechanismName(), null, SaslUtil.SASL_DEFAULT_REALM, - SaslUtil.SASL_PROPS, new SaslDigestCallbackHandler( + HBaseSaslRpcServer.getSaslProps(), new SaslDigestCallbackHandler( secretManager, this)); break; default: @@ -1462,7 +1455,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { public Object run() throws SaslException { saslServer = Sasl.createSaslServer(AuthMethod.KERBEROS .getMechanismName(), names[0], names[1], - SaslUtil.SASL_PROPS, new SaslGssCallbackHandler()); + HBaseSaslRpcServer.getSaslProps(), new SaslGssCallbackHandler()); return null; } }); @@ -1988,7 +1981,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } if (header.hasCellBlockMeta()) { buf.position(offset); - cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec, buf); + cellScanner = cellBlockBuilder.createCellScanner(this.codec, this.compressionCodec, buf); } } catch (Throwable t) { InetSocketAddress address = getListenerAddress(); @@ -2194,7 +2187,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true); this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive", true); - this.ipcUtil = new IPCUtil(conf); + this.cellBlockBuilder = new CellBlockBuilder(conf); // Create the responder here @@ -2346,7 +2339,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { status.setRPCPacket(param); status.resume("Servicing call"); //get an instance of the method arg type - PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cellScanner); + HBaseRpcController controller = new HBaseRpcControllerImpl(cellScanner); controller.setCallTimeout(timeout); Message result = service.callBlockingMethod(md, controller, param); long endTime = System.currentTimeMillis(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 61790d00b7a..71d03ce0cb4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -18,6 +18,10 @@ */ package org.apache.hadoop.hbase.master; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.net.ConnectException; import java.net.InetAddress; @@ -52,7 +56,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.ipc.FailedServerException; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; @@ -66,25 +70,21 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Triple; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounterFactory; +import org.apache.hadoop.hbase.util.Triple; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; -import com.google.protobuf.ServiceException; - /** * The ServerManager class manages info about region servers. *

@@ -871,7 +871,7 @@ public class ServerManager { } } - private PayloadCarryingRpcController newRpcController() { + private HBaseRpcController newRpcController() { return rpcControllerFactory == null ? null : rpcControllerFactory.newController(); } @@ -899,7 +899,7 @@ public class ServerManager { region.getRegionNameAsString() + " failed because no RPC connection found to this server"); } - PayloadCarryingRpcController controller = newRpcController(); + HBaseRpcController controller = newRpcController(); return ProtobufUtil.closeRegion(controller, admin, server, region.getRegionName(), versionOfClosingNode, dest, transitionInZK); } @@ -922,7 +922,7 @@ public class ServerManager { if (server == null) return; try { AdminService.BlockingInterface admin = getRsAdmin(server); - PayloadCarryingRpcController controller = newRpcController(); + HBaseRpcController controller = newRpcController(); ProtobufUtil.warmupRegion(controller, admin, region); } catch (IOException e) { LOG.error("Received exception in RPC for warmup server:" + @@ -938,7 +938,7 @@ public class ServerManager { public static void closeRegionSilentlyAndWait(ClusterConnection connection, ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException { AdminService.BlockingInterface rs = connection.getAdmin(server); - PayloadCarryingRpcController controller = connection.getRpcControllerFactory().newController(); + HBaseRpcController controller = connection.getRpcControllerFactory().newController(); try { ProtobufUtil.closeRegion(controller, rs, server, region.getRegionName(), false); } catch (IOException e) { @@ -946,6 +946,7 @@ public class ServerManager { } long expiration = timeout + System.currentTimeMillis(); while (System.currentTimeMillis() < expiration) { + controller.reset(); try { HRegionInfo rsRegion = ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName()); @@ -989,7 +990,7 @@ public class ServerManager { + region_b.getRegionNameAsString() + " failed because no RPC connection found to this server"); } - PayloadCarryingRpcController controller = newRpcController(); + HBaseRpcController controller = newRpcController(); ProtobufUtil.mergeRegions(controller, admin, region_a, region_b, forcible, user); } @@ -1008,7 +1009,7 @@ public class ServerManager { } } try { - PayloadCarryingRpcController controller = newRpcController(); + HBaseRpcController controller = newRpcController(); AdminService.BlockingInterface admin = getRsAdmin(server); if (admin != null) { ServerInfo info = ProtobufUtil.getServerInfo(controller, admin); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java index 7a1031c0d20..d13a79c0630 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.protobuf; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -35,7 +37,8 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.SizedCellScanner; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; @@ -46,8 +49,6 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALKey; -import com.google.protobuf.ServiceException; - @InterfaceAudience.Private public class ReplicationProtbufUtil { /** @@ -66,7 +67,7 @@ public class ReplicationProtbufUtil { Pair p = buildReplicateWALEntryRequest(entries, null, replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir); - PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond()); + HBaseRpcController controller = new HBaseRpcControllerImpl(p.getSecond()); try { admin.replicateWALEntry(controller, p.getFirst()); } catch (ServiceException se) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 9c89260bce9..49ce348ac36 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -90,7 +90,7 @@ import org.apache.hadoop.hbase.exceptions.ScannerResetException; import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.QosPriority; import org.apache.hadoop.hbase.ipc.RpcCallContext; @@ -392,7 +392,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } private void addResult(final MutateResponse.Builder builder, - final Result result, final PayloadCarryingRpcController rpcc) { + final Result result, final HBaseRpcController rpcc) { if (result == null) return; if (isClientCellBlockSupport()) { builder.setResult(ProtobufUtil.toResultNoData(result)); @@ -404,7 +404,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } private void addResults(final ScanResponse.Builder builder, final List results, - final PayloadCarryingRpcController controller, boolean isDefaultRegion) { + final HBaseRpcController controller, boolean isDefaultRegion) { builder.setStale(!isDefaultRegion); if (results.isEmpty()) { return; @@ -1795,7 +1795,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, public ReplicateWALEntryResponse replay(final RpcController controller, final ReplicateWALEntryRequest request) throws ServiceException { long before = EnvironmentEdgeManager.currentTime(); - CellScanner cells = ((PayloadCarryingRpcController) controller).cellScanner(); + CellScanner cells = ((HBaseRpcController) controller).cellScanner(); try { checkOpen(); List entries = request.getEntryList(); @@ -1900,7 +1900,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, if (regionServer.replicationSinkHandler != null) { requestCount.increment(); List entries = request.getEntryList(); - CellScanner cellScanner = ((PayloadCarryingRpcController)controller).cellScanner(); + CellScanner cellScanner = ((HBaseRpcController)controller).cellScanner(); regionServer.getRegionServerCoprocessorHost().preReplicateLogEntries(entries, cellScanner); regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner, request.getReplicationClusterId(), request.getSourceBaseNamespaceDirPath(), @@ -2129,10 +2129,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } else if (r != null) { ClientProtos.Result pbr; RpcCallContext call = RpcServer.getCurrentCall(); - if (isClientCellBlockSupport(call) && controller instanceof PayloadCarryingRpcController + if (isClientCellBlockSupport(call) && controller instanceof HBaseRpcController && VersionInfoUtil.hasMinimumVersion(call.getClientVersionInfo(), 1, 3)) { pbr = ProtobufUtil.toResultNoData(r); - ((PayloadCarryingRpcController) controller) + ((HBaseRpcController) controller) .setCellScanner(CellUtil.createCellScanner(r.rawCells())); addSize(call, r, null); } else { @@ -2175,7 +2175,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // rpc controller is how we bring in data via the back door; it is unprotobuf'ed data. // It is also the conduit via which we pass back data. - PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc; + HBaseRpcController controller = (HBaseRpcController)rpcc; CellScanner cellScanner = controller != null ? controller.cellScanner(): null; if (controller != null) { controller.setCellScanner(null); @@ -2305,7 +2305,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, final MutateRequest request) throws ServiceException { // rpc controller is how we bring in data via the back door; it is unprotobuf'ed data. // It is also the conduit via which we pass back data. - PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc; + HBaseRpcController controller = (HBaseRpcController)rpcc; CellScanner cellScanner = controller != null ? controller.cellScanner() : null; OperationQuota quota = null; RpcCallContext context = RpcServer.getCurrentCall(); @@ -2530,7 +2530,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - private long getTimeLimit(PayloadCarryingRpcController controller, + private long getTimeLimit(HBaseRpcController controller, boolean allowHeartbeatMessages) { // Set the time limit to be half of the more restrictive timeout value (one of the // timeout values must be positive). In the event that both values are positive, the @@ -2559,7 +2559,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } // return whether we have more results in region. - private boolean scan(PayloadCarryingRpcController controller, ScanRequest request, + private boolean scan(HBaseRpcController controller, ScanRequest request, RegionScannerHolder rsh, long maxQuotaResultSize, int maxResults, List results, ScanResponse.Builder builder, MutableObject lastBlock, RpcCallContext context) throws IOException { @@ -2714,9 +2714,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, @Override public ScanResponse scan(final RpcController controller, final ScanRequest request) throws ServiceException { - if (controller != null && !(controller instanceof PayloadCarryingRpcController)) { + if (controller != null && !(controller instanceof HBaseRpcController)) { throw new UnsupportedOperationException( - "We only do PayloadCarryingRpcControllers! FIX IF A PROBLEM: " + controller); + "We only do HBaseRpcController! FIX IF A PROBLEM: " + controller); } if (!request.hasScannerId() && !request.hasScan()) { throw new ServiceException( @@ -2839,7 +2839,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } if (!done) { - moreResultsInRegion = scan((PayloadCarryingRpcController) controller, request, rsh, + moreResultsInRegion = scan((HBaseRpcController) controller, request, rsh, maxQuotaResultSize, rows, results, builder, lastBlock, context); } } @@ -2858,7 +2858,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, // if we have reached the limit of rows moreResults = false; } - addResults(builder, results, (PayloadCarryingRpcController) controller, + addResults(builder, results, (HBaseRpcController) controller, RegionReplicaUtil.isDefaultReplica(region.getRegionInfo())); if (!moreResults || !moreResultsInRegion || closeScanner) { scannerClosed = true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java index 1314a4dccc5..5f6fd450279 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.regionserver.wal; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -27,7 +29,6 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; @@ -35,10 +36,11 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.RegionServerCallable; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; @@ -49,8 +51,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL.Entry; -import com.google.protobuf.ServiceException; - /** * This class is responsible for replaying the edits coming from a failed region server. *

@@ -214,7 +214,7 @@ public class WALEditsReplaySink { Pair p = ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond()); + HBaseRpcController controller = rpcControllerFactory.newController(p.getSecond()); try { remoteSvr.replay(controller, p.getFirst()); } catch (ServiceException se) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java index 069701397ed..235e27a6eba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java @@ -18,6 +18,11 @@ package org.apache.hadoop.hbase.replication.regionserver; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.collect.Lists; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; @@ -34,7 +39,6 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellScanner; @@ -45,29 +49,22 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionAdminServiceCallable; -import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RetryingCallable; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; -import org.apache.hadoop.hbase.wal.WALKey; -import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController; -import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer; -import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter; -import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; -import org.apache.hadoop.hbase.wal.WAL.Entry; -import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers; -import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink; import org.apache.hadoop.hbase.replication.BaseWALEntryFilter; import org.apache.hadoop.hbase.replication.ChainWALEntryFilter; import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; @@ -76,13 +73,14 @@ import org.apache.hadoop.hbase.replication.WALEntryFilter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers; +import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink; +import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController; +import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer; +import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter; import org.apache.hadoop.util.StringUtils; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import com.google.common.collect.Lists; -import com.google.protobuf.ServiceException; - /** * A {@link ReplicationEndpoint} endpoint which receives the WAL edits from the * WAL, and sends the edits to replicas of regions. @@ -658,7 +656,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location.getRegionInfo().getEncodedNameAsBytes(), null, null, null); try { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond()); + HBaseRpcController controller = rpcControllerFactory.newController(p.getSecond()); controller.setCallTimeout(timeout); controller.setPriority(tableName); return stub.replay(controller, p.getFirst()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java index 8b0fa708c58..644a70d9cbe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java @@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; import java.util.Locale; +import java.util.Map; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; @@ -33,14 +34,14 @@ import javax.security.sasl.RealmCallback; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager; -import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.SecretManager.InvalidToken; +import org.apache.hadoop.security.token.TokenIdentifier; /** * A utility class for dealing with SASL on RPC server @@ -49,11 +50,17 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken; public class HBaseSaslRpcServer { private static final Log LOG = LogFactory.getLog(HBaseSaslRpcServer.class); + private static Map saslProps = null; + public static void init(Configuration conf) { - SaslUtil.initSaslProperties(conf.get("hbase.rpc.protection", + saslProps = SaslUtil.initSaslProperties(conf.get("hbase.rpc.protection", QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT))); } + public static Map getSaslProps() { + return saslProps; + } + public static T getIdentifier(String id, SecretManager secretManager) throws InvalidToken { byte[] tokenId = SaslUtil.decodeIdentifier(id); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java index c537fe0576f..96ac5af9705 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java @@ -22,6 +22,9 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -32,7 +35,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; @@ -47,9 +50,6 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** * Test MetaTableAccessor but without spinning up a cluster. * We mock regionserver back and forth (we do spin up a zk cluster). @@ -163,7 +163,7 @@ public class TestMetaTableAccessorNoCluster { .thenThrow(new ServiceException("Server not running (3 of 3)")) .thenAnswer(new Answer() { public ScanResponse answer(InvocationOnMock invocation) throws Throwable { - ((PayloadCarryingRpcController) invocation.getArguments()[0]).setCellScanner(CellUtil + ((HBaseRpcController) invocation.getArguments()[0]).setCellScanner(CellUtil .createCellScanner(cellScannables)); return builder.setScannerId(1234567890L).setMoreResults(false).build(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java index de80a7b57e1..350dd846ba4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java @@ -19,9 +19,12 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; + +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.net.ConnectException; @@ -30,8 +33,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.RegionState; @@ -53,9 +57,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** * Test {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator} */ @@ -253,7 +254,7 @@ public class TestMetaTableLocator { thenReturn(implementation); RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); Mockito.when(controllerFactory.newController()).thenReturn( - Mockito.mock(PayloadCarryingRpcController.class)); + Mockito.mock(HBaseRpcController.class)); Mockito.when(connection.getRpcControllerFactory()).thenReturn(controllerFactory); ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index a475e5aab20..711f520dca1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -22,40 +22,38 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.net.SocketAddress; -import java.net.SocketTimeoutException; -import java.net.UnknownHostException; -import java.util.Random; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.ipc.AbstractRpcClient; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcClientImpl; -import org.apache.hadoop.hbase.security.User; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import com.google.protobuf.BlockingRpcChannel; import com.google.protobuf.Descriptors.MethodDescriptor; import com.google.protobuf.Message; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -@Category(MediumTests.class) +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.BlockingRpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({MediumTests.class, ClientTests.class}) public class TestClientTimeouts { - private static final Log LOG = LogFactory.getLog(TestClientTimeouts.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); protected static int SLAVES = 1; @@ -86,7 +84,6 @@ public class TestClientTimeouts { */ @Test public void testAdminTimeout() throws Exception { - Connection lastConnection = null; boolean lastFailed = false; int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get(); RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory @@ -98,12 +95,11 @@ public class TestClientTimeouts { // Ensure the HBaseAdmin uses a new connection by changing Configuration. Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - HBaseAdmin admin = null; + Admin admin = null; + Connection connection = null; try { - admin = new HBaseAdmin(conf); - Connection connection = admin.getConnection(); - assertFalse(connection == lastConnection); - lastConnection = connection; + connection = ConnectionFactory.createConnection(conf); + admin = connection.getAdmin(); // run some admin commands HBaseAdmin.checkHBaseAvailable(conf); admin.setBalancerRunning(false, false); @@ -112,10 +108,15 @@ public class TestClientTimeouts { // a MasterNotRunningException. It's a bug if we get other exceptions. lastFailed = true; } finally { - admin.close(); - if (admin.getConnection().isClosed()) { - rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + if(admin != null) { + admin.close(); + if (admin.getConnection().isClosed()) { + rpcClient = (RandomTimeoutRpcClient) RpcClientFactory + .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + } + } + if(connection != null) { + connection.close(); } } } @@ -130,7 +131,7 @@ public class TestClientTimeouts { /** * Rpc Channel implementation with RandomTimeoutBlockingRpcChannel */ - public static class RandomTimeoutRpcClient extends RpcClientImpl { + public static class RandomTimeoutRpcClient extends BlockingRpcClient { public RandomTimeoutRpcClient(Configuration conf, String clusterId, SocketAddress localAddr, MetricsConnection metrics) { super(conf, clusterId, localAddr, metrics); @@ -153,9 +154,9 @@ public class TestClientTimeouts { public static final double CHANCE_OF_TIMEOUT = 0.3; private static AtomicInteger invokations = new AtomicInteger(); - RandomTimeoutBlockingRpcChannel(final RpcClientImpl rpcClient, final ServerName sn, - final User ticket, final int rpcTimeout) throws UnknownHostException { - super(rpcClient, sn, ticket, rpcTimeout); + RandomTimeoutBlockingRpcChannel(final BlockingRpcClient rpcClient, final ServerName sn, + final User ticket, final int rpcTimeout) { + super(rpcClient, new InetSocketAddress(sn.getHostname(), sn.getPort()), ticket, rpcTimeout); } @Override @@ -172,4 +173,4 @@ public class TestClientTimeouts { return super.callBlockingMethod(md, controller, param, returnType); } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java index b788e358458..ad406b477c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java @@ -21,6 +21,9 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.util.ArrayList; @@ -31,12 +34,11 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest; @@ -49,6 +51,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,9 +60,6 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - @Category(SmallTests.class) public class TestHBaseAdminNoCluster { @@ -314,7 +314,7 @@ public class TestHBaseAdminNoCluster { RpcControllerFactory rpcControllerFactory = Mockito.mock(RpcControllerFactory.class); Mockito.when(connection.getRpcControllerFactory()).thenReturn(rpcControllerFactory); Mockito.when(rpcControllerFactory.newController()).thenReturn( - Mockito.mock(PayloadCarryingRpcController.class)); + Mockito.mock(HBaseRpcController.class)); // we need a real retrying caller RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(configuration); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index f468c16d588..6622ae9b50e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -18,6 +18,13 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import com.google.common.collect.Lists; import java.io.IOException; @@ -61,7 +68,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; -import org.apache.hadoop.hbase.ipc.CallTimeoutException; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.ServerTooBusyException; @@ -86,13 +92,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestRule; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - /** * This class is for testing HBaseConnectionManager features */ @@ -726,8 +725,11 @@ public class TestHCM { c2.setInt(HConstants.HBASE_CLIENT_PAUSE, 1); // don't wait between retries. c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt); - - final HTable table = new HTable(c2, tableName); + c2.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, 2000); + c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1000); + ConnectionManager.HConnectionImplementation conn = + (ConnectionManager.HConnectionImplementation) ConnectionManager.createConnection(c2); + final HTable table = (HTable) conn.getTable(tableName); Put put = new Put(ROW); put.add(FAM_NAM, ROW, ROW); @@ -749,6 +751,7 @@ public class TestHCM { done++; if (done % 100 == 0) LOG.info("done=" + done); + Thread.sleep(100); } } catch (Throwable t) { failed.set(t); @@ -766,8 +769,6 @@ public class TestHCM { }); ServerName sn = table.getRegionLocation(ROW).getServerName(); - ConnectionManager.HConnectionImplementation conn = - (ConnectionManager.HConnectionImplementation) table.getConnection(); RpcClient rpcClient = conn.getRpcClient(); LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java index eb989d2992a..1d49460a30f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java @@ -21,6 +21,8 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import com.google.common.collect.Lists; + import java.io.IOException; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -30,21 +32,19 @@ import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ProtobufCoprocessorService; -import org.apache.hadoop.hbase.ipc.DelegatingPayloadCarryingRpcController; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Lists; - @Category(MediumTests.class) public class TestRpcControllerFactory { @@ -55,27 +55,27 @@ public class TestRpcControllerFactory { } @Override - public PayloadCarryingRpcController newController() { + public HBaseRpcController newController() { return new CountingRpcController(super.newController()); } @Override - public PayloadCarryingRpcController newController(final CellScanner cellScanner) { + public HBaseRpcController newController(final CellScanner cellScanner) { return new CountingRpcController(super.newController(cellScanner)); } @Override - public PayloadCarryingRpcController newController(final List cellIterables) { + public HBaseRpcController newController(final List cellIterables) { return new CountingRpcController(super.newController(cellIterables)); } } - public static class CountingRpcController extends DelegatingPayloadCarryingRpcController { + public static class CountingRpcController extends DelegatingHBaseRpcController { private static AtomicInteger INT_PRIORITY = new AtomicInteger(); private static AtomicInteger TABLE_PRIORITY = new AtomicInteger(); - public CountingRpcController(PayloadCarryingRpcController delegate) { + public CountingRpcController(HBaseRpcController delegate) { super(delegate); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java index cdda28ab538..da12683c4c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java @@ -23,19 +23,24 @@ import com.google.protobuf.RpcController; import com.google.protobuf.Service; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.util.Threads; import java.io.IOException; /** * Test implementation of a coprocessor endpoint exposing the - * {@link TestRpcServiceProtos.TestProtobufRpcProto} service methods. For internal use by - * unit tests only. + * {@link TestRpcServiceProtos.TestProtobufRpcProto} service methods. For internal use by unit tests + * only. */ -public class ProtobufCoprocessorService - extends TestRpcServiceProtos.TestProtobufRpcProto +public class ProtobufCoprocessorService extends TestRpcServiceProtos.TestProtobufRpcProto implements CoprocessorService, Coprocessor { public ProtobufCoprocessorService() { } @@ -47,31 +52,46 @@ public class ProtobufCoprocessorService @Override public void ping(RpcController controller, TestProtos.EmptyRequestProto request, - RpcCallback done) { + RpcCallback done) { done.run(TestProtos.EmptyResponseProto.getDefaultInstance()); } @Override public void echo(RpcController controller, TestProtos.EchoRequestProto request, - RpcCallback done) { + RpcCallback done) { String message = request.getMessage(); done.run(TestProtos.EchoResponseProto.newBuilder().setMessage(message).build()); } @Override public void error(RpcController controller, TestProtos.EmptyRequestProto request, - RpcCallback done) { + RpcCallback done) { ResponseConverter.setControllerException(controller, new IOException("Test exception")); done.run(null); } + @Override + public void pause(RpcController controller, PauseRequestProto request, + RpcCallback done) { + Threads.sleep(request.getMs()); + done.run(EmptyResponseProto.getDefaultInstance()); + } + + @Override + public void addr(RpcController controller, EmptyRequestProto request, + RpcCallback done) { + done.run(AddrResponseProto.newBuilder().setAddr(RpcServer.getRemoteAddress().getHostAddress()) + .build()); + } + @Override public void start(CoprocessorEnvironment env) throws IOException { - //To change body of implemented methods use File | Settings | File Templates. + // To change body of implemented methods use File | Settings | File Templates. } @Override public void stop(CoprocessorEnvironment env) throws IOException { - //To change body of implemented methods use File | Settings | File Templates. + // To change body of implemented methods use File | Settings | File Templates. } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java index bbf8720f8ad..3c3d06d9b6d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java @@ -17,7 +17,13 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newStub; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.anyObject; @@ -25,11 +31,14 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.internal.verification.VerificationModeFactory.times; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.protobuf.ServiceException; + import java.io.IOException; -import java.net.ConnectException; -import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.List; @@ -39,36 +48,22 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; -import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; -import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.util.StringUtils; -import org.junit.Assert; import org.junit.Test; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.protobuf.BlockingRpcChannel; -import com.google.protobuf.BlockingService; -import com.google.protobuf.Descriptors.MethodDescriptor; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** * Some basic ipc tests. */ @@ -76,59 +71,11 @@ public abstract class AbstractTestIPC { private static final Log LOG = LogFactory.getLog(AbstractTestIPC.class); - private static byte[] CELL_BYTES = Bytes.toBytes("xyz"); - private static KeyValue CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES); + private static final byte[] CELL_BYTES = Bytes.toBytes("xyz"); + private static final KeyValue CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES); static byte[] BIG_CELL_BYTES = new byte[10 * 1024]; static KeyValue BIG_CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, BIG_CELL_BYTES); static final Configuration CONF = HBaseConfiguration.create(); - // We are using the test TestRpcServiceProtos generated classes and Service because they are - // available and basic with methods like 'echo', and ping. Below we make a blocking service - // by passing in implementation of blocking interface. We use this service in all tests that - // follow. - static final BlockingService SERVICE = - TestRpcServiceProtos.TestProtobufRpcProto - .newReflectiveBlockingService(new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() { - - @Override - public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public EmptyResponseProto error(RpcController controller, EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public EchoResponseProto echo(RpcController controller, EchoRequestProto request) - throws ServiceException { - if (controller instanceof PayloadCarryingRpcController) { - PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController) controller; - // If cells, scan them to check we are able to iterate what we were given and since - // this is - // an echo, just put them back on the controller creating a new block. Tests our - // block - // building. - CellScanner cellScanner = pcrc.cellScanner(); - List list = null; - if (cellScanner != null) { - list = new ArrayList(); - try { - while (cellScanner.advance()) { - list.add(cellScanner.current()); - } - } catch (IOException e) { - throw new ServiceException(e); - } - } - cellScanner = CellUtil.createCellScanner(list); - ((PayloadCarryingRpcController) controller).setCellScanner(cellScanner); - } - return EchoResponseProto.newBuilder().setMessage(request.getMessage()).build(); - } - }); /** * Instance of server. We actually don't do anything speical in here so could just use @@ -145,149 +92,106 @@ public abstract class AbstractTestIPC { } TestRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException { - super(null, "testRpcServer", Lists - .newArrayList(new BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress( - "localhost", 0), conf, scheduler); - } - - @Override - public Pair call(BlockingService service, MethodDescriptor md, - Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status) - throws IOException { - return super.call(service, md, param, cellScanner, receiveTime, status); + super(null, "testRpcServer", + Lists.newArrayList(new BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress("localhost", 0), conf, scheduler); } } - protected abstract AbstractRpcClient createRpcClientNoCodec(Configuration conf); + protected abstract AbstractRpcClient createRpcClientNoCodec(Configuration conf); /** * Ensure we do not HAVE TO HAVE a codec. - * @throws InterruptedException - * @throws IOException */ @Test - public void testNoCodec() throws InterruptedException, IOException { + public void testNoCodec() throws IOException, ServiceException { Configuration conf = HBaseConfiguration.create(); - AbstractRpcClient client = createRpcClientNoCodec(conf); TestRpcServer rpcServer = new TestRpcServer(); - try { + try (AbstractRpcClient client = createRpcClientNoCodec(conf)) { rpcServer.start(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - final String message = "hello"; - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message).build(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - Pair r = - client.call(null, md, param, md.getOutputType().toProto(), User.getCurrent(), address, - new MetricsConnection.CallStats()); - assertTrue(r.getSecond() == null); - // Silly assertion that the message is in the returned pb. - assertTrue(r.getFirst().toString().contains(message)); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + String message = "hello"; + assertEquals(message, + stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage(message).build()).getMessage()); + assertNull(pcrc.cellScanner()); } finally { - client.close(); rpcServer.stop(); } } - protected abstract AbstractRpcClient createRpcClient(Configuration conf); + protected abstract AbstractRpcClient createRpcClient(Configuration conf); /** * It is hard to verify the compression is actually happening under the wraps. Hope that if * unsupported, we'll get an exception out of some time (meantime, have to trace it manually to * confirm that compression is happening down in the client and server). - * @throws IOException - * @throws InterruptedException - * @throws SecurityException - * @throws NoSuchMethodException */ @Test - public void testCompressCellBlock() throws IOException, InterruptedException, SecurityException, - NoSuchMethodException, ServiceException { + public void testCompressCellBlock() throws IOException, ServiceException { Configuration conf = new Configuration(HBaseConfiguration.create()); - conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName()); - List cells = new ArrayList(); + // conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName()); + List cells = new ArrayList<>(); int count = 3; for (int i = 0; i < count; i++) { cells.add(CELL); } - AbstractRpcClient client = createRpcClient(conf); TestRpcServer rpcServer = new TestRpcServer(); - try { + try (AbstractRpcClient client = createRpcClient(conf)) { rpcServer.start(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - PayloadCarryingRpcController pcrc = - new PayloadCarryingRpcController(CellUtil.createCellScanner(cells)); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - Pair r = - client.call(pcrc, md, param, md.getOutputType().toProto(), User.getCurrent(), address, - new MetricsConnection.CallStats()); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(CellUtil.createCellScanner(cells)); + String message = "hello"; + assertEquals(message, + stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage(message).build()).getMessage()); int index = 0; - while (r.getSecond().advance()) { - assertTrue(CELL.equals(r.getSecond().current())); + CellScanner cellScanner = pcrc.cellScanner(); + assertNotNull(cellScanner); + while (cellScanner.advance()) { + assertEquals(CELL, cellScanner.current()); index++; } assertEquals(count, index); } finally { - client.close(); rpcServer.stop(); } } - protected abstract AbstractRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) - throws IOException; + protected abstract AbstractRpcClient createRpcClientRTEDuringConnectionSetup( + Configuration conf) throws IOException; @Test public void testRTEDuringConnectionSetup() throws Exception { Configuration conf = HBaseConfiguration.create(); TestRpcServer rpcServer = new TestRpcServer(); - AbstractRpcClient client = createRpcClientRTEDuringConnectionSetup(conf); - try { + try (AbstractRpcClient client = createRpcClientRTEDuringConnectionSetup(conf)) { rpcServer.start(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - client.call(null, md, param, null, User.getCurrent(), address, - new MetricsConnection.CallStats()); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + stub.ping(null, EmptyRequestProto.getDefaultInstance()); fail("Expected an exception to have been thrown!"); } catch (Exception e) { LOG.info("Caught expected exception: " + e.toString()); - assertTrue(StringUtils.stringifyException(e).contains("Injected fault")); + assertTrue(e.toString(), StringUtils.stringifyException(e).contains("Injected fault")); } finally { - client.close(); rpcServer.stop(); } } - /** Tests that the rpc scheduler is called when requests arrive. */ + /** + * Tests that the rpc scheduler is called when requests arrive. + */ @Test - public void testRpcScheduler() throws IOException, InterruptedException { + public void testRpcScheduler() throws IOException, ServiceException, InterruptedException { RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1)); RpcServer rpcServer = new TestRpcServer(scheduler, CONF); verify(scheduler).init((RpcScheduler.Context) anyObject()); - AbstractRpcClient client = createRpcClient(CONF); - try { + try (AbstractRpcClient client = createRpcClient(CONF)) { rpcServer.start(); verify(scheduler).start(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } for (int i = 0; i < 10; i++) { - client.call(new PayloadCarryingRpcController( - CellUtil.createCellScanner(ImmutableList. of(CELL))), md, param, - md.getOutputType().toProto(), User.getCurrent(), address, - new MetricsConnection.CallStats()); + stub.echo(null, param); } verify(scheduler, times(10)).dispatch((CallRunner) anyObject()); } finally { @@ -298,109 +202,194 @@ public abstract class AbstractTestIPC { /** Tests that the rpc scheduler is called when requests arrive. */ @Test - public void testRpcMaxRequestSize() throws IOException, InterruptedException { + public void testRpcMaxRequestSize() throws IOException, ServiceException { Configuration conf = new Configuration(CONF); conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000); RpcServer rpcServer = new TestRpcServer(conf); - AbstractRpcClient client = createRpcClient(conf); - try { + try (AbstractRpcClient client = createRpcClient(conf)) { rpcServer.start(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); StringBuilder message = new StringBuilder(1200); for (int i = 0; i < 200; i++) { message.append("hello."); } - // set total RPC size bigger than 1000 bytes + // set total RPC size bigger than 100 bytes EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message.toString()).build(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - try { - client.call(new PayloadCarryingRpcController( - CellUtil.createCellScanner(ImmutableList. of(CELL))), md, param, - md.getOutputType().toProto(), User.getCurrent(), address, - new MetricsConnection.CallStats()); - fail("RPC should have failed because it exceeds max request size"); - } catch(IOException e) { - LOG.info("Caught expected exception: " + e); - assertTrue(e.toString(), - StringUtils.stringifyException(e).contains("RequestTooBigException")); - } + stub.echo( + new HBaseRpcControllerImpl(CellUtil.createCellScanner(ImmutableList. of(CELL))), + param); + fail("RPC should have failed because it exceeds max request size"); + } catch (ServiceException e) { + LOG.info("Caught expected exception: " + e); + assertTrue(e.toString(), + StringUtils.stringifyException(e).contains("RequestTooBigException")); } finally { rpcServer.stop(); } } - /** - * Instance of RpcServer that echoes client hostAddress back to client - */ - static class TestRpcServer1 extends RpcServer { - - private static BlockingInterface SERVICE1 = - new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() { - @Override - public EmptyResponseProto ping(RpcController unused, EmptyRequestProto request) - throws ServiceException { - return EmptyResponseProto.newBuilder().build(); - } - - @Override - public EchoResponseProto echo(RpcController unused, EchoRequestProto request) - throws ServiceException { - final InetAddress remoteAddr = TestRpcServer1.getRemoteAddress(); - final String message = remoteAddr == null ? "NULL" : remoteAddr.getHostAddress(); - return EchoResponseProto.newBuilder().setMessage(message).build(); - } - - @Override - public EmptyResponseProto error(RpcController unused, EmptyRequestProto request) - throws ServiceException { - throw new ServiceException("error", new IOException("error")); - } - }; - - TestRpcServer1() throws IOException { - this(new FifoRpcScheduler(CONF, 1)); - } - - TestRpcServer1(RpcScheduler scheduler) throws IOException { - super(null, "testRemoteAddressInCallObject", Lists - .newArrayList(new BlockingServiceAndInterface(TestRpcServiceProtos.TestProtobufRpcProto - .newReflectiveBlockingService(SERVICE1), null)), - new InetSocketAddress("localhost", 0), CONF, scheduler); - } - } - /** * Tests that the RpcServer creates & dispatches CallRunner object to scheduler with non-null * remoteAddress set to its Call Object * @throws ServiceException */ @Test - public void testRpcServerForNotNullRemoteAddressInCallObject() throws IOException, - ServiceException { - final RpcScheduler scheduler = new FifoRpcScheduler(CONF, 1); - final TestRpcServer1 rpcServer = new TestRpcServer1(scheduler); - final InetSocketAddress localAddr = new InetSocketAddress("localhost", 0); - final AbstractRpcClient client = - new RpcClientImpl(CONF, HConstants.CLUSTER_ID_DEFAULT, localAddr, null); + public void testRpcServerForNotNullRemoteAddressInCallObject() + throws IOException, ServiceException { + TestRpcServer rpcServer = new TestRpcServer(); + InetSocketAddress localAddr = new InetSocketAddress("localhost", 0); + try (AbstractRpcClient client = createRpcClient(CONF)) { + rpcServer.start(); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + assertEquals(localAddr.getAddress().getHostAddress(), + stub.addr(null, EmptyRequestProto.getDefaultInstance()).getAddr()); + } finally { + rpcServer.stop(); + } + } + + @Test + public void testRemoteError() throws IOException, ServiceException { + TestRpcServer rpcServer = new TestRpcServer(); + try (AbstractRpcClient client = createRpcClient(CONF)) { + rpcServer.start(); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + stub.error(null, EmptyRequestProto.getDefaultInstance()); + } catch (ServiceException e) { + LOG.info("Caught expected exception: " + e); + IOException ioe = ProtobufUtil.handleRemoteException(e); + assertTrue(ioe instanceof DoNotRetryIOException); + assertTrue(ioe.getMessage().contains("server error!")); + } finally { + rpcServer.stop(); + } + } + + @Test + public void testTimeout() throws IOException { + TestRpcServer rpcServer = new TestRpcServer(); + try (AbstractRpcClient client = createRpcClient(CONF)) { + rpcServer.start(); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + int ms = 1000; + int timeout = 100; + for (int i = 0; i < 10; i++) { + pcrc.reset(); + pcrc.setCallTimeout(timeout); + long startTime = System.nanoTime(); + try { + stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build()); + } catch (ServiceException e) { + long waitTime = (System.nanoTime() - startTime) / 1000000; + // expected + LOG.info("Caught expected exception: " + e); + IOException ioe = ProtobufUtil.handleRemoteException(e); + assertTrue(ioe.getCause() instanceof CallTimeoutException); + // confirm that we got exception before the actual pause. + assertTrue(waitTime < ms); + } + } + } finally { + rpcServer.stop(); + } + } + + static class TestFailingRpcServer extends TestRpcServer { + + TestFailingRpcServer() throws IOException { + this(new FifoRpcScheduler(CONF, 1), CONF); + } + + TestFailingRpcServer(Configuration conf) throws IOException { + this(new FifoRpcScheduler(conf, 1), conf); + } + + TestFailingRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException { + super(scheduler, conf); + } + + class FailingConnection extends Connection { + public FailingConnection(SocketChannel channel, long lastContact) { + super(channel, lastContact); + } + + @Override + protected void processRequest(ByteBuffer buf) throws IOException, InterruptedException { + // this will throw exception after the connection header is read, and an RPC is sent + // from client + throw new DoNotRetryIOException("Failing for test"); + } + } + + @Override + protected Connection getConnection(SocketChannel channel, long time) { + return new FailingConnection(channel, time); + } + } + + /** Tests that the connection closing is handled by the client with outstanding RPC calls */ + @Test + public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException, IOException { + Configuration conf = new Configuration(CONF); + RpcServer rpcServer = new TestFailingRpcServer(conf); + try (AbstractRpcClient client = createRpcClient(conf)) { + rpcServer.start(); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); + stub.echo(null, param); + fail("RPC should have failed because connection closed"); + } catch (ServiceException e) { + LOG.info("Caught expected exception: " + e.toString()); + } finally { + rpcServer.stop(); + } + } + + @Test + public void testAsyncEcho() throws IOException { + Configuration conf = HBaseConfiguration.create(); + TestRpcServer rpcServer = new TestRpcServer(); + try (AbstractRpcClient client = createRpcClient(conf)) { + rpcServer.start(); + Interface stub = newStub(client, rpcServer.getListenerAddress()); + int num = 10; + List pcrcList = new ArrayList<>(); + List> callbackList = new ArrayList<>(); + for (int i = 0; i < num; i++) { + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + BlockingRpcCallback done = new BlockingRpcCallback<>(); + stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage("hello-" + i).build(), done); + pcrcList.add(pcrc); + callbackList.add(done); + } + for (int i = 0; i < num; i++) { + HBaseRpcController pcrc = pcrcList.get(i); + assertFalse(pcrc.failed()); + assertNull(pcrc.cellScanner()); + assertEquals("hello-" + i, callbackList.get(i).get().getMessage()); + } + } finally { + rpcServer.stop(); + } + } + + @Test + public void testAsyncRemoteError() throws IOException { + AbstractRpcClient client = createRpcClient(CONF); + TestRpcServer rpcServer = new TestRpcServer(); try { rpcServer.start(); - final InetSocketAddress isa = rpcServer.getListenerAddress(); - if (isa == null) { - throw new IOException("Listener channel is closed"); - } - final BlockingRpcChannel channel = - client.createBlockingRpcChannel( - ServerName.valueOf(isa.getHostName(), isa.getPort(), System.currentTimeMillis()), - User.getCurrent(), 0); - TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub = - TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(channel); - final EchoRequestProto echoRequest = - EchoRequestProto.newBuilder().setMessage("GetRemoteAddress").build(); - final EchoResponseProto echoResponse = stub.echo(null, echoRequest); - Assert.assertEquals(localAddr.getAddress().getHostAddress(), echoResponse.getMessage()); + Interface stub = newStub(client, rpcServer.getListenerAddress()); + BlockingRpcCallback callback = new BlockingRpcCallback<>(); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + stub.error(pcrc, EmptyRequestProto.getDefaultInstance(), callback); + assertNull(callback.get()); + assertTrue(pcrc.failed()); + LOG.info("Caught expected exception: " + pcrc.getFailed()); + IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed()); + assertTrue(ioe instanceof DoNotRetryIOException); + assertTrue(ioe.getMessage().contains("server error!")); } finally { client.close(); rpcServer.stop(); @@ -408,17 +397,38 @@ public abstract class AbstractTestIPC { } @Test - public void testWrapException() throws Exception { - AbstractRpcClient client = - (AbstractRpcClient) RpcClientFactory.createClient(CONF, "AbstractTestIPC"); - final InetSocketAddress address = InetSocketAddress.createUnresolved("localhost", 0); - assertTrue(client.wrapException(address, new ConnectException()) instanceof ConnectException); - assertTrue(client.wrapException(address, - new SocketTimeoutException()) instanceof SocketTimeoutException); - assertTrue(client.wrapException(address, new ConnectionClosingException( - "Test AbstractRpcClient#wrapException")) instanceof ConnectionClosingException); - assertTrue(client - .wrapException(address, new CallTimeoutException("Test AbstractRpcClient#wrapException")) - .getCause() instanceof CallTimeoutException); + public void testAsyncTimeout() throws IOException { + TestRpcServer rpcServer = new TestRpcServer(); + try (AbstractRpcClient client = createRpcClient(CONF)) { + rpcServer.start(); + Interface stub = newStub(client, rpcServer.getListenerAddress()); + List pcrcList = new ArrayList<>(); + List> callbackList = new ArrayList<>(); + int ms = 1000; + int timeout = 100; + long startTime = System.nanoTime(); + for (int i = 0; i < 10; i++) { + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + pcrc.setCallTimeout(timeout); + BlockingRpcCallback callback = new BlockingRpcCallback<>(); + stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build(), callback); + pcrcList.add(pcrc); + callbackList.add(callback); + } + for (BlockingRpcCallback callback : callbackList) { + assertNull(callback.get()); + } + long waitTime = (System.nanoTime() - startTime) / 1000000; + for (HBaseRpcController pcrc : pcrcList) { + assertTrue(pcrc.failed()); + LOG.info("Caught expected exception: " + pcrc.getFailed()); + IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed()); + assertTrue(ioe.getCause() instanceof CallTimeoutException); + } + // confirm that we got exception before the actual pause. + assertTrue(waitTime < ms); + } finally { + rpcServer.stop(); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java deleted file mode 100644 index d9b3e494e90..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java +++ /dev/null @@ -1,306 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInitializer; -import io.netty.channel.ChannelOutboundHandlerAdapter; -import io.netty.channel.ChannelPromise; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScannable; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.RowMutations; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.util.StringUtils; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors.MethodDescriptor; -import com.google.protobuf.Message; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcChannel; - -@RunWith(Parameterized.class) -@Category({ SmallTests.class }) -public class TestAsyncIPC extends AbstractTestIPC { - - private static final Log LOG = LogFactory.getLog(TestAsyncIPC.class); - - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - @Parameters - public static Collection parameters() { - List paramList = new ArrayList(); - paramList.add(new Object[] { false, false }); - paramList.add(new Object[] { false, true }); - paramList.add(new Object[] { true, false }); - paramList.add(new Object[] { true, true }); - return paramList; - } - - private final boolean useNativeTransport; - - private final boolean useGlobalEventLoopGroup; - - public TestAsyncIPC(boolean useNativeTransport, boolean useGlobalEventLoopGroup) { - this.useNativeTransport = useNativeTransport; - this.useGlobalEventLoopGroup = useGlobalEventLoopGroup; - } - - private void setConf(Configuration conf) { - conf.setBoolean(AsyncRpcClient.USE_NATIVE_TRANSPORT, useNativeTransport); - conf.setBoolean(AsyncRpcClient.USE_GLOBAL_EVENT_LOOP_GROUP, useGlobalEventLoopGroup); - if (useGlobalEventLoopGroup && AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP != null) { - if (useNativeTransport - && !(AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP.getFirst() instanceof EpollEventLoopGroup) - || (!useNativeTransport - && !(AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP.getFirst() instanceof NioEventLoopGroup))) { - AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP.getFirst().shutdownGracefully(); - AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP = null; - } - } - } - - @Override - protected AsyncRpcClient createRpcClientNoCodec(Configuration conf) { - setConf(conf); - return new AsyncRpcClient(conf) { - - @Override - Codec getCodec() { - return null; - } - - }; - } - - @Override - protected AsyncRpcClient createRpcClient(Configuration conf) { - setConf(conf); - return new AsyncRpcClient(conf); - } - - @Override - protected AsyncRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) { - setConf(conf); - return new AsyncRpcClient(conf, new ChannelInitializer() { - @Override - protected void initChannel(SocketChannel ch) throws Exception { - ch.pipeline().addFirst(new ChannelOutboundHandlerAdapter() { - @Override - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) - throws Exception { - promise.setFailure(new RuntimeException("Injected fault")); - } - }); - } - }); - } - - @Test - public void testAsyncConnectionSetup() throws Exception { - TestRpcServer rpcServer = new TestRpcServer(); - AsyncRpcClient client = createRpcClient(CONF); - try { - rpcServer.start(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - - RpcChannel channel = - client.createRpcChannel(ServerName.valueOf(address.getHostName(), address.getPort(), - System.currentTimeMillis()), User.getCurrent(), 0); - - final AtomicBoolean done = new AtomicBoolean(false); - - channel.callMethod(md, new PayloadCarryingRpcController(), param, md.getOutputType() - .toProto(), new RpcCallback() { - @Override - public void run(Message parameter) { - done.set(true); - } - }); - - TEST_UTIL.waitFor(1000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return done.get(); - } - }); - } finally { - client.close(); - rpcServer.stop(); - } - } - - @Test - public void testRTEDuringAsyncConnectionSetup() throws Exception { - TestRpcServer rpcServer = new TestRpcServer(); - AsyncRpcClient client = createRpcClientRTEDuringConnectionSetup(CONF); - try { - rpcServer.start(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - - RpcChannel channel = - client.createRpcChannel(ServerName.valueOf(address.getHostName(), address.getPort(), - System.currentTimeMillis()), User.getCurrent(), 0); - - final AtomicBoolean done = new AtomicBoolean(false); - - PayloadCarryingRpcController controller = new PayloadCarryingRpcController(); - controller.notifyOnFail(new RpcCallback() { - @Override - public void run(IOException e) { - done.set(true); - LOG.info("Caught expected exception: " + e.toString()); - assertTrue(StringUtils.stringifyException(e).contains("Injected fault")); - } - }); - - channel.callMethod(md, controller, param, md.getOutputType().toProto(), - new RpcCallback() { - @Override - public void run(Message parameter) { - done.set(true); - fail("Expected an exception to have been thrown!"); - } - }); - - TEST_UTIL.waitFor(1000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return done.get(); - } - }); - } finally { - client.close(); - rpcServer.stop(); - } - } - - public static void main(String[] args) throws IOException, SecurityException, - NoSuchMethodException, InterruptedException { - if (args.length != 2) { - System.out.println("Usage: TestAsyncIPC "); - return; - } - // ((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.INFO); - // ((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.INFO); - int cycles = Integer.parseInt(args[0]); - int cellcount = Integer.parseInt(args[1]); - Configuration conf = HBaseConfiguration.create(); - TestRpcServer rpcServer = new TestRpcServer(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - AsyncRpcClient client = new AsyncRpcClient(conf); - KeyValue kv = BIG_CELL; - Put p = new Put(CellUtil.cloneRow(kv)); - for (int i = 0; i < cellcount; i++) { - p.add(kv); - } - RowMutations rm = new RowMutations(CellUtil.cloneRow(kv)); - rm.add(p); - try { - rpcServer.start(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - long startTime = System.currentTimeMillis(); - User user = User.getCurrent(); - for (int i = 0; i < cycles; i++) { - List cells = new ArrayList(); - // Message param = RequestConverter.buildMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm); - ClientProtos.RegionAction.Builder builder = - RequestConverter.buildNoDataRegionAction(HConstants.EMPTY_BYTE_ARRAY, rm, cells, - RegionAction.newBuilder(), ClientProtos.Action.newBuilder(), - MutationProto.newBuilder()); - builder.setRegion(RegionSpecifier - .newBuilder() - .setType(RegionSpecifierType.REGION_NAME) - .setValue( - ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes()))); - if (i % 100000 == 0) { - LOG.info("" + i); - // Uncomment this for a thread dump every so often. - // ReflectionUtils.printThreadInfo(new PrintWriter(System.out), - // "Thread dump " + Thread.currentThread().getName()); - } - PayloadCarryingRpcController pcrc = - new PayloadCarryingRpcController(CellUtil.createCellScanner(cells)); - // Pair response = - client.call(pcrc, md, builder.build(), param, user, address, - new MetricsConnection.CallStats()); - /* - * int count = 0; while (p.getSecond().advance()) { count++; } assertEquals(cells.size(), - * count); - */ - } - LOG.info("Cycled " + cycles + " time(s) with " + cellcount + " cell(s) in " - + (System.currentTimeMillis() - startTime) + "ms"); - } finally { - client.close(); - rpcServer.stop(); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java new file mode 100644 index 00000000000..98efcfbebb2 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java @@ -0,0 +1,58 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.experimental.categories.Category; + +@Category({ RPCTests.class, SmallTests.class }) +public class TestBlockingIPC extends AbstractTestIPC { + + @Override + protected BlockingRpcClient createRpcClientNoCodec(Configuration conf) { + return new BlockingRpcClient(conf) { + @Override + Codec getCodec() { + return null; + } + }; + } + + @Override + protected BlockingRpcClient createRpcClient(Configuration conf) { + return new BlockingRpcClient(conf); + } + + @Override + protected BlockingRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) + throws IOException { + return new BlockingRpcClient(conf) { + + @Override + boolean isTcpNoDelay() { + throw new RuntimeException("Injected fault"); + } + }; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java deleted file mode 100644 index e2948305ae0..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertSame; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category({ SmallTests.class }) -public class TestGlobalEventLoopGroup { - - @Test - public void test() { - Configuration conf = HBaseConfiguration.create(); - conf.setBoolean(AsyncRpcClient.USE_GLOBAL_EVENT_LOOP_GROUP, true); - AsyncRpcClient client = new AsyncRpcClient(conf); - assertNotNull(AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP); - AsyncRpcClient client1 = new AsyncRpcClient(conf); - assertSame(client.bootstrap.group(), client1.bootstrap.group()); - client1.close(); - assertFalse(client.bootstrap.group().isShuttingDown()); - - conf.setBoolean(AsyncRpcClient.USE_GLOBAL_EVENT_LOOP_GROUP, false); - AsyncRpcClient client2 = new AsyncRpcClient(conf); - assertNotSame(client.bootstrap.group(), client2.bootstrap.group()); - client2.close(); - - client.close(); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java deleted file mode 100644 index d3dbd33e5af..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java +++ /dev/null @@ -1,170 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.ipc; - -import static org.mockito.Matchers.anyInt; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.spy; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.util.ArrayList; -import java.util.List; - -import javax.net.SocketFactory; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScannable; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.RowMutations; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.net.NetUtils; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors.MethodDescriptor; - -@Category({ SmallTests.class }) -public class TestIPC extends AbstractTestIPC { - - private static final Log LOG = LogFactory.getLog(TestIPC.class); - - @Override - protected RpcClientImpl createRpcClientNoCodec(Configuration conf) { - return new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT) { - @Override - Codec getCodec() { - return null; - } - }; - } - - @Override - protected RpcClientImpl createRpcClient(Configuration conf) { - return new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT); - } - - @Override - protected RpcClientImpl createRpcClientRTEDuringConnectionSetup(Configuration conf) - throws IOException { - SocketFactory spyFactory = spy(NetUtils.getDefaultSocketFactory(conf)); - Mockito.doAnswer(new Answer() { - @Override - public Socket answer(InvocationOnMock invocation) throws Throwable { - Socket s = spy((Socket) invocation.callRealMethod()); - doThrow(new RuntimeException("Injected fault")).when(s).setSoTimeout(anyInt()); - return s; - } - }).when(spyFactory).createSocket(); - - return new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory); - } - - public static void main(String[] args) throws IOException, SecurityException, - NoSuchMethodException, InterruptedException { - if (args.length != 2) { - System.out.println("Usage: TestIPC "); - return; - } - // ((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.INFO); - // ((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.INFO); - int cycles = Integer.parseInt(args[0]); - int cellcount = Integer.parseInt(args[1]); - Configuration conf = HBaseConfiguration.create(); - TestRpcServer rpcServer = new TestRpcServer(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - RpcClientImpl client = new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT); - KeyValue kv = BIG_CELL; - Put p = new Put(CellUtil.cloneRow(kv)); - for (int i = 0; i < cellcount; i++) { - p.add(kv); - } - RowMutations rm = new RowMutations(CellUtil.cloneRow(kv)); - rm.add(p); - try { - rpcServer.start(); - long startTime = System.currentTimeMillis(); - User user = User.getCurrent(); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - for (int i = 0; i < cycles; i++) { - List cells = new ArrayList(); - // Message param = RequestConverter.buildMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm); - ClientProtos.RegionAction.Builder builder = - RequestConverter.buildNoDataRegionAction(HConstants.EMPTY_BYTE_ARRAY, rm, cells, - RegionAction.newBuilder(), ClientProtos.Action.newBuilder(), - MutationProto.newBuilder()); - builder.setRegion(RegionSpecifier - .newBuilder() - .setType(RegionSpecifierType.REGION_NAME) - .setValue( - ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes()))); - if (i % 100000 == 0) { - LOG.info("" + i); - // Uncomment this for a thread dump every so often. - // ReflectionUtils.printThreadInfo(new PrintWriter(System.out), - // "Thread dump " + Thread.currentThread().getName()); - } - PayloadCarryingRpcController pcrc = - new PayloadCarryingRpcController(CellUtil.createCellScanner(cells)); - // Pair response = - client.call(pcrc, md, builder.build(), param, user, address, - new MetricsConnection.CallStats()); - /* - * int count = 0; while (p.getSecond().advance()) { count++; } assertEquals(cells.size(), - * count); - */ - } - LOG.info("Cycled " + cycles + " time(s) with " + cellcount + " cell(s) in " - + (System.currentTimeMillis() - startTime) + "ms"); - } finally { - client.close(); - rpcServer.stop(); - } - } - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java new file mode 100644 index 00000000000..3b323833713 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.epoll.EpollSocketChannel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioSocketChannel; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.JVM; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +@Category({ RPCTests.class, SmallTests.class }) +public class TestNettyIPC extends AbstractTestIPC { + + @Parameters(name = "{index}: EventLoop={0}") + public static Collection parameters() { + List params = new ArrayList<>(); + params.add(new Object[] { "nio" }); + params.add(new Object[] { "perClientNio" }); + if (JVM.isLinux() && JVM.isAmd64()) { + params.add(new Object[] { "epoll" }); + } + return params; + } + + @Parameter + public String eventLoopType; + + private static NioEventLoopGroup NIO; + + private static EpollEventLoopGroup EPOLL; + + @BeforeClass + public static void setUpBeforeClass() { + NIO = new NioEventLoopGroup(); + if (JVM.isLinux() && JVM.isAmd64()) { + EPOLL = new EpollEventLoopGroup(); + } + } + + @AfterClass + public static void tearDownAfterClass() { + if (NIO != null) { + NIO.shutdownGracefully(); + } + if (EPOLL != null) { + EPOLL.shutdownGracefully(); + } + } + + private void setConf(Configuration conf) { + switch (eventLoopType) { + case "nio": + NettyRpcClientConfigHelper.setEventLoopConfig(conf, NIO, NioSocketChannel.class); + break; + case "epoll": + NettyRpcClientConfigHelper.setEventLoopConfig(conf, EPOLL, EpollSocketChannel.class); + break; + case "perClientNio": + NettyRpcClientConfigHelper.createEventLoopPerClient(conf); + break; + default: + break; + } + } + + @Override + protected NettyRpcClient createRpcClientNoCodec(Configuration conf) { + setConf(conf); + return new NettyRpcClient(conf) { + + @Override + Codec getCodec() { + return null; + } + + }; + } + + @Override + protected NettyRpcClient createRpcClient(Configuration conf) { + setConf(conf); + return new NettyRpcClient(conf); + } + + @Override + protected NettyRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) { + setConf(conf); + return new NettyRpcClient(conf) { + + @Override + boolean isTcpNoDelay() { + throw new RuntimeException("Injected fault"); + } + }; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java index ffb3927193f..3df4cdc310b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java @@ -17,41 +17,39 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.net.InetSocketAddress; -import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; -import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.log4j.Level; import org.apache.log4j.Logger; -import org.junit.Assert; -import org.junit.Test; -import org.junit.Before; import org.junit.After; +import org.junit.Before; +import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.protobuf.BlockingRpcChannel; -import com.google.protobuf.BlockingService; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** - * Test for testing protocol buffer based RPC mechanism. - * This test depends on test.proto definition of types in src/test/protobuf/test.proto - * and protobuf service definition from src/test/protobuf/test_rpc_service.proto + * Test for testing protocol buffer based RPC mechanism. This test depends on test.proto definition + * of types in src/test/protobuf/test.proto and protobuf service definition from + * src/test/protobuf/test_rpc_service.proto */ -@Category(MediumTests.class) +@Category({ RPCTests.class, MediumTests.class }) public class TestProtoBufRpc { public final static String ADDRESS = "localhost"; public static int PORT = 0; @@ -59,47 +57,18 @@ public class TestProtoBufRpc { private Configuration conf; private RpcServerInterface server; - /** - * Implementation of the test service defined out in TestRpcServiceProtos - */ - static class PBServerImpl - implements TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface { - @Override - public EmptyResponseProto ping(RpcController unused, - EmptyRequestProto request) throws ServiceException { - return EmptyResponseProto.newBuilder().build(); - } - - @Override - public EchoResponseProto echo(RpcController unused, EchoRequestProto request) - throws ServiceException { - return EchoResponseProto.newBuilder().setMessage(request.getMessage()) - .build(); - } - - @Override - public EmptyResponseProto error(RpcController unused, - EmptyRequestProto request) throws ServiceException { - throw new ServiceException("error", new IOException("error")); - } - } - @Before - public void setUp() throws IOException { // Setup server for both protocols + public void setUp() throws IOException { // Setup server for both protocols this.conf = HBaseConfiguration.create(); Logger log = Logger.getLogger("org.apache.hadoop.ipc.HBaseServer"); log.setLevel(Level.DEBUG); log = Logger.getLogger("org.apache.hadoop.ipc.HBaseServer.trace"); log.setLevel(Level.TRACE); // Create server side implementation - PBServerImpl serverImpl = new PBServerImpl(); - BlockingService service = - TestRpcServiceProtos.TestProtobufRpcProto.newReflectiveBlockingService(serverImpl); // Get RPC server for server side implementation this.server = new RpcServer(null, "testrpc", - Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)), - new InetSocketAddress(ADDRESS, PORT), conf, - new FifoRpcScheduler(conf, 10)); + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), + new InetSocketAddress(ADDRESS, PORT), conf, new FifoRpcScheduler(conf, 10)); InetSocketAddress address = server.getListenerAddress(); if (address == null) { throw new IOException("Listener channel is closed"); @@ -113,31 +82,23 @@ public class TestProtoBufRpc { server.stop(); } - @Test + @Test(expected = ServiceException.class + /* Thrown when we call stub.error */) public void testProtoBufRpc() throws Exception { RpcClient rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT); try { - BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel( - ServerName.valueOf(this.isa.getHostName(), this.isa.getPort(), System.currentTimeMillis()), - User.getCurrent(), 0); - TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub = - TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(channel); + BlockingInterface stub = newBlockingStub(rpcClient, this.isa); // Test ping method - TestProtos.EmptyRequestProto emptyRequest = - TestProtos.EmptyRequestProto.newBuilder().build(); + TestProtos.EmptyRequestProto emptyRequest = TestProtos.EmptyRequestProto.newBuilder().build(); stub.ping(null, emptyRequest); // Test echo method EchoRequestProto echoRequest = EchoRequestProto.newBuilder().setMessage("hello").build(); EchoResponseProto echoResponse = stub.echo(null, echoRequest); - Assert.assertEquals(echoResponse.getMessage(), "hello"); + assertEquals(echoResponse.getMessage(), "hello"); - // Test error method - error should be thrown as RemoteException - try { - stub.error(null, emptyRequest); - Assert.fail("Expected exception is not thrown"); - } catch (ServiceException e) { - } + stub.error(null, emptyRequest); + fail("Expected exception is not thrown"); } finally { rpcClient.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java new file mode 100644 index 00000000000..8f947b1bb79 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + + +import com.google.protobuf.BlockingService; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.Threads; + +@InterfaceAudience.Private +public class TestProtobufRpcServiceImpl implements BlockingInterface { + + public static final BlockingService SERVICE = TestProtobufRpcProto + .newReflectiveBlockingService(new TestProtobufRpcServiceImpl()); + + public static BlockingInterface newBlockingStub(RpcClient client, InetSocketAddress addr) + throws IOException { + return newBlockingStub(client, addr, User.getCurrent()); + } + + public static BlockingInterface newBlockingStub(RpcClient client, InetSocketAddress addr, + User user) throws IOException { + return TestProtobufRpcProto.newBlockingStub(client.createBlockingRpcChannel( + ServerName.valueOf(addr.getHostName(), addr.getPort(), System.currentTimeMillis()), user, 0)); + } + + public static Interface newStub(RpcClient client, InetSocketAddress addr) throws IOException { + return TestProtobufRpcProto.newStub(client.createRpcChannel( + ServerName.valueOf(addr.getHostName(), addr.getPort(), System.currentTimeMillis()), + User.getCurrent(), 0)); + } + + @Override + public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request) + throws ServiceException { + return EmptyResponseProto.getDefaultInstance(); + } + + @Override + public EchoResponseProto echo(RpcController controller, EchoRequestProto request) + throws ServiceException { + if (controller instanceof HBaseRpcController) { + HBaseRpcController pcrc = (HBaseRpcController) controller; + // If cells, scan them to check we are able to iterate what we were given and since this is an + // echo, just put them back on the controller creating a new block. Tests our block building. + CellScanner cellScanner = pcrc.cellScanner(); + List list = null; + if (cellScanner != null) { + list = new ArrayList<>(); + try { + while (cellScanner.advance()) { + list.add(cellScanner.current()); + } + } catch (IOException e) { + throw new ServiceException(e); + } + } + cellScanner = CellUtil.createCellScanner(list); + pcrc.setCellScanner(cellScanner); + } + return EchoResponseProto.newBuilder().setMessage(request.getMessage()).build(); + } + + @Override + public EmptyResponseProto error(RpcController controller, EmptyRequestProto request) + throws ServiceException { + throw new ServiceException(new DoNotRetryIOException("server error!")); + } + + @Override + public EmptyResponseProto pause(RpcController controller, PauseRequestProto request) + throws ServiceException { + Threads.sleepWithoutInterrupt(request.getMs()); + return EmptyResponseProto.getDefaultInstance(); + } + + @Override + public AddrResponseProto addr(RpcController controller, EmptyRequestProto request) + throws ServiceException { + return AddrResponseProto.newBuilder().setAddr(RpcServer.getRemoteAddress().getHostAddress()) + .build(); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java index 596b8ab5b49..e4ecd1004b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java @@ -17,15 +17,20 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; +import static org.junit.Assert.assertTrue; + +import com.google.common.collect.Lists; + import java.io.IOException; import java.net.Socket; import java.net.SocketAddress; import java.util.List; -import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -35,28 +40,26 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; +import org.junit.rules.TestRule; -import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; -import static org.junit.Assert.*; - -@Category(SmallTests.class) +@Category(MediumTests.class) public class TestRpcClientLeaks { + @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). + withLookingForStuckThread(true).build(); - public static class MyRpcClientImpl extends RpcClientImpl { + public static class MyRpcClientImpl extends BlockingRpcClient { public static List savedSockets = Lists.newArrayList(); @Rule public ExpectedException thrown = ExpectedException.none(); - public MyRpcClientImpl(Configuration conf, String clusterId) { - super(conf, clusterId); + public MyRpcClientImpl(Configuration conf) { + super(conf); } public MyRpcClientImpl(Configuration conf, String clusterId, SocketAddress address, @@ -65,9 +68,8 @@ public class TestRpcClientLeaks { } @Override - protected Connection createConnection(ConnectionId remoteId, Codec codec, - CompressionCodec compressor) throws IOException { - return new Connection(remoteId, codec, compressor) { + protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException { + return new BlockingRpcConnection(this, remoteId) { @Override protected synchronized void setupConnection() throws IOException { super.setupConnection(); @@ -113,5 +115,4 @@ public class TestRpcClientLeaks { assertTrue("Socket + " + socket + " is not closed", socket.isClosed()); } } -} - +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java index 03e9e4ec9ad..d197bf27e90 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java @@ -17,107 +17,31 @@ */ package org.apache.hadoop.hbase.ipc; -import com.google.common.collect.ImmutableList; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; +import static org.mockito.Mockito.mock; + import com.google.common.collect.Lists; import com.google.protobuf.BlockingService; -import com.google.protobuf.Descriptors.MethodDescriptor; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.client.MetricsConnection; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; -import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Pair; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; import java.io.IOException; import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.List; -import static org.mockito.Mockito.mock; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; +import org.apache.hadoop.hbase.testclassification.RPCTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.experimental.categories.Category; -@Category({SmallTests.class}) +@Category({ RPCTests.class, SmallTests.class }) public class TestRpcHandlerException { - private static final Log LOG = LogFactory.getLog(TestRpcHandlerException.class); - static String example = "xyz"; - static byte[] CELL_BYTES = example.getBytes(); - static Cell CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES); private final static Configuration CONF = HBaseConfiguration.create(); - RpcExecutor rpcExecutor = Mockito.mock(RpcExecutor.class); - - // We are using the test TestRpcServiceProtos generated classes and Service because they are - // available and basic with methods like 'echo', and ping. Below we make a blocking service - // by passing in implementation of blocking interface. We use this service in all tests that - // follow. - private static final BlockingService SERVICE = - TestRpcServiceProtos.TestProtobufRpcProto - .newReflectiveBlockingService(new TestRpcServiceProtos - .TestProtobufRpcProto.BlockingInterface() { - - @Override - public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public EmptyResponseProto error(RpcController controller, EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public EchoResponseProto echo(RpcController controller, EchoRequestProto request) - throws Error, RuntimeException { - if (controller instanceof PayloadCarryingRpcController) { - PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController) controller; - // If cells, scan them to check we are able to iterate what we were given and since - // this is - // an echo, just put them back on the controller creating a new block. Tests our - // block - // building. - CellScanner cellScanner = pcrc.cellScanner(); - List list = null; - if (cellScanner != null) { - list = new ArrayList(); - try { - while (cellScanner.advance()) { - list.add(cellScanner.current()); - throw new StackOverflowError(); - } - } catch (StackOverflowError e) { - throw e; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - cellScanner = CellUtil.createCellScanner(list); - ((PayloadCarryingRpcController) controller).setCellScanner(cellScanner); - } - return EchoResponseProto.newBuilder().setMessage(request.getMessage()).build(); - } - }); /** * Instance of server. We actually don't do anything speical in here so could just use @@ -125,29 +49,18 @@ public class TestRpcHandlerException { */ private static class TestRpcServer extends RpcServer { - TestRpcServer() throws IOException { - this(new FifoRpcScheduler(CONF, 1)); - } - TestRpcServer(RpcScheduler scheduler) throws IOException { super(null, "testRpcServer", - Lists.newArrayList(new BlockingServiceAndInterface(SERVICE, null)), - new InetSocketAddress("localhost", 0), CONF, scheduler); - } - - @Override - public Pair call(BlockingService service, MethodDescriptor md, - Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status) - throws IOException { - return super.call(service, md, param, cellScanner, receiveTime, status); + Lists.newArrayList(new BlockingServiceAndInterface((BlockingService) SERVICE, null)), + new InetSocketAddress("localhost", 0), CONF, scheduler); } } - /** Tests that the rpc scheduler is called when requests arrive. - * When Rpc handler thread dies, the client will hang and the test will fail. - * The test is meant to be a unit test to test the behavior. - * - * */ + /** + * Tests that the rpc scheduler is called when requests arrive. When Rpc handler thread dies, the + * client will hang and the test will fail. The test is meant to be a unit test to test the + * behavior. + */ private class AbortServer implements Abortable { private boolean aborted = false; @@ -162,7 +75,8 @@ public class TestRpcHandlerException { } } - /* This is a unit test to make sure to abort region server when the number of Rpc handler thread + /* + * This is a unit test to make sure to abort region server when the number of Rpc handler thread * caught errors exceeds the threshold. Client will hang when RS aborts. */ @Ignore @@ -172,21 +86,12 @@ public class TestRpcHandlerException { Abortable abortable = new AbortServer(); RpcScheduler scheduler = new SimpleRpcScheduler(CONF, 2, 0, 0, qosFunction, abortable, 0); RpcServer rpcServer = new TestRpcServer(scheduler); - RpcClientImpl client = new RpcClientImpl(CONF, HConstants.CLUSTER_ID_DEFAULT); - try { + try (BlockingRpcClient client = new BlockingRpcClient(CONF)) { rpcServer.start(); - MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); - EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - PayloadCarryingRpcController controller = - new PayloadCarryingRpcController(CellUtil.createCellScanner(ImmutableList.of(CELL))); - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - client.call(controller, md, param, md.getOutputType().toProto(), User.getCurrent(), - address, new MetricsConnection.CallStats()); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + stub.echo(null, EchoRequestProto.newBuilder().setMessage("hello").build()); } catch (Throwable e) { - assert(abortable.isAborted() == true); + assert (abortable.isAborted() == true); } finally { rpcServer.stop(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java index 3da0254b2d8..d28945c4019 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java @@ -1718,6 +1718,965 @@ public final class TestProtos { // @@protoc_insertion_point(class_scope:EchoResponseProto) } + public interface PauseRequestProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required uint32 ms = 1; + /** + * required uint32 ms = 1; + */ + boolean hasMs(); + /** + * required uint32 ms = 1; + */ + int getMs(); + } + /** + * Protobuf type {@code PauseRequestProto} + */ + public static final class PauseRequestProto extends + com.google.protobuf.GeneratedMessage + implements PauseRequestProtoOrBuilder { + // Use PauseRequestProto.newBuilder() to construct. + private PauseRequestProto(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private PauseRequestProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final PauseRequestProto defaultInstance; + public static PauseRequestProto getDefaultInstance() { + return defaultInstance; + } + + public PauseRequestProto getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private PauseRequestProto( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + ms_ = input.readUInt32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public PauseRequestProto parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new PauseRequestProto(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required uint32 ms = 1; + public static final int MS_FIELD_NUMBER = 1; + private int ms_; + /** + * required uint32 ms = 1; + */ + public boolean hasMs() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint32 ms = 1; + */ + public int getMs() { + return ms_; + } + + private void initFields() { + ms_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMs()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt32(1, ms_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(1, ms_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto) obj; + + boolean result = true; + result = result && (hasMs() == other.hasMs()); + if (hasMs()) { + result = result && (getMs() + == other.getMs()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasMs()) { + hash = (37 * hash) + MS_FIELD_NUMBER; + hash = (53 * hash) + getMs(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code PauseRequestProto} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + ms_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_descriptor; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto build() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto buildPartial() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.ms_ = ms_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto) { + return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto other) { + if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.getDefaultInstance()) return this; + if (other.hasMs()) { + setMs(other.getMs()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasMs()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required uint32 ms = 1; + private int ms_ ; + /** + * required uint32 ms = 1; + */ + public boolean hasMs() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint32 ms = 1; + */ + public int getMs() { + return ms_; + } + /** + * required uint32 ms = 1; + */ + public Builder setMs(int value) { + bitField0_ |= 0x00000001; + ms_ = value; + onChanged(); + return this; + } + /** + * required uint32 ms = 1; + */ + public Builder clearMs() { + bitField0_ = (bitField0_ & ~0x00000001); + ms_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:PauseRequestProto) + } + + static { + defaultInstance = new PauseRequestProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:PauseRequestProto) + } + + public interface AddrResponseProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string addr = 1; + /** + * required string addr = 1; + */ + boolean hasAddr(); + /** + * required string addr = 1; + */ + java.lang.String getAddr(); + /** + * required string addr = 1; + */ + com.google.protobuf.ByteString + getAddrBytes(); + } + /** + * Protobuf type {@code AddrResponseProto} + */ + public static final class AddrResponseProto extends + com.google.protobuf.GeneratedMessage + implements AddrResponseProtoOrBuilder { + // Use AddrResponseProto.newBuilder() to construct. + private AddrResponseProto(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private AddrResponseProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final AddrResponseProto defaultInstance; + public static AddrResponseProto getDefaultInstance() { + return defaultInstance; + } + + public AddrResponseProto getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private AddrResponseProto( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + addr_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public AddrResponseProto parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new AddrResponseProto(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required string addr = 1; + public static final int ADDR_FIELD_NUMBER = 1; + private java.lang.Object addr_; + /** + * required string addr = 1; + */ + public boolean hasAddr() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string addr = 1; + */ + public java.lang.String getAddr() { + java.lang.Object ref = addr_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + addr_ = s; + } + return s; + } + } + /** + * required string addr = 1; + */ + public com.google.protobuf.ByteString + getAddrBytes() { + java.lang.Object ref = addr_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + addr_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + addr_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasAddr()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getAddrBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getAddrBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto) obj; + + boolean result = true; + result = result && (hasAddr() == other.hasAddr()); + if (hasAddr()) { + result = result && getAddr() + .equals(other.getAddr()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasAddr()) { + hash = (37 * hash) + ADDR_FIELD_NUMBER; + hash = (53 * hash) + getAddr().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code AddrResponseProto} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + addr_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_descriptor; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto build() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto buildPartial() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.addr_ = addr_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto) { + return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto other) { + if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance()) return this; + if (other.hasAddr()) { + bitField0_ |= 0x00000001; + addr_ = other.addr_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasAddr()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required string addr = 1; + private java.lang.Object addr_ = ""; + /** + * required string addr = 1; + */ + public boolean hasAddr() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string addr = 1; + */ + public java.lang.String getAddr() { + java.lang.Object ref = addr_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + addr_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string addr = 1; + */ + public com.google.protobuf.ByteString + getAddrBytes() { + java.lang.Object ref = addr_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + addr_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string addr = 1; + */ + public Builder setAddr( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + addr_ = value; + onChanged(); + return this; + } + /** + * required string addr = 1; + */ + public Builder clearAddr() { + bitField0_ = (bitField0_ & ~0x00000001); + addr_ = getDefaultInstance().getAddr(); + onChanged(); + return this; + } + /** + * required string addr = 1; + */ + public Builder setAddrBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + addr_ = value; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:AddrResponseProto) + } + + static { + defaultInstance = new AddrResponseProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:AddrResponseProto) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_EmptyRequestProto_descriptor; private static @@ -1738,6 +2697,16 @@ public final class TestProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_EchoResponseProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_PauseRequestProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_PauseRequestProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_AddrResponseProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_AddrResponseProto_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -1750,8 +2719,10 @@ public final class TestProtos { "\n\ntest.proto\"\023\n\021EmptyRequestProto\"\024\n\022Emp" + "tyResponseProto\"#\n\020EchoRequestProto\022\017\n\007m" + "essage\030\001 \002(\t\"$\n\021EchoResponseProto\022\017\n\007mes" + - "sage\030\001 \002(\tB?\n.org.apache.hadoop.hbase.ip" + - "c.protobuf.generatedB\nTestProtos\240\001\001" + "sage\030\001 \002(\t\"\037\n\021PauseRequestProto\022\n\n\002ms\030\001 " + + "\002(\r\"!\n\021AddrResponseProto\022\014\n\004addr\030\001 \002(\tB?" + + "\n.org.apache.hadoop.hbase.ipc.protobuf.g" + + "eneratedB\nTestProtos\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -1782,6 +2753,18 @@ public final class TestProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EchoResponseProto_descriptor, new java.lang.String[] { "Message", }); + internal_static_PauseRequestProto_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_PauseRequestProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_PauseRequestProto_descriptor, + new java.lang.String[] { "Ms", }); + internal_static_AddrResponseProto_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_AddrResponseProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_AddrResponseProto_descriptor, + new java.lang.String[] { "Addr", }); return null; } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java index 3fd34e96a6e..a5b1777bc7a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java @@ -45,6 +45,22 @@ public final class TestRpcServiceProtos { org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, com.google.protobuf.RpcCallback done); + /** + * rpc pause(.PauseRequestProto) returns (.EmptyResponseProto); + */ + public abstract void pause( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto request, + com.google.protobuf.RpcCallback done); + + /** + * rpc addr(.EmptyRequestProto) returns (.AddrResponseProto); + */ + public abstract void addr( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done); + } public static com.google.protobuf.Service newReflectiveService( @@ -74,6 +90,22 @@ public final class TestRpcServiceProtos { impl.error(controller, request, done); } + @java.lang.Override + public void pause( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto request, + com.google.protobuf.RpcCallback done) { + impl.pause(controller, request, done); + } + + @java.lang.Override + public void addr( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done) { + impl.addr(controller, request, done); + } + }; } @@ -102,6 +134,10 @@ public final class TestRpcServiceProtos { return impl.echo(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto)request); case 2: return impl.error(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request); + case 3: + return impl.pause(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto)request); + case 4: + return impl.addr(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request); default: throw new java.lang.AssertionError("Can't get here."); } @@ -122,6 +158,10 @@ public final class TestRpcServiceProtos { return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDefaultInstance(); case 2: return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.getDefaultInstance(); + case 4: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -142,6 +182,10 @@ public final class TestRpcServiceProtos { return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(); case 2: return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + case 4: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -174,6 +218,22 @@ public final class TestRpcServiceProtos { org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, com.google.protobuf.RpcCallback done); + /** + * rpc pause(.PauseRequestProto) returns (.EmptyResponseProto); + */ + public abstract void pause( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto request, + com.google.protobuf.RpcCallback done); + + /** + * rpc addr(.EmptyRequestProto) returns (.AddrResponseProto); + */ + public abstract void addr( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done); + public static final com.google.protobuf.Descriptors.ServiceDescriptor getDescriptor() { @@ -211,6 +271,16 @@ public final class TestRpcServiceProtos { com.google.protobuf.RpcUtil.specializeCallback( done)); return; + case 3: + this.pause(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 4: + this.addr(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; default: throw new java.lang.AssertionError("Can't get here."); } @@ -231,6 +301,10 @@ public final class TestRpcServiceProtos { return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDefaultInstance(); case 2: return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto.getDefaultInstance(); + case 4: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -251,6 +325,10 @@ public final class TestRpcServiceProtos { return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(); case 2: return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + case 4: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -316,6 +394,36 @@ public final class TestRpcServiceProtos { org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance())); } + + public void pause( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(3), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance())); + } + + public void addr( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(4), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance())); + } } public static BlockingInterface newBlockingStub( @@ -338,6 +446,16 @@ public final class TestRpcServiceProtos { com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto pause( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto addr( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) + throws com.google.protobuf.ServiceException; } private static final class BlockingStub implements BlockingInterface { @@ -382,6 +500,30 @@ public final class TestRpcServiceProtos { org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()); } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto pause( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto) channel.callBlockingMethod( + getDescriptor().getMethods().get(3), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto addr( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto) channel.callBlockingMethod( + getDescriptor().getMethods().get(4), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance()); + } + } // @@protoc_insertion_point(class_scope:TestProtobufRpcProto) @@ -396,14 +538,16 @@ public final class TestRpcServiceProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\026test_rpc_service.proto\032\ntest.proto2\250\001\n" + + "\n\026test_rpc_service.proto\032\ntest.proto2\212\002\n" + "\024TestProtobufRpcProto\022/\n\004ping\022\022.EmptyReq" + "uestProto\032\023.EmptyResponseProto\022-\n\004echo\022\021" + ".EchoRequestProto\032\022.EchoResponseProto\0220\n" + "\005error\022\022.EmptyRequestProto\032\023.EmptyRespon" + - "seProtoBL\n.org.apache.hadoop.hbase.ipc.p" + - "rotobuf.generatedB\024TestRpcServiceProtos\210" + - "\001\001\240\001\001" + "seProto\0220\n\005pause\022\022.PauseRequestProto\032\023.E" + + "mptyResponseProto\022.\n\004addr\022\022.EmptyRequest" + + "Proto\032\022.AddrResponseProtoBL\n.org.apache." + + "hadoop.hbase.ipc.protobuf.generatedB\024Tes" + + "tRpcServiceProtos\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 5cb7e8f5119..14bd2fde344 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hbase.master; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; @@ -43,7 +47,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.executor.ExecutorService; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -66,8 +70,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsReques import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest; @@ -80,6 +82,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfiguratio import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse; @@ -111,9 +115,7 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; -import com.google.protobuf.ServiceException; +import net.sf.ehcache.search.Results; /** * A mock RegionServer implementation. @@ -405,7 +407,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { builder.addCellsPerResult(result.size()); List results = new ArrayList(1); results.add(result); - ((PayloadCarryingRpcController) controller).setCellScanner(CellUtil + ((HBaseRpcController) controller).setCellScanner(CellUtil .createCellScanner(results)); builder.setMoreResults(true); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index dfb01cbd545..b6e242490a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -22,6 +22,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Iterators; +import com.google.common.collect.Sets; +import com.google.protobuf.ServiceException; + import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -56,7 +60,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; @@ -73,10 +77,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Iterators; -import com.google.common.collect.Sets; -import com.google.protobuf.ServiceException; - @Category(LargeTests.class) public class TestEndToEndSplitTransaction { private static final Log LOG = LogFactory.getLog(TestEndToEndSplitTransaction.class); @@ -179,7 +179,7 @@ public class TestEndToEndSplitTransaction { regionName, new Scan(row), 1, true); try { server.getRSRpcServices().scan( - new PayloadCarryingRpcController(), scanRequest); + new HBaseRpcControllerImpl(), scanRequest); } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java deleted file mode 100644 index 6db0b75b476..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java +++ /dev/null @@ -1,301 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.security; - -import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting; -import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting; -import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertSame; - -import java.io.File; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Properties; - -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import org.apache.commons.lang.RandomStringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcServer; -import org.apache.hadoop.hbase.ipc.RpcServerInterface; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos; -import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; -import org.apache.hadoop.minikdc.MiniKdc; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.mockito.Mockito; - -import com.google.common.collect.Lists; -import com.google.protobuf.BlockingRpcChannel; -import com.google.protobuf.BlockingService; - -import javax.security.sasl.SaslException; - -public abstract class AbstractTestSecureIPC { - - private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - private static final File KEYTAB_FILE = new File(TEST_UTIL.getDataTestDir("keytab").toUri() - .getPath()); - - static final BlockingService SERVICE = - TestRpcServiceProtos.TestProtobufRpcProto.newReflectiveBlockingService( - new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() { - - @Override - public TestProtos.EmptyResponseProto ping(RpcController controller, - TestProtos.EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public TestProtos.EmptyResponseProto error(RpcController controller, - TestProtos.EmptyRequestProto request) - throws ServiceException { - return null; - } - - @Override - public TestProtos.EchoResponseProto echo(RpcController controller, - TestProtos.EchoRequestProto request) - throws ServiceException { - if (controller instanceof PayloadCarryingRpcController) { - PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController) controller; - // If cells, scan them to check we are able to iterate what we were given and since - // this is - // an echo, just put them back on the controller creating a new block. Tests our - // block - // building. - CellScanner cellScanner = pcrc.cellScanner(); - List list = null; - if (cellScanner != null) { - list = new ArrayList(); - try { - while (cellScanner.advance()) { - list.add(cellScanner.current()); - } - } catch (IOException e) { - throw new ServiceException(e); - } - } - cellScanner = CellUtil.createCellScanner(list); - ((PayloadCarryingRpcController) controller).setCellScanner(cellScanner); - } - return TestProtos.EchoResponseProto.newBuilder() - .setMessage(request.getMessage()).build(); - } - }); - - private static MiniKdc KDC; - private static String HOST = "localhost"; - private static String PRINCIPAL; - - String krbKeytab; - String krbPrincipal; - UserGroupInformation ugi; - Configuration clientConf; - Configuration serverConf; - - abstract Class getRpcClientClass(); - - @Rule - public ExpectedException exception = ExpectedException.none(); - - @BeforeClass - public static void setUp() throws Exception { - KDC = TEST_UTIL.setupMiniKdc(KEYTAB_FILE); - PRINCIPAL = "hbase/" + HOST; - KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL); - HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm()); - } - - @AfterClass - public static void tearDown() throws IOException { - if (KDC != null) { - KDC.stop(); - } - TEST_UTIL.cleanupTestDir(); - } - - @Before - public void setUpTest() throws Exception { - krbKeytab = getKeytabFileForTesting(); - krbPrincipal = getPrincipalForTesting(); - ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal); - clientConf = getSecuredConfiguration(); - clientConf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, getRpcClientClass().getName()); - serverConf = getSecuredConfiguration(); - } - - @Test - public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception { - UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); - - // check that the login user is okay: - assertSame(ugi, ugi2); - assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); - assertEquals(krbPrincipal, ugi.getUserName()); - - callRpcService(User.create(ugi2)); - } - - @Test - public void testRpcFallbackToSimpleAuth() throws Exception { - String clientUsername = "testuser"; - UserGroupInformation clientUgi = UserGroupInformation.createUserForTesting(clientUsername, - new String[]{clientUsername}); - - // check that the client user is insecure - assertNotSame(ugi, clientUgi); - assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod()); - assertEquals(clientUsername, clientUgi.getUserName()); - - clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple"); - serverConf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, true); - callRpcService(User.create(clientUgi)); - } - - void setRpcProtection(String clientProtection, String serverProtection) { - clientConf.set("hbase.rpc.protection", clientProtection); - serverConf.set("hbase.rpc.protection", serverProtection); - } - - /** - * Test various qpos of Server and Client. - * @throws Exception - */ - @Test - public void testSaslWithCommonQop() throws Exception { - setRpcProtection("authentication", "authentication"); - callRpcService(User.create(ugi)); - - setRpcProtection("integrity", "integrity"); - callRpcService(User.create(ugi)); - - setRpcProtection("privacy", "privacy"); - callRpcService(User.create(ugi)); - } - - private UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal) - throws Exception { - Configuration cnf = new Configuration(); - cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - UserGroupInformation.setConfiguration(cnf); - UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab); - return UserGroupInformation.getLoginUser(); - } - - /** - * Sets up a RPC Server and a Client. Does a RPC checks the result. If an exception is thrown - * from the stub, this function will throw root cause of that exception. - */ - private void callRpcService(User clientUser) throws Exception { - SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class); - Mockito.when(securityInfoMock.getServerPrincipal()) - .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL); - SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock); - - InetSocketAddress isa = new InetSocketAddress(HOST, 0); - - RpcServerInterface rpcServer = - new RpcServer(null, "AbstractTestSecureIPC", - Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), isa, - serverConf, new FifoRpcScheduler(serverConf, 1)); - rpcServer.start(); - try (RpcClient rpcClient = RpcClientFactory.createClient(clientConf, - HConstants.DEFAULT_CLUSTER_ID.toString())) { - InetSocketAddress address = rpcServer.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } - BlockingRpcChannel channel = - rpcClient.createBlockingRpcChannel( - ServerName.valueOf(address.getHostName(), address.getPort(), - System.currentTimeMillis()), clientUser, 0); - TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub = - TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(channel); - TestThread th1 = new TestThread(stub); - final Throwable exception[] = new Throwable[1]; - Collections.synchronizedList(new ArrayList()); - Thread.UncaughtExceptionHandler exceptionHandler = - new Thread.UncaughtExceptionHandler() { - public void uncaughtException(Thread th, Throwable ex) { - exception[0] = ex; - } - }; - th1.setUncaughtExceptionHandler(exceptionHandler); - th1.start(); - th1.join(); - if (exception[0] != null) { - // throw root cause. - while (exception[0].getCause() != null) { - exception[0] = exception[0].getCause(); - } - throw (Exception) exception[0]; - } - } finally { - rpcServer.stop(); - } - } - - public static class TestThread extends Thread { - private final TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub; - - public TestThread(TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub) { - this.stub = stub; - } - - @Override - public void run() { - try { - int[] messageSize = new int[] {100, 1000, 10000}; - for (int i = 0; i < messageSize.length; i++) { - String input = RandomStringUtils.random(messageSize[i]); - String result = stub.echo(null, TestProtos.EchoRequestProto.newBuilder() - .setMessage(input).build()).getMessage(); - assertEquals(input, result); - } - } catch (ServiceException e) { - throw new RuntimeException(e); - } - } - } -} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java index baaa9859774..b016bd30370 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java @@ -1,4 +1,5 @@ /** + * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -15,19 +16,259 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.security; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; +import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting; +import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; + +import com.google.common.collect.Lists; +import com.google.protobuf.BlockingService; +import com.google.protobuf.ServiceException; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +import javax.security.sasl.SaslException; + +import org.apache.commons.lang.RandomStringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ipc.BlockingRpcClient; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientImpl; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerInterface; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.minikdc.MiniKdc; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; +import org.mockito.Mockito; +@RunWith(Parameterized.class) @Category({ SecurityTests.class, SmallTests.class }) -public class TestSecureIPC extends AbstractTestSecureIPC { +public class TestSecureIPC { - Class getRpcClientClass() { - return RpcClientImpl.class; + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static final File KEYTAB_FILE = + new File(TEST_UTIL.getDataTestDir("keytab").toUri().getPath()); + + private static MiniKdc KDC; + private static String HOST = "localhost"; + private static String PRINCIPAL; + + String krbKeytab; + String krbPrincipal; + UserGroupInformation ugi; + Configuration clientConf; + Configuration serverConf; + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Parameters(name = "{index}: rpcClientImpl={0}") + public static Collection parameters() { + return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() }, + new Object[] { NettyRpcClient.class.getName() }); + } + + @Parameter + public String rpcClientImpl; + + @BeforeClass + public static void setUp() throws Exception { + KDC = TEST_UTIL.setupMiniKdc(KEYTAB_FILE); + PRINCIPAL = "hbase/" + HOST; + KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL); + HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm()); + } + + @AfterClass + public static void tearDown() throws IOException { + if (KDC != null) { + KDC.stop(); + } + TEST_UTIL.cleanupTestDir(); + } + + @Before + public void setUpTest() throws Exception { + krbKeytab = getKeytabFileForTesting(); + krbPrincipal = getPrincipalForTesting(); + ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal); + clientConf = getSecuredConfiguration(); + clientConf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, rpcClientImpl); + serverConf = getSecuredConfiguration(); + } + + @Test + public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception { + UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser(); + + // check that the login user is okay: + assertSame(ugi, ugi2); + assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod()); + assertEquals(krbPrincipal, ugi.getUserName()); + + callRpcService(User.create(ugi2)); + } + + @Test + public void testRpcFallbackToSimpleAuth() throws Exception { + String clientUsername = "testuser"; + UserGroupInformation clientUgi = + UserGroupInformation.createUserForTesting(clientUsername, new String[] { clientUsername }); + + // check that the client user is insecure + assertNotSame(ugi, clientUgi); + assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod()); + assertEquals(clientUsername, clientUgi.getUserName()); + + clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple"); + serverConf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, true); + callRpcService(User.create(clientUgi)); + } + + void setRpcProtection(String clientProtection, String serverProtection) { + clientConf.set("hbase.rpc.protection", clientProtection); + serverConf.set("hbase.rpc.protection", serverProtection); + } + + /** + * Test various combinations of Server and Client qops. + * @throws Exception + */ + @Test + public void testSaslWithCommonQop() throws Exception { + setRpcProtection("privacy,authentication", "authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("authentication", "privacy,authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("integrity,authentication", "privacy,authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("integrity,authentication", "integrity,authentication"); + callRpcService(User.create(ugi)); + + setRpcProtection("privacy,authentication", "privacy,authentication"); + callRpcService(User.create(ugi)); + } + + @Ignore + @Test + public void testSaslNoCommonQop() throws Exception { + exception.expect(SaslException.class); + exception.expectMessage("No common protection layer between client and server"); + setRpcProtection("integrity", "privacy"); + callRpcService(User.create(ugi)); + } + + private UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal) + throws Exception { + Configuration cnf = new Configuration(); + cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(cnf); + UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab); + return UserGroupInformation.getLoginUser(); + } + + /** + * Sets up a RPC Server and a Client. Does a RPC checks the result. If an exception is thrown from + * the stub, this function will throw root cause of that exception. + */ + private void callRpcService(User clientUser) throws Exception { + SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class); + Mockito.when(securityInfoMock.getServerPrincipal()) + .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL); + SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock); + + InetSocketAddress isa = new InetSocketAddress(HOST, 0); + + RpcServerInterface rpcServer = new RpcServer(null, "AbstractTestSecureIPC", + Lists.newArrayList( + new RpcServer.BlockingServiceAndInterface((BlockingService) SERVICE, null)), + isa, serverConf, new FifoRpcScheduler(serverConf, 1)); + rpcServer.start(); + try (RpcClient rpcClient = + RpcClientFactory.createClient(clientConf, HConstants.DEFAULT_CLUSTER_ID.toString())) { + BlockingInterface stub = + newBlockingStub(rpcClient, rpcServer.getListenerAddress(), clientUser); + TestThread th1 = new TestThread(stub); + final Throwable exception[] = new Throwable[1]; + Collections.synchronizedList(new ArrayList()); + Thread.UncaughtExceptionHandler exceptionHandler = new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread th, Throwable ex) { + exception[0] = ex; + } + }; + th1.setUncaughtExceptionHandler(exceptionHandler); + th1.start(); + th1.join(); + if (exception[0] != null) { + // throw root cause. + while (exception[0].getCause() != null) { + exception[0] = exception[0].getCause(); + } + throw (Exception) exception[0]; + } + } finally { + rpcServer.stop(); + } + } + + public static class TestThread extends Thread { + private final BlockingInterface stub; + + public TestThread(BlockingInterface stub) { + this.stub = stub; + } + + @Override + public void run() { + try { + int[] messageSize = new int[] { 100, 1000, 10000 }; + for (int i = 0; i < messageSize.length; i++) { + String input = RandomStringUtils.random(messageSize[i]); + String result = + stub.echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage(input).build()) + .getMessage(); + assertEquals(input, result); + } + } catch (ServiceException e) { + throw new RuntimeException(e); + } + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java index 60d4a6a44b6..aa8c7ee38f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java @@ -19,6 +19,9 @@ package org.apache.hadoop.hbase.security.token; import com.google.protobuf.ServiceException; + +import java.io.IOException; + import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -29,10 +32,10 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.ipc.AsyncRpcClient; +import org.apache.hadoop.hbase.ipc.BlockingRpcClient; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcClientImpl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; @@ -43,7 +46,6 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; @Category({ SecurityTests.class, MediumTests.class }) public class TestDelegationTokenWithEncryption extends SecureTestCluster { @@ -84,8 +86,8 @@ public class TestDelegationTokenWithEncryption extends SecureTestCluster { tableDescriptor.addFamily(new HColumnDescriptor("family")); admin.createTable(tableDescriptor); - testPutGetWithDelegationToken(RpcClientImpl.class); - testPutGetWithDelegationToken(AsyncRpcClient.class); + testPutGetWithDelegationToken(BlockingRpcClient.class); + testPutGetWithDelegationToken(NettyRpcClient.class); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index 1d7c676d470..25e252ab49e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.security.token; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.protobuf.ServiceException; + import java.io.IOException; import org.apache.hadoop.hbase.HConstants; @@ -27,11 +29,11 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.ipc.AsyncRpcClient; +import org.apache.hadoop.hbase.ipc.BlockingRpcClient; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; +import org.apache.hadoop.hbase.ipc.NettyRpcClient; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcClientImpl; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest; @@ -47,8 +49,6 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.protobuf.ServiceException; - @Category({ SecurityTests.class, MediumTests.class }) public class TestGenerateDelegationToken extends SecureTestCluster { @@ -79,8 +79,8 @@ public class TestGenerateDelegationToken extends SecureTestCluster { try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { Token token = TokenUtil.obtainToken(conn); UserGroupInformation.getCurrentUser().addToken(token); - testTokenAuth(RpcClientImpl.class); - testTokenAuth(AsyncRpcClient.class); + testTokenAuth(BlockingRpcClient.class); + testTokenAuth(NettyRpcClient.class); } } } diff --git a/hbase-server/src/test/protobuf/test.proto b/hbase-server/src/test/protobuf/test.proto index 566b04b5a09..72b68e9aae6 100644 --- a/hbase-server/src/test/protobuf/test.proto +++ b/hbase-server/src/test/protobuf/test.proto @@ -33,3 +33,11 @@ message EchoRequestProto { message EchoResponseProto { required string message = 1; } + +message PauseRequestProto { + required uint32 ms = 1; +} + +message AddrResponseProto { + required string addr = 1; +} diff --git a/hbase-server/src/test/protobuf/test_rpc_service.proto b/hbase-server/src/test/protobuf/test_rpc_service.proto index 4ed0380224b..5f91dc4df4c 100644 --- a/hbase-server/src/test/protobuf/test_rpc_service.proto +++ b/hbase-server/src/test/protobuf/test_rpc_service.proto @@ -30,4 +30,6 @@ service TestProtobufRpcProto { rpc ping(EmptyRequestProto) returns (EmptyResponseProto); rpc echo(EchoRequestProto) returns (EchoResponseProto); rpc error(EmptyRequestProto) returns (EmptyResponseProto); + rpc pause(PauseRequestProto) returns (EmptyResponseProto); + rpc addr(EmptyRequestProto) returns (AddrResponseProto); } diff --git a/pom.xml b/pom.xml index bdccd28bb7b..03f885a1d2a 100644 --- a/pom.xml +++ b/pom.xml @@ -1207,7 +1207,7 @@ 4.0.3 2.4.1 1.3.3 - 4.0.23.Final + 4.1.8.Final 3.6.2.Final 2.1.2 1.0.8