HBASE-14107 Administrative Task: Provide an API to List all procedures (Stephen Yuan Jiang)

This commit is contained in:
Stephen Yuan Jiang 2015-09-23 21:48:32 -07:00
parent 5e26ae043d
commit 5b7894f92b
31 changed files with 2098 additions and 565 deletions

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@ -1044,6 +1045,14 @@ public interface Admin extends Abortable, Closeable {
final long procId,
final boolean mayInterruptIfRunning) throws IOException;
/**
* List procedures
* @return procedure list
* @throws IOException
*/
ProcedureInfo[] listProcedures()
throws IOException;
/**
* Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
*

View File

@ -1418,6 +1418,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return stub.abortProcedure(controller, request);
}
@Override
public MasterProtos.ListProceduresResponse listProcedures(
RpcController controller,
MasterProtos.ListProceduresRequest request) throws ServiceException {
return stub.listProcedures(controller, request);
}
@Override
public MasterProtos.AddColumnResponse addColumn(
RpcController controller,
@ -1933,7 +1940,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
* If the method returns it means that there is no error, and the 'results' array will
* contain no exception. On error, an exception is thrown, and the 'results' array will
* contain results and exceptions.
* @deprecated since 0.96 - Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
* @deprecated since 0.96 -
* Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
*/
@Override
@Deprecated

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
@ -124,6 +125,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshot
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
@ -144,6 +146,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
@ -2819,6 +2822,28 @@ public class HBaseAdmin implements Admin {
});
}
/**
* List procedures
* @return procedure list
* @throws IOException
*/
@Override
public ProcedureInfo[] listProcedures() throws IOException {
return
executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
@Override
public ProcedureInfo[] call(int callTimeout) throws Exception {
List<ProcedureProtos.Procedure> procList = master.listProcedures(
null, ListProceduresRequest.newBuilder().build()).getProcedureList();
ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
for (int i = 0; i < procList.size(); i++) {
procInfoList[i] = ProcedureInfo.convert(procList.get(i));
}
return procInfoList;
}
});
}
/**
* Get list of table descriptors by namespace
* @param name namespace name

View File

@ -0,0 +1,224 @@
/**
* 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 java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.NonceKey;
/**
* Procedure information
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ProcedureInfo {
private final long procId;
private final String procName;
private final String procOwner;
private final ProcedureState procState;
private final long parentId;
private final ForeignExceptionMessage exception;
private final long lastUpdate;
private final long startTime;
private final byte[] result;
private NonceKey nonceKey = null;
private long clientAckTime = -1;
public ProcedureInfo(
final long procId,
final String procName,
final String procOwner,
final ProcedureState procState,
final long parentId,
final ForeignExceptionMessage exception,
final long lastUpdate,
final long startTime,
final byte[] result) {
this.procId = procId;
this.procName = procName;
this.procOwner = procOwner;
this.procState = procState;
this.parentId = parentId;
this.lastUpdate = lastUpdate;
this.startTime = startTime;
// If the procedure is completed, we should treat exception and result differently
this.exception = exception;
this.result = result;
}
public long getProcId() {
return procId;
}
public String getProcName() {
return procName;
}
public String getProcOwner() {
return procOwner;
}
public ProcedureState getProcState() {
return procState;
}
public boolean hasParentId() {
return (parentId != -1);
}
public long getParentId() {
return parentId;
}
public NonceKey getNonceKey() {
return nonceKey;
}
public void setNonceKey(NonceKey nonceKey) {
this.nonceKey = nonceKey;
}
public boolean isFailed() {
return exception != null;
}
public IOException getException() {
if (isFailed()) {
return ForeignExceptionUtil.toIOException(exception);
}
return null;
}
@InterfaceAudience.Private
public ForeignExceptionMessage getForeignExceptionMessage() {
return exception;
}
public String getExceptionCause() {
assert isFailed();
return exception.getGenericException().getClassName();
}
public String getExceptionMessage() {
assert isFailed();
return exception.getGenericException().getMessage();
}
public String getExceptionFullMessage() {
assert isFailed();
return getExceptionCause() + " - " + getExceptionMessage();
}
public boolean hasResultData() {
return result != null;
}
public byte[] getResult() {
return result;
}
public long getStartTime() {
return startTime;
}
public long getLastUpdate() {
return lastUpdate;
}
public long executionTime() {
return lastUpdate - startTime;
}
@InterfaceAudience.Private
public boolean hasClientAckTime() {
return clientAckTime > 0;
}
@InterfaceAudience.Private
public long getClientAckTime() {
return clientAckTime;
}
@InterfaceAudience.Private
public void setClientAckTime(final long timestamp) {
this.clientAckTime = timestamp;
}
/**
* @return Convert the current {@link ProcedureInfo} into a Protocol Buffers Procedure
* instance.
*/
@InterfaceAudience.Private
public static ProcedureProtos.Procedure convertToProcedureProto(
final ProcedureInfo procInfo) {
ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder();
builder.setClassName(procInfo.getProcName());
builder.setProcId(procInfo.getProcId());
builder.setStartTime(procInfo.getStartTime());
builder.setState(procInfo.getProcState());
builder.setLastUpdate(procInfo.getLastUpdate());
if (procInfo.hasParentId()) {
builder.setParentId(procInfo.getParentId());
}
if (procInfo.getProcOwner() != null) {
builder.setOwner(procInfo.getProcOwner());
}
if (procInfo.isFailed()) {
builder.setException(procInfo.getForeignExceptionMessage());
}
if (procInfo.hasResultData()) {
builder.setResult(ByteStringer.wrap(procInfo.getResult()));
}
return builder.build();
}
/**
* Helper to convert the protobuf object.
* @return Convert the current Protocol Buffers Procedure to {@link ProcedureInfo}
* instance.
*/
@InterfaceAudience.Private
public static ProcedureInfo convert(final ProcedureProtos.Procedure procProto) {
return new ProcedureInfo(
procProto.getProcId(),
procProto.getClassName(),
procProto.getOwner(),
procProto.getState(),
procProto.hasParentId() ? procProto.getParentId() : -1,
procProto.getState() == ProcedureState.ROLLEDBACK ? procProto.getException() : null,
procProto.getLastUpdate(),
procProto.getStartTime(),
procProto.getState() == ProcedureState.FINISHED ? procProto.getResult().toByteArray() : null);
}
}

View File

@ -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.exceptions;
import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Exception thrown when an illegal argument is passed to a function/procedure.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
public class IllegalArgumentIOException extends IOException {
public IllegalArgumentIOException() {
super();
}
public IllegalArgumentIOException(final String message) {
super(message);
}
public IllegalArgumentIOException(final String message, final Throwable t) {
super(message, t);
}
public IllegalArgumentIOException(final Throwable t) {
super(t);
}
}

View File

@ -26,11 +26,12 @@ import java.lang.reflect.Modifier;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
@ -214,6 +215,13 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
return sb.toString();
}
protected String toStringClass() {
StringBuilder sb = new StringBuilder();
toStringClassDetails(sb);
return sb.toString();
}
/**
* Called from {@link #toString()} when interpolating {@link Procedure} state
* @param builder Append current {@link ProcedureState}
@ -395,7 +403,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
protected synchronized boolean setTimeoutFailure() {
if (state == ProcedureState.WAITING_TIMEOUT) {
long timeDiff = EnvironmentEdgeManager.currentTime() - lastUpdate;
setFailure("ProcedureExecutor", new TimeoutException(
setFailure("ProcedureExecutor", new TimeoutIOException(
"Operation timed out after " + StringUtils.humanTimeDiff(timeDiff)));
return true;
}
@ -625,6 +633,37 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
}
}
/**
* Helper to create the ProcedureInfo from Procedure.
*/
@InterfaceAudience.Private
public static ProcedureInfo createProcedureInfo(final Procedure proc, final NonceKey nonceKey) {
RemoteProcedureException exception;
if (proc.hasException()) {
exception = proc.getException();
} else {
exception = null;
}
ProcedureInfo procInfo = new ProcedureInfo(
proc.getProcId(),
proc.toStringClass(),
proc.getOwner(),
proc.getState(),
proc.hasParent() ? proc.getParentProcId() : -1,
exception != null ?
RemoteProcedureException.toProto(exception.getSource(), exception.getCause()) : null,
proc.getLastUpdate(),
proc.getStartTime(),
proc.getResult());
if (nonceKey != null) {
procInfo.setNonceKey(nonceKey);
}
return procInfo;
}
/**
* Helper to convert the procedure to protobuf.
* Used by ProcedureStore implementations.

View File

@ -40,8 +40,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
@ -135,13 +137,13 @@ public class ProcedureExecutor<TEnvironment> {
private static final String EVICT_ACKED_TTL_CONF_KEY ="hbase.procedure.cleaner.acked.evict.ttl";
private static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min
private final Map<Long, ProcedureResult> completed;
private final Map<Long, ProcedureInfo> completed;
private final Map<NonceKey, Long> nonceKeysToProcIdsMap;
private final ProcedureStore store;
private final Configuration conf;
public CompletedProcedureCleaner(final Configuration conf, final ProcedureStore store,
final Map<Long, ProcedureResult> completedMap,
final Map<Long, ProcedureInfo> completedMap,
final Map<NonceKey, Long> nonceKeysToProcIdsMap) {
// set the timeout interval that triggers the periodic-procedure
setTimeout(conf.getInt(CLEANER_INTERVAL_CONF_KEY, DEFAULT_CLEANER_INTERVAL));
@ -163,10 +165,10 @@ public class ProcedureExecutor<TEnvironment> {
final long evictAckTtl = conf.getInt(EVICT_ACKED_TTL_CONF_KEY, DEFAULT_ACKED_EVICT_TTL);
long now = EnvironmentEdgeManager.currentTime();
Iterator<Map.Entry<Long, ProcedureResult>> it = completed.entrySet().iterator();
Iterator<Map.Entry<Long, ProcedureInfo>> it = completed.entrySet().iterator();
while (it.hasNext() && store.isRunning()) {
Map.Entry<Long, ProcedureResult> entry = it.next();
ProcedureResult result = entry.getValue();
Map.Entry<Long, ProcedureInfo> entry = it.next();
ProcedureInfo result = entry.getValue();
// TODO: Select TTL based on Procedure type
if ((result.hasClientAckTime() && (now - result.getClientAckTime()) >= evictAckTtl) ||
@ -212,12 +214,12 @@ public class ProcedureExecutor<TEnvironment> {
}
/**
* Map the the procId returned by submitProcedure(), the Root-ProcID, to the ProcedureResult.
* Map the the procId returned by submitProcedure(), the Root-ProcID, to the ProcedureInfo.
* Once a Root-Procedure completes (success or failure), the result will be added to this map.
* The user of ProcedureExecutor should call getResult(procId) to get the result.
*/
private final ConcurrentHashMap<Long, ProcedureResult> completed =
new ConcurrentHashMap<Long, ProcedureResult>();
private final ConcurrentHashMap<Long, ProcedureInfo> completed =
new ConcurrentHashMap<Long, ProcedureInfo>();
/**
* Map the the procId returned by submitProcedure(), the Root-ProcID, to the RootProcedureState.
@ -364,7 +366,7 @@ public class ProcedureExecutor<TEnvironment> {
}
assert !rollbackStack.containsKey(proc.getProcId());
procedures.remove(proc.getProcId());
completed.put(proc.getProcId(), newResultFromProcedure(proc));
completed.put(proc.getProcId(), Procedure.createProcedureInfo(proc, proc.getNonceKey()));
continue;
}
@ -571,6 +573,26 @@ public class ProcedureExecutor<TEnvironment> {
return this.listeners.remove(listener);
}
/**
* List procedures.
* @return the procedures in a list
*/
public List<ProcedureInfo> listProcedures() {
List<ProcedureInfo> procedureLists =
new ArrayList<ProcedureInfo>(procedures.size() + completed.size());
for (java.util.Map.Entry<Long, Procedure> p: procedures.entrySet()) {
procedureLists.add(Procedure.createProcedureInfo(p.getValue(), null));
}
for (java.util.Map.Entry<Long, ProcedureInfo> e: completed.entrySet()) {
// Note: The procedure could show up twice in the list with different state, as
// it could complete after we walk through procedures list and insert into
// procedureList - it is ok, as we will use the information in the ProcedureInfo
// to figure it out; to prevent this would increase the complexity of the logic.
procedureLists.add(e.getValue());
}
return procedureLists;
}
/**
* Add a new root-procedure to the executor.
* @param proc the new procedure to execute.
@ -643,7 +665,7 @@ public class ProcedureExecutor<TEnvironment> {
return currentProcId;
}
public ProcedureResult getResult(final long procId) {
public ProcedureInfo getResult(final long procId) {
return completed.get(procId);
}
@ -676,7 +698,7 @@ public class ProcedureExecutor<TEnvironment> {
* @param procId the ID of the procedure to remove
*/
public void removeResult(final long procId) {
ProcedureResult result = completed.get(procId);
ProcedureInfo result = completed.get(procId);
if (result == null) {
assert !procedures.containsKey(procId) : "procId=" + procId + " is still running";
if (LOG.isDebugEnabled()) {
@ -718,7 +740,7 @@ public class ProcedureExecutor<TEnvironment> {
return false;
}
public Map<Long, ProcedureResult> getResults() {
public Map<Long, ProcedureInfo> getResults() {
return Collections.unmodifiableMap(completed);
}
@ -1033,7 +1055,7 @@ public class ProcedureExecutor<TEnvironment> {
if (subproc == null) {
String msg = "subproc[" + i + "] is null, aborting the procedure";
procedure.setFailure(new RemoteProcedureException(msg,
new IllegalArgumentException(msg)));
new IllegalArgumentIOException(msg)));
subprocs = null;
break;
}
@ -1212,7 +1234,7 @@ public class ProcedureExecutor<TEnvironment> {
}
// update the executor internal state maps
completed.put(proc.getProcId(), newResultFromProcedure(proc));
completed.put(proc.getProcId(), Procedure.createProcedureInfo(proc, proc.getNonceKey()));
rollbackStack.remove(proc.getProcId());
procedures.remove(proc.getProcId());
@ -1228,8 +1250,8 @@ public class ProcedureExecutor<TEnvironment> {
sendProcedureFinishedNotification(proc.getProcId());
}
public Pair<ProcedureResult, Procedure> getResultOrProcedure(final long procId) {
ProcedureResult result = completed.get(procId);
public Pair<ProcedureInfo, Procedure> getResultOrProcedure(final long procId) {
ProcedureInfo result = completed.get(procId);
Procedure proc = null;
if (result == null) {
proc = procedures.get(procId);
@ -1239,13 +1261,4 @@ public class ProcedureExecutor<TEnvironment> {
}
return new Pair(result, proc);
}
private static ProcedureResult newResultFromProcedure(final Procedure proc) {
if (proc.isFailed()) {
return new ProcedureResult(
proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getException());
}
return new ProcedureResult(
proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getResult());
}
}

View File

@ -1,115 +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.procedure2;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.NonceKey;
/**
* Once a Procedure completes the ProcedureExecutor takes all the useful
* information of the procedure (e.g. exception/result) and creates a ProcedureResult.
* The user of the Procedure framework will get the procedure result with
* procedureExecutor.getResult(procId)
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ProcedureResult {
private final NonceKey nonceKey;
private final RemoteProcedureException exception;
private final long lastUpdate;
private final long startTime;
private final byte[] result;
private long clientAckTime = -1;
public ProcedureResult(
final NonceKey nonceKey,
final long startTime,
final long lastUpdate,
final RemoteProcedureException exception) {
this(nonceKey, exception, lastUpdate, startTime, null);
}
public ProcedureResult(
final NonceKey nonceKey,
final long startTime,
final long lastUpdate,
final byte[] result) {
this(nonceKey, null, lastUpdate, startTime, result);
}
public ProcedureResult(
final NonceKey nonceKey,
final RemoteProcedureException exception,
final long lastUpdate,
final long startTime,
final byte[] result) {
this.nonceKey = nonceKey;
this.exception = exception;
this.lastUpdate = lastUpdate;
this.startTime = startTime;
this.result = result;
}
public NonceKey getNonceKey() {
return nonceKey;
}
public boolean isFailed() {
return exception != null;
}
public RemoteProcedureException getException() {
return exception;
}
public boolean hasResultData() {
return result != null;
}
public byte[] getResult() {
return result;
}
public long getStartTime() {
return startTime;
}
public long getLastUpdate() {
return lastUpdate;
}
public long executionTime() {
return lastUpdate - startTime;
}
public boolean hasClientAckTime() {
return clientAckTime > 0;
}
public long getClientAckTime() {
return clientAckTime;
}
@InterfaceAudience.Private
protected void setClientAckTime(final long timestamp) {
this.clientAckTime = timestamp;
}
}

View File

@ -28,15 +28,18 @@ 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.ProcedureInfo;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class ProcedureTestingUtility {
private static final Log LOG = LogFactory.getLog(ProcedureTestingUtility.class);
@ -166,23 +169,42 @@ public class ProcedureTestingUtility {
public static <TEnv> void assertProcNotFailed(ProcedureExecutor<TEnv> procExecutor,
long procId) {
ProcedureResult result = procExecutor.getResult(procId);
ProcedureInfo result = procExecutor.getResult(procId);
assertTrue("expected procedure result", result != null);
assertProcNotFailed(result);
}
public static void assertProcNotFailed(final ProcedureResult result) {
Exception exception = result.getException();
String msg = exception != null ? exception.toString() : "no exception found";
public static void assertProcNotFailed(final ProcedureInfo result) {
ForeignExceptionMessage exception = result.getForeignExceptionMessage();
String msg = exception != null ? result.getExceptionFullMessage() : "no exception found";
assertFalse(msg, result.isFailed());
}
public static void assertIsAbortException(final ProcedureResult result) {
LOG.info(result.getException());
public static void assertIsAbortException(final ProcedureInfo result) {
assertEquals(true, result.isFailed());
Throwable cause = result.getException().getCause();
assertTrue("expected abort exception, got "+ cause,
cause instanceof ProcedureAbortedException);
LOG.info(result.getExceptionFullMessage());
Throwable cause = getExceptionCause(result);
assertTrue("expected abort exception, got "+ cause, cause instanceof ProcedureAbortedException);
}
public static void assertIsTimeoutException(final ProcedureInfo result) {
assertEquals(true, result.isFailed());
LOG.info(result.getExceptionFullMessage());
Throwable cause = getExceptionCause(result);
assertTrue("expected TimeoutIOException, got " + cause, cause instanceof TimeoutIOException);
}
public static void assertIsIllegalArgumentException(final ProcedureInfo result) {
assertEquals(true, result.isFailed());
LOG.info(result.getExceptionFullMessage());
Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
assertTrue("expected IllegalArgumentIOException, got " + cause,
cause instanceof IllegalArgumentIOException);
}
public static Throwable getExceptionCause(final ProcedureInfo procInfo) {
assert procInfo.getForeignExceptionMessage() != null;
return RemoteProcedureException.fromProto(procInfo.getForeignExceptionMessage()).getCause();
}
public static class TestProcedure extends Procedure<Void> {

View File

@ -21,29 +21,25 @@ package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.junit.After;
import org.junit.Before;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({MasterTests.class, SmallTests.class})
public class TestProcedureExecution {
@ -81,7 +77,7 @@ public class TestProcedureExecution {
fs.delete(logDir, true);
}
private static class TestProcedureException extends Exception {
private static class TestProcedureException extends IOException {
public TestProcedureException(String msg) { super(msg); }
}
@ -142,11 +138,9 @@ public class TestProcedureExecution {
// subProc1 has a "null" subprocedure which is catched as InvalidArgument
// failed state with 2 execute and 2 rollback
LOG.info(state);
ProcedureResult result = procExecutor.getResult(rootId);
LOG.info(result.getException());
ProcedureInfo result = procExecutor.getResult(rootId);
assertTrue(state.toString(), result.isFailed());
assertTrue(result.getException().toString(),
result.getException().getCause() instanceof IllegalArgumentException);
ProcedureTestingUtility.assertIsIllegalArgumentException(result);
assertEquals(state.toString(), 4, state.size());
assertEquals("rootProc-execute", state.get(0));
@ -165,7 +159,7 @@ public class TestProcedureExecution {
// successful state, with 3 execute
LOG.info(state);
ProcedureResult result = procExecutor.getResult(rootId);
ProcedureInfo result = procExecutor.getResult(rootId);
ProcedureTestingUtility.assertProcNotFailed(result);
assertEquals(state.toString(), 3, state.size());
}
@ -181,11 +175,12 @@ public class TestProcedureExecution {
// the 3rd proc fail, rollback after 2 successful execution
LOG.info(state);
ProcedureResult result = procExecutor.getResult(rootId);
LOG.info(result.getException());
ProcedureInfo result = procExecutor.getResult(rootId);
assertTrue(state.toString(), result.isFailed());
assertTrue(result.getException().toString(),
result.getException().getCause() instanceof TestProcedureException);
LOG.info(result.getExceptionFullMessage());
Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
assertTrue("expected TestProcedureException, got " + cause,
cause instanceof TestProcedureException);
assertEquals(state.toString(), 6, state.size());
assertEquals("rootProc-execute", state.get(0));
@ -224,11 +219,12 @@ public class TestProcedureExecution {
public void testRollbackRetriableFailure() {
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, new TestFaultyRollback());
ProcedureResult result = procExecutor.getResult(procId);
LOG.info(result.getException());
ProcedureInfo result = procExecutor.getResult(procId);
assertTrue("expected a failure", result.isFailed());
assertTrue(result.getException().toString(),
result.getException().getCause() instanceof TestProcedureException);
LOG.info(result.getExceptionFullMessage());
Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
assertTrue("expected TestProcedureException, got " + cause,
cause instanceof TestProcedureException);
}
public static class TestWaitingProcedure extends SequentialProcedure<Void> {
@ -307,11 +303,9 @@ public class TestProcedureExecution {
long execTime = EnvironmentEdgeManager.currentTime() - startTime;
LOG.info(state);
assertTrue("we didn't wait enough execTime=" + execTime, execTime >= PROC_TIMEOUT_MSEC);
ProcedureResult result = procExecutor.getResult(rootId);
LOG.info(result.getException());
ProcedureInfo result = procExecutor.getResult(rootId);
assertTrue(state.toString(), result.isFailed());
assertTrue(result.getException().toString(),
result.getException().getCause() instanceof TimeoutException);
ProcedureTestingUtility.assertIsTimeoutException(result);
assertEquals(state.toString(), 2, state.size());
assertEquals("wproc-execute", state.get(0));
assertEquals("wproc-rollback", state.get(1));
@ -324,11 +318,9 @@ public class TestProcedureExecution {
proc.setTimeout(2500);
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
LOG.info(state);
ProcedureResult result = procExecutor.getResult(rootId);
LOG.info(result.getException());
ProcedureInfo result = procExecutor.getResult(rootId);
assertTrue(state.toString(), result.isFailed());
assertTrue(result.getException().toString(),
result.getException().getCause() instanceof TimeoutException);
ProcedureTestingUtility.assertIsTimeoutException(result);
assertEquals(state.toString(), 4, state.size());
assertEquals("wproc-execute", state.get(0));
assertEquals("wproc-child-execute", state.get(1));

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -197,7 +198,7 @@ public class TestProcedureRecovery {
long restartTs = EnvironmentEdgeManager.currentTime();
restart();
waitProcedure(procId);
ProcedureResult result = procExecutor.getResult(procId);
ProcedureInfo result = procExecutor.getResult(procId);
assertTrue(result.getLastUpdate() > restartTs);
ProcedureTestingUtility.assertProcNotFailed(result);
assertEquals(1, Bytes.toInt(result.getResult()));
@ -236,7 +237,7 @@ public class TestProcedureRecovery {
assertTrue(procExecutor.isRunning());
// The procedure is completed
ProcedureResult result = procExecutor.getResult(procId);
ProcedureInfo result = procExecutor.getResult(procId);
ProcedureTestingUtility.assertProcNotFailed(result);
}
@ -283,7 +284,7 @@ public class TestProcedureRecovery {
waitProcedure(procId);
// The procedure is completed
ProcedureResult result = procExecutor.getResult(procId);
ProcedureInfo result = procExecutor.getResult(procId);
ProcedureTestingUtility.assertIsAbortException(result);
}
@ -304,7 +305,7 @@ public class TestProcedureRecovery {
long procId2 = ProcedureTestingUtility.submitAndWait(procExecutor, proc2, nonceGroup, nonce);
assertTrue(procId == procId2);
ProcedureResult result = procExecutor.getResult(procId2);
ProcedureInfo result = procExecutor.getResult(procId2);
ProcedureTestingUtility.assertProcNotFailed(result);
}
@ -451,7 +452,7 @@ public class TestProcedureRecovery {
assertTrue(procExecutor.isRunning());
// The procedure is completed
ProcedureResult result = procExecutor.getResult(procId);
ProcedureInfo result = procExecutor.getResult(procId);
ProcedureTestingUtility.assertProcNotFailed(result);
assertEquals(15, Bytes.toInt(result.getResult()));
}
@ -505,7 +506,7 @@ public class TestProcedureRecovery {
assertTrue(procExecutor.isRunning());
// The procedure is completed
ProcedureResult result = procExecutor.getResult(procId);
ProcedureInfo result = procExecutor.getResult(procId);
ProcedureTestingUtility.assertIsAbortException(result);
}

View File

@ -35,14 +35,11 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

View File

@ -30,6 +30,7 @@ import "HBase.proto";
import "Client.proto";
import "ClusterStatus.proto";
import "ErrorHandling.proto";
import "Procedure.proto";
import "Quota.proto";
/* Column-level protobufs */
@ -440,6 +441,13 @@ message AbortProcedureResponse {
required bool is_procedure_aborted = 1;
}
message ListProceduresRequest {
}
message ListProceduresResponse {
repeated Procedure procedure = 1;
}
message SetQuotaRequest {
optional string user_name = 1;
optional string user_group = 2;
@ -720,4 +728,8 @@ service MasterService {
/** Abort a procedure */
rpc AbortProcedure(AbortProcedureRequest)
returns(AbortProcedureResponse);
/** returns a list of procedures */
rpc ListProcedures(ListProceduresRequest)
returns(ListProceduresResponse);
}

View File

@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
@ -74,7 +75,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
@ -2485,6 +2485,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return this.procedureExecutor.abort(procId, mayInterruptIfRunning);
}
@Override
public List<ProcedureInfo> listProcedures() throws IOException {
return this.procedureExecutor.listProcedures();
}
@Override
public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
ensureNamespaceExists(name);

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -51,7 +52,6 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
@ -123,6 +123,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneReq
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
@ -1032,15 +1034,15 @@ public class MasterRpcServices extends RSRpcServices
master.checkInitialized();
GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();
Pair<ProcedureResult, Procedure> v = master.getMasterProcedureExecutor()
Pair<ProcedureInfo, Procedure> v = master.getMasterProcedureExecutor()
.getResultOrProcedure(request.getProcId());
if (v.getFirst() != null) {
ProcedureResult result = v.getFirst();
ProcedureInfo result = v.getFirst();
builder.setState(GetProcedureResultResponse.State.FINISHED);
builder.setStartTime(result.getStartTime());
builder.setLastUpdate(result.getLastUpdate());
if (result.isFailed()) {
builder.setException(result.getException().convert());
builder.setException(result.getForeignExceptionMessage());
}
if (result.hasResultData()) {
builder.setResult(ByteStringer.wrap(result.getResult()));
@ -1088,6 +1090,22 @@ public class MasterRpcServices extends RSRpcServices
}
}
@Override
public ListProceduresResponse listProcedures(
RpcController rpcController,
ListProceduresRequest request) throws ServiceException {
try {
ListProceduresResponse.Builder response =
ListProceduresResponse.newBuilder();
for(ProcedureInfo p: master.listProcedures()) {
response.addProcedure(ProcedureInfo.convertToProcedureProto(p));
}
return response.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override
public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c,
ListTableDescriptorsByNamespaceRequest request) throws ServiceException {

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
@ -343,6 +344,13 @@ public interface MasterServices extends Server {
*/
public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException;
/**
* List procedures
* @return procedure list
* @throws IOException
*/
public List<ProcedureInfo> listProcedures() throws IOException;
/**
* Get list of table descriptors by namespace
* @param name namespace name

View File

@ -24,12 +24,12 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
@ -78,11 +78,12 @@ public final class ProcedureSyncWait {
// Dev Consideration: are we waiting forever, or we can set up some timeout value?
Threads.sleepWithoutInterrupt(250);
}
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
if (result != null) {
if (result.isFailed()) {
// If the procedure fails, we should always have an exception captured. Throw it.
throw result.getException().unwrapRemoteException();
throw RemoteProcedureException.fromProto(
result.getForeignExceptionMessage()).unwrapRemoteException();
}
return result.getResult();
} else {

View File

@ -26,7 +26,6 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.constraint.ConstraintException;
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.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.ClientTests;
@ -738,4 +737,10 @@ public class TestAdmin2 {
boolean abortResult = admin.abortProcedure(procId, true);
assertFalse(abortResult);
}
@Test(timeout = 300000)
public void testListProcedures() throws Exception {
ProcedureInfo[] procList = admin.listProcedures();
assertTrue(procList.length >= 0);
}
}

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaMockingUtil;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
@ -441,6 +442,11 @@ public class TestCatalogJanitor {
return false; //To change body of implemented methods use File | Settings | File Templates.
}
@Override
public List<ProcedureInfo> listProcedures() throws IOException {
return null; //To change body of implemented methods use File | Settings | File Templates.
}
@Override
public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
return null; //To change body of implemented methods use File | Settings | File Templates.

View File

@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -153,10 +153,11 @@ public class TestAddColumnFamilyProcedure {
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// Second add should fail with InvalidFamilyOperationException
ProcedureResult result = procExec.getResult(procId2);
ProcedureInfo result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Add failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
LOG.debug("Add failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
// Do the same add the existing column family - this time offline
UTIL.getHBaseAdmin().disableTable(tableName);
@ -170,8 +171,9 @@ public class TestAddColumnFamilyProcedure {
// Second add should fail with InvalidFamilyOperationException
result = procExec.getResult(procId3);
assertTrue(result.isFailed());
LOG.debug("Add failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
LOG.debug("Add failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)

View File

@ -32,9 +32,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceExistException;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -126,10 +126,11 @@ public class TestCreateNamespaceProcedure {
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// Second create should fail with NamespaceExistException
ProcedureResult result = procExec.getResult(procId2);
ProcedureInfo result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceExistException);
LOG.debug("Create namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceExistException);
}
@Test(timeout=60000)
@ -144,10 +145,11 @@ public class TestCreateNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceExistException);
LOG.debug("Create namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceExistException);
}
@Test(timeout=60000)
@ -166,10 +168,10 @@ public class TestCreateNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
LOG.debug("Create namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)
@ -188,10 +190,10 @@ public class TestCreateNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
LOG.debug("Create namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)

View File

@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -151,10 +151,11 @@ public class TestDeleteColumnFamilyProcedure {
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// Second delete should fail with InvalidFamilyOperationException
ProcedureResult result = procExec.getResult(procId2);
ProcedureInfo result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete online failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
LOG.debug("Delete online failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
// Try again, this time with table disabled.
UTIL.getHBaseAdmin().disableTable(tableName);
@ -167,8 +168,9 @@ public class TestDeleteColumnFamilyProcedure {
// Expect fail with InvalidFamilyOperationException
result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete offline failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
LOG.debug("Delete offline failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
@ -219,10 +221,11 @@ public class TestDeleteColumnFamilyProcedure {
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
ProcedureInfo result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Delete failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
LOG.debug("Delete failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)

View File

@ -32,10 +32,10 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -124,10 +124,11 @@ public class TestDeleteNamespaceProcedure {
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Expect fail with NamespaceNotFoundException
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Delete namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceNotFoundException);
LOG.debug("Delete namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceNotFoundException);
}
@Test(timeout=60000)
@ -141,10 +142,10 @@ public class TestDeleteNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Delete namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
LOG.debug("Delete namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)
@ -163,10 +164,10 @@ public class TestDeleteNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Delete namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
LOG.debug("Delete namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)

View File

@ -25,11 +25,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -142,10 +142,10 @@ public class TestDeleteTableProcedure {
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
// Second delete should fail with TableNotFound
ProcedureResult result = procExec.getResult(procId2);
ProcedureInfo result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotFoundException);
LOG.debug("Delete failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotFoundException);
}
@Test(timeout=60000)

View File

@ -26,10 +26,10 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -126,10 +126,11 @@ public class TestDisableTableProcedure {
procExec.getEnvironment(), tableName, false), nonceGroup + 1, nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureResult result = procExec.getResult(procId2);
ProcedureInfo result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Disable failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotEnabledException);
LOG.debug("Disable failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotEnabledException);
// Disable the table - expect failure from ProcedurePrepareLatch
try {

View File

@ -26,10 +26,10 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -141,10 +141,11 @@ public class TestEnableTableProcedure {
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
ProcedureInfo result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Enable failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotDisabledException);
LOG.debug("Enable failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotDisabledException);
// Enable the table with skipping table state check flag (simulate recovery scenario)
long procId2 = procExec.submitProcedure(

View File

@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -146,10 +146,11 @@ public class TestModifyColumnFamilyProcedure {
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
ProcedureInfo result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Modify failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
LOG.debug("Modify failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -150,10 +150,11 @@ public class TestModifyNamespaceProcedure {
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Expect fail with NamespaceNotFoundException
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("modify namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceNotFoundException);
LOG.debug("modify namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceNotFoundException);
}
@Test(timeout=60000)
@ -175,10 +176,10 @@ public class TestModifyNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Modify namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
LOG.debug("Modify namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)
@ -200,10 +201,10 @@ public class TestModifyNamespaceProcedure {
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Modify namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
LOG.debug("Modify namespace failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout = 60000)

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.master.procedure;
import java.util.Random;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -27,11 +28,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -181,6 +184,41 @@ public class TestProcedureAdmin {
assertFalse(abortResult);
}
@Test(timeout=60000)
public void testListProcedure() throws Exception {
final TableName tableName = TableName.valueOf("testListProcedure");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
long procId = procExec.submitProcedure(
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
List<ProcedureInfo> listProcedures = procExec.listProcedures();
assertTrue(listProcedures.size() >= 1);
boolean found = false;
for (ProcedureInfo procInfo: listProcedures) {
if (procInfo.getProcId() == procId) {
assertTrue(procInfo.getProcState() == ProcedureState.RUNNABLE);
found = true;
} else {
assertTrue(procInfo.getProcState() == ProcedureState.FINISHED);
}
}
assertTrue(found);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
listProcedures = procExec.listProcedures();
for (ProcedureInfo procInfo: listProcedures) {
assertTrue(procInfo.getProcState() == ProcedureState.FINISHED);
}
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}

View File

@ -25,11 +25,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -102,10 +102,10 @@ public class TestTruncateTableProcedure {
new TruncateTableProcedure(procExec.getEnvironment(), tableName, true));
// Second delete should fail with TableNotFound
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Truncate failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotFoundException);
LOG.debug("Truncate failed with exception: " + result.getExceptionFullMessage());
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotFoundException);
}
@Test(timeout=60000)
@ -119,10 +119,11 @@ public class TestTruncateTableProcedure {
new TruncateTableProcedure(procExec.getEnvironment(), tableName, false));
// Second delete should fail with TableNotDisabled
ProcedureResult result = procExec.getResult(procId);
ProcedureInfo result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Truncate failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotDisabledException);
LOG.debug("Truncate failed with exception: " + result.getExceptionFullMessage());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotDisabledException);
}
@Test(timeout=60000)