HBASE-8214 Remove proxy and engine, rely directly on pb generated Service

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1478637 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-05-03 03:52:15 +00:00
parent bdd511c7c1
commit d668363c23
120 changed files with 5516 additions and 5354 deletions

View File

@ -1,32 +0,0 @@
package org.apache.hadoop.hbase;
/**
* 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.
*/
/**
* Marker Interface used by ipc. We need a means of referring to
* ipc "protocols" generically. For example, we need to tell an rpc
* server the "protocols" it implements and it helps if all protocols
* implement a common 'type'. That is what this Interface is used for.
*/
// This Interface replaces the old VersionedProtocol Interface. Rather
// than redo a bunch of code its removal, instead we put in place this
// Interface and change all VP references to Protocol references.
// It is moved up here to top-level because it is ugly having members
// of super packages reach down into subpackages.
public interface IpcProtocol {}

View File

@ -1,36 +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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
import org.apache.hadoop.hbase.security.KerberosInfo;
import org.apache.hadoop.hbase.security.TokenInfo;
/**
* Protocol that a client uses to communicate with the Master (for admin purposes).
*/
@KerberosInfo(
serverPrincipal = "hbase.master.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface MasterAdminProtocol
extends MasterAdminService.BlockingInterface, MasterProtocol {}

View File

@ -1,36 +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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.MasterMonitorService;
import org.apache.hadoop.hbase.security.KerberosInfo;
import org.apache.hadoop.hbase.security.TokenInfo;
/**
* Protocol that a client uses to communicate with the Master (for monitoring purposes).
*/
@KerberosInfo(
serverPrincipal = "hbase.master.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface MasterMonitorProtocol
extends MasterMonitorService.BlockingInterface, MasterProtocol {}

View File

@ -98,7 +98,6 @@ public class RemoteExceptionHandler {
if (t instanceof IOException) {
i = (IOException) t;
} else {
i = new IOException("server error");
i.initCause(t);

View File

@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
@ -32,6 +31,7 @@ import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -265,7 +265,7 @@ public class CatalogTracker {
* @throws IOException
* @deprecated Use #getMetaServerConnection(long)
*/
public AdminProtocol waitForMetaServerConnection(long timeout)
public AdminService.BlockingInterface waitForMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getMetaServerConnection(timeout);
}
@ -281,7 +281,7 @@ public class CatalogTracker {
* @throws NotAllMetaRegionsOnlineException if timed out waiting
* @throws IOException
*/
AdminProtocol getMetaServerConnection(long timeout)
AdminService.BlockingInterface getMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getCachedConnection(waitForMeta(timeout));
}
@ -313,14 +313,14 @@ public class CatalogTracker {
* invocation, or may be null.
* @throws IOException
*/
private AdminProtocol getCachedConnection(ServerName sn)
private AdminService.BlockingInterface getCachedConnection(ServerName sn)
throws IOException {
if (sn == null) {
return null;
}
AdminProtocol protocol = null;
AdminService.BlockingInterface service = null;
try {
protocol = connection.getAdmin(sn);
service = connection.getAdmin(sn);
} catch (RetriesExhaustedException e) {
if (e.getCause() != null && e.getCause() instanceof ConnectException) {
// Catch this; presume it means the cached connection has gone bad.
@ -349,7 +349,7 @@ public class CatalogTracker {
}
}
return protocol;
return service;
}
/**
@ -367,7 +367,7 @@ public class CatalogTracker {
// rather than have to pass it in. Its made awkward by the fact that the
// HRI is likely a proxy against remote server so the getServerName needs
// to be fixed to go to a local method or to a cache before we can do this.
private boolean verifyRegionLocation(AdminProtocol hostingServer,
private boolean verifyRegionLocation(AdminService.BlockingInterface hostingServer,
final ServerName address, final byte [] regionName)
throws IOException {
if (hostingServer == null) {
@ -411,9 +411,9 @@ public class CatalogTracker {
*/
public boolean verifyMetaRegionLocation(final long timeout)
throws InterruptedException, IOException {
AdminProtocol connection = null;
AdminService.BlockingInterface service = null;
try {
connection = waitForMetaServerConnection(timeout);
service = waitForMetaServerConnection(timeout);
} catch (NotAllMetaRegionsOnlineException e) {
// Pass
} catch (ServerNotRunningYetException e) {
@ -421,8 +421,8 @@ public class CatalogTracker {
} catch (UnknownHostException e) {
// Pass -- server name doesn't resolve so it can't be assigned anything.
}
return (connection == null)? false:
verifyRegionLocation(connection,
return (service == null)? false:
verifyRegionLocation(service,
this.metaRegionTracker.getMetaRegionLocation(), META_REGION_NAME);
}

View File

@ -1,35 +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.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.security.KerberosInfo;
import org.apache.hadoop.hbase.security.TokenInfo;
/**
* Protocol that a HBase client uses to communicate with a region server.
*/
@KerberosInfo(
serverPrincipal = "hbase.regionserver.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Private
public interface AdminProtocol
extends AdminService.BlockingInterface, IpcProtocol {}

View File

@ -1,37 +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.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.security.KerberosInfo;
import org.apache.hadoop.hbase.security.TokenInfo;
/**
* Protocol that a HBase client uses to communicate with a region server.
*/
@KerberosInfo(
serverPrincipal = "hbase.regionserver.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ClientProtocol
extends ClientService.BlockingInterface, IpcProtocol {}

View File

@ -318,8 +318,8 @@ public class ClientScanner extends AbstractClientScanner {
if (retryAfterOutOfOrderException) {
retryAfterOutOfOrderException = false;
} else {
throw new DoNotRetryIOException("Failed after retry"
+ ", it could be cause by rpc timeout", e);
throw new DoNotRetryIOException("Failed after retry of " +
"OutOfOrderScannerNextException: was there a rpc timeout?", e);
}
}
// Clear region

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@ -78,6 +79,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRespo
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -570,7 +572,7 @@ public class HBaseAdmin implements Abortable, Closeable {
firstMetaServer.getRegionInfo().getRegionName(), scan, 1, true);
Result[] values = null;
// Get a batch at a time.
ClientProtocol server = connection.getClient(firstMetaServer.getServerName());
ClientService.BlockingInterface server = connection.getClient(firstMetaServer.getServerName());
try {
ScanResponse response = server.scan(null, request);
values = ResponseConverter.getResults(response);
@ -583,7 +585,7 @@ public class HBaseAdmin implements Abortable, Closeable {
if (values == null || values.length == 0) {
tableExists = false;
GetTableDescriptorsResponse htds;
MasterMonitorKeepAliveConnection master = connection.getKeepAliveMasterMonitor();
MasterMonitorKeepAliveConnection master = connection.getKeepAliveMasterMonitorService();
try {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(null);
@ -607,7 +609,7 @@ public class HBaseAdmin implements Abortable, Closeable {
if(tries == numRetries - 1) { // no more tries left
if (ex instanceof RemoteException) {
throw ((RemoteException) ex).unwrapRemoteException();
}else {
} else {
throw ex;
}
}
@ -1221,7 +1223,7 @@ public class HBaseAdmin implements Abortable, Closeable {
"The servername cannot be null or empty.");
}
ServerName sn = new ServerName(serverName);
AdminProtocol admin = this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
// Close the region without updating zk state.
CloseRegionRequest request =
RequestConverter.buildCloseRegionRequest(encodedRegionName, false);
@ -1246,8 +1248,7 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public void closeRegion(final ServerName sn, final HRegionInfo hri)
throws IOException {
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
// Close the region without updating zk state.
ProtobufUtil.closeRegion(admin, hri.getRegionName(), false);
}
@ -1257,8 +1258,7 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public List<HRegionInfo> getOnlineRegions(
final ServerName sn) throws IOException {
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
return ProtobufUtil.getOnlineRegions(admin);
}
@ -1320,8 +1320,7 @@ public class HBaseAdmin implements Abortable, Closeable {
private void flush(final ServerName sn, final HRegionInfo hri)
throws IOException {
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
FlushRegionRequest request =
RequestConverter.buildFlushRegionRequest(hri.getRegionName());
try {
@ -1490,8 +1489,7 @@ public class HBaseAdmin implements Abortable, Closeable {
private void compact(final ServerName sn, final HRegionInfo hri,
final boolean major, final byte [] family)
throws IOException {
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
CompactRegionRequest request =
RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
try {
@ -1518,10 +1516,11 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public void move(final byte [] encodedRegionName, final byte [] destServerName)
throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
try {
MoveRegionRequest request = RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
master.moveRegion(null,request);
MoveRegionRequest request =
RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
stub.moveRegion(null,request);
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof HBaseIOException) {
@ -1530,9 +1529,8 @@ public class HBaseAdmin implements Abortable, Closeable {
LOG.error("Unexpected exception: " + se + " from calling HMaster.moveRegion");
} catch (DeserializationException de) {
LOG.error("Could not parse destination server name: " + de);
}
finally {
master.close();
} finally {
stub.close();
}
}
@ -1587,7 +1585,7 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public void offline(final byte [] regionName)
throws IOException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdminService();
try {
master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
} catch (ServiceException se) {
@ -1605,11 +1603,11 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public boolean setBalancerRunning(final boolean on, final boolean synchronous)
throws MasterNotRunningException, ZooKeeperConnectionException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
try {
SetBalancerRunningRequest req =
RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
return master.setBalancerRunning(null, req).getPrevBalanceValue();
return stub.setBalancerRunning(null, req).getPrevBalanceValue();
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof MasterNotRunningException) {
@ -1623,7 +1621,7 @@ public class HBaseAdmin implements Abortable, Closeable {
// break interface by adding additional exception type.
throw new MasterNotRunningException("Unexpected exception when calling balanceSwitch",se);
} finally {
master.close();
stub.close();
}
}
@ -1635,11 +1633,11 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public boolean balancer()
throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
try {
return master.balance(null,RequestConverter.buildBalanceRequest()).getBalancerRan();
return stub.balance(null,RequestConverter.buildBalanceRequest()).getBalancerRan();
} finally {
master.close();
stub.close();
}
}
@ -1652,12 +1650,12 @@ public class HBaseAdmin implements Abortable, Closeable {
*/
public boolean enableCatalogJanitor(boolean enable)
throws ServiceException, MasterNotRunningException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
try {
return master.enableCatalogJanitor(null,
return stub.enableCatalogJanitor(null,
RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
} finally {
master.close();
stub.close();
}
}
@ -1668,12 +1666,12 @@ public class HBaseAdmin implements Abortable, Closeable {
* @throws MasterNotRunningException
*/
public int runCatalogScan() throws ServiceException, MasterNotRunningException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
try {
return master.runCatalogScan(null,
return stub.runCatalogScan(null,
RequestConverter.buildCatalogScanRequest()).getScanResult();
} finally {
master.close();
stub.close();
}
}
@ -1683,12 +1681,12 @@ public class HBaseAdmin implements Abortable, Closeable {
* @throws org.apache.hadoop.hbase.exceptions.MasterNotRunningException
*/
public boolean isCatalogJanitorEnabled() throws ServiceException, MasterNotRunningException {
MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
try {
return master.isCatalogJanitorEnabled(null,
return stub.isCatalogJanitorEnabled(null,
RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
} finally {
master.close();
stub.close();
}
}
@ -1704,7 +1702,7 @@ public class HBaseAdmin implements Abortable, Closeable {
final byte[] encodedNameOfRegionB, final boolean forcible)
throws IOException {
MasterAdminKeepAliveConnection master = connection
.getKeepAliveMasterAdmin();
.getKeepAliveMasterAdminService();
try {
DispatchMergingRegionsRequest request = RequestConverter
.buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
@ -1800,8 +1798,7 @@ public class HBaseAdmin implements Abortable, Closeable {
private void split(final ServerName sn, final HRegionInfo hri,
byte[] splitPoint) throws IOException {
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
ProtobufUtil.split(admin, hri, splitPoint);
}
@ -1924,7 +1921,7 @@ public class HBaseAdmin implements Abortable, Closeable {
throws IOException {
String hostname = Addressing.parseHostname(hostnamePort);
int port = Addressing.parsePort(hostnamePort);
AdminProtocol admin =
AdminService.BlockingInterface admin =
this.connection.getAdmin(new ServerName(hostname, port, 0));
StopServerRequest request = RequestConverter.buildStopServerRequest(
"Called by admin client " + this.connection.toString());
@ -2067,7 +2064,7 @@ public class HBaseAdmin implements Abortable, Closeable {
public synchronized byte[][] rollHLogWriter(String serverName)
throws IOException, FailedLogCloseException {
ServerName sn = new ServerName(serverName);
AdminProtocol admin = this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
try {
RollWALWriterResponse response = admin.rollWALWriter(null, request);
@ -2127,8 +2124,7 @@ public class HBaseAdmin implements Abortable, Closeable {
throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
} else {
ServerName sn = regionServerPair.getSecond();
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
regionServerPair.getFirst().getRegionName(), true);
GetRegionInfoResponse response = admin.getRegionInfo(null, request);
@ -2143,8 +2139,7 @@ public class HBaseAdmin implements Abortable, Closeable {
if (pair.getSecond() == null) continue;
try {
ServerName sn = pair.getSecond();
AdminProtocol admin =
this.connection.getAdmin(sn);
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
pair.getFirst().getRegionName(), true);
GetRegionInfoResponse response = admin.getRegionInfo(null, request);
@ -2607,7 +2602,7 @@ public class HBaseAdmin implements Abortable, Closeable {
* Create a {@link MasterAdminCallable} to use it.
*/
private <V> V execute(MasterAdminCallable<V> function) throws IOException {
function.masterAdmin = connection.getKeepAliveMasterAdmin();
function.masterAdmin = connection.getKeepAliveMasterAdminService();
try {
return executeCallable(function);
} finally {
@ -2621,7 +2616,7 @@ public class HBaseAdmin implements Abortable, Closeable {
* Create a {@link MasterAdminCallable} to use it.
*/
private <V> V execute(MasterMonitorCallable<V> function) throws IOException {
function.masterMonitor = connection.getKeepAliveMasterMonitor();
function.masterMonitor = connection.getKeepAliveMasterMonitorService();
try {
return executeCallable(function);
} finally {

View File

@ -0,0 +1,48 @@
/**
* Copyright The Apache Software Foundation
*
* 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.client;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
/**
* This class makes it convenient for one to execute a command in the context
* of a {@link HConnection} instance based on the given {@link Configuration}.
*
* <p>
* If you find yourself wanting to use a {@link HConnection} for a relatively
* short duration of time, and do not want to deal with the hassle of creating
* and cleaning up that resource, then you should consider using this
* convenience class.
*
* @param <T>
* the return type of the {@link HConnectable#connect(HConnection)}
* method.
*/
public abstract class HConnectable<T> {
public Configuration conf;
protected HConnectable(Configuration conf) {
this.conf = conf;
}
public abstract T connect(HConnection connection) throws IOException;
}

View File

@ -18,40 +18,43 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutorService;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterAdminProtocol;
import org.apache.hadoop.hbase.MasterMonitorProtocol;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.MasterMonitorService;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutorService;
/**
* Cluster connection. Hosts a connection to the ZooKeeper ensemble and
* thereafter into the HBase cluster. Knows how to locate regions out on the cluster,
* A cluster connection. Knows how to find the master, locate regions out on the cluster,
* keeps a cache of locations and then knows how to recalibrate after they move.
* {@link HConnectionManager} manages instances of this class.
* {@link HConnectionManager} manages instances of this class. This is NOT a connection to a
* particular server but to all servers in the cluster. An implementation takes care of individual
* connections at a lower level.
*
* <p>HConnections are used by {@link HTable} mostly but also by
* {@link HBaseAdmin}, {@link CatalogTracker},
* and {@link ZooKeeperWatcher}. HConnection instances can be shared. Sharing
* is usually what you want because rather than each HConnection instance
* having to do its own discovery of regions out on the cluster, instead, all
* clients get to share the one cache of locations. Sharing makes cleanup of
* HConnections awkward. See {@link HConnectionManager} for cleanup
* discussion.
* clients get to share the one cache of locations. {@link HConnectionManager} does the
* sharing for you if you go by it getting connections. Sharing makes cleanup of
* HConnections awkward. See {@link HConnectionManager} for cleanup discussion.
*
* @see HConnectionManager
*/
@ -213,29 +216,14 @@ public interface HConnection extends Abortable, Closeable {
final boolean offlined) throws IOException;
/**
* Returns a {@link MasterAdminProtocol} to the active master
* Returns a {@link MasterAdminKeepAliveConnection} to the active master
*/
public MasterAdminProtocol getMasterAdmin() throws IOException;
public MasterAdminService.BlockingInterface getMasterAdmin() throws IOException;
/**
* Returns an {@link MasterMonitorProtocol} to the active master
* Returns an {@link MasterMonitorKeepAliveConnection} to the active master
*/
public MasterMonitorProtocol getMasterMonitor() throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param hostname RegionServer hostname
* @param port RegionServer port
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
* @deprecated - use @link {#getAdmin(final ServerName serverName)} which takes into account
* the startCode
*/
@Deprecated
public AdminProtocol getAdmin(final String hostname, final int port)
throws IOException;
public MasterMonitorService.BlockingInterface getMasterMonitor() throws IOException;
/**
* Establishes a connection to the region server at the specified address.
@ -243,27 +231,10 @@ public interface HConnection extends Abortable, Closeable {
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
*/
public AdminProtocol getAdmin(final ServerName serverName)
throws IOException;
public AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
/**
* Establishes a connection to the region server at the specified address, and return
* a region client protocol.
*
* @param hostname RegionServer hostname
* @param port RegionServer port
* @return ClientProtocol proxy for RegionServer
* @throws IOException if a remote or network exception occurs
* @deprecated - use @link {#getClient(final ServerName serverName)} which takes into account
* the startCode
*/
@Deprecated
public ClientProtocol getClient(final String hostname, final int port)
throws IOException;
/**
* Establishes a connection to the region server at the specified address, and return
* Establishes a connection to the region server at the specified address, and returns
* a region client protocol.
*
* @param serverName
@ -271,30 +242,17 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
*
*/
public ClientProtocol getClient(final ServerName serverName) throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param hostname RegionServer hostname
* @param port RegionServer port
* @param getMaster - do we check if master is alive
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
* @deprecated use @link {#getAdmin(final ServerName serverName, boolean getMaster)}
* which takes into account the startCode.
*/
@Deprecated
public AdminProtocol getAdmin(final String hostname, final int port, boolean getMaster)
throws IOException;
public ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param serverName
* @param getMaster - do we check if master is alive
* @param getMaster do we check if master is alive
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
* @deprecated You can pass master flag but nothing special is done.
*/
public AdminProtocol getAdmin(final ServerName serverName, boolean getMaster)
public AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster)
throws IOException;
/**
@ -417,12 +375,13 @@ public interface HConnection extends Abortable, Closeable {
public void clearCaches(final ServerName sn);
/**
* This function allows HBaseAdminProtocol and potentially others to get a shared MasterMonitor
* This function allows HBaseAdmin and potentially others to get a shared MasterMonitor
* connection.
* @return The shared instance. Never returns null.
* @throws MasterNotRunningException
*/
public MasterMonitorKeepAliveConnection getKeepAliveMasterMonitor()
// TODO: Why is this in the public interface when the returned type is shutdown package access?
public MasterMonitorKeepAliveConnection getKeepAliveMasterMonitorService()
throws MasterNotRunningException;
/**
@ -431,7 +390,8 @@ public interface HConnection extends Abortable, Closeable {
* @return The shared instance. Never returns null.
* @throws MasterNotRunningException
*/
public MasterAdminKeepAliveConnection getKeepAliveMasterAdmin() throws MasterNotRunningException;
// TODO: Why is this in the public interface when the returned type is shutdown package access?
public MasterAdminKeepAliveConnection getKeepAliveMasterAdminService() throws MasterNotRunningException;
/**
* @param serverName
@ -439,4 +399,3 @@ public interface HConnection extends Abortable, Closeable {
*/
public boolean isDeadServer(ServerName serverName);
}

View File

@ -0,0 +1,140 @@
/**
* Copyright The Apache Software Foundation
*
* 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.client;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.security.User;
/**
* Denotes a unique key to an {@link HConnection} instance.
*
* In essence, this class captures the properties in {@link Configuration}
* that may be used in the process of establishing a connection. In light of
* that, if any new such properties are introduced into the mix, they must be
* added to the {@link HConnectionKey#properties} list.
*
*/
class HConnectionKey {
final static String[] CONNECTION_PROPERTIES = new String[] {
HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.ZOOKEEPER_CLIENT_PORT,
HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.HBASE_RPC_TIMEOUT_KEY,
HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
HConstants.HBASE_META_SCANNER_CACHING,
HConstants.HBASE_CLIENT_INSTANCE_ID };
private Map<String, String> properties;
private String username;
HConnectionKey(Configuration conf) {
Map<String, String> m = new HashMap<String, String>();
if (conf != null) {
for (String property : CONNECTION_PROPERTIES) {
String value = conf.get(property);
if (value != null) {
m.put(property, value);
}
}
}
this.properties = Collections.unmodifiableMap(m);
try {
User currentUser = User.getCurrent();
if (currentUser != null) {
username = currentUser.getName();
}
} catch (IOException ioe) {
HConnectionManager.LOG.warn("Error obtaining current user, skipping username in HConnectionKey", ioe);
}
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
if (username != null) {
result = username.hashCode();
}
for (String property : CONNECTION_PROPERTIES) {
String value = properties.get(property);
if (value != null) {
result = prime * result + value.hashCode();
}
}
return result;
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings (value="ES_COMPARING_STRINGS_WITH_EQ",
justification="Optimization")
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
HConnectionKey that = (HConnectionKey) obj;
if (this.username != null && !this.username.equals(that.username)) {
return false;
} else if (this.username == null && that.username != null) {
return false;
}
if (this.properties == null) {
if (that.properties != null) {
return false;
}
} else {
if (that.properties == null) {
return false;
}
for (String property : CONNECTION_PROPERTIES) {
String thisValue = this.properties.get(property);
String thatValue = that.properties.get(property);
//noinspection StringEquality
if (thisValue == thatValue) {
continue;
}
if (thisValue == null || !thisValue.equals(thatValue)) {
return false;
}
}
}
return true;
}
@Override
public String toString() {
return "HConnectionKey{" +
"properties=" + properties +
", username='" + username + '\'' +
'}';
}
}

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -534,7 +533,7 @@ public class HTable implements HTableInterface {
throws IOException {
return new ServerCallable<Result>(connection, tableName, row, operationTimeout) {
public Result call() throws IOException {
return ProtobufUtil.getRowOrBefore(server,
return ProtobufUtil.getRowOrBefore(stub,
location.getRegionInfo().getRegionName(), row, family);
}
}.withRetries();
@ -580,7 +579,7 @@ public class HTable implements HTableInterface {
public Result get(final Get get) throws IOException {
return new ServerCallable<Result>(connection, tableName, get.getRow(), operationTimeout) {
public Result call() throws IOException {
return ProtobufUtil.get(server,
return ProtobufUtil.get(stub,
location.getRegionInfo().getRegionName(), get);
}
}.withRetries();
@ -649,7 +648,7 @@ public class HTable implements HTableInterface {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), delete);
MutateResponse response = server.mutate(null, request);
MutateResponse response = stub.mutate(null, request);
return Boolean.valueOf(response.getProcessed());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
@ -726,7 +725,7 @@ public class HTable implements HTableInterface {
try {
MultiRequest request = RequestConverter.buildMultiRequest(
location.getRegionInfo().getRegionName(), rm);
server.multi(null, request);
stub.multi(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
@ -751,7 +750,7 @@ public class HTable implements HTableInterface {
location.getRegionInfo().getRegionName(), append);
PayloadCarryingRpcController rpcController =
new PayloadCarryingRpcController();
MutateResponse response = server.mutate(rpcController, request);
MutateResponse response = stub.mutate(rpcController, request);
if (!response.hasResult()) return null;
return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
} catch (ServiceException se) {
@ -776,7 +775,7 @@ public class HTable implements HTableInterface {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), increment);
PayloadCarryingRpcController rpcContoller = new PayloadCarryingRpcController();
MutateResponse response = server.mutate(rpcContoller, request);
MutateResponse response = stub.mutate(rpcContoller, request);
return ProtobufUtil.toResult(response.getResult(), rpcContoller.cellScanner());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
@ -821,7 +820,7 @@ public class HTable implements HTableInterface {
location.getRegionInfo().getRegionName(), row, family,
qualifier, amount, durability);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
MutateResponse response = server.mutate(rpcController, request);
MutateResponse response = stub.mutate(rpcController, request);
Result result =
ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
@ -846,7 +845,7 @@ public class HTable implements HTableInterface {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), row, family, qualifier,
new BinaryComparator(value), CompareType.EQUAL, put);
MutateResponse response = server.mutate(null, request);
MutateResponse response = stub.mutate(null, request);
return Boolean.valueOf(response.getProcessed());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
@ -870,7 +869,7 @@ public class HTable implements HTableInterface {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), row, family, qualifier,
new BinaryComparator(value), CompareType.EQUAL, delete);
MutateResponse response = server.mutate(null, request);
MutateResponse response = stub.mutate(null, request);
return Boolean.valueOf(response.getProcessed());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
@ -889,7 +888,7 @@ public class HTable implements HTableInterface {
try {
GetRequest request = RequestConverter.buildGetRequest(
location.getRegionInfo().getRegionName(), get, true);
GetResponse response = server.get(null, request);
GetResponse response = stub.get(null, request);
return response.getExists();
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
@ -992,7 +991,7 @@ public class HTable implements HTableInterface {
try {
MultiGetRequest requests = RequestConverter.buildMultiGetRequest(location
.getRegionInfo().getRegionName(), getsByRegionEntry.getValue(), true, false);
MultiGetResponse responses = server.multiGet(null, requests);
MultiGetResponse responses = stub.multiGet(null, requests);
return responses.getExistsList();
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);

View File

@ -20,25 +20,25 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.MasterAdminProtocol;
import java.io.Closeable;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
/**
* A KeepAlive connection is not physically closed immediately after the close,
* but rather kept alive for a few minutes. It makes sense only if it's shared.
* but rather kept alive for a few minutes. It makes sense only if it is shared.
*
* This interface is used by a dynamic proxy. It allows to have a #close
* function in a master client.
* <p>This interface is implemented on a stub. It allows to have a #close function in a master
* client.
*
* This class is intended to be used internally by HBase classes that need to
* speak the MasterAdminProtocol; but not by * final user code. Hence it's
* package protected.
* <p>This class is intended to be used internally by HBase classes that need to make invocations
* against the master on the MasterAdminProtos.MasterAdminService.BlockingInterface; but not by
* final user code. Hence it's package protected.
*/
interface MasterAdminKeepAliveConnection extends MasterAdminProtocol, Closeable {
@Override
interface MasterAdminKeepAliveConnection
extends MasterAdminProtos.MasterAdminService.BlockingInterface {
/**
* Close down all resources.
*/
// The Closeable Interface wants to throw an IOE out of a close.
// Thats a PITA. Do this below instead of Closeable.
public void close();
}

View File

@ -20,11 +20,10 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.MasterMonitorProtocol;
import java.io.Closeable;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos;
/**
* A KeepAlive connection is not physically closed immediately after the close,
* but rather kept alive for a few minutes. It makes sense only if it's shared.
@ -36,9 +35,5 @@ import java.io.Closeable;
* speak the MasterMonitorProtocol; but not by final user code. Hence it's
* package protected.
*/
interface MasterMonitorKeepAliveConnection extends MasterMonitorProtocol, Closeable {
@Override
public void close();
}
interface MasterMonitorKeepAliveConnection
extends MasterMonitorProtos.MasterMonitorService.BlockingInterface, Closeable {}

View File

@ -26,10 +26,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
import java.io.Closeable;
import java.io.IOException;
@ -275,7 +273,7 @@ public class MetaScanner {
public static List<HRegionInfo> listAllRegions(Configuration conf, final boolean offlined)
throws IOException {
final List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
MetaScannerVisitor visitor = new DefaultMetaScannerVisitor(conf) {
MetaScannerVisitor visitor = new DefaultMetaScannerVisitor() {
@Override
public boolean processRowInternal(Result result) throws IOException {
if (result == null || result.isEmpty()) {
@ -310,7 +308,7 @@ public class MetaScanner {
final byte [] tablename, final boolean offlined) throws IOException {
final NavigableMap<HRegionInfo, ServerName> regions =
new TreeMap<HRegionInfo, ServerName>();
MetaScannerVisitor visitor = new TableMetaScannerVisitor(conf, tablename) {
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tablename) {
@Override
public boolean processRowInternal(Result rowResult) throws IOException {
HRegionInfo info = getHRegionInfo(rowResult);
@ -354,10 +352,8 @@ public class MetaScanner {
public static abstract class DefaultMetaScannerVisitor
extends MetaScannerVisitorBase {
protected Configuration conf;
public DefaultMetaScannerVisitor(Configuration conf) {
this.conf = conf;
public DefaultMetaScannerVisitor() {
super();
}
public abstract boolean processRowInternal(Result rowResult) throws IOException;
@ -386,8 +382,8 @@ public class MetaScanner {
public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor {
private byte[] tableName;
public TableMetaScannerVisitor(Configuration conf, byte[] tableName) {
super(conf);
public TableMetaScannerVisitor(byte[] tableName) {
super();
this.tableName = tableName;
}
@ -402,6 +398,5 @@ public class MetaScanner {
}
return super.processRow(rowResult);
}
}
}

View File

@ -74,7 +74,7 @@ class MultiServerCallable<R> extends ServerCallable<MultiResponse> {
RequestConverter.buildNoDataMultiRequest(regionName, rms, cells);
// Carry the cells over the proxy/pb Service interface using the payload carrying
// rpc controller.
server.multi(new PayloadCarryingRpcController(cells), multiRequest);
stub.multi(new PayloadCarryingRpcController(cells), multiRequest);
// This multi call does not return results.
response.add(regionName, action.getOriginalIndex(), Result.EMPTY_RESULT);
} catch (ServiceException se) {
@ -99,7 +99,7 @@ class MultiServerCallable<R> extends ServerCallable<MultiResponse> {
// Controller optionally carries cell data over the proxy/service boundary and also
// optionally ferries cell response data back out again.
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
ClientProtos.MultiResponse responseProto = server.multi(controller, multiRequest);
ClientProtos.MultiResponse responseProto = stub.multi(controller, multiRequest);
results = ResponseConverter.getResults(responseProto, controller.cellScanner());
} catch (ServiceException se) {
ex = ProtobufUtil.getRemoteException(se);
@ -114,7 +114,7 @@ class MultiServerCallable<R> extends ServerCallable<MultiResponse> {
}
@Override
public void connect(boolean reload) throws IOException {
server = connection.getClient(loc.getServerName());
public void prepare(boolean reload) throws IOException {
stub = connection.getClient(loc.getServerName());
}
}

View File

@ -92,9 +92,9 @@ public class ScannerCallable extends ServerCallable<Result[]> {
* @throws IOException
*/
@Override
public void connect(boolean reload) throws IOException {
public void prepare(boolean reload) throws IOException {
if (!instantiated || reload) {
super.connect(reload);
super.prepare(reload);
checkIfRegionServerIsRemote();
instantiated = true;
}
@ -144,7 +144,7 @@ public class ScannerCallable extends ServerCallable<Result[]> {
RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq);
ScanResponse response = null;
try {
response = server.scan(null, request);
response = stub.scan(null, request);
// Client and RS maintain a nextCallSeq number during the scan. Every next() call
// from client to server will increment this number in both sides. Client passes this
// number along with the request and at RS side both the incoming nextCallSeq and its
@ -248,7 +248,7 @@ public class ScannerCallable extends ServerCallable<Result[]> {
ScanRequest request =
RequestConverter.buildScanRequest(this.scannerId, 0, true);
try {
server.scan(null, request);
stub.scan(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
@ -265,7 +265,7 @@ public class ScannerCallable extends ServerCallable<Result[]> {
this.location.getRegionInfo().getRegionName(),
this.scan, 0, false);
try {
ScanResponse response = server.scan(null, request);
ScanResponse response = stub.scan(null, request);
long id = response.getScannerId();
if (logScannerActivity) {
LOG.info("Open scanner=" + id + " for scan=" + scan.toString()

View File

@ -29,7 +29,8 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
@ -47,7 +48,7 @@ import java.util.concurrent.Callable;
* return type and method we actually invoke on remote Server. Usually
* used inside a try/catch that fields usual connection failures all wrapped
* up in a retry loop.
* <p>Call {@link #connect(boolean)} to connect to server hosting region
* <p>Call {@link #prepare(boolean)} to connect to server hosting region
* that contains the passed row in the passed table before invoking
* {@link #call()}.
* @see HConnection#getRegionServerWithoutRetries(ServerCallable)
@ -62,7 +63,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
protected final byte [] tableName;
protected final byte [] row;
protected HRegionLocation location;
protected ClientProtocol server;
protected ClientService.BlockingInterface stub;
protected int callTimeout;
protected long globalStartTime;
protected long startTime, endTime;
@ -86,13 +87,14 @@ public abstract class ServerCallable<T> implements Callable<T> {
}
/**
* Connect to the server hosting region with row from tablename.
* Prepare for connection to the server hosting region with row from tablename. Does lookup
* to find region location and hosting server.
* @param reload Set this to true if connection should re-find the region
* @throws IOException e
*/
public void connect(final boolean reload) throws IOException {
public void prepare(final boolean reload) throws IOException {
this.location = connection.getRegionLocation(tableName, row, reload);
this.server = connection.getClient(location.getServerName());
this.stub = connection.getClient(location.getServerName());
}
/** @return the server name
@ -127,11 +129,11 @@ public abstract class ServerCallable<T> implements Callable<T> {
// resetting to the minimum.
remaining = MIN_RPC_TIMEOUT;
}
HBaseClientRPC.setRpcTimeout(remaining);
RpcClient.setRpcTimeout(remaining);
}
public void afterCall() {
HBaseClientRPC.resetRpcTimeout();
RpcClient.resetRpcTimeout();
this.endTime = EnvironmentEdgeManager.currentTimeMillis();
}
@ -164,11 +166,11 @@ public abstract class ServerCallable<T> implements Callable<T> {
long expectedSleep = 0;
try {
beforeCall();
connect(tries != 0); // if called with false, check table status on ZK
prepare(tries != 0); // if called with false, check table status on ZK
return call();
} catch (Throwable t) {
LOG.warn("Received exception, tries=" + tries + ", numRetries=" + numRetries +
" message=" + t.getMessage());
LOG.warn("Received exception, tries=" + tries + ", numRetries=" + numRetries + ":" +
t.getMessage());
t = translateException(t);
// translateException throws an exception when we should not retry, i.e. when it's the
@ -237,7 +239,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
this.globalStartTime = EnvironmentEdgeManager.currentTimeMillis();
try {
beforeCall();
connect(false);
prepare(false);
return call();
} catch (Throwable t) {
Throwable t2 = translateException(t);

View File

@ -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
@ -18,17 +17,16 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
public class BadAuthException extends FatalConnectionException {
public BadAuthException() {
super();
}
import java.io.IOException;
public BadAuthException(String msg) {
super(msg);
}
/** An RPC implementation for the server. */
@InterfaceAudience.Private
interface RpcServerEngine {
/** Construct a server for a protocol implementation instance. */
RpcServer getServer(Object instance, Class<?>[] protocols,
String bindAddress, int port, int numHandlers, int metaHandlerCount,
boolean verbose, Configuration conf, int highPriorityLevel)
throws IOException;
public BadAuthException(String msg, Throwable t) {
super(msg, t);
}
}

View File

@ -1,152 +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.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
/**
* An RPC implementation. This class provides the client side.
*/
@InterfaceAudience.Private
public class HBaseClientRPC {
protected static final Log LOG =
LogFactory.getLog("org.apache.hadoop.ipc.HBaseClientRPC");
// thread-specific RPC timeout, which may override that of RpcEngine
private static ThreadLocal<Integer> rpcTimeout = new ThreadLocal<Integer>() {
@Override
protected Integer initialValue() {
return HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
}
};
/**
* @param protocol protocol interface
* @param addr address of remote service
* @param conf configuration
* @param maxAttempts max attempts
* @param rpcTimeout timeout for each RPC
* @param timeout timeout in milliseconds
* @return proxy
* @throws java.io.IOException e
*/
public static <T extends IpcProtocol> T waitForProxy(RpcClientEngine engine,
Class<T> protocol,
InetSocketAddress addr,
Configuration conf,
int maxAttempts,
int rpcTimeout,
long timeout)
throws IOException {
// HBase does limited number of reconnects which is different from hadoop.
long startTime = System.currentTimeMillis();
IOException ioe;
int reconnectAttempts = 0;
while (true) {
try {
return engine.getProxy(protocol, addr, conf, rpcTimeout);
} catch (SocketTimeoutException te) {
LOG.info("Problem connecting to server: " + addr);
ioe = te;
} catch (IOException ioex) {
// We only handle the ConnectException.
ConnectException ce = null;
if (ioex instanceof ConnectException) {
ce = (ConnectException) ioex;
ioe = ce;
} else if (ioex.getCause() != null
&& ioex.getCause() instanceof ConnectException) {
ce = (ConnectException) ioex.getCause();
ioe = ce;
} else if (ioex.getMessage().toLowerCase()
.contains("connection refused")) {
ce = new ConnectException(ioex.getMessage());
ioe = ce;
} else {
// This is the exception we can't handle.
ioe = ioex;
}
if (ce != null) {
handleConnectionException(++reconnectAttempts, maxAttempts, protocol,
addr, ce);
}
}
// check if timed out
if (System.currentTimeMillis() - timeout >= startTime) {
throw ioe;
}
// wait for retry
try {
Thread.sleep(1000);
} catch (InterruptedException ie) {
Thread.interrupted();
throw new InterruptedIOException();
}
}
}
/**
* @param retries current retried times.
* @param maxAttmpts max attempts
* @param protocol protocol interface
* @param addr address of remote service
* @param ce ConnectException
* @throws org.apache.hadoop.hbase.client.RetriesExhaustedException
*
*/
private static void handleConnectionException(int retries,
int maxAttmpts,
Class<?> protocol,
InetSocketAddress addr,
ConnectException ce)
throws RetriesExhaustedException {
if (maxAttmpts >= 0 && retries >= maxAttmpts) {
LOG.info("Server at " + addr + " could not be reached after "
+ maxAttmpts + " tries, giving up.");
throw new RetriesExhaustedException("Failed setting up proxy " + protocol
+ " to " + addr.toString() + " after attempts=" + maxAttmpts, ce);
}
}
public static void setRpcTimeout(int t) {
rpcTimeout.set(t);
}
public static int getRpcTimeout() {
return rpcTimeout.get();
}
public static void resetRpcTimeout() {
rpcTimeout.remove();
}
}

View File

@ -33,6 +33,12 @@ import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
@ -44,6 +50,7 @@ import com.google.common.base.Preconditions;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.CodedOutputStream;
import com.google.protobuf.Message;
import com.google.protobuf.TextFormat;
/**
* Utility to help ipc'ing.
@ -263,4 +270,24 @@ class IPCUtil {
Preconditions.checkArgument(totalSize < Integer.MAX_VALUE);
return totalSize;
}
/**
* Return short version of Param Message toString'd, shorter than TextFormat#regionServerStartup
* @param methodName
* @param request
* @return toString of passed <code>param</code>
*/
static String getRequestShortTextFormat(Message request) {
if (request instanceof ScanRequest) {
return TextFormat.shortDebugString(request);
} else if (request instanceof RegionServerReportRequest) {
// Print a short message only, just the servername and the requests, not the full load.
RegionServerReportRequest r = (RegionServerReportRequest)request;
return "server " + TextFormat.shortDebugString(r.getServer()) +
" load { numberOfRequests: " + r.getLoad().getNumberOfRequests() + " }";
} else if (request instanceof RegionServerStartupRequest) {
return TextFormat.shortDebugString(request);
}
return "TODO " + TextFormat.shortDebugString(request);
}
}

View File

@ -1,166 +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.Message;
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.CellScanner;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import java.io.IOException;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.net.InetSocketAddress;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
public class ProtobufRpcClientEngine implements RpcClientEngine {
private static final Log LOG =
LogFactory.getLog("org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine");
public HBaseClient getClient() {
return client;
}
protected HBaseClient client;
public ProtobufRpcClientEngine(Configuration conf, String clusterId) {
this.client = new HBaseClient(conf, clusterId);
}
@Override
public <T extends IpcProtocol> T getProxy(
Class<T> protocol, InetSocketAddress addr,
Configuration conf, int rpcTimeout) throws IOException {
final Invoker invoker = new Invoker(protocol, addr, User.getCurrent(), rpcTimeout, client);
return (T) Proxy.newProxyInstance(
protocol.getClassLoader(), new Class[]{protocol}, invoker);
}
@Override
public void close() {
this.client.stop();
}
static class Invoker implements InvocationHandler {
private static final Map<String, Message> returnTypes =
new ConcurrentHashMap<String, Message>();
private Class<? extends IpcProtocol> protocol;
private InetSocketAddress address;
private User ticket;
private HBaseClient client;
final private int rpcTimeout;
public Invoker(Class<? extends IpcProtocol> protocol, InetSocketAddress addr, User ticket,
int rpcTimeout, HBaseClient client)
throws IOException {
this.protocol = protocol;
this.address = addr;
this.ticket = ticket;
this.client = client;
this.rpcTimeout = rpcTimeout;
}
/**
* This is the client side invoker of RPC method. It only throws
* ServiceException, since the invocation proxy expects only
* ServiceException to be thrown by the method in case protobuf service.
*
* ServiceException has the following causes:
* <ol>
* <li>Exceptions encountered on the client side in this method are
* set as cause in ServiceException as is.</li>
* <li>Exceptions from the server are wrapped in RemoteException and are
* set as cause in ServiceException</li>
* </ol>
*
* <p>Note that the client calling protobuf RPC methods, must handle
* ServiceException by getting the cause from the ServiceException. If the
* cause is RemoteException, then unwrap it to get the exception thrown by
* the server.
*/
@Override
public Object invoke(Object proxy, Method method, Object[] args)
throws ServiceException {
long startTime = 0;
if (LOG.isTraceEnabled()) {
startTime = System.currentTimeMillis();
}
if (args.length != 2) {
throw new ServiceException(method.getName() + " didn't get two args: " + args.length);
}
// Get the controller. Often null. Presume payload carrying controller. Payload is optional.
// It is cells/data that we do not want to protobuf.
PayloadCarryingRpcController controller = (PayloadCarryingRpcController)args[0];
CellScanner cells = null;
if (controller != null) {
cells = controller.cellScanner();
// Clear it here so we don't by mistake try and these cells processing results.
controller.setCellScanner(null);
}
// The request parameter
Message param = (Message)args[1];
Pair<Message, CellScanner> val = null;
try {
val = client.call(method, param, cells, address, protocol, ticket, rpcTimeout);
if (controller != null) {
// Shove the results into controller so can be carried across the proxy/pb service void.
if (val.getSecond() != null) controller.setCellScanner(val.getSecond());
} else if (val.getSecond() != null) {
throw new ServiceException("Client dropping data on the floor!");
}
if (LOG.isTraceEnabled()) {
long callTime = System.currentTimeMillis() - startTime;
if (LOG.isTraceEnabled()) LOG.trace("Call: " + method.getName() + " " + callTime);
}
return val.getFirst();
} catch (Throwable e) {
if (e instanceof RemoteException) {
Throwable cause = ((RemoteException)e).unwrapRemoteException();
throw new ServiceException("methodName=" + method.getName(), cause);
}
throw new ServiceException(e);
}
}
static Message getReturnProtoType(Method method) throws Exception {
if (returnTypes.containsKey(method.getName())) {
return returnTypes.get(method.getName());
}
Class<?> returnType = method.getReturnType();
if (returnType.getName().equals("void")) return null;
Method newInstMethod = returnType.getMethod("getDefaultInstance");
newInstMethod.setAccessible(true);
Message protoType = (Message) newInstMethod.invoke(null, (Object[]) null);
returnTypes.put(method.getName(), protoType);
return protoType;
}
}
}

View File

@ -1,76 +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.lang.reflect.Method;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.hbase.IpcProtocol;
import com.google.protobuf.Message;
/**
* Save on relection by keeping around method, method argument, and constructor instances
*/
class ReflectionCache {
private final Map<String, Message> methodArgCache = new ConcurrentHashMap<String, Message>();
private final Map<String, Method> methodInstanceCache = new ConcurrentHashMap<String, Method>();
public ReflectionCache() {
super();
}
Method getMethod(Class<? extends IpcProtocol> protocol, String methodName) {
Method method = this.methodInstanceCache.get(methodName);
if (method != null) return method;
Method [] methods = protocol.getMethods();
for (Method m : methods) {
if (m.getName().equals(methodName)) {
m.setAccessible(true);
this.methodInstanceCache.put(methodName, m);
return m;
}
}
return null;
}
Message getMethodArgType(Method method) throws Exception {
Message protoType = this.methodArgCache.get(method.getName());
if (protoType != null) return protoType;
Class<?>[] args = method.getParameterTypes();
Class<?> arg;
if (args.length == 2) {
// RpcController + Message in the method args
// (generated code from RPC bits in .proto files have RpcController)
arg = args[1];
} else if (args.length == 1) {
arg = args[0];
} else {
//unexpected
return null;
}
//in the protobuf methods, args[1] is the only significant argument
Method newInstMethod = arg.getMethod("getDefaultInstance");
newInstMethod.setAccessible(true);
protoType = (Message) newInstMethod.invoke(null, (Object[]) null);
this.methodArgCache.put(method.getName(), protoType);
return protoType;
}
}

View File

@ -79,7 +79,7 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
new ServerCallable<CoprocessorServiceResponse>(connection, table, row) {
public CoprocessorServiceResponse call() throws Exception {
byte[] regionName = location.getRegionInfo().getRegionName();
return ProtobufUtil.execService(server, call, regionName);
return ProtobufUtil.execService(stub, call, regionName);
}
};
CoprocessorServiceResponse result = callable.withRetries();

View File

@ -22,8 +22,9 @@ import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.ipc.RemoteException;
/**
* An {@link RemoteException} with some extra information. If source exception
* A {@link RemoteException} with some extra information. If source exception
* was a {@link DoNotRetryIOException}, {@link #isDoNotRetry()} will return true.
* <p>A {@link RemoteException} hosts exceptions we got from the server.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private

View File

@ -27,7 +27,6 @@ import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.lang.reflect.Method;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.Socket;
@ -55,9 +54,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.codec.KeyValueCodec;
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;
@ -67,8 +67,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
import org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo;
import org.apache.hadoop.hbase.security.AuthMethod;
import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
import org.apache.hadoop.hbase.security.KerberosInfo;
import org.apache.hadoop.hbase.security.TokenInfo;
import org.apache.hadoop.hbase.security.SecurityInfo;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
@ -89,23 +88,25 @@ import org.apache.hadoop.security.token.TokenSelector;
import org.cloudera.htrace.Span;
import org.cloudera.htrace.Trace;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.Message.Builder;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/**
* A client for an IPC service. IPC calls take a single Protobuf message as a
* request and returns a single Protobuf message as result. A service runs on
* a port and is defined by a parameter class and a value class.
*
* Does RPC against a cluster. Manages connections per regionserver in the cluster.
* <p>See HBaseServer
*/
@InterfaceAudience.Private
public class HBaseClient {
public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.HBaseClient");
public class RpcClient {
// The LOG key is intentionally not from this package to avoid ipc logging at DEBUG (all under
// o.a.h.hbase is set to DEBUG as default).
public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RpcClient");
protected final PoolMap<ConnectionId, Connection> connections;
private ReflectionCache reflectionCache = new ReflectionCache();
protected int counter; // counter for call ids
protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
@ -117,7 +118,6 @@ public class HBaseClient {
protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
protected final boolean tcpKeepAlive; // if T then use keepalives
protected int pingInterval; // how often sends ping to the server in msecs
protected int socketTimeout; // socket timeout
protected FailedServers failedServers;
private final Codec codec;
private final CompressionCodec compressor;
@ -135,6 +135,15 @@ public class HBaseClient {
public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
// thread-specific RPC timeout, which may override that of what was passed in.
// TODO: Verify still being used.
private static ThreadLocal<Integer> rpcTimeout = new ThreadLocal<Integer>() {
@Override
protected Integer initialValue() {
return HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
}
};
/**
* A class to manage a list of servers that failed recently.
*/
@ -242,24 +251,28 @@ public class HBaseClient {
*/
CellScanner cells;
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
boolean done; // true when call is done
long startTime;
final Method method;
final MethodDescriptor md;
protected Call(final Method method, Message param, final CellScanner cells) {
protected Call(final MethodDescriptor md, Message param, final CellScanner cells,
final Message responseDefaultType) {
this.param = param;
this.method = method;
this.md = md;
this.cells = cells;
this.startTime = System.currentTimeMillis();
synchronized (HBaseClient.this) {
this.responseDefaultType = responseDefaultType;
synchronized (RpcClient.this) {
this.id = counter++;
}
}
@Override
public String toString() {
return "callId: " + this.id + " methodName: " + this.method.getName() + " param {" +
return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" +
(this.param != null? TextFormat.shortDebugString(this.param): "") + "}";
}
@ -275,18 +288,19 @@ public class HBaseClient {
*
* @param error exception thrown by the call; either local or remote
*/
public synchronized void setException(IOException error) {
public void setException(IOException error) {
this.error = error;
callComplete();
}
/** Set the return value when there is no error.
/**
* 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 synchronized void setResponse(Message response, final CellScanner cells) {
public void setResponse(Message response, final CellScanner cells) {
this.response = response;
this.cells = cells;
callComplete();
@ -297,10 +311,11 @@ public class HBaseClient {
}
}
protected final static Map<String,TokenSelector<? extends TokenIdentifier>> tokenHandlers =
new HashMap<String,TokenSelector<? extends TokenIdentifier>>();
protected final static Map<AuthenticationProtos.TokenIdentifier.Kind,
TokenSelector<? extends TokenIdentifier>> tokenHandlers =
new HashMap<AuthenticationProtos.TokenIdentifier.Kind, TokenSelector<? extends TokenIdentifier>>();
static {
tokenHandlers.put(AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE.toString(),
tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
new AuthenticationTokenSelector());
}
@ -352,33 +367,30 @@ public class HBaseClient {
this.compressor = compressor;
UserGroupInformation ticket = remoteId.getTicket().getUGI();
Class<?> protocol = remoteId.getProtocol();
SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
this.useSasl = User.isHBaseSecurityEnabled(conf);
if (useSasl && protocol != null) {
TokenInfo tokenInfo = protocol.getAnnotation(TokenInfo.class);
if (tokenInfo != null) {
if (useSasl && securityInfo != null) {
AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
if (tokenKind != null) {
TokenSelector<? extends TokenIdentifier> tokenSelector =
tokenHandlers.get(tokenInfo.value());
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 "+tokenInfo.value());
LOG.debug("No token selector found for type "+tokenKind);
}
}
KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
if (krbInfo != null) {
String serverKey = krbInfo.serverPrincipal();
String serverKey = securityInfo.getServerPrincipal();
if (serverKey == null) {
throw new IOException(
"Can't obtain server Kerberos config key from KerberosInfo");
"Can't obtain server Kerberos config key from SecurityInfo");
}
serverPrincipal = SecurityUtil.getServerPrincipal(
conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
if (LOG.isDebugEnabled()) {
LOG.debug("RPC Server Kerberos principal name for protocol="
+ protocol.getCanonicalName() + " is " + serverPrincipal);
}
LOG.debug("RPC Server Kerberos principal name for service="
+ remoteId.getServiceName() + " is " + serverPrincipal);
}
}
@ -391,14 +403,14 @@ public class HBaseClient {
}
if (LOG.isDebugEnabled()) {
LOG.debug("Use " + authMethod + " authentication for protocol "
+ (protocol == null ? "null" : protocol.getSimpleName()));
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.setProtocol(protocol == null ? "" : protocol.getName());
builder.setServiceName(remoteId.getServiceName());
UserInformation userInfoPB;
if ((userInfoPB = getUserInfo(ticket)) != null) {
builder.setUserInfo(userInfoPB);
@ -464,9 +476,11 @@ public class HBaseClient {
}
} else {
calls.put(call.id, call);
synchronized (call) {
notify();
}
}
}
/** This class sends a ping to the remote side when timeout on
* reading. If no failure is detected, it retries until at least
@ -483,8 +497,7 @@ public class HBaseClient {
* otherwise, throw the timeout exception.
*/
private void handleTimeout(SocketTimeoutException e) throws IOException {
if (shouldCloseConnection.get() || !running.get() ||
remoteId.rpcTimeout > 0) {
if (shouldCloseConnection.get() || !running.get() || remoteId.rpcTimeout > 0) {
throw e;
}
sendPing();
@ -580,8 +593,8 @@ public class HBaseClient {
* @param ioe failure reason
* @throws IOException if max number of retries is reached
*/
private void handleConnectionFailure(
int curRetries, int maxRetries, IOException ioe) throws IOException {
private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
throws IOException {
closeConnection();
@ -657,7 +670,7 @@ public class HBaseClient {
}
try {
while (waitForWork()) {//wait here for work - read or close connection
while (waitForWork()) { // Wait here for work - read or close connection
readResponse();
}
} catch (Throwable t) {
@ -811,19 +824,15 @@ public class HBaseClient {
ticket = ticket.getRealUser();
}
}
boolean continueSasl;
boolean continueSasl = false;
if (ticket == null) throw new FatalConnectionException("ticket/user is null");
try {
if (ticket == null) {
throw new NullPointerException("ticket is null");
} else {
continueSasl =
ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
@Override
public Boolean run() throws IOException {
return setupSaslConnection(in2, out2);
}
});
}
} catch (Exception ex) {
if (rand == null) {
rand = new Random();
@ -855,7 +864,7 @@ public class HBaseClient {
}
} catch (Throwable t) {
failedServers.addToFailedServers(remoteId.address);
IOException e;
IOException e = null;
if (t instanceof IOException) {
e = (IOException)t;
markClosed(e);
@ -891,10 +900,12 @@ public class HBaseClient {
* Out is not synchronized because only the first thread does this.
*/
private void writeConnectionHeader() throws IOException {
synchronized (this.out) {
this.out.writeInt(this.header.getSerializedSize());
this.header.writeTo(this.out);
this.out.flush();
}
}
/** Close the connection. */
protected synchronized void close() {
@ -913,7 +924,9 @@ public class HBaseClient {
// close the streams and therefore the socket
IOUtils.closeStream(out);
this.out = null;
IOUtils.closeStream(in);
this.in = null;
disposeSasl();
// clean up all calls
@ -930,7 +943,7 @@ public class HBaseClient {
// log the info
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": closing ipc connection to " + server + ": " +
closeException.getMessage(),closeException);
closeException.getMessage(), closeException);
}
// cleanup calls
@ -957,7 +970,7 @@ public class HBaseClient {
builder.setTraceInfo(RPCTInfo.newBuilder().
setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
}
builder.setMethodName(call.method.getName());
builder.setMethodName(call.md.getName());
builder.setRequestParam(call.param != null);
ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells);
if (cellBlock != null) {
@ -970,8 +983,8 @@ public class HBaseClient {
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
IPCUtil.write(this.out, header, call.param, cellBlock);
}
if (LOG.isTraceEnabled()) {
LOG.trace(getName() + ": wrote request header " + TextFormat.shortDebugString(header));
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": wrote request header " + TextFormat.shortDebugString(header));
}
} catch(IOException e) {
markClosed(e);
@ -984,20 +997,32 @@ public class HBaseClient {
protected void readResponse() {
if (shouldCloseConnection.get()) return;
touch();
int totalSize = -1;
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();
totalSize = in.readInt();
// Read the header
ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
int id = responseHeader.getCallId();
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": got response header " +
TextFormat.shortDebugString(responseHeader));
TextFormat.shortDebugString(responseHeader) + ", totalSize: " + totalSize + " bytes");
}
Call call = calls.get(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;
LOG.debug("Unknown callId: " + id + ", skipping over this response of " +
whatIsLeftToRead + " bytes");
IOUtils.skipFully(in, whatIsLeftToRead);
}
if (responseHeader.hasException()) {
ExceptionResponse exceptionResponse = responseHeader.getException();
RemoteException re = createRemoteException(exceptionResponse);
@ -1007,20 +1032,10 @@ public class HBaseClient {
if (call != null) call.setException(re);
}
} else {
Message rpcResponseType = null;
if (call != null){
try {
// TODO: Why pb engine pollution in here in this class? FIX.
rpcResponseType =
ProtobufRpcClientEngine.Invoker.getReturnProtoType(
reflectionCache.getMethod(remoteId.getProtocol(), call.method.getName()));
} catch (Exception e) {
throw new RuntimeException(e); //local exception
}
}
Message value = null;
if (rpcResponseType != null) {
Builder builder = rpcResponseType.newBuilderForType();
// Call may be null because it may have timedout and been cleaned up on this side already
if (call != null && call.responseDefaultType != null) {
Builder builder = call.responseDefaultType.newBuilderForType();
builder.mergeDelimitedFrom(in);
value = builder.build();
}
@ -1028,7 +1043,7 @@ public class HBaseClient {
if (responseHeader.hasCellBlockMeta()) {
int size = responseHeader.getCellBlockMeta().getLength();
byte [] cellBlock = new byte[size];
IPCUtil.readChunked(this.in, cellBlock, 0, size);
IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock);
}
// it's possible that this call may have been cleaned up due to a RPC
@ -1043,8 +1058,7 @@ public class HBaseClient {
// {@link ConnectionId#rpcTimeout}.
closeException = e;
} else {
// Since the server did not respond within the default ping interval
// time, treat this as a fatal condition and close this connection
// Treat this as a fatal condition and close this connection
markClosed(e);
}
} finally {
@ -1146,31 +1160,42 @@ public class HBaseClient {
}
/**
* Construct an IPC client whose values are of the {@link Message}
* class.
* Construct an IPC cluster client whose values are of the {@link Message} class.
* @param conf configuration
* @param factory socket factory
*/
public HBaseClient(Configuration conf, String clusterId, SocketFactory factory) {
this.maxIdleTime =
conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
RpcClient(Configuration conf, String clusterId, SocketFactory factory) {
this.maxIdleTime = conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
this.failureSleep = conf.getInt("hbase.client.pause", 1000);
this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true);
this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
this.pingInterval = getPingInterval(conf);
if (LOG.isDebugEnabled()) {
LOG.debug("Ping interval: " + this.pingInterval + "ms.");
}
this.ipcUtil = new IPCUtil(conf);
this.conf = conf;
this.codec = getCodec(conf);
this.compressor = getCompressor(conf);
this.socketFactory = factory;
this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
this.connections = new PoolMap<ConnectionId, Connection>(
getPoolType(conf), getPoolSize(conf));
this.connections = new PoolMap<ConnectionId, Connection>(getPoolType(conf), getPoolSize(conf));
this.failedServers = new FailedServers(conf);
if (LOG.isDebugEnabled()) {
LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor +
", tcpKeepAlive=" + this.tcpKeepAlive +
", tcpNoDelay=" + this.tcpNoDelay +
", maxIdleTime=" + this.maxIdleTime +
", maxRetries=" + this.maxRetries +
", ping interval=" + this.pingInterval + "ms.");
}
}
/**
* Construct an IPC client for the cluster <code>clusterId</code> with the default SocketFactory
* @param conf configuration
* @param clusterId
*/
public RpcClient(Configuration conf, String clusterId) {
this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf));
}
/**
@ -1202,14 +1227,6 @@ public class HBaseClient {
}
}
/**
* Construct an IPC client with the default SocketFactory
* @param conf configuration
*/
public HBaseClient(Configuration conf, String clusterId) {
this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf));
}
/**
* Return the pool type specified in the configuration, which must be set to
* either {@link PoolType#RoundRobin} or {@link PoolType#ThreadLocal},
@ -1217,7 +1234,7 @@ public class HBaseClient {
*
* 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 HBaseClient} instances
* thread-local pool. In any case, the number of {@link RpcClient} instances
* should not exceed the operating system's hard limit on the number of
* connections.
*
@ -1252,13 +1269,8 @@ public class HBaseClient {
/** Stop all threads related to this client. No further calls may be made
* using this client. */
public void stop() {
if (LOG.isDebugEnabled()) {
LOG.debug("Stopping client");
}
if (!running.compareAndSet(true, false)) {
return;
}
if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client");
if (!running.compareAndSet(true, false)) return;
// wake up all connections
synchronized (connections) {
@ -1281,11 +1293,11 @@ public class HBaseClient {
* with the <code>ticket</code> credentials, returning the value.
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
* @param method
* @param md
* @param param
* @param cells
* @param addr
* @param protocol
* @param returnType
* @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
* {@link User#getCurrent()} makes a new instance of User each time so will be a new Connection
* each time.
@ -1294,12 +1306,13 @@ public class HBaseClient {
* @throws InterruptedException
* @throws IOException
*/
public Pair<Message, CellScanner> call(Method method, Message param, CellScanner cells,
InetSocketAddress addr, Class<? extends IpcProtocol> protocol, User ticket, int rpcTimeout)
Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
Message returnType, User ticket, InetSocketAddress addr,
int rpcTimeout)
throws InterruptedException, IOException {
Call call = new Call(method, param, cells);
Call call = new Call(md, param, cells, returnType);
Connection connection =
getConnection(addr, protocol, ticket, rpcTimeout, call, this.codec, this.compressor);
getConnection(ticket, call, addr, rpcTimeout, this.codec, this.compressor);
connection.writeRequest(call); // send the parameter
boolean interrupted = false;
//noinspection SynchronizationOnLocalVariableOrMethodParameter
@ -1347,11 +1360,10 @@ public class HBaseClient {
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);
"Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
} else if (exception instanceof SocketTimeoutException) {
return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
" failed on socket timeout exception: " + exception).initCause(exception);
" failed because " + exception).initCause(exception);
} else {
return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
exception).initCause(exception);
@ -1372,34 +1384,25 @@ public class HBaseClient {
if (connection.isAlive() &&
connection.getRemoteAddress().getPort() == port &&
connection.getRemoteAddress().getHostName().equals(hostname)) {
if (connection.shouldCloseConnection.compareAndSet(false, true)) {
LOG.info("The server on " + hostname + ":" + port +
" is dead - closing the connection " + connection.remoteId);
connection.closeException = ioe;
connection.close();
" is dead - stopping the connection " + connection.remoteId);
connection.closeConnection();
// We could do a connection.interrupt(), but it's safer not to do it, as the
// interrupted exception behavior is not defined nor enforced enough.
}
}
}
}
}
/* 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(InetSocketAddress addr, Class<? extends IpcProtocol> protocol,
User ticket, int rpcTimeout, Call call, final Codec codec, final CompressionCodec compressor)
protected Connection getConnection(User ticket, Call call, InetSocketAddress addr,
int rpcTimeout, final Codec codec, final CompressionCodec compressor)
throws IOException, InterruptedException {
if (!running.get()) {
// the client is stopped
throw new IOException("The client is stopped");
}
if (!running.get()) throw new StoppedRpcClientException();
Connection connection;
/* we could avoid this allocation for each RPC by having a
* connectionsId object and with set() method. We need to manage the
* refs for keys in HashMap properly. For now its ok.
*/
ConnectionId remoteId = new ConnectionId(addr, protocol, ticket, rpcTimeout);
ConnectionId remoteId =
new ConnectionId(ticket, call.md.getService().getName(), addr, rpcTimeout);
synchronized (connections) {
connection = connections.get(remoteId);
if (connection == null) {
@ -1421,40 +1424,41 @@ public class HBaseClient {
}
/**
* This class holds the address and the user ticket. The client connections
* to servers are uniquely identified by <remoteAddress, ticket>
* This class holds the address and the user ticket, etc. The client connections
* to servers are uniquely identified by <remoteAddress, ticket, serviceName, rpcTimeout>
*/
protected static class ConnectionId {
final InetSocketAddress address;
final User ticket;
final int rpcTimeout;
Class<? extends IpcProtocol> protocol;
private static final int PRIME = 16777619;
final String serviceName;
ConnectionId(InetSocketAddress address, Class<? extends IpcProtocol> protocol,
User ticket,
ConnectionId(User ticket,
String serviceName,
InetSocketAddress address,
int rpcTimeout) {
this.protocol = protocol;
this.address = address;
this.ticket = ticket;
this.rpcTimeout = rpcTimeout;
this.serviceName = serviceName;
}
String getServiceName() {
return this.serviceName;
}
InetSocketAddress getAddress() {
return address;
}
Class<? extends IpcProtocol> getProtocol() {
return protocol;
}
User getTicket() {
return ticket;
}
@Override
public String toString() {
return this.address.toString() + "/" + this.protocol + "/" + this.ticket + "/" +
return this.address.toString() + "/" + this.serviceName + "/" + this.ticket + "/" +
this.rpcTimeout;
}
@ -1462,18 +1466,126 @@ public class HBaseClient {
public boolean equals(Object obj) {
if (obj instanceof ConnectionId) {
ConnectionId id = (ConnectionId) obj;
return address.equals(id.address) && protocol == id.protocol &&
return address.equals(id.address) &&
((ticket != null && ticket.equals(id.ticket)) ||
(ticket == id.ticket)) && rpcTimeout == id.rpcTimeout;
(ticket == id.ticket)) && rpcTimeout == id.rpcTimeout &&
this.serviceName == id.serviceName;
}
return false;
}
@Override // simply use the default Object#hashcode() ?
public int hashCode() {
int hashcode = (address.hashCode() + PRIME * (PRIME * System.identityHashCode(protocol) ^
(ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout;
int hashcode = (address.hashCode() +
PRIME * (PRIME * this.serviceName.hashCode() ^
(ticket == null ? 0 : ticket.hashCode()) )) ^
rpcTimeout;
return hashcode;
}
}
public static void setRpcTimeout(int t) {
rpcTimeout.set(t);
}
public static int getRpcTimeout() {
return rpcTimeout.get();
}
public static void resetRpcTimeout() {
rpcTimeout.remove();
}
/** Make a blocking call.
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
* @param md
* @param controller
* @param param
* @param returnType
* @param isa
* @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
* {@link User#getCurrent()} makes a new instance of User each time so will be a new Connection
* each time.
* @param rpcTimeout
* @return A pair with the Message response and the Cell data (if any).
* @throws InterruptedException
* @throws IOException
*/
Message callBlockingMethod(MethodDescriptor md, RpcController controller,
Message param, Message returnType, final User ticket, final InetSocketAddress isa,
final int rpcTimeout)
throws ServiceException {
long startTime = 0;
if (LOG.isTraceEnabled()) {
startTime = System.currentTimeMillis();
}
PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController)controller;
CellScanner cells = null;
if (pcrc != null) {
cells = pcrc.cellScanner();
// Clear it here so we don't by mistake try and these cells processing results.
pcrc.setCellScanner(null);
}
Pair<Message, CellScanner> val = null;
try {
val = call(md, param, cells, returnType, ticket, isa, rpcTimeout);
if (pcrc != null) {
// Shove the results into controller so can be carried across the proxy/pb service void.
if (val.getSecond() != null) pcrc.setCellScanner(val.getSecond());
} else if (val.getSecond() != null) {
throw new ServiceException("Client dropping data on the floor!");
}
if (LOG.isTraceEnabled()) {
long callTime = System.currentTimeMillis() - startTime;
if (LOG.isTraceEnabled()) {
LOG.trace("Call: " + md.getName() + ", callTime: " + callTime + "ms");
}
}
return val.getFirst();
} catch (Throwable e) {
throw new ServiceException(e);
}
}
/**
* Creates a "channel" that can be used by a blocking protobuf service. Useful setting up
* protobuf blocking stubs.
* @param sn
* @param ticket
* @param rpcTimeout
* @return A blocking rpc channel that goes via this rpc client instance.
*/
public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn,
final User ticket, final int rpcTimeout) {
return new BlockingRpcChannelImplementation(this, sn, ticket, rpcTimeout);
}
/**
* Blocking rpc channel that goes via hbase rpc.
*/
// Public so can be subclassed for tests.
public static class BlockingRpcChannelImplementation implements BlockingRpcChannel {
private final InetSocketAddress isa;
private volatile RpcClient rpcClient;
private final int rpcTimeout;
private final User ticket;
protected BlockingRpcChannelImplementation(final RpcClient rpcClient, final ServerName sn,
final User ticket, final int rpcTimeout) {
this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
this.rpcClient = rpcClient;
this.rpcTimeout = rpcTimeout;
this.ticket = ticket;
}
@Override
public Message callBlockingMethod(MethodDescriptor md, RpcController controller,
Message param, Message returnType)
throws ServiceException {
return this.rpcClient.callBlockingMethod(md, controller, param, returnType, this.ticket,
this.isa, this.rpcTimeout);
}
}
}

View File

@ -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
@ -16,25 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.exceptions.HBaseIOException;
import java.io.IOException;
import java.net.InetSocketAddress;
public class StoppedRpcClientException extends HBaseIOException {
public StoppedRpcClientException() {
super();
}
/** An RPC implementation for the client */
@InterfaceAudience.Private
public interface RpcClientEngine {
/** Construct a client-side proxy object. */
<T extends IpcProtocol> T getProxy(Class<T> protocol, InetSocketAddress addr,
Configuration conf, int rpcTimeout) throws IOException;
/** Shutdown this instance */
void close();
public HBaseClient getClient();
public StoppedRpcClientException(String msg) {
super(msg);
}
}

View File

@ -1,4 +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
@ -15,24 +15,18 @@
* 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;
public class UnsupportedCellCodecException extends FatalConnectionException {
public UnsupportedCellCodecException() {
super();
}
import org.apache.hadoop.classification.InterfaceAudience;
public UnsupportedCellCodecException(String msg) {
super(msg);
}
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
* Indicates Token related information to be used in authorizing connections
* over a given RPC protocol interface.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
@InterfaceAudience.Private
public @interface TokenInfo {
/** The type of Token.getKind() to be handled */
String value();
public UnsupportedCellCodecException(String msg, Throwable t) {
super(msg, t);
}
}

View File

@ -15,15 +15,18 @@
* 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;
public class UnsupportedCompressionCodecException extends FatalConnectionException {
public UnsupportedCompressionCodecException() {
super();
}
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
public UnsupportedCompressionCodecException(String msg) {
super(msg);
}
/**
* Functions implemented by all the master protocols: e.g. {@link MasterAdminProtocol}
* and {@link MasterMonitorProtocol}. Currently, the only shared method
* {@link #isMasterRunning(com.google.protobuf.RpcController, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest)}
* which is used on connection setup to check if the master has been stopped.
*/
public interface MasterProtocol extends IpcProtocol, MasterService.BlockingInterface {}
public UnsupportedCompressionCodecException(String msg, Throwable t) {
super(msg, t);
}
}

View File

@ -0,0 +1,32 @@
/**
* 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;
public class WrongVersionException extends FatalConnectionException {
public WrongVersionException() {
super();
}
public WrongVersionException(String msg) {
super(msg);
}
public WrongVersionException(String msg, Throwable t) {
super(msg, t);
}
}

View File

@ -46,11 +46,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.MasterAdminProtocol;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
@ -67,6 +64,7 @@ import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@ -85,6 +83,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
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;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
@ -105,6 +104,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.security.access.Permission;
@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.util.DynamicClassLoader;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.token.Token;
import com.google.common.collect.ArrayListMultimap;
@ -222,6 +223,9 @@ public final class ProtobufUtil {
if (e == null) {
return new IOException(se);
}
if (e instanceof RemoteException) {
e = ((RemoteException)e).unwrapRemoteException();
}
return e instanceof IOException ? (IOException) e : new IOException(se);
}
@ -1206,7 +1210,7 @@ public final class ProtobufUtil {
* @return the result of the Get
* @throws IOException
*/
public static Result get(final ClientProtocol client,
public static Result get(final ClientService.BlockingInterface client,
final byte[] regionName, final Get get) throws IOException {
GetRequest request =
RequestConverter.buildGetRequest(regionName, get);
@ -1229,7 +1233,7 @@ public final class ProtobufUtil {
* @return the row or the closestRowBefore if it doesn't exist
* @throws IOException
*/
public static Result getRowOrBefore(final ClientProtocol client,
public static Result getRowOrBefore(final ClientService.BlockingInterface client,
final byte[] regionName, final byte[] row,
final byte[] family) throws IOException {
GetRequest request =
@ -1254,7 +1258,7 @@ public final class ProtobufUtil {
* @return true if all are loaded
* @throws IOException
*/
public static boolean bulkLoadHFile(final ClientProtocol client,
public static boolean bulkLoadHFile(final ClientService.BlockingInterface client,
final List<Pair<byte[], String>> familyPaths,
final byte[] regionName, boolean assignSeqNum) throws IOException {
BulkLoadHFileRequest request =
@ -1268,7 +1272,7 @@ public final class ProtobufUtil {
}
}
public static CoprocessorServiceResponse execService(final ClientProtocol client,
public static CoprocessorServiceResponse execService(final ClientService.BlockingInterface client,
final CoprocessorServiceCall call, final byte[] regionName) throws IOException {
CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
.setCall(call).setRegion(
@ -1282,8 +1286,9 @@ public final class ProtobufUtil {
}
}
public static CoprocessorServiceResponse execService(final MasterAdminProtocol client,
final CoprocessorServiceCall call) throws IOException {
public static CoprocessorServiceResponse execService(
final MasterAdminService.BlockingInterface client, final CoprocessorServiceCall call)
throws IOException {
CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
.setCall(call).setRegion(
RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
@ -1315,7 +1320,7 @@ public final class ProtobufUtil {
* @return the retrieved region info
* @throws IOException
*/
public static HRegionInfo getRegionInfo(final AdminProtocol admin,
public static HRegionInfo getRegionInfo(final AdminService.BlockingInterface admin,
final byte[] regionName) throws IOException {
try {
GetRegionInfoRequest request =
@ -1337,7 +1342,7 @@ public final class ProtobufUtil {
* @param transitionInZK
* @throws IOException
*/
public static void closeRegion(final AdminProtocol admin,
public static void closeRegion(final AdminService.BlockingInterface admin,
final byte[] regionName, final boolean transitionInZK) throws IOException {
CloseRegionRequest closeRegionRequest =
RequestConverter.buildCloseRegionRequest(regionName, transitionInZK);
@ -1358,7 +1363,8 @@ public final class ProtobufUtil {
* @return true if the region is closed
* @throws IOException
*/
public static boolean closeRegion(final AdminProtocol admin, final byte[] regionName,
public static boolean closeRegion(final AdminService.BlockingInterface admin,
final byte[] regionName,
final int versionOfClosingNode, final ServerName destinationServer,
final boolean transitionInZK) throws IOException {
CloseRegionRequest closeRegionRequest =
@ -1379,7 +1385,7 @@ public final class ProtobufUtil {
* @param region
* @throws IOException
*/
public static void openRegion(final AdminProtocol admin,
public static void openRegion(final AdminService.BlockingInterface admin,
final HRegionInfo region) throws IOException {
OpenRegionRequest request =
RequestConverter.buildOpenRegionRequest(region, -1);
@ -1398,7 +1404,8 @@ public final class ProtobufUtil {
* @return a list of online region info
* @throws IOException
*/
public static List<HRegionInfo> getOnlineRegions(final AdminProtocol admin) throws IOException {
public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
throws IOException {
GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
GetOnlineRegionResponse response = null;
try {
@ -1431,8 +1438,8 @@ public final class ProtobufUtil {
* @return the server name
* @throws IOException
*/
public static ServerInfo getServerInfo(
final AdminProtocol admin) throws IOException {
public static ServerInfo getServerInfo(final AdminService.BlockingInterface admin)
throws IOException {
GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest();
try {
GetServerInfoResponse response = admin.getServerInfo(null, request);
@ -1452,8 +1459,9 @@ public final class ProtobufUtil {
* @return the list of store files
* @throws IOException
*/
public static List<String> getStoreFiles(final AdminProtocol admin,
final byte[] regionName, final byte[] family) throws IOException {
public static List<String> getStoreFiles(final AdminService.BlockingInterface admin,
final byte[] regionName, final byte[] family)
throws IOException {
GetStoreFileRequest request =
RequestConverter.buildGetStoreFileRequest(regionName, family);
try {
@ -1472,7 +1480,7 @@ public final class ProtobufUtil {
* @param splitPoint
* @throws IOException
*/
public static void split(final AdminProtocol admin,
public static void split(final AdminService.BlockingInterface admin,
final HRegionInfo hri, byte[] splitPoint) throws IOException {
SplitRegionRequest request =
RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
@ -1493,7 +1501,7 @@ public final class ProtobufUtil {
* two adjacent regions
* @throws IOException
*/
public static void mergeRegions(final AdminProtocol admin,
public static void mergeRegions(final AdminService.BlockingInterface admin,
final HRegionInfo region_a, final HRegionInfo region_b,
final boolean forcible) throws IOException {
MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(

View File

@ -1,39 +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 org.apache.hadoop.classification.InterfaceAudience;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
* Indicates Kerberos related information to be used for authorizing connections
* over a given RPC protocol interface.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
@InterfaceAudience.Private
public @interface KerberosInfo {
/** Key for getting server's Kerberos principal name from Configuration */
String serverPrincipal();
String clientPrincipal() default "";
}

View File

@ -0,0 +1,81 @@
/*
* 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 org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* Maps RPC protocol interfaces to required configuration
*/
public class SecurityInfo {
/** Maps RPC service names to authentication information */
private static ConcurrentMap<String,SecurityInfo> infos = new ConcurrentHashMap<String,SecurityInfo>();
// populate info for known services
static {
infos.put(AdminProtos.AdminService.getDescriptor().getName(),
new SecurityInfo("hbase.regionserver.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
infos.put(ClientProtos.ClientService.getDescriptor().getName(),
new SecurityInfo("hbase.regionserver.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
infos.put(MasterAdminProtos.MasterAdminService.getDescriptor().getName(),
new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
infos.put(MasterMonitorProtos.MasterMonitorService.getDescriptor().getName(),
new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
infos.put(RegionServerStatusProtos.RegionServerStatusService.getDescriptor().getName(),
new SecurityInfo("hbase.master.kerberos.principal", Kind.HBASE_AUTH_TOKEN));
}
/**
* Adds a security configuration for a new service name. Note that this will have no effect if
* the service name was already registered.
*/
public static void addInfo(String serviceName, SecurityInfo securityInfo) {
infos.putIfAbsent(serviceName, securityInfo);
}
/**
* Returns the security configuration associated with the given service name.
*/
public static SecurityInfo getInfo(String serviceName) {
return infos.get(serviceName);
}
private final String serverPrincipal;
private final Kind tokenKind;
public SecurityInfo(String serverPrincipal, Kind tokenKind) {
this.serverPrincipal = serverPrincipal;
this.tokenKind = tokenKind;
}
public String getServerPrincipal() {
return serverPrincipal;
}
public Kind getTokenKind() {
return tokenKind;
}
}

View File

@ -116,7 +116,8 @@ public class RecoverableZooKeeper {
// the identifier = processID@hostName
identifier = ManagementFactory.getRuntimeMXBean().getName();
}
LOG.info("The identifier of this process is " + identifier);
LOG.info("Process identifier=" + identifier +
" connecting to ZooKeeper ensemble=" + quorumServers);
this.identifier = identifier;
this.id = Bytes.toBytes(identifier);

View File

@ -56,8 +56,7 @@ public class ZKConfig {
* @return Properties holding mappings representing ZooKeeper config file.
*/
public static Properties makeZKProps(Configuration conf) {
if (conf.getBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG,
false)) {
if (conf.getBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG, false)) {
LOG.warn(
"Parsing ZooKeeper's " + HConstants.ZOOKEEPER_CONFIG_NAME +
" file for ZK properties " +
@ -80,12 +79,9 @@ public class ZKConfig {
}
}
} else {
if (LOG.isDebugEnabled()) {
LOG.debug(
"Skipped reading ZK properties file '" +
HConstants.ZOOKEEPER_CONFIG_NAME +
"' since '" + HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG +
"' was not set to true");
if (LOG.isTraceEnabled()) {
LOG.trace("Skipped reading ZK properties file '" + HConstants.ZOOKEEPER_CONFIG_NAME +
"' since '" + HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG + "' was not set to true");
}
}

View File

@ -116,8 +116,9 @@ public class ZKUtil {
}
int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
LOG.debug(identifier + " opening connection to ZooKeeper with ensemble (" +
ensemble + ")");
if (LOG.isTraceEnabled()) {
LOG.debug(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
}
int retry = conf.getInt("zookeeper.recovery.retry", 3);
int retryIntervalMillis =
conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
@ -419,9 +420,9 @@ public class ZKUtil {
Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
boolean exists = s != null ? true : false;
if (exists) {
LOG.debug(zkw.prefix("Set watcher on existing znode " + znode));
LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
} else {
LOG.debug(zkw.prefix(znode+" does not exist. Watcher is set."));
LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
}
return exists;
} catch (KeeperException e) {

View File

@ -78,7 +78,7 @@ public class TestSnapshotFromAdmin {
// mock the master admin to our mock
MasterAdminKeepAliveConnection mockMaster = Mockito.mock(MasterAdminKeepAliveConnection.class);
Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
Mockito.when(mockConnection.getKeepAliveMasterAdmin()).thenReturn(mockMaster);
Mockito.when(mockConnection.getKeepAliveMasterAdminService()).thenReturn(mockMaster);
// set the max wait time for the snapshot to complete
TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder()
.setExpectedTimeout(maxWaitTime)
@ -135,7 +135,7 @@ public class TestSnapshotFromAdmin {
// mock the master connection
MasterAdminKeepAliveConnection master = Mockito.mock(MasterAdminKeepAliveConnection.class);
Mockito.when(mockConnection.getKeepAliveMasterAdmin()).thenReturn(master);
Mockito.when(mockConnection.getKeepAliveMasterAdminService()).thenReturn(master);
TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder().setExpectedTimeout(0).build();
Mockito.when(
master.snapshot((RpcController) Mockito.isNull(), Mockito.any(TakeSnapshotRequest.class)))

View File

@ -572,17 +572,6 @@ public final class HConstants {
*/
public static int DEFAULT_HBASE_CLIENT_RETRIES_NUMBER = 10;
/**
* Parameter name for maximum attempts, used to limit the number of times the
* client will try to obtain the proxy for a given region server.
*/
public static String HBASE_CLIENT_RPC_MAXATTEMPTS = "hbase.client.rpc.maxattempts";
/**
* Default value of {@link #HBASE_CLIENT_RPC_MAXATTEMPTS}.
*/
public static int DEFAULT_HBASE_CLIENT_RPC_MAXATTEMPTS = 1;
/**
* Parameter name for client prefetch limit, used as the maximum number of regions
* info that will be prefetched.

View File

@ -22,15 +22,14 @@ import java.io.File;
import java.io.FileFilter;
import java.io.FileInputStream;
import java.io.IOException;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.net.URL;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.jar.*;
import java.util.jar.JarEntry;
import java.util.jar.JarInputStream;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

View File

@ -23,15 +23,17 @@ import java.util.HashMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterManager.ServiceType;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@ -86,12 +88,14 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
@Override
public AdminProtocol getAdminProtocol(ServerName serverName) throws IOException {
public AdminProtos.AdminService.BlockingInterface getAdminProtocol(ServerName serverName)
throws IOException {
return admin.getConnection().getAdmin(serverName);
}
@Override
public ClientProtocol getClientProtocol(ServerName serverName) throws IOException {
public ClientProtos.ClientService.BlockingInterface getClientProtocol(ServerName serverName)
throws IOException {
return admin.getConnection().getClient(serverName);
}
@ -133,13 +137,15 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
@Override
public MasterAdminProtocol getMasterAdmin() throws IOException {
public MasterAdminProtos.MasterAdminService.BlockingInterface getMasterAdmin()
throws IOException {
HConnection conn = HConnectionManager.getConnection(conf);
return conn.getMasterAdmin();
}
@Override
public MasterMonitorProtocol getMasterMonitor() throws IOException {
public MasterMonitorProtos.MasterMonitorService.BlockingInterface getMasterMonitor()
throws IOException {
HConnection conn = HConnectionManager.getConnection(conf);
return conn.getMasterMonitor();
}
@ -195,7 +201,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
return null;
}
AdminProtocol client = connection.getAdmin(regionLoc.getServerName());
AdminProtos.AdminService.BlockingInterface client =
connection.getAdmin(regionLoc.getServerName());
ServerInfo info = ProtobufUtil.getServerInfo(client);
return ProtobufUtil.toServerName(info.getServerName());
}

View File

@ -103,7 +103,7 @@ public class IntegrationTestRebalanceAndKillServersTargeted extends IngestIntegr
@SuppressWarnings("unchecked")
public void setUp() throws Exception {
Configuration conf = HBaseConfiguration.create();
conf.set(HConnectionManager.RETRIES_BY_SERVER, "true");
conf.set(HConnectionManager.RETRIES_BY_SERVER_KEY, "true");
super.setUp(NUM_SLAVES_BASE, conf);
ChaosMonkey.Policy chaosPolicy = new ChaosMonkey.PeriodicRandomActionPolicy(

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import java.util.regex.Pattern;
@ -113,5 +112,4 @@ public class IntegrationTestsDriver extends AbstractHBaseTool {
return result.wasSuccessful() ? 0 : 1;
}
}

View File

@ -20646,6 +20646,11 @@ public final class MasterAdminProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse> done);
public abstract void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done);
}
public static com.google.protobuf.Service newReflectiveService(
@ -20867,6 +20872,14 @@ public final class MasterAdminProtos {
impl.isRestoreSnapshotDone(controller, request, done);
}
@java.lang.Override
public void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done) {
impl.isMasterRunning(controller, request, done);
}
};
}
@ -20943,6 +20956,8 @@ public final class MasterAdminProtos {
return impl.restoreSnapshot(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotRequest)request);
case 26:
return impl.isRestoreSnapshotDone(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest)request);
case 27:
return impl.isMasterRunning(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest)request);
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -21011,6 +21026,8 @@ public final class MasterAdminProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotRequest.getDefaultInstance();
case 26:
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest.getDefaultInstance();
case 27:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -21079,6 +21096,8 @@ public final class MasterAdminProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotResponse.getDefaultInstance();
case 26:
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse.getDefaultInstance();
case 27:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -21222,6 +21241,11 @@ public final class MasterAdminProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse> done);
public abstract void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done);
public static final
com.google.protobuf.Descriptors.ServiceDescriptor
getDescriptor() {
@ -21379,6 +21403,11 @@ public final class MasterAdminProtos {
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse>specializeCallback(
done));
return;
case 27:
this.isMasterRunning(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest)request,
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse>specializeCallback(
done));
return;
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -21447,6 +21476,8 @@ public final class MasterAdminProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotRequest.getDefaultInstance();
case 26:
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest.getDefaultInstance();
case 27:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -21515,6 +21546,8 @@ public final class MasterAdminProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.RestoreSnapshotResponse.getDefaultInstance();
case 26:
return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse.getDefaultInstance();
case 27:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -21940,6 +21973,21 @@ public final class MasterAdminProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse.class,
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse.getDefaultInstance()));
}
public void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done) {
channel.callMethod(
getDescriptor().getMethods().get(27),
controller,
request,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance(),
com.google.protobuf.RpcUtil.generalizeCallback(
done,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.class,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance()));
}
}
public static BlockingInterface newBlockingStub(
@ -22082,6 +22130,11 @@ public final class MasterAdminProtos {
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneRequest request)
throws com.google.protobuf.ServiceException;
public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request)
throws com.google.protobuf.ServiceException;
}
private static final class BlockingStub implements BlockingInterface {
@ -22414,6 +22467,18 @@ public final class MasterAdminProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsRestoreSnapshotDoneResponse.getDefaultInstance());
}
public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request)
throws com.google.protobuf.ServiceException {
return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(27),
controller,
request,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance());
}
}
}
@ -22686,114 +22751,117 @@ public final class MasterAdminProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
"\n\021MasterAdmin.proto\032\013hbase.proto\032\014Client" +
".proto\"R\n\020AddColumnRequest\022\021\n\ttableName\030" +
"\001 \002(\014\022+\n\016columnFamilies\030\002 \002(\0132\023.ColumnFa" +
"milySchema\"\023\n\021AddColumnResponse\"<\n\023Delet" +
"eColumnRequest\022\021\n\ttableName\030\001 \002(\014\022\022\n\ncol" +
"umnName\030\002 \002(\014\"\026\n\024DeleteColumnResponse\"U\n" +
"\023ModifyColumnRequest\022\021\n\ttableName\030\001 \002(\014\022" +
"+\n\016columnFamilies\030\002 \002(\0132\023.ColumnFamilySc" +
"hema\"\026\n\024ModifyColumnResponse\"Z\n\021MoveRegi" +
"onRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecif",
"ier\022#\n\016destServerName\030\002 \001(\0132\013.ServerName" +
"\"\024\n\022MoveRegionResponse\"~\n\035DispatchMergin" +
"gRegionsRequest\022!\n\007regionA\030\001 \002(\0132\020.Regio" +
"nSpecifier\022!\n\007regionB\030\002 \002(\0132\020.RegionSpec" +
"ifier\022\027\n\010forcible\030\003 \001(\010:\005false\" \n\036Dispat" +
"chMergingRegionsResponse\"7\n\023AssignRegion" +
"Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" +
"r\"\026\n\024AssignRegionResponse\"O\n\025UnassignReg" +
"\n\021MasterAdmin.proto\032\014Master.proto\032\013hbase" +
".proto\032\014Client.proto\"R\n\020AddColumnRequest" +
"\022\021\n\ttableName\030\001 \002(\014\022+\n\016columnFamilies\030\002 " +
"\002(\0132\023.ColumnFamilySchema\"\023\n\021AddColumnRes" +
"ponse\"<\n\023DeleteColumnRequest\022\021\n\ttableNam" +
"e\030\001 \002(\014\022\022\n\ncolumnName\030\002 \002(\014\"\026\n\024DeleteCol" +
"umnResponse\"U\n\023ModifyColumnRequest\022\021\n\tta" +
"bleName\030\001 \002(\014\022+\n\016columnFamilies\030\002 \002(\0132\023." +
"ColumnFamilySchema\"\026\n\024ModifyColumnRespon" +
"se\"Z\n\021MoveRegionRequest\022 \n\006region\030\001 \002(\0132",
"\020.RegionSpecifier\022#\n\016destServerName\030\002 \001(" +
"\0132\013.ServerName\"\024\n\022MoveRegionResponse\"~\n\035" +
"DispatchMergingRegionsRequest\022!\n\007regionA" +
"\030\001 \002(\0132\020.RegionSpecifier\022!\n\007regionB\030\002 \002(" +
"\0132\020.RegionSpecifier\022\027\n\010forcible\030\003 \001(\010:\005f" +
"alse\" \n\036DispatchMergingRegionsResponse\"7" +
"\n\023AssignRegionRequest\022 \n\006region\030\001 \002(\0132\020." +
"RegionSpecifier\"\026\n\024AssignRegionResponse\"" +
"O\n\025UnassignRegionRequest\022 \n\006region\030\001 \002(\013" +
"2\020.RegionSpecifier\022\024\n\005force\030\002 \001(\010:\005false",
"\"\030\n\026UnassignRegionResponse\"8\n\024OfflineReg" +
"ionRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpeci" +
"fier\022\024\n\005force\030\002 \001(\010:\005false\"\030\n\026UnassignRe",
"gionResponse\"8\n\024OfflineRegionRequest\022 \n\006" +
"region\030\001 \002(\0132\020.RegionSpecifier\"\027\n\025Offlin" +
"eRegionResponse\"J\n\022CreateTableRequest\022!\n" +
"\013tableSchema\030\001 \002(\0132\014.TableSchema\022\021\n\tspli" +
"tKeys\030\002 \003(\014\"\025\n\023CreateTableResponse\"\'\n\022De" +
"leteTableRequest\022\021\n\ttableName\030\001 \002(\014\"\025\n\023D" +
"eleteTableResponse\"\'\n\022EnableTableRequest" +
"\022\021\n\ttableName\030\001 \002(\014\"\025\n\023EnableTableRespon" +
"se\"(\n\023DisableTableRequest\022\021\n\ttableName\030\001" +
" \002(\014\"\026\n\024DisableTableResponse\"J\n\022ModifyTa",
"bleRequest\022\021\n\ttableName\030\001 \002(\014\022!\n\013tableSc" +
"hema\030\002 \002(\0132\014.TableSchema\"\025\n\023ModifyTableR" +
"esponse\"\021\n\017ShutdownRequest\"\022\n\020ShutdownRe" +
"sponse\"\023\n\021StopMasterRequest\"\024\n\022StopMaste" +
"rResponse\"\020\n\016BalanceRequest\"&\n\017BalanceRe" +
"sponse\022\023\n\013balancerRan\030\001 \002(\010\"<\n\031SetBalanc" +
"erRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchron" +
"ous\030\002 \001(\010\"6\n\032SetBalancerRunningResponse\022" +
"\030\n\020prevBalanceValue\030\001 \001(\010\"\024\n\022CatalogScan" +
"Request\")\n\023CatalogScanResponse\022\022\n\nscanRe",
"sult\030\001 \001(\005\"-\n\033EnableCatalogJanitorReques" +
"t\022\016\n\006enable\030\001 \002(\010\"1\n\034EnableCatalogJanito" +
"rResponse\022\021\n\tprevValue\030\001 \001(\010\" \n\036IsCatalo" +
"gJanitorEnabledRequest\"0\n\037IsCatalogJanit" +
"orEnabledResponse\022\r\n\005value\030\001 \002(\010\"=\n\023Take" +
"SnapshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Snap" +
"shotDescription\"/\n\024TakeSnapshotResponse\022" +
"\027\n\017expectedTimeout\030\001 \002(\003\"\025\n\023ListSnapshot" +
"Request\"?\n\024ListSnapshotResponse\022\'\n\tsnaps" +
"hots\030\001 \003(\0132\024.SnapshotDescription\"?\n\025Dele",
"teSnapshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Sn" +
"apshotDescription\"\030\n\026DeleteSnapshotRespo" +
"nse\"@\n\026RestoreSnapshotRequest\022&\n\010snapsho" +
"t\030\001 \002(\0132\024.SnapshotDescription\"\031\n\027Restore" +
"SnapshotResponse\"?\n\025IsSnapshotDoneReques" +
"t\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescriptio" +
"n\"U\n\026IsSnapshotDoneResponse\022\023\n\004done\030\001 \001(" +
"\010:\005false\022&\n\010snapshot\030\002 \001(\0132\024.SnapshotDes" +
"cription\"F\n\034IsRestoreSnapshotDoneRequest" +
"\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescription",
"\"3\n\035IsRestoreSnapshotDoneResponse\022\022\n\004don" +
"e\030\001 \001(\010:\004true2\377\r\n\022MasterAdminService\0222\n\t" +
"addColumn\022\021.AddColumnRequest\032\022.AddColumn" +
"Response\022;\n\014deleteColumn\022\024.DeleteColumnR" +
"equest\032\025.DeleteColumnResponse\022;\n\014modifyC" +
"olumn\022\024.ModifyColumnRequest\032\025.ModifyColu" +
"mnResponse\0225\n\nmoveRegion\022\022.MoveRegionReq" +
"uest\032\023.MoveRegionResponse\022Y\n\026dispatchMer" +
"gingRegions\022\036.DispatchMergingRegionsRequ" +
"est\032\037.DispatchMergingRegionsResponse\022;\n\014",
"assignRegion\022\024.AssignRegionRequest\032\025.Ass" +
"ignRegionResponse\022A\n\016unassignRegion\022\026.Un" +
"assignRegionRequest\032\027.UnassignRegionResp" +
"onse\022>\n\rofflineRegion\022\025.OfflineRegionReq" +
"uest\032\026.OfflineRegionResponse\0228\n\013deleteTa" +
"ble\022\023.DeleteTableRequest\032\024.DeleteTableRe" +
"sponse\0228\n\013enableTable\022\023.EnableTableReque" +
"st\032\024.EnableTableResponse\022;\n\014disableTable" +
"\022\024.DisableTableRequest\032\025.DisableTableRes" +
"ponse\0228\n\013modifyTable\022\023.ModifyTableReques",
"t\032\024.ModifyTableResponse\0228\n\013createTable\022\023" +
".CreateTableRequest\032\024.CreateTableRespons" +
"e\022/\n\010shutdown\022\020.ShutdownRequest\032\021.Shutdo" +
"wnResponse\0225\n\nstopMaster\022\022.StopMasterReq" +
"uest\032\023.StopMasterResponse\022,\n\007balance\022\017.B" +
"alanceRequest\032\020.BalanceResponse\022M\n\022setBa" +
"lancerRunning\022\032.SetBalancerRunningReques" +
"t\032\033.SetBalancerRunningResponse\022;\n\016runCat" +
"alogScan\022\023.CatalogScanRequest\032\024.CatalogS" +
"canResponse\022S\n\024enableCatalogJanitor\022\034.En",
"ableCatalogJanitorRequest\032\035.EnableCatalo" +
"gJanitorResponse\022\\\n\027isCatalogJanitorEnab" +
"led\022\037.IsCatalogJanitorEnabledRequest\032 .I" +
"sCatalogJanitorEnabledResponse\022L\n\021execMa" +
"sterService\022\032.CoprocessorServiceRequest\032" +
"\033.CoprocessorServiceResponse\0227\n\010snapshot" +
"\022\024.TakeSnapshotRequest\032\025.TakeSnapshotRes" +
"ponse\022D\n\025getCompletedSnapshots\022\024.ListSna" +
"pshotRequest\032\025.ListSnapshotResponse\022A\n\016d" +
"eleteSnapshot\022\026.DeleteSnapshotRequest\032\027.",
"DeleteSnapshotResponse\022A\n\016isSnapshotDone" +
"\022\026.IsSnapshotDoneRequest\032\027.IsSnapshotDon" +
"eResponse\022D\n\017restoreSnapshot\022\027.RestoreSn" +
"apshotRequest\032\030.RestoreSnapshotResponse\022" +
"V\n\025isRestoreSnapshotDone\022\035.IsRestoreSnap" +
"shotDoneRequest\032\036.IsRestoreSnapshotDoneR" +
"esponseBG\n*org.apache.hadoop.hbase.proto" +
"buf.generatedB\021MasterAdminProtosH\001\210\001\001\240\001\001"
"fier\"\027\n\025OfflineRegionResponse\"J\n\022CreateT" +
"ableRequest\022!\n\013tableSchema\030\001 \002(\0132\014.Table" +
"Schema\022\021\n\tsplitKeys\030\002 \003(\014\"\025\n\023CreateTable" +
"Response\"\'\n\022DeleteTableRequest\022\021\n\ttableN" +
"ame\030\001 \002(\014\"\025\n\023DeleteTableResponse\"\'\n\022Enab" +
"leTableRequest\022\021\n\ttableName\030\001 \002(\014\"\025\n\023Ena" +
"bleTableResponse\"(\n\023DisableTableRequest\022" +
"\021\n\ttableName\030\001 \002(\014\"\026\n\024DisableTableRespon",
"se\"J\n\022ModifyTableRequest\022\021\n\ttableName\030\001 " +
"\002(\014\022!\n\013tableSchema\030\002 \002(\0132\014.TableSchema\"\025" +
"\n\023ModifyTableResponse\"\021\n\017ShutdownRequest" +
"\"\022\n\020ShutdownResponse\"\023\n\021StopMasterReques" +
"t\"\024\n\022StopMasterResponse\"\020\n\016BalanceReques" +
"t\"&\n\017BalanceResponse\022\023\n\013balancerRan\030\001 \002(" +
"\010\"<\n\031SetBalancerRunningRequest\022\n\n\002on\030\001 \002" +
"(\010\022\023\n\013synchronous\030\002 \001(\010\"6\n\032SetBalancerRu" +
"nningResponse\022\030\n\020prevBalanceValue\030\001 \001(\010\"" +
"\024\n\022CatalogScanRequest\")\n\023CatalogScanResp",
"onse\022\022\n\nscanResult\030\001 \001(\005\"-\n\033EnableCatalo" +
"gJanitorRequest\022\016\n\006enable\030\001 \002(\010\"1\n\034Enabl" +
"eCatalogJanitorResponse\022\021\n\tprevValue\030\001 \001" +
"(\010\" \n\036IsCatalogJanitorEnabledRequest\"0\n\037" +
"IsCatalogJanitorEnabledResponse\022\r\n\005value" +
"\030\001 \002(\010\"=\n\023TakeSnapshotRequest\022&\n\010snapsho" +
"t\030\001 \002(\0132\024.SnapshotDescription\"/\n\024TakeSna" +
"pshotResponse\022\027\n\017expectedTimeout\030\001 \002(\003\"\025" +
"\n\023ListSnapshotRequest\"?\n\024ListSnapshotRes" +
"ponse\022\'\n\tsnapshots\030\001 \003(\0132\024.SnapshotDescr",
"iption\"?\n\025DeleteSnapshotRequest\022&\n\010snaps" +
"hot\030\001 \002(\0132\024.SnapshotDescription\"\030\n\026Delet" +
"eSnapshotResponse\"@\n\026RestoreSnapshotRequ" +
"est\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotDescript" +
"ion\"\031\n\027RestoreSnapshotResponse\"?\n\025IsSnap" +
"shotDoneRequest\022&\n\010snapshot\030\001 \001(\0132\024.Snap" +
"shotDescription\"U\n\026IsSnapshotDoneRespons" +
"e\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010snapshot\030\002 \001(\013" +
"2\024.SnapshotDescription\"F\n\034IsRestoreSnaps" +
"hotDoneRequest\022&\n\010snapshot\030\001 \001(\0132\024.Snaps",
"hotDescription\"3\n\035IsRestoreSnapshotDoneR" +
"esponse\022\022\n\004done\030\001 \001(\010:\004true2\305\016\n\022MasterAd" +
"minService\0222\n\taddColumn\022\021.AddColumnReque" +
"st\032\022.AddColumnResponse\022;\n\014deleteColumn\022\024" +
".DeleteColumnRequest\032\025.DeleteColumnRespo" +
"nse\022;\n\014modifyColumn\022\024.ModifyColumnReques" +
"t\032\025.ModifyColumnResponse\0225\n\nmoveRegion\022\022" +
".MoveRegionRequest\032\023.MoveRegionResponse\022" +
"Y\n\026dispatchMergingRegions\022\036.DispatchMerg" +
"ingRegionsRequest\032\037.DispatchMergingRegio",
"nsResponse\022;\n\014assignRegion\022\024.AssignRegio" +
"nRequest\032\025.AssignRegionResponse\022A\n\016unass" +
"ignRegion\022\026.UnassignRegionRequest\032\027.Unas" +
"signRegionResponse\022>\n\rofflineRegion\022\025.Of" +
"flineRegionRequest\032\026.OfflineRegionRespon" +
"se\0228\n\013deleteTable\022\023.DeleteTableRequest\032\024" +
".DeleteTableResponse\0228\n\013enableTable\022\023.En" +
"ableTableRequest\032\024.EnableTableResponse\022;" +
"\n\014disableTable\022\024.DisableTableRequest\032\025.D" +
"isableTableResponse\0228\n\013modifyTable\022\023.Mod",
"ifyTableRequest\032\024.ModifyTableResponse\0228\n" +
"\013createTable\022\023.CreateTableRequest\032\024.Crea" +
"teTableResponse\022/\n\010shutdown\022\020.ShutdownRe" +
"quest\032\021.ShutdownResponse\0225\n\nstopMaster\022\022" +
".StopMasterRequest\032\023.StopMasterResponse\022" +
",\n\007balance\022\017.BalanceRequest\032\020.BalanceRes" +
"ponse\022M\n\022setBalancerRunning\022\032.SetBalance" +
"rRunningRequest\032\033.SetBalancerRunningResp" +
"onse\022;\n\016runCatalogScan\022\023.CatalogScanRequ" +
"est\032\024.CatalogScanResponse\022S\n\024enableCatal",
"ogJanitor\022\034.EnableCatalogJanitorRequest\032" +
"\035.EnableCatalogJanitorResponse\022\\\n\027isCata" +
"logJanitorEnabled\022\037.IsCatalogJanitorEnab" +
"ledRequest\032 .IsCatalogJanitorEnabledResp" +
"onse\022L\n\021execMasterService\022\032.CoprocessorS" +
"erviceRequest\032\033.CoprocessorServiceRespon" +
"se\0227\n\010snapshot\022\024.TakeSnapshotRequest\032\025.T" +
"akeSnapshotResponse\022D\n\025getCompletedSnaps" +
"hots\022\024.ListSnapshotRequest\032\025.ListSnapsho" +
"tResponse\022A\n\016deleteSnapshot\022\026.DeleteSnap",
"shotRequest\032\027.DeleteSnapshotResponse\022A\n\016" +
"isSnapshotDone\022\026.IsSnapshotDoneRequest\032\027" +
".IsSnapshotDoneResponse\022D\n\017restoreSnapsh" +
"ot\022\027.RestoreSnapshotRequest\032\030.RestoreSna" +
"pshotResponse\022V\n\025isRestoreSnapshotDone\022\035" +
".IsRestoreSnapshotDoneRequest\032\036.IsRestor" +
"eSnapshotDoneResponse\022D\n\017isMasterRunning" +
"\022\027.IsMasterRunningRequest\032\030.IsMasterRunn" +
"ingResponseBG\n*org.apache.hadoop.hbase.p" +
"rotobuf.generatedB\021MasterAdminProtosH\001\210\001",
"\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -23222,6 +23290,7 @@ public final class MasterAdminProtos {
com.google.protobuf.Descriptors.FileDescriptor
.internalBuildGeneratedFileFrom(descriptorData,
new com.google.protobuf.Descriptors.FileDescriptor[] {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(),
}, assigner);

View File

@ -2632,6 +2632,11 @@ public final class MasterMonitorProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse> done);
public abstract void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done);
}
public static com.google.protobuf.Service newReflectiveService(
@ -2661,6 +2666,14 @@ public final class MasterMonitorProtos {
impl.getClusterStatus(controller, request, done);
}
@java.lang.Override
public void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done) {
impl.isMasterRunning(controller, request, done);
}
};
}
@ -2689,6 +2702,8 @@ public final class MasterMonitorProtos {
return impl.getTableDescriptors(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest)request);
case 2:
return impl.getClusterStatus(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest)request);
case 3:
return impl.isMasterRunning(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest)request);
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -2709,6 +2724,8 @@ public final class MasterMonitorProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest.getDefaultInstance();
case 2:
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest.getDefaultInstance();
case 3:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -2729,6 +2746,8 @@ public final class MasterMonitorProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse.getDefaultInstance();
case 2:
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse.getDefaultInstance();
case 3:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -2752,6 +2771,11 @@ public final class MasterMonitorProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse> done);
public abstract void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done);
public static final
com.google.protobuf.Descriptors.ServiceDescriptor
getDescriptor() {
@ -2789,6 +2813,11 @@ public final class MasterMonitorProtos {
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse>specializeCallback(
done));
return;
case 3:
this.isMasterRunning(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest)request,
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse>specializeCallback(
done));
return;
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -2809,6 +2838,8 @@ public final class MasterMonitorProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest.getDefaultInstance();
case 2:
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest.getDefaultInstance();
case 3:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -2829,6 +2860,8 @@ public final class MasterMonitorProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse.getDefaultInstance();
case 2:
return org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse.getDefaultInstance();
case 3:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@ -2894,6 +2927,21 @@ public final class MasterMonitorProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse.class,
org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse.getDefaultInstance()));
}
public void isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse> done) {
channel.callMethod(
getDescriptor().getMethods().get(3),
controller,
request,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance(),
com.google.protobuf.RpcUtil.generalizeCallback(
done,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.class,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance()));
}
}
public static BlockingInterface newBlockingStub(
@ -2916,6 +2964,11 @@ public final class MasterMonitorProtos {
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest request)
throws com.google.protobuf.ServiceException;
public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request)
throws com.google.protobuf.ServiceException;
}
private static final class BlockingStub implements BlockingInterface {
@ -2960,6 +3013,18 @@ public final class MasterMonitorProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse.getDefaultInstance());
}
public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse isMasterRunning(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest request)
throws com.google.protobuf.ServiceException {
return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(3),
controller,
request,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance());
}
}
}
@ -3002,25 +3067,27 @@ public final class MasterMonitorProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
"\n\023MasterMonitor.proto\032\013hbase.proto\032\023Clus" +
"terStatus.proto\"0\n\033GetSchemaAlterStatusR" +
"equest\022\021\n\ttableName\030\001 \002(\014\"P\n\034GetSchemaAl" +
"terStatusResponse\022\032\n\022yetToUpdateRegions\030" +
"\001 \001(\r\022\024\n\014totalRegions\030\002 \001(\r\"0\n\032GetTableD" +
"escriptorsRequest\022\022\n\ntableNames\030\001 \003(\t\"@\n" +
"\033GetTableDescriptorsResponse\022!\n\013tableSch" +
"ema\030\001 \003(\0132\014.TableSchema\"\031\n\027GetClusterSta" +
"tusRequest\"A\n\030GetClusterStatusResponse\022%" +
"\n\rclusterStatus\030\001 \002(\0132\016.ClusterStatus2\206\002",
"\n\024MasterMonitorService\022S\n\024getSchemaAlter" +
"Status\022\034.GetSchemaAlterStatusRequest\032\035.G" +
"etSchemaAlterStatusResponse\022P\n\023getTableD" +
"escriptors\022\033.GetTableDescriptorsRequest\032" +
"\034.GetTableDescriptorsResponse\022G\n\020getClus" +
"terStatus\022\030.GetClusterStatusRequest\032\031.Ge" +
"tClusterStatusResponseBI\n*org.apache.had" +
"oop.hbase.protobuf.generatedB\023MasterMoni" +
"torProtosH\001\210\001\001\240\001\001"
"\n\023MasterMonitor.proto\032\014Master.proto\032\013hba" +
"se.proto\032\023ClusterStatus.proto\"0\n\033GetSche" +
"maAlterStatusRequest\022\021\n\ttableName\030\001 \002(\014\"" +
"P\n\034GetSchemaAlterStatusResponse\022\032\n\022yetTo" +
"UpdateRegions\030\001 \001(\r\022\024\n\014totalRegions\030\002 \001(" +
"\r\"0\n\032GetTableDescriptorsRequest\022\022\n\ntable" +
"Names\030\001 \003(\t\"@\n\033GetTableDescriptorsRespon" +
"se\022!\n\013tableSchema\030\001 \003(\0132\014.TableSchema\"\031\n" +
"\027GetClusterStatusRequest\"A\n\030GetClusterSt" +
"atusResponse\022%\n\rclusterStatus\030\001 \002(\0132\016.Cl",
"usterStatus2\314\002\n\024MasterMonitorService\022S\n\024" +
"getSchemaAlterStatus\022\034.GetSchemaAlterSta" +
"tusRequest\032\035.GetSchemaAlterStatusRespons" +
"e\022P\n\023getTableDescriptors\022\033.GetTableDescr" +
"iptorsRequest\032\034.GetTableDescriptorsRespo" +
"nse\022G\n\020getClusterStatus\022\030.GetClusterStat" +
"usRequest\032\031.GetClusterStatusResponse\022D\n\017" +
"isMasterRunning\022\027.IsMasterRunningRequest" +
"\032\030.IsMasterRunningResponseBI\n*org.apache" +
".hadoop.hbase.protobuf.generatedB\023Master",
"MonitorProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -3081,6 +3148,7 @@ public final class MasterMonitorProtos {
com.google.protobuf.Descriptors.FileDescriptor
.internalBuildGeneratedFileFrom(descriptorData,
new com.google.protobuf.Descriptors.FileDescriptor[] {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.getDescriptor(),
}, assigner);

View File

@ -535,9 +535,9 @@ public final class RPCProtos {
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
boolean hasProtocol();
String getProtocol();
// optional string serviceName = 2;
boolean hasServiceName();
String getServiceName();
// optional string cellBlockCodecClass = 3 [default = "org.apache.hadoop.hbase.codec.KeyValueCodec"];
boolean hasCellBlockCodecClass();
@ -589,14 +589,14 @@ public final class RPCProtos {
return userInfo_;
}
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
public static final int PROTOCOL_FIELD_NUMBER = 2;
private java.lang.Object protocol_;
public boolean hasProtocol() {
// optional string serviceName = 2;
public static final int SERVICENAME_FIELD_NUMBER = 2;
private java.lang.Object serviceName_;
public boolean hasServiceName() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getProtocol() {
java.lang.Object ref = protocol_;
public String getServiceName() {
java.lang.Object ref = serviceName_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -604,17 +604,17 @@ public final class RPCProtos {
(com.google.protobuf.ByteString) ref;
String s = bs.toStringUtf8();
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
protocol_ = s;
serviceName_ = s;
}
return s;
}
}
private com.google.protobuf.ByteString getProtocolBytes() {
java.lang.Object ref = protocol_;
private com.google.protobuf.ByteString getServiceNameBytes() {
java.lang.Object ref = serviceName_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
protocol_ = b;
serviceName_ = b;
return b;
} else {
return (com.google.protobuf.ByteString) ref;
@ -687,7 +687,7 @@ public final class RPCProtos {
private void initFields() {
userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol";
serviceName_ = "";
cellBlockCodecClass_ = "org.apache.hadoop.hbase.codec.KeyValueCodec";
cellBlockCompressorClass_ = "";
}
@ -713,7 +713,7 @@ public final class RPCProtos {
output.writeMessage(1, userInfo_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeBytes(2, getProtocolBytes());
output.writeBytes(2, getServiceNameBytes());
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeBytes(3, getCellBlockCodecClassBytes());
@ -736,7 +736,7 @@ public final class RPCProtos {
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
.computeBytesSize(2, getProtocolBytes());
.computeBytesSize(2, getServiceNameBytes());
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
size += com.google.protobuf.CodedOutputStream
@ -774,10 +774,10 @@ public final class RPCProtos {
result = result && getUserInfo()
.equals(other.getUserInfo());
}
result = result && (hasProtocol() == other.hasProtocol());
if (hasProtocol()) {
result = result && getProtocol()
.equals(other.getProtocol());
result = result && (hasServiceName() == other.hasServiceName());
if (hasServiceName()) {
result = result && getServiceName()
.equals(other.getServiceName());
}
result = result && (hasCellBlockCodecClass() == other.hasCellBlockCodecClass());
if (hasCellBlockCodecClass()) {
@ -802,9 +802,9 @@ public final class RPCProtos {
hash = (37 * hash) + USERINFO_FIELD_NUMBER;
hash = (53 * hash) + getUserInfo().hashCode();
}
if (hasProtocol()) {
hash = (37 * hash) + PROTOCOL_FIELD_NUMBER;
hash = (53 * hash) + getProtocol().hashCode();
if (hasServiceName()) {
hash = (37 * hash) + SERVICENAME_FIELD_NUMBER;
hash = (53 * hash) + getServiceName().hashCode();
}
if (hasCellBlockCodecClass()) {
hash = (37 * hash) + CELLBLOCKCODECCLASS_FIELD_NUMBER;
@ -937,7 +937,7 @@ public final class RPCProtos {
userInfoBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol";
serviceName_ = "";
bitField0_ = (bitField0_ & ~0x00000002);
cellBlockCodecClass_ = "org.apache.hadoop.hbase.codec.KeyValueCodec";
bitField0_ = (bitField0_ & ~0x00000004);
@ -992,7 +992,7 @@ public final class RPCProtos {
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
result.protocol_ = protocol_;
result.serviceName_ = serviceName_;
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
to_bitField0_ |= 0x00000004;
}
@ -1020,8 +1020,8 @@ public final class RPCProtos {
if (other.hasUserInfo()) {
mergeUserInfo(other.getUserInfo());
}
if (other.hasProtocol()) {
setProtocol(other.getProtocol());
if (other.hasServiceName()) {
setServiceName(other.getServiceName());
}
if (other.hasCellBlockCodecClass()) {
setCellBlockCodecClass(other.getCellBlockCodecClass());
@ -1077,7 +1077,7 @@ public final class RPCProtos {
}
case 18: {
bitField0_ |= 0x00000002;
protocol_ = input.readBytes();
serviceName_ = input.readBytes();
break;
}
case 26: {
@ -1186,39 +1186,39 @@ public final class RPCProtos {
return userInfoBuilder_;
}
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
private java.lang.Object protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol";
public boolean hasProtocol() {
// optional string serviceName = 2;
private java.lang.Object serviceName_ = "";
public boolean hasServiceName() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getProtocol() {
java.lang.Object ref = protocol_;
public String getServiceName() {
java.lang.Object ref = serviceName_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
protocol_ = s;
serviceName_ = s;
return s;
} else {
return (String) ref;
}
}
public Builder setProtocol(String value) {
public Builder setServiceName(String value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000002;
protocol_ = value;
serviceName_ = value;
onChanged();
return this;
}
public Builder clearProtocol() {
public Builder clearServiceName() {
bitField0_ = (bitField0_ & ~0x00000002);
protocol_ = getDefaultInstance().getProtocol();
serviceName_ = getDefaultInstance().getServiceName();
onChanged();
return this;
}
void setProtocol(com.google.protobuf.ByteString value) {
void setServiceName(com.google.protobuf.ByteString value) {
bitField0_ |= 0x00000002;
protocol_ = value;
serviceName_ = value;
onChanged();
}
@ -3982,25 +3982,23 @@ public final class RPCProtos {
java.lang.String[] descriptorData = {
"\n\tRPC.proto\032\rTracing.proto\032\013hbase.proto\"" +
":\n\017UserInformation\022\025\n\reffectiveUser\030\001 \002(" +
"\t\022\020\n\010realUser\030\002 \001(\t\"\343\001\n\020ConnectionHeader" +
"\022\"\n\010userInfo\030\001 \001(\0132\020.UserInformation\022?\n\010" +
"protocol\030\002 \001(\t:-org.apache.hadoop.hbase." +
"client.ClientProtocol\022H\n\023cellBlockCodecC" +
"lass\030\003 \001(\t:+org.apache.hadoop.hbase.code" +
"c.KeyValueCodec\022 \n\030cellBlockCompressorCl" +
"ass\030\004 \001(\t\"\037\n\rCellBlockMeta\022\016\n\006length\030\001 \001" +
"(\r\"w\n\021ExceptionResponse\022\032\n\022exceptionClas",
"sName\030\001 \001(\t\022\022\n\nstackTrace\030\002 \001(\t\022\020\n\010hostn" +
"ame\030\003 \001(\t\022\014\n\004port\030\004 \001(\005\022\022\n\ndoNotRetry\030\005 " +
"\001(\010\"\216\001\n\rRequestHeader\022\016\n\006callId\030\001 \001(\r\022\034\n" +
"\ttraceInfo\030\002 \001(\0132\t.RPCTInfo\022\022\n\nmethodNam" +
"e\030\003 \001(\t\022\024\n\014requestParam\030\004 \001(\010\022%\n\rcellBlo" +
"ckMeta\030\005 \001(\0132\016.CellBlockMeta\"n\n\016Response" +
"Header\022\016\n\006callId\030\001 \001(\r\022%\n\texception\030\002 \001(" +
"\0132\022.ExceptionResponse\022%\n\rcellBlockMeta\030\003" +
" \001(\0132\016.CellBlockMetaB<\n*org.apache.hadoo" +
"p.hbase.protobuf.generatedB\tRPCProtosH\001\240",
"\001\001"
"\t\022\020\n\010realUser\030\002 \001(\t\"\267\001\n\020ConnectionHeader" +
"\022\"\n\010userInfo\030\001 \001(\0132\020.UserInformation\022\023\n\013" +
"serviceName\030\002 \001(\t\022H\n\023cellBlockCodecClass" +
"\030\003 \001(\t:+org.apache.hadoop.hbase.codec.Ke" +
"yValueCodec\022 \n\030cellBlockCompressorClass\030" +
"\004 \001(\t\"\037\n\rCellBlockMeta\022\016\n\006length\030\001 \001(\r\"w" +
"\n\021ExceptionResponse\022\032\n\022exceptionClassNam" +
"e\030\001 \001(\t\022\022\n\nstackTrace\030\002 \001(\t\022\020\n\010hostname\030",
"\003 \001(\t\022\014\n\004port\030\004 \001(\005\022\022\n\ndoNotRetry\030\005 \001(\010\"" +
"\216\001\n\rRequestHeader\022\016\n\006callId\030\001 \001(\r\022\034\n\ttra" +
"ceInfo\030\002 \001(\0132\t.RPCTInfo\022\022\n\nmethodName\030\003 " +
"\001(\t\022\024\n\014requestParam\030\004 \001(\010\022%\n\rcellBlockMe" +
"ta\030\005 \001(\0132\016.CellBlockMeta\"n\n\016ResponseHead" +
"er\022\016\n\006callId\030\001 \001(\r\022%\n\texception\030\002 \001(\0132\022." +
"ExceptionResponse\022%\n\rcellBlockMeta\030\003 \001(\013" +
"2\016.CellBlockMetaB<\n*org.apache.hadoop.hb" +
"ase.protobuf.generatedB\tRPCProtosH\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -4020,7 +4018,7 @@ public final class RPCProtos {
internal_static_ConnectionHeader_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ConnectionHeader_descriptor,
new java.lang.String[] { "UserInfo", "Protocol", "CellBlockCodecClass", "CellBlockCompressorClass", },
new java.lang.String[] { "UserInfo", "ServiceName", "CellBlockCodecClass", "CellBlockCompressorClass", },
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.class,
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.Builder.class);
internal_static_CellBlockMeta_descriptor =

View File

@ -18,6 +18,8 @@
// This file contains protocol buffers that are used for MasterAdminProtocol.
import "Master.proto";
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "MasterAdminProtos";
option java_generic_services = true;
@ -354,7 +356,7 @@ service MasterAdminService {
/**
* List completed snapshots.
* @return a list of snapshot descriptors for completed snapshots
* Returns a list of snapshot descriptors for completed snapshots
*/
rpc getCompletedSnapshots(ListSnapshotRequest) returns(ListSnapshotResponse);
@ -379,4 +381,7 @@ service MasterAdminService {
* Determine if the snapshot restore is done yet.
*/
rpc isRestoreSnapshotDone(IsRestoreSnapshotDoneRequest) returns(IsRestoreSnapshotDoneResponse);
/** return true if master is available */
rpc isMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse);
}

View File

@ -17,6 +17,7 @@
*/
// This file contains protocol buffers that are used for MasterMonitorProtocol.
import "Master.proto";
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "MasterMonitorProtos";
@ -63,4 +64,7 @@ service MasterMonitorService {
/** Return cluster status. */
rpc getClusterStatus(GetClusterStatusRequest)
returns(GetClusterStatusResponse);
/** return true if master is available */
rpc isMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse);
}

View File

@ -79,7 +79,7 @@ message UserInformation {
// This is sent on connection setup after the connection preamble is sent.
message ConnectionHeader {
optional UserInformation userInfo = 1;
optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
optional string serviceName = 2;
// Cell block codec we will use sending over optional cell blocks. Server throws exception
// if cannot deal.
optional string cellBlockCodecClass = 3 [default = "org.apache.hadoop.hbase.codec.KeyValueCodec"];

View File

@ -1,36 +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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
import org.apache.hadoop.hbase.security.TokenInfo;
import org.apache.hadoop.hbase.security.KerberosInfo;
/**
* Protocol that a RegionServer uses to communicate its status to the Master.
*/
@KerberosInfo(
serverPrincipal = "hbase.master.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface RegionServerStatusProtocol
extends RegionServerStatusService.BlockingInterface, IpcProtocol {}

View File

@ -53,10 +53,9 @@ public class ZNodeClearer {
*/
public static void writeMyEphemeralNodeOnDisk(String fileContent) {
String fileName = ZNodeClearer.getMyEphemeralNodeFileName();
if (fileName == null) {
LOG.warn("No filename given to save the znode used, it won't be saved " +
"(Environment variable HBASE_ZNODE_FILE is not set).");
LOG.warn("Environment variable HBASE_ZNODE_FILE not set; znodes will not be cleared " +
"on crash by start scripts (Longer MTTR!)");
return;
}

View File

@ -196,8 +196,6 @@ public class HFileSystem extends FilterFileSystem {
* @return true if the interceptor was added, false otherwise.
*/
static boolean addLocationsOrderInterceptor(Configuration conf, final ReorderBlocks lrb) {
LOG.debug("Starting addLocationsOrderInterceptor with class " + lrb.getClass());
if (!conf.getBoolean("hbase.filesystem.reorder.blocks", true)) { // activated by default
LOG.debug("addLocationsOrderInterceptor configured to false");
return false;
@ -212,8 +210,8 @@ public class HFileSystem extends FilterFileSystem {
}
if (!(fs instanceof DistributedFileSystem)) {
LOG.warn("The file system is not a DistributedFileSystem." +
"Not adding block location reordering");
LOG.debug("The file system is not a DistributedFileSystem. " +
"Skipping on block location reordering");
return false;
}
@ -243,7 +241,8 @@ public class HFileSystem extends FilterFileSystem {
ClientProtocol cp1 = createReorderingProxy(namenode, lrb, conf);
nf.set(dfsc, cp1);
LOG.info("Added intercepting call to namenode#getBlockLocations");
LOG.info("Added intercepting call to namenode#getBlockLocations so can do block reordering" +
" using class " + lrb.getClass());
} catch (NoSuchFieldException e) {
LOG.warn("Can't modify the DFSClient#namenode field to add the location reorder.", e);
return false;

View File

@ -347,8 +347,7 @@ public class CacheConfig {
* @param conf The current configuration.
* @return The block cache or <code>null</code>.
*/
private static synchronized BlockCache instantiateBlockCache(
Configuration conf) {
private static synchronized BlockCache instantiateBlockCache(Configuration conf) {
if (globalBlockCache != null) return globalBlockCache;
if (blockCacheDisabled) return null;
@ -366,14 +365,12 @@ public class CacheConfig {
// Calculate the amount of heap to give the heap.
MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
long lruCacheSize = (long) (mu.getMax() * cachePercentage);
int blockSize = conf.getInt("hbase.offheapcache.minblocksize",
HConstants.DEFAULT_BLOCKSIZE);
int blockSize = conf.getInt("hbase.offheapcache.minblocksize", HConstants.DEFAULT_BLOCKSIZE);
long offHeapCacheSize =
(long) (conf.getFloat("hbase.offheapcache.percentage", (float) 0) *
DirectMemoryUtils.getDirectMemorySize());
if (offHeapCacheSize <= 0) {
String bucketCacheIOEngineName = conf
.get(BUCKET_CACHE_IOENGINE_KEY, null);
String bucketCacheIOEngineName = conf.get(BUCKET_CACHE_IOENGINE_KEY, null);
float bucketCachePercentage = conf.getFloat(BUCKET_CACHE_SIZE_KEY, 0F);
// A percentage of max heap size or a absolute value with unit megabytes
long bucketCacheSize = (long) (bucketCachePercentage < 1 ? mu.getMax()
@ -407,10 +404,9 @@ public class CacheConfig {
throw new RuntimeException(ioex);
}
}
LOG.info("Allocating LruBlockCache with maximum size "
+ StringUtils.humanReadableInt(lruCacheSize));
LruBlockCache lruCache = new LruBlockCache(lruCacheSize,
StoreFile.DEFAULT_BLOCKSIZE_SMALL);
LOG.info("Allocating LruBlockCache with maximum size " +
StringUtils.humanReadableInt(lruCacheSize));
LruBlockCache lruCache = new LruBlockCache(lruCacheSize, StoreFile.DEFAULT_BLOCKSIZE_SMALL);
lruCache.setVictimCache(bucketCache);
if (bucketCache != null && combinedWithLru) {
globalBlockCache = new CombinedBlockCache(lruCache, bucketCache);

View File

@ -0,0 +1,21 @@
/**
* 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;
@SuppressWarnings("serial")
public class EmptyServiceNameException extends FatalConnectionException {}

View File

@ -1,123 +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.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.util.ReflectionUtils;
/**
* A simple RPC mechanism.
*
* This is a local hbase copy of the hadoop RPC so we can do things like
* address HADOOP-414 for hbase-only and try other hbase-specific
* optimizations. Class has been renamed to avoid confusing it w/ hadoop
* versions.
* <p>
*
*
* A <i>protocol</i> is a Java interface. All parameters and return types must
* be Protobuf objects.
* All methods in the protocol should throw only IOException. No field data of
* the protocol instance is transmitted.
*
* This class provides the server side implementation.
*/
@InterfaceAudience.Private
public class HBaseServerRPC {
// Leave this out in the hadoop ipc package but keep class name. Do this
// so that we dont' get the logging of this class's invocations by doing our
// blanket enabling DEBUG on the o.a.h.h. package.
protected static final Log LOG =
LogFactory.getLog("org.apache.hadoop.ipc.HBaseServerRPC");
// cache of RpcEngines by protocol
private static final Map<Class<? extends IpcProtocol>, RpcServerEngine> PROTOCOL_ENGINES =
new HashMap<Class<? extends IpcProtocol>, RpcServerEngine>();
/**
* Configuration key for the {@link org.apache.hadoop.hbase.ipc.RpcServerEngine} implementation to
* load to handle connection protocols. Handlers for individual protocols can be
* configured using {@code "hbase.rpc.server.engine." + protocol.class.name}.
*/
public static final String RPC_ENGINE_PROP = "hbase.rpc.server.engine";
private HBaseServerRPC() {
super();
} // no public ctor
// set a protocol to use a non-default RpcEngine
static void setProtocolEngine(Configuration conf,
Class<? extends IpcProtocol> protocol, Class<? extends RpcServerEngine> engine) {
conf.setClass(RPC_ENGINE_PROP + "." + protocol.getName(), engine, RpcServerEngine.class);
}
// return the RpcEngine configured to handle a protocol
static synchronized RpcServerEngine getProtocolEngine(Class<? extends IpcProtocol> protocol,
Configuration conf) {
RpcServerEngine engine = PROTOCOL_ENGINES.get(protocol);
if (engine == null) {
// check for a configured default engine
Class<?> defaultEngine =
conf.getClass(RPC_ENGINE_PROP, ProtobufRpcServerEngine.class);
// check for a per interface override
Class<?> impl = conf.getClass(RPC_ENGINE_PROP + "." + protocol.getName(),
defaultEngine);
LOG.debug("Using " + impl.getName() + " for " + protocol.getName());
engine = (RpcServerEngine) ReflectionUtils.newInstance(impl, conf);
PROTOCOL_ENGINES.put(protocol, engine);
}
return engine;
}
/**
* Construct a server for a protocol implementation instance.
*/
public static RpcServer getServer(Class<? extends IpcProtocol> protocol,
final Object instance,
final Class<?>[] ifaces,
String bindAddress,
int port,
final int numHandlers,
int metaHandlerCount,
final boolean verbose,
Configuration conf,
int highPriorityLevel)
throws IOException {
return getProtocolEngine(protocol, conf).
getServer(instance,
ifaces,
bindAddress,
port,
numHandlers,
metaHandlerCount,
verbose,
conf,
highPriorityLevel);
}
}

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.ipc;
public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper {
private HBaseServer server;
private RpcServer server;
MetricsHBaseServerWrapperImpl(HBaseServer server) {
MetricsHBaseServerWrapperImpl(RpcServer server) {
this.server = server;
}

View File

@ -1,302 +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.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.client.Operation;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.HBasePolicyProvider;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.codehaus.jackson.map.ObjectMapper;
import com.google.protobuf.Message;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/**
* The {@link RpcServerEngine} implementation for ProtoBuf-based RPCs.
*/
@InterfaceAudience.Private
class ProtobufRpcServerEngine implements RpcServerEngine {
ProtobufRpcServerEngine() {
super();
}
@Override
public Server getServer(Object instance, Class<?>[] ifaces,
String bindAddress, int port, int numHandlers, int metaHandlerCount,
boolean verbose, Configuration conf, int highPriorityLevel)
throws IOException {
return new Server(instance, ifaces, conf, bindAddress, port, numHandlers,
metaHandlerCount, verbose, highPriorityLevel);
}
public static class Server extends HBaseServer {
boolean verbose;
Object instance;
Class<?> implementation;
private static final String WARN_RESPONSE_TIME =
"hbase.ipc.warn.response.time";
private static final String WARN_RESPONSE_SIZE =
"hbase.ipc.warn.response.size";
/** Default value for above params */
private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
private final int warnResponseTime;
private final int warnResponseSize;
private static String classNameBase(String className) {
String[] names = className.split("\\.", -1);
if (names == null || names.length == 0) {
return className;
}
return names[names.length-1];
}
public Server(Object instance, final Class<?>[] ifaces,
Configuration conf, String bindAddress, int port,
int numHandlers, int metaHandlerCount, boolean verbose,
int highPriorityLevel)
throws IOException {
super(bindAddress, port, numHandlers, metaHandlerCount,
conf, classNameBase(instance.getClass().getName()),
highPriorityLevel);
this.instance = instance;
this.implementation = instance.getClass();
this.verbose = verbose;
this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME,
DEFAULT_WARN_RESPONSE_TIME);
this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE,
DEFAULT_WARN_RESPONSE_SIZE);
this.verbose = verbose;
this.instance = instance;
this.implementation = instance.getClass();
}
private AuthenticationTokenSecretManager createSecretManager(){
if (!isSecurityEnabled ||
!(instance instanceof org.apache.hadoop.hbase.Server)) {
return null;
}
org.apache.hadoop.hbase.Server server =
(org.apache.hadoop.hbase.Server)instance;
Configuration conf = server.getConfiguration();
long keyUpdateInterval =
conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000);
long maxAge =
conf.getLong("hbase.auth.token.max.lifetime", 7*24*60*60*1000);
return new AuthenticationTokenSecretManager(conf, server.getZooKeeper(),
server.getServerName().toString(), keyUpdateInterval, maxAge);
}
@Override
public void startThreads() {
AuthenticationTokenSecretManager mgr = createSecretManager();
if (mgr != null) {
setSecretManager(mgr);
mgr.start();
}
this.authManager = new ServiceAuthorizationManager();
HBasePolicyProvider.init(conf, authManager);
// continue with base startup
super.startThreads();
}
@Override
/**
* This is a server side method, which is invoked over RPC. On success
* the return response has protobuf response payload. On failure, the
* exception name and the stack trace are returned in the protobuf response.
*/
public Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol,
Method method, Message param, CellScanner cellScanner, long receiveTime,
MonitoredRPCHandler status)
throws IOException {
try {
if (verbose) {
LOG.info("callId: " + CurCall.get().id + " protocol: " + protocol.getName() +
" method: " + method.getName());
}
status.setRPC(method.getName(), new Object[]{param}, receiveTime);
// TODO: Review after we add in encoded data blocks.
status.setRPCPacket(param);
status.resume("Servicing call");
//get an instance of the method arg type
Message result;
Object impl = null;
if (protocol.isAssignableFrom(this.implementation)) {
impl = this.instance;
} else {
throw new UnknownProtocolException(protocol);
}
PayloadCarryingRpcController controller = null;
long startTime = System.currentTimeMillis();
if (method.getParameterTypes().length == 2) {
// Always create a controller. Some invocations may not pass data in but will pass
// data out and they'll need a controller instance to carry it for them.
controller = new PayloadCarryingRpcController(cellScanner);
result = (Message)method.invoke(impl, controller, param);
} else {
throw new ServiceException("Wrong number of parameters for method: [" +
method.getName() + "]" + ", wanted: 2, actual: " + method.getParameterTypes().length);
}
int processingTime = (int) (System.currentTimeMillis() - startTime);
int qTime = (int) (startTime-receiveTime);
if (LOG.isTraceEnabled()) {
LOG.trace(CurCall.get().toString() +
" response: " + TextFormat.shortDebugString(result) +
" served: " + protocol.getSimpleName() +
" queueTime: " + qTime +
" processingTime: " + processingTime);
}
metrics.dequeuedCall(qTime);
metrics.processedCall(processingTime);
if (verbose) {
log("Return " + TextFormat.shortDebugString(result), LOG);
}
long responseSize = result.getSerializedSize();
// log any RPC responses that are slower than the configured warn
// response time or larger than configured warning size
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
if (tooSlow || tooLarge) {
// when tagging, we let TooLarge trump TooSmall to keep output simple
// note that large responses will often also be slow.
// TOOD: This output is useless.... output the serialized pb as toString but do a
// short form, shorter than TextFormat.shortDebugString(proto).
StringBuilder buffer = new StringBuilder(256);
buffer.append(method.getName());
buffer.append("(");
buffer.append(param.getClass().getName());
buffer.append(")");
logResponse(new Object[]{param},
method.getName(), buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"),
status.getClient(), startTime, processingTime, qTime,
responseSize);
}
return new Pair<Message, CellScanner>(result,
controller != null? controller.cellScanner(): null);
} catch (InvocationTargetException e) {
Throwable target = e.getTargetException();
if (target instanceof IOException) {
throw (IOException)target;
}
if (target instanceof ServiceException) {
throw ProtobufUtil.getRemoteException((ServiceException)target);
}
IOException ioe = new IOException(target.toString());
ioe.setStackTrace(target.getStackTrace());
throw ioe;
} catch (Throwable e) {
if (!(e instanceof IOException)) {
LOG.error("Unexpected throwable object ", e);
}
IOException ioe = new IOException(e.toString());
ioe.setStackTrace(e.getStackTrace());
throw ioe;
}
}
/**
* Logs an RPC response to the LOG file, producing valid JSON objects for
* client Operations.
* @param params The parameters received in the call.
* @param methodName The name of the method invoked
* @param call The string representation of the call
* @param tag The tag that will be used to indicate this event in the log.
* @param clientAddress The address of the client who made this call.
* @param startTime The time that the call was initiated, in ms.
* @param processingTime The duration that the call took to run, in ms.
* @param qTime The duration that the call spent on the queue
* prior to being initiated, in ms.
* @param responseSize The size in bytes of the response buffer.
*/
void logResponse(Object[] params, String methodName, String call, String tag,
String clientAddress, long startTime, int processingTime, int qTime,
long responseSize)
throws IOException {
// for JSON encoding
ObjectMapper mapper = new ObjectMapper();
// base information that is reported regardless of type of call
Map<String, Object> responseInfo = new HashMap<String, Object>();
responseInfo.put("starttimems", startTime);
responseInfo.put("processingtimems", processingTime);
responseInfo.put("queuetimems", qTime);
responseInfo.put("responsesize", responseSize);
responseInfo.put("client", clientAddress);
responseInfo.put("class", instance.getClass().getSimpleName());
responseInfo.put("method", methodName);
if (params.length == 2 && instance instanceof HRegionServer &&
params[0] instanceof byte[] &&
params[1] instanceof Operation) {
// if the slow process is a query, we want to log its table as well
// as its own fingerprint
byte [] tableName =
HRegionInfo.parseRegionName((byte[]) params[0])[0];
responseInfo.put("table", Bytes.toStringBinary(tableName));
// annotate the response map with operation details
responseInfo.putAll(((Operation) params[1]).toMap());
// report to the log file
LOG.warn("(operation" + tag + "): " +
mapper.writeValueAsString(responseInfo));
} else if (params.length == 1 && instance instanceof HRegionServer &&
params[0] instanceof Operation) {
// annotate the response map with operation details
responseInfo.putAll(((Operation) params[0]).toMap());
// report to the log file
LOG.warn("(operation" + tag + "): " +
mapper.writeValueAsString(responseInfo));
} else {
// can't get JSON details, so just report call.toString() along with
// a more generic tag.
responseInfo.put("call", call);
LOG.warn("(response" + tag + "): " +
mapper.writeValueAsString(responseInfo));
}
}
protected static void log(String value, Log LOG) {
String v = value;
final int max = 100;
if (v != null && v.length() > max)
v = v.substring(0, max) + "...";
LOG.info(v);
}
}
}

View File

@ -21,9 +21,10 @@
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.security.User;
import com.google.protobuf.BlockingService;
import java.net.InetAddress;
/**
@ -87,15 +88,14 @@ public class RequestContext {
* Initializes the client credentials for the current request.
* @param user
* @param remoteAddress
* @param protocol
* @param service
*/
public static void set(User user,
InetAddress remoteAddress,
Class<? extends IpcProtocol> protocol) {
InetAddress remoteAddress, BlockingService service) {
RequestContext ctx = instance.get();
ctx.user = user;
ctx.remoteAddress = remoteAddress;
ctx.protocol = protocol;
ctx.service = service;
ctx.inRequest = true;
}
@ -106,21 +106,20 @@ public class RequestContext {
RequestContext ctx = instance.get();
ctx.user = null;
ctx.remoteAddress = null;
ctx.protocol = null;
ctx.service = null;
ctx.inRequest = false;
}
private User user;
private InetAddress remoteAddress;
private Class<? extends IpcProtocol> protocol;
private BlockingService service;
// indicates we're within a RPC request invocation
private boolean inRequest;
private RequestContext(User user, InetAddress remoteAddr,
Class<? extends IpcProtocol> protocol) {
private RequestContext(User user, InetAddress remoteAddr, BlockingService service) {
this.user = user;
this.remoteAddress = remoteAddr;
this.protocol = protocol;
this.service = service;
}
public User getUser() {
@ -131,8 +130,8 @@ public class RequestContext {
return remoteAddress;
}
public Class<? extends IpcProtocol> getProtocol() {
return protocol;
public BlockingService getService() {
return this.service;
}
public boolean isInRequest() {

View File

@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
/**
* Utility for managing the flag byte passed in response to a
* {@link HBaseServer.Call}
* {@link RpcServer.Call}
*/
@InterfaceAudience.Private
class ResponseFlag {

View File

@ -20,12 +20,10 @@ package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.hbase.exceptions.CallerDisconnectedException;
public interface RpcCallContext extends Delayable {
/**
* Throw an exception if the caller who made this IPC call has disconnected.
* If called from outside the context of IPC, this does nothing.
* @throws CallerDisconnectedException
*/
void throwExceptionIfCallerDisconnected() throws CallerDisconnectedException;
}

View File

@ -0,0 +1,76 @@
/*
*
* 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.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.authorize.PolicyProvider;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.protobuf.BlockingService;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
public interface RpcServerInterface {
// TODO: Needs cleanup. Why a 'start', and then a 'startThreads' and an 'openServer'?
void setSocketSendBufSize(int size);
void start();
void stop();
void join() throws InterruptedException;
InetSocketAddress getListenerAddress();
Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
throws IOException, ServiceException;
void setErrorHandler(HBaseRPCErrorHandler handler);
void openServer();
void startThreads();
/**
* Returns the metrics instance for reporting RPC call statistics
*/
MetricsHBaseServer getMetrics();
public void setQosFunction(Function<Pair<RequestHeader, Message>, Integer> newFunc);
/**
* Refresh autentication manager policy.
* @param pp
*/
@VisibleForTesting
void refreshAuthManager(PolicyProvider pp);
}

View File

@ -0,0 +1,25 @@
/**
* 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;
@SuppressWarnings("serial")
public class UnknownServiceException extends FatalConnectionException {
UnknownServiceException(final String msg) {
super(msg);
}
}

View File

@ -543,7 +543,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
+ Bytes.toStringBinary(row));
byte[] regionName = location.getRegionInfo().getRegionName();
if(!useSecure) {
success = ProtobufUtil.bulkLoadHFile(server, famPaths, regionName, assignSeqIds);
success = ProtobufUtil.bulkLoadHFile(stub, famPaths, regionName, assignSeqIds);
} else {
HTable table = new HTable(conn.getConfiguration(), tableName);
secureClient = new SecureBulkLoadClient(table);

View File

@ -24,9 +24,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;

View File

@ -156,7 +156,7 @@ class ActiveMasterManager extends ZooKeeperListener {
// We are the master, return
startupStatus.setStatus("Successfully registered as active master.");
this.clusterHasActiveMaster.set(true);
LOG.info("Master=" + this.sn);
LOG.info("Registered Active Master=" + this.sn);
return true;
}

View File

@ -53,9 +53,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HealthCheckChore;
import org.apache.hadoop.hbase.MasterAdminProtocol;
import org.apache.hadoop.hbase.MasterMonitorProtocol;
import org.apache.hadoop.hbase.RegionServerStatusProtocol;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
@ -79,9 +76,9 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.exceptions.UnknownRegionException;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.HBaseServerRPC;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.balancer.BalancerChore;
import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
@ -109,6 +106,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
@ -161,6 +159,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshot
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest;
@ -169,6 +168,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDe
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@ -227,16 +227,15 @@ import com.google.protobuf.ServiceException;
*
* <p>You can also shutdown just this master. Call {@link #stopMaster()}.
*
* @see MasterMonitorProtocol
* @see MasterAdminProtocol
* @see RegionServerStatusProtocol
* @see Watcher
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
public class HMaster extends HasThread
implements MasterMonitorProtocol, MasterAdminProtocol, RegionServerStatusProtocol, MasterServices,
Server {
implements MasterMonitorProtos.MasterMonitorService.BlockingInterface,
MasterAdminProtos.MasterAdminService.BlockingInterface,
RegionServerStatusProtos.RegionServerStatusService.BlockingInterface,
MasterServices, Server {
private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
// MASTER is name of the webapp and the attribute name used stuffing this
@ -260,7 +259,7 @@ Server {
private LoadBalancerTracker loadBalancerTracker;
// RPC server for the HMaster
private final RpcServer rpcServer;
private final RpcServerInterface rpcServer;
// Set after we've called HBaseServer#openServer and ready to receive RPCs.
// Set back to false after we stop rpcServer. Used by tests.
private volatile boolean rpcServerOpen = false;
@ -367,8 +366,6 @@ Server {
this.conf = new Configuration(conf);
// Disable the block cache on the master
this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
// Set how many times to retry talking to another server over HConnection.
HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
// Server to handle client requests.
String hostname = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
conf.get("hbase.master.dns.interface", "default"),
@ -387,21 +384,20 @@ Server {
throw new IllegalArgumentException("Failed resolve of bind address " + initialIsa);
}
}
String name = "master/" + initialIsa.toString();
// Set how many times to retry talking to another server over HConnection.
HConnectionManager.setServerSideHConnectionRetries(this.conf, name, LOG);
int numHandlers = conf.getInt("hbase.master.handler.count",
conf.getInt("hbase.regionserver.handler.count", 25));
this.rpcServer = HBaseServerRPC.getServer(MasterMonitorProtocol.class, this,
new Class<?>[]{MasterMonitorProtocol.class,
MasterAdminProtocol.class, RegionServerStatusProtocol.class},
initialIsa.getHostName(), // This is bindAddress if set else it's hostname
initialIsa.getPort(),
this.rpcServer = new RpcServer(this, name, getServices(),
initialIsa, // BindAddress is IP we got for this server.
numHandlers,
0, // we dont use high priority handlers in master
conf.getBoolean("hbase.rpc.verbose", false), conf,
conf,
0); // this is a DNC w/o high priority handlers
// Set our address.
this.isa = this.rpcServer.getListenerAddress();
this.serverName = new ServerName(hostname,
this.isa.getPort(), System.currentTimeMillis());
this.serverName = new ServerName(hostname, this.isa.getPort(), System.currentTimeMillis());
this.rsFatals = new MemoryBoundedLogMessageBuffer(
conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
@ -457,6 +453,23 @@ Server {
}
}
/**
* @return list of blocking services and their security info classes that this server supports
*/
private List<BlockingServiceAndInterface> getServices() {
List<BlockingServiceAndInterface> bssi = new ArrayList<BlockingServiceAndInterface>(3);
bssi.add(new BlockingServiceAndInterface(
MasterMonitorProtos.MasterMonitorService.newReflectiveBlockingService(this),
MasterMonitorProtos.MasterMonitorService.BlockingInterface.class));
bssi.add(new BlockingServiceAndInterface(
MasterAdminProtos.MasterAdminService.newReflectiveBlockingService(this),
MasterAdminProtos.MasterAdminService.BlockingInterface.class));
bssi.add(new BlockingServiceAndInterface(
RegionServerStatusProtos.RegionServerStatusService.newReflectiveBlockingService(this),
RegionServerStatusProtos.RegionServerStatusService.BlockingInterface.class));
return bssi;
}
/**
* Stall startup if we are designated a backup master; i.e. we want someone
* else to become the master before proceeding.
@ -612,10 +625,10 @@ Server {
boolean wasUp = this.clusterStatusTracker.isClusterUp();
if (!wasUp) this.clusterStatusTracker.setClusterUp();
LOG.info("Server active/primary master; " + this.serverName +
LOG.info("Server active/primary master=" + this.serverName +
", sessionid=0x" +
Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
", cluster-up flag was=" + wasUp);
", setting cluster-up flag (Was=" + wasUp + ")");
// create the snapshot manager
this.snapshotManager = new SnapshotManager(this, this.metricsMaster);
@ -995,7 +1008,6 @@ Server {
* need to install an unexpected exception handler.
*/
void startServiceThreads() throws IOException{
// Start the executor service pools
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
conf.getInt("hbase.master.executor.openregion.threads", 5));
@ -1045,14 +1057,14 @@ Server {
// Start allowing requests to happen.
this.rpcServer.openServer();
this.rpcServerOpen = true;
if (LOG.isDebugEnabled()) {
LOG.debug("Started service threads");
if (LOG.isTraceEnabled()) {
LOG.trace("Started service threads");
}
}
/**
* Use this when trying to figure when its ok to send in rpcs. Used by tests.
* @return True if we have successfully run {@link HBaseServer#openServer()}
* @return True if we have successfully run {@link RpcServer#openServer()}
*/
boolean isRpcServerOpen() {
return this.rpcServerOpen;
@ -1141,7 +1153,7 @@ Server {
throws UnknownHostException {
// Do it out here in its own little method so can fake an address when
// mocking up in tests.
return HBaseServer.getRemoteIp();
return RpcServer.getRemoteIp();
}
/**

View File

@ -477,7 +477,7 @@ public class MasterFileSystem {
private static void bootstrap(final Path rd, final Configuration c)
throws IOException {
LOG.info("BOOTSTRAP: creating first META region");
LOG.info("BOOTSTRAP: creating META region");
try {
// Bootstrapping, make sure blockcache is off. Else, one will be
// created here in bootstap and it'll need to be cleaned up. Better to

View File

@ -155,10 +155,7 @@ public interface MasterServices extends Server {
public boolean isServerShutdownHandlerEnabled();
/**
* Registers a new protocol buffer {@link Service} subclass as a master coprocessor endpoint to
* be available for handling
* {@link org.apache.hadoop.hbase.MasterAdminProtocol#execMasterService(com.google.protobuf.RpcController,
* org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)} calls.
* Registers a new protocol buffer {@link Service} subclass as a master coprocessor endpoint.
*
* <p>
* Only a single instance may be registered for a given {@link Service} subclass (the

View File

@ -27,9 +27,9 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.SortedMap;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
@ -37,25 +37,25 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.exceptions.ClockOutOfSyncException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.exceptions.PleaseHoldException;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.YouAreDeadException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.ClockOutOfSyncException;
import org.apache.hadoop.hbase.exceptions.PleaseHoldException;
import org.apache.hadoop.hbase.exceptions.YouAreDeadException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
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.regionserver.RegionOpeningState;
@ -112,12 +112,12 @@ public class ServerManager {
private final Map<ServerName, ServerLoad> onlineServers =
new ConcurrentHashMap<ServerName, ServerLoad>();
// TODO: This is strange to have two maps but HSI above is used on both sides
/**
* Map from full server-instance name to the RPC connection for this server.
* Map of admin interfaces per registered regionserver; these interfaces we use to control
* regionservers out on the cluster
*/
private final Map<ServerName, AdminProtocol> serverConnections =
new HashMap<ServerName, AdminProtocol>();
private final Map<ServerName, AdminService.BlockingInterface> rsAdmins =
new HashMap<ServerName, AdminService.BlockingInterface>();
/**
* List of region servers <ServerName> that should not get any more new
@ -351,7 +351,7 @@ public class ServerManager {
void recordNewServer(final ServerName serverName, final ServerLoad sl) {
LOG.info("Registering server=" + serverName);
this.onlineServers.put(serverName, sl);
this.serverConnections.remove(serverName);
this.rsAdmins.remove(serverName);
}
public long getLastFlushedSequenceId(byte[] regionName) {
@ -472,7 +472,7 @@ public class ServerManager {
synchronized (onlineServers) {
onlineServers.notifyAll();
}
this.serverConnections.remove(serverName);
this.rsAdmins.remove(serverName);
// If cluster is going down, yes, servers are going to be expiring; don't
// process as a dead server
if (this.clusterShutdown) {
@ -591,7 +591,7 @@ public class ServerManager {
public RegionOpeningState sendRegionOpen(final ServerName server,
HRegionInfo region, int versionOfOfflineNode)
throws IOException {
AdminProtocol admin = getServerConnection(server);
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
" failed because no RPC connection found to this server");
@ -619,7 +619,7 @@ public class ServerManager {
public List<RegionOpeningState> sendRegionOpen(ServerName server,
List<Pair<HRegionInfo, Integer>> regionOpenInfos)
throws IOException {
AdminProtocol admin = getServerConnection(server);
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
" failed because no RPC connection found to this server");
@ -653,7 +653,7 @@ public class ServerManager {
public boolean sendRegionClose(ServerName server, HRegionInfo region,
int versionOfClosingNode, ServerName dest, boolean transitionInZK) throws IOException {
if (server == null) throw new NullPointerException("Passed server is null");
AdminProtocol admin = getServerConnection(server);
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
throw new IOException("Attempting to send CLOSE RPC to server " +
server.toString() + " for region " +
@ -688,7 +688,7 @@ public class ServerManager {
throw new NullPointerException("Passed server is null");
if (region_a == null || region_b == null)
throw new NullPointerException("Passed region is null");
AdminProtocol admin = getServerConnection(server);
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
throw new IOException("Attempting to send MERGE REGIONS RPC to server "
+ server.toString() + " for region "
@ -701,18 +701,17 @@ public class ServerManager {
/**
* @param sn
* @return
* @return Admin interface for the remote regionserver named <code>sn</code>
* @throws IOException
* @throws RetriesExhaustedException wrapping a ConnectException if failed
* putting up proxy.
*/
private AdminProtocol getServerConnection(final ServerName sn)
private AdminService.BlockingInterface getRsAdmin(final ServerName sn)
throws IOException {
AdminProtocol admin = this.serverConnections.get(sn);
AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
if (admin == null) {
LOG.debug("New connection to " + sn.toString());
LOG.debug("New admin connection to " + sn.toString());
admin = this.connection.getAdmin(sn);
this.serverConnections.put(sn, admin);
this.rsAdmins.put(sn, admin);
}
return admin;
}

View File

@ -178,8 +178,7 @@ public class SplitLogManager extends ZooKeeperListener {
this.timeout = conf.getInt("hbase.splitlog.manager.timeout", DEFAULT_TIMEOUT);
this.unassignedTimeout =
conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
LOG.info("timeout = " + timeout);
LOG.info("unassigned timeout = " + unassignedTimeout);
LOG.info("timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout);
this.serverName = serverName;
this.timeoutMonitor =
@ -855,7 +854,7 @@ public class SplitLogManager extends ZooKeeperListener {
}
getDataSetWatch(nodepath, zkretries);
}
LOG.info("found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
LOG.info("Found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
rescan_nodes + " rescan nodes");
}

View File

@ -22,12 +22,12 @@ package org.apache.hadoop.hbase.protobuf;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -79,7 +79,7 @@ public class ReplicationProtbufUtil {
* @param entries
* @throws java.io.IOException
*/
public static void replicateWALEntry(final AdminProtocol admin,
public static void replicateWALEntry(final AdminService.BlockingInterface admin,
final HLog.Entry[] entries) throws IOException {
AdminProtos.ReplicateWALEntryRequest request =
buildReplicateWALEntryRequest(entries);

View File

@ -108,7 +108,7 @@ import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcCallContext;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@ -3576,7 +3576,7 @@ public class HRegion implements HeapSize { // , Writable{
if (!results.isEmpty()) {
throw new IllegalArgumentException("First parameter should be an empty list");
}
RpcCallContext rpcCall = HBaseServer.getCurrentCall();
RpcCallContext rpcCall = RpcServer.getCurrentCall();
// The loop here is used only when at some point during the next we determine
// that due to effects of filters or otherwise, we have an empty row in the result.
// Then we loop and try again. Otherwise, we must get out on the first iteration via return,

View File

@ -65,7 +65,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HealthCheckChore;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RegionServerStatusProtocol;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
@ -74,9 +73,7 @@ import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -108,19 +105,19 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
import org.apache.hadoop.hbase.ipc.HBaseServerRPC;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine;
import org.apache.hadoop.hbase.ipc.RpcClientEngine;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@ -179,6 +176,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLa
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
@ -218,6 +216,7 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.cliffc.high_scale_lib.Counter;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
@ -230,8 +229,9 @@ import com.google.protobuf.TextFormat;
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
public class HRegionServer implements ClientProtocol,
AdminProtocol, Runnable, RegionServerServices, HBaseRPCErrorHandler, LastSequenceId {
public class HRegionServer implements ClientProtos.ClientService.BlockingInterface,
AdminProtos.AdminService.BlockingInterface, Runnable, RegionServerServices,
HBaseRPCErrorHandler, LastSequenceId {
public static final Log LOG = LogFactory.getLog(HRegionServer.class);
@ -326,15 +326,14 @@ public class HRegionServer implements ClientProtocol,
protected final int numRegionsToReport;
// Remote HMaster
private RegionServerStatusProtocol hbaseMaster;
// Stub to do region server status calls against the master.
private RegionServerStatusService.BlockingInterface rssStub;
// RPC client. Used to make the stub above that does region server status checking.
RpcClient rpcClient;
// Server to handle client requests. Default access so can be accessed by
// unit tests.
RpcServer rpcServer;
// RPC client for communicating with master
RpcClientEngine rpcClientEngine;
RpcServerInterface rpcServer;
private final InetSocketAddress isa;
private UncaughtExceptionHandler uncaughtExceptionHandler;
@ -460,15 +459,12 @@ public class HRegionServer implements ClientProtocol,
throws IOException, InterruptedException {
this.fsOk = true;
this.conf = conf;
// Set how many times to retry talking to another server over HConnection.
HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
this.isOnline = false;
checkCodecs(this.conf);
// do we use checksum verification in the hbase? If hbase checksum verification
// is enabled, then we automatically switch off hdfs checksum verification.
this.useHBaseChecksum = conf.getBoolean(
HConstants.HBASE_CHECKSUM_VERIFICATION, false);
this.useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, false);
// Config'ed params
this.numRetries = conf.getInt("hbase.client.retries.number", 10);
@ -506,18 +502,17 @@ public class HRegionServer implements ClientProtocol,
if (initialIsa.getAddress() == null) {
throw new IllegalArgumentException("Failed resolve of " + initialIsa);
}
this.rand = new Random(initialIsa.hashCode());
this.rpcServer = HBaseServerRPC.getServer(AdminProtocol.class, this,
new Class<?>[]{ClientProtocol.class,
AdminProtocol.class, HBaseRPCErrorHandler.class,
OnlineRegions.class},
initialIsa.getHostName(), // BindAddress is IP we got for this server.
initialIsa.getPort(),
String name = "regionserver/" + initialIsa.toString();
// Set how many times to retry talking to another server over HConnection.
HConnectionManager.setServerSideHConnectionRetries(this.conf, name, LOG);
this.rpcServer = new RpcServer(this, name, getServices(),
/*HBaseRPCErrorHandler.class, OnlineRegions.class},*/
initialIsa, // BindAddress is IP we got for this server.
conf.getInt("hbase.regionserver.handler.count", 10),
conf.getInt("hbase.regionserver.metahandler.count", 10),
conf.getBoolean("hbase.rpc.verbose", false),
conf, HConstants.QOS_THRESHOLD);
// Set our address.
this.isa = this.rpcServer.getListenerAddress();
@ -542,6 +537,20 @@ public class HRegionServer implements ClientProtocol,
this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
}
/**
* @return list of blocking services and their security info classes that this server supports
*/
private List<BlockingServiceAndInterface> getServices() {
List<BlockingServiceAndInterface> bssi = new ArrayList<BlockingServiceAndInterface>(2);
bssi.add(new BlockingServiceAndInterface(
ClientProtos.ClientService.newReflectiveBlockingService(this),
ClientProtos.ClientService.BlockingInterface.class));
bssi.add(new BlockingServiceAndInterface(
AdminProtos.AdminService.newReflectiveBlockingService(this),
AdminProtos.AdminService.BlockingInterface.class));
return bssi;
}
/**
* Run test on configured codecs to make sure supporting libs are in place.
* @param c
@ -706,7 +715,7 @@ public class HRegionServer implements ClientProtocol,
movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this);
// Setup RPC client for master communication
rpcClientEngine = new ProtobufRpcClientEngine(conf, clusterId);
rpcClient = new RpcClient(conf, clusterId);
}
/**
@ -870,10 +879,10 @@ public class HRegionServer implements ClientProtocol,
}
// Make sure the proxy is down.
if (this.hbaseMaster != null) {
this.hbaseMaster = null;
if (this.rssStub != null) {
this.rssStub = null;
}
this.rpcClientEngine.close();
this.rpcClient.stop();
this.leases.close();
if (!killed) {
@ -920,7 +929,7 @@ public class HRegionServer implements ClientProtocol,
this.serverNameFromMasterPOV.getVersionedBytes());
request.setServer(ProtobufUtil.toServerName(sn));
request.setLoad(sl);
this.hbaseMaster.regionServerReport(null, request.build());
this.rssStub.regionServerReport(null, request.build());
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof YouAreDeadException) {
@ -929,7 +938,9 @@ public class HRegionServer implements ClientProtocol,
}
// Couldn't connect to the master, get location from zk and reconnect
// Method blocks until new master is found or we are stopped
getMaster();
Pair<ServerName, RegionServerStatusService.BlockingInterface> p =
createRegionServerStatusStub();
this.rssStub = p.getSecond();
}
}
@ -1078,9 +1089,11 @@ public class HRegionServer implements ClientProtocol,
String hostnameFromMasterPOV = e.getValue();
this.serverNameFromMasterPOV = new ServerName(hostnameFromMasterPOV,
this.isa.getPort(), this.startcode);
LOG.info("Master passed us hostname to use. Was=" +
this.isa.getHostName() + ", Now=" +
if (!this.serverNameFromMasterPOV.equals(this.isa.getHostName())) {
LOG.info("Master passed us a different hostname to use; was=" +
this.isa.getHostName() + ", but now=" +
this.serverNameFromMasterPOV.getHostname());
}
continue;
}
String value = e.getValue();
@ -1629,7 +1642,7 @@ public class HRegionServer implements ClientProtocol,
}
@Override
public RpcServer getRpcServer() {
public RpcServerInterface getRpcServer() {
return rpcServer;
}
@ -1662,14 +1675,14 @@ public class HRegionServer implements ClientProtocol,
msg += "\nCause:\n" + StringUtils.stringifyException(cause);
}
// Report to the master but only if we have already registered with the master.
if (hbaseMaster != null && this.serverNameFromMasterPOV != null) {
if (rssStub != null && this.serverNameFromMasterPOV != null) {
ReportRSFatalErrorRequest.Builder builder =
ReportRSFatalErrorRequest.newBuilder();
ServerName sn =
ServerName.parseVersionedServerName(this.serverNameFromMasterPOV.getVersionedBytes());
builder.setServer(ProtobufUtil.toServerName(sn));
builder.setErrorMessage(msg);
hbaseMaster.reportRSFatalError(null, builder.build());
rssStub.reportRSFatalError(null, builder.build());
}
} catch (Throwable t) {
LOG.warn("Unable to report fatal error to master", t);
@ -1753,14 +1766,16 @@ public class HRegionServer implements ClientProtocol,
*
* @return master + port, or null if server has been stopped
*/
private ServerName getMaster() {
ServerName masterServerName = null;
private Pair<ServerName, RegionServerStatusService.BlockingInterface>
createRegionServerStatusStub() {
ServerName sn = null;
long previousLogTime = 0;
RegionServerStatusProtocol master = null;
RegionServerStatusService.BlockingInterface master = null;
boolean refresh = false; // for the first time, use cached data
RegionServerStatusService.BlockingInterface intf = null;
while (keepLooping() && master == null) {
masterServerName = this.masterAddressManager.getMasterAddress(refresh);
if (masterServerName == null) {
sn = this.masterAddressManager.getMasterAddress(refresh);
if (sn == null) {
if (!keepLooping()) {
// give up with no connection.
LOG.debug("No master found and cluster is stopped; bailing out");
@ -1769,22 +1784,20 @@ public class HRegionServer implements ClientProtocol,
LOG.debug("No master found; retry");
previousLogTime = System.currentTimeMillis();
refresh = true; // let's try pull it from ZK directly
sleeper.sleep();
continue;
}
InetSocketAddress isa =
new InetSocketAddress(masterServerName.getHostname(), masterServerName.getPort());
new InetSocketAddress(sn.getHostname(), sn.getPort());
LOG.info("Attempting connect to Master server at " +
this.masterAddressManager.getMasterAddress());
try {
// Do initial RPC setup. The final argument indicates that the RPC
// should retry indefinitely.
master = HBaseClientRPC.waitForProxy(rpcClientEngine, RegionServerStatusProtocol.class,
isa, this.conf, -1, this.rpcTimeout, this.rpcTimeout);
LOG.info("Connected to master at " + isa);
BlockingRpcChannel channel = this.rpcClient.createBlockingRpcChannel(sn,
User.getCurrent(), this.rpcTimeout);
intf = RegionServerStatusService.newBlockingStub(channel);
break;
} catch (IOException e) {
e = e instanceof RemoteException ?
((RemoteException)e).unwrapRemoteException() : e;
@ -1805,8 +1818,7 @@ public class HRegionServer implements ClientProtocol,
}
}
}
this.hbaseMaster = master;
return masterServerName;
return new Pair<ServerName, RegionServerStatusService.BlockingInterface>(sn, intf);
}
/**
@ -1826,7 +1838,10 @@ public class HRegionServer implements ClientProtocol,
*/
private RegionServerStartupResponse reportForDuty() throws IOException {
RegionServerStartupResponse result = null;
ServerName masterServerName = getMaster();
Pair<ServerName, RegionServerStatusService.BlockingInterface> p =
createRegionServerStatusStub();
this.rssStub = p.getSecond();
ServerName masterServerName = p.getFirst();
if (masterServerName == null) return result;
try {
this.requestCount.set(0);
@ -1838,7 +1853,7 @@ public class HRegionServer implements ClientProtocol,
request.setPort(port);
request.setServerStartCode(this.startcode);
request.setServerCurrentTime(now);
result = this.hbaseMaster.regionServerStartup(null, request.build());
result = this.rssStub.regionServerStartup(null, request.build());
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof ClockOutOfSyncException) {
@ -1858,7 +1873,7 @@ public class HRegionServer implements ClientProtocol,
try {
GetLastFlushedSequenceIdRequest req =
RequestConverter.buildGetLastFlushedSequenceIdRequest(region);
lastFlushedSequenceId = hbaseMaster.getLastFlushedSequenceId(null, req)
lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req)
.getLastFlushedSequenceId();
} catch (ServiceException e) {
lastFlushedSequenceId = -1l;
@ -3062,8 +3077,7 @@ public class HRegionServer implements ClientProtocol,
builder.setMoreResults(moreResults);
return builder.build();
} catch (Throwable t) {
if (scannerName != null &&
t instanceof NotServingRegionException) {
if (scannerName != null && t instanceof NotServingRegionException) {
scanners.remove(scannerName);
}
throw convertThrowableToIOE(cleanup(t));

View File

@ -24,9 +24,10 @@ import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.zookeeper.KeeperException;
@ -80,7 +81,7 @@ public interface RegionServerServices extends OnlineRegions {
/**
* Returns a reference to the region server's RPC server
*/
public RpcServer getRpcServer();
public RpcServerInterface getRpcServer();
/**
* Get the regions that are currently being opened or closed in the RS

View File

@ -441,7 +441,7 @@ class FSHLog implements HLog, Syncable {
}
}
if (m != null) {
LOG.info("Using getNumCurrentReplicas--HDFS-826");
if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas--HDFS-826");
}
return m;
}

View File

@ -49,12 +49,12 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -718,7 +718,7 @@ public class ReplicationSource extends Thread
continue;
}
try {
AdminProtocol rrs = getRS();
AdminService.BlockingInterface rrs = getRS();
ReplicationProtbufUtil.replicateWALEntry(rrs,
Arrays.copyOf(this.entriesArray, currentNbEntries));
if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
@ -848,7 +848,7 @@ public class ReplicationSource extends Thread
* @return
* @throws IOException
*/
private AdminProtocol getRS() throws IOException {
private AdminService.BlockingInterface getRS() throws IOException {
if (this.currentPeers.size() == 0) {
throw new IOException(this.peerClusterZnode + " has 0 region servers");
}
@ -867,7 +867,7 @@ public class ReplicationSource extends Thread
Thread pingThread = new Thread() {
public void run() {
try {
AdminProtocol rrs = getRS();
AdminService.BlockingInterface rrs = getRS();
// Dummy call which should fail
ProtobufUtil.getServerInfo(rrs);
latch.countDown();

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.hbase.security;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.MasterMonitorProtocol;
import org.apache.hadoop.hbase.MasterAdminProtocol;
import org.apache.hadoop.hbase.RegionServerStatusProtocol;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.MasterMonitorService;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
@ -33,11 +33,11 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
*/
public class HBasePolicyProvider extends PolicyProvider {
protected final static Service[] services = {
new Service("security.client.protocol.acl", ClientProtocol.class),
new Service("security.client.protocol.acl", AdminProtocol.class),
new Service("security.admin.protocol.acl", MasterMonitorProtocol.class),
new Service("security.admin.protocol.acl", MasterAdminProtocol.class),
new Service("security.masterregion.protocol.acl", RegionServerStatusProtocol.class)
new Service("security.client.protocol.acl", ClientService.BlockingInterface.class),
new Service("security.client.protocol.acl", AdminService.BlockingInterface.class),
new Service("security.admin.protocol.acl", MasterMonitorService.BlockingInterface.class),
new Service("security.admin.protocol.acl", MasterAdminService.BlockingInterface.class),
new Service("security.masterregion.protocol.acl", RegionServerStatusService.BlockingInterface.class)
};
@Override
@ -45,12 +45,10 @@ public class HBasePolicyProvider extends PolicyProvider {
return services;
}
public static void init(Configuration conf,
ServiceAuthorizationManager authManager) {
public static void init(Configuration conf, ServiceAuthorizationManager authManager) {
// set service-level authorization security policy
System.setProperty("hadoop.policy.file", "hbase-policy.xml");
if (conf.getBoolean(
ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false)) {
if (conf.getBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false)) {
authManager.refresh(conf, new HBasePolicyProvider());
}
}

View File

@ -34,7 +34,7 @@ import javax.security.sasl.Sasl;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
@ -96,11 +96,11 @@ public class HBaseSaslRpcServer {
/** CallbackHandler for SASL DIGEST-MD5 mechanism */
public static class SaslDigestCallbackHandler implements CallbackHandler {
private SecretManager<TokenIdentifier> secretManager;
private HBaseServer.Connection connection;
private RpcServer.Connection connection;
public SaslDigestCallbackHandler(
SecretManager<TokenIdentifier> secretManager,
HBaseServer.Connection connection) {
RpcServer.Connection connection) {
this.secretManager = secretManager;
this.connection = connection;
}

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RequestContext;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RequestContext;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
@ -60,8 +60,8 @@ public class TokenProvider implements AuthenticationProtos.AuthenticationService
if (env instanceof RegionCoprocessorEnvironment) {
RegionCoprocessorEnvironment regionEnv =
(RegionCoprocessorEnvironment)env;
RpcServer server = regionEnv.getRegionServerServices().getRpcServer();
SecretManager<?> mgr = ((HBaseServer)server).getSecretManager();
RpcServerInterface server = regionEnv.getRegionServerServices().getRpcServer();
SecretManager<?> mgr = ((RpcServer)server).getSecretManager();
if (mgr instanceof AuthenticationTokenSecretManager) {
secretManager = (AuthenticationTokenSecretManager)mgr;
}

View File

@ -66,13 +66,12 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
@ -86,6 +85,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@ -3063,8 +3063,7 @@ public class HBaseFsck extends Configured implements Tool {
public synchronized Void call() throws IOException {
errors.progress();
try {
AdminProtocol server =
connection.getAdmin(rsinfo);
BlockingInterface server = connection.getAdmin(rsinfo);
// list all online regions from this region server
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);

View File

@ -32,14 +32,14 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.zookeeper.KeeperException;
@ -149,7 +149,7 @@ public class HBaseFsckRepair {
public static void closeRegionSilentlyAndWait(HBaseAdmin admin,
ServerName server, HRegionInfo region) throws IOException, InterruptedException {
HConnection connection = admin.getConnection();
AdminProtocol rs = connection.getAdmin(server);
AdminService.BlockingInterface rs = connection.getAdmin(server);
ProtobufUtil.closeRegion(rs, region.getRegionName(), false);
long timeout = admin.getConfiguration()
.getLong("hbase.hbck.close.timeout", 120000);

View File

@ -39,9 +39,9 @@ import org.apache.hadoop.hbase.exceptions.TableNotDisabledException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;

View File

@ -25,8 +25,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.MasterMonitorService;
import org.apache.hadoop.hbase.util.Threads;
/**
@ -94,26 +96,28 @@ public abstract class HBaseCluster implements Closeable, Configurable {
}
/**
* Returns an {@link MasterAdminProtocol} to the active master
* Returns an {@link MasterAdminService.BlockingInterface} to the active master
*/
public abstract MasterAdminProtocol getMasterAdmin()
public abstract MasterAdminService.BlockingInterface getMasterAdmin()
throws IOException;
/**
* Returns an {@link MasterMonitorProtocol} to the active master
* Returns an {@link MasterMonitorService.BlockingInterface} to the active master
*/
public abstract MasterMonitorProtocol getMasterMonitor()
public abstract MasterMonitorService.BlockingInterface getMasterMonitor()
throws IOException;
/**
* Returns an AdminProtocol interface to the regionserver
*/
public abstract AdminProtocol getAdminProtocol(ServerName serverName) throws IOException;
public abstract AdminService.BlockingInterface getAdminProtocol(ServerName serverName)
throws IOException;
/**
* Returns a ClientProtocol interface to the regionserver
*/
public abstract ClientProtocol getClientProtocol(ServerName serverName) throws IOException;
public abstract ClientService.BlockingInterface getClientProtocol(ServerName serverName)
throws IOException;
/**
* Starts a new region server on the given hostname or if this is a mini/local cluster,

View File

@ -29,10 +29,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.MasterMonitorService;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -372,12 +374,12 @@ public class MiniHBaseCluster extends HBaseCluster {
}
@Override
public MasterAdminProtocol getMasterAdmin() {
public MasterAdminService.BlockingInterface getMasterAdmin() {
return this.hbaseCluster.getActiveMaster();
}
@Override
public MasterMonitorProtocol getMasterMonitor() {
public MasterMonitorService.BlockingInterface getMasterMonitor() {
return this.hbaseCluster.getActiveMaster();
}
@ -712,12 +714,13 @@ public class MiniHBaseCluster extends HBaseCluster {
}
@Override
public AdminProtocol getAdminProtocol(ServerName serverName) throws IOException {
public AdminService.BlockingInterface getAdminProtocol(ServerName serverName) throws IOException {
return getRegionServer(getRegionServerIndex(serverName));
}
@Override
public ClientProtocol getClientProtocol(ServerName serverName) throws IOException {
public ClientService.BlockingInterface getClientProtocol(ServerName serverName)
throws IOException {
return getRegionServer(getRegionServerIndex(serverName));
}
}

View File

@ -22,8 +22,6 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.ConnectException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import junit.framework.Assert;
@ -37,25 +35,20 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.Progressable;
import org.apache.zookeeper.KeeperException;
@ -63,7 +56,6 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
@ -149,7 +141,8 @@ public class TestCatalogTracker {
*/
@Test public void testInterruptWaitOnMeta()
throws IOException, InterruptedException, ServiceException {
final ClientProtocol client = Mockito.mock(ClientProtocol.class);
final ClientProtos.ClientService.BlockingInterface client =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
HConnection connection = mockConnection(null, client);
try {
Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
@ -183,7 +176,8 @@ public class TestCatalogTracker {
private void testVerifyMetaRegionLocationWithException(Exception ex)
throws IOException, InterruptedException, KeeperException, ServiceException {
// Mock an ClientProtocol.
final ClientProtocol implementation = Mockito.mock(ClientProtocol.class);
final ClientProtos.ClientService.BlockingInterface implementation =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
HConnection connection = mockConnection(null, implementation);
try {
// If a 'get' is called on mocked interface, throw connection refused.
@ -253,8 +247,8 @@ public class TestCatalogTracker {
HConnection connection = Mockito.mock(HConnection.class);
ServiceException connectException =
new ServiceException(new ConnectException("Connection refused"));
final AdminProtocol implementation =
Mockito.mock(AdminProtocol.class);
final AdminProtos.AdminService.BlockingInterface implementation =
Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
Mockito.when(implementation.getRegionInfo((RpcController)Mockito.any(),
(GetRegionInfoRequest)Mockito.any())).thenThrow(connectException);
Mockito.when(connection.getAdmin(Mockito.any(ServerName.class), Mockito.anyBoolean())).
@ -309,22 +303,23 @@ public class TestCatalogTracker {
}
/**
* @param admin An {@link AdminProtocol} instance; you'll likely
* @param admin An {@link AdminProtos.AdminService.BlockingInterface} instance; you'll likely
* want to pass a mocked HRS; can be null.
* @param client A mocked ClientProtocol instance, can be null
* @return Mock up a connection that returns a {@link Configuration} when
* {@link HConnection#getConfiguration()} is called, a 'location' when
* {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
* and that returns the passed {@link AdminProtocol} instance when
* and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
* {@link HConnection#getAdmin(ServerName)} is called, returns the passed
* {@link ClientProtocol} instance when {@link HConnection#getClient(ServerName)}
* is called (Be sure call
* {@link ClientProtos.ClientService.BlockingInterface} instance when
* {@link HConnection#getClient(ServerName)} is called (Be sure to call
* {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration)}
* when done with this mocked Connection.
* @throws IOException
*/
private HConnection mockConnection(final AdminProtocol admin,
final ClientProtocol client) throws IOException {
private HConnection mockConnection(final AdminProtos.AdminService.BlockingInterface admin,
final ClientProtos.ClientService.BlockingInterface client)
throws IOException {
HConnection connection =
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
Mockito.doNothing().when(connection).close();

View File

@ -30,12 +30,14 @@ 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.*;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -43,8 +45,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.log4j.Level;
/**
* Test {@link MetaReader}, {@link MetaEditor}.
@ -71,9 +71,6 @@ public class TestMetaReaderEditor {
};
@BeforeClass public static void beforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
UTIL.startMiniCluster(3);
Configuration c = new Configuration(UTIL.getConfiguration());

View File

@ -37,12 +37,12 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
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;
import org.apache.hadoop.hbase.util.Bytes;
@ -138,7 +138,8 @@ public class TestMetaReaderEditorNoCluster {
try {
// Mock an ClientProtocol. Our mock implementation will fail a few
// times when we go to open a scanner.
final ClientProtocol implementation = Mockito.mock(ClientProtocol.class);
final ClientProtos.ClientService.BlockingInterface implementation =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
// When scan called throw IOE 'Server not running' a few times
// before we return a scanner id. Whats WEIRD is that these
// exceptions do not show in the log because they are caught and only

View File

@ -20,13 +20,13 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionKey;
import org.mockito.Mockito;
/**
@ -52,13 +52,13 @@ public class HConnectionTestingUtility {
public static HConnection getMockedConnection(final Configuration conf)
throws ZooKeeperConnectionException {
HConnectionKey connectionKey = new HConnectionKey(conf);
synchronized (HConnectionManager.HBASE_INSTANCES) {
synchronized (HConnectionManager.CONNECTION_INSTANCES) {
HConnectionImplementation connection =
HConnectionManager.HBASE_INSTANCES.get(connectionKey);
HConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
connection = Mockito.mock(HConnectionImplementation.class);
Mockito.when(connection.getConfiguration()).thenReturn(conf);
HConnectionManager.HBASE_INSTANCES.put(connectionKey, connection);
HConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
}
return connection;
}
@ -84,16 +84,17 @@ public class HConnectionTestingUtility {
* @return Mock up a connection that returns a {@link Configuration} when
* {@link HConnection#getConfiguration()} is called, a 'location' when
* {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
* and that returns the passed {@link AdminProtocol} instance when
* and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
* {@link HConnection#getAdmin(ServerName)} is called, returns the passed
* {@link ClientProtocol} instance when {@link HConnection#getClient(ServerName)}
* is called (Be sure call
* {@link ClientProtos.ClientService.BlockingInterface} instance when
* {@link HConnection#getClient(ServerName)} is called (Be sure to call
* {@link HConnectionManager#deleteConnection(HConnectionKey, boolean)}
* when done with this mocked Connection.
* @throws IOException
*/
public static HConnection getMockedConnectionAndDecorate(final Configuration conf,
final AdminProtocol admin, final ClientProtocol client,
final AdminProtos.AdminService.BlockingInterface admin,
final ClientProtos.ClientService.BlockingInterface client,
final ServerName sn, final HRegionInfo hri)
throws IOException {
HConnection c = HConnectionTestingUtility.getMockedConnection(conf);
@ -133,12 +134,12 @@ public class HConnectionTestingUtility {
public static HConnection getSpiedConnection(final Configuration conf)
throws IOException {
HConnectionKey connectionKey = new HConnectionKey(conf);
synchronized (HConnectionManager.HBASE_INSTANCES) {
synchronized (HConnectionManager.CONNECTION_INSTANCES) {
HConnectionImplementation connection =
HConnectionManager.HBASE_INSTANCES.get(connectionKey);
HConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
connection = Mockito.spy(new HConnectionImplementation(conf, true));
HConnectionManager.HBASE_INSTANCES.put(connectionKey, connection);
HConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
}
return connection;
}
@ -148,8 +149,8 @@ public class HConnectionTestingUtility {
* @return Count of extant connection instances
*/
public static int getConnectionCount() {
synchronized (HConnectionManager.HBASE_INSTANCES) {
return HConnectionManager.HBASE_INSTANCES.size();
synchronized (HConnectionManager.CONNECTION_INSTANCES) {
return HConnectionManager.CONNECTION_INSTANCES.size();
}
}
}

View File

@ -52,8 +52,8 @@ import org.apache.hadoop.hbase.exceptions.TableNotEnabledException;
import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;

View File

@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.util.Bytes;
@ -59,10 +59,12 @@ public class TestClientScannerRPCTimeout {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
Configuration conf = TEST_UTIL.getConfiguration();
// Don't report so often so easier to see other rpcs
conf.setInt("hbase.regionserver.msginterval", 3 * 10000);
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, rpcTimeout);
conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithScanTimeout.class.getName());
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES_NUMBER);
@ -84,12 +86,14 @@ public class TestClientScannerRPCTimeout {
putToTable(ht, r1);
putToTable(ht, r2);
putToTable(ht, r3);
LOG.info("Wrote our three values");
RegionServerWithScanTimeout.seqNoToSleepOn = 1;
Scan scan = new Scan();
scan.setCaching(1);
ResultScanner scanner = ht.getScanner(scan);
Result result = scanner.next();
assertTrue("Expected row: row-1", Bytes.equals(r1, result.getRow()));
LOG.info("Got expected first row");
long t1 = System.currentTimeMillis();
result = scanner.next();
assertTrue((System.currentTimeMillis() - t1) > rpcTimeout);
@ -127,7 +131,8 @@ public class TestClientScannerRPCTimeout {
private static boolean sleepAlways = false;
private static int tryNumber = 0;
public RegionServerWithScanTimeout(Configuration conf) throws IOException, InterruptedException {
public RegionServerWithScanTimeout(Configuration conf)
throws IOException, InterruptedException {
super(conf);
}
@ -139,6 +144,7 @@ public class TestClientScannerRPCTimeout {
if (this.tableScannerId == request.getScannerId() &&
(sleepAlways || (!slept && seqNoToSleepOn == request.getNextCallSeq()))) {
try {
LOG.info("SLEEPING " + (rpcTimeout + 500));
Thread.sleep(rpcTimeout + 500);
} catch (InterruptedException e) {
}

View File

@ -19,7 +19,12 @@
*/
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.*;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.net.SocketTimeoutException;
import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -27,14 +32,22 @@ 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.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ipc.RandomTimeoutRpcEngine;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.ipc.RpcClient;
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)
public class TestClientTimeouts {
final Log LOG = LogFactory.getLog(getClass());
@ -46,7 +59,6 @@ public class TestClientTimeouts {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(SLAVES);
}
@ -68,22 +80,29 @@ public class TestClientTimeouts {
long lastLimit = HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT;
HConnection lastConnection = null;
boolean lastFailed = false;
int initialInvocations = RandomTimeoutRpcEngine.getNumberOfInvocations();
RandomTimeoutRpcEngine engine = new RandomTimeoutRpcEngine(TEST_UTIL.getConfiguration());
int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get();
RpcClient rpcClient = new RpcClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()) {
// Return my own instance, one that does random timeouts
@Override
public BlockingRpcChannel createBlockingRpcChannel(ServerName sn,
User ticket, int rpcTimeout) {
return new RandomTimeoutBlockingRpcChannel(this, sn, ticket, rpcTimeout);
}
};
try {
for (int i = 0; i < 5 || (lastFailed && i < 100); ++i) {
lastFailed = false;
// Ensure the HBaseAdmin uses a new connection by changing Configuration.
Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
conf.setLong(HConstants.HBASE_CLIENT_PREFETCH_LIMIT, ++lastLimit);
HBaseAdmin admin = null;
try {
HBaseAdmin admin = new HBaseAdmin(conf);
admin = new HBaseAdmin(conf);
HConnection connection = admin.getConnection();
assertFalse(connection == lastConnection);
lastConnection = connection;
// override the connection's rpc engine for timeout testing
((HConnectionManager.HConnectionImplementation)connection).setRpcEngine(engine);
// Override the connection's rpc client for timeout testing
((HConnectionManager.HConnectionImplementation)connection).setRpcClient(rpcClient);
// run some admin commands
HBaseAdmin.checkHBaseAvailable(conf);
admin.setBalancerRunning(false, false);
@ -91,13 +110,43 @@ public class TestClientTimeouts {
// Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
// a MasterNotRunningException. It's a bug if we get other exceptions.
lastFailed = true;
} finally {
admin.close();
}
}
// Ensure the RandomTimeoutRpcEngine is actually being used.
assertFalse(lastFailed);
assertTrue(RandomTimeoutRpcEngine.getNumberOfInvocations() > initialInvocations);
assertTrue(RandomTimeoutBlockingRpcChannel.invokations.get() > initialInvocations);
} finally {
engine.close();
rpcClient.stop();
}
}
/**
* Blocking rpc channel that goes via hbase rpc.
*/
static class RandomTimeoutBlockingRpcChannel extends RpcClient.BlockingRpcChannelImplementation {
private static final Random RANDOM = new Random(System.currentTimeMillis());
public static final double CHANCE_OF_TIMEOUT = 0.3;
private static AtomicInteger invokations = new AtomicInteger();
RandomTimeoutBlockingRpcChannel(final RpcClient rpcClient, final ServerName sn,
final User ticket, final int rpcTimeout) {
super(rpcClient, sn, ticket, rpcTimeout);
}
@Override
public Message callBlockingMethod(MethodDescriptor md,
RpcController controller, Message param, Message returnType)
throws ServiceException {
invokations.getAndIncrement();
if (RANDOM.nextFloat() < CHANCE_OF_TIMEOUT) {
// throw a ServiceException, becuase that is the only exception type that
// {@link ProtobufRpcEngine} throws. If this RpcEngine is used with a different
// "actual" type, this may not properly mimic the underlying RpcEngine.
throw new ServiceException(new SocketTimeoutException("fake timeout"));
}
return super.callBlockingMethod(md, controller, param, returnType);
}
}
}

View File

@ -78,8 +78,8 @@ import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
@ -124,8 +124,8 @@ public class TestFromClientSide {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
Configuration conf = TEST_UTIL.getConfiguration();
conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,

View File

@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.After;
@ -116,7 +116,7 @@ public class TestFromClientSide3 {
HConnection conn = HConnectionManager.getConnection(TEST_UTIL
.getConfiguration());
HRegionLocation loc = table.getRegionLocation(row, true);
AdminProtocol server = conn.getAdmin(loc.getServerName());
AdminProtos.AdminService.BlockingInterface server = conn.getAdmin(loc.getServerName());
byte[] regName = loc.getRegionInfo().getRegionName();
for (int i = 0; i < nFlushes; i++) {
@ -163,7 +163,8 @@ public class TestFromClientSide3 {
// Verify we have multiple store files.
HRegionLocation loc = hTable.getRegionLocation(row, true);
byte[] regionName = loc.getRegionInfo().getRegionName();
AdminProtocol server = connection.getAdmin(loc.getServerName());
AdminProtos.AdminService.BlockingInterface server =
connection.getAdmin(loc.getServerName());
assertTrue(ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() > 1);

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionKey;
import org.apache.hadoop.hbase.exceptions.RegionServerStoppedException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.master.ClusterStatusPublisher;
@ -99,7 +98,7 @@ public class TestHCM {
IllegalArgumentException, NoSuchFieldException,
IllegalAccessException, InterruptedException, ZooKeeperConnectionException, IOException {
HConnection last = null;
for (int i = 0; i <= (HConnectionManager.MAX_CACHED_HBASE_INSTANCES * 2); i++) {
for (int i = 0; i <= (HConnectionManager.MAX_CACHED_CONNECTION_INSTANCES * 2); i++) {
// set random key to differentiate the connection from previous ones
Configuration configuration = HBaseConfiguration.create();
configuration.set("somekey", String.valueOf(_randy.nextInt()));
@ -186,9 +185,9 @@ public class TestHCM {
// Save off current HConnections
Map<HConnectionKey, HConnectionImplementation> oldHBaseInstances =
new HashMap<HConnectionKey, HConnectionImplementation>();
oldHBaseInstances.putAll(HConnectionManager.HBASE_INSTANCES);
oldHBaseInstances.putAll(HConnectionManager.CONNECTION_INSTANCES);
HConnectionManager.HBASE_INSTANCES.clear();
HConnectionManager.CONNECTION_INSTANCES.clear();
try {
HConnection connection = HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
@ -198,8 +197,8 @@ public class TestHCM {
HConnectionManager.getConnection(TEST_UTIL.getConfiguration()));
} finally {
// Put original HConnections back
HConnectionManager.HBASE_INSTANCES.clear();
HConnectionManager.HBASE_INSTANCES.putAll(oldHBaseInstances);
HConnectionManager.CONNECTION_INSTANCES.clear();
HConnectionManager.CONNECTION_INSTANCES.putAll(oldHBaseInstances);
}
}

View File

@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
@ -53,8 +53,8 @@ import org.junit.experimental.categories.Category;
public class TestMultiParallel {
private static final Log LOG = LogFactory.getLog(TestMultiParallel.class);
{
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
}
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final byte[] VALUE = Bytes.toBytes("value");
@ -68,8 +68,8 @@ public class TestMultiParallel {
private static final int slaves = 2; // also used for testing HTable pool size
@BeforeClass public static void beforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
UTIL.startMiniCluster(slaves);
HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY));

View File

@ -34,8 +34,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -177,8 +177,8 @@ public class TestFilterWithScanLimits {
@BeforeClass
public static void setUp() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
TEST_UTIL.startMiniCluster(1);
initialize(TEST_UTIL.getConfiguration());

View File

@ -40,32 +40,33 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.log4j.Level;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.commons.logging.impl.Log4JLogger;
/**
* Tests changing data block encoding settings of a column family.
*/
@Category(LargeTests.class)
public class TestChangingEncoding {
private static final Log LOG = LogFactory.getLog(TestChangingEncoding.class);
static final String CF = "EncodingTestCF";
static final byte[] CF_BYTES = Bytes.toBytes(CF);
private static final int NUM_ROWS_PER_BATCH = 100;
private static final int NUM_COLS_PER_ROW = 20;
private static final HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final Configuration conf = TEST_UTIL.getConfiguration();
private static final int TIMEOUT_MS = 240000;
@ -100,6 +101,8 @@ public class TestChangingEncoding {
public static void setUpBeforeClass() throws Exception {
// Use a small flush size to create more HFiles.
conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 1024);
// ((Log4JLogger)RpcServerImplementation.LOG).getLogger().setLevel(Level.TRACE);
// ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.TRACE);
TEST_UTIL.startMiniCluster();
}
@ -190,6 +193,7 @@ public class TestChangingEncoding {
prepareTest("ChangingEncoding");
for (boolean encodeOnDisk : new boolean[]{false, true}) {
for (DataBlockEncoding encoding : ENCODINGS_TO_ITERATE) {
LOG.info("encoding=" + encoding + ", encodeOnDisk=" + encodeOnDisk);
setEncodingConf(encoding, encodeOnDisk);
writeSomeNewData();
verifyAllData();

Some files were not shown because too many files have changed in this diff Show More