Merge remote-tracking branch 'origin/master'
This commit is contained in:
commit
8e13fea268
|
@ -0,0 +1,46 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.HBaseException;
|
||||
|
||||
/**
|
||||
* Thrown by operations requiring coordination state access or manipulation
|
||||
* when internal error within coordination engine (or other internal implementation) occurs.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@SuppressWarnings("serial")
|
||||
public class CoordinatedStateException extends HBaseException {
|
||||
public CoordinatedStateException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public CoordinatedStateException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public CoordinatedStateException(final String message, final Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public CoordinatedStateException(final Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
|
@ -22,37 +22,45 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
/**
|
||||
* Implementations of this interface will keep and return to clients
|
||||
* implementations of classes providing API to execute
|
||||
* coordinated operations. This interface is client-sise, so it does NOT
|
||||
* include method to retrieve the particular consensus providers.
|
||||
* coordinated operations. This interface is client-side, so it does NOT
|
||||
* include methods to retrieve the particular interface implementations.
|
||||
*
|
||||
* For each coarse-grained area of operations there will be a separate
|
||||
* interface with implementation, providing API for relevant operations
|
||||
* requiring coordination.
|
||||
*
|
||||
* Property hbase.consensus.provider.class in hbase-site.xml controls
|
||||
* Property hbase.coordinated.state.manager.class in hbase-site.xml controls
|
||||
* which provider to use.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ConsensusProvider {
|
||||
public interface CoordinatedStateManager {
|
||||
|
||||
/**
|
||||
* Initialize consensus service.
|
||||
* Initialize coordinated state management service.
|
||||
* @param server server instance to run within.
|
||||
*/
|
||||
void initialize(Server server);
|
||||
|
||||
/**
|
||||
* Starts consensus service.
|
||||
* Starts service.
|
||||
*/
|
||||
void start();
|
||||
|
||||
/**
|
||||
* Stop consensus provider.
|
||||
* Stops service.
|
||||
*/
|
||||
void stop();
|
||||
|
||||
/**
|
||||
* @return instance of Server consensus runs within
|
||||
* @return instance of Server coordinated state manager runs within
|
||||
*/
|
||||
Server getServer();
|
||||
|
||||
/**
|
||||
* Returns implementation of TableStateManager.
|
||||
* @throws InterruptedException if operation is interrupted
|
||||
* @throws CoordinatedStateException if error happens in underlying coordination mechanism
|
||||
*/
|
||||
TableStateManager getTableStateManager() throws InterruptedException,
|
||||
CoordinatedStateException;
|
||||
}
|
|
@ -50,7 +50,7 @@ public interface Server extends Abortable, Stoppable {
|
|||
ServerName getServerName();
|
||||
|
||||
/**
|
||||
* Get ConsensusProvider instance for this server.
|
||||
* Get CoordinatedStateManager instance for this server.
|
||||
*/
|
||||
ConsensusProvider getConsensusProvider();
|
||||
CoordinatedStateManager getCoordinatedStateManager();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,115 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Helper class for table state management for operations running inside
|
||||
* RegionServer or HMaster.
|
||||
* Depending on implementation, fetches information from HBase system table,
|
||||
* local data store, ZooKeeper ensemble or somewhere else.
|
||||
* Code running on client side (with no coordinated state context) shall instead use
|
||||
* {@link org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader}
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface TableStateManager {
|
||||
|
||||
/**
|
||||
* Sets the table into desired state. Fails silently if the table is already in this state.
|
||||
* @param tableName table to process
|
||||
* @param state new state of this table
|
||||
* @throws CoordinatedStateException if error happened when trying to set table state
|
||||
*/
|
||||
void setTableState(TableName tableName, ZooKeeperProtos.Table.State state)
|
||||
throws CoordinatedStateException;
|
||||
|
||||
/**
|
||||
* Sets the specified table into the newState, but only if the table is already in
|
||||
* one of the possibleCurrentStates (otherwise no operation is performed).
|
||||
* @param tableName table to process
|
||||
* @param newState new state for the table
|
||||
* @param states table should be in one of these states for the operation
|
||||
* to be performed
|
||||
* @throws CoordinatedStateException if error happened while performing operation
|
||||
* @return true if operation succeeded, false otherwise
|
||||
*/
|
||||
boolean setTableStateIfInStates(TableName tableName, ZooKeeperProtos.Table.State newState,
|
||||
ZooKeeperProtos.Table.State... states)
|
||||
throws CoordinatedStateException;
|
||||
|
||||
/**
|
||||
* Sets the specified table into the newState, but only if the table is NOT in
|
||||
* one of the possibleCurrentStates (otherwise no operation is performed).
|
||||
* @param tableName table to process
|
||||
* @param newState new state for the table
|
||||
* @param states table should NOT be in one of these states for the operation
|
||||
* to be performed
|
||||
* @throws CoordinatedStateException if error happened while performing operation
|
||||
* @return true if operation succeeded, false otherwise
|
||||
*/
|
||||
boolean setTableStateIfNotInStates(TableName tableName, ZooKeeperProtos.Table.State newState,
|
||||
ZooKeeperProtos.Table.State... states)
|
||||
throws CoordinatedStateException;
|
||||
|
||||
/**
|
||||
* @return true if the table is in any one of the listed states, false otherwise.
|
||||
*/
|
||||
boolean isTableState(TableName tableName, ZooKeeperProtos.Table.State... states);
|
||||
|
||||
/**
|
||||
* Mark table as deleted. Fails silently if the table is not currently marked as disabled.
|
||||
* @param tableName table to be deleted
|
||||
* @throws CoordinatedStateException if error happened while performing operation
|
||||
*/
|
||||
void setDeletedTable(TableName tableName) throws CoordinatedStateException;
|
||||
|
||||
/**
|
||||
* Checks if table is present.
|
||||
*
|
||||
* @param tableName table we're checking
|
||||
* @return true if the table is present, false otherwise
|
||||
*/
|
||||
boolean isTablePresent(TableName tableName);
|
||||
|
||||
/**
|
||||
* @return set of tables which are in any one of the listed states, empty Set if none
|
||||
*/
|
||||
Set<TableName> getTablesInStates(ZooKeeperProtos.Table.State... states)
|
||||
throws InterruptedIOException, CoordinatedStateException;
|
||||
|
||||
/**
|
||||
* If the table is found in the given state the in-memory state is removed. This
|
||||
* helps in cases where CreateTable is to be retried by the client in case of
|
||||
* failures. If deletePermanentState is true - the flag kept permanently is
|
||||
* also reset.
|
||||
*
|
||||
* @param tableName table we're working on
|
||||
* @param states if table isn't in any one of these states, operation aborts
|
||||
* @param deletePermanentState if true, reset the permanent flag
|
||||
* @throws CoordinatedStateException if error happened in underlying coordination engine
|
||||
*/
|
||||
void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states,
|
||||
boolean deletePermanentState)
|
||||
throws CoordinatedStateException;
|
||||
}
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.htrace.Trace;
|
||||
|
@ -120,6 +121,7 @@ class AsyncProcess {
|
|||
|
||||
protected final ClusterConnection hConnection;
|
||||
protected final RpcRetryingCallerFactory rpcCallerFactory;
|
||||
protected final RpcControllerFactory rpcFactory;
|
||||
protected final BatchErrors globalErrors;
|
||||
protected final ExecutorService pool;
|
||||
|
||||
|
@ -188,7 +190,7 @@ class AsyncProcess {
|
|||
}
|
||||
|
||||
public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool,
|
||||
RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors) {
|
||||
RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors, RpcControllerFactory rpcFactory) {
|
||||
if (hc == null) {
|
||||
throw new IllegalArgumentException("HConnection cannot be null.");
|
||||
}
|
||||
|
@ -242,8 +244,8 @@ class AsyncProcess {
|
|||
serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
|
||||
}
|
||||
|
||||
|
||||
this.rpcCallerFactory = rpcCaller;
|
||||
this.rpcFactory = rpcFactory;
|
||||
}
|
||||
|
||||
private ExecutorService getPool(ExecutorService pool) {
|
||||
|
@ -950,7 +952,7 @@ class AsyncProcess {
|
|||
@VisibleForTesting
|
||||
protected MultiServerCallable<Row> createCallable(final ServerName server,
|
||||
TableName tableName, final MultiAction<Row> multi) {
|
||||
return new MultiServerCallable<Row>(hConnection, tableName, server, multi);
|
||||
return new MultiServerCallable<Row>(hConnection, tableName, server, this.rpcFactory, multi);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||
|
@ -66,6 +67,7 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
protected final int scannerTimeout;
|
||||
protected boolean scanMetricsPublished = false;
|
||||
protected RpcRetryingCaller<Result []> caller;
|
||||
protected RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
/**
|
||||
* Create a new ClientScanner for the specified table. An HConnection will be
|
||||
|
@ -93,27 +95,36 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
|
||||
|
||||
/**
|
||||
* Create a new ClientScanner for the specified table
|
||||
* Note that the passed {@link Scan}'s start row maybe changed changed.
|
||||
*
|
||||
* @param conf The {@link Configuration} to use.
|
||||
* @param scan {@link Scan} to use in this scanner
|
||||
* @param tableName The table that we wish to scan
|
||||
* @param connection Connection identifying the cluster
|
||||
* @throws IOException
|
||||
*/
|
||||
* @deprecated use
|
||||
* {@link #ClientScanner(Configuration, Scan, TableName, HConnection, RpcRetryingCallerFactory, RpcControllerFactory)}
|
||||
*/
|
||||
@Deprecated
|
||||
public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
HConnection connection) throws IOException {
|
||||
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
|
||||
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf),
|
||||
RpcControllerFactory.instantiate(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #ClientScanner(Configuration, Scan, TableName, HConnection)}
|
||||
* @deprecated Use
|
||||
* {@link #ClientScanner(Configuration, Scan, TableName, HConnection, RpcRetryingCallerFactory, RpcControllerFactory)}
|
||||
*/
|
||||
@Deprecated
|
||||
public ClientScanner(final Configuration conf, final Scan scan, final byte [] tableName,
|
||||
HConnection connection) throws IOException {
|
||||
this(conf, scan, TableName.valueOf(tableName), connection, new RpcRetryingCallerFactory(conf));
|
||||
this(conf, scan, TableName.valueOf(tableName), connection, new RpcRetryingCallerFactory(conf),
|
||||
RpcControllerFactory.instantiate(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use
|
||||
* {@link #ClientScanner(Configuration, Scan, TableName, HConnection, RpcRetryingCallerFactory, RpcControllerFactory)
|
||||
* instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
HConnection connection, RpcRetryingCallerFactory rpcFactory) throws IOException {
|
||||
this(conf, scan, tableName, connection, rpcFactory, RpcControllerFactory.instantiate(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -126,7 +137,8 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
* @throws IOException
|
||||
*/
|
||||
public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
HConnection connection, RpcRetryingCallerFactory rpcFactory) throws IOException {
|
||||
HConnection connection, RpcRetryingCallerFactory rpcFactory,
|
||||
RpcControllerFactory controllerFactory) throws IOException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Scan table=" + tableName
|
||||
+ ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
|
||||
|
@ -159,7 +171,8 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
|
||||
}
|
||||
|
||||
this.caller = rpcFactory.<Result[]> newCaller();
|
||||
this.caller = rpcFactory.<Result[]> newCaller();
|
||||
this.rpcControllerFactory = controllerFactory;
|
||||
|
||||
initializeScannerInConstruction();
|
||||
}
|
||||
|
@ -277,8 +290,9 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
protected ScannerCallable getScannerCallable(byte [] localStartKey,
|
||||
int nbRows) {
|
||||
scan.setStartRow(localStartKey);
|
||||
ScannerCallable s = new ScannerCallable(getConnection(),
|
||||
getTable(), scan, this.scanMetrics);
|
||||
ScannerCallable s =
|
||||
new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
|
||||
this.rpcControllerFactory);
|
||||
s.setCaching(nbRows);
|
||||
return s;
|
||||
}
|
||||
|
|
|
@ -109,7 +109,7 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
}
|
||||
|
||||
smallScanCallable = ClientSmallScanner.getSmallScanCallable(
|
||||
scan, getConnection(), getTable(), localStartKey, cacheNum);
|
||||
scan, getConnection(), getTable(), localStartKey, cacheNum, this.rpcControllerFactory);
|
||||
|
||||
if (this.scanMetrics != null && skipRowOfFirstResult == null) {
|
||||
this.scanMetrics.countOfRegions.incrementAndGet();
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
|
@ -85,9 +86,20 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
*/
|
||||
public ClientSmallScanner(final Configuration conf, final Scan scan,
|
||||
final TableName tableName, HConnection connection) throws IOException {
|
||||
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
|
||||
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf),
|
||||
new RpcControllerFactory(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated use
|
||||
* {@link #ClientSmallScanner(Configuration, Scan, TableName, HConnection, RpcRetryingCallerFactory, RpcControllerFactory)
|
||||
* instead
|
||||
*/
|
||||
public ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
HConnection connection, RpcRetryingCallerFactory rpcFactory) throws IOException {
|
||||
this(conf, scan, tableName, connection, rpcFactory, RpcControllerFactory.instantiate(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new ShortClientScanner for the specified table Note that the
|
||||
* passed {@link Scan}'s start row maybe changed changed.
|
||||
|
@ -99,10 +111,10 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
* @param rpcFactory
|
||||
* @throws IOException
|
||||
*/
|
||||
public ClientSmallScanner(final Configuration conf, final Scan scan,
|
||||
final TableName tableName, HConnection connection,
|
||||
RpcRetryingCallerFactory rpcFactory) throws IOException {
|
||||
super(conf, scan, tableName, connection, rpcFactory);
|
||||
public ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName,
|
||||
HConnection connection, RpcRetryingCallerFactory rpcFactory,
|
||||
RpcControllerFactory controllerFactory) throws IOException {
|
||||
super(conf, scan, tableName, connection, rpcFactory, controllerFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -154,7 +166,7 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
+ Bytes.toStringBinary(localStartKey) + "'");
|
||||
}
|
||||
smallScanCallable = getSmallScanCallable(
|
||||
scan, getConnection(), getTable(), localStartKey, cacheNum);
|
||||
scan, getConnection(), getTable(), localStartKey, cacheNum, rpcControllerFactory);
|
||||
if (this.scanMetrics != null && skipRowOfFirstResult == null) {
|
||||
this.scanMetrics.countOfRegions.incrementAndGet();
|
||||
}
|
||||
|
@ -163,14 +175,14 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
|
||||
static RegionServerCallable<Result[]> getSmallScanCallable(
|
||||
final Scan sc, HConnection connection, TableName table,
|
||||
byte[] localStartKey, final int cacheNum) {
|
||||
byte[] localStartKey, final int cacheNum, final RpcControllerFactory rpcControllerFactory) {
|
||||
sc.setStartRow(localStartKey);
|
||||
RegionServerCallable<Result[]> callable = new RegionServerCallable<Result[]>(
|
||||
connection, table, sc.getStartRow()) {
|
||||
public Result[] call(int callTimeout) throws IOException {
|
||||
ScanRequest request = RequestConverter.buildScanRequest(getLocation()
|
||||
.getRegionInfo().getRegionName(), sc, cacheNum, true);
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
|
|
@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
|||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
|
@ -2200,8 +2201,8 @@ class ConnectionManager {
|
|||
// For tests to override.
|
||||
protected AsyncProcess createAsyncProcess(Configuration conf) {
|
||||
// No default pool available.
|
||||
return new AsyncProcess(
|
||||
this, conf, this.batchPool, RpcRetryingCallerFactory.instantiate(conf), false);
|
||||
return new AsyncProcess(this, conf, this.batchPool,
|
||||
RpcRetryingCallerFactory.instantiate(conf), false, RpcControllerFactory.instantiate(conf));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
|||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
|
@ -146,6 +147,7 @@ public class HTable implements HTableInterface {
|
|||
/** The Async process for batch */
|
||||
protected AsyncProcess multiAp;
|
||||
private RpcRetryingCallerFactory rpcCallerFactory;
|
||||
private RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
/**
|
||||
* Creates an object to access a HBase table.
|
||||
|
@ -362,8 +364,9 @@ public class HTable implements HTableInterface {
|
|||
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
|
||||
|
||||
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(configuration);
|
||||
this.rpcControllerFactory = RpcControllerFactory.instantiate(configuration);
|
||||
// puts need to track errors globally due to how the APIs currently work.
|
||||
ap = new AsyncProcess(connection, configuration, pool, rpcCallerFactory, true);
|
||||
ap = new AsyncProcess(connection, configuration, pool, rpcCallerFactory, true, rpcControllerFactory);
|
||||
multiAp = this.connection.getAsyncProcess();
|
||||
|
||||
this.maxKeyValueSize = this.configuration.getInt(
|
||||
|
@ -725,7 +728,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
|
||||
tableName, row) {
|
||||
public Result call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
ClientProtos.GetRequest request = RequestConverter.buildGetRowOrBeforeRequest(
|
||||
|
@ -763,10 +766,10 @@ public class HTable implements HTableInterface {
|
|||
|
||||
if (scan.isSmall()) {
|
||||
return new ClientSmallScanner(getConfiguration(), scan, getName(),
|
||||
this.connection, this.rpcCallerFactory);
|
||||
this.connection, this.rpcCallerFactory, this.rpcControllerFactory);
|
||||
} else {
|
||||
return new ClientScanner(getConfiguration(), scan,
|
||||
getName(), this.connection);
|
||||
return new ClientScanner(getConfiguration(), scan, getName(), this.connection,
|
||||
this.rpcCallerFactory, this.rpcControllerFactory);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -801,7 +804,7 @@ public class HTable implements HTableInterface {
|
|||
public Result call(int callTimeout) throws IOException {
|
||||
ClientProtos.GetRequest request =
|
||||
RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), get);
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -902,7 +905,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
|
||||
tableName, delete.getRow()) {
|
||||
public Boolean call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
|
@ -1042,7 +1045,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Void> callable =
|
||||
new RegionServerCallable<Void>(connection, getName(), rm.getRow()) {
|
||||
public Void call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1076,7 +1079,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Result> callable =
|
||||
new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
|
||||
public Result call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1107,7 +1110,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
|
||||
getName(), increment.getRow()) {
|
||||
public Result call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1118,8 +1121,8 @@ public class HTable implements HTableInterface {
|
|||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
|
||||
}
|
||||
|
||||
|
@ -1170,7 +1173,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Long> callable =
|
||||
new RegionServerCallable<Long>(connection, getName(), row) {
|
||||
public Long call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1200,7 +1203,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Boolean> callable =
|
||||
new RegionServerCallable<Boolean>(connection, getName(), row) {
|
||||
public Boolean call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1257,7 +1260,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Boolean> callable =
|
||||
new RegionServerCallable<Boolean>(connection, getName(), row) {
|
||||
public Boolean call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1285,7 +1288,7 @@ public class HTable implements HTableInterface {
|
|||
RegionServerCallable<Boolean> callable =
|
||||
new RegionServerCallable<Boolean>(connection, getName(), row) {
|
||||
public Boolean call(int callTimeout) throws IOException {
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(tableName);
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
@ -1761,8 +1764,10 @@ public class HTable implements HTableInterface {
|
|||
final List<String> callbackErrorServers = new ArrayList<String>();
|
||||
Object[] results = new Object[execs.size()];
|
||||
|
||||
AsyncProcess asyncProcess = new AsyncProcess(connection, configuration, pool,
|
||||
RpcRetryingCallerFactory.instantiate(configuration), true);
|
||||
AsyncProcess asyncProcess =
|
||||
new AsyncProcess(connection, configuration, pool,
|
||||
RpcRetryingCallerFactory.instantiate(configuration), true,
|
||||
RpcControllerFactory.instantiate(configuration));
|
||||
AsyncRequestFuture future = asyncProcess.submitAll(tableName, execs,
|
||||
new Callback<ClientProtos.CoprocessorServiceResult>() {
|
||||
@Override
|
||||
|
|
|
@ -31,14 +31,15 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -51,10 +52,12 @@ import com.google.protobuf.ServiceException;
|
|||
class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
|
||||
private final MultiAction<R> multiAction;
|
||||
private final boolean cellBlock;
|
||||
private RpcControllerFactory rpcFactory;
|
||||
|
||||
MultiServerCallable(final HConnection connection, final TableName tableName,
|
||||
final ServerName location, final MultiAction<R> multi) {
|
||||
final ServerName location, RpcControllerFactory rpcFactory, final MultiAction<R> multi) {
|
||||
super(connection, tableName, null);
|
||||
this.rpcFactory = rpcFactory;
|
||||
this.multiAction = multi;
|
||||
// RegionServerCallable has HRegionLocation field, but this is a multi-region request.
|
||||
// Using region info from parent HRegionLocation would be a mistake for this class; so
|
||||
|
@ -115,7 +118,7 @@ class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
|
|||
|
||||
// Controller optionally carries cell data over the proxy/service boundary and also
|
||||
// optionally ferries cell response data back out again.
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
|
||||
PayloadCarryingRpcController controller = rpcFactory.newController(cells);
|
||||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(callTimeout);
|
||||
ClientProtos.MultiResponse responseProto;
|
||||
|
|
|
@ -125,8 +125,9 @@ public class ReversedClientScanner extends ClientScanner {
|
|||
protected ScannerCallable getScannerCallable(byte[] localStartKey,
|
||||
int nbRows, byte[] locateStartRow) {
|
||||
scan.setStartRow(localStartKey);
|
||||
ScannerCallable s = new ReversedScannerCallable(getConnection(),
|
||||
getTable(), scan, this.scanMetrics, locateStartRow);
|
||||
ScannerCallable s =
|
||||
new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
|
||||
locateStartRow, this.rpcControllerFactory);
|
||||
s.setCaching(nbRows);
|
||||
return s;
|
||||
}
|
||||
|
|
|
@ -29,8 +29,11 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* A reversed ScannerCallable which supports backward scanning.
|
||||
*/
|
||||
|
@ -45,17 +48,28 @@ public class ReversedScannerCallable extends ScannerCallable {
|
|||
protected final byte[] locateStartRow;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param connection
|
||||
* @param tableName
|
||||
* @param scan
|
||||
* @param scanMetrics
|
||||
* @param locateStartRow The start row for locating regions
|
||||
* @param rpcFactory to create an {@link RpcController} to talk to the regionserver
|
||||
*/
|
||||
public ReversedScannerCallable(HConnection connection, TableName tableName, Scan scan,
|
||||
ScanMetrics scanMetrics, byte[] locateStartRow, RpcControllerFactory rpcFactory) {
|
||||
super(connection, tableName, scan, scanMetrics, rpcFactory);
|
||||
this.locateStartRow = locateStartRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated use
|
||||
* {@link #ReversedScannerCallable(HConnection, TableName, Scan, ScanMetrics, byte[], RpcControllerFactory)}
|
||||
*/
|
||||
@Deprecated
|
||||
public ReversedScannerCallable(HConnection connection, TableName tableName,
|
||||
Scan scan, ScanMetrics scanMetrics, byte[] locateStartRow) {
|
||||
super(connection, tableName, scan, scanMetrics);
|
||||
this.locateStartRow = locateStartRow;
|
||||
this(connection, tableName, scan, scanMetrics, locateStartRow, RpcControllerFactory
|
||||
.instantiate(connection.getConfiguration()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
|
@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
|||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.DNS;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
|
@ -81,22 +83,25 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
// indicate if it is a remote server call
|
||||
protected boolean isRegionServerRemote = true;
|
||||
private long nextCallSeq = 0;
|
||||
private RpcControllerFactory rpcFactory;
|
||||
|
||||
/**
|
||||
* @param connection which connection
|
||||
* @param tableName table callable is on
|
||||
* @param scan the scan to execute
|
||||
* @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable
|
||||
* won't collect metrics
|
||||
* @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable won't collect
|
||||
* metrics
|
||||
* @param rpcControllerFactory factory to use when creating {@link RpcController}
|
||||
*/
|
||||
public ScannerCallable (HConnection connection, TableName tableName, Scan scan,
|
||||
ScanMetrics scanMetrics) {
|
||||
ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory) {
|
||||
super(connection, tableName, scan.getStartRow());
|
||||
this.scan = scan;
|
||||
this.scanMetrics = scanMetrics;
|
||||
Configuration conf = connection.getConfiguration();
|
||||
logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false);
|
||||
logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000);
|
||||
this.rpcFactory = rpcControllerFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -105,7 +110,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
@Deprecated
|
||||
public ScannerCallable (HConnection connection, final byte [] tableName, Scan scan,
|
||||
ScanMetrics scanMetrics) {
|
||||
this(connection, TableName.valueOf(tableName), scan, scanMetrics);
|
||||
this(connection, TableName.valueOf(tableName), scan, scanMetrics, RpcControllerFactory
|
||||
.instantiate(connection.getConfiguration()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -161,7 +167,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
incRPCcallsMetrics();
|
||||
request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq);
|
||||
ScanResponse response = null;
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
PayloadCarryingRpcController controller = rpcFactory.newController();
|
||||
controller.setPriority(getTableName());
|
||||
controller.setCallTimeout(callTimeout);
|
||||
try {
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -101,9 +101,9 @@ class ZooKeeperRegistry implements Registry {
|
|||
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
|
||||
try {
|
||||
if (enabled) {
|
||||
return ZKTableReadOnly.isEnabledTable(zkw, tableName);
|
||||
return ZKTableStateClientSideReader.isEnabledTable(zkw, tableName);
|
||||
}
|
||||
return ZKTableReadOnly.isDisabledTable(zkw, tableName);
|
||||
return ZKTableStateClientSideReader.isDisabledTable(zkw, tableName);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Enable/Disable failed", e);
|
||||
} catch (InterruptedException e) {
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
||||
/**
|
||||
* Simple delegating controller for use with the {@link RpcControllerFactory} to help override
|
||||
* standard behavior of a {@link PayloadCarryingRpcController}.
|
||||
*/
|
||||
public class DelegatingPayloadCarryingRpcController extends PayloadCarryingRpcController {
|
||||
private PayloadCarryingRpcController delegate;
|
||||
|
||||
public DelegatingPayloadCarryingRpcController(PayloadCarryingRpcController delegate) {
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CellScanner cellScanner() {
|
||||
return delegate.cellScanner();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setCellScanner(final CellScanner cellScanner) {
|
||||
delegate.setCellScanner(cellScanner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPriority(int priority) {
|
||||
delegate.setPriority(priority);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPriority(final TableName tn) {
|
||||
delegate.setPriority(tn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority() {
|
||||
return delegate.getPriority();
|
||||
}
|
||||
}
|
|
@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
|
||||
import org.apache.hadoop.hbase.security.AuthMethod;
|
||||
import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
|
||||
import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
|
||||
import org.apache.hadoop.hbase.security.SecurityInfo;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
|
@ -804,7 +805,9 @@ public class RpcClient {
|
|||
|
||||
private synchronized boolean setupSaslConnection(final InputStream in2,
|
||||
final OutputStream out2) throws IOException {
|
||||
saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed);
|
||||
saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
|
||||
conf.get("hbase.rpc.protection",
|
||||
QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
|
||||
return saslRpcClient.saslConnect(in2, out2);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,59 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
* Factory to create a {@link PayloadCarryingRpcController}
|
||||
*/
|
||||
public class RpcControllerFactory {
|
||||
|
||||
public static final String CUSTOM_CONTROLLER_CONF_KEY = "hbase.rpc.controllerfactory.class";
|
||||
protected final Configuration conf;
|
||||
|
||||
public RpcControllerFactory(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
public PayloadCarryingRpcController newController() {
|
||||
return new PayloadCarryingRpcController();
|
||||
}
|
||||
|
||||
public PayloadCarryingRpcController newController(final CellScanner cellScanner) {
|
||||
return new PayloadCarryingRpcController(cellScanner);
|
||||
}
|
||||
|
||||
public PayloadCarryingRpcController newController(final List<CellScannable> cellIterables) {
|
||||
return new PayloadCarryingRpcController(cellIterables);
|
||||
}
|
||||
|
||||
|
||||
public static RpcControllerFactory instantiate(Configuration configuration) {
|
||||
String rpcControllerFactoryClazz =
|
||||
configuration.get(CUSTOM_CONTROLLER_CONF_KEY,
|
||||
RpcControllerFactory.class.getName());
|
||||
return ReflectionUtils.instantiateWithCustomCtor(rpcControllerFactoryClazz,
|
||||
new Class[] { Configuration.class }, new Object[] { configuration });
|
||||
}
|
||||
}
|
|
@ -57,7 +57,6 @@ public class HBaseSaslRpcClient {
|
|||
|
||||
private final SaslClient saslClient;
|
||||
private final boolean fallbackAllowed;
|
||||
|
||||
/**
|
||||
* Create a HBaseSaslRpcClient for an authentication method
|
||||
*
|
||||
|
@ -65,11 +64,37 @@ public class HBaseSaslRpcClient {
|
|||
* the requested authentication method
|
||||
* @param token
|
||||
* token to use if needed by the authentication method
|
||||
* @param serverPrincipal
|
||||
* the server principal that we are trying to set the connection up to
|
||||
* @param fallbackAllowed
|
||||
* does the client allow fallback to simple authentication
|
||||
* @throws IOException
|
||||
*/
|
||||
public HBaseSaslRpcClient(AuthMethod method,
|
||||
Token<? extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed)
|
||||
throws IOException {
|
||||
this(method, token, serverPrincipal, fallbackAllowed, "authentication");
|
||||
}
|
||||
/**
|
||||
* Create a HBaseSaslRpcClient for an authentication method
|
||||
*
|
||||
* @param method
|
||||
* the requested authentication method
|
||||
* @param token
|
||||
* token to use if needed by the authentication method
|
||||
* @param serverPrincipal
|
||||
* the server principal that we are trying to set the connection up to
|
||||
* @param fallbackAllowed
|
||||
* does the client allow fallback to simple authentication
|
||||
* @param rpcProtection
|
||||
* the protection level ("authentication", "integrity" or "privacy")
|
||||
* @throws IOException
|
||||
*/
|
||||
public HBaseSaslRpcClient(AuthMethod method,
|
||||
Token<? extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed,
|
||||
String rpcProtection) throws IOException {
|
||||
this.fallbackAllowed = fallbackAllowed;
|
||||
SaslUtil.initSaslProperties(rpcProtection);
|
||||
switch (method) {
|
||||
case DIGEST:
|
||||
if (LOG.isDebugEnabled())
|
||||
|
|
|
@ -23,12 +23,30 @@ import org.apache.commons.codec.binary.Base64;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import javax.security.sasl.Sasl;
|
||||
|
||||
public class SaslUtil {
|
||||
public static final String SASL_DEFAULT_REALM = "default";
|
||||
public static final Map<String, String> SASL_PROPS =
|
||||
new TreeMap<String, String>();
|
||||
public static final int SWITCH_TO_SIMPLE_AUTH = -88;
|
||||
|
||||
public static enum QualityOfProtection {
|
||||
AUTHENTICATION("auth"),
|
||||
INTEGRITY("auth-int"),
|
||||
PRIVACY("auth-conf");
|
||||
|
||||
public final String saslQop;
|
||||
|
||||
private QualityOfProtection(String saslQop) {
|
||||
this.saslQop = saslQop;
|
||||
}
|
||||
|
||||
public String getSaslQop() {
|
||||
return saslQop;
|
||||
}
|
||||
}
|
||||
|
||||
/** Splitting fully qualified Kerberos name into parts */
|
||||
public static String[] splitKerberosName(String fullName) {
|
||||
return fullName.split("[/@]");
|
||||
|
@ -45,4 +63,18 @@ public class SaslUtil {
|
|||
static char[] encodePassword(byte[] password) {
|
||||
return new String(Base64.encodeBase64(password)).toCharArray();
|
||||
}
|
||||
|
||||
static void initSaslProperties(String rpcProtection) {
|
||||
QualityOfProtection saslQOP = QualityOfProtection.AUTHENTICATION;
|
||||
if (QualityOfProtection.INTEGRITY.name().toLowerCase()
|
||||
.equals(rpcProtection)) {
|
||||
saslQOP = QualityOfProtection.INTEGRITY;
|
||||
} else if (QualityOfProtection.PRIVACY.name().toLowerCase().equals(
|
||||
rpcProtection)) {
|
||||
saslQOP = QualityOfProtection.PRIVACY;
|
||||
}
|
||||
|
||||
SaslUtil.SASL_PROPS.put(Sasl.QOP, saslQOP.getSaslQop());
|
||||
SaslUtil.SASL_PROPS.put(Sasl.SERVER_AUTH, "true");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,406 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.zookeeper;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Helper class for table state tracking for use by AssignmentManager.
|
||||
* Reads, caches and sets state up in zookeeper. If multiple read/write
|
||||
* clients, will make for confusion. Read-only clients other than
|
||||
* AssignmentManager interested in learning table state can use the
|
||||
* read-only utility methods in {@link ZKTableReadOnly}.
|
||||
*
|
||||
* <p>To save on trips to the zookeeper ensemble, internally we cache table
|
||||
* state.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKTable {
|
||||
// A znode will exist under the table directory if it is in any of the
|
||||
// following states: {@link TableState#ENABLING} , {@link TableState#DISABLING},
|
||||
// or {@link TableState#DISABLED}. If {@link TableState#ENABLED}, there will
|
||||
// be no entry for a table in zk. Thats how it currently works.
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ZKTable.class);
|
||||
private final ZooKeeperWatcher watcher;
|
||||
|
||||
/**
|
||||
* Cache of what we found in zookeeper so we don't have to go to zk ensemble
|
||||
* for every query. Synchronize access rather than use concurrent Map because
|
||||
* synchronization needs to span query of zk.
|
||||
*/
|
||||
private final Map<TableName, ZooKeeperProtos.Table.State> cache =
|
||||
new HashMap<TableName, ZooKeeperProtos.Table.State>();
|
||||
|
||||
// TODO: Make it so always a table znode. Put table schema here as well as table state.
|
||||
// Have watcher on table znode so all are notified of state or schema change.
|
||||
|
||||
public ZKTable(final ZooKeeperWatcher zkw) throws KeeperException, InterruptedException {
|
||||
super();
|
||||
this.watcher = zkw;
|
||||
populateTableStates();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private void populateTableStates()
|
||||
throws KeeperException, InterruptedException {
|
||||
synchronized (this.cache) {
|
||||
List<String> children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
|
||||
if (children == null) return;
|
||||
for (String child: children) {
|
||||
TableName tableName = TableName.valueOf(child);
|
||||
ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(this.watcher, tableName);
|
||||
if (state != null) this.cache.put(tableName, state);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as DISABLED in zookeeper. Fails silently if the
|
||||
* table is already disabled in zookeeper. Sets no watches.
|
||||
* @param tableName
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public void setDisabledTable(TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (!isDisablingOrDisabledTable(tableName)) {
|
||||
LOG.warn("Moving table " + tableName + " state to disabled but was " +
|
||||
"not first in disabling state: " + this.cache.get(tableName));
|
||||
}
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as DISABLING in zookeeper. Fails silently if the
|
||||
* table is already disabled in zookeeper. Sets no watches.
|
||||
* @param tableName
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public void setDisablingTable(final TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (!isEnabledOrDisablingTable(tableName)) {
|
||||
LOG.warn("Moving table " + tableName + " state to disabling but was " +
|
||||
"not first in enabled state: " + this.cache.get(tableName));
|
||||
}
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as ENABLING in zookeeper. Fails silently if the
|
||||
* table is already disabled in zookeeper. Sets no watches.
|
||||
* @param tableName
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public void setEnablingTable(final TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (!isDisabledOrEnablingTable(tableName)) {
|
||||
LOG.warn("Moving table " + tableName + " state to enabling but was " +
|
||||
"not first in disabled state: " + this.cache.get(tableName));
|
||||
}
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as ENABLING in zookeeper atomically
|
||||
* If the table is already in ENABLING state, no operation is performed
|
||||
* @param tableName
|
||||
* @return if the operation succeeds or not
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public boolean checkAndSetEnablingTable(final TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (isEnablingOrEnabledTable(tableName)) {
|
||||
return false;
|
||||
}
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as ENABLING in zookeeper atomically
|
||||
* If the table isn't in DISABLED state, no operation is performed
|
||||
* @param tableName
|
||||
* @return if the operation succeeds or not
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public boolean checkDisabledAndSetEnablingTable(final TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (!isDisabledTable(tableName)) {
|
||||
return false;
|
||||
}
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as DISABLING in zookeeper atomically
|
||||
* If the table isn't in ENABLED state, no operation is performed
|
||||
* @param tableName
|
||||
* @return if the operation succeeds or not
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public boolean checkEnabledAndSetDisablingTable(final TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (this.cache.get(tableName) != null && !isEnabledTable(tableName)) {
|
||||
return false;
|
||||
}
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private void setTableState(final TableName tableName, final ZooKeeperProtos.Table.State state)
|
||||
throws KeeperException {
|
||||
String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString());
|
||||
if (ZKUtil.checkExists(this.watcher, znode) == -1) {
|
||||
ZKUtil.createAndFailSilent(this.watcher, znode);
|
||||
}
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
|
||||
builder.setState(state);
|
||||
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||
ZKUtil.setData(this.watcher, znode, data);
|
||||
this.cache.put(tableName, state);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isDisabledTable(final TableName tableName) {
|
||||
return isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
|
||||
}
|
||||
|
||||
public boolean isDisablingTable(final TableName tableName) {
|
||||
return isTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
|
||||
}
|
||||
|
||||
public boolean isEnablingTable(final TableName tableName) {
|
||||
return isTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
|
||||
}
|
||||
|
||||
public boolean isEnabledTable(TableName tableName) {
|
||||
return isTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
|
||||
}
|
||||
|
||||
public boolean isDisablingOrDisabledTable(final TableName tableName) {
|
||||
synchronized (this.cache) {
|
||||
return isDisablingTable(tableName) || isDisabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isEnablingOrEnabledTable(final TableName tableName) {
|
||||
synchronized (this.cache) {
|
||||
return isEnablingTable(tableName) || isEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isEnabledOrDisablingTable(final TableName tableName) {
|
||||
synchronized (this.cache) {
|
||||
return isEnabledTable(tableName) || isDisablingTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isDisabledOrEnablingTable(final TableName tableName) {
|
||||
synchronized (this.cache) {
|
||||
return isDisabledTable(tableName) || isEnablingTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isTableState(final TableName tableName, final ZooKeeperProtos.Table.State state) {
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
|
||||
return ZKTableReadOnly.isTableState(currentState, state);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the table in zookeeper. Fails silently if the
|
||||
* table is not currently disabled in zookeeper. Sets no watches.
|
||||
* @param tableName
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public void setDeletedTable(final TableName tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (this.cache.remove(tableName) == null) {
|
||||
LOG.warn("Moving table " + tableName + " state to deleted but was " +
|
||||
"already deleted");
|
||||
}
|
||||
ZKUtil.deleteNodeFailSilent(this.watcher,
|
||||
ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ENABLED state in the cache and creates or force updates a node to
|
||||
* ENABLED state for the specified table
|
||||
*
|
||||
* @param tableName
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public void setEnabledTable(final TableName tableName) throws KeeperException {
|
||||
setTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
* check if table is present .
|
||||
*
|
||||
* @param tableName
|
||||
* @return true if the table is present
|
||||
*/
|
||||
public boolean isTablePresent(final TableName tableName) {
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.State state = this.cache.get(tableName);
|
||||
return !(state == null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @return Set of disabled tables, empty Set if none
|
||||
*/
|
||||
public Set<TableName> getDisabledTables() {
|
||||
Set<TableName> disabledTables = new HashSet<TableName>();
|
||||
synchronized (this.cache) {
|
||||
Set<TableName> tables = this.cache.keySet();
|
||||
for (TableName table: tables) {
|
||||
if (isDisabledTable(table)) disabledTables.add(table);
|
||||
}
|
||||
}
|
||||
return disabledTables;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @return Set of disabled tables, empty Set if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static Set<TableName> getDisabledTables(ZooKeeperWatcher zkw)
|
||||
throws KeeperException, InterruptedIOException {
|
||||
return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabling in zookeeper.
|
||||
* @return Set of disabling tables, empty Set if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static Set<TableName> getDisablingTables(ZooKeeperWatcher zkw)
|
||||
throws KeeperException, InterruptedIOException {
|
||||
return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as enabling in zookeeper.
|
||||
* @return Set of enabling tables, empty Set if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static Set<TableName> getEnablingTables(ZooKeeperWatcher zkw)
|
||||
throws KeeperException, InterruptedIOException {
|
||||
return getAllTables(zkw, ZooKeeperProtos.Table.State.ENABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @return Set of disabled tables, empty Set if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static Set<TableName> getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
|
||||
throws KeeperException, InterruptedIOException {
|
||||
return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
* If the table is found in ENABLING state the inmemory state is removed. This
|
||||
* helps in cases where CreateTable is to be retried by the client incase of
|
||||
* failures. If deleteZNode is true - the znode is also deleted
|
||||
*
|
||||
* @param tableName
|
||||
* @param deleteZNode
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public void removeEnablingTable(final TableName tableName, boolean deleteZNode)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (isEnablingTable(tableName)) {
|
||||
this.cache.remove(tableName);
|
||||
if (deleteZNode) {
|
||||
ZKUtil.deleteNodeFailSilent(this.watcher,
|
||||
ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables of specified states in zookeeper.
|
||||
* @return Set of tables of specified states, empty Set if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
static Set<TableName> getAllTables(final ZooKeeperWatcher zkw,
|
||||
final ZooKeeperProtos.Table.State... states) throws KeeperException, InterruptedIOException {
|
||||
Set<TableName> allTables = new HashSet<TableName>();
|
||||
List<String> children =
|
||||
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
|
||||
if(children == null) return allTables;
|
||||
for (String child: children) {
|
||||
TableName tableName = TableName.valueOf(child);
|
||||
ZooKeeperProtos.Table.State state = null;
|
||||
try {
|
||||
state = ZKTableReadOnly.getTableState(zkw, tableName);
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
for (ZooKeeperProtos.Table.State expectedState: states) {
|
||||
if (state == expectedState) {
|
||||
allTables.add(tableName);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return allTables;
|
||||
}
|
||||
}
|
|
@ -33,23 +33,24 @@ import java.util.List;
|
|||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Non-instantiable class that provides helper functions for
|
||||
* clients other than AssignmentManager for reading the
|
||||
* state of a table in ZK.
|
||||
* Non-instantiable class that provides helper functions to learn
|
||||
* about HBase table state for code running on client side (hence, not having
|
||||
* access to consensus context).
|
||||
*
|
||||
* <p>Does not cache state like {@link ZKTable}, actually reads from ZK each call.
|
||||
* Doesn't cache any table state, just goes directly to ZooKeeper.
|
||||
* TODO: decouple this class from ZooKeeper.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKTableReadOnly {
|
||||
public class ZKTableStateClientSideReader {
|
||||
|
||||
private ZKTableReadOnly() {}
|
||||
private ZKTableStateClientSideReader() {}
|
||||
|
||||
/**
|
||||
* Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#DISABLED}.
|
||||
* This method does not use cache.
|
||||
* This method is for clients other than AssignmentManager
|
||||
* @param zkw
|
||||
* @param tableName
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @param tableName table we're checking
|
||||
* @return True if table is enabled.
|
||||
* @throws KeeperException
|
||||
*/
|
||||
|
@ -64,8 +65,8 @@ public class ZKTableReadOnly {
|
|||
* Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#ENABLED}.
|
||||
* This method does not use cache.
|
||||
* This method is for clients other than AssignmentManager
|
||||
* @param zkw
|
||||
* @param tableName
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @param tableName table we're checking
|
||||
* @return True if table is enabled.
|
||||
* @throws KeeperException
|
||||
*/
|
||||
|
@ -80,8 +81,8 @@ public class ZKTableReadOnly {
|
|||
* of {@code ZooKeeperProtos.Table.State#DISABLED}.
|
||||
* This method does not use cache.
|
||||
* This method is for clients other than AssignmentManager.
|
||||
* @param zkw
|
||||
* @param tableName
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @param tableName table we're checking
|
||||
* @return True if table is enabled.
|
||||
* @throws KeeperException
|
||||
*/
|
||||
|
@ -139,8 +140,8 @@ public class ZKTableReadOnly {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param zkw
|
||||
* @param tableName
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @param tableName table we're checking
|
||||
* @return Null or {@link ZooKeeperProtos.Table.State} found in znode.
|
||||
* @throws KeeperException
|
||||
*/
|
|
@ -0,0 +1,330 @@
|
|||
/**
|
||||
*
|
||||
* 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.zookeeper;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Implementation of TableStateManager which reads, caches and sets state
|
||||
* up in ZooKeeper. If multiple read/write clients, will make for confusion.
|
||||
* Code running on client side without consensus context should use
|
||||
* {@link ZKTableStateClientSideReader} instead.
|
||||
*
|
||||
* <p>To save on trips to the zookeeper ensemble, internally we cache table
|
||||
* state.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKTableStateManager implements TableStateManager {
|
||||
// A znode will exist under the table directory if it is in any of the
|
||||
// following states: {@link TableState#ENABLING} , {@link TableState#DISABLING},
|
||||
// or {@link TableState#DISABLED}. If {@link TableState#ENABLED}, there will
|
||||
// be no entry for a table in zk. Thats how it currently works.
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ZKTableStateManager.class);
|
||||
private final ZooKeeperWatcher watcher;
|
||||
|
||||
/**
|
||||
* Cache of what we found in zookeeper so we don't have to go to zk ensemble
|
||||
* for every query. Synchronize access rather than use concurrent Map because
|
||||
* synchronization needs to span query of zk.
|
||||
*/
|
||||
private final Map<TableName, ZooKeeperProtos.Table.State> cache =
|
||||
new HashMap<TableName, ZooKeeperProtos.Table.State>();
|
||||
|
||||
public ZKTableStateManager(final ZooKeeperWatcher zkw) throws KeeperException,
|
||||
InterruptedException {
|
||||
super();
|
||||
this.watcher = zkw;
|
||||
populateTableStates();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabled in zookeeper.
|
||||
* @throws KeeperException, InterruptedException
|
||||
*/
|
||||
private void populateTableStates() throws KeeperException, InterruptedException {
|
||||
synchronized (this.cache) {
|
||||
List<String> children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
|
||||
if (children == null) return;
|
||||
for (String child: children) {
|
||||
TableName tableName = TableName.valueOf(child);
|
||||
ZooKeeperProtos.Table.State state = getTableState(this.watcher, tableName);
|
||||
if (state != null) this.cache.put(tableName, state);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets table state in ZK. Sets no watches.
|
||||
*
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void setTableState(TableName tableName, ZooKeeperProtos.Table.State state)
|
||||
throws CoordinatedStateException {
|
||||
synchronized (this.cache) {
|
||||
LOG.warn("Moving table " + tableName + " state from " + this.cache.get(tableName)
|
||||
+ " to " + state);
|
||||
try {
|
||||
setTableStateInZK(tableName, state);
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks and sets table state in ZK. Sets no watches.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean setTableStateIfInStates(TableName tableName,
|
||||
ZooKeeperProtos.Table.State newState,
|
||||
ZooKeeperProtos.Table.State... states)
|
||||
throws CoordinatedStateException {
|
||||
synchronized (this.cache) {
|
||||
// Transition ENABLED->DISABLING has to be performed with a hack, because
|
||||
// we treat empty state as enabled in this case because 0.92- clusters.
|
||||
if (
|
||||
(newState == ZooKeeperProtos.Table.State.DISABLING) &&
|
||||
this.cache.get(tableName) != null && !isTableState(tableName, states) ||
|
||||
(newState != ZooKeeperProtos.Table.State.DISABLING &&
|
||||
!isTableState(tableName, states) )) {
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
setTableStateInZK(tableName, newState);
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks and sets table state in ZK. Sets no watches.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean setTableStateIfNotInStates(TableName tableName,
|
||||
ZooKeeperProtos.Table.State newState,
|
||||
ZooKeeperProtos.Table.State... states)
|
||||
throws CoordinatedStateException {
|
||||
synchronized (this.cache) {
|
||||
if (isTableState(tableName, states)) {
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
setTableStateInZK(tableName, newState);
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private void setTableStateInZK(final TableName tableName,
|
||||
final ZooKeeperProtos.Table.State state)
|
||||
throws KeeperException {
|
||||
String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString());
|
||||
if (ZKUtil.checkExists(this.watcher, znode) == -1) {
|
||||
ZKUtil.createAndFailSilent(this.watcher, znode);
|
||||
}
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
|
||||
builder.setState(state);
|
||||
byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
|
||||
ZKUtil.setData(this.watcher, znode, data);
|
||||
this.cache.put(tableName, state);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if table is marked in specified state in ZK.
|
||||
*
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public boolean isTableState(final TableName tableName,
|
||||
final ZooKeeperProtos.Table.State... states) {
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
|
||||
return isTableInState(Arrays.asList(states), currentState);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the table in zookeeper. Fails silently if the
|
||||
* table is not currently disabled in zookeeper. Sets no watches.
|
||||
*
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void setDeletedTable(final TableName tableName)
|
||||
throws CoordinatedStateException {
|
||||
synchronized (this.cache) {
|
||||
if (this.cache.remove(tableName) == null) {
|
||||
LOG.warn("Moving table " + tableName + " state to deleted but was " +
|
||||
"already deleted");
|
||||
}
|
||||
try {
|
||||
ZKUtil.deleteNodeFailSilent(this.watcher,
|
||||
ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* check if table is present.
|
||||
*
|
||||
* @param tableName table we're working on
|
||||
* @return true if the table is present
|
||||
*/
|
||||
@Override
|
||||
public boolean isTablePresent(final TableName tableName) {
|
||||
synchronized (this.cache) {
|
||||
ZooKeeperProtos.Table.State state = this.cache.get(tableName);
|
||||
return !(state == null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables set as disabling in zookeeper.
|
||||
* @return Set of disabling tables, empty Set if none
|
||||
* @throws CoordinatedStateException if error happened in underlying coordination engine
|
||||
*/
|
||||
@Override
|
||||
public Set<TableName> getTablesInStates(ZooKeeperProtos.Table.State... states)
|
||||
throws InterruptedIOException, CoordinatedStateException {
|
||||
try {
|
||||
return getAllTables(states);
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states,
|
||||
boolean deletePermanentState)
|
||||
throws CoordinatedStateException {
|
||||
synchronized (this.cache) {
|
||||
if (isTableState(tableName, states)) {
|
||||
this.cache.remove(tableName);
|
||||
if (deletePermanentState) {
|
||||
try {
|
||||
ZKUtil.deleteNodeFailSilent(this.watcher,
|
||||
ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a list of all the tables of specified states in zookeeper.
|
||||
* @return Set of tables of specified states, empty Set if none
|
||||
* @throws KeeperException
|
||||
*/
|
||||
Set<TableName> getAllTables(final ZooKeeperProtos.Table.State... states)
|
||||
throws KeeperException, InterruptedIOException {
|
||||
|
||||
Set<TableName> allTables = new HashSet<TableName>();
|
||||
List<String> children =
|
||||
ZKUtil.listChildrenNoWatch(watcher, watcher.tableZNode);
|
||||
if(children == null) return allTables;
|
||||
for (String child: children) {
|
||||
TableName tableName = TableName.valueOf(child);
|
||||
ZooKeeperProtos.Table.State state;
|
||||
try {
|
||||
state = getTableState(watcher, tableName);
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
for (ZooKeeperProtos.Table.State expectedState: states) {
|
||||
if (state == expectedState) {
|
||||
allTables.add(tableName);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return allTables;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets table state from ZK.
|
||||
* @param zkw ZooKeeperWatcher instance to use
|
||||
* @param tableName table we're checking
|
||||
* @return Null or {@link ZooKeeperProtos.Table.State} found in znode.
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
|
||||
final TableName tableName)
|
||||
throws KeeperException, InterruptedException {
|
||||
String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
|
||||
byte [] data = ZKUtil.getData(zkw, znode);
|
||||
if (data == null || data.length <= 0) return null;
|
||||
try {
|
||||
ProtobufUtil.expectPBMagicPrefix(data);
|
||||
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
|
||||
int magicLen = ProtobufUtil.lengthOfPBMagic();
|
||||
ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
|
||||
return t.getState();
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
KeeperException ke = new KeeperException.DataInconsistencyException();
|
||||
ke.initCause(e);
|
||||
throw ke;
|
||||
} catch (DeserializationException e) {
|
||||
throw ZKUtil.convert(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if current state isn't null and is contained
|
||||
* in the list of expected states.
|
||||
*/
|
||||
private boolean isTableInState(final List<ZooKeeperProtos.Table.State> expectedStates,
|
||||
final ZooKeeperProtos.Table.State currentState) {
|
||||
return currentState != null && expectedStates.contains(currentState);
|
||||
}
|
||||
}
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Assert;
|
||||
|
@ -125,14 +126,14 @@ public class TestAsyncProcess {
|
|||
public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
|
||||
super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
|
||||
new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
|
||||
new RpcRetryingCallerFactory(conf), false);
|
||||
new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf));
|
||||
}
|
||||
|
||||
public MyAsyncProcess(
|
||||
ClusterConnection hc, Configuration conf, boolean useGlobalErrors) {
|
||||
super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
|
||||
new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())),
|
||||
new RpcRetryingCallerFactory(conf), useGlobalErrors);
|
||||
new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -989,7 +989,8 @@ public final class HConstants {
|
|||
public static final String REPLICATION_CODEC_CONF_KEY = "hbase.replication.rpc.codec";
|
||||
|
||||
/** Config for pluggable consensus provider */
|
||||
public static final String HBASE_CONSENSUS_PROVIDER_CLASS = "hbase.consensus.provider.class";
|
||||
public static final String HBASE_COORDINATED_STATE_MANAGER_CLASS =
|
||||
"hbase.coordinated.state.manager.class";
|
||||
|
||||
private HConstants() {
|
||||
// Can't be instantiated with this ctor.
|
||||
|
|
|
@ -360,7 +360,8 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
ByteBufferUtils.readCompressedInt(block); // commonLength
|
||||
int pos = block.position();
|
||||
block.reset();
|
||||
return ByteBuffer.wrap(block.array(), pos, keyLength).slice();
|
||||
return ByteBuffer.wrap(block.array(), block.arrayOffset() + pos, keyLength)
|
||||
.slice();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -137,7 +137,8 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
}
|
||||
int pos = block.position();
|
||||
block.reset();
|
||||
return ByteBuffer.wrap(block.array(), pos, keyLength).slice();
|
||||
return ByteBuffer.wrap(block.array(), block.arrayOffset() + pos, keyLength)
|
||||
.slice();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1180,8 +1180,8 @@ possible configurations would overwhelm and obscure the important.
|
|||
and add the fully qualified class name here.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.consensus.provider.class</name>
|
||||
<value>org.apache.hadoop.hbase.consensus.ZkConsensusProvider</value>
|
||||
<description>Fully qualified name of class implementing consensus.</description>
|
||||
<name>hbase.coordinated.state.manager.class</name>
|
||||
<value>org.apache.hadoop.hbase.consensus.ZkCoordinatedStateManager</value>
|
||||
<description>Fully qualified name of class implementing coordinated state manager.</description>
|
||||
</property>
|
||||
</configuration>
|
||||
|
|
|
@ -179,7 +179,7 @@ public interface MetricsRegionServerSource extends BaseSource {
|
|||
String BLOCK_CACHE_EVICTION_COUNT = "blockCacheEvictionCount";
|
||||
String BLOCK_CACHE_EVICTION_COUNT_DESC =
|
||||
"Count of the number of blocks evicted from the block cache.";
|
||||
String BLOCK_CACHE_HIT_PERCENT = "blockCountHitPercent";
|
||||
String BLOCK_CACHE_HIT_PERCENT = "blockCacheCountHitPercent";
|
||||
String BLOCK_CACHE_HIT_PERCENT_DESC =
|
||||
"Percent of block cache requests that are hits";
|
||||
String BLOCK_CACHE_EXPRESS_HIT_PERCENT = "blockCacheExpressHitPercent";
|
||||
|
|
|
@ -200,7 +200,7 @@ public interface MetricsRegionServerWrapper {
|
|||
/**
|
||||
* Get the percent of all requests that hit the block cache.
|
||||
*/
|
||||
int getBlockCacheHitPercent();
|
||||
double getBlockCacheHitPercent();
|
||||
|
||||
/**
|
||||
* Get the percent of requests with the block cache turned on that hit the block cache.
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.security.access.AccessController;
|
||||
|
@ -45,8 +46,10 @@ public class IntegrationTestIngestWithACL extends IntegrationTestIngest {
|
|||
private static final int SPECIAL_PERM_CELL_INSERTION_FACTOR = 100;
|
||||
public static final String OPT_SUPERUSER = "superuser";
|
||||
public static final String OPT_USERS = "userlist";
|
||||
public static final String OPT_AUTHN = "authinfo";
|
||||
private String superUser = "owner";
|
||||
private String userNames = "user1,user2,user3,user4";
|
||||
private String authnFileName;
|
||||
@Override
|
||||
public void setUpCluster() throws Exception {
|
||||
util = getTestingUtil(null);
|
||||
|
@ -66,6 +69,10 @@ public class IntegrationTestIngestWithACL extends IntegrationTestIngest {
|
|||
tmp.add(HYPHEN + LoadTestTool.OPT_GENERATOR);
|
||||
StringBuilder sb = new StringBuilder(LoadTestDataGeneratorWithACL.class.getName());
|
||||
sb.append(COLON);
|
||||
if (LoadTestTool.isSecure(getConf())) {
|
||||
sb.append(authnFileName);
|
||||
sb.append(COLON);
|
||||
}
|
||||
sb.append(superUser);
|
||||
sb.append(COLON);
|
||||
sb.append(userNames);
|
||||
|
@ -80,7 +87,15 @@ public class IntegrationTestIngestWithACL extends IntegrationTestIngest {
|
|||
super.addOptWithArg(OPT_SUPERUSER,
|
||||
"Super user name used to add the ACL permissions");
|
||||
super.addOptWithArg(OPT_USERS,
|
||||
"List of users to be added with the ACLs. Should be comma seperated.");
|
||||
"List of users to be added with the ACLs. Should be comma seperated.");
|
||||
super
|
||||
.addOptWithArg(
|
||||
OPT_AUTHN,
|
||||
"The name of the properties file that contains kerberos key tab file and principal definitions. " +
|
||||
"The principal key in the file should be of the form hbase.<username>.kerberos.principal." +
|
||||
" The keytab key in the file should be of the form hbase.<username>.keytab.file. Example: " +
|
||||
"hbase.user1.kerberos.principal=user1/fully.qualified.domain.name@YOUR-REALM.COM, " +
|
||||
"hbase.user1.keytab.file=<filelocation>.");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -92,6 +107,21 @@ public class IntegrationTestIngestWithACL extends IntegrationTestIngest {
|
|||
if (cmd.hasOption(OPT_USERS)) {
|
||||
userNames = cmd.getOptionValue(OPT_USERS);
|
||||
}
|
||||
if (LoadTestTool.isSecure(getConf())) {
|
||||
boolean authFileNotFound = false;
|
||||
if (cmd.hasOption(OPT_AUTHN)) {
|
||||
authnFileName = cmd.getOptionValue(OPT_AUTHN);
|
||||
if (StringUtils.isEmpty(authnFileName)) {
|
||||
authFileNotFound = true;
|
||||
}
|
||||
} else {
|
||||
authFileNotFound = true;
|
||||
}
|
||||
if (authFileNotFound) {
|
||||
super.printUsage();
|
||||
System.exit(EXIT_FAILURE);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
|
|
@ -19,25 +19,25 @@ package org.apache.hadoop.hbase;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.consensus.ZkConsensusProvider;
|
||||
import org.apache.hadoop.hbase.consensus.ZkCoordinatedStateManager;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
* Creates instance of {@link ConsensusProvider}
|
||||
* Creates instance of {@link CoordinatedStateManager}
|
||||
* based on configuration.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ConsensusProviderFactory {
|
||||
public class CoordinatedStateManagerFactory {
|
||||
|
||||
/**
|
||||
* Creates consensus provider from the given configuration.
|
||||
* @param conf Configuration
|
||||
* @return A {@link ConsensusProvider}
|
||||
* @return Implementation of {@link CoordinatedStateManager}
|
||||
*/
|
||||
public static ConsensusProvider getConsensusProvider(Configuration conf) {
|
||||
Class<? extends ConsensusProvider> consensusKlass =
|
||||
conf.getClass(HConstants.HBASE_CONSENSUS_PROVIDER_CLASS, ZkConsensusProvider.class,
|
||||
ConsensusProvider.class);
|
||||
return ReflectionUtils.newInstance(consensusKlass, conf);
|
||||
public static CoordinatedStateManager getCoordinatedStateManager(Configuration conf) {
|
||||
Class<? extends CoordinatedStateManager> coordinatedStateMgrKlass =
|
||||
conf.getClass(HConstants.HBASE_COORDINATED_STATE_MANAGER_CLASS,
|
||||
ZkCoordinatedStateManager.class, CoordinatedStateManager.class);
|
||||
return ReflectionUtils.newInstance(coordinatedStateMgrKlass, conf);
|
||||
}
|
||||
}
|
|
@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
|
@ -174,10 +173,10 @@ public class LocalHBaseCluster {
|
|||
// its HConnection instance rather than share (see HBASE_INSTANCES down in
|
||||
// the guts of HConnectionManager.
|
||||
|
||||
// Also, create separate ConsensusProvider instance per Server.
|
||||
// This is special case when we have to have more than 1 ConsensusProvider
|
||||
// Also, create separate CoordinatedStateManager instance per Server.
|
||||
// This is special case when we have to have more than 1 CoordinatedStateManager
|
||||
// within 1 process.
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
|
||||
JVMClusterUtil.RegionServerThread rst =
|
||||
JVMClusterUtil.createRegionServerThread(config, cp,
|
||||
|
@ -207,10 +206,10 @@ public class LocalHBaseCluster {
|
|||
// its HConnection instance rather than share (see HBASE_INSTANCES down in
|
||||
// the guts of HConnectionManager.
|
||||
|
||||
// Also, create separate ConsensusProvider instance per Server.
|
||||
// This is special case when we have to have more than 1 ConsensusProvider
|
||||
// Also, create separate CoordinatedStateManager instance per Server.
|
||||
// This is special case when we have to have more than 1 CoordinatedStateManager
|
||||
// within 1 process.
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
|
||||
JVMClusterUtil.MasterThread mt = JVMClusterUtil.createMasterThread(c, cp,
|
||||
(Class<? extends HMaster>) conf.getClass(HConstants.MASTER_IMPL, this.masterClass), index);
|
||||
|
|
|
@ -18,17 +18,19 @@
|
|||
package org.apache.hadoop.hbase.consensus;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
|
||||
/**
|
||||
* Base class for {@link org.apache.hadoop.hbase.ConsensusProvider} implementations.
|
||||
* Defines methods to retrieve consensus objects for relevant areas. ConsensusProvider
|
||||
* Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations.
|
||||
* Defines methods to retrieve consensus objects for relevant areas. CoordinatedStateManager
|
||||
* reference returned from Server interface has to be casted to this type to
|
||||
* access those methods.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class BaseConsensusProvider implements ConsensusProvider {
|
||||
public abstract class BaseCoordinatedStateManager implements CoordinatedStateManager {
|
||||
|
||||
@Override
|
||||
public void initialize(Server server) {
|
||||
|
@ -46,4 +48,8 @@ public abstract class BaseConsensusProvider implements ConsensusProvider {
|
|||
public Server getServer() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract TableStateManager getTableStateManager() throws InterruptedException,
|
||||
CoordinatedStateException;
|
||||
}
|
|
@ -17,15 +17,22 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.consensus;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.ConsensusProvider}.
|
||||
* ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZkConsensusProvider extends BaseConsensusProvider {
|
||||
public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
|
||||
private static final Log LOG = LogFactory.getLog(ZkCoordinatedStateManager.class);
|
||||
private Server server;
|
||||
private ZooKeeperWatcher watcher;
|
||||
|
||||
|
@ -39,4 +46,14 @@ public class ZkConsensusProvider extends BaseConsensusProvider {
|
|||
public Server getServer() {
|
||||
return server;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableStateManager getTableStateManager() throws InterruptedException,
|
||||
CoordinatedStateException {
|
||||
try {
|
||||
return new ZKTableStateManager(server.getZooKeeper());
|
||||
} catch (KeeperException e) {
|
||||
throw new CoordinatedStateException(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -51,7 +51,7 @@ public class KeyValueSortReducer extends Reducer<ImmutableBytesWritable, KeyValu
|
|||
int index = 0;
|
||||
for (KeyValue kv: map) {
|
||||
context.write(row, kv);
|
||||
if (index > 0 && index % 100 == 0) context.setStatus("Wrote " + index);
|
||||
if (++index % 100 == 0) context.setStatus("Wrote " + index);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -43,6 +43,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -52,6 +53,7 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
|
@ -69,6 +71,7 @@ import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
|
|||
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
|
@ -82,7 +85,6 @@ import org.apache.hadoop.hbase.util.Threads;
|
|||
import org.apache.hadoop.hbase.util.Triple;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
@ -161,7 +163,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
final NavigableMap<String, RegionPlan> regionPlans =
|
||||
new TreeMap<String, RegionPlan>();
|
||||
|
||||
private final ZKTable zkTable;
|
||||
private final TableStateManager tableStateManager;
|
||||
|
||||
private final ExecutorService executorService;
|
||||
|
||||
|
@ -233,7 +235,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
public AssignmentManager(Server server, ServerManager serverManager,
|
||||
CatalogTracker catalogTracker, final LoadBalancer balancer,
|
||||
final ExecutorService service, MetricsMaster metricsMaster,
|
||||
final TableLockManager tableLockManager) throws KeeperException, IOException {
|
||||
final TableLockManager tableLockManager) throws KeeperException,
|
||||
IOException, CoordinatedStateException {
|
||||
super(server.getZooKeeper());
|
||||
this.server = server;
|
||||
this.serverManager = serverManager;
|
||||
|
@ -247,7 +250,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
|
||||
FavoredNodeLoadBalancer.class);
|
||||
try {
|
||||
this.zkTable = new ZKTable(this.watcher);
|
||||
if (server.getCoordinatedStateManager() != null) {
|
||||
this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager();
|
||||
} else {
|
||||
this.tableStateManager = null;
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
|
@ -277,12 +284,12 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Instance of ZKTable.
|
||||
* @return Instance of ZKTableStateManager.
|
||||
*/
|
||||
public ZKTable getZKTable() {
|
||||
public TableStateManager getTableStateManager() {
|
||||
// These are 'expensive' to make involving trip to zk ensemble so allow
|
||||
// sharing.
|
||||
return this.zkTable;
|
||||
return this.tableStateManager;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -387,9 +394,10 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
* @throws CoordinatedStateException
|
||||
*/
|
||||
void joinCluster() throws IOException,
|
||||
KeeperException, InterruptedException {
|
||||
KeeperException, InterruptedException, CoordinatedStateException {
|
||||
// Concurrency note: In the below the accesses on regionsInTransition are
|
||||
// outside of a synchronization block where usually all accesses to RIT are
|
||||
// synchronized. The presumption is that in this case it is safe since this
|
||||
|
@ -400,8 +408,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
// Scan hbase:meta to build list of existing regions, servers, and assignment
|
||||
// Returns servers who have not checked in (assumed dead) and their regions
|
||||
Map<ServerName, List<HRegionInfo>> deadServers = rebuildUserRegions();
|
||||
Map<ServerName, List<HRegionInfo>> deadServers;
|
||||
|
||||
deadServers = rebuildUserRegions();
|
||||
// This method will assign all user regions if a clean server startup or
|
||||
// it will reconstruct master state and cleanup any leftovers from
|
||||
// previous master process.
|
||||
|
@ -424,7 +433,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
*/
|
||||
void processDeadServersAndRegionsInTransition(
|
||||
final Map<ServerName, List<HRegionInfo>> deadServers)
|
||||
throws KeeperException, IOException, InterruptedException {
|
||||
throws KeeperException, IOException, InterruptedException, CoordinatedStateException {
|
||||
List<String> nodes = ZKUtil.listChildrenNoWatch(watcher,
|
||||
watcher.assignmentZNode);
|
||||
|
||||
|
@ -1134,7 +1143,10 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
HRegionInfo regionInfo = rs.getRegion();
|
||||
String regionNameStr = regionInfo.getRegionNameAsString();
|
||||
LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
|
||||
boolean disabled = getZKTable().isDisablingOrDisabledTable(regionInfo.getTable());
|
||||
|
||||
boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
|
||||
|
||||
ServerName serverName = rs.getServerName();
|
||||
if (serverManager.isServerOnline(serverName)) {
|
||||
if (rs.isOnServer(serverName)
|
||||
|
@ -1819,7 +1831,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// assignRegion then we need to make the table ENABLED. Hence in such case the table
|
||||
// will not be in ENABLING or ENABLED state.
|
||||
TableName tableName = region.getTable();
|
||||
if (!zkTable.isEnablingTable(tableName) && !zkTable.isEnabledTable(tableName)) {
|
||||
if (!tableStateManager.isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED, ZooKeeperProtos.Table.State.ENABLING)) {
|
||||
LOG.debug("Setting table " + tableName + " to ENABLED state.");
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
|
@ -2001,11 +2014,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
|
||||
TableName tableName = region.getTable();
|
||||
boolean disabled = this.zkTable.isDisabledTable(tableName);
|
||||
if (disabled || this.zkTable.isDisablingTable(tableName)) {
|
||||
LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
|
||||
" skipping assign of " + region.getRegionNameAsString());
|
||||
if (this.tableStateManager.isTableState(region.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
LOG.info("Table " + region.getTable() + " is disabled or disabling;"
|
||||
+ " skipping assign of " + region.getRegionNameAsString());
|
||||
offlineDisabledRegion(region);
|
||||
return true;
|
||||
}
|
||||
|
@ -2455,7 +2468,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws KeeperException
|
||||
*/
|
||||
private void assignAllUserRegions()
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
|
||||
// Cleanup any existing ZK nodes and start watching
|
||||
ZKAssign.deleteAllNodes(watcher);
|
||||
ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
|
||||
|
@ -2465,8 +2478,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Skip assignment for regions of tables in DISABLING state because during clean cluster startup
|
||||
// no RS is alive and regions map also doesn't have any information about the regions.
|
||||
// See HBASE-6281.
|
||||
Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
|
||||
disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
|
||||
Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
|
||||
// Scan hbase:meta for all user regions, skipping any disabled tables
|
||||
Map<HRegionInfo, ServerName> allRegions;
|
||||
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
|
||||
|
@ -2488,7 +2504,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
for (HRegionInfo hri : allRegions.keySet()) {
|
||||
TableName tableName = hri.getTable();
|
||||
if (!zkTable.isEnabledTable(tableName)) {
|
||||
if (!tableStateManager.isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
@ -2527,12 +2544,17 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* in META
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<ServerName, List<HRegionInfo>> rebuildUserRegions() throws IOException, KeeperException {
|
||||
Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
|
||||
Set<TableName> disabledOrEnablingTables = ZKTable.getDisabledTables(watcher);
|
||||
disabledOrEnablingTables.addAll(enablingTables);
|
||||
Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisablingTables(watcher);
|
||||
disabledOrDisablingOrEnabling.addAll(disabledOrEnablingTables);
|
||||
Map<ServerName, List<HRegionInfo>> rebuildUserRegions() throws
|
||||
IOException, KeeperException, CoordinatedStateException {
|
||||
Set<TableName> enablingTables = tableStateManager.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING);
|
||||
|
||||
Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
|
||||
// Region assignment from META
|
||||
List<Result> results = MetaReader.fullScan(this.catalogTracker);
|
||||
|
@ -2584,7 +2606,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// need to enable the table if not disabled or disabling or enabling
|
||||
// this will be used in rolling restarts
|
||||
if (!disabledOrDisablingOrEnabling.contains(tableName)
|
||||
&& !getZKTable().isEnabledTable(tableName)) {
|
||||
&& !getTableStateManager().isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
} else {
|
||||
|
@ -2598,7 +2621,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// need to enable the table if not disabled or disabling or enabling
|
||||
// this will be used in rolling restarts
|
||||
if (!disabledOrDisablingOrEnabling.contains(tableName)
|
||||
&& !getZKTable().isEnabledTable(tableName)) {
|
||||
&& !getTableStateManager().isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
}
|
||||
|
@ -2615,8 +2639,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
*/
|
||||
private void recoverTableInDisablingState()
|
||||
throws KeeperException, TableNotFoundException, IOException {
|
||||
Set<TableName> disablingTables = ZKTable.getDisablingTables(watcher);
|
||||
throws KeeperException, IOException, CoordinatedStateException {
|
||||
Set<TableName> disablingTables =
|
||||
tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING);
|
||||
if (disablingTables.size() != 0) {
|
||||
for (TableName tableName : disablingTables) {
|
||||
// Recover by calling DisableTableHandler
|
||||
|
@ -2638,8 +2663,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
*/
|
||||
private void recoverTableInEnablingState()
|
||||
throws KeeperException, TableNotFoundException, IOException {
|
||||
Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
|
||||
throws KeeperException, IOException, CoordinatedStateException {
|
||||
Set<TableName> enablingTables = tableStateManager.
|
||||
getTablesInStates(ZooKeeperProtos.Table.State.ENABLING);
|
||||
if (enablingTables.size() != 0) {
|
||||
for (TableName tableName : enablingTables) {
|
||||
// Recover by calling EnableTableHandler
|
||||
|
@ -2876,7 +2902,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
} catch (KeeperException ke) {
|
||||
server.abort("Unexpected ZK exception deleting node " + hri, ke);
|
||||
}
|
||||
if (zkTable.isDisablingOrDisabledTable(hri.getTable())) {
|
||||
if (tableStateManager.isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
regionStates.regionOffline(hri);
|
||||
it.remove();
|
||||
continue;
|
||||
|
@ -2897,7 +2924,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
public void balance(final RegionPlan plan) {
|
||||
HRegionInfo hri = plan.getRegionInfo();
|
||||
TableName tableName = hri.getTable();
|
||||
if (zkTable.isDisablingOrDisabledTable(tableName)) {
|
||||
if (tableStateManager.isTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
LOG.info("Ignored moving region of disabling/disabled table "
|
||||
+ tableName);
|
||||
return;
|
||||
|
@ -2940,8 +2968,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
protected void setEnabledTable(TableName tableName) {
|
||||
try {
|
||||
this.zkTable.setEnabledTable(tableName);
|
||||
} catch (KeeperException e) {
|
||||
this.tableStateManager.setTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED);
|
||||
} catch (CoordinatedStateException e) {
|
||||
// here we can abort as it is the start up flow
|
||||
String errorMsg = "Unable to ensure that the table " + tableName
|
||||
+ " will be" + " enabled because of a ZooKeeper issue";
|
||||
|
@ -3113,7 +3142,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
+ hri_b.getRegionNameAsString() + ", on " + sn);
|
||||
|
||||
// User could disable the table before master knows the new region.
|
||||
if (zkTable.isDisablingOrDisabledTable(p.getTable())) {
|
||||
if (tableStateManager.isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
unassign(p);
|
||||
}
|
||||
}
|
||||
|
@ -3239,7 +3269,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
+ hri_b.getRegionNameAsString() + ", on " + sn);
|
||||
|
||||
// User could disable the table before master knows the new region.
|
||||
if (zkTable.isDisablingOrDisabledTable(p.getTable())) {
|
||||
if (tableStateManager.isTableState(p.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
unassign(hri_a);
|
||||
unassign(hri_b);
|
||||
}
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -68,7 +69,7 @@ import org.apache.hadoop.hbase.client.MetaScanner;
|
|||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorType;
|
||||
|
@ -99,6 +100,7 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
|||
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
|
||||
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
|
||||
|
@ -257,9 +259,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
* @throws KeeperException
|
||||
* @throws IOException
|
||||
*/
|
||||
public HMaster(final Configuration conf, ConsensusProvider consensusProvider)
|
||||
public HMaster(final Configuration conf, CoordinatedStateManager csm)
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
super(conf, consensusProvider);
|
||||
super(conf, csm);
|
||||
this.rsFatals = new MemoryBoundedLogMessageBuffer(
|
||||
conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
|
||||
|
||||
|
@ -397,9 +399,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
* Initialize all ZK based system trackers.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* @throws KeeperException
|
||||
* @throws CoordinatedStateException
|
||||
*/
|
||||
void initializeZKBasedSystemTrackers() throws IOException,
|
||||
InterruptedException, KeeperException {
|
||||
InterruptedException, KeeperException, CoordinatedStateException {
|
||||
this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
|
||||
this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
|
||||
this.loadBalancerTracker.start();
|
||||
|
@ -453,9 +457,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
* @throws KeeperException
|
||||
* @throws CoordinatedStateException
|
||||
*/
|
||||
private void finishActiveMasterInitialization(MonitoredTask status)
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
|
||||
|
||||
isActiveMaster = true;
|
||||
|
||||
|
@ -765,7 +770,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
}
|
||||
|
||||
private void enableMeta(TableName metaTableName) {
|
||||
if (!this.assignmentManager.getZKTable().isEnabledTable(metaTableName)) {
|
||||
if (!this.assignmentManager.getTableStateManager().isTableState(metaTableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
this.assignmentManager.setEnabledTable(metaTableName);
|
||||
}
|
||||
}
|
||||
|
@ -1477,8 +1483,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
if (!MetaReader.tableExists(getCatalogTracker(), tableName)) {
|
||||
throw new TableNotFoundException(tableName);
|
||||
}
|
||||
if (!getAssignmentManager().getZKTable().
|
||||
isDisabledTable(tableName)) {
|
||||
if (!getAssignmentManager().getTableStateManager().
|
||||
isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
throw new TableNotDisabledException(tableName);
|
||||
}
|
||||
}
|
||||
|
@ -1770,10 +1776,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
* @return HMaster instance.
|
||||
*/
|
||||
public static HMaster constructMaster(Class<? extends HMaster> masterClass,
|
||||
final Configuration conf, final ConsensusProvider cp) {
|
||||
final Configuration conf, final CoordinatedStateManager cp) {
|
||||
try {
|
||||
Constructor<? extends HMaster> c =
|
||||
masterClass.getConstructor(Configuration.class, ConsensusProvider.class);
|
||||
masterClass.getConstructor(Configuration.class, CoordinatedStateManager.class);
|
||||
return c.newInstance(conf, cp);
|
||||
} catch (InvocationTargetException ite) {
|
||||
Throwable target = ite.getTargetException() != null?
|
||||
|
|
|
@ -30,14 +30,14 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.ZNodeClearer;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.LocalHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.util.ServerCommandLine;
|
||||
|
@ -187,8 +187,9 @@ public class HMasterCommandLine extends ServerCommandLine {
|
|||
waitOnMasterThreads(cluster);
|
||||
} else {
|
||||
logProcessInfo(getConf());
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
HMaster master = HMaster.constructMaster(masterClass, conf, cp);
|
||||
CoordinatedStateManager csm =
|
||||
CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
HMaster master = HMaster.constructMaster(masterClass, conf, csm);
|
||||
if (master.isStopped()) {
|
||||
LOG.info("Won't bring the Master up as a shutdown is requested");
|
||||
return 1;
|
||||
|
@ -258,9 +259,9 @@ public class HMasterCommandLine extends ServerCommandLine {
|
|||
public static class LocalHMaster extends HMaster {
|
||||
private MiniZooKeeperCluster zkcluster = null;
|
||||
|
||||
public LocalHMaster(Configuration conf, ConsensusProvider consensusProvider)
|
||||
public LocalHMaster(Configuration conf, CoordinatedStateManager csm)
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
super(conf, consensusProvider);
|
||||
super(conf, csm);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -123,7 +123,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
private long resubmit_threshold;
|
||||
private long timeout;
|
||||
private long unassignedTimeout;
|
||||
private long lastNodeCreateTime = Long.MAX_VALUE;
|
||||
private long lastTaskCreateTime = Long.MAX_VALUE;
|
||||
public boolean ignoreZKDeleteForTesting = false;
|
||||
private volatile long lastRecoveringNodeCreationTime = 0;
|
||||
// When lastRecoveringNodeCreationTime is older than the following threshold, we'll check
|
||||
|
@ -247,7 +247,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
* if there was an error while splitting any log file
|
||||
* @return cumulative size of the logfiles split
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
*/
|
||||
public long splitLogDistributed(final Path logDir) throws IOException {
|
||||
List<Path> logDirs = new ArrayList<Path>();
|
||||
|
@ -348,7 +348,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
} catch (IOException ioe) {
|
||||
FileStatus[] files = fs.listStatus(logDir);
|
||||
if (files != null && files.length > 0) {
|
||||
LOG.warn("returning success without actually splitting and " +
|
||||
LOG.warn("returning success without actually splitting and " +
|
||||
"deleting all the log files in path " + logDir);
|
||||
} else {
|
||||
LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
|
||||
|
@ -366,7 +366,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
|
||||
/**
|
||||
* Add a task entry to splitlog znode if it is not already there.
|
||||
*
|
||||
*
|
||||
* @param taskname the path of the log to be split
|
||||
* @param batch the batch this task belongs to
|
||||
* @return true if a new entry is created, false if it is already there.
|
||||
|
@ -376,6 +376,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// This is a znode path under the splitlog dir with the rest of the path made up of an
|
||||
// url encoding of the passed in log to split.
|
||||
String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
|
||||
lastTaskCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
Task oldtask = createTaskIfAbsent(path, batch);
|
||||
if (oldtask == null) {
|
||||
// publish the task in zk
|
||||
|
@ -505,7 +506,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
if(isMetaRecovery != null) {
|
||||
if ((isMetaRecovery && !region.equalsIgnoreCase(metaEncodeRegionName))
|
||||
|| (!isMetaRecovery && region.equalsIgnoreCase(metaEncodeRegionName))) {
|
||||
// skip non-meta regions when recovering the meta region or
|
||||
// skip non-meta regions when recovering the meta region or
|
||||
// skip the meta region when recovering user regions
|
||||
continue;
|
||||
}
|
||||
|
@ -689,7 +690,6 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private void createNodeSuccess(String path) {
|
||||
lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
LOG.debug("put up splitlog task at znode " + path);
|
||||
getDataSetWatch(path, zkretries);
|
||||
}
|
||||
|
@ -940,7 +940,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
/**
|
||||
* signal the workers that a task was resubmitted by creating the
|
||||
* RESCAN node.
|
||||
* @throws KeeperException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private void createRescanNode(long retries) {
|
||||
// The RESCAN node will be deleted almost immediately by the
|
||||
|
@ -950,6 +950,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// might miss the watch-trigger that creation of RESCAN node provides.
|
||||
// Since the TimeoutMonitor will keep resubmitting UNASSIGNED tasks
|
||||
// therefore this behavior is safe.
|
||||
lastTaskCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
SplitLogTask slt = new SplitLogTask.Done(this.serverName);
|
||||
this.watcher.getRecoverableZooKeeper().getZooKeeper().
|
||||
create(ZKSplitLog.getRescanNode(watcher), slt.toByteArray(),
|
||||
|
@ -958,7 +959,6 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private void createRescanSuccess(String path) {
|
||||
lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
SplitLogCounters.tot_mgr_rescan.incrementAndGet();
|
||||
getDataSetWatch(path, zkretries);
|
||||
}
|
||||
|
@ -1217,7 +1217,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// when SplitLogWorker recovers a region by directly replaying unflushed WAL edits,
|
||||
// last flushed sequence Id changes when newly assigned RS flushes writes to the region.
|
||||
// If the newly assigned RS fails again(a chained RS failures scenario), the last flushed
|
||||
// sequence Id name space (sequence Id only valid for a particular RS instance), changes
|
||||
// sequence Id name space (sequence Id only valid for a particular RS instance), changes
|
||||
// when different newly assigned RS flushes the region.
|
||||
// Therefore, in this mode we need to fetch last sequence Ids from ZK where we keep history of
|
||||
// last flushed sequence Id for each failed RS instance.
|
||||
|
@ -1410,7 +1410,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// master should spawn both a manager and a worker thread to guarantee
|
||||
// that there is always one worker in the system
|
||||
if (tot > 0 && !found_assigned_task &&
|
||||
((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
|
||||
((EnvironmentEdgeManager.currentTimeMillis() - lastTaskCreateTime) >
|
||||
unassignedTimeout)) {
|
||||
for (Map.Entry<String, Task> e : tasks.entrySet()) {
|
||||
String path = e.getKey();
|
||||
|
@ -1518,15 +1518,10 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
if (rc == KeeperException.Code.NONODE.intValue()) {
|
||||
SplitLogCounters.tot_mgr_get_data_nonode.incrementAndGet();
|
||||
// The task znode has been deleted. Must be some pending delete
|
||||
// that deleted the task. Assume success because a task-znode is
|
||||
// is only deleted after TaskFinisher is successful.
|
||||
LOG.warn("task znode " + path + " vanished.");
|
||||
try {
|
||||
getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Deserialization problem", e);
|
||||
}
|
||||
LOG.warn("task znode " + path + " vanished or not created yet.");
|
||||
// ignore since we should not end up in a case where there is in-memory task,
|
||||
// but no znode. The only case is between the time task is created in-memory
|
||||
// and the znode is created. See HBASE-11217.
|
||||
return;
|
||||
}
|
||||
Long retry_count = (Long) ctx;
|
||||
|
@ -1675,7 +1670,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
TerminationStatus(String msg) {
|
||||
statusMsg = msg;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return statusMsg;
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.executor.EventHandler;
|
|||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
/**
|
||||
* Handles CLOSED region event on Master.
|
||||
|
@ -91,8 +92,8 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
public void process() {
|
||||
LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
|
||||
// Check if this table is being disabled or not
|
||||
if (this.assignmentManager.getZKTable().
|
||||
isDisablingOrDisabledTable(this.regionInfo.getTable())) {
|
||||
if (this.assignmentManager.getTableStateManager().isTableState(this.regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
assignmentManager.offlineDisabledRegion(regionInfo);
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -46,10 +47,10 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Handler to create a table.
|
||||
|
@ -130,10 +131,13 @@ public class CreateTableHandler extends EventHandler {
|
|||
// We could have cleared the hbase.rootdir and not zk. How can we detect this case?
|
||||
// Having to clean zk AND hdfs is awkward.
|
||||
try {
|
||||
if (!assignmentManager.getZKTable().checkAndSetEnablingTable(tableName)) {
|
||||
if (!assignmentManager.getTableStateManager().setTableStateIfNotInStates(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
throw new TableExistsException(tableName);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
" enabling because of a ZooKeeper issue", e);
|
||||
}
|
||||
|
@ -146,8 +150,9 @@ public class CreateTableHandler extends EventHandler {
|
|||
// again with the same Active master
|
||||
// It will block the creation saying TableAlreadyExists.
|
||||
try {
|
||||
assignmentManager.getZKTable().removeEnablingTable(tableName, false);
|
||||
} catch (KeeperException e) {
|
||||
assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING, false);
|
||||
} catch (CoordinatedStateException e) {
|
||||
// Keeper exception should not happen here
|
||||
LOG.error("Got a keeper exception while removing the ENABLING table znode "
|
||||
+ tableName, e);
|
||||
|
@ -211,7 +216,7 @@ public class CreateTableHandler extends EventHandler {
|
|||
* - Update ZooKeeper with the enabled state
|
||||
*/
|
||||
private void handleCreateTable(TableName tableName)
|
||||
throws IOException, KeeperException {
|
||||
throws IOException, CoordinatedStateException {
|
||||
Path tempdir = fileSystemManager.getTempDir();
|
||||
FileSystem fs = fileSystemManager.getFileSystem();
|
||||
|
||||
|
@ -239,8 +244,9 @@ public class CreateTableHandler extends EventHandler {
|
|||
|
||||
// 6. Set table enabled flag up in zk.
|
||||
try {
|
||||
assignmentManager.getZKTable().setEnabledTable(tableName);
|
||||
} catch (KeeperException e) {
|
||||
assignmentManager.getTableStateManager().setTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED);
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Unable to ensure that " + tableName + " will be" +
|
||||
" enabled because of a ZooKeeper issue", e);
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class DeleteTableHandler extends TableEventHandler {
|
||||
|
@ -62,7 +62,7 @@ public class DeleteTableHandler extends TableEventHandler {
|
|||
}
|
||||
|
||||
protected void waitRegionInTransition(final List<HRegionInfo> regions)
|
||||
throws IOException, KeeperException {
|
||||
throws IOException, CoordinatedStateException {
|
||||
AssignmentManager am = this.masterServices.getAssignmentManager();
|
||||
RegionStates states = am.getRegionStates();
|
||||
long waitTime = server.getConfiguration().
|
||||
|
@ -93,7 +93,7 @@ public class DeleteTableHandler extends TableEventHandler {
|
|||
|
||||
@Override
|
||||
protected void handleTableOperation(List<HRegionInfo> regions)
|
||||
throws IOException, KeeperException {
|
||||
throws IOException, CoordinatedStateException {
|
||||
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
|
||||
if (cpHost != null) {
|
||||
cpHost.preDeleteTableHandler(this.tableName);
|
||||
|
@ -118,7 +118,7 @@ public class DeleteTableHandler extends TableEventHandler {
|
|||
|
||||
// 5. If entry for this table in zk, and up in AssignmentManager, remove it.
|
||||
LOG.debug("Marking '" + tableName + "' as deleted.");
|
||||
am.getZKTable().setDeletedTable(tableName);
|
||||
am.getTableStateManager().setDeletedTable(tableName);
|
||||
}
|
||||
|
||||
if (cpHost != null) {
|
||||
|
@ -130,7 +130,7 @@ public class DeleteTableHandler extends TableEventHandler {
|
|||
* Removes the table from .META. and archives the HDFS files.
|
||||
*/
|
||||
protected void removeTableData(final List<HRegionInfo> regions)
|
||||
throws IOException, KeeperException {
|
||||
throws IOException, CoordinatedStateException {
|
||||
// 1. Remove regions from META
|
||||
LOG.debug("Deleting regions from META");
|
||||
MetaEditor.deleteRegions(this.server.getCatalogTracker(), regions);
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
|
@ -43,7 +44,7 @@ import org.apache.hadoop.hbase.master.RegionStates;
|
|||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.htrace.Trace;
|
||||
|
||||
/**
|
||||
|
@ -94,14 +95,15 @@ public class DisableTableHandler extends EventHandler {
|
|||
//TODO: reevaluate this since we have table locks now
|
||||
if (!skipTableStateCheck) {
|
||||
try {
|
||||
if (!this.assignmentManager.getZKTable().checkEnabledAndSetDisablingTable
|
||||
(this.tableName)) {
|
||||
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
|
||||
this.tableName, ZooKeeperProtos.Table.State.DISABLING,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't enabled; skipping disable");
|
||||
throw new TableNotEnabledException(this.tableName);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
" disabling because of a ZooKeeper issue", e);
|
||||
" disabling because of a coordination engine issue", e);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
@ -139,7 +141,7 @@ public class DisableTableHandler extends EventHandler {
|
|||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error trying to disable table " + this.tableName, e);
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
LOG.error("Error trying to disable table " + this.tableName, e);
|
||||
} finally {
|
||||
releaseTableLock();
|
||||
|
@ -156,9 +158,10 @@ public class DisableTableHandler extends EventHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private void handleDisableTable() throws IOException, KeeperException {
|
||||
private void handleDisableTable() throws IOException, CoordinatedStateException {
|
||||
// Set table disabling flag up in zk.
|
||||
this.assignmentManager.getZKTable().setDisablingTable(this.tableName);
|
||||
this.assignmentManager.getTableStateManager().setTableState(this.tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
boolean done = false;
|
||||
while (true) {
|
||||
// Get list of online regions that are of this table. Regions that are
|
||||
|
@ -186,7 +189,8 @@ public class DisableTableHandler extends EventHandler {
|
|||
}
|
||||
}
|
||||
// Flip the table to disabled if success.
|
||||
if (done) this.assignmentManager.getZKTable().setDisabledTable(this.tableName);
|
||||
if (done) this.assignmentManager.getTableStateManager().setTableState(this.tableName,
|
||||
ZooKeeperProtos.Table.State.DISABLED);
|
||||
LOG.info("Disabled table is done=" + done);
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
|
@ -45,8 +46,8 @@ import org.apache.hadoop.hbase.master.RegionStates;
|
|||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Handler to run enable of a table.
|
||||
|
@ -88,9 +89,10 @@ public class EnableTableHandler extends EventHandler {
|
|||
throw new TableNotFoundException(tableName);
|
||||
}
|
||||
try {
|
||||
this.assignmentManager.getZKTable().removeEnablingTable(tableName, true);
|
||||
this.assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING, true);
|
||||
throw new TableNotFoundException(tableName);
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
// TODO : Use HBCK to clear such nodes
|
||||
LOG.warn("Failed to delete the ENABLING node for the table " + tableName
|
||||
+ ". The table will remain unusable. Run HBCK to manually fix the problem.");
|
||||
|
@ -103,14 +105,15 @@ public class EnableTableHandler extends EventHandler {
|
|||
// DISABLED or ENABLED.
|
||||
if (!skipTableStateCheck) {
|
||||
try {
|
||||
if (!this.assignmentManager.getZKTable().checkDisabledAndSetEnablingTable
|
||||
(this.tableName)) {
|
||||
if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
|
||||
this.tableName, ZooKeeperProtos.Table.State.ENABLING,
|
||||
ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
LOG.info("Table " + tableName + " isn't disabled; skipping enable");
|
||||
throw new TableNotDisabledException(this.tableName);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
" enabling because of a ZooKeeper issue", e);
|
||||
" enabling because of a coordination engine issue", e);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
@ -147,7 +150,7 @@ public class EnableTableHandler extends EventHandler {
|
|||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error trying to enable the table " + this.tableName, e);
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
LOG.error("Error trying to enable the table " + this.tableName, e);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Error trying to enable the table " + this.tableName, e);
|
||||
|
@ -166,12 +169,14 @@ public class EnableTableHandler extends EventHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private void handleEnableTable() throws IOException, KeeperException, InterruptedException {
|
||||
private void handleEnableTable() throws IOException, CoordinatedStateException,
|
||||
InterruptedException {
|
||||
// I could check table is disabling and if so, not enable but require
|
||||
// that user first finish disabling but that might be obnoxious.
|
||||
|
||||
// Set table enabling flag up in zk.
|
||||
this.assignmentManager.getZKTable().setEnablingTable(this.tableName);
|
||||
this.assignmentManager.getTableStateManager().setTableState(this.tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLING);
|
||||
boolean done = false;
|
||||
ServerManager serverManager = ((HMaster)this.server).getServerManager();
|
||||
// Get the regions of this table. We're done when all listed
|
||||
|
@ -206,8 +211,8 @@ public class EnableTableHandler extends EventHandler {
|
|||
}
|
||||
if (done) {
|
||||
// Flip the table to enabled.
|
||||
this.assignmentManager.getZKTable().setEnabledTable(
|
||||
this.tableName);
|
||||
this.assignmentManager.getTableStateManager().setTableState(
|
||||
this.tableName, ZooKeeperProtos.Table.State.ENABLED);
|
||||
LOG.info("Table '" + this.tableName
|
||||
+ "' was successfully enabled. Status: done=" + done);
|
||||
} else {
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.executor.EventHandler;
|
|||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -113,7 +114,8 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
" because regions is NOT in RIT -- presuming this is because it SPLIT");
|
||||
}
|
||||
if (!openedNodeDeleted) {
|
||||
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(regionInfo.getTable())) {
|
||||
if (this.assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
debugLog(regionInfo, "Opened region "
|
||||
+ regionInfo.getShortNameToLog() + " but "
|
||||
+ "this table is disabled, triggering close of region");
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.master.RegionState;
|
|||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -257,7 +258,8 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
toAssignRegions.add(hri);
|
||||
} else if (rit != null) {
|
||||
if (rit.isPendingCloseOrClosing()
|
||||
&& am.getZKTable().isDisablingOrDisabledTable(hri.getTable())) {
|
||||
&& am.getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
|
||||
// If the table was partially disabled and the RS went down, we should clear the RIT
|
||||
// and remove the node for the region.
|
||||
// The rit that we use may be stale in case the table was in DISABLING state
|
||||
|
@ -334,14 +336,15 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
public static boolean processDeadRegion(HRegionInfo hri, Result result,
|
||||
AssignmentManager assignmentManager, CatalogTracker catalogTracker)
|
||||
throws IOException {
|
||||
boolean tablePresent = assignmentManager.getZKTable().isTablePresent(hri.getTable());
|
||||
boolean tablePresent = assignmentManager.getTableStateManager().isTablePresent(hri.getTable());
|
||||
if (!tablePresent) {
|
||||
LOG.info("The table " + hri.getTable()
|
||||
+ " was deleted. Hence not proceeding.");
|
||||
return false;
|
||||
}
|
||||
// If table is not disabled but the region is offlined,
|
||||
boolean disabled = assignmentManager.getZKTable().isDisabledTable(hri.getTable());
|
||||
boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLED);
|
||||
if (disabled){
|
||||
LOG.info("The table " + hri.getTable()
|
||||
+ " was disabled. Hence not proceeding.");
|
||||
|
@ -353,7 +356,8 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
//to the dead server. We don't have to do anything.
|
||||
return false;
|
||||
}
|
||||
boolean disabling = assignmentManager.getZKTable().isDisablingTable(hri.getTable());
|
||||
boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
|
||||
ZooKeeperProtos.Table.State.DISABLING);
|
||||
if (disabling) {
|
||||
LOG.info("The table " + hri.getTable()
|
||||
+ " is disabled. Hence not assigning region" + hri.getEncodedName());
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.TreeMap;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -44,8 +45,8 @@ import org.apache.hadoop.hbase.executor.EventType;
|
|||
import org.apache.hadoop.hbase.master.BulkReOpen;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
@ -128,8 +129,8 @@ public abstract class TableEventHandler extends EventHandler {
|
|||
tableName);
|
||||
handleTableOperation(hris);
|
||||
if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
|
||||
getAssignmentManager().getZKTable().
|
||||
isEnabledTable(tableName)) {
|
||||
getAssignmentManager().getTableStateManager().isTableState(
|
||||
tableName, ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
if (reOpenAllRegions(hris)) {
|
||||
LOG.info("Completed table operation " + eventType + " on table " +
|
||||
tableName);
|
||||
|
@ -141,7 +142,7 @@ public abstract class TableEventHandler extends EventHandler {
|
|||
} catch (IOException e) {
|
||||
LOG.error("Error manipulating table " + tableName, e);
|
||||
completed(e);
|
||||
} catch (KeeperException e) {
|
||||
} catch (CoordinatedStateException e) {
|
||||
LOG.error("Error manipulating table " + tableName, e);
|
||||
completed(e);
|
||||
} finally {
|
||||
|
@ -249,5 +250,5 @@ public abstract class TableEventHandler extends EventHandler {
|
|||
}
|
||||
|
||||
protected abstract void handleTableOperation(List<HRegionInfo> regions)
|
||||
throws IOException, KeeperException;
|
||||
throws IOException, CoordinatedStateException;
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.master.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -27,25 +26,20 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
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.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionStates;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Truncate the table by removing META and the HDFS files and recreating it.
|
||||
|
@ -67,7 +61,7 @@ public class TruncateTableHandler extends DeleteTableHandler {
|
|||
|
||||
@Override
|
||||
protected void handleTableOperation(List<HRegionInfo> regions)
|
||||
throws IOException, KeeperException {
|
||||
throws IOException, CoordinatedStateException {
|
||||
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
|
||||
if (cpHost != null) {
|
||||
cpHost.preTruncateTableHandler(this.tableName);
|
||||
|
@ -137,8 +131,9 @@ public class TruncateTableHandler extends DeleteTableHandler {
|
|||
|
||||
// 6. Set table enabled flag up in zk.
|
||||
try {
|
||||
assignmentManager.getZKTable().setEnabledTable(tableName);
|
||||
} catch (KeeperException e) {
|
||||
assignmentManager.getTableStateManager().setTableState(tableName,
|
||||
ZooKeeperProtos.Table.State.ENABLED);
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Unable to ensure that " + tableName + " will be" +
|
||||
" enabled because of a ZooKeeper issue", e);
|
||||
}
|
||||
|
|
|
@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
|
||||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
|
||||
|
@ -556,13 +557,15 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
// if the table is enabled, then have the RS run actually the snapshot work
|
||||
TableName snapshotTable = TableName.valueOf(snapshot.getTable());
|
||||
AssignmentManager assignmentMgr = master.getAssignmentManager();
|
||||
if (assignmentMgr.getZKTable().isEnabledTable(snapshotTable)) {
|
||||
if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
|
||||
ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
LOG.debug("Table enabled, starting distributed snapshot.");
|
||||
snapshotEnabledTable(snapshot);
|
||||
LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
|
||||
}
|
||||
// For disabled table, snapshot is created by the master
|
||||
else if (assignmentMgr.getZKTable().isDisabledTable(snapshotTable)) {
|
||||
else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
|
||||
ZooKeeperProtos.Table.State.DISABLED)) {
|
||||
LOG.debug("Table is disabled, running snapshot entirely on master.");
|
||||
snapshotDisabledTable(snapshot);
|
||||
LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
|
||||
|
@ -692,8 +695,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
|
||||
// Execute the restore/clone operation
|
||||
if (MetaReader.tableExists(master.getCatalogTracker(), tableName)) {
|
||||
if (master.getAssignmentManager().getZKTable().isEnabledTable(
|
||||
TableName.valueOf(fsSnapshot.getTable()))) {
|
||||
if (master.getAssignmentManager().getTableStateManager().isTableState(
|
||||
TableName.valueOf(fsSnapshot.getTable()), ZooKeeperProtos.Table.State.ENABLED)) {
|
||||
throw new UnsupportedOperationException("Table '" +
|
||||
TableName.valueOf(fsSnapshot.getTable()) + "' must be disabled in order to " +
|
||||
"perform a restore operation" +
|
||||
|
|
|
@ -58,6 +58,8 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -75,8 +77,6 @@ import org.apache.hadoop.hbase.catalog.MetaEditor;
|
|||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
|
||||
|
@ -392,7 +392,7 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
protected final RSRpcServices rpcServices;
|
||||
|
||||
protected ConsensusProvider consensusProvider;
|
||||
protected CoordinatedStateManager csm;
|
||||
|
||||
/**
|
||||
* Starts a HRegionServer at the default location.
|
||||
|
@ -401,17 +401,17 @@ public class HRegionServer extends HasThread implements
|
|||
* @throws InterruptedException
|
||||
*/
|
||||
public HRegionServer(Configuration conf) throws IOException, InterruptedException {
|
||||
this(conf, ConsensusProviderFactory.getConsensusProvider(conf));
|
||||
this(conf, CoordinatedStateManagerFactory.getCoordinatedStateManager(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts a HRegionServer at the default location
|
||||
* @param conf
|
||||
* @param consensusProvider implementation of ConsensusProvider to be used
|
||||
* @param csm implementation of CoordinatedStateManager to be used
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public HRegionServer(Configuration conf, ConsensusProvider consensusProvider)
|
||||
public HRegionServer(Configuration conf, CoordinatedStateManager csm)
|
||||
throws IOException, InterruptedException {
|
||||
this.fsOk = true;
|
||||
this.conf = conf;
|
||||
|
@ -483,9 +483,9 @@ public class HRegionServer extends HasThread implements
|
|||
zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" +
|
||||
rpcServices.isa.getPort(), this, canCreateBaseZNode());
|
||||
|
||||
this.consensusProvider = consensusProvider;
|
||||
this.consensusProvider.initialize(this);
|
||||
this.consensusProvider.start();
|
||||
this.csm = csm;
|
||||
this.csm.initialize(this);
|
||||
this.csm.start();
|
||||
|
||||
tableLockManager = TableLockManager.createTableLockManager(
|
||||
conf, zooKeeper, serverName);
|
||||
|
@ -2155,8 +2155,8 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
return consensusProvider;
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return csm;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2253,10 +2253,10 @@ public class HRegionServer extends HasThread implements
|
|||
*/
|
||||
public static HRegionServer constructRegionServer(
|
||||
Class<? extends HRegionServer> regionServerClass,
|
||||
final Configuration conf2, ConsensusProvider cp) {
|
||||
final Configuration conf2, CoordinatedStateManager cp) {
|
||||
try {
|
||||
Constructor<? extends HRegionServer> c = regionServerClass
|
||||
.getConstructor(Configuration.class, ConsensusProvider.class);
|
||||
.getConstructor(Configuration.class, CoordinatedStateManager.class);
|
||||
return c.newInstance(conf2, cp);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Failed construction of " + "Regionserver: "
|
||||
|
|
|
@ -23,10 +23,10 @@ import org.apache.commons.logging.LogFactory;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.LocalHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.util.ServerCommandLine;
|
||||
|
||||
/**
|
||||
|
@ -52,7 +52,7 @@ public class HRegionServerCommandLine extends ServerCommandLine {
|
|||
|
||||
private int start() throws Exception {
|
||||
Configuration conf = getConf();
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
try {
|
||||
// If 'local', don't start a region server here. Defer to
|
||||
// LocalHBaseCluster. It manages 'local' clusters.
|
||||
|
|
|
@ -242,7 +242,7 @@ class MetricsRegionServerWrapperImpl
|
|||
}
|
||||
|
||||
@Override
|
||||
public int getBlockCacheHitPercent() {
|
||||
public double getBlockCacheHitPercent() {
|
||||
if (this.cacheStats == null) {
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
|
@ -123,7 +122,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
}
|
||||
|
||||
public SplitLogWorker(final ZooKeeperWatcher watcher, final Configuration conf,
|
||||
RegionServerServices server, final LastSequenceId sequenceIdChecker) {
|
||||
final RegionServerServices server, final LastSequenceId sequenceIdChecker) {
|
||||
this(watcher, conf, server, new TaskExecutor() {
|
||||
@Override
|
||||
public Status exec(String filename, CancelableProgressable p) {
|
||||
|
@ -141,7 +140,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
// encountered a bad non-retry-able persistent error.
|
||||
try {
|
||||
if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)),
|
||||
fs, conf, p, sequenceIdChecker, watcher)) {
|
||||
fs, conf, p, sequenceIdChecker, watcher, server.getCoordinatedStateManager())) {
|
||||
return Status.PREEMPTED;
|
||||
}
|
||||
} catch (InterruptedIOException iioe) {
|
||||
|
|
|
@ -56,6 +56,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -66,6 +68,7 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
|
@ -89,6 +92,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -104,10 +108,8 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -135,6 +137,7 @@ public class HLogSplitter {
|
|||
private Set<TableName> disablingOrDisabledTables =
|
||||
new HashSet<TableName>();
|
||||
private ZooKeeperWatcher watcher;
|
||||
private CoordinatedStateManager csm;
|
||||
|
||||
// If an exception is thrown by one of the other threads, it will be
|
||||
// stored here.
|
||||
|
@ -168,7 +171,8 @@ public class HLogSplitter {
|
|||
private final int minBatchSize;
|
||||
|
||||
HLogSplitter(Configuration conf, Path rootDir,
|
||||
FileSystem fs, LastSequenceId idChecker, ZooKeeperWatcher zkw) {
|
||||
FileSystem fs, LastSequenceId idChecker, ZooKeeperWatcher zkw,
|
||||
CoordinatedStateManager csm) {
|
||||
this.conf = HBaseConfiguration.create(conf);
|
||||
String codecClassName = conf
|
||||
.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
|
||||
|
@ -177,6 +181,7 @@ public class HLogSplitter {
|
|||
this.fs = fs;
|
||||
this.sequenceIdChecker = idChecker;
|
||||
this.watcher = zkw;
|
||||
this.csm = csm;
|
||||
|
||||
entryBuffers = new EntryBuffers(
|
||||
this.conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
|
||||
|
@ -188,7 +193,7 @@ public class HLogSplitter {
|
|||
this.distributedLogReplay = HLogSplitter.isDistributedLogReplay(this.conf);
|
||||
|
||||
this.numWriterThreads = this.conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
|
||||
if (zkw != null && this.distributedLogReplay) {
|
||||
if (zkw != null && csm != null && this.distributedLogReplay) {
|
||||
outputSink = new LogReplayOutputSink(numWriterThreads);
|
||||
} else {
|
||||
if (this.distributedLogReplay) {
|
||||
|
@ -219,8 +224,9 @@ public class HLogSplitter {
|
|||
*/
|
||||
public static boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs,
|
||||
Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
|
||||
ZooKeeperWatcher zkw) throws IOException {
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, fs, idChecker, zkw);
|
||||
ZooKeeperWatcher zkw, CoordinatedStateManager cp) throws IOException {
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, fs, idChecker, zkw,
|
||||
cp);
|
||||
return s.splitLogFile(logfile, reporter);
|
||||
}
|
||||
|
||||
|
@ -234,7 +240,7 @@ public class HLogSplitter {
|
|||
List<Path> splits = new ArrayList<Path>();
|
||||
if (logfiles != null && logfiles.length > 0) {
|
||||
for (FileStatus logfile: logfiles) {
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, fs, null, null);
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, fs, null, null, null);
|
||||
if (s.splitLogFile(logfile, null)) {
|
||||
finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
|
||||
if (s.outputSink.splits != null) {
|
||||
|
@ -288,10 +294,12 @@ public class HLogSplitter {
|
|||
LOG.warn("Nothing to split in log file " + logPath);
|
||||
return true;
|
||||
}
|
||||
if(watcher != null) {
|
||||
if(watcher != null && csm != null) {
|
||||
try {
|
||||
disablingOrDisabledTables = ZKTable.getDisabledOrDisablingTables(watcher);
|
||||
} catch (KeeperException e) {
|
||||
TableStateManager tsm = csm.getTableStateManager();
|
||||
disablingOrDisabledTables = tsm.getTablesInStates(
|
||||
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new IOException("Can't get disabling/disabled tables", e);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.HConnection;
|
|||
import org.apache.hadoop.hbase.client.RegionServerCallable;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
|
@ -69,6 +70,7 @@ public class WALEditsReplaySink {
|
|||
private final AtomicLong totalReplayedEdits = new AtomicLong();
|
||||
private final boolean skipErrors;
|
||||
private final int replayTimeout;
|
||||
private RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
/**
|
||||
* Create a sink for WAL log entries replay
|
||||
|
@ -87,6 +89,7 @@ public class WALEditsReplaySink {
|
|||
HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS);
|
||||
// a single replay operation time out and default is 60 seconds
|
||||
this.replayTimeout = conf.getInt("hbase.regionserver.logreplay.timeout", 60000);
|
||||
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -211,7 +214,7 @@ public class WALEditsReplaySink {
|
|||
|
||||
Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
|
||||
ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond());
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
|
||||
try {
|
||||
remoteSvr.replay(controller, p.getFirst());
|
||||
} catch (ServiceException se) {
|
||||
|
|
|
@ -29,9 +29,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
|
@ -146,7 +144,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -29,12 +29,12 @@ import javax.security.auth.callback.PasswordCallback;
|
|||
import javax.security.auth.callback.UnsupportedCallbackException;
|
||||
import javax.security.sasl.AuthorizeCallback;
|
||||
import javax.security.sasl.RealmCallback;
|
||||
import javax.security.sasl.Sasl;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
|
@ -46,36 +46,9 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|||
public class HBaseSaslRpcServer {
|
||||
public static final Log LOG = LogFactory.getLog(HBaseSaslRpcServer.class);
|
||||
|
||||
public static enum QualityOfProtection {
|
||||
AUTHENTICATION("auth"),
|
||||
INTEGRITY("auth-int"),
|
||||
PRIVACY("auth-conf");
|
||||
|
||||
public final String saslQop;
|
||||
|
||||
private QualityOfProtection(String saslQop) {
|
||||
this.saslQop = saslQop;
|
||||
}
|
||||
|
||||
public String getSaslQop() {
|
||||
return saslQop;
|
||||
}
|
||||
}
|
||||
|
||||
public static void init(Configuration conf) {
|
||||
QualityOfProtection saslQOP = QualityOfProtection.AUTHENTICATION;
|
||||
String rpcProtection = conf.get("hbase.rpc.protection",
|
||||
QualityOfProtection.AUTHENTICATION.name().toLowerCase());
|
||||
if (QualityOfProtection.INTEGRITY.name().toLowerCase()
|
||||
.equals(rpcProtection)) {
|
||||
saslQOP = QualityOfProtection.INTEGRITY;
|
||||
} else if (QualityOfProtection.PRIVACY.name().toLowerCase().equals(
|
||||
rpcProtection)) {
|
||||
saslQOP = QualityOfProtection.PRIVACY;
|
||||
}
|
||||
|
||||
SaslUtil.SASL_PROPS.put(Sasl.QOP, saslQOP.getSaslQop());
|
||||
SaslUtil.SASL_PROPS.put(Sasl.SERVER_AUTH, "true");
|
||||
SaslUtil.initSaslProperties(conf.get("hbase.rpc.protection",
|
||||
QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
|
||||
}
|
||||
|
||||
public static <T extends TokenIdentifier> T getIdentifier(String id,
|
||||
|
|
|
@ -236,7 +236,11 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
done.run(null);
|
||||
return;
|
||||
}
|
||||
ugi.addToken(targetfsDelegationToken.getUserToken());
|
||||
Token<?> targetFsToken = targetfsDelegationToken.getUserToken();
|
||||
if (targetFsToken != null && (userToken == null
|
||||
|| !targetFsToken.getService().equals(userToken.getService()))) {
|
||||
ugi.addToken(targetFsToken);
|
||||
}
|
||||
|
||||
loaded = ugi.doAs(new PrivilegedAction<Boolean>() {
|
||||
@Override
|
||||
|
|
|
@ -103,7 +103,7 @@ import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
|
|||
import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
|
||||
import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
|
@ -1385,7 +1385,7 @@ public class HBaseFsck extends Configured {
|
|||
ZooKeeperWatcher zkw = createZooKeeperWatcher();
|
||||
try {
|
||||
for (TableName tableName :
|
||||
ZKTableReadOnly.getDisabledOrDisablingTables(zkw)) {
|
||||
ZKTableStateClientSideReader.getDisabledOrDisablingTables(zkw)) {
|
||||
disabledTables.add(tableName);
|
||||
}
|
||||
} catch (KeeperException ke) {
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
@ -81,14 +81,14 @@ public class JVMClusterUtil {
|
|||
* @return Region server added.
|
||||
*/
|
||||
public static JVMClusterUtil.RegionServerThread createRegionServerThread(
|
||||
final Configuration c, ConsensusProvider cp, final Class<? extends HRegionServer> hrsc,
|
||||
final Configuration c, CoordinatedStateManager cp, final Class<? extends HRegionServer> hrsc,
|
||||
final int index)
|
||||
throws IOException {
|
||||
HRegionServer server;
|
||||
try {
|
||||
|
||||
Constructor<? extends HRegionServer> ctor = hrsc.getConstructor(Configuration.class,
|
||||
ConsensusProvider.class);
|
||||
CoordinatedStateManager.class);
|
||||
ctor.setAccessible(true);
|
||||
server = ctor.newInstance(c, cp);
|
||||
} catch (InvocationTargetException ite) {
|
||||
|
@ -133,12 +133,12 @@ public class JVMClusterUtil {
|
|||
* @return Master added.
|
||||
*/
|
||||
public static JVMClusterUtil.MasterThread createMasterThread(
|
||||
final Configuration c, ConsensusProvider cp, final Class<? extends HMaster> hmc,
|
||||
final Configuration c, CoordinatedStateManager cp, final Class<? extends HMaster> hmc,
|
||||
final int index)
|
||||
throws IOException {
|
||||
HMaster server;
|
||||
try {
|
||||
server = hmc.getConstructor(Configuration.class, ConsensusProvider.class).
|
||||
server = hmc.getConstructor(Configuration.class, CoordinatedStateManager.class).
|
||||
newInstance(c, cp);
|
||||
} catch (InvocationTargetException ite) {
|
||||
Throwable target = ite.getTargetException();
|
||||
|
|
|
@ -1,25 +0,0 @@
|
|||
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
|
||||
<html>
|
||||
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<head />
|
||||
<body bgcolor="white">
|
||||
Provides the hbase data+index+metadata file.
|
||||
</body>
|
||||
</html>
|
|
@ -132,11 +132,12 @@ import org.apache.zookeeper.ZooKeeper.States;
|
|||
public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
||||
private MiniZooKeeperCluster zkCluster = null;
|
||||
|
||||
public static final String REGIONS_PER_SERVER_KEY = "hbase.test.regions-per-server";
|
||||
/**
|
||||
* The default number of regions per regionserver when creating a pre-split
|
||||
* table.
|
||||
*/
|
||||
private static int DEFAULT_REGIONS_PER_SERVER = 5;
|
||||
public static final int DEFAULT_REGIONS_PER_SERVER = 5;
|
||||
|
||||
/**
|
||||
* Set if we were passed a zkCluster. If so, we won't shutdown zk as
|
||||
|
@ -3196,10 +3197,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
throw new IllegalStateException("No live regionservers");
|
||||
}
|
||||
|
||||
totalNumberOfRegions = numberOfServers * DEFAULT_REGIONS_PER_SERVER;
|
||||
int regionsPerServer = conf.getInt(REGIONS_PER_SERVER_KEY, DEFAULT_REGIONS_PER_SERVER);
|
||||
totalNumberOfRegions = numberOfServers * regionsPerServer;
|
||||
LOG.info("Number of live regionservers: " + numberOfServers + ", " +
|
||||
"pre-splitting table into " + totalNumberOfRegions + " regions " +
|
||||
"(default regions per server: " + DEFAULT_REGIONS_PER_SERVER + ")");
|
||||
"(default regions per server: " + regionsPerServer + ")");
|
||||
|
||||
byte[][] splits = new RegionSplitter.HexStringSplit().split(
|
||||
totalNumberOfRegions);
|
||||
|
|
|
@ -110,7 +110,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
private User user = null;
|
||||
public static boolean TEST_SKIP_CLOSE = false;
|
||||
|
||||
public MiniHBaseClusterRegionServer(Configuration conf, ConsensusProvider cp)
|
||||
public MiniHBaseClusterRegionServer(Configuration conf, CoordinatedStateManager cp)
|
||||
throws IOException, InterruptedException {
|
||||
super(conf, cp);
|
||||
this.user = User.getCurrent();
|
||||
|
|
|
@ -136,7 +136,7 @@ class MockRegionServerServices implements RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.consensus.ZkCoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorType;
|
||||
|
@ -108,6 +109,12 @@ public class TestDrainingServer {
|
|||
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
|
||||
|
||||
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(server);
|
||||
cp.start();
|
||||
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
|
||||
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
|
||||
Mockito.when(serverManager.getOnlineServersList())
|
||||
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
|
@ -204,6 +211,12 @@ public class TestDrainingServer {
|
|||
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
|
||||
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
|
||||
|
||||
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(server);
|
||||
cp.start();
|
||||
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
|
||||
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
|
||||
Mockito.when(serverManager.getOnlineServersList()).thenReturn(
|
||||
new ArrayList<ServerName>(onlineServers.keySet()));
|
||||
|
@ -291,4 +304,4 @@ public class TestDrainingServer {
|
|||
executor.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, 3);
|
||||
return executor;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,7 +64,7 @@ public class TestLocalHBaseCluster {
|
|||
* running in local mode.
|
||||
*/
|
||||
public static class MyHMaster extends HMaster {
|
||||
public MyHMaster(Configuration conf, ConsensusProvider cp)
|
||||
public MyHMaster(Configuration conf, CoordinatedStateManager cp)
|
||||
throws IOException, KeeperException,
|
||||
InterruptedException {
|
||||
super(conf, cp);
|
||||
|
@ -80,7 +80,7 @@ public class TestLocalHBaseCluster {
|
|||
*/
|
||||
public static class MyHRegionServer extends MiniHBaseCluster.MiniHBaseClusterRegionServer {
|
||||
|
||||
public MyHRegionServer(Configuration conf, ConsensusProvider cp) throws IOException,
|
||||
public MyHRegionServer(Configuration conf, CoordinatedStateManager cp) throws IOException,
|
||||
InterruptedException {
|
||||
super(conf, cp);
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
|
@ -121,8 +122,9 @@ public class HConnectionTestingUtility {
|
|||
}
|
||||
NonceGenerator ng = Mockito.mock(NonceGenerator.class);
|
||||
Mockito.when(c.getNonceGenerator()).thenReturn(ng);
|
||||
Mockito.when(c.getAsyncProcess()).thenReturn(new AsyncProcess(
|
||||
c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false));
|
||||
Mockito.when(c.getAsyncProcess()).thenReturn(
|
||||
new AsyncProcess(c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false,
|
||||
RpcControllerFactory.instantiate(conf)));
|
||||
Mockito.doNothing().when(c).incCount();
|
||||
Mockito.doNothing().when(c).decCount();
|
||||
return c;
|
||||
|
|
|
@ -60,12 +60,13 @@ import org.apache.hadoop.hbase.executor.EventHandler;
|
|||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtilsForTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -255,8 +256,8 @@ public class TestAdmin {
|
|||
|
||||
this.admin.disableTable(ht.getName());
|
||||
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getZKTable().isDisabledTable(
|
||||
ht.getName()));
|
||||
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
|
||||
ht.getName(), ZooKeeperProtos.Table.State.DISABLED));
|
||||
|
||||
// Test that table is disabled
|
||||
get = new Get(row);
|
||||
|
@ -270,8 +271,8 @@ public class TestAdmin {
|
|||
assertTrue(ok);
|
||||
this.admin.enableTable(table);
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getZKTable().isEnabledTable(
|
||||
ht.getName()));
|
||||
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
|
||||
ht.getName(), ZooKeeperProtos.Table.State.ENABLED));
|
||||
|
||||
// Test that table is enabled
|
||||
try {
|
||||
|
@ -343,8 +344,8 @@ public class TestAdmin {
|
|||
tables = this.admin.listTables();
|
||||
assertEquals(numTables + 1, tables.length);
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
|
||||
.getMaster().getAssignmentManager().getZKTable().isEnabledTable(
|
||||
TableName.valueOf("testCreateTable")));
|
||||
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
|
||||
TableName.valueOf("testCreateTable"), ZooKeeperProtos.Table.State.ENABLED));
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
|
@ -1125,8 +1126,8 @@ public class TestAdmin {
|
|||
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
|
||||
TableName tableName = TableName.valueOf("testMasterAdmin");
|
||||
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
|
||||
while (!ZKTableReadOnly.isEnabledTable(zkw,
|
||||
TableName.valueOf("testMasterAdmin"))) {
|
||||
while (!ZKTableStateClientSideReader.isEnabledTable(zkw,
|
||||
TableName.valueOf("testMasterAdmin"))) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
this.admin.disableTable(tableName);
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||
|
@ -130,7 +130,7 @@ public class TestClientScannerRPCTimeout {
|
|||
}
|
||||
|
||||
private static class RegionServerWithScanTimeout extends MiniHBaseClusterRegionServer {
|
||||
public RegionServerWithScanTimeout(Configuration conf, ConsensusProvider cp)
|
||||
public RegionServerWithScanTimeout(Configuration conf, CoordinatedStateManager cp)
|
||||
throws IOException, InterruptedException {
|
||||
super(conf, cp);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,203 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ProtobufCoprocessorService;
|
||||
import org.apache.hadoop.hbase.ipc.DelegatingPayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestRpcControllerFactory {
|
||||
|
||||
public static class StaticRpcControllerFactory extends RpcControllerFactory {
|
||||
|
||||
public StaticRpcControllerFactory(Configuration conf) {
|
||||
super(conf);
|
||||
}
|
||||
|
||||
public PayloadCarryingRpcController newController() {
|
||||
return new CountingRpcController(super.newController());
|
||||
}
|
||||
|
||||
public PayloadCarryingRpcController newController(final CellScanner cellScanner) {
|
||||
return new CountingRpcController(super.newController(cellScanner));
|
||||
}
|
||||
|
||||
public PayloadCarryingRpcController newController(final List<CellScannable> cellIterables) {
|
||||
return new CountingRpcController(super.newController(cellIterables));
|
||||
}
|
||||
}
|
||||
|
||||
public static class CountingRpcController extends DelegatingPayloadCarryingRpcController {
|
||||
|
||||
private static AtomicInteger INT_PRIORITY = new AtomicInteger();
|
||||
private static AtomicInteger TABLE_PRIORITY = new AtomicInteger();
|
||||
|
||||
public CountingRpcController(PayloadCarryingRpcController delegate) {
|
||||
super(delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPriority(int priority) {
|
||||
super.setPriority(priority);
|
||||
INT_PRIORITY.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPriority(TableName tn) {
|
||||
super.setPriority(tn);
|
||||
// ignore counts for system tables - it could change and we really only want to check on what
|
||||
// the client should change
|
||||
if (!tn.isSystemTable()) {
|
||||
TABLE_PRIORITY.incrementAndGet();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
// load an endpoint so we have an endpoint to test - it doesn't matter which one, but
|
||||
// this is already in tests, so we can just use it.
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
|
||||
ProtobufCoprocessorService.class.getName());
|
||||
|
||||
UTIL.startMiniCluster();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardown() throws Exception {
|
||||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* check some of the methods and make sure we are incrementing each time. Its a bit tediuous to
|
||||
* cover all methods here and really is a bit brittle since we can always add new methods but
|
||||
* won't be sure to add them here. So we just can cover the major ones.
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
@Test
|
||||
public void testCountController() throws Exception {
|
||||
Configuration conf = new Configuration(UTIL.getConfiguration());
|
||||
// setup our custom controller
|
||||
conf.set(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY,
|
||||
StaticRpcControllerFactory.class.getName());
|
||||
|
||||
TableName name = TableName.valueOf("testcustomcontroller");
|
||||
UTIL.createTable(name, fam1).close();
|
||||
|
||||
// change one of the connection properties so we get a new HConnection with our configuration
|
||||
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1);
|
||||
|
||||
HTable table = new HTable(conf, name);
|
||||
table.setAutoFlushTo(false);
|
||||
byte[] row = Bytes.toBytes("row");
|
||||
Put p = new Put(row);
|
||||
p.add(fam1, fam1, Bytes.toBytes("val0"));
|
||||
table.put(p);
|
||||
table.flushCommits();
|
||||
Integer counter = 1;
|
||||
counter = verifyCount(counter);
|
||||
|
||||
Delete d = new Delete(row);
|
||||
d.deleteColumn(fam1, fam1);
|
||||
table.delete(d);
|
||||
counter = verifyCount(counter);
|
||||
|
||||
Put p2 = new Put(row);
|
||||
p2.add(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
|
||||
table.batch(Lists.newArrayList(p, p2), new Object[2]);
|
||||
// this only goes to a single server, so we don't need to change the count here
|
||||
counter = verifyCount(counter);
|
||||
|
||||
Append append = new Append(row);
|
||||
append.add(fam1, fam1, Bytes.toBytes("val2"));
|
||||
table.append(append);
|
||||
counter = verifyCount(counter);
|
||||
|
||||
// and check the major lookup calls as well
|
||||
Get g = new Get(row);
|
||||
table.get(g);
|
||||
counter = verifyCount(counter);
|
||||
|
||||
ResultScanner scan = table.getScanner(fam1);
|
||||
scan.next();
|
||||
scan.close();
|
||||
counter = verifyCount(counter);
|
||||
|
||||
Get g2 = new Get(row);
|
||||
table.get(Lists.newArrayList(g, g2));
|
||||
// same server, so same as above for not changing count
|
||||
counter = verifyCount(counter);
|
||||
|
||||
// make sure all the scanner types are covered
|
||||
Scan scanInfo = new Scan(row);
|
||||
// regular small
|
||||
scanInfo.setSmall(true);
|
||||
counter = doScan(table, scanInfo, counter);
|
||||
|
||||
// reversed, small
|
||||
scanInfo.setReversed(true);
|
||||
counter = doScan(table, scanInfo, counter);
|
||||
|
||||
// reversed, regular
|
||||
scanInfo.setSmall(false);
|
||||
counter = doScan(table, scanInfo, counter);
|
||||
|
||||
table.close();
|
||||
}
|
||||
|
||||
int doScan(HTable table, Scan scan, int expectedCount) throws IOException {
|
||||
ResultScanner results = table.getScanner(scan);
|
||||
results.next();
|
||||
results.close();
|
||||
return verifyCount(expectedCount);
|
||||
}
|
||||
|
||||
int verifyCount(Integer counter) {
|
||||
assertEquals(counter.intValue(), CountingRpcController.TABLE_PRIORITY.get());
|
||||
assertEquals(0, CountingRpcController.INT_PRIORITY.get());
|
||||
return counter + 1;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,176 @@
|
|||
/**
|
||||
*
|
||||
* 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.io.hfile;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mortbay.log.Log;
|
||||
|
||||
/**
|
||||
* Tests that {@link CacheConfig} does as expected.
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestCacheConfig {
|
||||
private Configuration conf;
|
||||
|
||||
static class DataCacheEntry implements Cacheable {
|
||||
private static final int SIZE = 1;
|
||||
private static DataCacheEntry SINGLETON = new DataCacheEntry();
|
||||
|
||||
private final CacheableDeserializer<Cacheable> deserializer =
|
||||
new CacheableDeserializer<Cacheable>() {
|
||||
@Override
|
||||
public int getDeserialiserIdentifier() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cacheable deserialize(ByteBuffer b, boolean reuse) throws IOException {
|
||||
Log.info("Deserialized " + b + ", reuse=" + reuse);
|
||||
return SINGLETON;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cacheable deserialize(ByteBuffer b) throws IOException {
|
||||
Log.info("Deserialized " + b);
|
||||
return SINGLETON;
|
||||
}
|
||||
};
|
||||
|
||||
public String toString() {
|
||||
return "size=" + SIZE + ", type=" + getBlockType();
|
||||
};
|
||||
|
||||
@Override
|
||||
public long heapSize() {
|
||||
return SIZE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSerializedLength() {
|
||||
return SIZE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(ByteBuffer destination) {
|
||||
Log.info("Serialized " + this + " to " + destination);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheableDeserializer<Cacheable> getDeserializer() {
|
||||
return this.deserializer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlockType getBlockType() {
|
||||
return BlockType.DATA;
|
||||
}
|
||||
};
|
||||
|
||||
static class MetaCacheEntry extends DataCacheEntry {
|
||||
@Override
|
||||
public BlockType getBlockType() {
|
||||
return BlockType.INTERMEDIATE_INDEX;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
|
||||
this.conf = HBaseConfiguration.create();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
// Let go of current block cache.
|
||||
CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cc
|
||||
* @param doubling If true, addition of element ups counter by 2, not 1, because element added
|
||||
* to onheap and offheap caches.
|
||||
* @param sizing True if we should run sizing test (doesn't always apply).
|
||||
*/
|
||||
private void basicBlockCacheOps(final CacheConfig cc, final boolean doubling,
|
||||
final boolean sizing) {
|
||||
assertTrue(cc.isBlockCacheEnabled());
|
||||
assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
|
||||
BlockCache bc = cc.getBlockCache();
|
||||
BlockCacheKey bck = new BlockCacheKey("f", 0);
|
||||
Cacheable c = new DataCacheEntry();
|
||||
// Do asserts on block counting.
|
||||
long initialBlockCount = bc.getBlockCount();
|
||||
bc.cacheBlock(bck, c);
|
||||
assertEquals(doubling? 2: 1, bc.getBlockCount() - initialBlockCount);
|
||||
bc.evictBlock(bck);
|
||||
assertEquals(initialBlockCount, bc.getBlockCount());
|
||||
// Do size accounting. Do it after the above 'warm-up' because it looks like some
|
||||
// buffers do lazy allocation so sizes are off on first go around.
|
||||
if (sizing) {
|
||||
long originalSize = bc.getCurrentSize();
|
||||
bc.cacheBlock(bck, c);
|
||||
long size = bc.getCurrentSize();
|
||||
assertTrue(bc.getCurrentSize() > originalSize);
|
||||
bc.evictBlock(bck);
|
||||
size = bc.getCurrentSize();
|
||||
assertEquals(originalSize, size);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCacheConfigDefaultLRUBlockCache() {
|
||||
CacheConfig cc = new CacheConfig(this.conf);
|
||||
assertTrue(cc.isBlockCacheEnabled());
|
||||
assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
|
||||
basicBlockCacheOps(cc, false, true);
|
||||
assertTrue(cc.getBlockCache() instanceof LruBlockCache);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSlabCacheConfig() {
|
||||
this.conf.setFloat(CacheConfig.SLAB_CACHE_OFFHEAP_PERCENTAGE_KEY, 0.1f);
|
||||
CacheConfig cc = new CacheConfig(this.conf);
|
||||
basicBlockCacheOps(cc, true, true);
|
||||
assertTrue(cc.getBlockCache() instanceof DoubleBlockCache);
|
||||
// TODO Assert sizes allocated are right.
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBucketCacheConfig() {
|
||||
this.conf.set(CacheConfig.BUCKET_CACHE_IOENGINE_KEY, "offheap");
|
||||
this.conf.setInt(CacheConfig.BUCKET_CACHE_SIZE_KEY, 100);
|
||||
this.conf.setFloat(CacheConfig.BUCKET_CACHE_COMBINED_PERCENTAGE_KEY, 0.8f);
|
||||
CacheConfig cc = new CacheConfig(this.conf);
|
||||
basicBlockCacheOps(cc, false, false);
|
||||
assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
|
||||
// TODO: Assert sizes allocated are right and proportions.
|
||||
}
|
||||
}
|
|
@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
|||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||
|
@ -269,7 +269,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -28,12 +28,12 @@ import java.util.concurrent.Semaphore;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||
|
@ -282,7 +282,7 @@ public class TestActiveMasterManager {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -32,6 +32,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
|
@ -50,8 +52,8 @@ import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
|
|||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.consensus.ZkCoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -116,6 +118,7 @@ public class TestAssignmentManager {
|
|||
private Server server;
|
||||
private ServerManager serverManager;
|
||||
private ZooKeeperWatcher watcher;
|
||||
private CoordinatedStateManager cp;
|
||||
private LoadBalancer balancer;
|
||||
private HMaster master;
|
||||
|
||||
|
@ -146,6 +149,12 @@ public class TestAssignmentManager {
|
|||
Mockito.doThrow(new RuntimeException("Aborted")).
|
||||
when(server).abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
|
||||
|
||||
cp = new ZkCoordinatedStateManager();
|
||||
cp.initialize(this.server);
|
||||
cp.start();
|
||||
|
||||
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
|
||||
|
||||
// Mock a ServerManager. Say server SERVERNAME_{A,B} are online. Also
|
||||
// make it so if close or open, we return 'success'.
|
||||
this.serverManager = Mockito.mock(ServerManager.class);
|
||||
|
@ -184,6 +193,7 @@ public class TestAssignmentManager {
|
|||
// Clean up all znodes
|
||||
ZKAssign.deleteAllNodes(this.watcher);
|
||||
this.watcher.close();
|
||||
this.cp.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -197,7 +207,8 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test(timeout = 60000)
|
||||
public void testBalanceOnMasterFailoverScenarioWithOpenedNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException {
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException,
|
||||
DeserializationException, CoordinatedStateException {
|
||||
AssignmentManagerWithExtrasForTesting am =
|
||||
setUpMockedAssignmentManager(this.server, this.serverManager);
|
||||
try {
|
||||
|
@ -244,8 +255,9 @@ public class TestAssignmentManager {
|
|||
}
|
||||
|
||||
@Test(timeout = 60000)
|
||||
public void testBalanceOnMasterFailoverScenarioWithClosedNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException {
|
||||
public void testBalanceOnMasterFailoverScenarioWithClosedNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException,
|
||||
DeserializationException, CoordinatedStateException {
|
||||
AssignmentManagerWithExtrasForTesting am =
|
||||
setUpMockedAssignmentManager(this.server, this.serverManager);
|
||||
try {
|
||||
|
@ -294,7 +306,8 @@ public class TestAssignmentManager {
|
|||
|
||||
@Test(timeout = 60000)
|
||||
public void testBalanceOnMasterFailoverScenarioWithOfflineNode()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException {
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException,
|
||||
DeserializationException, CoordinatedStateException {
|
||||
AssignmentManagerWithExtrasForTesting am =
|
||||
setUpMockedAssignmentManager(this.server, this.serverManager);
|
||||
try {
|
||||
|
@ -359,8 +372,8 @@ public class TestAssignmentManager {
|
|||
* @throws DeserializationException
|
||||
*/
|
||||
@Test
|
||||
public void testBalance()
|
||||
throws IOException, KeeperException, DeserializationException, InterruptedException {
|
||||
public void testBalance() throws IOException, KeeperException, DeserializationException,
|
||||
InterruptedException, CoordinatedStateException {
|
||||
// Create and startup an executor. This is used by AssignmentManager
|
||||
// handling zk callbacks.
|
||||
ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
|
||||
|
@ -435,7 +448,7 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test
|
||||
public void testShutdownHandler()
|
||||
throws KeeperException, IOException, ServiceException {
|
||||
throws KeeperException, IOException, CoordinatedStateException, ServiceException {
|
||||
// Create and startup an executor. This is used by AssignmentManager
|
||||
// handling zk callbacks.
|
||||
ExecutorService executor = startupMasterExecutor("testShutdownHandler");
|
||||
|
@ -466,7 +479,7 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test
|
||||
public void testSSHWhenDisableTableInProgress() throws KeeperException, IOException,
|
||||
ServiceException {
|
||||
CoordinatedStateException, ServiceException {
|
||||
testCaseWithPartiallyDisabledState(Table.State.DISABLING);
|
||||
testCaseWithPartiallyDisabledState(Table.State.DISABLED);
|
||||
}
|
||||
|
@ -488,7 +501,8 @@ public class TestAssignmentManager {
|
|||
}
|
||||
|
||||
private void testCaseWithSplitRegionPartial(boolean regionSplitDone) throws KeeperException,
|
||||
IOException, NodeExistsException, InterruptedException, ServiceException {
|
||||
IOException, InterruptedException,
|
||||
CoordinatedStateException, ServiceException {
|
||||
// Create and startup an executor. This is used by AssignmentManager
|
||||
// handling zk callbacks.
|
||||
ExecutorService executor = startupMasterExecutor("testSSHWhenSplitRegionInProgress");
|
||||
|
@ -504,7 +518,8 @@ public class TestAssignmentManager {
|
|||
// adding region in pending close.
|
||||
am.getRegionStates().updateRegionState(
|
||||
REGIONINFO, State.SPLITTING, SERVERNAME_A);
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLED);
|
||||
RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
|
||||
REGIONINFO.getRegionName(), SERVERNAME_A);
|
||||
String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
|
||||
|
@ -536,7 +551,7 @@ public class TestAssignmentManager {
|
|||
}
|
||||
|
||||
private void testCaseWithPartiallyDisabledState(Table.State state) throws KeeperException,
|
||||
IOException, NodeExistsException, ServiceException {
|
||||
IOException, CoordinatedStateException, ServiceException {
|
||||
// Create and startup an executor. This is used by AssignmentManager
|
||||
// handling zk callbacks.
|
||||
ExecutorService executor = startupMasterExecutor("testSSHWhenDisableTableInProgress");
|
||||
|
@ -553,9 +568,11 @@ public class TestAssignmentManager {
|
|||
// adding region in pending close.
|
||||
am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
|
||||
if (state == Table.State.DISABLING) {
|
||||
am.getZKTable().setDisablingTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.DISABLING);
|
||||
} else {
|
||||
am.getZKTable().setDisabledTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.DISABLED);
|
||||
}
|
||||
RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
|
||||
REGIONINFO.getRegionName(), SERVERNAME_A);
|
||||
|
@ -668,7 +685,8 @@ public class TestAssignmentManager {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testUnassignWithSplitAtSameTime() throws KeeperException, IOException {
|
||||
public void testUnassignWithSplitAtSameTime() throws KeeperException,
|
||||
IOException, CoordinatedStateException {
|
||||
// Region to use in test.
|
||||
final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
|
||||
// First amend the servermanager mock so that when we do send close of the
|
||||
|
@ -713,7 +731,8 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test(timeout = 60000)
|
||||
public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
|
||||
throws IOException, KeeperException, InterruptedException, ServiceException {
|
||||
throws IOException, KeeperException, CoordinatedStateException,
|
||||
InterruptedException, ServiceException {
|
||||
final RecoverableZooKeeper recoverableZk = Mockito
|
||||
.mock(RecoverableZooKeeper.class);
|
||||
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
|
||||
|
@ -744,7 +763,7 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test(timeout = 60000)
|
||||
public void testRegionPlanIsUpdatedWhenRegionFailsToOpen() throws IOException, KeeperException,
|
||||
ServiceException, InterruptedException {
|
||||
ServiceException, InterruptedException, CoordinatedStateException {
|
||||
this.server.getConfiguration().setClass(
|
||||
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MockedLoadBalancer.class,
|
||||
LoadBalancer.class);
|
||||
|
@ -842,7 +861,7 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test(timeout = 60000)
|
||||
public void testRegionInOpeningStateOnDeadRSWhileMasterFailover() throws IOException,
|
||||
KeeperException, ServiceException, InterruptedException {
|
||||
KeeperException, ServiceException, CoordinatedStateException, InterruptedException {
|
||||
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
|
||||
this.server, this.serverManager);
|
||||
ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A);
|
||||
|
@ -858,7 +877,8 @@ public class TestAssignmentManager {
|
|||
am.gate.set(false);
|
||||
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
|
||||
assertFalse(am.processRegionsInTransition(rt, REGIONINFO, version));
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLED);
|
||||
processServerShutdownHandler(ct, am, false);
|
||||
// Waiting for the assignment to get completed.
|
||||
while (!am.gate.get()) {
|
||||
|
@ -889,8 +909,9 @@ public class TestAssignmentManager {
|
|||
// To avoid cast exception in DisableTableHandler process.
|
||||
HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
|
||||
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(HTU.getConfiguration());
|
||||
Server server = new HMaster(HTU.getConfiguration(), cp);
|
||||
CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
HTU.getConfiguration());
|
||||
Server server = new HMaster(HTU.getConfiguration(), csm);
|
||||
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
|
||||
this.serverManager);
|
||||
AtomicBoolean gate = new AtomicBoolean(false);
|
||||
|
@ -899,7 +920,8 @@ public class TestAssignmentManager {
|
|||
}
|
||||
try{
|
||||
// set table in disabling state.
|
||||
am.getZKTable().setDisablingTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.DISABLING);
|
||||
am.joinCluster();
|
||||
// should not call retainAssignment if we get empty regions in assignAllUserRegions.
|
||||
assertFalse(
|
||||
|
@ -907,12 +929,14 @@ public class TestAssignmentManager {
|
|||
gate.get());
|
||||
// need to change table state from disabling to disabled.
|
||||
assertTrue("Table should be disabled.",
|
||||
am.getZKTable().isDisabledTable(REGIONINFO.getTable()));
|
||||
am.getTableStateManager().isTableState(REGIONINFO.getTable(),
|
||||
Table.State.DISABLED));
|
||||
} finally {
|
||||
this.server.getConfiguration().setClass(
|
||||
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, SimpleLoadBalancer.class,
|
||||
LoadBalancer.class);
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLED);
|
||||
am.shutdown();
|
||||
}
|
||||
}
|
||||
|
@ -932,24 +956,28 @@ public class TestAssignmentManager {
|
|||
Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
|
||||
Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
|
||||
HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(HTU.getConfiguration());
|
||||
Server server = new HMaster(HTU.getConfiguration(), cp);
|
||||
CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
HTU.getConfiguration());
|
||||
Server server = new HMaster(HTU.getConfiguration(), csm);
|
||||
Whitebox.setInternalState(server, "serverManager", this.serverManager);
|
||||
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
|
||||
this.serverManager);
|
||||
try {
|
||||
// set table in enabling state.
|
||||
am.getZKTable().setEnablingTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLING);
|
||||
new EnableTableHandler(server, REGIONINFO.getTable(),
|
||||
am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare()
|
||||
.process();
|
||||
assertEquals("Number of assignments should be 1.", 1, assignmentCount);
|
||||
assertTrue("Table should be enabled.",
|
||||
am.getZKTable().isEnabledTable(REGIONINFO.getTable()));
|
||||
am.getTableStateManager().isTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLED));
|
||||
} finally {
|
||||
enabling = false;
|
||||
assignmentCount = 0;
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLED);
|
||||
am.shutdown();
|
||||
ZKAssign.deleteAllNodes(this.watcher);
|
||||
}
|
||||
|
@ -964,24 +992,26 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test
|
||||
public void testMasterRestartShouldRemoveStaleZnodesOfUnknownTableAsForMeta()
|
||||
throws KeeperException, IOException, Exception {
|
||||
throws Exception {
|
||||
List<ServerName> destServers = new ArrayList<ServerName>(1);
|
||||
destServers.add(SERVERNAME_A);
|
||||
Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
|
||||
Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
|
||||
HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(HTU.getConfiguration());
|
||||
Server server = new HMaster(HTU.getConfiguration(), cp);
|
||||
CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
HTU.getConfiguration());
|
||||
Server server = new HMaster(HTU.getConfiguration(), csm);
|
||||
Whitebox.setInternalState(server, "serverManager", this.serverManager);
|
||||
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
|
||||
this.serverManager);
|
||||
try {
|
||||
TableName tableName = TableName.valueOf("dummyTable");
|
||||
// set table in enabling state.
|
||||
am.getZKTable().setEnablingTable(tableName);
|
||||
am.getTableStateManager().setTableState(tableName,
|
||||
Table.State.ENABLING);
|
||||
am.joinCluster();
|
||||
assertFalse("Table should not be present in zookeeper.",
|
||||
am.getZKTable().isTablePresent(tableName));
|
||||
am.getTableStateManager().isTablePresent(tableName));
|
||||
} finally {
|
||||
}
|
||||
}
|
||||
|
@ -992,7 +1022,7 @@ public class TestAssignmentManager {
|
|||
*/
|
||||
@Test
|
||||
public void testSSHTimesOutOpeningRegionTransition()
|
||||
throws KeeperException, IOException, ServiceException {
|
||||
throws KeeperException, IOException, CoordinatedStateException, ServiceException {
|
||||
// We need a mocked catalog tracker.
|
||||
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
|
||||
// Create an AM.
|
||||
|
@ -1006,7 +1036,8 @@ public class TestAssignmentManager {
|
|||
// adding region plan
|
||||
am.regionPlans.put(REGIONINFO.getEncodedName(),
|
||||
new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
|
||||
Table.State.ENABLED);
|
||||
|
||||
try {
|
||||
am.assignInvoked = false;
|
||||
|
@ -1102,7 +1133,8 @@ public class TestAssignmentManager {
|
|||
* @throws KeeperException
|
||||
*/
|
||||
private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server,
|
||||
final ServerManager manager) throws IOException, KeeperException, ServiceException {
|
||||
final ServerManager manager) throws IOException, KeeperException,
|
||||
ServiceException, CoordinatedStateException {
|
||||
// We need a mocked catalog tracker. Its used by our AM instance.
|
||||
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
|
||||
// Make an RS Interface implementation. Make it so a scanner can go against
|
||||
|
@ -1173,7 +1205,7 @@ public class TestAssignmentManager {
|
|||
final Server master, final ServerManager serverManager,
|
||||
final CatalogTracker catalogTracker, final LoadBalancer balancer,
|
||||
final ExecutorService service, final TableLockManager tableLockManager)
|
||||
throws KeeperException, IOException {
|
||||
throws KeeperException, IOException, CoordinatedStateException {
|
||||
super(master, serverManager, catalogTracker, balancer, service, null, tableLockManager);
|
||||
this.es = service;
|
||||
this.ct = catalogTracker;
|
||||
|
@ -1267,6 +1299,8 @@ public class TestAssignmentManager {
|
|||
throw new RuntimeException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
} catch (CoordinatedStateException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -1309,7 +1343,8 @@ public class TestAssignmentManager {
|
|||
* assignment). So during master failover, we can ignored such events too.
|
||||
*/
|
||||
@Test
|
||||
public void testAssignmentEventIgnoredIfNotExpected() throws KeeperException, IOException {
|
||||
public void testAssignmentEventIgnoredIfNotExpected() throws KeeperException, IOException,
|
||||
CoordinatedStateException {
|
||||
// Region to use in test.
|
||||
final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
|
||||
// Need a mocked catalog tracker.
|
||||
|
|
|
@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.Waiter;
|
|||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
|
@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
|||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -567,7 +568,7 @@ public class TestAssignmentManagerOnCluster {
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
am.getZKTable().setDisablingTable(table);
|
||||
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLING);
|
||||
List<HRegionInfo> toAssignRegions = am.processServerShutdown(destServerName);
|
||||
assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty());
|
||||
assertTrue("Regions to be assigned should be empty.", am.getRegionStates()
|
||||
|
@ -576,7 +577,7 @@ public class TestAssignmentManagerOnCluster {
|
|||
if (hri != null && serverName != null) {
|
||||
am.regionOnline(hri, serverName);
|
||||
}
|
||||
am.getZKTable().setDisabledTable(table);
|
||||
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLED);
|
||||
TEST_UTIL.deleteTable(table);
|
||||
}
|
||||
}
|
||||
|
@ -838,7 +839,7 @@ public class TestAssignmentManagerOnCluster {
|
|||
public static class MyMaster extends HMaster {
|
||||
AtomicBoolean enabled = new AtomicBoolean(true);
|
||||
|
||||
public MyMaster(Configuration conf, ConsensusProvider cp)
|
||||
public MyMaster(Configuration conf, CoordinatedStateManager cp)
|
||||
throws IOException, KeeperException,
|
||||
InterruptedException {
|
||||
super(conf, cp);
|
||||
|
|
|
@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.client.HConnection;
|
|||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
|
||||
|
@ -165,7 +165,7 @@ public class TestCatalogJanitor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -249,7 +249,7 @@ public class TestCatalogJanitor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -62,7 +62,7 @@ public class TestClockSkewDetection {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,8 +26,8 @@ import java.net.SocketTimeoutException;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
|
||||
|
@ -48,7 +48,7 @@ public class TestHMasterRPCException {
|
|||
TEST_UTIL.startMiniZKCluster();
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.set(HConstants.MASTER_PORT, "0");
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
HMaster hm = new HMaster(conf, cp);
|
||||
ServerName sm = hm.getServerName();
|
||||
RpcClient rpcClient = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT);
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.*;
|
|||
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.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
|
@ -67,7 +68,8 @@ public class TestMaster {
|
|||
HMaster m = cluster.getMaster();
|
||||
|
||||
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
|
||||
assertTrue(m.assignmentManager.getZKTable().isEnabledTable(TABLENAME));
|
||||
assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME,
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
TEST_UTIL.loadTable(ht, FAMILYNAME, false);
|
||||
ht.close();
|
||||
|
||||
|
|
|
@ -47,9 +47,11 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
|
|||
import org.apache.hadoop.hbase.RegionTransition;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableStateManager;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
|
||||
|
@ -63,7 +65,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
|||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.junit.Test;
|
||||
|
@ -305,8 +307,8 @@ public class TestMasterFailover {
|
|||
log("Beginning to mock scenarios");
|
||||
|
||||
// Disable the disabledTable in ZK
|
||||
ZKTable zktable = new ZKTable(zkw);
|
||||
zktable.setDisabledTable(disabledTable);
|
||||
TableStateManager zktable = new ZKTableStateManager(zkw);
|
||||
zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED);
|
||||
|
||||
/*
|
||||
* ZK = OFFLINE
|
||||
|
@ -620,7 +622,8 @@ public class TestMasterFailover {
|
|||
log("Assignment completed");
|
||||
|
||||
assertTrue(" Table must be enabled.", master.getAssignmentManager()
|
||||
.getZKTable().isEnabledTable(TableName.valueOf("enabledTable")));
|
||||
.getTableStateManager().isTableState(TableName.valueOf("enabledTable"),
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
// we also need regions assigned out on the dead server
|
||||
List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
|
||||
enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6));
|
||||
|
@ -694,11 +697,12 @@ public class TestMasterFailover {
|
|||
log("Beginning to mock scenarios");
|
||||
|
||||
// Disable the disabledTable in ZK
|
||||
ZKTable zktable = new ZKTable(zkw);
|
||||
zktable.setDisabledTable(disabledTable);
|
||||
TableStateManager zktable = new ZKTableStateManager(zkw);
|
||||
zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED);
|
||||
|
||||
assertTrue(" The enabled table should be identified on master fail over.",
|
||||
zktable.isEnabledTable(TableName.valueOf("enabledTable")));
|
||||
zktable.isTableState(TableName.valueOf("enabledTable"),
|
||||
ZooKeeperProtos.Table.State.ENABLED));
|
||||
|
||||
/*
|
||||
* ZK = CLOSING
|
||||
|
|
|
@ -23,11 +23,11 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CompatibilityFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
|
||||
|
@ -50,7 +50,7 @@ public class TestMasterMetrics {
|
|||
private static HBaseTestingUtility TEST_UTIL;
|
||||
|
||||
public static class MyMaster extends HMaster {
|
||||
public MyMaster(Configuration conf, ConsensusProvider cp) throws IOException,
|
||||
public MyMaster(Configuration conf, CoordinatedStateManager cp) throws IOException,
|
||||
KeeperException, InterruptedException {
|
||||
super(conf, cp);
|
||||
}
|
||||
|
|
|
@ -32,6 +32,9 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -46,8 +49,6 @@ import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
|
|||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
||||
|
@ -125,7 +126,7 @@ public class TestMasterNoCluster {
|
|||
@Test (timeout=30000)
|
||||
public void testStopDuringStart()
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
TESTUTIL.getConfiguration());
|
||||
HMaster master = new HMaster(TESTUTIL.getConfiguration(), cp);
|
||||
master.start();
|
||||
|
@ -177,7 +178,7 @@ public class TestMasterNoCluster {
|
|||
// and get notification on transitions. We need to fake out any rpcs the
|
||||
// master does opening/closing regions. Also need to fake out the address
|
||||
// of the 'remote' mocked up regionservers.
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
TESTUTIL.getConfiguration());
|
||||
HMaster master = new HMaster(conf, cp) {
|
||||
InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
|
||||
|
@ -260,7 +261,7 @@ public class TestMasterNoCluster {
|
|||
final ServerName deadServer = ServerName.valueOf("test.sample", 1, 100);
|
||||
final MockRegionServer rs0 = new MockRegionServer(conf, newServer);
|
||||
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
TESTUTIL.getConfiguration());
|
||||
HMaster master = new HMaster(conf, cp) {
|
||||
@Override
|
||||
|
@ -269,7 +270,7 @@ public class TestMasterNoCluster {
|
|||
|
||||
@Override
|
||||
void initializeZKBasedSystemTrackers() throws IOException,
|
||||
InterruptedException, KeeperException {
|
||||
InterruptedException, KeeperException, CoordinatedStateException {
|
||||
super.initializeZKBasedSystemTrackers();
|
||||
// Record a newer server in server manager at first
|
||||
serverManager.recordNewServerWithLock(newServer, ServerLoad.EMPTY_SERVERLOAD);
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
|
@ -96,8 +97,9 @@ public class TestMasterRestartAfterDisablingTable {
|
|||
cluster.waitForActiveAndReadyMaster();
|
||||
|
||||
assertTrue("The table should not be in enabled state", cluster.getMaster()
|
||||
.getAssignmentManager().getZKTable().isDisablingOrDisabledTable(
|
||||
TableName.valueOf("tableRestart")));
|
||||
.getAssignmentManager().getTableStateManager().isTableState(
|
||||
TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.DISABLED,
|
||||
ZooKeeperProtos.Table.State.DISABLING));
|
||||
log("Enabling table\n");
|
||||
// Need a new Admin, the previous one is on the old master
|
||||
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
|
||||
|
@ -111,8 +113,8 @@ public class TestMasterRestartAfterDisablingTable {
|
|||
+ " switch except for the catalog and namespace tables.",
|
||||
6, regions.size());
|
||||
assertTrue("The table should be in enabled state", cluster.getMaster()
|
||||
.getAssignmentManager().getZKTable()
|
||||
.isEnabledTable(TableName.valueOf("tableRestart")));
|
||||
.getAssignmentManager().getTableStateManager()
|
||||
.isTableState(TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.ENABLED));
|
||||
ht.close();
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.MockServer;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -130,7 +130,7 @@ public class TestOpenedRegionHandler {
|
|||
// create a node with OPENED state
|
||||
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
|
||||
region, server.getServerName());
|
||||
when(am.getZKTable()).thenReturn(new ZKTable(zkw));
|
||||
when(am.getTableStateManager()).thenReturn(new ZKTableStateManager(zkw));
|
||||
Stat stat = new Stat();
|
||||
String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo()
|
||||
.getEncodedName());
|
||||
|
|
|
@ -491,51 +491,6 @@ public class TestSplitLogManager {
|
|||
assertFalse(fs.exists(emptyLogDirPath));
|
||||
}
|
||||
|
||||
@Test(timeout=45000)
|
||||
public void testVanishingTaskZNode() throws Exception {
|
||||
LOG.info("testVanishingTaskZNode");
|
||||
|
||||
conf.setInt("hbase.splitlog.manager.unassigned.timeout", 0);
|
||||
conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 1000);
|
||||
slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
final Path logDir = new Path(fs.getWorkingDirectory(),
|
||||
UUID.randomUUID().toString());
|
||||
fs.mkdirs(logDir);
|
||||
Thread thread = null;
|
||||
try {
|
||||
Path logFile = new Path(logDir, UUID.randomUUID().toString());
|
||||
fs.createNewFile(logFile);
|
||||
thread = new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
// this call will block because there are no SplitLogWorkers,
|
||||
// until the task znode is deleted below. Then the call will
|
||||
// complete successfully, assuming the log is split.
|
||||
slm.splitLogDistributed(logDir);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("splitLogDistributed failed", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
thread.start();
|
||||
waitForCounter(tot_mgr_node_create_result, 0, 1, 10000);
|
||||
String znode = ZKSplitLog.getEncodedNodeName(zkw, logFile.toString());
|
||||
// remove the task znode, to finish the distributed log splitting
|
||||
ZKUtil.deleteNode(zkw, znode);
|
||||
waitForCounter(tot_mgr_get_data_nonode, 0, 1, 30000);
|
||||
waitForCounter(tot_mgr_log_split_batch_success, 0, 1, to/2);
|
||||
assertTrue(fs.exists(logFile));
|
||||
} finally {
|
||||
if (thread != null) {
|
||||
// interrupt the thread in case the test fails in the middle.
|
||||
// it has no effect if the thread is already terminated.
|
||||
thread.interrupt();
|
||||
}
|
||||
fs.delete(logDir, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The following test case is aiming to test the situation when distributedLogReplay is turned off
|
||||
* and restart a cluster there should no recovery regions in ZK left.
|
||||
|
|
|
@ -29,13 +29,13 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -210,7 +210,7 @@ public class TestHFileCleaner {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
|
@ -146,7 +146,7 @@ public class TestHFileLinkCleaner {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
||||
|
@ -152,7 +152,7 @@ public class TestLogsCleaner {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -181,7 +181,7 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
|
|||
}
|
||||
|
||||
@Override
|
||||
public int getBlockCacheHitPercent() {
|
||||
public double getBlockCacheHitPercent() {
|
||||
return 98;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.List;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
|
||||
|
@ -42,7 +42,7 @@ import com.google.protobuf.ServiceException;
|
|||
public class OOMERegionServer extends HRegionServer {
|
||||
private List<Put> retainer = new ArrayList<Put>();
|
||||
|
||||
public OOMERegionServer(HBaseConfiguration conf, ConsensusProvider cp)
|
||||
public OOMERegionServer(HBaseConfiguration conf, CoordinatedStateManager cp)
|
||||
throws IOException, InterruptedException {
|
||||
super(conf, cp);
|
||||
}
|
||||
|
|
|
@ -27,11 +27,11 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
|
@ -72,7 +72,7 @@ public class TestClusterId {
|
|||
TEST_UTIL.startMiniDFSCluster(1);
|
||||
|
||||
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
//start region server, needs to be separate
|
||||
//so we get an unset clusterId
|
||||
rst = JVMClusterUtil.createRegionServerThread(conf,cp,
|
||||
|
|
|
@ -28,12 +28,12 @@ import java.lang.management.ManagementFactory;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheStats;
|
||||
|
@ -420,7 +420,7 @@ public class TestHeapMemoryManager {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsensusProvider getConsensusProvider() {
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -83,7 +83,7 @@ public class TestMetricsRegionServer {
|
|||
HELPER.assertCounter("blockCacheHitCount", 416, serverSource);
|
||||
HELPER.assertCounter("blockCacheMissCount", 417, serverSource);
|
||||
HELPER.assertCounter("blockCacheEvictionCount", 418, serverSource);
|
||||
HELPER.assertGauge("blockCountHitPercent", 98, serverSource);
|
||||
HELPER.assertGauge("blockCacheCountHitPercent", 98, serverSource);
|
||||
HELPER.assertGauge("blockCacheExpressHitPercent", 97, serverSource);
|
||||
HELPER.assertCounter("updatesBlockedTime", 419, serverSource);
|
||||
}
|
||||
|
|
|
@ -25,12 +25,12 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.ipc.PriorityFunction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
|
||||
|
@ -58,7 +58,7 @@ public class TestPriorityRpc {
|
|||
public void setup() {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.setBoolean("hbase.testing.nocluster", true); // No need to do ZK
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(conf);
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
|
||||
regionServer = HRegionServer.constructRegionServer(HRegionServer.class, conf, cp);
|
||||
priority = regionServer.rpcServices.getPriority();
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.List;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.LargeTests;
|
|||
import org.apache.hadoop.hbase.LocalHBaseCluster;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
|
||||
|
@ -104,7 +104,7 @@ public class TestRSKilledWhenInitializing {
|
|||
|
||||
public static class MockedRegionServer extends MiniHBaseCluster.MiniHBaseClusterRegionServer {
|
||||
|
||||
public MockedRegionServer(Configuration conf, ConsensusProvider cp)
|
||||
public MockedRegionServer(Configuration conf, CoordinatedStateManager cp)
|
||||
throws IOException, InterruptedException {
|
||||
super(conf, cp);
|
||||
}
|
||||
|
|
|
@ -31,6 +31,8 @@ import java.util.List;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -42,8 +44,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.ConsensusProviderFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -220,7 +220,7 @@ public class TestRegionMergeTransaction {
|
|||
|
||||
// Run the execute. Look at what it returns.
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
TEST_UTIL.getConfiguration());
|
||||
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
|
||||
HRegion mergedRegion = mt.execute(mockServer, null);
|
||||
|
@ -269,7 +269,7 @@ public class TestRegionMergeTransaction {
|
|||
// Run the execute. Look at what it returns.
|
||||
boolean expectedException = false;
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
TEST_UTIL.getConfiguration());
|
||||
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
|
||||
try {
|
||||
|
@ -330,7 +330,7 @@ public class TestRegionMergeTransaction {
|
|||
// Run the execute. Look at what it returns.
|
||||
boolean expectedException = false;
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
|
||||
ConsensusProvider cp = ConsensusProviderFactory.getConsensusProvider(
|
||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||
TEST_UTIL.getConfiguration());
|
||||
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
|
||||
try {
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.filter.FilterList;
|
|||
import org.apache.hadoop.hbase.filter.FilterList.Operator;
|
||||
import org.apache.hadoop.hbase.filter.PageFilter;
|
||||
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
|
@ -88,28 +89,32 @@ public class TestReversibleScanners {
|
|||
TEST_UTIL.getDataTestDir("testReversibleStoreFileScanner"),
|
||||
"regionname"), "familyname");
|
||||
CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
|
||||
HFileContextBuilder hcBuilder = new HFileContextBuilder();
|
||||
hcBuilder.withBlockSize(2 * 1024);
|
||||
HFileContext hFileContext = hcBuilder.build();
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(
|
||||
TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir(
|
||||
hfilePath).withFileContext(hFileContext).build();
|
||||
writeStoreFile(writer);
|
||||
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
|
||||
HFileContextBuilder hcBuilder = new HFileContextBuilder();
|
||||
hcBuilder.withBlockSize(2 * 1024);
|
||||
hcBuilder.withDataBlockEncoding(encoding);
|
||||
HFileContext hFileContext = hcBuilder.build();
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(
|
||||
TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir(hfilePath)
|
||||
.withFileContext(hFileContext).build();
|
||||
writeStoreFile(writer);
|
||||
|
||||
StoreFile sf = new StoreFile(fs, writer.getPath(),
|
||||
TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
|
||||
StoreFile sf = new StoreFile(fs, writer.getPath(),
|
||||
TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
|
||||
|
||||
List<StoreFileScanner> scanners = StoreFileScanner
|
||||
.getScannersForStoreFiles(Collections.singletonList(sf), false, true,
|
||||
false, Long.MAX_VALUE);
|
||||
StoreFileScanner scanner = scanners.get(0);
|
||||
seekTestOfReversibleKeyValueScanner(scanner);
|
||||
for (int readPoint = 0; readPoint < MAXMVCC; readPoint++) {
|
||||
LOG.info("Setting read point to " + readPoint);
|
||||
scanners = StoreFileScanner.getScannersForStoreFiles(
|
||||
Collections.singletonList(sf), false, true, false, readPoint);
|
||||
seekTestOfReversibleKeyValueScannerWithMVCC(scanners.get(0), readPoint);
|
||||
List<StoreFileScanner> scanners = StoreFileScanner
|
||||
.getScannersForStoreFiles(Collections.singletonList(sf), false, true,
|
||||
false, Long.MAX_VALUE);
|
||||
StoreFileScanner scanner = scanners.get(0);
|
||||
seekTestOfReversibleKeyValueScanner(scanner);
|
||||
for (int readPoint = 0; readPoint < MAXMVCC; readPoint++) {
|
||||
LOG.info("Setting read point to " + readPoint);
|
||||
scanners = StoreFileScanner.getScannersForStoreFiles(
|
||||
Collections.singletonList(sf), false, true, false, readPoint);
|
||||
seekTestOfReversibleKeyValueScannerWithMVCC(scanners.get(0), readPoint);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
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.ConsensusProvider;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
|
@ -1298,7 +1298,7 @@ public class TestSplitTransactionOnCluster {
|
|||
|
||||
public static class MockMasterWithoutCatalogJanitor extends HMaster {
|
||||
|
||||
public MockMasterWithoutCatalogJanitor(Configuration conf, ConsensusProvider cp)
|
||||
public MockMasterWithoutCatalogJanitor(Configuration conf, CoordinatedStateManager cp)
|
||||
throws IOException, KeeperException,
|
||||
InterruptedException {
|
||||
super(conf, cp);
|
||||
|
|
|
@ -112,7 +112,7 @@ public class TestHLogMethods {
|
|||
public void testEntrySink() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
HLogSplitter splitter = new HLogSplitter(
|
||||
conf, mock(Path.class), mock(FileSystem.class), null, null);
|
||||
conf, mock(Path.class), mock(FileSystem.class), null, null, null);
|
||||
|
||||
EntryBuffers sink = splitter.new EntryBuffers(1*1024*1024);
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue