HBASE-5889 Remove HRegionInterface

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1334314 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-05-05 04:22:52 +00:00
parent 49731c73dc
commit 007d6da28c
44 changed files with 2831 additions and 3960 deletions

View File

@ -24,7 +24,7 @@
<property>
<name>security.client.protocol.acl</name>
<value>*</value>
<description>ACL for HRegionInterface protocol implementations (ie.
<description>ACL for ClientProtocol and AdminProtocol implementations (ie.
clients talking to HRegionServers)
The ACL is a comma-separated list of user and group names. The user and
group list is separated by a blank. For e.g. "alice,bob users,wheel".

View File

@ -18,8 +18,9 @@
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.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.ipc.RegionServerStatusProtocol;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service;
@ -31,7 +32,8 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
*/
public class HBasePolicyProvider extends PolicyProvider {
protected static Service[] services = {
new Service("security.client.protocol.acl", HRegionInterface.class),
new Service("security.client.protocol.acl", ClientProtocol.class),
new Service("security.client.protocol.acl", AdminProtocol.class),
new Service("security.admin.protocol.acl", HMasterInterface.class),
new Service("security.masterregion.protocol.acl", RegionServerStatusProtocol.class)
};

View File

@ -24,31 +24,25 @@ String format = "html";
</%args>
<%import>
java.util.*;
java.io.IOException;
org.apache.hadoop.io.Text;
org.apache.hadoop.hbase.regionserver.HRegionServer;
org.apache.hadoop.hbase.regionserver.HRegion;
org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
org.apache.hadoop.hbase.util.Bytes;
org.apache.hadoop.hbase.HConstants;
org.apache.hadoop.hbase.HServerInfo;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerLoad;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.protobuf.ProtobufUtil;
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
</%import>
<%if format.equals("json") %>
<& ../common/TaskMonitorTmpl; filter = filter; format = "json" &>
<%java return; %>
</%if>
<%java>
HServerInfo serverInfo = null;
ServerName serverName = null;
serverInfo = regionServer.getHServerInfo();
serverName = regionServer.getServerName();
RegionServerMetrics metrics = regionServer.getMetrics();
List<HRegionInfo> onlineRegions = regionServer.getOnlineRegions();
ServerInfo serverInfo = ProtobufUtil.getServerInfo(regionServer);
ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer);
int masterInfoPort = regionServer.getConfiguration().getInt("hbase.master.info.port", 60010);
</%java>
<?xml version="1.0" encoding="UTF-8" ?>
@ -59,7 +53,7 @@ org.apache.hadoop.hbase.HBaseConfiguration;
-->
<html xmlns="http://www.w3.org/1999/xhtml">
<head><meta http-equiv="Content-Type" content="text/html;charset=UTF-8"/>
<title>HBase Region Server: <% serverName %>:<% serverInfo.getServerAddress().getPort() %></title>
<title>HBase Region Server: <% serverName %></title>
<link rel="stylesheet" type="text/css" href="/static/hbase.css" />
</head>

View File

@ -27,7 +27,6 @@ import java.util.regex.Pattern;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -184,15 +183,9 @@ public final class HConstants {
public static final String REGIONSERVER_INFO_PORT_AUTO =
REGIONSERVER_INFO_PORT + ".auto";
/** Parameter name for what region server interface to use. */
public static final String REGION_SERVER_CLASS = "hbase.regionserver.class";
/** Parameter name for what region server implementation to use. */
public static final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
/** Default region server interface class name. */
public static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName();
/** Parameter name for what master implementation to use. */
public static final String MASTER_IMPL= "hbase.master.impl";

View File

@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionReque
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
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.SplitRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
@ -1507,13 +1506,7 @@ public class HBaseAdmin implements Abortable, Closeable {
byte[] splitPoint) throws IOException {
AdminProtocol admin =
this.connection.getAdmin(sn.getHostname(), sn.getPort());
SplitRegionRequest request =
RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
try {
admin.splitRegion(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
ProtobufUtil.split(admin, hri, splitPoint);
}
/**

View File

@ -23,6 +23,8 @@ 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.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.ipc.VersionedProtocol;
import org.apache.hadoop.metrics.MetricsContext;
import org.apache.hadoop.metrics.MetricsRecord;
@ -66,7 +68,8 @@ public class HBaseRpcMetrics implements Updater {
initMethods(HMasterInterface.class);
initMethods(RegionServerStatusProtocol.class);
initMethods(HRegionInterface.class);
initMethods(ClientProtocol.class);
initMethods(AdminProtocol.class);
rpcStatistics = new HBaseRPCStatistics(this.registry, hostName, port);
}

View File

@ -1,630 +0,0 @@
/**
* Copyright 2010 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.ipc;
import java.io.IOException;
import java.net.ConnectException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MultiAction;
import org.apache.hadoop.hbase.client.MultiResponse;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.Exec;
import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.security.TokenInfo;
import org.apache.hadoop.hbase.security.KerberosInfo;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.hbase.ipc.VersionedProtocol;
/**
* Clients interact with HRegionServers using a handle to the HRegionInterface.
*
* <p>NOTE: if you change the interface, you must change the RPC version
* number in HBaseRPCProtocolVersion
*/
@KerberosInfo(
serverPrincipal = "hbase.regionserver.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface HRegionInterface extends VersionedProtocol, Stoppable, Abortable {
/**
* This Interfaces' version. Version changes when the Interface changes.
*/
// All HBase Interfaces used derive from HBaseRPCProtocolVersion. It
// maintained a single global version number on all HBase Interfaces. This
// meant all HBase RPC was broke though only one of the three RPC Interfaces
// had changed. This has since been undone.
public static final long VERSION = 29L;
/**
* Get metainfo about an HRegion
*
* @param regionName name of the region
* @return HRegionInfo object for region
* @throws NotServingRegionException
* @throws ConnectException
* @throws IOException This can manifest as an Hadoop ipc {@link RemoteException}
*/
public HRegionInfo getRegionInfo(final byte [] regionName)
throws NotServingRegionException, ConnectException, IOException;
/**
* Flush the given region
* @param region name
*/
public void flushRegion(byte[] regionName)
throws IllegalArgumentException, IOException;
/**
* Flush the given region if lastFlushTime < ifOlderThanTS
* @param region name
* @param timestamp
*/
public void flushRegion(byte[] regionName, long ifOlderThanTS)
throws IllegalArgumentException, IOException;
/**
* Gets last flush time for the given region
* @return the last flush time for a region
*/
public long getLastFlushTime(byte[] regionName);
/**
* Get a list of store files for a particular CF in a particular region
* @param region name
* @param CF name
* @return the list of store files
*/
public List<String> getStoreFileList(byte[] regionName, byte[] columnFamily)
throws IllegalArgumentException;
/**
* Get a list of store files for a set of CFs in a particular region
* @param region name
* @param CF names
* @return the list of store files
*/
public List<String> getStoreFileList(byte[] regionName, byte[][] columnFamilies)
throws IllegalArgumentException;
/**
* Get a list of store files for all CFs in a particular region
* @param region name
* @return the list of store files
*/
public List<String> getStoreFileList(byte[] regionName)
throws IllegalArgumentException;
/**
* Return all the data for the row that matches <i>row</i> exactly,
* or the one that immediately preceeds it.
*
* @param regionName region name
* @param row row key
* @param family Column family to look for row in.
* @return map of values
* @throws IOException e
*/
public Result getClosestRowBefore(final byte [] regionName,
final byte [] row, final byte [] family)
throws IOException;
/**
* Perform Get operation.
* @param regionName name of region to get from
* @param get Get operation
* @return Result
* @throws IOException e
*/
public Result get(byte [] regionName, Get get) throws IOException;
/**
* Perform exists operation.
* @param regionName name of region to get from
* @param get Get operation describing cell to test
* @return true if exists
* @throws IOException e
*/
public boolean exists(byte [] regionName, Get get) throws IOException;
/**
* Put data into the specified region
* @param regionName region name
* @param put the data to be put
* @throws IOException e
*/
public void put(final byte [] regionName, final Put put)
throws IOException;
/**
* Put an array of puts into the specified region
*
* @param regionName region name
* @param puts List of puts to execute
* @return The number of processed put's. Returns -1 if all Puts
* processed successfully.
* @throws IOException e
*/
public int put(final byte[] regionName, final List<Put> puts)
throws IOException;
/**
* Deletes all the KeyValues that match those found in the Delete object,
* if their ts <= to the Delete. In case of a delete with a specific ts it
* only deletes that specific KeyValue.
* @param regionName region name
* @param delete delete object
* @throws IOException e
*/
public void delete(final byte[] regionName, final Delete delete)
throws IOException;
/**
* Put an array of deletes into the specified region
*
* @param regionName region name
* @param deletes delete List to execute
* @return The number of processed deletes. Returns -1 if all Deletes
* processed successfully.
* @throws IOException e
*/
public int delete(final byte[] regionName, final List<Delete> deletes)
throws IOException;
/**
* Atomically checks if a row/family/qualifier value match the expectedValue.
* If it does, it adds the put. If passed expected value is null, then the
* check is for non-existance of the row/column.
*
* @param regionName region name
* @param row row to check
* @param family column family
* @param qualifier column qualifier
* @param value the expected value
* @param put data to put if check succeeds
* @throws IOException e
* @return true if the new put was execute, false otherwise
*/
public boolean checkAndPut(final byte[] regionName, final byte [] row,
final byte [] family, final byte [] qualifier, final byte [] value,
final Put put)
throws IOException;
/**
* Atomically checks if a row/family/qualifier value match the expectedValue.
* If it does, it adds the delete. If passed expected value is null, then the
* check is for non-existance of the row/column.
*
* @param regionName region name
* @param row row to check
* @param family column family
* @param qualifier column qualifier
* @param value the expected value
* @param delete data to delete if check succeeds
* @throws IOException e
* @return true if the new delete was execute, false otherwise
*/
public boolean checkAndDelete(final byte[] regionName, final byte [] row,
final byte [] family, final byte [] qualifier, final byte [] value,
final Delete delete)
throws IOException;
/**
* Atomically increments a column value. If the column value isn't long-like,
* this could throw an exception. If passed expected value is null, then the
* check is for non-existance of the row/column.
*
* @param regionName region name
* @param row row to check
* @param family column family
* @param qualifier column qualifier
* @param amount long amount to increment
* @param writeToWAL whether to write the increment to the WAL
* @return new incremented column value
* @throws IOException e
*/
public long incrementColumnValue(byte [] regionName, byte [] row,
byte [] family, byte [] qualifier, long amount, boolean writeToWAL)
throws IOException;
public void mutateRow(byte[] regionName, RowMutations rm)
throws IOException;
/**
* Appends values to one or more columns values in a row. Optionally
* Returns the updated keys after the append.
* <p>
* This operation does not appear atomic to readers. Appends are done
* under a row lock but readers do not take row locks.
* @param regionName region name
* @param append Append operation
* @return changed cells (maybe null)
*/
public Result append(byte[] regionName, Append append)
throws IOException;
/**
* Increments one or more columns values in a row. Returns the
* updated keys after the increment.
* <p>
* This operation does not appear atomic to readers. Increments are done
* under a row lock but readers do not take row locks.
* @param regionName region name
* @param increment increment operation
* @return incremented cells
*/
public Result increment(byte[] regionName, Increment increment)
throws IOException;
//
// remote scanner interface
//
/**
* Opens a remote scanner with a RowFilter.
*
* @param regionName name of region to scan
* @param scan configured scan object
* @return scannerId scanner identifier used in other calls
* @throws IOException e
*/
public long openScanner(final byte [] regionName, final Scan scan)
throws IOException;
/**
* Get the next set of values
* @param scannerId clientId passed to openScanner
* @return map of values; returns null if no results.
* @throws IOException e
*/
public Result next(long scannerId) throws IOException;
/**
* Get the next set of values
* @param scannerId clientId passed to openScanner
* @param numberOfRows the number of rows to fetch
* @return Array of Results (map of values); array is empty if done with this
* region and null if we are NOT to go to the next region (happens when a
* filter rules that the scan is done).
* @throws IOException e
*/
public Result [] next(long scannerId, int numberOfRows) throws IOException;
/**
* Close a scanner
*
* @param scannerId the scanner id returned by openScanner
* @throws IOException e
*/
public void close(long scannerId) throws IOException;
/**
* Opens a remote row lock.
*
* @param regionName name of region
* @param row row to lock
* @return lockId lock identifier
* @throws IOException e
*/
public long lockRow(final byte [] regionName, final byte [] row)
throws IOException;
/**
* Releases a remote row lock.
*
* @param regionName region name
* @param lockId the lock id returned by lockRow
* @throws IOException e
*/
public void unlockRow(final byte [] regionName, final long lockId)
throws IOException;
/**
* @return All regions online on this region server
* @throws IOException e
*/
public List<HRegionInfo> getOnlineRegions() throws IOException;
/**
* Method used when a master is taking the place of another failed one.
* @return This servers {@link HServerInfo}; it has RegionServer POV on the
* hostname which may not agree w/ how the Master sees this server.
* @throws IOException e
* @deprecated
*/
public HServerInfo getHServerInfo() throws IOException;
/**
* Method used for doing multiple actions(Deletes, Gets and Puts) in one call
* @param multi
* @return MultiResult
* @throws IOException
*/
public <R> MultiResponse multi(MultiAction<R> multi) throws IOException;
/**
* Atomically bulk load multiple HFiles (say from different column families)
* into an open region.
*
* @param familyPaths List of (family, hfile path) pairs
* @param regionName name of region to load hfiles into
* @return true if successful, false if failed recoverably
* @throws IOException if fails unrecoverably
*/
public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, byte[] regionName)
throws IOException;
// Master methods
/**
* Opens the specified region.
*
* @param region
* region to open
* @return RegionOpeningState
* OPENED - if region open request was successful.
* ALREADY_OPENED - if the region was already opened.
* FAILED_OPENING - if region opening failed.
*
* @throws IOException
*/
public RegionOpeningState openRegion(final HRegionInfo region) throws IOException;
/**
* Opens the specified region.
* @param region
* region to open
* @param versionOfOfflineNode
* the version of znode to compare when RS transitions the znode from
* OFFLINE state.
* @return RegionOpeningState
* OPENED - if region open request was successful.
* ALREADY_OPENED - if the region was already opened.
* FAILED_OPENING - if region opening failed.
* @throws IOException
*/
public RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode)
throws IOException;
/**
* Opens the specified regions.
* @param regions regions to open
* @throws IOException
*/
public void openRegions(final List<HRegionInfo> regions) throws IOException;
/**
* Closes the specified region.
* @param region region to close
* @return true if closing region, false if not
* @throws IOException
*/
public boolean closeRegion(final HRegionInfo region)
throws IOException;
/**
* Closes the specified region.
* @param region region to close
* @param versionOfClosingNode
* the version of znode to compare when RS transitions the znode
* from CLOSING state.
* @return true if closing region, false if not
* @throws IOException
*/
public boolean closeRegion(final HRegionInfo region,
final int versionOfClosingNode)
throws IOException;
/**
* Closes the specified region and will use or not use ZK during the close
* according to the specified flag.
* @param region region to close
* @param zk true if transitions should be done in ZK, false if not
* @return true if closing region, false if not
* @throws IOException
*/
public boolean closeRegion(final HRegionInfo region, final boolean zk)
throws IOException;
/**
* Closes the region in the RS with the specified encoded regionName and will
* use or not use ZK during the close according to the specified flag. Note
* that the encoded region name is in byte format.
*
* @param encodedRegionName
* in bytes
* @param zk
* true if to use zookeeper, false if need not.
* @return true if region is closed, false if not.
* @throws IOException
*/
public boolean closeRegion(byte[] encodedRegionName, final boolean zk)
throws IOException;
// Region administrative methods
/**
* Flushes the MemStore of the specified region.
* <p>
* This method is synchronous.
* @param regionInfo region to flush
* @throws NotServingRegionException
* @throws IOException
* @deprecated use {@link #flushRegion(byte[])} instead
*/
void flushRegion(HRegionInfo regionInfo)
throws NotServingRegionException, IOException;
/**
* Splits the specified region.
* <p>
* This method currently flushes the region and then forces a compaction which
* will then trigger a split. The flush is done synchronously but the
* compaction is asynchronous.
* @param regionInfo region to split
* @throws NotServingRegionException
* @throws IOException
*/
void splitRegion(HRegionInfo regionInfo)
throws NotServingRegionException, IOException;
/**
* Splits the specified region.
* <p>
* This method currently flushes the region and then forces a compaction which
* will then trigger a split. The flush is done synchronously but the
* compaction is asynchronous.
* @param regionInfo region to split
* @param splitPoint the explicit row to split on
* @throws NotServingRegionException
* @throws IOException
*/
void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
throws NotServingRegionException, IOException;
/**
* Compacts the specified region. Performs a major compaction if specified.
* <p>
* This method is asynchronous.
* @param regionInfo region to compact
* @param major true to force major compaction
* @throws NotServingRegionException
* @throws IOException
*/
void compactRegion(HRegionInfo regionInfo, boolean major)
throws NotServingRegionException, IOException;
/**
* Replicates the given entries. The guarantee is that the given entries
* will be durable on the slave cluster if this method returns without
* any exception.
* hbase.replication has to be set to true for this to work.
*
* @param entries entries to replicate
* @throws IOException
*/
public void replicateLogEntries(HLog.Entry[] entries) throws IOException;
/**
* Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
* method using the registered protocol handlers.
* {@link CoprocessorProtocol} implementations must be registered via the
* {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
* method before they are available.
*
* @param regionName name of the region against which the invocation is executed
* @param call an {@code Exec} instance identifying the protocol, method name,
* and parameters for the method invocation
* @return an {@code ExecResult} instance containing the region name of the
* invocation and the return value
* @throws IOException if no registered protocol handler is found or an error
* occurs during the invocation
* @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
*/
ExecResult execCoprocessor(byte[] regionName, Exec call)
throws IOException;
/**
* Atomically checks if a row/family/qualifier value match the expectedValue.
* If it does, it adds the put. If passed expected value is null, then the
* check is for non-existance of the row/column.
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param compareOp
* @param comparator
* @param put
* @throws IOException
* @return true if the new put was execute, false otherwise
*/
public boolean checkAndPut(final byte[] regionName, final byte[] row,
final byte[] family, final byte[] qualifier, final CompareOp compareOp,
final WritableByteArrayComparable comparator, final Put put)
throws IOException;
/**
* Atomically checks if a row/family/qualifier value match the expectedValue.
* If it does, it adds the delete. If passed expected value is null, then the
* check is for non-existance of the row/column.
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param compareOp
* @param comparator
* @param delete
* @throws IOException
* @return true if the new put was execute, false otherwise
*/
public boolean checkAndDelete(final byte[] regionName, final byte[] row,
final byte[] family, final byte[] qualifier, final CompareOp compareOp,
final WritableByteArrayComparable comparator, final Delete delete)
throws IOException;
/**
* Performs a BlockCache summary and returns a List of BlockCacheColumnFamilySummary objects.
* This method could be fairly heavyweight in that it evaluates the entire HBase file-system
* against what is in the RegionServer BlockCache.
*
* @return BlockCacheColumnFamilySummary
* @throws IOException exception
*/
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries() throws IOException;
/**
* Roll the log writer. That is, start writing log messages to a new file.
*
* @throws IOException
* @throws FailedLogCloseException
* @return If lots of logs, flush the returned regions so next time through
* we can clean logs. Returns null if nothing to flush. Names are actual
* region names as returned by {@link HRegionInfo#getEncodedName()}
*/
public byte[][] rollHLogWriter() throws IOException, FailedLogCloseException;
@Override
public void stop(String why);
}

View File

@ -196,7 +196,6 @@ implements Configurable {
ZKUtil.applyClusterKeyToConf(this.conf, address);
}
if (serverClass != null) {
this.conf.set(HConstants.REGION_SERVER_CLASS, serverClass);
this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl);
}
if (zkClientPort != 0) {

View File

@ -194,7 +194,6 @@ class CatalogJanitor extends Chore {
/**
* If daughters no longer hold reference to the parents, delete the parent.
* @param server HRegionInterface of meta server to talk to
* @param parent HRegionInfo of split offlined parent
* @param rowContent Content of <code>parent</code> row in
* <code>metaRegionName</code>

View File

@ -38,8 +38,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HConstants;
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.OpenRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.Server;
@ -55,6 +58,8 @@ import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import com.google.protobuf.ServiceException;
/**
* The ServerManager class manages info about region servers.
* <p>
@ -484,7 +489,14 @@ public class ServerManager {
" failed because no RPC connection found to this server");
return RegionOpeningState.FAILED_OPENING;
}
return ProtobufUtil.openRegion(admin, region, versionOfOfflineNode);
OpenRegionRequest request =
RequestConverter.buildOpenRegionRequest(region, versionOfOfflineNode);
try {
OpenRegionResponse response = admin.openRegion(null, request);
return ResponseConverter.getRegionOpeningState(response);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
/**

View File

@ -77,8 +77,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRespo
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UUID;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry.WALEdit.FamilyScope;
@ -102,7 +103,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -1287,19 +1287,16 @@ public final class ProtobufUtil {
*
* @param admin
* @param region
* @param versionOfOfflineNode
* @return the region opening state
* @throws IOException
*/
public static RegionOpeningState openRegion(final AdminProtocol admin,
final HRegionInfo region, final int versionOfOfflineNode) throws IOException {
public static void openRegion(final AdminProtocol admin,
final HRegionInfo region) throws IOException {
OpenRegionRequest request =
RequestConverter.buildOpenRegionRequest(region, versionOfOfflineNode);
RequestConverter.buildOpenRegionRequest(region, -1);
try {
OpenRegionResponse response = admin.openRegion(null, request);
return ResponseConverter.getRegionOpeningState(response);
admin.openRegion(null, request);
} catch (ServiceException se) {
throw getRemoteException(se);
throw ProtobufUtil.getRemoteException(se);
}
}
@ -1337,8 +1334,10 @@ public final class ProtobufUtil {
GetOnlineRegionResponse response =
admin.getOnlineRegion(null, request);
regions = new ArrayList<HRegionInfo>();
for (RegionInfo regionInfo: response.getRegionInfoList()) {
regions.add(toRegionInfo(regionInfo));
if (response != null) { // it can be null only mockup testing region sever
for (RegionInfo regionInfo: response.getRegionInfoList()) {
regions.add(toRegionInfo(regionInfo));
}
}
return regions;
} catch (ServiceException se) {
@ -1353,12 +1352,12 @@ public final class ProtobufUtil {
* @return the server name
* @throws IOException
*/
public static ServerName getServerInfo(
public static ServerInfo getServerInfo(
final AdminProtocol admin) throws IOException {
GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest();
try {
GetServerInfoResponse response = admin.getServerInfo(null, request);
return toServerName(response.getServerName());
return response.getServerInfo();
} catch (ServiceException se) {
throw getRemoteException(se);
}
@ -1404,6 +1403,27 @@ public final class ProtobufUtil {
}
}
/**
* A helper to split a region using admin protocol.
*
* @param admin
* @param hri
* @param splitPoint
* @throws IOException
*/
public static void split(final AdminProtocol admin,
final HRegionInfo hri, byte[] splitPoint) throws IOException {
SplitRegionRequest request =
RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
try {
admin.splitRegion(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
// End helpers for Admin
/*
* Get the total (read + write) requests from a RegionLoad pb
* @param rl - RegionLoad pb
@ -1416,6 +1436,4 @@ public final class ProtobufUtil {
return rl.getReadRequestsCount() + rl.getWriteRequestsCount();
}
// End helpers for Admin
}

View File

@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
/**
* Helper utility to build protocol buffer requests,

View File

@ -23,11 +23,14 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
@ -170,5 +173,39 @@ public final class ResponseConverter {
return proto.getClosed();
}
/**
* A utility to build a GetServerInfoResponse.
*
* @param serverName
* @param webuiPort
* @return the response
*/
public static GetServerInfoResponse buildGetServerInfoResponse(
final ServerName serverName, final int webuiPort) {
GetServerInfoResponse.Builder builder = GetServerInfoResponse.newBuilder();
ServerInfo.Builder serverInfoBuilder = ServerInfo.newBuilder();
serverInfoBuilder.setServerName(ProtobufUtil.toServerName(serverName));
if (webuiPort >= 0) {
serverInfoBuilder.setWebuiPort(webuiPort);
}
builder.setServerInfo(serverInfoBuilder.build());
return builder.build();
}
/**
* A utility to build a GetOnlineRegionResponse.
*
* @param regions
* @return the response
*/
public static GetOnlineRegionResponse buildGetOnlineRegionResponse(
final List<HRegionInfo> regions) {
GetOnlineRegionResponse.Builder builder = GetOnlineRegionResponse.newBuilder();
for (HRegionInfo region: regions) {
builder.addRegionInfo(ProtobufUtil.toRegionInfo(region));
}
return builder.build();
}
// End utilities for Admin
}

View File

@ -126,13 +126,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -799,13 +801,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -2066,13 +2070,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -2558,13 +2564,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -3265,13 +3273,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -3960,13 +3970,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -4963,13 +4975,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -5477,13 +5491,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -6229,13 +6245,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -7490,13 +7508,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -8241,13 +8261,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -8994,13 +9016,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -10447,13 +10471,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -11251,13 +11277,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -11992,13 +12020,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -12556,13 +12586,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -13011,13 +13043,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -13495,13 +13529,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -13832,12 +13868,12 @@ public final class ClientProtos {
// required string path = 2;
public static final int PATH_FIELD_NUMBER = 2;
private Object path_;
private java.lang.Object path_;
public boolean hasPath() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getPath() {
Object ref = path_;
java.lang.Object ref = path_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -13851,7 +13887,7 @@ public final class ClientProtos {
}
}
private com.google.protobuf.ByteString getPathBytes() {
Object ref = path_;
java.lang.Object ref = path_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -13914,13 +13950,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -14225,12 +14263,12 @@ public final class ClientProtos {
}
// required string path = 2;
private Object path_ = "";
private java.lang.Object path_ = "";
public boolean hasPath() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getPath() {
Object ref = path_;
java.lang.Object ref = path_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
path_ = s;
@ -14364,13 +14402,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -15073,13 +15113,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -15440,12 +15482,12 @@ public final class ClientProtos {
// required string protocolName = 2;
public static final int PROTOCOLNAME_FIELD_NUMBER = 2;
private Object protocolName_;
private java.lang.Object protocolName_;
public boolean hasProtocolName() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getProtocolName() {
Object ref = protocolName_;
java.lang.Object ref = protocolName_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -15459,7 +15501,7 @@ public final class ClientProtos {
}
}
private com.google.protobuf.ByteString getProtocolNameBytes() {
Object ref = protocolName_;
java.lang.Object ref = protocolName_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -15472,12 +15514,12 @@ public final class ClientProtos {
// required string methodName = 3;
public static final int METHODNAME_FIELD_NUMBER = 3;
private Object methodName_;
private java.lang.Object methodName_;
public boolean hasMethodName() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
public String getMethodName() {
Object ref = methodName_;
java.lang.Object ref = methodName_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -15491,7 +15533,7 @@ public final class ClientProtos {
}
}
private com.google.protobuf.ByteString getMethodNameBytes() {
Object ref = methodName_;
java.lang.Object ref = methodName_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -15636,13 +15678,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -16094,12 +16138,12 @@ public final class ClientProtos {
}
// required string protocolName = 2;
private Object protocolName_ = "";
private java.lang.Object protocolName_ = "";
public boolean hasProtocolName() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getProtocolName() {
Object ref = protocolName_;
java.lang.Object ref = protocolName_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
protocolName_ = s;
@ -16130,12 +16174,12 @@ public final class ClientProtos {
}
// required string methodName = 3;
private Object methodName_ = "";
private java.lang.Object methodName_ = "";
public boolean hasMethodName() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
public String getMethodName() {
Object ref = methodName_;
java.lang.Object ref = methodName_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
methodName_ = s;
@ -16676,13 +16720,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -17281,13 +17327,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -17814,13 +17862,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -18577,13 +18627,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -19245,13 +19297,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -20016,13 +20070,15 @@ public final class ClientProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}

View File

@ -289,13 +289,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -994,13 +996,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -4222,13 +4226,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -4757,13 +4763,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -5343,12 +5351,12 @@ public final class HBaseProtos {
private int bitField0_;
// required string hostName = 1;
public static final int HOSTNAME_FIELD_NUMBER = 1;
private Object hostName_;
private java.lang.Object hostName_;
public boolean hasHostName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getHostName() {
Object ref = hostName_;
java.lang.Object ref = hostName_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -5362,7 +5370,7 @@ public final class HBaseProtos {
}
}
private com.google.protobuf.ByteString getHostNameBytes() {
Object ref = hostName_;
java.lang.Object ref = hostName_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -5449,13 +5457,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -5755,12 +5765,12 @@ public final class HBaseProtos {
private int bitField0_;
// required string hostName = 1;
private Object hostName_ = "";
private java.lang.Object hostName_ = "";
public boolean hasHostName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getHostName() {
Object ref = hostName_;
java.lang.Object ref = hostName_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
hostName_ = s;
@ -6301,12 +6311,12 @@ public final class HBaseProtos {
private int bitField0_;
// required string name = 1;
public static final int NAME_FIELD_NUMBER = 1;
private Object name_;
private java.lang.Object name_;
public boolean hasName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getName() {
Object ref = name_;
java.lang.Object ref = name_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -6320,7 +6330,7 @@ public final class HBaseProtos {
}
}
private com.google.protobuf.ByteString getNameBytes() {
Object ref = name_;
java.lang.Object ref = name_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -6333,12 +6343,12 @@ public final class HBaseProtos {
// required string value = 2;
public static final int VALUE_FIELD_NUMBER = 2;
private Object value_;
private java.lang.Object value_;
public boolean hasValue() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getValue() {
Object ref = value_;
java.lang.Object ref = value_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -6352,7 +6362,7 @@ public final class HBaseProtos {
}
}
private com.google.protobuf.ByteString getValueBytes() {
Object ref = value_;
java.lang.Object ref = value_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -6415,13 +6425,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -6702,12 +6714,12 @@ public final class HBaseProtos {
private int bitField0_;
// required string name = 1;
private Object name_ = "";
private java.lang.Object name_ = "";
public boolean hasName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getName() {
Object ref = name_;
java.lang.Object ref = name_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
name_ = s;
@ -6738,12 +6750,12 @@ public final class HBaseProtos {
}
// required string value = 2;
private Object value_ = "";
private java.lang.Object value_ = "";
public boolean hasValue() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getValue() {
Object ref = value_;
java.lang.Object ref = value_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
value_ = s;
@ -6826,12 +6838,12 @@ public final class HBaseProtos {
private int bitField0_;
// required string name = 1;
public static final int NAME_FIELD_NUMBER = 1;
private Object name_;
private java.lang.Object name_;
public boolean hasName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getName() {
Object ref = name_;
java.lang.Object ref = name_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -6845,7 +6857,7 @@ public final class HBaseProtos {
}
}
private com.google.protobuf.ByteString getNameBytes() {
Object ref = name_;
java.lang.Object ref = name_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -6914,13 +6926,15 @@ public final class HBaseProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -7197,12 +7211,12 @@ public final class HBaseProtos {
private int bitField0_;
// required string name = 1;
private Object name_ = "";
private java.lang.Object name_ = "";
public boolean hasName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getName() {
Object ref = name_;
java.lang.Object ref = name_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
name_ = s;

View File

@ -50,12 +50,12 @@ public final class RPCProtos {
private int bitField0_;
// required string effectiveUser = 1;
public static final int EFFECTIVEUSER_FIELD_NUMBER = 1;
private Object effectiveUser_;
private java.lang.Object effectiveUser_;
public boolean hasEffectiveUser() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getEffectiveUser() {
Object ref = effectiveUser_;
java.lang.Object ref = effectiveUser_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -69,7 +69,7 @@ public final class RPCProtos {
}
}
private com.google.protobuf.ByteString getEffectiveUserBytes() {
Object ref = effectiveUser_;
java.lang.Object ref = effectiveUser_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -82,12 +82,12 @@ public final class RPCProtos {
// required string realUser = 2;
public static final int REALUSER_FIELD_NUMBER = 2;
private Object realUser_;
private java.lang.Object realUser_;
public boolean hasRealUser() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getRealUser() {
Object ref = realUser_;
java.lang.Object ref = realUser_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -101,7 +101,7 @@ public final class RPCProtos {
}
}
private com.google.protobuf.ByteString getRealUserBytes() {
Object ref = realUser_;
java.lang.Object ref = realUser_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -164,13 +164,15 @@ public final class RPCProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -451,12 +453,12 @@ public final class RPCProtos {
private int bitField0_;
// required string effectiveUser = 1;
private Object effectiveUser_ = "";
private java.lang.Object effectiveUser_ = "";
public boolean hasEffectiveUser() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getEffectiveUser() {
Object ref = effectiveUser_;
java.lang.Object ref = effectiveUser_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
effectiveUser_ = s;
@ -487,12 +489,12 @@ public final class RPCProtos {
}
// required string realUser = 2;
private Object realUser_ = "";
private java.lang.Object realUser_ = "";
public boolean hasRealUser() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getRealUser() {
Object ref = realUser_;
java.lang.Object ref = realUser_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
realUser_ = s;
@ -541,7 +543,7 @@ 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.ipc.HRegionInterface"];
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
boolean hasProtocol();
String getProtocol();
}
@ -587,14 +589,14 @@ public final class RPCProtos {
return userInfo_;
}
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"];
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
public static final int PROTOCOL_FIELD_NUMBER = 2;
private Object protocol_;
private java.lang.Object protocol_;
public boolean hasProtocol() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getProtocol() {
Object ref = protocol_;
java.lang.Object ref = protocol_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -608,7 +610,7 @@ public final class RPCProtos {
}
}
private com.google.protobuf.ByteString getProtocolBytes() {
Object ref = protocol_;
java.lang.Object ref = protocol_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -621,7 +623,7 @@ public final class RPCProtos {
private void initFields() {
userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
protocol_ = "org.apache.hadoop.hbase.ipc.HRegionInterface";
protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol";
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -669,13 +671,15 @@ public final class RPCProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -835,7 +839,7 @@ public final class RPCProtos {
userInfoBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
protocol_ = "org.apache.hadoop.hbase.ipc.HRegionInterface";
protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol";
bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
@ -1056,13 +1060,13 @@ public final class RPCProtos {
return userInfoBuilder_;
}
// optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"];
private Object protocol_ = "org.apache.hadoop.hbase.ipc.HRegionInterface";
// 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() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getProtocol() {
Object ref = protocol_;
java.lang.Object ref = protocol_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
protocol_ = s;
@ -1211,13 +1215,15 @@ public final class RPCProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -1591,12 +1597,12 @@ public final class RPCProtos {
private int bitField0_;
// required string exceptionName = 1;
public static final int EXCEPTIONNAME_FIELD_NUMBER = 1;
private Object exceptionName_;
private java.lang.Object exceptionName_;
public boolean hasExceptionName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getExceptionName() {
Object ref = exceptionName_;
java.lang.Object ref = exceptionName_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -1610,7 +1616,7 @@ public final class RPCProtos {
}
}
private com.google.protobuf.ByteString getExceptionNameBytes() {
Object ref = exceptionName_;
java.lang.Object ref = exceptionName_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -1623,12 +1629,12 @@ public final class RPCProtos {
// optional string stackTrace = 2;
public static final int STACKTRACE_FIELD_NUMBER = 2;
private Object stackTrace_;
private java.lang.Object stackTrace_;
public boolean hasStackTrace() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getStackTrace() {
Object ref = stackTrace_;
java.lang.Object ref = stackTrace_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -1642,7 +1648,7 @@ public final class RPCProtos {
}
}
private com.google.protobuf.ByteString getStackTraceBytes() {
Object ref = stackTrace_;
java.lang.Object ref = stackTrace_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -1701,13 +1707,15 @@ public final class RPCProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -1984,12 +1992,12 @@ public final class RPCProtos {
private int bitField0_;
// required string exceptionName = 1;
private Object exceptionName_ = "";
private java.lang.Object exceptionName_ = "";
public boolean hasExceptionName() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getExceptionName() {
Object ref = exceptionName_;
java.lang.Object ref = exceptionName_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
exceptionName_ = s;
@ -2020,12 +2028,12 @@ public final class RPCProtos {
}
// optional string stackTrace = 2;
private Object stackTrace_ = "";
private java.lang.Object stackTrace_ = "";
public boolean hasStackTrace() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public String getStackTrace() {
Object ref = stackTrace_;
java.lang.Object ref = stackTrace_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
stackTrace_ = s;
@ -2232,13 +2240,15 @@ public final class RPCProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -2785,17 +2795,17 @@ public final class RPCProtos {
static {
java.lang.String[] descriptorData = {
"\n\tRPC.proto\":\n\017UserInformation\022\025\n\reffect" +
"iveUser\030\001 \002(\t\022\020\n\010realUser\030\002 \002(\t\"v\n\020Conne" +
"iveUser\030\001 \002(\t\022\020\n\010realUser\030\002 \002(\t\"w\n\020Conne" +
"ctionHeader\022\"\n\010userInfo\030\001 \001(\0132\020.UserInfo" +
"rmation\022>\n\010protocol\030\002 \001(\t:,org.apache.ha" +
"doop.hbase.ipc.HRegionInterface\"-\n\nRpcRe" +
"quest\022\016\n\006callId\030\001 \002(\005\022\017\n\007request\030\002 \001(\014\"9" +
"\n\014RpcException\022\025\n\rexceptionName\030\001 \002(\t\022\022\n" +
"\nstackTrace\030\002 \001(\t\"`\n\013RpcResponse\022\016\n\006call" +
"Id\030\001 \002(\005\022\r\n\005error\030\002 \002(\010\022\020\n\010response\030\003 \001(" +
"\014\022 \n\texception\030\004 \001(\0132\r.RpcExceptionB<\n*o",
"rg.apache.hadoop.hbase.protobuf.generate" +
"dB\tRPCProtosH\001\240\001\001"
"rmation\022?\n\010protocol\030\002 \001(\t:-org.apache.ha" +
"doop.hbase.client.ClientProtocol\"-\n\nRpcR" +
"equest\022\016\n\006callId\030\001 \002(\005\022\017\n\007request\030\002 \001(\014\"" +
"9\n\014RpcException\022\025\n\rexceptionName\030\001 \002(\t\022\022" +
"\n\nstackTrace\030\002 \001(\t\"`\n\013RpcResponse\022\016\n\006cal" +
"lId\030\001 \002(\005\022\r\n\005error\030\002 \002(\010\022\020\n\010response\030\003 \001" +
"(\014\022 \n\texception\030\004 \001(\0132\r.RpcExceptionB<\n*",
"org.apache.hadoop.hbase.protobuf.generat" +
"edB\tRPCProtosH\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

View File

@ -102,13 +102,15 @@ public final class ZooKeeperProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -573,13 +575,15 @@ public final class ZooKeeperProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -988,12 +992,12 @@ public final class ZooKeeperProtos {
private int bitField0_;
// required string clusterId = 1;
public static final int CLUSTERID_FIELD_NUMBER = 1;
private Object clusterId_;
private java.lang.Object clusterId_;
public boolean hasClusterId() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getClusterId() {
Object ref = clusterId_;
java.lang.Object ref = clusterId_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -1007,7 +1011,7 @@ public final class ZooKeeperProtos {
}
}
private com.google.protobuf.ByteString getClusterIdBytes() {
Object ref = clusterId_;
java.lang.Object ref = clusterId_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -1058,13 +1062,15 @@ public final class ZooKeeperProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -1318,12 +1324,12 @@ public final class ZooKeeperProtos {
private int bitField0_;
// required string clusterId = 1;
private Object clusterId_ = "";
private java.lang.Object clusterId_ = "";
public boolean hasClusterId() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getClusterId() {
Object ref = clusterId_;
java.lang.Object ref = clusterId_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
clusterId_ = s;
@ -1402,12 +1408,12 @@ public final class ZooKeeperProtos {
private int bitField0_;
// required string startDate = 1;
public static final int STARTDATE_FIELD_NUMBER = 1;
private Object startDate_;
private java.lang.Object startDate_;
public boolean hasStartDate() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getStartDate() {
Object ref = startDate_;
java.lang.Object ref = startDate_;
if (ref instanceof String) {
return (String) ref;
} else {
@ -1421,7 +1427,7 @@ public final class ZooKeeperProtos {
}
}
private com.google.protobuf.ByteString getStartDateBytes() {
Object ref = startDate_;
java.lang.Object ref = startDate_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
@ -1472,13 +1478,15 @@ public final class ZooKeeperProtos {
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected Object writeReplace() throws java.io.ObjectStreamException {
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final Object obj) {
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
@ -1732,12 +1740,12 @@ public final class ZooKeeperProtos {
private int bitField0_;
// required string startDate = 1;
private Object startDate_ = "";
private java.lang.Object startDate_ = "";
public boolean hasStartDate() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public String getStartDate() {
Object ref = startDate_;
java.lang.Object ref = startDate_;
if (!(ref instanceof String)) {
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
startDate_ = s;

View File

@ -57,14 +57,14 @@ public class HRegionThriftServer extends Thread {
public static final Log LOG = LogFactory.getLog(HRegionThriftServer.class);
private final RegionServer rs;
private final HRegionServer rs;
private final ThriftServerRunner serverRunner;
/**
* Create an instance of the glue object that connects the
* RegionServer with the standard ThriftServer implementation
*/
HRegionThriftServer(RegionServer regionServer, Configuration conf)
HRegionThriftServer(HRegionServer regionServer, Configuration conf)
throws IOException {
super("Region Thrift Server");
this.rs = regionServer;

View File

@ -174,7 +174,6 @@ message ReplicateWALEntryRequest {
message ReplicateWALEntryResponse {
}
// Replacement for rollHLogWriter in HRegionInterface
message RollWALWriterRequest {
}
@ -193,8 +192,13 @@ message StopServerResponse {
message GetServerInfoRequest {
}
message GetServerInfoResponse {
message ServerInfo {
required ServerName serverName = 1;
optional uint32 webuiPort = 2;
}
message GetServerInfoResponse {
required ServerInfo serverInfo = 1;
}
service AdminService {

View File

@ -20,7 +20,7 @@
* Specification of (unsecure) HBase RPC:
*
* Client needs to set up a connection first to a server serving a certain
* HBase protocol (like HRegionInterface). Once the connection is set up, the
* HBase protocol (like ClientProtocol). Once the connection is set up, the
* client and server communicates on that channel for RPC requests/responses.
* The sections below describe the flow.
*
@ -58,7 +58,7 @@ message ConnectionHeader {
/** Protocol name for next rpc layer
* the client created a proxy with this protocol name
*/
optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"];
optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
}

View File

@ -108,13 +108,6 @@
<description>The address for the HBase RegionServer web UI
</description>
</property>
<property>
<name>hbase.regionserver.class</name>
<value>org.apache.hadoop.hbase.ipc.HRegionInterface</value>
<description>The RegionServer interface to use.
Used by the client opening proxy to remote region server.
</description>
</property>
<property>
<name>hbase.client.pause</name>
<value>1000</value>

View File

@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -80,7 +81,7 @@ public class TestDrainingServer {
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
HRegionServer hrs = cluster.getRegionServer(i);
Assert.assertFalse(hrs.getOnlineRegions().isEmpty());
Assert.assertFalse(ProtobufUtil.getOnlineRegions(hrs).isEmpty());
}
}
@ -128,7 +129,7 @@ public class TestDrainingServer {
final int regionsOnDrainingServer =
drainingServer.getNumberOfOnlineRegions();
Assert.assertTrue(regionsOnDrainingServer > 0);
List<HRegionInfo> hris = drainingServer.getOnlineRegions();
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(drainingServer);
for (HRegionInfo hri : hris) {
// Pass null and AssignmentManager will chose a random server BUT it
// should exclude draining servers.

View File

@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
@ -81,7 +82,7 @@ public class TestGlobalMemStoreSize {
for (HRegionServer server : getOnlineRegionServers()) {
long globalMemStoreSize = 0;
for (HRegionInfo regionInfo : server.getOnlineRegions()) {
for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server)) {
globalMemStoreSize +=
server.getFromOnlineRegions(regionInfo.getEncodedName()).
getMemstoreSize().get();
@ -96,7 +97,7 @@ public class TestGlobalMemStoreSize {
LOG.info("Starting flushes on " + server.getServerName() +
", size=" + server.getRegionServerAccounting().getGlobalMemstoreSize());
for (HRegionInfo regionInfo : server.getOnlineRegions()) {
for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server)) {
HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName());
flush(r, server);
}
@ -111,7 +112,7 @@ public class TestGlobalMemStoreSize {
if (size > 0) {
// If size > 0, see if its because the meta region got edits while
// our test was running....
for (HRegionInfo regionInfo : server.getOnlineRegions()) {
for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server)) {
HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName());
long l = r.getMemstoreSize().longValue();
if (l > 0) {
@ -148,7 +149,7 @@ public class TestGlobalMemStoreSize {
private int getRegionCount() throws IOException {
int total = 0;
for (HRegionServer server : getOnlineRegionServers()) {
total += server.getOnlineRegions().size();
total += ProtobufUtil.getOnlineRegions(server).size();
}
return total;
}

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
@ -135,7 +136,7 @@ public class TestRegionRebalancing {
private int getRegionCount() throws IOException {
int total = 0;
for (HRegionServer server : getOnlineRegionServers()) {
total += server.getOnlineRegions().size();
total += ProtobufUtil.getOnlineRegions(server).size();
}
return total;
}
@ -167,11 +168,11 @@ public class TestRegionRebalancing {
+ ", up border: " + avgLoadPlusSlop + "; attempt: " + i);
for (HRegionServer server : servers) {
int serverLoad = server.getOnlineRegions().size();
int serverLoad = ProtobufUtil.getOnlineRegions(server).size();
LOG.debug(server.getServerName() + " Avg: " + avg + " actual: " + serverLoad);
if (!(avg > 2.0 && serverLoad <= avgLoadPlusSlop
&& serverLoad >= avgLoadMinusSlop)) {
for (HRegionInfo hri : server.getOnlineRegions()) {
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(server)) {
if (hri.isMetaRegion() || hri.isRootRegion()) serverLoad--;
// LOG.debug(hri.getRegionNameAsString());
}

View File

@ -50,7 +50,6 @@ 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.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
@ -454,8 +453,9 @@ public class TestCatalogTracker {
* @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 HRegionInterface} instance when
* {@link HConnection#getHRegionConnection(String, int)}
* and that returns the passed {@link AdminProtocol} instance when
* {@link HConnection#getAdmin(String, int)} is called, returns the passed
* {@link ClientProtocol} instance when {@link HConnection#getClient(String, int)}
* is called (Be sure call
* {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration, boolean)}
* when done with this mocked Connection.

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionKey;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.mockito.Mockito;
/**
@ -86,8 +85,9 @@ 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 HRegionInterface} instance when
* {@link HConnection#getHRegionConnection(String, int)}
* and that returns the passed {@link AdminProtocol} instance when
* {@link HConnection#getAdmin(String, int)} is called, returns the passed
* {@link ClientProtocol} instance when {@link HConnection#getClient(String, int)}
* is called (Be sure call
* {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration, boolean)}
* when done with this mocked Connection.

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests;
@ -1224,7 +1225,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
info = regionInfo;
@ -1233,7 +1234,7 @@ public class TestAdmin {
}
}
Thread.sleep(1000);
onlineRegions = rs.getOnlineRegions();
onlineRegions = ProtobufUtil.getOnlineRegions(rs);
assertFalse("The region should not be present in online regions list.",
onlineRegions.contains(info));
}
@ -1245,7 +1246,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion1")) {
@ -1259,7 +1260,7 @@ public class TestAdmin {
}
}
}
onlineRegions = rs.getOnlineRegions();
onlineRegions = ProtobufUtil.getOnlineRegions(rs);
assertTrue("The region should be present in online regions list.",
onlineRegions.contains(info));
}
@ -1271,7 +1272,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
@ -1283,11 +1284,11 @@ public class TestAdmin {
}
}
boolean isInList = rs.getOnlineRegions().contains(info);
boolean isInList = ProtobufUtil.getOnlineRegions(rs).contains(info);
long timeout = System.currentTimeMillis() + 2000;
while ((System.currentTimeMillis() < timeout) && (isInList)) {
Thread.sleep(100);
isInList = rs.getOnlineRegions().contains(info);
isInList = ProtobufUtil.getOnlineRegions(rs).contains(info);
}
assertFalse("The region should not be present in online regions list.",
@ -1302,7 +1303,7 @@ public class TestAdmin {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
try {
List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString()
@ -1326,7 +1327,7 @@ public class TestAdmin {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
try {
List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString()
@ -1349,7 +1350,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = rs.getOnlineRegions();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion4")) {
@ -1363,7 +1364,7 @@ public class TestAdmin {
}
}
}
onlineRegions = rs.getOnlineRegions();
onlineRegions = ProtobufUtil.getOnlineRegions(rs);
assertTrue("The region should be present in online regions list.",
onlineRegions.contains(info));
}

View File

@ -28,6 +28,7 @@ import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.AfterClass;
@ -260,7 +261,7 @@ public class TestMultiParallel {
Assert.assertEquals("Server count=" + count + ", abort=" + doAbort,
(doAbort ? 1 : 2), count);
for (JVMClusterUtil.RegionServerThread t: liveRSs) {
int regions = t.getRegionServer().getOnlineRegions().size();
int regions = ProtobufUtil.getOnlineRegions(t.getRegionServer()).size();
Assert.assertTrue("Count of regions=" + regions, regions > 10);
}
table.close();

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
@ -684,7 +685,7 @@ public class TestMasterObserver {
// move half the open regions from RS 0 to RS 1
HRegionServer rs = cluster.getRegionServer(0);
byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName().toString());
List<HRegionInfo> openRegions = rs.getOnlineRegions();
List<HRegionInfo> openRegions = ProtobufUtil.getOnlineRegions(rs);
int moveCnt = openRegions.size()/2;
for (int i=0; i<moveCnt; i++) {
HRegionInfo info = openRegions.get(i);

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
@ -451,7 +452,7 @@ public class TestRegionObserverInterface {
Object value[]) throws IOException {
try {
for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
for (HRegionInfo r : t.getRegionServer().getOnlineRegions()) {
for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer())) {
if (!Arrays.equals(r.getTableName(), tableName)) {
continue;
}

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
@ -153,10 +154,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
HRegionServer hrs = util.getRSForFirstRegionInTable(Bytes
.toBytes(table));
for (HRegionInfo hri : hrs.getOnlineRegions()) {
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
if (Bytes.equals(hri.getTableName(), Bytes.toBytes(table))) {
// splitRegion doesn't work if startkey/endkey are null
hrs.splitRegion(hri, rowkey(ROWCOUNT / 2)); // hard code split
ProtobufUtil.split(hrs, hri, rowkey(ROWCOUNT / 2)); // hard code split
}
}
@ -164,7 +165,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
int regions;
do {
regions = 0;
for (HRegionInfo hri : hrs.getOnlineRegions()) {
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
if (Bytes.equals(hri.getTableName(), Bytes.toBytes(table))) {
regions++;
}

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
@ -129,7 +130,7 @@ public class TestDistributedLogSplitting {
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
hrs = rsts.get(i).getRegionServer();
regions = hrs.getOnlineRegions();
regions = ProtobufUtil.getOnlineRegions(hrs);
if (regions.size() != 0) break;
}
final Path logDir = new Path(rootdir, HLog.getHLogDirectoryName(hrs
@ -189,7 +190,7 @@ public class TestDistributedLogSplitting {
installTable(new ZooKeeperWatcher(conf, "table-creation", null),
"table", "family", 40);
makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs), "table",
NUM_LOG_LINES, 100);
new Thread() {
@ -378,7 +379,7 @@ public class TestDistributedLogSplitting {
for (RegionServerThread rst : rsts) {
HRegionServer hrs = rst.getRegionServer();
List<HRegionInfo> hris = hrs.getOnlineRegions();
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo hri : hris) {
if (hri.isMetaTable()) {
continue;
@ -459,7 +460,7 @@ public class TestDistributedLogSplitting {
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
online.add(region.getRegionNameAsString());
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
@ -311,7 +312,7 @@ public class TestMasterFailover {
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
ZKAssign.createNodeOffline(zkw, region, serverName);
hrs.openRegion(region);
ProtobufUtil.openRegion(hrs, region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -325,7 +326,7 @@ public class TestMasterFailover {
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
ZKAssign.createNodeOffline(zkw, region, serverName);
hrs.openRegion(region);
ProtobufUtil.openRegion(hrs, region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -361,7 +362,7 @@ public class TestMasterFailover {
Set<HRegionInfo> onlineRegions = new TreeSet<HRegionInfo>();
for (JVMClusterUtil.RegionServerThread rst :
cluster.getRegionServerThreads()) {
onlineRegions.addAll(rst.getRegionServer().getOnlineRegions());
onlineRegions.addAll(ProtobufUtil.getOnlineRegions(rst.getRegionServer()));
}
// Now, everything that should be online should be online
@ -671,7 +672,7 @@ public class TestMasterFailover {
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
hrsDead.openRegion(region);
ProtobufUtil.openRegion(hrsDead, region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -687,7 +688,7 @@ public class TestMasterFailover {
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
hrsDead.openRegion(region);
ProtobufUtil.openRegion(hrsDead, region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -707,7 +708,7 @@ public class TestMasterFailover {
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
hrsDead.openRegion(region);
ProtobufUtil.openRegion(hrsDead, region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -725,7 +726,7 @@ public class TestMasterFailover {
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
hrsDead.openRegion(region);
ProtobufUtil.openRegion(hrsDead, region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -819,7 +820,7 @@ public class TestMasterFailover {
for (JVMClusterUtil.RegionServerThread rst :
cluster.getRegionServerThreads()) {
try {
onlineRegions.addAll(rst.getRegionServer().getOnlineRegions());
onlineRegions.addAll(ProtobufUtil.getOnlineRegions(rst.getRegionServer()));
} catch (org.apache.hadoop.hbase.regionserver.RegionServerStoppedException e) {
LOG.info("Got RegionServerStoppedException", e);
}

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@ -132,7 +133,7 @@ public class TestMasterRestartAfterDisablingTable {
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
online.add(region.getRegionNameAsString());
}
}

View File

@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@ -340,7 +341,7 @@ public class TestRollingRestart {
private RegionServerThread getServerHosting(MiniHBaseCluster cluster,
HRegionInfo region) throws IOException {
for (RegionServerThread rst : cluster.getRegionServerThreads()) {
if (rst.getRegionServer().getOnlineRegions().contains(region)) {
if (ProtobufUtil.getOnlineRegions(rst.getRegionServer()).contains(region)) {
return rst;
}
}
@ -386,7 +387,7 @@ public class TestRollingRestart {
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
online.add(region.getRegionNameAsString());
}
}
@ -398,7 +399,7 @@ public class TestRollingRestart {
NavigableSet<String> online = new TreeSet<String>();
NavigableSet<String> doubled = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
if(!online.add(region.getRegionNameAsString())) {
doubled.add(region.getRegionNameAsString());
}

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.handler.TotesHRegionInfo;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@ -107,7 +108,7 @@ public class TestZKBasedOpenCloseRegion {
int rsIdx = 0;
HRegionServer regionServer =
TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx);
HRegionInfo hri = getNonMetaRegion(regionServer.getOnlineRegions());
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(regionServer));
LOG.debug("Asking RS to close region " + hri.getRegionNameAsString());
AtomicBoolean closeEventProcessed = new AtomicBoolean(false);
@ -242,7 +243,7 @@ public class TestZKBasedOpenCloseRegion {
cluster.getLiveRegionServerThreads().get(0).getRegionServer();
HRegionServer hr1 =
cluster.getLiveRegionServerThreads().get(1).getRegionServer();
HRegionInfo hri = getNonMetaRegion(hr0.getOnlineRegions());
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr0));
// fake that hr1 is processing the region
hr1.getRegionsInTransitionInRS().putIfAbsent(hri.getEncodedNameAsBytes(), true);
@ -266,7 +267,7 @@ public class TestZKBasedOpenCloseRegion {
reopenEventProcessed.set(false);
// now try moving a region when there is no region in transition.
hri = getNonMetaRegion(hr1.getOnlineRegions());
hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr1));
openListener =
new ReopenEventListener(hri.getRegionNameAsString(),
@ -295,7 +296,7 @@ public class TestZKBasedOpenCloseRegion {
int rsIdx = 0;
HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx);
HRegionInfo hri = getNonMetaRegion(regionServer.getOnlineRegions());
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(regionServer));
LOG.debug("Asking RS to close region " + hri.getRegionNameAsString());
AtomicBoolean closeEventProcessed = new AtomicBoolean(false);
@ -327,7 +328,7 @@ public class TestZKBasedOpenCloseRegion {
Whitebox.setInternalState(regionServer, "tableDescriptors", htd);
Mockito.doThrow(new IOException()).when(htd).get((byte[]) Mockito.any());
try {
regionServer.openRegion(REGIONINFO);
ProtobufUtil.openRegion(regionServer, REGIONINFO);
fail("It should throw IOException ");
} catch (IOException e) {
}

View File

@ -30,6 +30,9 @@ import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.junit.AfterClass;
@ -37,6 +40,8 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.ServiceException;
@Category(LargeTests.class)
public class TestEndToEndSplitTransaction {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@ -119,8 +124,14 @@ public class TestEndToEndSplitTransaction {
byte[] regionName = con.relocateRegion(tableName, row).getRegionInfo()
.getRegionName();
// get and scan should now succeed without exception
server.get(regionName, new Get(row));
server.openScanner(regionName, new Scan(row));
ProtobufUtil.get(server, regionName, new Get(row));
ScanRequest scanRequest = RequestConverter.buildScanRequest(
regionName, new Scan(row), 1, true);
try {
server.scan(null, scanRequest);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
} catch (IOException x) {
return false;
}

View File

@ -25,11 +25,13 @@ import java.util.List;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
import org.apache.hadoop.hbase.tmpl.regionserver.RSStatusTmpl;
import org.apache.hadoop.hbase.util.Bytes;
@ -41,6 +43,8 @@ import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.common.collect.Lists;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Tests for the region server status page and its template.
@ -52,24 +56,22 @@ public class TestRSStatusServlet {
static final int FAKE_IPC_PORT = 1585;
static final int FAKE_WEB_PORT = 1586;
@SuppressWarnings("deprecation")
private final HServerAddress fakeAddress =
new HServerAddress("localhost", FAKE_IPC_PORT);
@SuppressWarnings("deprecation")
private final HServerInfo fakeInfo =
new HServerInfo(fakeAddress, FAKE_WEB_PORT);
private final ServerName fakeServerName =
new ServerName("localhost", FAKE_IPC_PORT, 11111);
private final GetServerInfoResponse fakeResponse =
ResponseConverter.buildGetServerInfoResponse(fakeServerName, FAKE_WEB_PORT);
private final RegionServerMetrics metrics =
new RegionServerMetrics();
private final ServerName fakeMasterAddress =
new ServerName("localhost", 60010, 1212121212);
@SuppressWarnings("deprecation")
@Before
public void setupBasicMocks() throws IOException {
public void setupBasicMocks() throws IOException, ServiceException {
rs = Mockito.mock(HRegionServer.class);
Mockito.doReturn(HBaseConfiguration.create())
.when(rs).getConfiguration();
Mockito.doReturn(fakeInfo).when(rs).getHServerInfo();
Mockito.doReturn(fakeResponse).when(rs).getServerInfo(
(RpcController)Mockito.any(), (GetServerInfoRequest)Mockito.any());
Mockito.doReturn(metrics).when(rs).getMetrics();
// Fake ZKW
@ -84,18 +86,20 @@ public class TestRSStatusServlet {
}
@Test
public void testBasic() throws IOException {
public void testBasic() throws IOException, ServiceException {
new RSStatusTmpl().render(new StringWriter(), rs);
}
@Test
public void testWithRegions() throws IOException {
public void testWithRegions() throws IOException, ServiceException {
HTableDescriptor htd = new HTableDescriptor("mytable");
List<HRegionInfo> regions = Lists.newArrayList(
new HRegionInfo(htd.getName(), Bytes.toBytes("a"), Bytes.toBytes("d")),
new HRegionInfo(htd.getName(), Bytes.toBytes("d"), Bytes.toBytes("z"))
);
Mockito.doReturn(regions).when(rs).getOnlineRegions();
Mockito.doReturn(ResponseConverter.buildGetOnlineRegionResponse(
regions)).when(rs).getOnlineRegion((RpcController)Mockito.any(),
(GetOnlineRegionRequest)Mockito.any());
new RSStatusTmpl().render(new StringWriter(), rs);
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
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.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.
@ -234,7 +235,7 @@ public class TestRegionServerMetrics {
final HRegionServer rs =
TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
assertEquals(NUM_REGIONS + META_AND_ROOT, rs.getOnlineRegions().size());
assertEquals(NUM_REGIONS + META_AND_ROOT, ProtobufUtil.getOnlineRegions(rs).size());
rs.doMetrics();
for (HRegion r : TEST_UTIL.getMiniHBaseCluster().getRegions(

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
@ -122,7 +123,7 @@ public class TestSplitTransactionOnCluster {
// Get region pre-split.
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
int regionCount = server.getOnlineRegions().size();
int regionCount = ProtobufUtil.getOnlineRegions(server).size();
// Now, before we split, set special flag in master, a flag that has
// it FAIL the processing of split.
SplitRegionHandler.TEST_SKIP = true;
@ -192,7 +193,7 @@ public class TestSplitTransactionOnCluster {
// Get region pre-split.
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
int regionCount = server.getOnlineRegions().size();
int regionCount = ProtobufUtil.getOnlineRegions(server).size();
// Insert into zk a blocking znode, a znode of same name as region
// so it gets in way of our splitting.
ZKAssign.createNodeClosing(TESTING_UTIL.getZooKeeperWatcher(),
@ -205,7 +206,7 @@ public class TestSplitTransactionOnCluster {
// Wait around a while and assert count of regions remains constant.
for (int i = 0; i < 10; i++) {
Thread.sleep(100);
assertEquals(regionCount, server.getOnlineRegions().size());
assertEquals(regionCount, ProtobufUtil.getOnlineRegions(server).size());
}
// Now clear the zknode
ZKAssign.deleteClosingNode(TESTING_UTIL.getZooKeeperWatcher(), hri);
@ -251,7 +252,7 @@ public class TestSplitTransactionOnCluster {
// Get region pre-split.
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
int regionCount = server.getOnlineRegions().size();
int regionCount = ProtobufUtil.getOnlineRegions(server).size();
// Now split.
split(hri, server, regionCount);
// Get daughters
@ -308,14 +309,14 @@ public class TestSplitTransactionOnCluster {
// Get region pre-split.
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
int regionCount = server.getOnlineRegions().size();
int regionCount = ProtobufUtil.getOnlineRegions(server).size();
// Now split.
split(hri, server, regionCount);
// Get daughters
List<HRegion> daughters = cluster.getRegions(tableName);
assertTrue(daughters.size() >= 2);
// Now split one of the daughters.
regionCount = server.getOnlineRegions().size();
regionCount = ProtobufUtil.getOnlineRegions(server).size();
HRegionInfo daughter = daughters.get(0).getRegionInfo();
// Compact first to ensure we have cleaned up references -- else the split
// will fail.
@ -358,7 +359,7 @@ public class TestSplitTransactionOnCluster {
final int regionCount)
throws IOException, InterruptedException {
this.admin.split(hri.getRegionNameAsString());
while (server.getOnlineRegions().size() <= regionCount) {
while (ProtobufUtil.getOnlineRegions(server).size() <= regionCount) {
LOG.debug("Waiting on region to split");
Thread.sleep(100);
}
@ -444,7 +445,7 @@ public class TestSplitTransactionOnCluster {
private void printOutRegions(final HRegionServer hrs, final String prefix)
throws IOException {
List<HRegionInfo> regions = hrs.getOnlineRegions();
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region: regions) {
LOG.info(prefix + region.getRegionNameAsString());
}

View File

@ -916,7 +916,7 @@ public class TestHBaseFsck {
TEST_UTIL.getHBaseAdmin().disableTable(table);
HRegionInfo region = disabledRegions.remove(0);
ZKAssign.createNodeOffline(zkw, region, serverName);
hrs.openRegion(region);
ProtobufUtil.openRegion(hrs, region);
int iTimes = 0;
while (true) {