HBASE-13202 Procedure v2 - core framework
This commit is contained in:
parent
d20c08ea58
commit
04246c6c3d
|
@ -120,7 +120,7 @@ public class StreamUtils {
|
|||
|
||||
/**
|
||||
* Reads a varInt value stored in an array.
|
||||
*
|
||||
*
|
||||
* @param input
|
||||
* Input array where the varInt is available
|
||||
* @param offset
|
||||
|
@ -198,4 +198,14 @@ public class StreamUtils {
|
|||
out.write((byte) (0xff & (v >> 8)));
|
||||
out.write((byte) (0xff & v));
|
||||
}
|
||||
|
||||
public static long readLong(InputStream in) throws IOException {
|
||||
long result = 0;
|
||||
for (int shift = 56; shift >= 0; shift -= 8) {
|
||||
long x = in.read();
|
||||
if (x < 0) throw new IOException("EOF");
|
||||
result |= (x << shift);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,109 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
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.ErrorHandlingProtos.GenericExceptionMessage;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage;
|
||||
|
||||
/**
|
||||
* Helper to convert Exceptions and StackTraces from/to protobuf.
|
||||
* (see ErrorHandling.proto for the internal of the proto messages)
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public final class ForeignExceptionUtil {
|
||||
private ForeignExceptionUtil() { }
|
||||
|
||||
public static IOException toIOException(final ForeignExceptionMessage eem) {
|
||||
GenericExceptionMessage gem = eem.getGenericException();
|
||||
StackTraceElement[] trace = toStackTrace(gem.getTraceList());
|
||||
RemoteException re = new RemoteException(gem.getClassName(), gem.getMessage());
|
||||
re.setStackTrace(trace);
|
||||
return re.unwrapRemoteException();
|
||||
}
|
||||
|
||||
public static ForeignExceptionMessage toProtoForeignException(String source, Throwable t) {
|
||||
GenericExceptionMessage.Builder gemBuilder = GenericExceptionMessage.newBuilder();
|
||||
gemBuilder.setClassName(t.getClass().getName());
|
||||
if (t.getMessage() != null) {
|
||||
gemBuilder.setMessage(t.getMessage());
|
||||
}
|
||||
// set the stack trace, if there is one
|
||||
List<StackTraceElementMessage> stack = toProtoStackTraceElement(t.getStackTrace());
|
||||
if (stack != null) {
|
||||
gemBuilder.addAllTrace(stack);
|
||||
}
|
||||
GenericExceptionMessage payload = gemBuilder.build();
|
||||
ForeignExceptionMessage.Builder exception = ForeignExceptionMessage.newBuilder();
|
||||
exception.setGenericException(payload).setSource(source);
|
||||
return exception.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a stack trace to list of {@link StackTraceElement}.
|
||||
* @param trace the stack trace to convert to protobuf message
|
||||
* @return <tt>null</tt> if the passed stack is <tt>null</tt>.
|
||||
*/
|
||||
public static List<StackTraceElementMessage> toProtoStackTraceElement(StackTraceElement[] trace) {
|
||||
// if there is no stack trace, ignore it and just return the message
|
||||
if (trace == null) return null;
|
||||
// build the stack trace for the message
|
||||
List<StackTraceElementMessage> pbTrace = new ArrayList<StackTraceElementMessage>(trace.length);
|
||||
for (StackTraceElement elem : trace) {
|
||||
StackTraceElementMessage.Builder stackBuilder = StackTraceElementMessage.newBuilder();
|
||||
stackBuilder.setDeclaringClass(elem.getClassName());
|
||||
if (elem.getFileName() != null) {
|
||||
stackBuilder.setFileName(elem.getFileName());
|
||||
}
|
||||
stackBuilder.setLineNumber(elem.getLineNumber());
|
||||
stackBuilder.setMethodName(elem.getMethodName());
|
||||
pbTrace.add(stackBuilder.build());
|
||||
}
|
||||
return pbTrace;
|
||||
}
|
||||
|
||||
/**
|
||||
* Unwind a serialized array of {@link StackTraceElementMessage}s to a
|
||||
* {@link StackTraceElement}s.
|
||||
* @param traceList list that was serialized
|
||||
* @return the deserialized list or <tt>null</tt> if it couldn't be unwound (e.g. wasn't set on
|
||||
* the sender).
|
||||
*/
|
||||
public static StackTraceElement[] toStackTrace(List<StackTraceElementMessage> traceList) {
|
||||
if (traceList == null || traceList.size() == 0) {
|
||||
return new StackTraceElement[0]; // empty array
|
||||
}
|
||||
StackTraceElement[] trace = new StackTraceElement[traceList.size()];
|
||||
for (int i = 0; i < traceList.size(); i++) {
|
||||
StackTraceElementMessage elem = traceList.get(i);
|
||||
trace[i] = new StackTraceElement(
|
||||
elem.getDeclaringClass(), elem.getMethodName(),
|
||||
elem.hasFileName() ? elem.getFileName() : null,
|
||||
elem.getLineNumber());
|
||||
}
|
||||
return trace;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,181 @@
|
|||
<?xml version="1.0"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<!--
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
-->
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<artifactId>hbase</artifactId>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
<relativePath>..</relativePath>
|
||||
</parent>
|
||||
|
||||
<artifactId>hbase-procedure</artifactId>
|
||||
<name>HBase - Procedure</name>
|
||||
<description>Procedure Framework</description>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-site-plugin</artifactId>
|
||||
<configuration>
|
||||
<skip>true</skip>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<!-- Make a jar and put the sources in the jar -->
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-source-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<!--Make it so assembly:single does nothing in here-->
|
||||
<artifactId>maven-assembly-plugin</artifactId>
|
||||
<version>${maven.assembly.version}</version>
|
||||
<configuration>
|
||||
<skipAssembly>true</skipAssembly>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<!-- Always skip the second part executions, since we only run
|
||||
simple unit tests in this module. -->
|
||||
<executions>
|
||||
<execution>
|
||||
<id>secondPartTestsExecution</id>
|
||||
<phase>test</phase>
|
||||
<goals>
|
||||
<goal>test</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<skip>true</skip>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-annotations</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-protocol</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-logging</groupId>
|
||||
<artifactId>commons-logging</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<profiles>
|
||||
<!-- Profiles for building against different hadoop versions -->
|
||||
<profile>
|
||||
<id>hadoop-1.1</id>
|
||||
<activation>
|
||||
<property>
|
||||
<!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
|
||||
<!--h1--><name>hadoop.profile</name><value>1.1</value>
|
||||
</property>
|
||||
</activation>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-core</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>hadoop-1.0</id>
|
||||
<activation>
|
||||
<property>
|
||||
<name>hadoop.profile</name>
|
||||
<value>1.0</value>
|
||||
</property>
|
||||
</activation>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-core</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</profile>
|
||||
<!--
|
||||
profile for building against Hadoop 2.0.0-alpha. Activate using:
|
||||
mvn -Dhadoop.profile=2.0
|
||||
-->
|
||||
<profile>
|
||||
<id>hadoop-2.0</id>
|
||||
<activation>
|
||||
<property>
|
||||
<!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
|
||||
<!--h2--><name>!hadoop.profile</name>
|
||||
</property>
|
||||
</activation>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</profile>
|
||||
<!--
|
||||
profile for building against Hadoop 3.0.x. Activate using:
|
||||
mvn -Dhadoop.profile=3.0
|
||||
-->
|
||||
<profile>
|
||||
<id>hadoop-3.0</id>
|
||||
<activation>
|
||||
<property>
|
||||
<name>hadoop.profile</name>
|
||||
<value>3.0</value>
|
||||
</property>
|
||||
</activation>
|
||||
<properties>
|
||||
<hadoop.version>3.0-SNAPSHOT</hadoop.version>
|
||||
</properties>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</profile>
|
||||
</profiles>
|
||||
</project>
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class OnePhaseProcedure<TEnvironment> extends Procedure<TEnvironment> {
|
||||
// TODO (e.g. used by online snapshots)
|
||||
}
|
|
@ -0,0 +1,680 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.lang.reflect.Constructor;
|
||||
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.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
/**
|
||||
* Base Procedure class responsible to handle the Procedure Metadata
|
||||
* e.g. state, startTime, lastUpdate, stack-indexes, ...
|
||||
*
|
||||
* execute() is called each time the procedure is executed.
|
||||
* it may be called multiple times in case of failure and restart, so the
|
||||
* code must be idempotent.
|
||||
* the return is a set of sub-procedures or null in case the procedure doesn't
|
||||
* have sub-procedures. Once the sub-procedures are successfully completed
|
||||
* the execute() method is called again, you should think at it as a stack:
|
||||
* -> step 1
|
||||
* ---> step 2
|
||||
* -> step 1
|
||||
*
|
||||
* rollback() is called when the procedure or one of the sub-procedures is failed.
|
||||
* the rollback step is supposed to cleanup the resources created during the
|
||||
* execute() step. in case of failure and restart rollback() may be called
|
||||
* multiple times, so the code must be idempotent.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
|
||||
// unchanged after initialization
|
||||
private String owner = null;
|
||||
private Long parentProcId = null;
|
||||
private Long procId = null;
|
||||
private long startTime;
|
||||
|
||||
// runtime state, updated every operation
|
||||
private ProcedureState state = ProcedureState.INITIALIZING;
|
||||
private Integer timeout = null;
|
||||
private int[] stackIndexes = null;
|
||||
private int childrenLatch = 0;
|
||||
private long lastUpdate;
|
||||
|
||||
private RemoteProcedureException exception = null;
|
||||
private byte[] result = null;
|
||||
|
||||
/**
|
||||
* The main code of the procedure. It must be idempotent since execute()
|
||||
* may be called multiple time in case of machine failure in the middle
|
||||
* of the execution.
|
||||
* @return a set of sub-procedures or null if there is nothing else to execute.
|
||||
*/
|
||||
protected abstract Procedure[] execute(TEnvironment env)
|
||||
throws ProcedureYieldException;
|
||||
|
||||
/**
|
||||
* The code to undo what done by the execute() code.
|
||||
* It is called when the procedure or one of the sub-procedure failed or an
|
||||
* abort was requested. It should cleanup all the resources created by
|
||||
* the execute() call. The implementation must be idempotent since rollback()
|
||||
* may be called multiple time in case of machine failure in the middle
|
||||
* of the execution.
|
||||
* @throws IOException temporary failure, the rollback will retry later
|
||||
*/
|
||||
protected abstract void rollback(TEnvironment env)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* The abort() call is asynchronous and each procedure must decide how to deal
|
||||
* with that, if they want to be abortable. The simplest implementation
|
||||
* is to have an AtomicBoolean set in the abort() method and then the execute()
|
||||
* will check if the abort flag is set or not.
|
||||
* abort() may be called multiple times from the client, so the implementation
|
||||
* must be idempotent.
|
||||
*
|
||||
* NOTE: abort() is not like Thread.interrupt() it is just a notification
|
||||
* that allows the procedure implementor where to abort to avoid leak and
|
||||
* have a better control on what was executed and what not.
|
||||
*/
|
||||
protected abstract boolean abort(TEnvironment env);
|
||||
|
||||
/**
|
||||
* The user-level code of the procedure may have some state to
|
||||
* persist (e.g. input arguments) to be able to resume on failure.
|
||||
* @param stream the stream that will contain the user serialized data
|
||||
*/
|
||||
protected abstract void serializeStateData(final OutputStream stream)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Called on store load to allow the user to decode the previously serialized
|
||||
* state.
|
||||
* @param stream the stream that contains the user serialized data
|
||||
*/
|
||||
protected abstract void deserializeStateData(final InputStream stream)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* The user should override this method, and try to take a lock if necessary.
|
||||
* A lock can be anything, and it is up to the implementor.
|
||||
* Example: in our Master we can execute request in parallel for different tables
|
||||
* create t1 and create t2 can be executed at the same time.
|
||||
* anything else on t1/t2 is queued waiting that specific table create to happen.
|
||||
*
|
||||
* @return true if the lock was acquired and false otherwise
|
||||
*/
|
||||
protected boolean acquireLock(final TEnvironment env) {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* The user should override this method, and release lock if necessary.
|
||||
*/
|
||||
protected void releaseLock(final TEnvironment env) {
|
||||
// no-op
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when the procedure is loaded for replay.
|
||||
* The procedure implementor may use this method to perform some quick
|
||||
* operation before replay.
|
||||
* e.g. failing the procedure if the state on replay may be unknown.
|
||||
*/
|
||||
protected void beforeReplay(final TEnvironment env) {
|
||||
// no-op
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when the procedure is marked as completed (success or rollback).
|
||||
* The procedure implementor may use this method to cleanup in-memory states.
|
||||
* This operation will not be retried on failure.
|
||||
*/
|
||||
protected void completionCleanup(final TEnvironment env) {
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
toStringClassDetails(sb);
|
||||
|
||||
if (procId != null) {
|
||||
sb.append(" id=");
|
||||
sb.append(getProcId());
|
||||
}
|
||||
|
||||
if (hasParent()) {
|
||||
sb.append(" parent=");
|
||||
sb.append(getParentProcId());
|
||||
}
|
||||
|
||||
if (hasOwner()) {
|
||||
sb.append(" owner=");
|
||||
sb.append(getOwner());
|
||||
}
|
||||
|
||||
sb.append(" state=");
|
||||
sb.append(getState());
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Extend the toString() information with the procedure details
|
||||
* e.g. className and parameters
|
||||
* @param builder the string builder to use to append the proc specific information
|
||||
*/
|
||||
protected void toStringClassDetails(StringBuilder builder) {
|
||||
builder.append(getClass().getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the serialized result if any, otherwise null
|
||||
*/
|
||||
public byte[] getResult() {
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* The procedure may leave a "result" on completion.
|
||||
* @param result the serialized result that will be passed to the client
|
||||
*/
|
||||
protected void setResult(final byte[] result) {
|
||||
this.result = result;
|
||||
}
|
||||
|
||||
public long getProcId() {
|
||||
return procId;
|
||||
}
|
||||
|
||||
public boolean hasParent() {
|
||||
return parentProcId != null;
|
||||
}
|
||||
|
||||
public boolean hasException() {
|
||||
return exception != null;
|
||||
}
|
||||
|
||||
public boolean hasTimeout() {
|
||||
return timeout != null;
|
||||
}
|
||||
|
||||
public long getParentProcId() {
|
||||
return parentProcId;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the procedure has failed.
|
||||
* true may mean failed but not yet rolledback or failed and rolledback.
|
||||
*/
|
||||
public synchronized boolean isFailed() {
|
||||
return exception != null || state == ProcedureState.ROLLEDBACK;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the procedure is finished successfully.
|
||||
*/
|
||||
public synchronized boolean isSuccess() {
|
||||
return state == ProcedureState.FINISHED && exception == null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the procedure is finished. The Procedure may be completed
|
||||
* successfuly or failed and rolledback.
|
||||
*/
|
||||
public synchronized boolean isFinished() {
|
||||
switch (state) {
|
||||
case ROLLEDBACK:
|
||||
return true;
|
||||
case FINISHED:
|
||||
return exception == null;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the procedure is waiting for a child to finish or for an external event.
|
||||
*/
|
||||
public synchronized boolean isWaiting() {
|
||||
switch (state) {
|
||||
case WAITING:
|
||||
case WAITING_TIMEOUT:
|
||||
return true;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public synchronized RemoteProcedureException getException() {
|
||||
return exception;
|
||||
}
|
||||
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
|
||||
public synchronized long getLastUpdate() {
|
||||
return lastUpdate;
|
||||
}
|
||||
|
||||
public synchronized long elapsedTime() {
|
||||
return lastUpdate - startTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param timeout timeout in msec
|
||||
*/
|
||||
protected void setTimeout(final int timeout) {
|
||||
this.timeout = timeout;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the timeout in msec
|
||||
*/
|
||||
public int getTimeout() {
|
||||
return timeout;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the remaining time before the timeout
|
||||
*/
|
||||
public long getTimeRemaining() {
|
||||
return Math.max(0, timeout - (EnvironmentEdgeManager.currentTime() - startTime));
|
||||
}
|
||||
|
||||
protected void setOwner(final String owner) {
|
||||
this.owner = StringUtils.isEmpty(owner) ? null : owner;
|
||||
}
|
||||
|
||||
public String getOwner() {
|
||||
return owner;
|
||||
}
|
||||
|
||||
public boolean hasOwner() {
|
||||
return owner != null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized void setState(final ProcedureState state) {
|
||||
this.state = state;
|
||||
updateTimestamp();
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized ProcedureState getState() {
|
||||
return state;
|
||||
}
|
||||
|
||||
protected void setFailure(final String source, final Throwable cause) {
|
||||
setFailure(new RemoteProcedureException(source, cause));
|
||||
}
|
||||
|
||||
protected synchronized void setFailure(final RemoteProcedureException exception) {
|
||||
this.exception = exception;
|
||||
if (!isFinished()) {
|
||||
setState(ProcedureState.FINISHED);
|
||||
}
|
||||
}
|
||||
|
||||
protected void setAbortFailure(final String source, final String msg) {
|
||||
setFailure(source, new ProcedureAbortedException(msg));
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized boolean setTimeoutFailure() {
|
||||
if (state == ProcedureState.WAITING_TIMEOUT) {
|
||||
long timeDiff = EnvironmentEdgeManager.currentTime() - lastUpdate;
|
||||
setFailure("ProcedureExecutor", new TimeoutException(
|
||||
"Operation timed out after " + StringUtils.humanTimeDiff(timeDiff)));
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to assign the ID to the newly created procedure.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected void setProcId(final long procId) {
|
||||
this.procId = procId;
|
||||
this.startTime = EnvironmentEdgeManager.currentTime();
|
||||
setState(ProcedureState.RUNNABLE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to assign the parent to the newly created procedure.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected void setParentProcId(final long parentProcId) {
|
||||
this.parentProcId = parentProcId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal method called by the ProcedureExecutor that starts the
|
||||
* user-level code execute().
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected Procedure[] doExecute(final TEnvironment env)
|
||||
throws ProcedureYieldException {
|
||||
try {
|
||||
updateTimestamp();
|
||||
return execute(env);
|
||||
} finally {
|
||||
updateTimestamp();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal method called by the ProcedureExecutor that starts the
|
||||
* user-level code rollback().
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected void doRollback(final TEnvironment env) throws IOException {
|
||||
try {
|
||||
updateTimestamp();
|
||||
rollback(env);
|
||||
} finally {
|
||||
updateTimestamp();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called on store load to initialize the Procedure internals after
|
||||
* the creation/deserialization.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected void setStartTime(final long startTime) {
|
||||
this.startTime = startTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called on store load to initialize the Procedure internals after
|
||||
* the creation/deserialization.
|
||||
*/
|
||||
private synchronized void setLastUpdate(final long lastUpdate) {
|
||||
this.lastUpdate = lastUpdate;
|
||||
}
|
||||
|
||||
protected synchronized void updateTimestamp() {
|
||||
this.lastUpdate = EnvironmentEdgeManager.currentTime();
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor on procedure-load to restore the latch state
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized void setChildrenLatch(final int numChildren) {
|
||||
this.childrenLatch = numChildren;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor on procedure-load to restore the latch state
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized void incChildrenLatch() {
|
||||
// TODO: can this be inferred from the stack? I think so...
|
||||
this.childrenLatch++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to notify that one of the sub-procedures
|
||||
* has completed.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized boolean childrenCountDown() {
|
||||
assert childrenLatch > 0;
|
||||
return --childrenLatch == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the RootProcedureState on procedure execution.
|
||||
* Each procedure store its stack-index positions.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized void addStackIndex(final int index) {
|
||||
if (stackIndexes == null) {
|
||||
stackIndexes = new int[] { index };
|
||||
} else {
|
||||
int count = stackIndexes.length;
|
||||
stackIndexes = Arrays.copyOf(stackIndexes, count + 1);
|
||||
stackIndexes[count] = index;
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized boolean removeStackIndex() {
|
||||
if (stackIndexes.length > 1) {
|
||||
stackIndexes = Arrays.copyOf(stackIndexes, stackIndexes.length - 1);
|
||||
return false;
|
||||
} else {
|
||||
stackIndexes = null;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called on store load to initialize the Procedure internals after
|
||||
* the creation/deserialization.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized void setStackIndexes(final List<Integer> stackIndexes) {
|
||||
this.stackIndexes = new int[stackIndexes.size()];
|
||||
for (int i = 0; i < this.stackIndexes.length; ++i) {
|
||||
this.stackIndexes[i] = stackIndexes.get(i);
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized boolean wasExecuted() {
|
||||
return stackIndexes != null;
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized int[] getStackIndexes() {
|
||||
return stackIndexes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(final Procedure other) {
|
||||
long diff = getProcId() - other.getProcId();
|
||||
return (diff < 0) ? -1 : (diff > 0) ? 1 : 0;
|
||||
}
|
||||
|
||||
/*
|
||||
* Helper to lookup the root Procedure ID given a specified procedure.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected static Long getRootProcedureId(final Map<Long, Procedure> procedures, Procedure proc) {
|
||||
while (proc.hasParent()) {
|
||||
proc = procedures.get(proc.getParentProcId());
|
||||
if (proc == null) return null;
|
||||
}
|
||||
return proc.getProcId();
|
||||
}
|
||||
|
||||
protected static Procedure newInstance(final String className) throws IOException {
|
||||
try {
|
||||
Class<?> clazz = Class.forName(className);
|
||||
if (!Modifier.isPublic(clazz.getModifiers())) {
|
||||
throw new Exception("the " + clazz + " class is not public");
|
||||
}
|
||||
|
||||
Constructor<?> ctor = clazz.getConstructor();
|
||||
assert ctor != null : "no constructor found";
|
||||
if (!Modifier.isPublic(ctor.getModifiers())) {
|
||||
throw new Exception("the " + clazz + " constructor is not public");
|
||||
}
|
||||
return (Procedure)ctor.newInstance();
|
||||
} catch (Exception e) {
|
||||
throw new IOException("The procedure class " + className +
|
||||
" must be accessible and have an empty constructor", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void validateClass(final Procedure proc) throws IOException {
|
||||
try {
|
||||
Class<?> clazz = proc.getClass();
|
||||
if (!Modifier.isPublic(clazz.getModifiers())) {
|
||||
throw new Exception("the " + clazz + " class is not public");
|
||||
}
|
||||
|
||||
Constructor<?> ctor = clazz.getConstructor();
|
||||
assert ctor != null;
|
||||
if (!Modifier.isPublic(ctor.getModifiers())) {
|
||||
throw new Exception("the " + clazz + " constructor is not public");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new IOException("The procedure class " + proc.getClass().getName() +
|
||||
" must be accessible and have an empty constructor", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to convert the procedure to protobuf.
|
||||
* Used by ProcedureStore implementations.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public static ProcedureProtos.Procedure convert(final Procedure proc)
|
||||
throws IOException {
|
||||
Preconditions.checkArgument(proc != null);
|
||||
validateClass(proc);
|
||||
|
||||
ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder()
|
||||
.setClassName(proc.getClass().getName())
|
||||
.setProcId(proc.getProcId())
|
||||
.setState(proc.getState())
|
||||
.setStartTime(proc.getStartTime())
|
||||
.setLastUpdate(proc.getLastUpdate());
|
||||
|
||||
if (proc.hasParent()) {
|
||||
builder.setParentId(proc.getParentProcId());
|
||||
}
|
||||
|
||||
if (proc.hasTimeout()) {
|
||||
builder.setTimeout(proc.getTimeout());
|
||||
}
|
||||
|
||||
if (proc.hasOwner()) {
|
||||
builder.setOwner(proc.getOwner());
|
||||
}
|
||||
|
||||
int[] stackIds = proc.getStackIndexes();
|
||||
if (stackIds != null) {
|
||||
for (int i = 0; i < stackIds.length; ++i) {
|
||||
builder.addStackId(stackIds[i]);
|
||||
}
|
||||
}
|
||||
|
||||
if (proc.hasException()) {
|
||||
RemoteProcedureException exception = proc.getException();
|
||||
builder.setException(
|
||||
RemoteProcedureException.toProto(exception.getSource(), exception.getCause()));
|
||||
}
|
||||
|
||||
byte[] result = proc.getResult();
|
||||
if (result != null) {
|
||||
builder.setResult(ByteStringer.wrap(result));
|
||||
}
|
||||
|
||||
ByteString.Output stateStream = ByteString.newOutput();
|
||||
proc.serializeStateData(stateStream);
|
||||
if (stateStream.size() > 0) {
|
||||
builder.setStateData(stateStream.toByteString());
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to convert the protobuf procedure.
|
||||
* Used by ProcedureStore implementations.
|
||||
*
|
||||
* TODO: OPTIMIZATION: some of the field never change during the execution
|
||||
* (e.g. className, procId, parentId, ...).
|
||||
* We can split in 'data' and 'state', and the store
|
||||
* may take advantage of it by storing the data only on insert().
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public static Procedure convert(final ProcedureProtos.Procedure proto)
|
||||
throws IOException {
|
||||
// Procedure from class name
|
||||
Procedure proc = Procedure.newInstance(proto.getClassName());
|
||||
|
||||
// set fields
|
||||
proc.setProcId(proto.getProcId());
|
||||
proc.setState(proto.getState());
|
||||
proc.setStartTime(proto.getStartTime());
|
||||
proc.setLastUpdate(proto.getLastUpdate());
|
||||
|
||||
if (proto.hasParentId()) {
|
||||
proc.setParentProcId(proto.getParentId());
|
||||
}
|
||||
|
||||
if (proto.hasOwner()) {
|
||||
proc.setOwner(proto.getOwner());
|
||||
}
|
||||
|
||||
if (proto.hasTimeout()) {
|
||||
proc.setTimeout(proto.getTimeout());
|
||||
}
|
||||
|
||||
if (proto.getStackIdCount() > 0) {
|
||||
proc.setStackIndexes(proto.getStackIdList());
|
||||
}
|
||||
|
||||
if (proto.hasException()) {
|
||||
assert proc.getState() == ProcedureState.FINISHED ||
|
||||
proc.getState() == ProcedureState.ROLLEDBACK :
|
||||
"The procedure must be failed (waiting to rollback) or rolledback";
|
||||
proc.setFailure(RemoteProcedureException.fromProto(proto.getException()));
|
||||
}
|
||||
|
||||
if (proto.hasResult()) {
|
||||
proc.setResult(proto.getResult().toByteArray());
|
||||
}
|
||||
|
||||
// we want to call deserialize even when the stream is empty, mainly for testing.
|
||||
proc.deserializeStateData(proto.getStateData().newInput());
|
||||
|
||||
return proc;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Thrown when a procedure is aborted
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class ProcedureAbortedException extends ProcedureException {
|
||||
/** default constructor */
|
||||
public ProcedureAbortedException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param s message
|
||||
*/
|
||||
public ProcedureAbortedException(String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class ProcedureException extends IOException {
|
||||
/** default constructor */
|
||||
public ProcedureException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param s message
|
||||
*/
|
||||
public ProcedureException(String s) {
|
||||
super(s);
|
||||
}
|
||||
|
||||
public ProcedureException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,173 @@
|
|||
/**
|
||||
* 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 java.util.Map;
|
||||
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* This class is a container of queues that allows to select a queue
|
||||
* in a round robin fashion, considering priority of the queue.
|
||||
*
|
||||
* the quantum is just how many poll() will return the same object.
|
||||
* e.g. if quantum is 1 and you have A and B as object you'll get: A B A B
|
||||
* e.g. if quantum is 2 and you have A and B as object you'll get: A A B B A A B B
|
||||
* then the object priority is just a priority * quantum
|
||||
*
|
||||
* Example:
|
||||
* - three queues (A, B, C) with priorities (1, 1, 2)
|
||||
* - The first poll() will return A
|
||||
* - The second poll() will return B
|
||||
* - The third and forth poll() will return C
|
||||
* - and so on again and again.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class ProcedureFairRunQueues<TKey, TQueue extends ProcedureFairRunQueues.FairObject> {
|
||||
private ConcurrentSkipListMap<TKey, TQueue> objMap =
|
||||
new ConcurrentSkipListMap<TKey, TQueue>();
|
||||
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
private final int quantum;
|
||||
|
||||
private Map.Entry<TKey, TQueue> current = null;
|
||||
private int currentQuantum = 0;
|
||||
|
||||
public interface FairObject {
|
||||
boolean isAvailable();
|
||||
int getPriority();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param quantum how many poll() will return the same object.
|
||||
*/
|
||||
public ProcedureFairRunQueues(final int quantum) {
|
||||
this.quantum = quantum;
|
||||
}
|
||||
|
||||
public TQueue get(final TKey key) {
|
||||
return objMap.get(key);
|
||||
}
|
||||
|
||||
public TQueue add(final TKey key, final TQueue queue) {
|
||||
TQueue oldq = objMap.putIfAbsent(key, queue);
|
||||
return oldq != null ? oldq : queue;
|
||||
}
|
||||
|
||||
public TQueue remove(final TKey key) {
|
||||
TQueue queue = objMap.get(key);
|
||||
if (queue != null) {
|
||||
lock.lock();
|
||||
try {
|
||||
queue = objMap.remove(key);
|
||||
if (current != null && queue == current.getValue()) {
|
||||
currentQuantum = 0;
|
||||
current = null;
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
return queue;
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
lock.lock();
|
||||
try {
|
||||
current = null;
|
||||
objMap.clear();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the next available item if present
|
||||
*/
|
||||
public TQueue poll() {
|
||||
lock.lock();
|
||||
try {
|
||||
TQueue queue;
|
||||
if (currentQuantum == 0) {
|
||||
if (nextObject() == null) {
|
||||
// nothing here
|
||||
return null;
|
||||
}
|
||||
|
||||
queue = current.getValue();
|
||||
currentQuantum = calculateQuantum(queue) - 1;
|
||||
} else {
|
||||
currentQuantum--;
|
||||
queue = current.getValue();
|
||||
}
|
||||
|
||||
if (!queue.isAvailable()) {
|
||||
Map.Entry<TKey, TQueue> last = current;
|
||||
// Try the next one
|
||||
do {
|
||||
if (nextObject() == null)
|
||||
return null;
|
||||
} while (current.getValue() != last.getValue() && !current.getValue().isAvailable());
|
||||
|
||||
queue = current.getValue();
|
||||
currentQuantum = calculateQuantum(queue) - 1;
|
||||
}
|
||||
|
||||
return queue;
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append('{');
|
||||
for (Map.Entry<TKey, TQueue> entry: objMap.entrySet()) {
|
||||
builder.append(entry.getKey());
|
||||
builder.append(':');
|
||||
builder.append(entry.getValue());
|
||||
}
|
||||
builder.append('}');
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
private Map.Entry<TKey, TQueue> nextObject() {
|
||||
Map.Entry<TKey, TQueue> next = null;
|
||||
|
||||
// If we have already a key, try the next one
|
||||
if (current != null) {
|
||||
next = objMap.higherEntry(current.getKey());
|
||||
}
|
||||
|
||||
// if there is no higher key, go back to the first
|
||||
current = (next != null) ? next : objMap.firstEntry();
|
||||
return current;
|
||||
}
|
||||
|
||||
private int calculateQuantum(final TQueue fairObject) {
|
||||
// TODO
|
||||
return Math.max(1, fairObject.getPriority() * quantum);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* 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.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class ProcedureResult {
|
||||
private final RemoteProcedureException exception;
|
||||
private final long lastUpdate;
|
||||
private final long startTime;
|
||||
private final byte[] result;
|
||||
|
||||
private long clientAckTime = -1;
|
||||
|
||||
public ProcedureResult(final long startTime, final long lastUpdate,
|
||||
final RemoteProcedureException exception) {
|
||||
this.lastUpdate = lastUpdate;
|
||||
this.startTime = startTime;
|
||||
this.exception = exception;
|
||||
this.result = null;
|
||||
}
|
||||
|
||||
public ProcedureResult(final long startTime, final long lastUpdate, final byte[] result) {
|
||||
this.lastUpdate = lastUpdate;
|
||||
this.startTime = startTime;
|
||||
this.exception = null;
|
||||
this.result = result;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Keep track of the runnable procedures
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public interface ProcedureRunnableSet {
|
||||
/**
|
||||
* Inserts the specified element at the front of this queue.
|
||||
* @param proc the Procedure to add
|
||||
*/
|
||||
void addFront(Procedure proc);
|
||||
|
||||
/**
|
||||
* Inserts the specified element at the end of this queue.
|
||||
* @param proc the Procedure to add
|
||||
*/
|
||||
void addBack(Procedure proc);
|
||||
|
||||
/**
|
||||
* The procedure can't run at the moment.
|
||||
* add it back to the queue, giving priority to someone else.
|
||||
* @param proc the Procedure to add back to the list
|
||||
*/
|
||||
void yield(Procedure proc);
|
||||
|
||||
/**
|
||||
* The procedure in execution completed.
|
||||
* This can be implemented to perform cleanups.
|
||||
* @param proc the Procedure that completed the execution.
|
||||
*/
|
||||
void completionCleanup(Procedure proc);
|
||||
|
||||
/**
|
||||
* Fetch one Procedure from the queue
|
||||
* @return the Procedure ID to execute, or null if nothing present.
|
||||
*/
|
||||
Long poll();
|
||||
|
||||
/**
|
||||
* In case the class is blocking on poll() waiting for items to be added,
|
||||
* this method should awake poll() and poll() should return.
|
||||
*/
|
||||
void signalAll();
|
||||
|
||||
/**
|
||||
* Returns the number of elements in this collection.
|
||||
* @return the number of elements in this collection.
|
||||
*/
|
||||
int size();
|
||||
|
||||
/**
|
||||
* Removes all of the elements from this collection.
|
||||
*/
|
||||
void clear();
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.procedure2;
|
||||
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Deque;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Simple runqueue for the procedures
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class ProcedureSimpleRunQueue implements ProcedureRunnableSet {
|
||||
private final Deque<Long> runnables = new ArrayDeque<Long>();
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
private final Condition waitCond = lock.newCondition();
|
||||
|
||||
@Override
|
||||
public void addFront(final Procedure proc) {
|
||||
lock.lock();
|
||||
try {
|
||||
runnables.addFirst(proc.getProcId());
|
||||
waitCond.signal();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBack(final Procedure proc) {
|
||||
lock.lock();
|
||||
try {
|
||||
runnables.addLast(proc.getProcId());
|
||||
waitCond.signal();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void yield(final Procedure proc) {
|
||||
addBack(proc);
|
||||
}
|
||||
|
||||
@Override
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
|
||||
public Long poll() {
|
||||
lock.lock();
|
||||
try {
|
||||
if (runnables.isEmpty()) {
|
||||
waitCond.await();
|
||||
if (!runnables.isEmpty()) {
|
||||
return runnables.pop();
|
||||
}
|
||||
} else {
|
||||
return runnables.pop();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
return null;
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void signalAll() {
|
||||
lock.lock();
|
||||
try {
|
||||
waitCond.signalAll();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
lock.lock();
|
||||
try {
|
||||
runnables.clear();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
lock.lock();
|
||||
try {
|
||||
return runnables.size();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completionCleanup(Procedure proc) {
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.procedure2;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
// TODO: Not used yet
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class ProcedureYieldException extends ProcedureException {
|
||||
/** default constructor */
|
||||
public ProcedureYieldException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param s message
|
||||
*/
|
||||
public ProcedureYieldException(String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,116 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
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.util.ForeignExceptionUtil;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* A RemoteProcedureException is an exception from another thread or process.
|
||||
* <p>
|
||||
* RemoteProcedureExceptions are sent to 'remote' peers to signal an abort in the face of failures.
|
||||
* When serialized for transmission we encode using Protobufs to ensure version compatibility.
|
||||
* <p>
|
||||
* RemoteProcedureException exceptions contain a Throwable as its cause.
|
||||
* This can be a "regular" exception generated locally or a ProxyThrowable that is a representation
|
||||
* of the original exception created on original 'remote' source. These ProxyThrowables have their
|
||||
* their stacks traces and messages overridden to reflect the original 'remote' exception.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@SuppressWarnings("serial")
|
||||
public class RemoteProcedureException extends ProcedureException {
|
||||
|
||||
/**
|
||||
* Name of the throwable's source such as a host or thread name. Must be non-null.
|
||||
*/
|
||||
private final String source;
|
||||
|
||||
/**
|
||||
* Create a new RemoteProcedureException that can be serialized.
|
||||
* It is assumed that this came form a local source.
|
||||
* @param source
|
||||
* @param cause
|
||||
*/
|
||||
public RemoteProcedureException(String source, Throwable cause) {
|
||||
super(cause);
|
||||
assert source != null;
|
||||
assert cause != null;
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
public String getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
public IOException unwrapRemoteException() {
|
||||
if (getCause() instanceof RemoteException) {
|
||||
return ((RemoteException)getCause()).unwrapRemoteException();
|
||||
}
|
||||
if (getCause() instanceof IOException) {
|
||||
return (IOException)getCause();
|
||||
}
|
||||
return new IOException(getCause());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String className = getCause().getClass().getName();
|
||||
return className + " via " + getSource() + ":" + getLocalizedMessage();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a RemoteProcedureException to an array of bytes.
|
||||
* @param source the name of the external exception source
|
||||
* @param t the "local" external exception (local)
|
||||
* @return protobuf serialized version of RemoteProcedureException
|
||||
*/
|
||||
public static byte[] serialize(String source, Throwable t) {
|
||||
return toProto(source, t).toByteArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* Takes a series of bytes and tries to generate an RemoteProcedureException instance for it.
|
||||
* @param bytes
|
||||
* @return the ForeignExcpetion instance
|
||||
* @throws InvalidProtocolBufferException if there was deserialization problem this is thrown.
|
||||
*/
|
||||
public static RemoteProcedureException deserialize(byte[] bytes)
|
||||
throws InvalidProtocolBufferException {
|
||||
return fromProto(ForeignExceptionMessage.parseFrom(bytes));
|
||||
}
|
||||
|
||||
public ForeignExceptionMessage convert() {
|
||||
return ForeignExceptionUtil.toProtoForeignException(getSource(), getCause());
|
||||
}
|
||||
|
||||
public static ForeignExceptionMessage toProto(String source, Throwable t) {
|
||||
return ForeignExceptionUtil.toProtoForeignException(source, t);
|
||||
}
|
||||
|
||||
public static RemoteProcedureException fromProto(final ForeignExceptionMessage eem) {
|
||||
return new RemoteProcedureException(eem.getSource(), ForeignExceptionUtil.toIOException(eem));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,185 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
|
||||
|
||||
/**
|
||||
* Internal state of the ProcedureExecutor that describes the state of a "Root Procedure".
|
||||
* A "Root Procedure" is a Procedure without parent, each subprocedure will be
|
||||
* added to the "Root Procedure" stack (or rollback-stack).
|
||||
*
|
||||
* RootProcedureState is used and managed only by the ProcedureExecutor.
|
||||
* Long rootProcId = getRootProcedureId(proc);
|
||||
* rollbackStack.get(rootProcId).acquire(proc)
|
||||
* rollbackStack.get(rootProcId).release(proc)
|
||||
* ...
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
class RootProcedureState {
|
||||
private static final Log LOG = LogFactory.getLog(RootProcedureState.class);
|
||||
|
||||
private enum State {
|
||||
RUNNING, // The Procedure is running or ready to run
|
||||
FAILED, // The Procedure failed, waiting for the rollback executing
|
||||
ROLLINGBACK, // The Procedure failed and the execution was rolledback
|
||||
}
|
||||
|
||||
private ArrayList<Procedure> subprocedures = null;
|
||||
private State state = State.RUNNING;
|
||||
private int running = 0;
|
||||
|
||||
public synchronized boolean isFailed() {
|
||||
switch (state) {
|
||||
case ROLLINGBACK:
|
||||
case FAILED:
|
||||
return true;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public synchronized boolean isRollingback() {
|
||||
return state == State.ROLLINGBACK;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to mark rollback execution
|
||||
*/
|
||||
protected synchronized boolean setRollback() {
|
||||
if (running == 0 && state == State.FAILED) {
|
||||
state = State.ROLLINGBACK;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to mark rollback execution
|
||||
*/
|
||||
protected synchronized void unsetRollback() {
|
||||
assert state == State.ROLLINGBACK;
|
||||
state = State.FAILED;
|
||||
}
|
||||
|
||||
protected synchronized List<Procedure> getSubprocedures() {
|
||||
return subprocedures;
|
||||
}
|
||||
|
||||
protected synchronized RemoteProcedureException getException() {
|
||||
if (subprocedures != null) {
|
||||
for (Procedure proc: subprocedures) {
|
||||
if (proc.hasException()) {
|
||||
return proc.getException();
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to mark the procedure step as running.
|
||||
*/
|
||||
protected synchronized boolean acquire(final Procedure proc) {
|
||||
if (state != State.RUNNING) return false;
|
||||
|
||||
running++;
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor to mark the procedure step as finished.
|
||||
*/
|
||||
protected synchronized void release(final Procedure proc) {
|
||||
running--;
|
||||
}
|
||||
|
||||
protected synchronized void abort() {
|
||||
if (state == State.RUNNING) {
|
||||
state = State.FAILED;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the ProcedureExecutor after the procedure step is completed,
|
||||
* to add the step to the rollback list (or procedure stack)
|
||||
*/
|
||||
protected synchronized void addRollbackStep(final Procedure proc) {
|
||||
if (proc.isFailed()) {
|
||||
state = State.FAILED;
|
||||
}
|
||||
if (subprocedures == null) {
|
||||
subprocedures = new ArrayList<Procedure>();
|
||||
}
|
||||
proc.addStackIndex(subprocedures.size());
|
||||
subprocedures.add(proc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called on store load by the ProcedureExecutor to load part of the stack.
|
||||
*
|
||||
* Each procedure has its own stack-positions. Which means we have to write
|
||||
* to the store only the Procedure we executed, and nothing else.
|
||||
* on load we recreate the full stack by aggregating each procedure stack-positions.
|
||||
*/
|
||||
protected synchronized void loadStack(final Procedure proc) {
|
||||
int[] stackIndexes = proc.getStackIndexes();
|
||||
if (stackIndexes != null) {
|
||||
if (subprocedures == null) {
|
||||
subprocedures = new ArrayList<Procedure>();
|
||||
}
|
||||
int diff = (1 + stackIndexes[stackIndexes.length - 1]) - subprocedures.size();
|
||||
if (diff > 0) {
|
||||
subprocedures.ensureCapacity(1 + stackIndexes[stackIndexes.length - 1]);
|
||||
while (diff-- > 0) subprocedures.add(null);
|
||||
}
|
||||
for (int i = 0; i < stackIndexes.length; ++i) {
|
||||
subprocedures.set(stackIndexes[i], proc);
|
||||
}
|
||||
}
|
||||
if (proc.getState() == ProcedureState.ROLLEDBACK) {
|
||||
state = State.ROLLINGBACK;
|
||||
} else if (proc.isFailed()) {
|
||||
state = State.FAILED;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called on store load by the ProcedureExecutor to validate the procedure stack.
|
||||
*/
|
||||
protected synchronized boolean isValid() {
|
||||
if (subprocedures != null) {
|
||||
for (Procedure proc: subprocedures) {
|
||||
if (proc == null) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.procedure2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData;
|
||||
|
||||
/**
|
||||
* A SequentialProcedure describes one step in a procedure chain.
|
||||
* -> Step 1 -> Step 2 -> Step 3
|
||||
*
|
||||
* The main difference from a base Procedure is that the execute() of a
|
||||
* SequentialProcedure will be called only once, there will be no second
|
||||
* execute() call once the child are finished. which means once the child
|
||||
* of a SequentialProcedure are completed the SequentialProcedure is completed too.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class SequentialProcedure<TEnvironment> extends Procedure<TEnvironment> {
|
||||
private boolean executed = false;
|
||||
|
||||
@Override
|
||||
protected Procedure[] doExecute(final TEnvironment env)
|
||||
throws ProcedureYieldException {
|
||||
updateTimestamp();
|
||||
try {
|
||||
Procedure[] children = !executed ? execute(env) : null;
|
||||
executed = !executed;
|
||||
return children;
|
||||
} finally {
|
||||
updateTimestamp();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRollback(final TEnvironment env) throws IOException {
|
||||
updateTimestamp();
|
||||
if (executed) {
|
||||
try {
|
||||
rollback(env);
|
||||
executed = !executed;
|
||||
} finally {
|
||||
updateTimestamp();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serializeStateData(final OutputStream stream) throws IOException {
|
||||
SequentialProcedureData.Builder data = SequentialProcedureData.newBuilder();
|
||||
data.setExecuted(executed);
|
||||
data.build().writeDelimitedTo(stream);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void deserializeStateData(final InputStream stream) throws IOException {
|
||||
SequentialProcedureData data = SequentialProcedureData.parseDelimitedFrom(stream);
|
||||
executed = data.getExecuted();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,166 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.StateMachineProcedureData;
|
||||
|
||||
/**
|
||||
* Procedure described by a series of steps.
|
||||
*
|
||||
* The procedure implementor must have an enum of 'states', describing
|
||||
* the various step of the procedure.
|
||||
* Once the procedure is running, the procedure-framework will call executeFromState()
|
||||
* using the 'state' provided by the user. The first call to executeFromState()
|
||||
* will be performed with 'state = null'. The implementor can jump between
|
||||
* states using setNextState(MyStateEnum.ordinal()).
|
||||
* The rollback will call rollbackState() for each state that was executed, in reverse order.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class StateMachineProcedure<TEnvironment, TState>
|
||||
extends Procedure<TEnvironment> {
|
||||
private int stateCount = 0;
|
||||
private int[] states = null;
|
||||
|
||||
protected enum Flow {
|
||||
HAS_MORE_STATE,
|
||||
NO_MORE_STATE,
|
||||
}
|
||||
|
||||
/**
|
||||
* called to perform a single step of the specified 'state' of the procedure
|
||||
* @param state state to execute
|
||||
* @return Flow.NO_MORE_STATE if the procedure is completed,
|
||||
* Flow.HAS_MORE_STATE if there is another step.
|
||||
*/
|
||||
protected abstract Flow executeFromState(TEnvironment env, TState state)
|
||||
throws ProcedureYieldException;
|
||||
|
||||
/**
|
||||
* called to perform the rollback of the specified state
|
||||
* @param state state to rollback
|
||||
* @throws IOException temporary failure, the rollback will retry later
|
||||
*/
|
||||
protected abstract void rollbackState(TEnvironment env, TState state)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Convert an ordinal (or state id) to an Enum (or more descriptive) state object.
|
||||
* @param stateId the ordinal() of the state enum (or state id)
|
||||
* @return the state enum object
|
||||
*/
|
||||
protected abstract TState getState(int stateId);
|
||||
|
||||
/**
|
||||
* Convert the Enum (or more descriptive) state object to an ordinal (or state id).
|
||||
* @param state the state enum object
|
||||
* @return stateId the ordinal() of the state enum (or state id)
|
||||
*/
|
||||
protected abstract int getStateId(TState state);
|
||||
|
||||
/**
|
||||
* Return the initial state object that will be used for the first call to executeFromState().
|
||||
* @return the initial state enum object
|
||||
*/
|
||||
protected abstract TState getInitialState();
|
||||
|
||||
/**
|
||||
* Set the next state for the procedure.
|
||||
* @param state the state enum object
|
||||
*/
|
||||
protected void setNextState(final TState state) {
|
||||
setNextState(getStateId(state));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(final TEnvironment env)
|
||||
throws ProcedureYieldException {
|
||||
updateTimestamp();
|
||||
try {
|
||||
TState state = stateCount > 0 ? getState(states[stateCount-1]) : getInitialState();
|
||||
if (stateCount == 0) {
|
||||
setNextState(getStateId(state));
|
||||
}
|
||||
if (executeFromState(env, state) == Flow.NO_MORE_STATE) {
|
||||
// completed
|
||||
return null;
|
||||
}
|
||||
return (isWaiting() || isFailed()) ? null : new Procedure[] {this};
|
||||
} finally {
|
||||
updateTimestamp();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(final TEnvironment env) throws IOException {
|
||||
try {
|
||||
updateTimestamp();
|
||||
rollbackState(env, stateCount > 0 ? getState(states[stateCount-1]) : getInitialState());
|
||||
stateCount--;
|
||||
} finally {
|
||||
updateTimestamp();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the next state for the procedure.
|
||||
* @param stateId the ordinal() of the state enum (or state id)
|
||||
*/
|
||||
private void setNextState(final int stateId) {
|
||||
if (states == null || states.length == stateCount) {
|
||||
int newCapacity = stateCount + 8;
|
||||
if (states != null) {
|
||||
states = Arrays.copyOf(states, newCapacity);
|
||||
} else {
|
||||
states = new int[newCapacity];
|
||||
}
|
||||
}
|
||||
states[stateCount++] = stateId;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serializeStateData(final OutputStream stream) throws IOException {
|
||||
StateMachineProcedureData.Builder data = StateMachineProcedureData.newBuilder();
|
||||
for (int i = 0; i < stateCount; ++i) {
|
||||
data.addState(states[i]);
|
||||
}
|
||||
data.build().writeDelimitedTo(stream);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void deserializeStateData(final InputStream stream) throws IOException {
|
||||
StateMachineProcedureData data = StateMachineProcedureData.parseDelimitedFrom(stream);
|
||||
stateCount = data.getStateCount();
|
||||
if (stateCount > 0) {
|
||||
states = new int[stateCount];
|
||||
for (int i = 0; i < stateCount; ++i) {
|
||||
states[i] = data.getState(i);
|
||||
}
|
||||
} else {
|
||||
states = null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class TwoPhaseProcedure<TEnvironment> extends Procedure<TEnvironment> {
|
||||
// TODO (e.g. used by ACLs/VisibilityTags updates)
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.procedure2.store;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
|
||||
/**
|
||||
* The ProcedureStore is used by the executor to persist the state of each procedure execution.
|
||||
* This allows to resume the execution of pending/in-progress procedures in case
|
||||
* of machine failure or service shutdown.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public interface ProcedureStore {
|
||||
/**
|
||||
* Store listener interface.
|
||||
* The main process should register a listener and respond to the store events.
|
||||
*/
|
||||
public interface ProcedureStoreListener {
|
||||
/**
|
||||
* triggered when the store is not able to write out data.
|
||||
* the main process should abort.
|
||||
*/
|
||||
void abortProcess();
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the listener to the notification list.
|
||||
* @param listener The AssignmentListener to register
|
||||
*/
|
||||
void registerListener(ProcedureStoreListener listener);
|
||||
|
||||
/**
|
||||
* Remove the listener from the notification list.
|
||||
* @param listener The AssignmentListener to unregister
|
||||
* @return true if the listner was in the list and it was removed, otherwise false.
|
||||
*/
|
||||
boolean unregisterListener(ProcedureStoreListener listener);
|
||||
|
||||
/**
|
||||
* Start/Open the procedure store
|
||||
* @param numThreads
|
||||
*/
|
||||
void start(int numThreads) throws IOException;
|
||||
|
||||
/**
|
||||
* Stop/Close the procedure store
|
||||
* @param abort true if the stop is an abort
|
||||
*/
|
||||
void stop(boolean abort);
|
||||
|
||||
/**
|
||||
* @return true if the store is running, otherwise false.
|
||||
*/
|
||||
boolean isRunning();
|
||||
|
||||
/**
|
||||
* @return the number of threads/slots passed to start()
|
||||
*/
|
||||
int getNumThreads();
|
||||
|
||||
/**
|
||||
* Acquire the lease for the procedure store.
|
||||
*/
|
||||
void recoverLease() throws IOException;
|
||||
|
||||
/**
|
||||
* Load the Procedures in the store.
|
||||
* @return the set of procedures present in the store
|
||||
*/
|
||||
Iterator<Procedure> load() throws IOException;
|
||||
|
||||
/**
|
||||
* When a procedure is submitted to the executor insert(proc, null) will be called.
|
||||
* 'proc' has a 'RUNNABLE' state and the initial information required to start up.
|
||||
*
|
||||
* When a procedure is executed and it returns children insert(proc, subprocs) will be called.
|
||||
* 'proc' has a 'WAITING' state and an update state.
|
||||
* 'subprocs' are the children in 'RUNNABLE' state with the initial information.
|
||||
*
|
||||
* @param proc the procedure to serialize and write to the store.
|
||||
* @param subprocs the newly created child of the proc.
|
||||
*/
|
||||
void insert(Procedure proc, Procedure[] subprocs);
|
||||
|
||||
/**
|
||||
* The specified procedure was executed,
|
||||
* and the new state should be written to the store.
|
||||
* @param proc the procedure to serialize and write to the store.
|
||||
*/
|
||||
void update(Procedure proc);
|
||||
|
||||
/**
|
||||
* The specified procId was removed from the executor,
|
||||
* due to completion, abort or failure.
|
||||
* The store implementor should remove all the information about the specified procId.
|
||||
* @param procId the ID of the procedure to remove.
|
||||
*/
|
||||
void delete(long procId);
|
||||
}
|
|
@ -0,0 +1,540 @@
|
|||
/**
|
||||
* 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.store;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
|
||||
|
||||
/**
|
||||
* Keeps track of live procedures.
|
||||
*
|
||||
* It can be used by the ProcedureStore to identify which procedures are already
|
||||
* deleted/completed to avoid the deserialization step on restart.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class ProcedureStoreTracker {
|
||||
private final TreeMap<Long, BitSetNode> map = new TreeMap<Long, BitSetNode>();
|
||||
|
||||
private boolean keepDeletes = false;
|
||||
private boolean partial = false;
|
||||
|
||||
public enum DeleteState { YES, NO, MAYBE }
|
||||
|
||||
public static class BitSetNode {
|
||||
private final static long WORD_MASK = 0xffffffffffffffffL;
|
||||
private final static int ADDRESS_BITS_PER_WORD = 6;
|
||||
private final static int BITS_PER_WORD = 1 << ADDRESS_BITS_PER_WORD;
|
||||
private final static int MAX_NODE_SIZE = 4 << ADDRESS_BITS_PER_WORD;
|
||||
|
||||
private long[] updated;
|
||||
private long[] deleted;
|
||||
private long start;
|
||||
|
||||
public void dump() {
|
||||
System.out.printf("%06d:%06d min=%d max=%d%n", getStart(), getEnd(),
|
||||
getMinProcId(), getMaxProcId());
|
||||
System.out.println("Update:");
|
||||
for (int i = 0; i < updated.length; ++i) {
|
||||
for (int j = 0; j < BITS_PER_WORD; ++j) {
|
||||
System.out.print((updated[i] & (1L << j)) != 0 ? "1" : "0");
|
||||
}
|
||||
System.out.println(" " + i);
|
||||
}
|
||||
System.out.println();
|
||||
System.out.println("Delete:");
|
||||
for (int i = 0; i < deleted.length; ++i) {
|
||||
for (int j = 0; j < BITS_PER_WORD; ++j) {
|
||||
System.out.print((deleted[i] & (1L << j)) != 0 ? "1" : "0");
|
||||
}
|
||||
System.out.println(" " + i);
|
||||
}
|
||||
System.out.println();
|
||||
}
|
||||
|
||||
public BitSetNode(final long procId, final boolean partial) {
|
||||
start = alignDown(procId);
|
||||
|
||||
int count = 2;
|
||||
updated = new long[count];
|
||||
deleted = new long[count];
|
||||
for (int i = 0; i < count; ++i) {
|
||||
updated[i] = 0;
|
||||
deleted[i] = partial ? 0 : WORD_MASK;
|
||||
}
|
||||
|
||||
updateState(procId, false);
|
||||
}
|
||||
|
||||
protected BitSetNode(final long start, final long[] updated, final long[] deleted) {
|
||||
this.start = start;
|
||||
this.updated = updated;
|
||||
this.deleted = deleted;
|
||||
}
|
||||
|
||||
public void update(final long procId) {
|
||||
updateState(procId, false);
|
||||
}
|
||||
|
||||
public void delete(final long procId) {
|
||||
updateState(procId, true);
|
||||
}
|
||||
|
||||
public Long getStart() {
|
||||
return start;
|
||||
}
|
||||
|
||||
public Long getEnd() {
|
||||
return start + (updated.length << ADDRESS_BITS_PER_WORD) - 1;
|
||||
}
|
||||
|
||||
public boolean contains(final long procId) {
|
||||
return start <= procId && procId <= getEnd();
|
||||
}
|
||||
|
||||
public DeleteState isDeleted(final long procId) {
|
||||
int bitmapIndex = getBitmapIndex(procId);
|
||||
int wordIndex = bitmapIndex >> ADDRESS_BITS_PER_WORD;
|
||||
if (wordIndex >= deleted.length) {
|
||||
return DeleteState.MAYBE;
|
||||
}
|
||||
return (deleted[wordIndex] & (1L << bitmapIndex)) != 0 ? DeleteState.YES : DeleteState.NO;
|
||||
}
|
||||
|
||||
private boolean isUpdated(final long procId) {
|
||||
int bitmapIndex = getBitmapIndex(procId);
|
||||
int wordIndex = bitmapIndex >> ADDRESS_BITS_PER_WORD;
|
||||
if (wordIndex >= updated.length) {
|
||||
return false;
|
||||
}
|
||||
return (updated[wordIndex] & (1L << bitmapIndex)) != 0;
|
||||
}
|
||||
|
||||
public boolean isUpdated() {
|
||||
// TODO: cache the value
|
||||
for (int i = 0; i < updated.length; ++i) {
|
||||
long deleteMask = ~deleted[i];
|
||||
if ((updated[i] & deleteMask) != (WORD_MASK & deleteMask)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
// TODO: cache the value
|
||||
for (int i = 0; i < deleted.length; ++i) {
|
||||
if (deleted[i] != WORD_MASK) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public void resetUpdates() {
|
||||
for (int i = 0; i < updated.length; ++i) {
|
||||
updated[i] = 0;
|
||||
}
|
||||
}
|
||||
|
||||
public void undeleteAll() {
|
||||
for (int i = 0; i < updated.length; ++i) {
|
||||
deleted[i] = 0;
|
||||
}
|
||||
}
|
||||
|
||||
public ProcedureProtos.ProcedureStoreTracker.TrackerNode convert() {
|
||||
ProcedureProtos.ProcedureStoreTracker.TrackerNode.Builder builder =
|
||||
ProcedureProtos.ProcedureStoreTracker.TrackerNode.newBuilder();
|
||||
builder.setStartId(start);
|
||||
for (int i = 0; i < updated.length; ++i) {
|
||||
builder.addUpdated(updated[i]);
|
||||
builder.addDeleted(deleted[i]);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public static BitSetNode convert(ProcedureProtos.ProcedureStoreTracker.TrackerNode data) {
|
||||
long start = data.getStartId();
|
||||
int size = data.getUpdatedCount();
|
||||
long[] updated = new long[size];
|
||||
long[] deleted = new long[size];
|
||||
for (int i = 0; i < size; ++i) {
|
||||
updated[i] = data.getUpdated(i);
|
||||
deleted[i] = data.getDeleted(i);
|
||||
}
|
||||
return new BitSetNode(start, updated, deleted);
|
||||
}
|
||||
|
||||
// ========================================================================
|
||||
// Grow/Merge Helpers
|
||||
// ========================================================================
|
||||
public boolean canGrow(final long procId) {
|
||||
return (procId - start) < MAX_NODE_SIZE;
|
||||
}
|
||||
|
||||
public boolean canMerge(final BitSetNode rightNode) {
|
||||
return (start + rightNode.getEnd()) < MAX_NODE_SIZE;
|
||||
}
|
||||
|
||||
public void grow(final long procId) {
|
||||
int delta, offset;
|
||||
|
||||
if (procId < start) {
|
||||
// add to head
|
||||
long newStart = alignDown(procId);
|
||||
delta = (int)(start - newStart) >> ADDRESS_BITS_PER_WORD;
|
||||
offset = delta;
|
||||
} else {
|
||||
// Add to tail
|
||||
long newEnd = alignUp(procId + 1);
|
||||
delta = (int)(newEnd - getEnd()) >> ADDRESS_BITS_PER_WORD;
|
||||
offset = 0;
|
||||
}
|
||||
|
||||
long[] newBitmap;
|
||||
int oldSize = updated.length;
|
||||
|
||||
newBitmap = new long[oldSize + delta];
|
||||
System.arraycopy(updated, 0, newBitmap, offset, oldSize);
|
||||
updated = newBitmap;
|
||||
|
||||
newBitmap = new long[deleted.length + delta];
|
||||
System.arraycopy(deleted, 0, newBitmap, offset, oldSize);
|
||||
deleted = newBitmap;
|
||||
|
||||
for (int i = 0; i < delta; ++i) {
|
||||
updated[oldSize + i] = 0;
|
||||
deleted[oldSize + i] = WORD_MASK;
|
||||
}
|
||||
}
|
||||
|
||||
public void merge(final BitSetNode rightNode) {
|
||||
int delta = (int)(rightNode.getEnd() - getEnd()) >> ADDRESS_BITS_PER_WORD;
|
||||
|
||||
long[] newBitmap;
|
||||
int oldSize = updated.length;
|
||||
int newSize = (delta - rightNode.updated.length);
|
||||
int offset = oldSize + newSize;
|
||||
|
||||
newBitmap = new long[oldSize + delta];
|
||||
System.arraycopy(updated, 0, newBitmap, 0, oldSize);
|
||||
System.arraycopy(rightNode.updated, 0, newBitmap, offset, rightNode.updated.length);
|
||||
updated = newBitmap;
|
||||
|
||||
newBitmap = new long[oldSize + delta];
|
||||
System.arraycopy(deleted, 0, newBitmap, 0, oldSize);
|
||||
System.arraycopy(rightNode.deleted, 0, newBitmap, offset, rightNode.deleted.length);
|
||||
deleted = newBitmap;
|
||||
|
||||
for (int i = 0; i < newSize; ++i) {
|
||||
updated[offset + i] = 0;
|
||||
deleted[offset + i] = WORD_MASK;
|
||||
}
|
||||
}
|
||||
|
||||
// ========================================================================
|
||||
// Min/Max Helpers
|
||||
// ========================================================================
|
||||
public long getMinProcId() {
|
||||
long minProcId = start;
|
||||
for (int i = 0; i < deleted.length; ++i) {
|
||||
if (deleted[i] == 0) {
|
||||
return(minProcId);
|
||||
}
|
||||
|
||||
if (deleted[i] != WORD_MASK) {
|
||||
for (int j = 0; j < BITS_PER_WORD; ++j) {
|
||||
if ((deleted[i] & (1L << j)) != 0) {
|
||||
return minProcId + j;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
minProcId += BITS_PER_WORD;
|
||||
}
|
||||
return minProcId;
|
||||
}
|
||||
|
||||
public long getMaxProcId() {
|
||||
long maxProcId = getEnd();
|
||||
for (int i = deleted.length - 1; i >= 0; --i) {
|
||||
if (deleted[i] == 0) {
|
||||
return maxProcId;
|
||||
}
|
||||
|
||||
if (deleted[i] != WORD_MASK) {
|
||||
for (int j = BITS_PER_WORD - 1; j >= 0; --j) {
|
||||
if ((deleted[i] & (1L << j)) == 0) {
|
||||
return maxProcId - (BITS_PER_WORD - 1 - j);
|
||||
}
|
||||
}
|
||||
}
|
||||
maxProcId -= BITS_PER_WORD;
|
||||
}
|
||||
return maxProcId;
|
||||
}
|
||||
|
||||
// ========================================================================
|
||||
// Bitmap Helpers
|
||||
// ========================================================================
|
||||
private int getBitmapIndex(final long procId) {
|
||||
return (int)(procId - start);
|
||||
}
|
||||
|
||||
private void updateState(final long procId, final boolean isDeleted) {
|
||||
int bitmapIndex = getBitmapIndex(procId);
|
||||
int wordIndex = bitmapIndex >> ADDRESS_BITS_PER_WORD;
|
||||
long value = (1L << bitmapIndex);
|
||||
|
||||
if (isDeleted) {
|
||||
updated[wordIndex] |= value;
|
||||
deleted[wordIndex] |= value;
|
||||
} else {
|
||||
updated[wordIndex] |= value;
|
||||
deleted[wordIndex] &= ~value;
|
||||
}
|
||||
}
|
||||
|
||||
// ========================================================================
|
||||
// Helpers
|
||||
// ========================================================================
|
||||
private static long alignUp(final long x) {
|
||||
return (x + (BITS_PER_WORD - 1)) & -BITS_PER_WORD;
|
||||
}
|
||||
|
||||
private static long alignDown(final long x) {
|
||||
return x & -BITS_PER_WORD;
|
||||
}
|
||||
}
|
||||
|
||||
public void insert(final Procedure proc, final Procedure[] subprocs) {
|
||||
insert(proc.getProcId());
|
||||
if (subprocs != null) {
|
||||
for (int i = 0; i < subprocs.length; ++i) {
|
||||
insert(subprocs[i].getProcId());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void update(final Procedure proc) {
|
||||
update(proc.getProcId());
|
||||
}
|
||||
|
||||
public void insert(long procId) {
|
||||
BitSetNode node = getOrCreateNode(procId);
|
||||
node.update(procId);
|
||||
}
|
||||
|
||||
public void update(long procId) {
|
||||
Map.Entry<Long, BitSetNode> entry = map.floorEntry(procId);
|
||||
assert entry != null : "expected node to update procId=" + procId;
|
||||
|
||||
BitSetNode node = entry.getValue();
|
||||
assert node.contains(procId);
|
||||
node.update(procId);
|
||||
}
|
||||
|
||||
public void delete(long procId) {
|
||||
Map.Entry<Long, BitSetNode> entry = map.floorEntry(procId);
|
||||
assert entry != null : "expected node to delete procId=" + procId;
|
||||
|
||||
BitSetNode node = entry.getValue();
|
||||
assert node.contains(procId) : "expected procId in the node";
|
||||
node.delete(procId);
|
||||
|
||||
if (!keepDeletes && node.isEmpty()) {
|
||||
// TODO: RESET if (map.size() == 1)
|
||||
map.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public void setDeleted(final long procId, final boolean isDeleted) {
|
||||
BitSetNode node = getOrCreateNode(procId);
|
||||
node.updateState(procId, isDeleted);
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
this.map.clear();
|
||||
}
|
||||
|
||||
public DeleteState isDeleted(long procId) {
|
||||
Map.Entry<Long, BitSetNode> entry = map.floorEntry(procId);
|
||||
if (entry != null) {
|
||||
BitSetNode node = entry.getValue();
|
||||
DeleteState state = node.isDeleted(procId);
|
||||
return partial && !node.isUpdated(procId) ? DeleteState.MAYBE : state;
|
||||
}
|
||||
return partial ? DeleteState.MAYBE : DeleteState.YES;
|
||||
}
|
||||
|
||||
public long getMinProcId() {
|
||||
// TODO: Cache?
|
||||
Map.Entry<Long, BitSetNode> entry = map.firstEntry();
|
||||
return entry == null ? 0 : entry.getValue().getMinProcId();
|
||||
}
|
||||
|
||||
public void setKeepDeletes(boolean keepDeletes) {
|
||||
this.keepDeletes = keepDeletes;
|
||||
if (!keepDeletes) {
|
||||
Iterator<Map.Entry<Long, BitSetNode>> it = map.entrySet().iterator();
|
||||
while (it.hasNext()) {
|
||||
Map.Entry<Long, BitSetNode> entry = it.next();
|
||||
if (entry.getValue().isEmpty()) {
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void setPartialFlag(boolean isPartial) {
|
||||
this.partial = isPartial;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
|
||||
if (entry.getValue().isEmpty() == false) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean isUpdated() {
|
||||
for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
|
||||
if (entry.getValue().isUpdated() == false) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public void resetUpdates() {
|
||||
for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
|
||||
entry.getValue().resetUpdates();
|
||||
}
|
||||
}
|
||||
|
||||
public void undeleteAll() {
|
||||
for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
|
||||
entry.getValue().undeleteAll();
|
||||
}
|
||||
}
|
||||
|
||||
private BitSetNode getOrCreateNode(final long procId) {
|
||||
// can procId fit in the left node?
|
||||
BitSetNode leftNode = null;
|
||||
boolean leftCanGrow = false;
|
||||
Map.Entry<Long, BitSetNode> leftEntry = map.floorEntry(procId);
|
||||
if (leftEntry != null) {
|
||||
leftNode = leftEntry.getValue();
|
||||
if (leftNode.contains(procId)) {
|
||||
return leftNode;
|
||||
}
|
||||
leftCanGrow = leftNode.canGrow(procId);
|
||||
}
|
||||
|
||||
BitSetNode rightNode = null;
|
||||
boolean rightCanGrow = false;
|
||||
Map.Entry<Long, BitSetNode> rightEntry = map.ceilingEntry(procId);
|
||||
if (rightEntry != null) {
|
||||
rightNode = rightEntry.getValue();
|
||||
rightCanGrow = rightNode.canGrow(procId);
|
||||
if (leftNode != null) {
|
||||
if (leftNode.canMerge(rightNode)) {
|
||||
// merge left and right node
|
||||
return mergeNodes(leftNode, rightNode);
|
||||
}
|
||||
|
||||
if (leftCanGrow && rightCanGrow) {
|
||||
if ((procId - leftNode.getEnd()) <= (rightNode.getStart() - procId)) {
|
||||
// grow the left node
|
||||
return growNode(leftNode, procId);
|
||||
}
|
||||
// grow the right node
|
||||
return growNode(rightNode, procId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// grow the left node
|
||||
if (leftCanGrow) {
|
||||
return growNode(leftNode, procId);
|
||||
}
|
||||
|
||||
// grow the right node
|
||||
if (rightCanGrow) {
|
||||
return growNode(rightNode, procId);
|
||||
}
|
||||
|
||||
// add new node
|
||||
BitSetNode node = new BitSetNode(procId, partial);
|
||||
map.put(node.getStart(), node);
|
||||
return node;
|
||||
}
|
||||
|
||||
private BitSetNode growNode(BitSetNode node, long procId) {
|
||||
map.remove(node.getStart());
|
||||
node.grow(procId);
|
||||
map.put(node.getStart(), node);
|
||||
return node;
|
||||
}
|
||||
|
||||
private BitSetNode mergeNodes(BitSetNode leftNode, BitSetNode rightNode) {
|
||||
leftNode.merge(rightNode);
|
||||
map.remove(rightNode.getStart());
|
||||
return leftNode;
|
||||
}
|
||||
|
||||
public void dump() {
|
||||
System.out.println("map " + map.size());
|
||||
for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
|
||||
entry.getValue().dump();
|
||||
}
|
||||
}
|
||||
|
||||
public void writeTo(final OutputStream stream) throws IOException {
|
||||
ProcedureProtos.ProcedureStoreTracker.Builder builder =
|
||||
ProcedureProtos.ProcedureStoreTracker.newBuilder();
|
||||
for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
|
||||
builder.addNode(entry.getValue().convert());
|
||||
}
|
||||
builder.build().writeDelimitedTo(stream);
|
||||
}
|
||||
|
||||
public void readFrom(final InputStream stream) throws IOException {
|
||||
ProcedureProtos.ProcedureStoreTracker data =
|
||||
ProcedureProtos.ProcedureStoreTracker.parseDelimitedFrom(stream);
|
||||
map.clear();
|
||||
for (ProcedureProtos.ProcedureStoreTracker.TrackerNode protoNode: data.getNodeList()) {
|
||||
BitSetNode node = BitSetNode.convert(protoNode);
|
||||
map.put(node.getStart(), node);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* 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.store.wal;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Thrown when a procedure WAL is corrupted
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class CorruptedWALProcedureStoreException extends HBaseIOException {
|
||||
/** default constructor */
|
||||
public CorruptedWALProcedureStoreException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param s message
|
||||
*/
|
||||
public CorruptedWALProcedureStoreException(String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,152 @@
|
|||
/**
|
||||
* 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.store.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
|
||||
|
||||
/**
|
||||
* Describes a WAL File
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
|
||||
private static final Log LOG = LogFactory.getLog(ProcedureWALFile.class);
|
||||
|
||||
private ProcedureWALHeader header;
|
||||
private FSDataInputStream stream;
|
||||
private FileStatus logStatus;
|
||||
private FileSystem fs;
|
||||
private Path logFile;
|
||||
private long startPos;
|
||||
|
||||
public ProcedureWALFile(final FileSystem fs, final FileStatus logStatus) {
|
||||
this.fs = fs;
|
||||
this.logStatus = logStatus;
|
||||
this.logFile = logStatus.getPath();
|
||||
}
|
||||
|
||||
public ProcedureWALFile(FileSystem fs, Path logFile, ProcedureWALHeader header, long startPos) {
|
||||
this.fs = fs;
|
||||
this.logFile = logFile;
|
||||
this.header = header;
|
||||
this.startPos = startPos;
|
||||
}
|
||||
|
||||
public void open() throws IOException {
|
||||
if (stream == null) {
|
||||
stream = fs.open(logFile);
|
||||
}
|
||||
|
||||
if (header == null) {
|
||||
header = ProcedureWALFormat.readHeader(stream);
|
||||
startPos = stream.getPos();
|
||||
} else {
|
||||
stream.seek(startPos);
|
||||
}
|
||||
}
|
||||
|
||||
public ProcedureWALTrailer readTrailer() throws IOException {
|
||||
try {
|
||||
return ProcedureWALFormat.readTrailer(stream, startPos, logStatus.getLen());
|
||||
} finally {
|
||||
stream.seek(startPos);
|
||||
}
|
||||
}
|
||||
|
||||
public void readTracker(ProcedureStoreTracker tracker) throws IOException {
|
||||
ProcedureWALTrailer trailer = readTrailer();
|
||||
try {
|
||||
stream.seek(trailer.getTrackerPos());
|
||||
tracker.readFrom(stream);
|
||||
} finally {
|
||||
stream.seek(startPos);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
if (stream == null) return;
|
||||
try {
|
||||
stream.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("unable to close the wal file: " + logFile, e);
|
||||
} finally {
|
||||
stream = null;
|
||||
}
|
||||
}
|
||||
|
||||
public FSDataInputStream getStream() {
|
||||
return stream;
|
||||
}
|
||||
|
||||
public ProcedureWALHeader getHeader() {
|
||||
return header;
|
||||
}
|
||||
|
||||
public boolean isCompacted() {
|
||||
return header.getType() == ProcedureWALFormat.LOG_TYPE_COMPACTED;
|
||||
}
|
||||
|
||||
public long getLogId() {
|
||||
return header.getLogId();
|
||||
}
|
||||
|
||||
public long getSize() {
|
||||
return logStatus.getLen();
|
||||
}
|
||||
|
||||
public void removeFile() throws IOException {
|
||||
close();
|
||||
fs.delete(logFile, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(final ProcedureWALFile other) {
|
||||
long diff = header.getLogId() - other.header.getLogId();
|
||||
return (diff < 0) ? -1 : (diff > 0) ? 1 : 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (!(o instanceof ProcedureWALFile)) return false;
|
||||
return compareTo((ProcedureWALFile)o) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return logFile.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return logFile.toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,234 @@
|
|||
/**
|
||||
* 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.store.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.hbase.io.util.StreamUtils;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
|
||||
import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* Helper class that contains the WAL serialization utils.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public final class ProcedureWALFormat {
|
||||
static final byte LOG_TYPE_STREAM = 0;
|
||||
static final byte LOG_TYPE_COMPACTED = 1;
|
||||
static final byte LOG_TYPE_MAX_VALID = 1;
|
||||
|
||||
static final byte HEADER_VERSION = 1;
|
||||
static final byte TRAILER_VERSION = 1;
|
||||
static final long HEADER_MAGIC = 0x31764c4157637250L;
|
||||
static final long TRAILER_MAGIC = 0x50726357414c7631L;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public static class InvalidWALDataException extends IOException {
|
||||
public InvalidWALDataException(String s) {
|
||||
super(s);
|
||||
}
|
||||
|
||||
public InvalidWALDataException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
||||
|
||||
interface Loader {
|
||||
void removeLog(ProcedureWALFile log);
|
||||
void markCorruptedWAL(ProcedureWALFile log, IOException e);
|
||||
}
|
||||
|
||||
private ProcedureWALFormat() {}
|
||||
|
||||
public static Iterator<Procedure> load(final Iterator<ProcedureWALFile> logs,
|
||||
final ProcedureStoreTracker tracker, final Loader loader) throws IOException {
|
||||
ProcedureWALFormatReader reader = new ProcedureWALFormatReader(tracker);
|
||||
tracker.setKeepDeletes(true);
|
||||
try {
|
||||
while (logs.hasNext()) {
|
||||
ProcedureWALFile log = logs.next();
|
||||
log.open();
|
||||
try {
|
||||
reader.read(log, loader);
|
||||
} finally {
|
||||
log.close();
|
||||
}
|
||||
}
|
||||
// The tracker is now updated with all the procedures read from the logs
|
||||
tracker.setPartialFlag(false);
|
||||
tracker.resetUpdates();
|
||||
} finally {
|
||||
tracker.setKeepDeletes(false);
|
||||
}
|
||||
// TODO: Write compacted version?
|
||||
return reader.getProcedures();
|
||||
}
|
||||
|
||||
public static void writeHeader(OutputStream stream, ProcedureWALHeader header)
|
||||
throws IOException {
|
||||
header.writeDelimitedTo(stream);
|
||||
}
|
||||
|
||||
/*
|
||||
* +-----------------+
|
||||
* | END OF WAL DATA | <---+
|
||||
* +-----------------+ |
|
||||
* | | |
|
||||
* | Tracker | |
|
||||
* | | |
|
||||
* +-----------------+ |
|
||||
* | version | |
|
||||
* +-----------------+ |
|
||||
* | TRAILER_MAGIC | |
|
||||
* +-----------------+ |
|
||||
* | offset |-----+
|
||||
* +-----------------+
|
||||
*/
|
||||
public static void writeTrailer(FSDataOutputStream stream, ProcedureStoreTracker tracker)
|
||||
throws IOException {
|
||||
long offset = stream.getPos();
|
||||
|
||||
// Write EOF Entry
|
||||
ProcedureWALEntry.newBuilder()
|
||||
.setType(ProcedureWALEntry.Type.EOF)
|
||||
.build().writeDelimitedTo(stream);
|
||||
|
||||
// Write Tracker
|
||||
tracker.writeTo(stream);
|
||||
|
||||
stream.write(TRAILER_VERSION);
|
||||
StreamUtils.writeLong(stream, TRAILER_MAGIC);
|
||||
StreamUtils.writeLong(stream, offset);
|
||||
}
|
||||
|
||||
public static ProcedureWALHeader readHeader(InputStream stream)
|
||||
throws IOException {
|
||||
ProcedureWALHeader header;
|
||||
try {
|
||||
header = ProcedureWALHeader.parseDelimitedFrom(stream);
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new InvalidWALDataException(e);
|
||||
}
|
||||
|
||||
if (header == null) {
|
||||
throw new InvalidWALDataException("No data available to read the Header");
|
||||
}
|
||||
|
||||
if (header.getVersion() < 0 || header.getVersion() != HEADER_VERSION) {
|
||||
throw new InvalidWALDataException("Invalid Header version. got " + header.getVersion() +
|
||||
" expected " + HEADER_VERSION);
|
||||
}
|
||||
|
||||
if (header.getType() < 0 || header.getType() > LOG_TYPE_MAX_VALID) {
|
||||
throw new InvalidWALDataException("Invalid header type. got " + header.getType());
|
||||
}
|
||||
|
||||
return header;
|
||||
}
|
||||
|
||||
public static ProcedureWALTrailer readTrailer(FSDataInputStream stream, long startPos, long size)
|
||||
throws IOException {
|
||||
long trailerPos = size - 17; // Beginning of the Trailer Jump
|
||||
|
||||
if (trailerPos < startPos) {
|
||||
throw new InvalidWALDataException("Missing trailer: size=" + size + " startPos=" + startPos);
|
||||
}
|
||||
|
||||
stream.seek(trailerPos);
|
||||
int version = stream.read();
|
||||
if (version != TRAILER_VERSION) {
|
||||
throw new InvalidWALDataException("Invalid Trailer version. got " + version +
|
||||
" expected " + TRAILER_VERSION);
|
||||
}
|
||||
|
||||
long magic = StreamUtils.readLong(stream);
|
||||
if (magic != TRAILER_MAGIC) {
|
||||
throw new InvalidWALDataException("Invalid Trailer magic. got " + magic +
|
||||
" expected " + TRAILER_MAGIC);
|
||||
}
|
||||
|
||||
long trailerOffset = StreamUtils.readLong(stream);
|
||||
stream.seek(trailerOffset);
|
||||
|
||||
ProcedureWALEntry entry = readEntry(stream);
|
||||
if (entry.getType() != ProcedureWALEntry.Type.EOF) {
|
||||
throw new InvalidWALDataException("Invalid Trailer begin");
|
||||
}
|
||||
|
||||
ProcedureWALTrailer trailer = ProcedureWALTrailer.newBuilder()
|
||||
.setVersion(version)
|
||||
.setTrackerPos(stream.getPos())
|
||||
.build();
|
||||
return trailer;
|
||||
}
|
||||
|
||||
public static ProcedureWALEntry readEntry(InputStream stream) throws IOException {
|
||||
return ProcedureWALEntry.parseDelimitedFrom(stream);
|
||||
}
|
||||
|
||||
public static void writeEntry(ByteSlot slot, ProcedureWALEntry.Type type,
|
||||
Procedure proc, Procedure[] subprocs) throws IOException {
|
||||
ProcedureWALEntry.Builder builder = ProcedureWALEntry.newBuilder();
|
||||
builder.setType(type);
|
||||
builder.addProcedure(Procedure.convert(proc));
|
||||
if (subprocs != null) {
|
||||
for (int i = 0; i < subprocs.length; ++i) {
|
||||
builder.addProcedure(Procedure.convert(subprocs[i]));
|
||||
}
|
||||
}
|
||||
builder.build().writeDelimitedTo(slot);
|
||||
}
|
||||
|
||||
public static void writeInsert(ByteSlot slot, Procedure proc)
|
||||
throws IOException {
|
||||
writeEntry(slot, ProcedureWALEntry.Type.INIT, proc, null);
|
||||
}
|
||||
|
||||
public static void writeInsert(ByteSlot slot, Procedure proc, Procedure[] subprocs)
|
||||
throws IOException {
|
||||
writeEntry(slot, ProcedureWALEntry.Type.INSERT, proc, subprocs);
|
||||
}
|
||||
|
||||
public static void writeUpdate(ByteSlot slot, Procedure proc)
|
||||
throws IOException {
|
||||
writeEntry(slot, ProcedureWALEntry.Type.UPDATE, proc, null);
|
||||
}
|
||||
|
||||
public static void writeDelete(ByteSlot slot, long procId)
|
||||
throws IOException {
|
||||
ProcedureWALEntry.Builder builder = ProcedureWALEntry.newBuilder();
|
||||
builder.setType(ProcedureWALEntry.Type.DELETE);
|
||||
builder.setProcId(procId);
|
||||
builder.build().writeDelimitedTo(slot);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,166 @@
|
|||
/**
|
||||
* 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.store.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
|
||||
|
||||
/**
|
||||
* Helper class that loads the procedures stored in a WAL
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class ProcedureWALFormatReader {
|
||||
private static final Log LOG = LogFactory.getLog(ProcedureWALFormatReader.class);
|
||||
|
||||
private final ProcedureStoreTracker tracker;
|
||||
//private final long compactionLogId;
|
||||
|
||||
private final Map<Long, Procedure> procedures = new HashMap<Long, Procedure>();
|
||||
private final Map<Long, ProcedureProtos.Procedure> localProcedures =
|
||||
new HashMap<Long, ProcedureProtos.Procedure>();
|
||||
|
||||
private long maxProcId = 0;
|
||||
|
||||
public ProcedureWALFormatReader(final ProcedureStoreTracker tracker) {
|
||||
this.tracker = tracker;
|
||||
}
|
||||
|
||||
public void read(ProcedureWALFile log, ProcedureWALFormat.Loader loader) throws IOException {
|
||||
FSDataInputStream stream = log.getStream();
|
||||
try {
|
||||
boolean hasMore = true;
|
||||
while (hasMore) {
|
||||
ProcedureWALEntry entry = ProcedureWALFormat.readEntry(stream);
|
||||
if (entry == null) {
|
||||
LOG.warn("nothing left to decode. exiting with missing EOF");
|
||||
hasMore = false;
|
||||
break;
|
||||
}
|
||||
switch (entry.getType()) {
|
||||
case INIT:
|
||||
readInitEntry(entry);
|
||||
break;
|
||||
case INSERT:
|
||||
readInsertEntry(entry);
|
||||
break;
|
||||
case UPDATE:
|
||||
case COMPACT:
|
||||
readUpdateEntry(entry);
|
||||
break;
|
||||
case DELETE:
|
||||
readDeleteEntry(entry);
|
||||
break;
|
||||
case EOF:
|
||||
hasMore = false;
|
||||
break;
|
||||
default:
|
||||
throw new CorruptedWALProcedureStoreException("Invalid entry: " + entry);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("got an exception while reading the procedure WAL: " + log, e);
|
||||
loader.markCorruptedWAL(log, e);
|
||||
}
|
||||
|
||||
if (localProcedures.isEmpty()) {
|
||||
LOG.info("No active entry found in state log " + log + ". removing it");
|
||||
loader.removeLog(log);
|
||||
} else {
|
||||
Iterator<Map.Entry<Long, ProcedureProtos.Procedure>> itd =
|
||||
localProcedures.entrySet().iterator();
|
||||
while (itd.hasNext()) {
|
||||
Map.Entry<Long, ProcedureProtos.Procedure> entry = itd.next();
|
||||
itd.remove();
|
||||
|
||||
// Deserialize the procedure
|
||||
Procedure proc = Procedure.convert(entry.getValue());
|
||||
procedures.put(entry.getKey(), proc);
|
||||
}
|
||||
|
||||
// TODO: Some procedure may be already runnables (see readInitEntry())
|
||||
// (we can also check the "update map" in the log trackers)
|
||||
}
|
||||
}
|
||||
|
||||
public Iterator<Procedure> getProcedures() {
|
||||
return procedures.values().iterator();
|
||||
}
|
||||
|
||||
private void loadEntries(final ProcedureWALEntry entry) {
|
||||
for (ProcedureProtos.Procedure proc: entry.getProcedureList()) {
|
||||
maxProcId = Math.max(maxProcId, proc.getProcId());
|
||||
if (isRequired(proc.getProcId())) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("read " + entry.getType() + " entry " + proc.getProcId());
|
||||
}
|
||||
localProcedures.put(proc.getProcId(), proc);
|
||||
tracker.setDeleted(proc.getProcId(), false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void readInitEntry(final ProcedureWALEntry entry)
|
||||
throws IOException {
|
||||
assert entry.getProcedureCount() == 1 : "Expected only one procedure";
|
||||
// TODO: Make it runnable, before reading other files
|
||||
loadEntries(entry);
|
||||
}
|
||||
|
||||
private void readInsertEntry(final ProcedureWALEntry entry) throws IOException {
|
||||
assert entry.getProcedureCount() >= 1 : "Expected one or more procedures";
|
||||
loadEntries(entry);
|
||||
}
|
||||
|
||||
private void readUpdateEntry(final ProcedureWALEntry entry) throws IOException {
|
||||
assert entry.getProcedureCount() == 1 : "Expected only one procedure";
|
||||
loadEntries(entry);
|
||||
}
|
||||
|
||||
private void readDeleteEntry(final ProcedureWALEntry entry) throws IOException {
|
||||
assert entry.getProcedureCount() == 0 : "Expected no procedures";
|
||||
assert entry.hasProcId() : "expected ProcID";
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("read delete entry " + entry.getProcId());
|
||||
}
|
||||
maxProcId = Math.max(maxProcId, entry.getProcId());
|
||||
localProcedures.remove(entry.getProcId());
|
||||
tracker.setDeleted(entry.getProcId(), true);
|
||||
}
|
||||
|
||||
private boolean isDeleted(final long procId) {
|
||||
return tracker.isDeleted(procId) == ProcedureStoreTracker.DeleteState.YES;
|
||||
}
|
||||
|
||||
private boolean isRequired(final long procId) {
|
||||
return !isDeleted(procId) && !procedures.containsKey(procId);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,721 @@
|
|||
/**
|
||||
* 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.store.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
|
||||
import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
|
||||
import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
|
||||
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
|
||||
|
||||
/**
|
||||
* WAL implementation of the ProcedureStore.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class WALProcedureStore implements ProcedureStore {
|
||||
private static final Log LOG = LogFactory.getLog(WALProcedureStore.class);
|
||||
|
||||
public interface LeaseRecovery {
|
||||
void recoverFileLease(FileSystem fs, Path path) throws IOException;
|
||||
}
|
||||
|
||||
private static final int MAX_RETRIES_BEFORE_ABORT = 3;
|
||||
|
||||
private static final String SYNC_WAIT_MSEC_CONF_KEY = "hbase.procedure.store.wal.sync.wait.msec";
|
||||
private static final int DEFAULT_SYNC_WAIT_MSEC = 100;
|
||||
|
||||
private final CopyOnWriteArrayList<ProcedureStoreListener> listeners =
|
||||
new CopyOnWriteArrayList<ProcedureStoreListener>();
|
||||
|
||||
private final LinkedList<ProcedureWALFile> logs = new LinkedList<ProcedureWALFile>();
|
||||
private final ProcedureStoreTracker storeTracker = new ProcedureStoreTracker();
|
||||
private final AtomicBoolean running = new AtomicBoolean(false);
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
private final Condition waitCond = lock.newCondition();
|
||||
private final Condition slotCond = lock.newCondition();
|
||||
private final Condition syncCond = lock.newCondition();
|
||||
|
||||
private final LeaseRecovery leaseRecovery;
|
||||
private final Configuration conf;
|
||||
private final FileSystem fs;
|
||||
private final Path logDir;
|
||||
|
||||
private AtomicBoolean inSync = new AtomicBoolean(false);
|
||||
private ArrayBlockingQueue<ByteSlot> slotsCache = null;
|
||||
private Set<ProcedureWALFile> corruptedLogs = null;
|
||||
private FSDataOutputStream stream = null;
|
||||
private long totalSynced = 0;
|
||||
private long flushLogId = 0;
|
||||
private int slotIndex = 0;
|
||||
private Thread syncThread;
|
||||
private ByteSlot[] slots;
|
||||
private int syncWaitMsec;
|
||||
|
||||
public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path logDir,
|
||||
final LeaseRecovery leaseRecovery) {
|
||||
this.fs = fs;
|
||||
this.conf = conf;
|
||||
this.logDir = logDir;
|
||||
this.leaseRecovery = leaseRecovery;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start(int numSlots) throws IOException {
|
||||
if (running.getAndSet(true)) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Init buffer slots
|
||||
slots = new ByteSlot[numSlots];
|
||||
slotsCache = new ArrayBlockingQueue(numSlots, true);
|
||||
while (slotsCache.remainingCapacity() > 0) {
|
||||
slotsCache.offer(new ByteSlot());
|
||||
}
|
||||
|
||||
// Tunings
|
||||
syncWaitMsec = conf.getInt(SYNC_WAIT_MSEC_CONF_KEY, DEFAULT_SYNC_WAIT_MSEC);
|
||||
|
||||
// Init sync thread
|
||||
syncThread = new Thread("WALProcedureStoreSyncThread") {
|
||||
@Override
|
||||
public void run() {
|
||||
while (running.get()) {
|
||||
try {
|
||||
syncLoop();
|
||||
} catch (IOException e) {
|
||||
LOG.error("got an exception from the sync-loop", e);
|
||||
sendAbortProcessSignal();
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
syncThread.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(boolean abort) {
|
||||
if (!running.getAndSet(false)) {
|
||||
return;
|
||||
}
|
||||
|
||||
LOG.info("Stopping the WAL Procedure Store");
|
||||
if (lock.tryLock()) {
|
||||
try {
|
||||
waitCond.signalAll();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
if (!abort) {
|
||||
try {
|
||||
syncThread.join();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
// Close the writer
|
||||
closeStream();
|
||||
|
||||
// Close the old logs
|
||||
// they should be already closed, this is just in case the load fails
|
||||
// and we call start() and then stop()
|
||||
for (ProcedureWALFile log: logs) {
|
||||
log.close();
|
||||
}
|
||||
logs.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRunning() {
|
||||
return running.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumThreads() {
|
||||
return slots == null ? 0 : slots.length;
|
||||
}
|
||||
|
||||
public ProcedureStoreTracker getStoreTracker() {
|
||||
return storeTracker;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerListener(ProcedureStoreListener listener) {
|
||||
this.listeners.add(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean unregisterListener(ProcedureStoreListener listener) {
|
||||
return this.listeners.remove(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void recoverLease() throws IOException {
|
||||
LOG.info("Starting WAL Procedure Store lease recovery");
|
||||
FileStatus[] oldLogs = getLogFiles();
|
||||
while (running.get()) {
|
||||
// Get Log-MaxID and recover lease on old logs
|
||||
flushLogId = initOldLogs(oldLogs) + 1;
|
||||
|
||||
// Create new state-log
|
||||
if (!rollWriter(flushLogId)) {
|
||||
// someone else has already created this log
|
||||
LOG.debug("someone else has already created log " + flushLogId);
|
||||
continue;
|
||||
}
|
||||
|
||||
// We have the lease on the log
|
||||
oldLogs = getLogFiles();
|
||||
if (getMaxLogId(oldLogs) > flushLogId) {
|
||||
// Someone else created new logs
|
||||
LOG.debug("someone else created new logs. expected maxLogId < " + flushLogId);
|
||||
logs.getLast().removeFile();
|
||||
continue;
|
||||
}
|
||||
|
||||
LOG.info("lease acquired flushLogId=" + flushLogId);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Procedure> load() throws IOException {
|
||||
if (logs.isEmpty()) {
|
||||
throw new RuntimeException("recoverLease() must be called before loading data");
|
||||
}
|
||||
|
||||
// Nothing to do, If we have only the current log.
|
||||
if (logs.size() == 1) {
|
||||
LOG.debug("No state logs to replay");
|
||||
return null;
|
||||
}
|
||||
|
||||
// Load the old logs
|
||||
final ArrayList<ProcedureWALFile> toRemove = new ArrayList<ProcedureWALFile>();
|
||||
Iterator<ProcedureWALFile> it = logs.descendingIterator();
|
||||
it.next(); // Skip the current log
|
||||
try {
|
||||
return ProcedureWALFormat.load(it, storeTracker, new ProcedureWALFormat.Loader() {
|
||||
@Override
|
||||
public void removeLog(ProcedureWALFile log) {
|
||||
toRemove.add(log);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markCorruptedWAL(ProcedureWALFile log, IOException e) {
|
||||
if (corruptedLogs == null) {
|
||||
corruptedLogs = new HashSet<ProcedureWALFile>();
|
||||
}
|
||||
corruptedLogs.add(log);
|
||||
// TODO: sideline corrupted log
|
||||
}
|
||||
});
|
||||
} finally {
|
||||
if (!toRemove.isEmpty()) {
|
||||
for (ProcedureWALFile log: toRemove) {
|
||||
removeLogFile(log);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void insert(final Procedure proc, final Procedure[] subprocs) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("insert " + proc + " subproc=" + Arrays.toString(subprocs));
|
||||
}
|
||||
|
||||
ByteSlot slot = acquireSlot();
|
||||
long logId = -1;
|
||||
try {
|
||||
// Serialize the insert
|
||||
if (subprocs != null) {
|
||||
ProcedureWALFormat.writeInsert(slot, proc, subprocs);
|
||||
} else {
|
||||
assert !proc.hasParent();
|
||||
ProcedureWALFormat.writeInsert(slot, proc);
|
||||
}
|
||||
|
||||
// Push the transaction data and wait until it is persisted
|
||||
logId = pushData(slot);
|
||||
} catch (IOException e) {
|
||||
// We are not able to serialize the procedure.
|
||||
// this is a code error, and we are not able to go on.
|
||||
LOG.fatal("Unable to serialize one of the procedure: proc=" + proc +
|
||||
" subprocs=" + Arrays.toString(subprocs), e);
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
releaseSlot(slot);
|
||||
}
|
||||
|
||||
// Update the store tracker
|
||||
synchronized (storeTracker) {
|
||||
if (logId == flushLogId) {
|
||||
storeTracker.insert(proc, subprocs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void update(final Procedure proc) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("update " + proc);
|
||||
}
|
||||
|
||||
ByteSlot slot = acquireSlot();
|
||||
long logId = -1;
|
||||
try {
|
||||
// Serialize the update
|
||||
ProcedureWALFormat.writeUpdate(slot, proc);
|
||||
|
||||
// Push the transaction data and wait until it is persisted
|
||||
logId = pushData(slot);
|
||||
} catch (IOException e) {
|
||||
// We are not able to serialize the procedure.
|
||||
// this is a code error, and we are not able to go on.
|
||||
LOG.fatal("Unable to serialize the procedure: " + proc, e);
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
releaseSlot(slot);
|
||||
}
|
||||
|
||||
// Update the store tracker
|
||||
boolean removeOldLogs = false;
|
||||
synchronized (storeTracker) {
|
||||
if (logId == flushLogId) {
|
||||
storeTracker.update(proc);
|
||||
removeOldLogs = storeTracker.isUpdated();
|
||||
}
|
||||
}
|
||||
|
||||
if (removeOldLogs) {
|
||||
removeAllLogs(logId - 1);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(final long procId) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("delete " + procId);
|
||||
}
|
||||
|
||||
ByteSlot slot = acquireSlot();
|
||||
long logId = -1;
|
||||
try {
|
||||
// Serialize the delete
|
||||
ProcedureWALFormat.writeDelete(slot, procId);
|
||||
|
||||
// Push the transaction data and wait until it is persisted
|
||||
logId = pushData(slot);
|
||||
} catch (IOException e) {
|
||||
// We are not able to serialize the procedure.
|
||||
// this is a code error, and we are not able to go on.
|
||||
LOG.fatal("Unable to serialize the procedure: " + procId, e);
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
releaseSlot(slot);
|
||||
}
|
||||
|
||||
boolean removeOldLogs = false;
|
||||
synchronized (storeTracker) {
|
||||
if (logId == flushLogId) {
|
||||
storeTracker.delete(procId);
|
||||
if (storeTracker.isEmpty()) {
|
||||
removeOldLogs = rollWriterOrDie(logId + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (removeOldLogs) {
|
||||
removeAllLogs(logId);
|
||||
}
|
||||
}
|
||||
|
||||
private ByteSlot acquireSlot() {
|
||||
ByteSlot slot = slotsCache.poll();
|
||||
return slot != null ? slot : new ByteSlot();
|
||||
}
|
||||
|
||||
private void releaseSlot(final ByteSlot slot) {
|
||||
slot.reset();
|
||||
slotsCache.offer(slot);
|
||||
}
|
||||
|
||||
private long pushData(final ByteSlot slot) {
|
||||
assert !logs.isEmpty() : "recoverLease() must be called before inserting data";
|
||||
long logId = -1;
|
||||
|
||||
lock.lock();
|
||||
try {
|
||||
// Wait for the sync to be completed
|
||||
while (true) {
|
||||
if (inSync.get()) {
|
||||
syncCond.await();
|
||||
} else if (slotIndex == slots.length) {
|
||||
slotCond.signal();
|
||||
syncCond.await();
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
slots[slotIndex++] = slot;
|
||||
logId = flushLogId;
|
||||
|
||||
// Notify that there is new data
|
||||
if (slotIndex == 1) {
|
||||
waitCond.signal();
|
||||
}
|
||||
|
||||
// Notify that the slots are full
|
||||
if (slotIndex == slots.length) {
|
||||
slotCond.signal();
|
||||
}
|
||||
syncCond.await();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
sendAbortProcessSignal();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
return logId;
|
||||
}
|
||||
|
||||
private void syncLoop() throws IOException {
|
||||
inSync.set(false);
|
||||
while (running.get()) {
|
||||
lock.lock();
|
||||
try {
|
||||
// Wait until new data is available
|
||||
if (slotIndex == 0) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Waiting for data. flushed=" + StringUtils.humanSize(totalSynced));
|
||||
}
|
||||
waitCond.await();
|
||||
if (slotIndex == 0) {
|
||||
// no data.. probably a stop()
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
// Wait SYNC_WAIT_MSEC or the signal of "slots full" before flushing
|
||||
slotCond.await(syncWaitMsec, TimeUnit.MILLISECONDS);
|
||||
|
||||
inSync.set(true);
|
||||
totalSynced += syncSlots();
|
||||
slotIndex = 0;
|
||||
inSync.set(false);
|
||||
syncCond.signalAll();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
sendAbortProcessSignal();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private long syncSlots() {
|
||||
int retry = 0;
|
||||
long totalSynced = 0;
|
||||
do {
|
||||
try {
|
||||
totalSynced = syncSlots(stream, slots, 0, slotIndex);
|
||||
break;
|
||||
} catch (Throwable e) {
|
||||
if (++retry == MAX_RETRIES_BEFORE_ABORT) {
|
||||
LOG.error("sync slot failed, abort.", e);
|
||||
sendAbortProcessSignal();
|
||||
}
|
||||
}
|
||||
} while (running.get());
|
||||
return totalSynced;
|
||||
}
|
||||
|
||||
protected long syncSlots(FSDataOutputStream stream, ByteSlot[] slots, int offset, int count)
|
||||
throws IOException {
|
||||
long totalSynced = 0;
|
||||
for (int i = 0; i < count; ++i) {
|
||||
ByteSlot data = slots[offset + i];
|
||||
data.writeTo(stream);
|
||||
totalSynced += data.size();
|
||||
}
|
||||
stream.hsync();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Sync slots=" + count + '/' + slots.length +
|
||||
" flushed=" + StringUtils.humanSize(totalSynced));
|
||||
}
|
||||
return totalSynced;
|
||||
}
|
||||
|
||||
private void sendAbortProcessSignal() {
|
||||
if (!this.listeners.isEmpty()) {
|
||||
for (ProcedureStoreListener listener : this.listeners) {
|
||||
listener.abortProcess();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean rollWriterOrDie(final long logId) {
|
||||
try {
|
||||
return rollWriter(logId);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to roll the log", e);
|
||||
sendAbortProcessSignal();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private boolean rollWriter(final long logId) throws IOException {
|
||||
ProcedureWALHeader header = ProcedureWALHeader.newBuilder()
|
||||
.setVersion(ProcedureWALFormat.HEADER_VERSION)
|
||||
.setType(ProcedureWALFormat.LOG_TYPE_STREAM)
|
||||
.setMinProcId(storeTracker.getMinProcId())
|
||||
.setLogId(logId)
|
||||
.build();
|
||||
|
||||
FSDataOutputStream newStream = null;
|
||||
Path newLogFile = null;
|
||||
long startPos = -1;
|
||||
try {
|
||||
newLogFile = getLogFilePath(logId);
|
||||
newStream = fs.create(newLogFile, false);
|
||||
ProcedureWALFormat.writeHeader(newStream, header);
|
||||
startPos = newStream.getPos();
|
||||
} catch (FileAlreadyExistsException e) {
|
||||
LOG.error("Log file with id=" + logId + " already exists", e);
|
||||
return false;
|
||||
}
|
||||
lock.lock();
|
||||
try {
|
||||
closeStream();
|
||||
synchronized (storeTracker) {
|
||||
storeTracker.resetUpdates();
|
||||
}
|
||||
stream = newStream;
|
||||
flushLogId = logId;
|
||||
totalSynced = 0;
|
||||
logs.add(new ProcedureWALFile(fs, newLogFile, header, startPos));
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
LOG.info("Roll new state log: " + logId);
|
||||
return true;
|
||||
}
|
||||
|
||||
private void closeStream() {
|
||||
try {
|
||||
if (stream != null) {
|
||||
try {
|
||||
ProcedureWALFormat.writeTrailer(stream, storeTracker);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to write the trailer: " + e.getMessage());
|
||||
}
|
||||
stream.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Unable to close the stream", e);
|
||||
} finally {
|
||||
stream = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void removeAllLogs(long lastLogId) {
|
||||
LOG.info("Remove all state logs with ID less then " + lastLogId);
|
||||
while (!logs.isEmpty()) {
|
||||
ProcedureWALFile log = logs.getFirst();
|
||||
if (lastLogId < log.getLogId()) {
|
||||
break;
|
||||
}
|
||||
|
||||
removeLogFile(log);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean removeLogFile(final ProcedureWALFile log) {
|
||||
try {
|
||||
LOG.debug("remove log: " + log);
|
||||
log.removeFile();
|
||||
logs.remove(log);
|
||||
} catch (IOException e) {
|
||||
LOG.error("unable to remove log " + log, e);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public Set<ProcedureWALFile> getCorruptedLogs() {
|
||||
return corruptedLogs;
|
||||
}
|
||||
|
||||
// ==========================================================================
|
||||
// FileSystem Log Files helpers
|
||||
// ==========================================================================
|
||||
public Path getLogDir() {
|
||||
return this.logDir;
|
||||
}
|
||||
|
||||
public FileSystem getFileSystem() {
|
||||
return this.fs;
|
||||
}
|
||||
|
||||
protected Path getLogFilePath(final long logId) throws IOException {
|
||||
return new Path(logDir, String.format("state-%020d.log", logId));
|
||||
}
|
||||
|
||||
private static long getLogIdFromName(final String name) {
|
||||
int end = name.lastIndexOf(".log");
|
||||
int start = name.lastIndexOf('-') + 1;
|
||||
while (start < end) {
|
||||
if (name.charAt(start) != '0')
|
||||
break;
|
||||
start++;
|
||||
}
|
||||
return Long.parseLong(name.substring(start, end));
|
||||
}
|
||||
|
||||
private FileStatus[] getLogFiles() throws IOException {
|
||||
try {
|
||||
return fs.listStatus(logDir, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
String name = path.getName();
|
||||
return name.startsWith("state-") && name.endsWith(".log");
|
||||
}
|
||||
});
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.warn("log directory not found: " + e.getMessage());
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private long getMaxLogId(final FileStatus[] logFiles) {
|
||||
long maxLogId = 0;
|
||||
if (logFiles != null && logFiles.length > 0) {
|
||||
for (int i = 0; i < logFiles.length; ++i) {
|
||||
maxLogId = Math.max(maxLogId, getLogIdFromName(logFiles[i].getPath().getName()));
|
||||
}
|
||||
}
|
||||
return maxLogId;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Max-LogID of the specified log file set
|
||||
*/
|
||||
private long initOldLogs(final FileStatus[] logFiles) throws IOException {
|
||||
this.logs.clear();
|
||||
|
||||
long maxLogId = 0;
|
||||
if (logFiles != null && logFiles.length > 0) {
|
||||
for (int i = 0; i < logFiles.length; ++i) {
|
||||
final Path logPath = logFiles[i].getPath();
|
||||
leaseRecovery.recoverFileLease(fs, logPath);
|
||||
maxLogId = Math.max(maxLogId, getLogIdFromName(logPath.getName()));
|
||||
|
||||
ProcedureWALFile log = initOldLog(logFiles[i]);
|
||||
if (log != null) {
|
||||
this.logs.add(log);
|
||||
}
|
||||
}
|
||||
Collections.sort(this.logs);
|
||||
initTrackerFromOldLogs();
|
||||
}
|
||||
return maxLogId;
|
||||
}
|
||||
|
||||
private void initTrackerFromOldLogs() {
|
||||
// TODO: Load the most recent tracker available
|
||||
if (!logs.isEmpty()) {
|
||||
ProcedureWALFile log = logs.getLast();
|
||||
try {
|
||||
log.readTracker(storeTracker);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Unable to read tracker for " + log, e);
|
||||
// try the next one...
|
||||
storeTracker.clear();
|
||||
storeTracker.setPartialFlag(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ProcedureWALFile initOldLog(final FileStatus logFile) throws IOException {
|
||||
ProcedureWALFile log = new ProcedureWALFile(fs, logFile);
|
||||
if (logFile.getLen() == 0) {
|
||||
LOG.warn("Remove uninitialized log " + logFile);
|
||||
log.removeFile();
|
||||
return null;
|
||||
}
|
||||
|
||||
LOG.debug("opening state-log: " + logFile);
|
||||
try {
|
||||
log.open();
|
||||
} catch (ProcedureWALFormat.InvalidWALDataException e) {
|
||||
LOG.warn("Remove uninitialized log " + logFile, e);
|
||||
log.removeFile();
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
String msg = "Unable to read state log: " + logFile;
|
||||
LOG.error(msg, e);
|
||||
throw new IOException(msg, e);
|
||||
}
|
||||
|
||||
if (log.isCompacted()) {
|
||||
try {
|
||||
log.readTrailer();
|
||||
} catch (IOException e) {
|
||||
// unfinished compacted log throw it away
|
||||
LOG.warn("Unfinished compacted log " + logFile, e);
|
||||
log.removeFile();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return log;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Similar to the ByteArrayOutputStream, with the exception that we can prepend an header.
|
||||
* e.g. you write some data and you want to prepend an header that contains the data len or cksum.
|
||||
* <code>
|
||||
* ByteSlot slot = new ByteSlot();
|
||||
* // write data
|
||||
* slot.write(...);
|
||||
* slot.write(...);
|
||||
* // write header with the size of the written data
|
||||
* slot.markHead();
|
||||
* slot.write(Bytes.toBytes(slot.size()));
|
||||
* // flush to stream as [header, data]
|
||||
* slot.writeTo(stream);
|
||||
* </code>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class ByteSlot extends OutputStream {
|
||||
private static final int DOUBLE_GROW_LIMIT = 1 << 20;
|
||||
private static final int GROW_ALIGN = 128;
|
||||
|
||||
private byte[] buf;
|
||||
private int head;
|
||||
private int size;
|
||||
|
||||
public void reset() {
|
||||
head = 0;
|
||||
size = 0;
|
||||
}
|
||||
|
||||
public void markHead() {
|
||||
head = size;
|
||||
}
|
||||
|
||||
public int getHead() {
|
||||
return head;
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public byte[] getBuffer() {
|
||||
return buf;
|
||||
}
|
||||
|
||||
public void writeAt(int offset, int b) {
|
||||
head = Math.min(head, offset);
|
||||
buf[offset] = (byte)b;
|
||||
}
|
||||
|
||||
public void write(int b) {
|
||||
ensureCapacity(size + 1);
|
||||
buf[size++] = (byte)b;
|
||||
}
|
||||
|
||||
public void write(byte[] b, int off, int len) {
|
||||
ensureCapacity(size + len);
|
||||
System.arraycopy(b, off, buf, size, len);
|
||||
size += len;
|
||||
}
|
||||
|
||||
public void writeTo(final OutputStream stream) throws IOException {
|
||||
if (head != 0) {
|
||||
stream.write(buf, head, size - head);
|
||||
stream.write(buf, 0, head);
|
||||
} else {
|
||||
stream.write(buf, 0, size);
|
||||
}
|
||||
}
|
||||
|
||||
private void ensureCapacity(int minCapacity) {
|
||||
minCapacity = (minCapacity + (GROW_ALIGN - 1)) & -GROW_ALIGN;
|
||||
if (buf == null) {
|
||||
buf = new byte[minCapacity];
|
||||
} else if (minCapacity > buf.length) {
|
||||
int newCapacity = buf.length << 1;
|
||||
if (minCapacity > newCapacity || newCapacity > DOUBLE_GROW_LIMIT) {
|
||||
newCapacity = minCapacity;
|
||||
}
|
||||
buf = Arrays.copyOf(buf, newCapacity);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public final class StringUtils {
|
||||
private StringUtils() {}
|
||||
|
||||
public static String humanTimeDiff(long timeDiff) {
|
||||
StringBuilder buf = new StringBuilder();
|
||||
long hours = timeDiff / (60*60*1000);
|
||||
long rem = (timeDiff % (60*60*1000));
|
||||
long minutes = rem / (60*1000);
|
||||
rem = rem % (60*1000);
|
||||
float seconds = rem / 1000.0f;
|
||||
|
||||
if (hours != 0){
|
||||
buf.append(hours);
|
||||
buf.append("hrs, ");
|
||||
}
|
||||
if (minutes != 0){
|
||||
buf.append(minutes);
|
||||
buf.append("mins, ");
|
||||
}
|
||||
if (hours > 0 || minutes > 0) {
|
||||
buf.append(seconds);
|
||||
buf.append("sec");
|
||||
} else {
|
||||
buf.append(String.format("%.4fsec", seconds));
|
||||
}
|
||||
return buf.toString();
|
||||
}
|
||||
|
||||
public static String humanSize(double size) {
|
||||
if (size >= (1L << 40)) return String.format("%.1fT", size / (1L << 40));
|
||||
if (size >= (1L << 30)) return String.format("%.1fG", size / (1L << 30));
|
||||
if (size >= (1L << 20)) return String.format("%.1fM", size / (1L << 20));
|
||||
if (size >= (1L << 10)) return String.format("%.1fK", size / (1L << 10));
|
||||
return String.format("%.0f", size);
|
||||
}
|
||||
|
||||
public static boolean isEmpty(final String input) {
|
||||
return input == null || input.length() == 0;
|
||||
}
|
||||
|
||||
public static String buildString(final String... parts) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (int i = 0; i < parts.length; ++i) {
|
||||
sb.append(parts[i]);
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public static StringBuilder appendStrings(final StringBuilder sb, final String... parts) {
|
||||
for (int i = 0; i < parts.length; ++i) {
|
||||
sb.append(parts[i]);
|
||||
}
|
||||
return sb;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,217 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class TimeoutBlockingQueue<E> {
|
||||
public static interface TimeoutRetriever<T> {
|
||||
long getTimeout(T object);
|
||||
TimeUnit getTimeUnit(T object);
|
||||
}
|
||||
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
private final Condition waitCond = lock.newCondition();
|
||||
private final TimeoutRetriever<? super E> timeoutRetriever;
|
||||
|
||||
private E[] objects;
|
||||
private int head = 0;
|
||||
private int tail = 0;
|
||||
|
||||
public TimeoutBlockingQueue(TimeoutRetriever<? super E> timeoutRetriever) {
|
||||
this(32, timeoutRetriever);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public TimeoutBlockingQueue(int capacity, TimeoutRetriever<? super E> timeoutRetriever) {
|
||||
this.objects = (E[])new Object[capacity];
|
||||
this.timeoutRetriever = timeoutRetriever;
|
||||
}
|
||||
|
||||
public void dump() {
|
||||
for (int i = 0; i < objects.length; ++i) {
|
||||
if (i == head) {
|
||||
System.out.print("[" + objects[i] + "] ");
|
||||
} else if (i == tail) {
|
||||
System.out.print("]" + objects[i] + "[ ");
|
||||
} else {
|
||||
System.out.print(objects[i] + " ");
|
||||
}
|
||||
}
|
||||
System.out.println();
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
lock.lock();
|
||||
try {
|
||||
if (head != tail) {
|
||||
for (int i = head; i < tail; ++i) {
|
||||
objects[i] = null;
|
||||
}
|
||||
head = 0;
|
||||
tail = 0;
|
||||
waitCond.signal();
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public void add(E e) {
|
||||
if (e == null) throw new NullPointerException();
|
||||
|
||||
lock.lock();
|
||||
try {
|
||||
addElement(e);
|
||||
waitCond.signal();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
|
||||
public E poll() {
|
||||
lock.lock();
|
||||
try {
|
||||
if (isEmpty()) {
|
||||
waitCond.await();
|
||||
return null;
|
||||
}
|
||||
|
||||
E elem = objects[head];
|
||||
long nanos = getNanosTimeout(elem);
|
||||
nanos = waitCond.awaitNanos(nanos);
|
||||
return nanos > 0 ? null : removeFirst();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
return null;
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return tail - head;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return (tail - head) == 0;
|
||||
}
|
||||
|
||||
public void signalAll() {
|
||||
lock.lock();
|
||||
try {
|
||||
waitCond.signalAll();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private void addElement(E elem) {
|
||||
int size = (tail - head);
|
||||
if ((objects.length - size) == 0) {
|
||||
int capacity = size + ((size < 64) ? (size + 2) : (size >> 1));
|
||||
E[] newObjects = (E[])new Object[capacity];
|
||||
|
||||
if (compareTimeouts(objects[tail - 1], elem) <= 0) {
|
||||
// Append
|
||||
System.arraycopy(objects, head, newObjects, 0, tail);
|
||||
tail -= head;
|
||||
newObjects[tail++] = elem;
|
||||
} else if (compareTimeouts(objects[head], elem) > 0) {
|
||||
// Prepend
|
||||
System.arraycopy(objects, head, newObjects, 1, tail);
|
||||
newObjects[0] = elem;
|
||||
tail -= (head - 1);
|
||||
} else {
|
||||
// Insert in the middle
|
||||
int index = upperBound(head, tail - 1, elem);
|
||||
int newIndex = (index - head);
|
||||
System.arraycopy(objects, head, newObjects, 0, newIndex);
|
||||
newObjects[newIndex] = elem;
|
||||
System.arraycopy(objects, index, newObjects, newIndex + 1, tail - index);
|
||||
tail -= (head - 1);
|
||||
}
|
||||
head = 0;
|
||||
objects = newObjects;
|
||||
} else {
|
||||
if (tail == objects.length) {
|
||||
// shift down |-----AAAAAAA|
|
||||
tail -= head;
|
||||
System.arraycopy(objects, head, objects, 0, tail);
|
||||
head = 0;
|
||||
}
|
||||
|
||||
if (tail == head || compareTimeouts(objects[tail - 1], elem) <= 0) {
|
||||
// Append
|
||||
objects[tail++] = elem;
|
||||
} else if (head > 0 && compareTimeouts(objects[head], elem) > 0) {
|
||||
// Prepend
|
||||
objects[--head] = elem;
|
||||
} else {
|
||||
// Insert in the middle
|
||||
int index = upperBound(head, tail - 1, elem);
|
||||
System.arraycopy(objects, index, objects, index + 1, tail - index);
|
||||
objects[index] = elem;
|
||||
tail++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private E removeFirst() {
|
||||
E elem = objects[head];
|
||||
objects[head] = null;
|
||||
head = (head + 1) % objects.length;
|
||||
if (head == 0) tail = 0;
|
||||
return elem;
|
||||
}
|
||||
|
||||
private int upperBound(int start, int end, E key) {
|
||||
while (start < end) {
|
||||
int mid = (start + end) >>> 1;
|
||||
E mitem = objects[mid];
|
||||
int cmp = compareTimeouts(mitem, key);
|
||||
if (cmp > 0) {
|
||||
end = mid;
|
||||
} else {
|
||||
start = mid + 1;
|
||||
}
|
||||
}
|
||||
return start;
|
||||
}
|
||||
|
||||
private int compareTimeouts(final E a, final E b) {
|
||||
long t1 = getNanosTimeout(a);
|
||||
long t2 = getNanosTimeout(b);
|
||||
return (t1 < t2) ? -1 : (t1 > t2) ? 1 : 0;
|
||||
}
|
||||
|
||||
private long getNanosTimeout(final E obj) {
|
||||
TimeUnit unit = timeoutRetriever.getTimeUnit(obj);
|
||||
long timeout = timeoutRetriever.getTimeout(obj);
|
||||
return unit.toNanos(timeout);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,163 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
|
||||
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
|
||||
|
||||
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);
|
||||
|
||||
private ProcedureTestingUtility() {
|
||||
}
|
||||
|
||||
public static ProcedureStore createStore(final Configuration conf, final FileSystem fs,
|
||||
final Path baseDir) throws IOException {
|
||||
return createWalStore(conf, fs, baseDir);
|
||||
}
|
||||
|
||||
public static WALProcedureStore createWalStore(final Configuration conf, final FileSystem fs,
|
||||
final Path logDir) throws IOException {
|
||||
return new WALProcedureStore(conf, fs, logDir, new WALProcedureStore.LeaseRecovery() {
|
||||
@Override
|
||||
public void recoverFileLease(FileSystem fs, Path path) throws IOException {
|
||||
// no-op
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static <TEnv> void restart(ProcedureExecutor<TEnv> procExecutor)
|
||||
throws Exception {
|
||||
restart(procExecutor, null);
|
||||
}
|
||||
|
||||
public static <TEnv> void restart(ProcedureExecutor<TEnv> procExecutor,
|
||||
Runnable beforeStartAction) throws Exception {
|
||||
ProcedureStore procStore = procExecutor.getStore();
|
||||
int storeThreads = procExecutor.getNumThreads();
|
||||
int execThreads = procExecutor.getNumThreads();
|
||||
// stop
|
||||
procExecutor.stop();
|
||||
procStore.stop(false);
|
||||
procExecutor.join();
|
||||
// nothing running...
|
||||
if (beforeStartAction != null) {
|
||||
beforeStartAction.run();
|
||||
}
|
||||
// re-start
|
||||
procStore.start(storeThreads);
|
||||
procExecutor.start(execThreads);
|
||||
}
|
||||
|
||||
public static <TEnv> void setKillBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor,
|
||||
boolean value) {
|
||||
if (procExecutor.testing == null) {
|
||||
procExecutor.testing = new ProcedureExecutor.Testing();
|
||||
}
|
||||
procExecutor.testing.killBeforeStoreUpdate = value;
|
||||
LOG.warn("Set Kill before store update to: " + procExecutor.testing.killBeforeStoreUpdate);
|
||||
}
|
||||
|
||||
public static <TEnv> void setToggleKillBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor,
|
||||
boolean value) {
|
||||
if (procExecutor.testing == null) {
|
||||
procExecutor.testing = new ProcedureExecutor.Testing();
|
||||
}
|
||||
procExecutor.testing.toggleKillBeforeStoreUpdate = value;
|
||||
}
|
||||
|
||||
public static <TEnv> void toggleKillBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor) {
|
||||
if (procExecutor.testing == null) {
|
||||
procExecutor.testing = new ProcedureExecutor.Testing();
|
||||
}
|
||||
procExecutor.testing.killBeforeStoreUpdate = !procExecutor.testing.killBeforeStoreUpdate;
|
||||
LOG.warn("Set Kill before store update to: " + procExecutor.testing.killBeforeStoreUpdate);
|
||||
}
|
||||
|
||||
public static <TEnv> void setKillAndToggleBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor,
|
||||
boolean value) {
|
||||
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, value);
|
||||
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, value);
|
||||
}
|
||||
|
||||
public static <TEnv> long submitAndWait(ProcedureExecutor<TEnv> procExecutor, Procedure proc) {
|
||||
long procId = procExecutor.submitProcedure(proc);
|
||||
waitProcedure(procExecutor, procId);
|
||||
return procId;
|
||||
}
|
||||
|
||||
public static <TEnv> void waitProcedure(ProcedureExecutor<TEnv> procExecutor, long procId) {
|
||||
while (!procExecutor.isFinished(procId) && procExecutor.isRunning()) {
|
||||
Threads.sleepWithoutInterrupt(250);
|
||||
}
|
||||
}
|
||||
|
||||
public static <TEnv> void waitNoProcedureRunning(ProcedureExecutor<TEnv> procExecutor) {
|
||||
int stableRuns = 0;
|
||||
while (stableRuns < 10) {
|
||||
if (procExecutor.getActiveExecutorCount() > 0 || procExecutor.getRunnableSet().size() > 0) {
|
||||
stableRuns = 0;
|
||||
Threads.sleepWithoutInterrupt(100);
|
||||
} else {
|
||||
stableRuns++;
|
||||
Threads.sleepWithoutInterrupt(25);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static <TEnv> void assertProcNotYetCompleted(ProcedureExecutor<TEnv> procExecutor,
|
||||
long procId) {
|
||||
assertFalse("expected a running proc", procExecutor.isFinished(procId));
|
||||
assertEquals(null, procExecutor.getResult(procId));
|
||||
}
|
||||
|
||||
public static <TEnv> void assertProcNotFailed(ProcedureExecutor<TEnv> procExecutor,
|
||||
long procId) {
|
||||
ProcedureResult 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";
|
||||
assertFalse(msg, result.isFailed());
|
||||
}
|
||||
|
||||
public static void assertIsAbortException(final ProcedureResult result) {
|
||||
LOG.info(result.getException());
|
||||
assertEquals(true, result.isFailed());
|
||||
Throwable cause = result.getException().getCause();
|
||||
assertTrue("expected abort exception, got "+ cause,
|
||||
cause instanceof ProcedureAbortedException);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,338 @@
|
|||
/**
|
||||
* 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 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.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 {
|
||||
private static final Log LOG = LogFactory.getLog(TestProcedureExecution.class);
|
||||
|
||||
private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
|
||||
private static final Procedure NULL_PROC = null;
|
||||
|
||||
private ProcedureExecutor<Void> procExecutor;
|
||||
private ProcedureStore procStore;
|
||||
|
||||
private HBaseCommonTestingUtility htu;
|
||||
private FileSystem fs;
|
||||
private Path testDir;
|
||||
private Path logDir;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
htu = new HBaseCommonTestingUtility();
|
||||
testDir = htu.getDataTestDir();
|
||||
fs = testDir.getFileSystem(htu.getConfiguration());
|
||||
assertTrue(testDir.depth() > 1);
|
||||
|
||||
logDir = new Path(testDir, "proc-logs");
|
||||
procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
|
||||
procExecutor = new ProcedureExecutor(htu.getConfiguration(), null, procStore);
|
||||
procStore.start(PROCEDURE_EXECUTOR_SLOTS);
|
||||
procExecutor.start(PROCEDURE_EXECUTOR_SLOTS);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
procExecutor.stop();
|
||||
procStore.stop(false);
|
||||
fs.delete(logDir, true);
|
||||
}
|
||||
|
||||
private static class TestProcedureException extends Exception {
|
||||
public TestProcedureException(String msg) { super(msg); }
|
||||
}
|
||||
|
||||
public static class TestSequentialProcedure extends SequentialProcedure<Void> {
|
||||
private final Procedure[] subProcs;
|
||||
private final List<String> state;
|
||||
private final Exception failure;
|
||||
private final String name;
|
||||
|
||||
public TestSequentialProcedure() {
|
||||
throw new UnsupportedOperationException("recovery should not be triggered here");
|
||||
}
|
||||
|
||||
public TestSequentialProcedure(String name, List<String> state, Procedure... subProcs) {
|
||||
this.state = state;
|
||||
this.subProcs = subProcs;
|
||||
this.name = name;
|
||||
this.failure = null;
|
||||
}
|
||||
|
||||
public TestSequentialProcedure(String name, List<String> state, Exception failure) {
|
||||
this.state = state;
|
||||
this.subProcs = null;
|
||||
this.name = name;
|
||||
this.failure = failure;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
state.add(name + "-execute");
|
||||
if (failure != null) {
|
||||
setFailure(new RemoteProcedureException(name + "-failure", failure));
|
||||
return null;
|
||||
}
|
||||
return subProcs;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) {
|
||||
state.add(name + "-rollback");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) {
|
||||
state.add(name + "-abort");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testBadSubprocList() {
|
||||
List<String> state = new ArrayList<String>();
|
||||
Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
|
||||
Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2, NULL_PROC);
|
||||
Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
|
||||
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
|
||||
|
||||
// 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());
|
||||
assertTrue(state.toString(), result.isFailed());
|
||||
assertTrue(result.getException().toString(),
|
||||
result.getException().getCause() instanceof IllegalArgumentException);
|
||||
|
||||
assertEquals(state.toString(), 4, state.size());
|
||||
assertEquals("rootProc-execute", state.get(0));
|
||||
assertEquals("subProc1-execute", state.get(1));
|
||||
assertEquals("subProc1-rollback", state.get(2));
|
||||
assertEquals("rootProc-rollback", state.get(3));
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testSingleSequentialProc() {
|
||||
List<String> state = new ArrayList<String>();
|
||||
Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
|
||||
Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
|
||||
Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
|
||||
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
|
||||
|
||||
// successful state, with 3 execute
|
||||
LOG.info(state);
|
||||
ProcedureResult result = procExecutor.getResult(rootId);
|
||||
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||
assertEquals(state.toString(), 3, state.size());
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testSingleSequentialProcRollback() {
|
||||
List<String> state = new ArrayList<String>();
|
||||
Procedure subProc2 = new TestSequentialProcedure("subProc2", state,
|
||||
new TestProcedureException("fail test"));
|
||||
Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
|
||||
Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
|
||||
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
|
||||
|
||||
// the 3rd proc fail, rollback after 2 successful execution
|
||||
LOG.info(state);
|
||||
ProcedureResult result = procExecutor.getResult(rootId);
|
||||
LOG.info(result.getException());
|
||||
assertTrue(state.toString(), result.isFailed());
|
||||
assertTrue(result.getException().toString(),
|
||||
result.getException().getCause() instanceof TestProcedureException);
|
||||
|
||||
assertEquals(state.toString(), 6, state.size());
|
||||
assertEquals("rootProc-execute", state.get(0));
|
||||
assertEquals("subProc1-execute", state.get(1));
|
||||
assertEquals("subProc2-execute", state.get(2));
|
||||
assertEquals("subProc2-rollback", state.get(3));
|
||||
assertEquals("subProc1-rollback", state.get(4));
|
||||
assertEquals("rootProc-rollback", state.get(5));
|
||||
}
|
||||
|
||||
public static class TestFaultyRollback extends SequentialProcedure<Void> {
|
||||
private int retries = 0;
|
||||
|
||||
public TestFaultyRollback() { }
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
setFailure("faulty-rollback-test", new TestProcedureException("test faulty rollback"));
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) throws IOException {
|
||||
if (++retries < 3) {
|
||||
LOG.info("inject rollback failure " + retries);
|
||||
throw new IOException("injected failure number " + retries);
|
||||
}
|
||||
LOG.info("execute non faulty rollback step retries=" + retries);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) { return false; }
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testRollbackRetriableFailure() {
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, new TestFaultyRollback());
|
||||
|
||||
ProcedureResult result = procExecutor.getResult(procId);
|
||||
LOG.info(result.getException());
|
||||
assertTrue("expected a failure", result.isFailed());
|
||||
assertTrue(result.getException().toString(),
|
||||
result.getException().getCause() instanceof TestProcedureException);
|
||||
}
|
||||
|
||||
public static class TestWaitingProcedure extends SequentialProcedure<Void> {
|
||||
private final List<String> state;
|
||||
private final boolean hasChild;
|
||||
private final String name;
|
||||
|
||||
public TestWaitingProcedure() {
|
||||
throw new UnsupportedOperationException("recovery should not be triggered here");
|
||||
}
|
||||
|
||||
public TestWaitingProcedure(String name, List<String> state, boolean hasChild) {
|
||||
this.hasChild = hasChild;
|
||||
this.state = state;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
state.add(name + "-execute");
|
||||
setState(ProcedureState.WAITING_TIMEOUT);
|
||||
return hasChild ? new Procedure[] { new TestWaitChild(name, state) } : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) {
|
||||
state.add(name + "-rollback");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) {
|
||||
state.add(name + "-abort");
|
||||
return true;
|
||||
}
|
||||
|
||||
public static class TestWaitChild extends SequentialProcedure<Void> {
|
||||
private final List<String> state;
|
||||
private final String name;
|
||||
|
||||
public TestWaitChild() {
|
||||
throw new UnsupportedOperationException("recovery should not be triggered here");
|
||||
}
|
||||
|
||||
public TestWaitChild(String name, List<String> state) {
|
||||
this.name = name;
|
||||
this.state = state;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
state.add(name + "-child-execute");
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) {
|
||||
state.add(name + "-child-rollback");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) {
|
||||
state.add(name + "-child-abort");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testAbortTimeout() {
|
||||
final int PROC_TIMEOUT_MSEC = 2500;
|
||||
List<String> state = new ArrayList<String>();
|
||||
Procedure proc = new TestWaitingProcedure("wproc", state, false);
|
||||
proc.setTimeout(PROC_TIMEOUT_MSEC);
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
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());
|
||||
assertTrue(state.toString(), result.isFailed());
|
||||
assertTrue(result.getException().toString(),
|
||||
result.getException().getCause() instanceof TimeoutException);
|
||||
assertEquals(state.toString(), 2, state.size());
|
||||
assertEquals("wproc-execute", state.get(0));
|
||||
assertEquals("wproc-rollback", state.get(1));
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testAbortTimeoutWithChildren() {
|
||||
List<String> state = new ArrayList<String>();
|
||||
Procedure proc = new TestWaitingProcedure("wproc", state, true);
|
||||
proc.setTimeout(2500);
|
||||
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
LOG.info(state);
|
||||
ProcedureResult result = procExecutor.getResult(rootId);
|
||||
LOG.info(result.getException());
|
||||
assertTrue(state.toString(), result.isFailed());
|
||||
assertTrue(result.getException().toString(),
|
||||
result.getException().getCause() instanceof TimeoutException);
|
||||
assertEquals(state.toString(), 4, state.size());
|
||||
assertEquals("wproc-execute", state.get(0));
|
||||
assertEquals("wproc-child-execute", state.get(1));
|
||||
assertEquals("wproc-child-rollback", state.get(2));
|
||||
assertEquals("wproc-rollback", state.get(3));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,155 @@
|
|||
/**
|
||||
* 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.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
@Category({MasterTests.class, SmallTests.class})
|
||||
public class TestProcedureFairRunQueues {
|
||||
private static class TestRunQueue implements ProcedureFairRunQueues.FairObject {
|
||||
private final int priority;
|
||||
private final String name;
|
||||
|
||||
private boolean available = true;
|
||||
|
||||
public TestRunQueue(String name, int priority) {
|
||||
this.name = name;
|
||||
this.priority = priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return name;
|
||||
}
|
||||
|
||||
private void setAvailable(boolean available) {
|
||||
this.available = available;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAvailable() {
|
||||
return available;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority() {
|
||||
return priority;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyFairQueues() throws Exception {
|
||||
ProcedureFairRunQueues<String, TestRunQueue> fairq
|
||||
= new ProcedureFairRunQueues<String, TestRunQueue>(1);
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
assertEquals(null, fairq.poll());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFairQueues() throws Exception {
|
||||
ProcedureFairRunQueues<String, TestRunQueue> fairq
|
||||
= new ProcedureFairRunQueues<String, TestRunQueue>(1);
|
||||
TestRunQueue a = fairq.add("A", new TestRunQueue("A", 1));
|
||||
TestRunQueue b = fairq.add("B", new TestRunQueue("B", 1));
|
||||
TestRunQueue m = fairq.add("M", new TestRunQueue("M", 2));
|
||||
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
assertEquals(a, fairq.poll());
|
||||
assertEquals(b, fairq.poll());
|
||||
assertEquals(m, fairq.poll());
|
||||
assertEquals(m, fairq.poll());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFairQueuesNotAvailable() throws Exception {
|
||||
ProcedureFairRunQueues<String, TestRunQueue> fairq
|
||||
= new ProcedureFairRunQueues<String, TestRunQueue>(1);
|
||||
TestRunQueue a = fairq.add("A", new TestRunQueue("A", 1));
|
||||
TestRunQueue b = fairq.add("B", new TestRunQueue("B", 1));
|
||||
TestRunQueue m = fairq.add("M", new TestRunQueue("M", 2));
|
||||
|
||||
// m is not available
|
||||
m.setAvailable(false);
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
assertEquals(a, fairq.poll());
|
||||
assertEquals(b, fairq.poll());
|
||||
}
|
||||
|
||||
// m is available
|
||||
m.setAvailable(true);
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
assertEquals(m, fairq.poll());
|
||||
assertEquals(m, fairq.poll());
|
||||
assertEquals(a, fairq.poll());
|
||||
assertEquals(b, fairq.poll());
|
||||
}
|
||||
|
||||
// b is not available
|
||||
b.setAvailable(false);
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
assertEquals(m, fairq.poll());
|
||||
assertEquals(m, fairq.poll());
|
||||
assertEquals(a, fairq.poll());
|
||||
}
|
||||
|
||||
assertEquals(m, fairq.poll());
|
||||
m.setAvailable(false);
|
||||
// m should be fetched next, but is no longer available
|
||||
assertEquals(a, fairq.poll());
|
||||
assertEquals(a, fairq.poll());
|
||||
b.setAvailable(true);
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
assertEquals(b, fairq.poll());
|
||||
assertEquals(a, fairq.poll());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFairQueuesDelete() throws Exception {
|
||||
ProcedureFairRunQueues<String, TestRunQueue> fairq
|
||||
= new ProcedureFairRunQueues<String, TestRunQueue>(1);
|
||||
TestRunQueue a = fairq.add("A", new TestRunQueue("A", 1));
|
||||
TestRunQueue b = fairq.add("B", new TestRunQueue("B", 1));
|
||||
TestRunQueue m = fairq.add("M", new TestRunQueue("M", 2));
|
||||
|
||||
// Fetch A and then remove it
|
||||
assertEquals(a, fairq.poll());
|
||||
assertEquals(a, fairq.remove("A"));
|
||||
|
||||
// Fetch B and then remove it
|
||||
assertEquals(b, fairq.poll());
|
||||
assertEquals(b, fairq.remove("B"));
|
||||
|
||||
// Fetch M and then remove it
|
||||
assertEquals(m, fairq.poll());
|
||||
assertEquals(m, fairq.remove("M"));
|
||||
|
||||
// nothing left
|
||||
assertEquals(null, fairq.poll());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,488 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.procedure2.store.ProcedureStore;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
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 TestProcedureRecovery {
|
||||
private static final Log LOG = LogFactory.getLog(TestProcedureRecovery.class);
|
||||
|
||||
private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
|
||||
private static final Procedure NULL_PROC = null;
|
||||
|
||||
private static ProcedureExecutor<Void> procExecutor;
|
||||
private static ProcedureStore procStore;
|
||||
private static int procSleepInterval;
|
||||
|
||||
private HBaseCommonTestingUtility htu;
|
||||
private FileSystem fs;
|
||||
private Path testDir;
|
||||
private Path logDir;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
htu = new HBaseCommonTestingUtility();
|
||||
testDir = htu.getDataTestDir();
|
||||
fs = testDir.getFileSystem(htu.getConfiguration());
|
||||
assertTrue(testDir.depth() > 1);
|
||||
|
||||
logDir = new Path(testDir, "proc-logs");
|
||||
procStore = ProcedureTestingUtility.createStore(htu.getConfiguration(), fs, logDir);
|
||||
procExecutor = new ProcedureExecutor(htu.getConfiguration(), null, procStore);
|
||||
procExecutor.testing = new ProcedureExecutor.Testing();
|
||||
procStore.start(PROCEDURE_EXECUTOR_SLOTS);
|
||||
procExecutor.start(PROCEDURE_EXECUTOR_SLOTS);
|
||||
procSleepInterval = 0;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
procExecutor.stop();
|
||||
procStore.stop(false);
|
||||
fs.delete(logDir, true);
|
||||
}
|
||||
|
||||
private void restart() throws Exception {
|
||||
dumpLogDirState();
|
||||
ProcedureTestingUtility.restart(procExecutor);
|
||||
dumpLogDirState();
|
||||
}
|
||||
|
||||
public static class TestSingleStepProcedure extends SequentialProcedure<Void> {
|
||||
private int step = 0;
|
||||
|
||||
public TestSingleStepProcedure() { }
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
LOG.debug("execute procedure " + this + " step=" + step);
|
||||
step++;
|
||||
setResult(Bytes.toBytes(step));
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) { }
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) { return true; }
|
||||
}
|
||||
|
||||
public static class BaseTestStepProcedure extends SequentialProcedure<Void> {
|
||||
private AtomicBoolean abort = new AtomicBoolean(false);
|
||||
private int step = 0;
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
LOG.debug("execute procedure " + this + " step=" + step);
|
||||
ProcedureTestingUtility.toggleKillBeforeStoreUpdate(procExecutor);
|
||||
step++;
|
||||
Threads.sleepWithoutInterrupt(procSleepInterval);
|
||||
if (isAborted()) {
|
||||
setFailure(new RemoteProcedureException(getClass().getName(),
|
||||
new ProcedureAbortedException(
|
||||
"got an abort at " + getClass().getName() + " step=" + step)));
|
||||
return null;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) {
|
||||
LOG.debug("rollback procedure " + this + " step=" + step);
|
||||
ProcedureTestingUtility.toggleKillBeforeStoreUpdate(procExecutor);
|
||||
step++;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) {
|
||||
abort.set(true);
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean isAborted() {
|
||||
boolean aborted = abort.get();
|
||||
BaseTestStepProcedure proc = this;
|
||||
while (proc.hasParent() && !aborted) {
|
||||
proc = (BaseTestStepProcedure)procExecutor.getProcedure(proc.getParentProcId());
|
||||
aborted = proc.isAborted();
|
||||
}
|
||||
return aborted;
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestMultiStepProcedure extends BaseTestStepProcedure {
|
||||
public TestMultiStepProcedure() { }
|
||||
|
||||
@Override
|
||||
public Procedure[] execute(Void env) {
|
||||
super.execute(env);
|
||||
return isFailed() ? null : new Procedure[] { new Step1Procedure() };
|
||||
}
|
||||
|
||||
public static class Step1Procedure extends BaseTestStepProcedure {
|
||||
public Step1Procedure() { }
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) {
|
||||
super.execute(env);
|
||||
return isFailed() ? null : new Procedure[] { new Step2Procedure() };
|
||||
}
|
||||
}
|
||||
|
||||
public static class Step2Procedure extends BaseTestStepProcedure {
|
||||
public Step2Procedure() { }
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoopLoad() throws Exception {
|
||||
restart();
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testSingleStepProcRecovery() throws Exception {
|
||||
Procedure proc = new TestSingleStepProcedure();
|
||||
procExecutor.testing.killBeforeStoreUpdate = true;
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
procExecutor.testing.killBeforeStoreUpdate = false;
|
||||
|
||||
// Restart and verify that the procedures restart
|
||||
long restartTs = EnvironmentEdgeManager.currentTime();
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureResult result = procExecutor.getResult(procId);
|
||||
assertTrue(result.getLastUpdate() > restartTs);
|
||||
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||
assertEquals(1, Bytes.toInt(result.getResult()));
|
||||
long resultTs = result.getLastUpdate();
|
||||
|
||||
// Verify that after another restart the result is still there
|
||||
restart();
|
||||
result = procExecutor.getResult(procId);
|
||||
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||
assertEquals(resultTs, result.getLastUpdate());
|
||||
assertEquals(1, Bytes.toInt(result.getResult()));
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testMultiStepProcRecovery() throws Exception {
|
||||
// Step 0 - kill
|
||||
Procedure proc = new TestMultiStepProcedure();
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 0 exec && Step 1 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 1 exec && step 2 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 2 exec
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
assertTrue(procExecutor.isRunning());
|
||||
|
||||
// The procedure is completed
|
||||
ProcedureResult result = procExecutor.getResult(procId);
|
||||
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testMultiStepRollbackRecovery() throws Exception {
|
||||
// Step 0 - kill
|
||||
Procedure proc = new TestMultiStepProcedure();
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 0 exec && Step 1 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 1 exec && step 2 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 2 exec - rollback - kill
|
||||
procSleepInterval = 2500;
|
||||
restart();
|
||||
assertTrue(procExecutor.abort(procId));
|
||||
waitProcedure(procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// rollback - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// rollback - complete
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Restart the executor and get the result
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
|
||||
// The procedure is completed
|
||||
ProcedureResult result = procExecutor.getResult(procId);
|
||||
ProcedureTestingUtility.assertIsAbortException(result);
|
||||
}
|
||||
|
||||
public static class TestStateMachineProcedure
|
||||
extends StateMachineProcedure<Void, TestStateMachineProcedure.State> {
|
||||
enum State { STATE_1, STATE_2, STATE_3, DONE }
|
||||
|
||||
public TestStateMachineProcedure() {}
|
||||
|
||||
private AtomicBoolean aborted = new AtomicBoolean(false);
|
||||
private int iResult = 0;
|
||||
|
||||
@Override
|
||||
protected StateMachineProcedure.Flow executeFromState(Void env, State state) {
|
||||
switch (state) {
|
||||
case STATE_1:
|
||||
LOG.info("execute step 1 " + this);
|
||||
setNextState(State.STATE_2);
|
||||
iResult += 3;
|
||||
break;
|
||||
case STATE_2:
|
||||
LOG.info("execute step 2 " + this);
|
||||
setNextState(State.STATE_3);
|
||||
iResult += 5;
|
||||
break;
|
||||
case STATE_3:
|
||||
LOG.info("execute step 3 " + this);
|
||||
Threads.sleepWithoutInterrupt(procSleepInterval);
|
||||
if (aborted.get()) {
|
||||
LOG.info("aborted step 3 " + this);
|
||||
setAbortFailure("test", "aborted");
|
||||
break;
|
||||
}
|
||||
setNextState(State.DONE);
|
||||
iResult += 7;
|
||||
setResult(Bytes.toBytes(iResult));
|
||||
return Flow.NO_MORE_STATE;
|
||||
default:
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
return Flow.HAS_MORE_STATE;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollbackState(Void env, final State state) {
|
||||
switch (state) {
|
||||
case STATE_1:
|
||||
LOG.info("rollback step 1 " + this);
|
||||
break;
|
||||
case STATE_2:
|
||||
LOG.info("rollback step 2 " + this);
|
||||
break;
|
||||
case STATE_3:
|
||||
LOG.info("rollback step 3 " + this);
|
||||
break;
|
||||
default:
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected State getState(final int stateId) {
|
||||
return State.values()[stateId];
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getStateId(final State state) {
|
||||
return state.ordinal();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected State getInitialState() {
|
||||
return State.STATE_1;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) {
|
||||
aborted.set(true);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serializeStateData(final OutputStream stream) throws IOException {
|
||||
super.serializeStateData(stream);
|
||||
stream.write(Bytes.toBytes(iResult));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void deserializeStateData(final InputStream stream) throws IOException {
|
||||
super.deserializeStateData(stream);
|
||||
byte[] data = new byte[4];
|
||||
stream.read(data);
|
||||
iResult = Bytes.toInt(data);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testStateMachineRecovery() throws Exception {
|
||||
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
|
||||
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
|
||||
|
||||
// Step 1 - kill
|
||||
Procedure proc = new TestStateMachineProcedure();
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 1 exec && Step 2 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 2 exec && step 3 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 3 exec
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
assertTrue(procExecutor.isRunning());
|
||||
|
||||
// The procedure is completed
|
||||
ProcedureResult result = procExecutor.getResult(procId);
|
||||
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||
assertEquals(15, Bytes.toInt(result.getResult()));
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testStateMachineRollbackRecovery() throws Exception {
|
||||
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
|
||||
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
|
||||
|
||||
// Step 1 - kill
|
||||
Procedure proc = new TestStateMachineProcedure();
|
||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 1 exec && Step 2 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 2 exec && step 3 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Step 3 exec - rollback step 3 - kill
|
||||
procSleepInterval = 2500;
|
||||
restart();
|
||||
assertTrue(procExecutor.abort(procId));
|
||||
waitProcedure(procId);
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
|
||||
// Rollback step 3 - rollback step 2 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
|
||||
// Rollback step 2 - step 1 - kill
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
assertFalse(procExecutor.isRunning());
|
||||
ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
|
||||
|
||||
// Rollback step 1 - complete
|
||||
restart();
|
||||
waitProcedure(procId);
|
||||
assertTrue(procExecutor.isRunning());
|
||||
|
||||
// The procedure is completed
|
||||
ProcedureResult result = procExecutor.getResult(procId);
|
||||
ProcedureTestingUtility.assertIsAbortException(result);
|
||||
}
|
||||
|
||||
private void waitProcedure(final long procId) {
|
||||
ProcedureTestingUtility.waitProcedure(procExecutor, procId);
|
||||
dumpLogDirState();
|
||||
}
|
||||
|
||||
private void dumpLogDirState() {
|
||||
try {
|
||||
FileStatus[] files = fs.listStatus(logDir);
|
||||
if (files != null && files.length > 0) {
|
||||
for (FileStatus file: files) {
|
||||
assertTrue(file.toString(), file.isFile());
|
||||
LOG.debug("log file " + file.getPath() + " size=" + file.getLen());
|
||||
}
|
||||
} else {
|
||||
LOG.debug("no files under: " + logDir);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to dump " + logDir, e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,226 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
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.procedure2.store.ProcedureStore;
|
||||
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;
|
||||
|
||||
@Category({MasterTests.class, LargeTests.class})
|
||||
public class TestProcedureReplayOrder {
|
||||
private static final Log LOG = LogFactory.getLog(TestProcedureReplayOrder.class);
|
||||
|
||||
private static final Procedure NULL_PROC = null;
|
||||
|
||||
private ProcedureExecutor<Void> procExecutor;
|
||||
private TestProcedureEnv procEnv;
|
||||
private ProcedureStore procStore;
|
||||
|
||||
private HBaseCommonTestingUtility htu;
|
||||
private FileSystem fs;
|
||||
private Path testDir;
|
||||
private Path logDir;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
htu = new HBaseCommonTestingUtility();
|
||||
htu.getConfiguration().setInt("hbase.procedure.store.wal.sync.wait.msec", 10);
|
||||
|
||||
testDir = htu.getDataTestDir();
|
||||
fs = testDir.getFileSystem(htu.getConfiguration());
|
||||
assertTrue(testDir.depth() > 1);
|
||||
|
||||
logDir = new Path(testDir, "proc-logs");
|
||||
procEnv = new TestProcedureEnv();
|
||||
procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
|
||||
procExecutor = new ProcedureExecutor(htu.getConfiguration(), procEnv, procStore);
|
||||
procStore.start(24);
|
||||
procExecutor.start(1);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
procExecutor.stop();
|
||||
procStore.stop(false);
|
||||
fs.delete(logDir, true);
|
||||
}
|
||||
|
||||
@Test(timeout=90000)
|
||||
public void testSingleStepReplyOrder() throws Exception {
|
||||
// avoid the procedure to be runnable
|
||||
procEnv.setAcquireLock(false);
|
||||
|
||||
// submit the procedures
|
||||
submitProcedures(16, 25, TestSingleStepProcedure.class);
|
||||
|
||||
// restart the executor and allow the procedures to run
|
||||
ProcedureTestingUtility.restart(procExecutor, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
procEnv.setAcquireLock(true);
|
||||
}
|
||||
});
|
||||
|
||||
// wait the execution of all the procedures and
|
||||
// assert that the execution order was sorted by procId
|
||||
ProcedureTestingUtility.waitNoProcedureRunning(procExecutor);
|
||||
procEnv.assertSortedExecList();
|
||||
|
||||
// TODO: FIXME: This should be revisited
|
||||
}
|
||||
|
||||
@Ignore
|
||||
@Test(timeout=90000)
|
||||
public void testMultiStepReplyOrder() throws Exception {
|
||||
// avoid the procedure to be runnable
|
||||
procEnv.setAcquireLock(false);
|
||||
|
||||
// submit the procedures
|
||||
submitProcedures(16, 10, TestTwoStepProcedure.class);
|
||||
|
||||
// restart the executor and allow the procedures to run
|
||||
ProcedureTestingUtility.restart(procExecutor, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
procEnv.setAcquireLock(true);
|
||||
}
|
||||
});
|
||||
|
||||
fail("TODO: FIXME: NOT IMPLEMENT REPLAY ORDER");
|
||||
}
|
||||
|
||||
private void submitProcedures(final int nthreads, final int nprocPerThread,
|
||||
final Class<?> procClazz) throws Exception {
|
||||
Thread[] submitThreads = new Thread[nthreads];
|
||||
for (int i = 0; i < submitThreads.length; ++i) {
|
||||
submitThreads[i] = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
for (int i = 0; i < nprocPerThread; ++i) {
|
||||
try {
|
||||
procExecutor.submitProcedure((Procedure)procClazz.newInstance());
|
||||
} catch (InstantiationException|IllegalAccessException e) {
|
||||
LOG.error("unable to instantiate the procedure", e);
|
||||
fail("failure during the proc.newInstance(): " + e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
for (int i = 0; i < submitThreads.length; ++i) {
|
||||
submitThreads[i].start();
|
||||
}
|
||||
|
||||
for (int i = 0; i < submitThreads.length; ++i) {
|
||||
submitThreads[i].join();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TestProcedureEnv {
|
||||
private ArrayList<Long> execList = new ArrayList<Long>();
|
||||
private boolean acquireLock = true;
|
||||
|
||||
public void setAcquireLock(boolean acquireLock) {
|
||||
this.acquireLock = acquireLock;
|
||||
}
|
||||
|
||||
public boolean canAcquireLock() {
|
||||
return acquireLock;
|
||||
}
|
||||
|
||||
public void addToExecList(final Procedure proc) {
|
||||
execList.add(proc.getProcId());
|
||||
}
|
||||
|
||||
public ArrayList<Long> getExecList() {
|
||||
return execList;
|
||||
}
|
||||
|
||||
public void assertSortedExecList() {
|
||||
LOG.debug("EXEC LIST: " + execList);
|
||||
for (int i = 1; i < execList.size(); ++i) {
|
||||
assertTrue("exec list not sorted: " + execList.get(i-1) + " >= " + execList.get(i),
|
||||
execList.get(i-1) < execList.get(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestSingleStepProcedure extends SequentialProcedure<TestProcedureEnv> {
|
||||
public TestSingleStepProcedure() { }
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(TestProcedureEnv env) {
|
||||
LOG.debug("execute procedure " + this);
|
||||
env.addToExecList(this);
|
||||
return null;
|
||||
}
|
||||
|
||||
protected boolean acquireLock(final TestProcedureEnv env) {
|
||||
return env.canAcquireLock();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(TestProcedureEnv env) { }
|
||||
|
||||
@Override
|
||||
protected boolean abort(TestProcedureEnv env) { return true; }
|
||||
}
|
||||
|
||||
public static class TestTwoStepProcedure extends SequentialProcedure<TestProcedureEnv> {
|
||||
public TestTwoStepProcedure() { }
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(TestProcedureEnv env) {
|
||||
LOG.debug("execute procedure " + this);
|
||||
env.addToExecList(this);
|
||||
return new Procedure[] { new TestSingleStepProcedure() };
|
||||
}
|
||||
|
||||
protected boolean acquireLock(final TestProcedureEnv env) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void rollback(TestProcedureEnv env) { }
|
||||
|
||||
@Override
|
||||
protected boolean abort(TestProcedureEnv env) { return true; }
|
||||
}
|
||||
}
|
|
@ -0,0 +1,168 @@
|
|||
/**
|
||||
* 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.store;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
||||
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 TestProcedureStoreTracker {
|
||||
private static final Log LOG = LogFactory.getLog(TestProcedureStoreTracker.class);
|
||||
|
||||
static class TestProcedure extends Procedure<Void> {
|
||||
public TestProcedure(long procId) {
|
||||
setProcId(procId);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) { return null; }
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) { /* no-op */ }
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) { return false; }
|
||||
|
||||
@Override
|
||||
protected void serializeStateData(final OutputStream stream) { /* no-op */ }
|
||||
|
||||
@Override
|
||||
protected void deserializeStateData(final InputStream stream) { /* no-op */ }
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSeqInsertAndDelete() {
|
||||
ProcedureStoreTracker tracker = new ProcedureStoreTracker();
|
||||
assertTrue(tracker.isEmpty());
|
||||
|
||||
final int MIN_PROC = 1;
|
||||
final int MAX_PROC = 1 << 10;
|
||||
|
||||
// sequential insert
|
||||
for (int i = MIN_PROC; i < MAX_PROC; ++i) {
|
||||
tracker.insert(i);
|
||||
|
||||
// All the proc that we inserted should not be deleted
|
||||
for (int j = MIN_PROC; j <= i; ++j) {
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.NO, tracker.isDeleted(j));
|
||||
}
|
||||
// All the proc that are not yet inserted should be result as deleted
|
||||
for (int j = i + 1; j < MAX_PROC; ++j) {
|
||||
assertTrue(tracker.isDeleted(j) != ProcedureStoreTracker.DeleteState.NO);
|
||||
}
|
||||
}
|
||||
|
||||
// sequential delete
|
||||
for (int i = MIN_PROC; i < MAX_PROC; ++i) {
|
||||
tracker.delete(i);
|
||||
|
||||
// All the proc that we deleted should be deleted
|
||||
for (int j = MIN_PROC; j <= i; ++j) {
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.YES, tracker.isDeleted(j));
|
||||
}
|
||||
// All the proc that are not yet deleted should be result as not deleted
|
||||
for (int j = i + 1; j < MAX_PROC; ++j) {
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.NO, tracker.isDeleted(j));
|
||||
}
|
||||
}
|
||||
assertTrue(tracker.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPartialTracker() {
|
||||
ProcedureStoreTracker tracker = new ProcedureStoreTracker();
|
||||
tracker.setPartialFlag(true);
|
||||
|
||||
// nothing in the tracker, the state is unknown
|
||||
assertTrue(tracker.isEmpty());
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(1));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(579));
|
||||
|
||||
// Mark 1 as deleted, now that is a known state
|
||||
tracker.setDeleted(1, true);
|
||||
tracker.dump();
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.YES, tracker.isDeleted(1));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(2));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(579));
|
||||
|
||||
// Mark 579 as non-deleted, now that is a known state
|
||||
tracker.setDeleted(579, false);
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.YES, tracker.isDeleted(1));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(2));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.NO, tracker.isDeleted(579));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(577));
|
||||
assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(580));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBasicCRUD() {
|
||||
ProcedureStoreTracker tracker = new ProcedureStoreTracker();
|
||||
assertTrue(tracker.isEmpty());
|
||||
|
||||
Procedure[] procs = new TestProcedure[] {
|
||||
new TestProcedure(1), new TestProcedure(2), new TestProcedure(3),
|
||||
new TestProcedure(4), new TestProcedure(5), new TestProcedure(6),
|
||||
};
|
||||
|
||||
tracker.insert(procs[0], null);
|
||||
tracker.insert(procs[1], new Procedure[] { procs[2], procs[3], procs[4] });
|
||||
assertFalse(tracker.isEmpty());
|
||||
assertTrue(tracker.isUpdated());
|
||||
|
||||
tracker.resetUpdates();
|
||||
assertFalse(tracker.isUpdated());
|
||||
|
||||
for (int i = 0; i < 4; ++i) {
|
||||
tracker.update(procs[i]);
|
||||
assertFalse(tracker.isEmpty());
|
||||
assertFalse(tracker.isUpdated());
|
||||
}
|
||||
|
||||
tracker.update(procs[4]);
|
||||
assertFalse(tracker.isEmpty());
|
||||
assertTrue(tracker.isUpdated());
|
||||
|
||||
tracker.update(procs[5]);
|
||||
assertFalse(tracker.isEmpty());
|
||||
assertTrue(tracker.isUpdated());
|
||||
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
tracker.delete(procs[i].getProcId());
|
||||
assertFalse(tracker.isEmpty());
|
||||
assertTrue(tracker.isUpdated());
|
||||
}
|
||||
tracker.delete(procs[5].getProcId());
|
||||
assertTrue(tracker.isEmpty());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,267 @@
|
|||
/**
|
||||
* 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.store.wal;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Iterator;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.procedure2.SequentialProcedure;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
||||
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 TestWALProcedureStore {
|
||||
private static final Log LOG = LogFactory.getLog(TestWALProcedureStore.class);
|
||||
|
||||
private static final int PROCEDURE_STORE_SLOTS = 1;
|
||||
private static final Procedure NULL_PROC = null;
|
||||
|
||||
private WALProcedureStore procStore;
|
||||
|
||||
private HBaseCommonTestingUtility htu;
|
||||
private FileSystem fs;
|
||||
private Path testDir;
|
||||
private Path logDir;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
htu = new HBaseCommonTestingUtility();
|
||||
testDir = htu.getDataTestDir();
|
||||
fs = testDir.getFileSystem(htu.getConfiguration());
|
||||
assertTrue(testDir.depth() > 1);
|
||||
|
||||
logDir = new Path(testDir, "proc-logs");
|
||||
procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
|
||||
procStore.start(PROCEDURE_STORE_SLOTS);
|
||||
procStore.recoverLease();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
procStore.stop(false);
|
||||
fs.delete(logDir, true);
|
||||
}
|
||||
|
||||
private Iterator<Procedure> storeRestart() throws Exception {
|
||||
procStore.stop(false);
|
||||
procStore.start(PROCEDURE_STORE_SLOTS);
|
||||
procStore.recoverLease();
|
||||
return procStore.load();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyLogLoad() throws Exception {
|
||||
Iterator<Procedure> loader = storeRestart();
|
||||
assertEquals(0, countProcedures(loader));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLoad() throws Exception {
|
||||
Set<Long> procIds = new HashSet<>();
|
||||
|
||||
// Insert something in the log
|
||||
Procedure proc1 = new TestSequentialProcedure();
|
||||
procIds.add(proc1.getProcId());
|
||||
procStore.insert(proc1, null);
|
||||
|
||||
Procedure proc2 = new TestSequentialProcedure();
|
||||
Procedure[] child2 = new Procedure[2];
|
||||
child2[0] = new TestSequentialProcedure();
|
||||
child2[1] = new TestSequentialProcedure();
|
||||
|
||||
procIds.add(proc2.getProcId());
|
||||
procIds.add(child2[0].getProcId());
|
||||
procIds.add(child2[1].getProcId());
|
||||
procStore.insert(proc2, child2);
|
||||
|
||||
// Verify that everything is there
|
||||
verifyProcIdsOnRestart(procIds);
|
||||
|
||||
// Update and delete something
|
||||
procStore.update(proc1);
|
||||
procStore.update(child2[1]);
|
||||
procStore.delete(child2[1].getProcId());
|
||||
procIds.remove(child2[1].getProcId());
|
||||
|
||||
// Verify that everything is there
|
||||
verifyProcIdsOnRestart(procIds);
|
||||
|
||||
// Remove 4 byte from the trailers
|
||||
procStore.stop(false);
|
||||
FileStatus[] logs = fs.listStatus(logDir);
|
||||
assertEquals(3, logs.length);
|
||||
for (int i = 0; i < logs.length; ++i) {
|
||||
corruptLog(logs[i], 4);
|
||||
}
|
||||
verifyProcIdsOnRestart(procIds);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCorruptedTrailer() throws Exception {
|
||||
// Insert something
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
procStore.insert(new TestSequentialProcedure(), null);
|
||||
}
|
||||
|
||||
// Stop the store
|
||||
procStore.stop(false);
|
||||
|
||||
// Remove 4 byte from the trailer
|
||||
FileStatus[] logs = fs.listStatus(logDir);
|
||||
assertEquals(1, logs.length);
|
||||
corruptLog(logs[0], 4);
|
||||
|
||||
int count = countProcedures(storeRestart());
|
||||
assertEquals(100, count);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCorruptedEntries() throws Exception {
|
||||
// Insert something
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
procStore.insert(new TestSequentialProcedure(), null);
|
||||
}
|
||||
|
||||
// Stop the store
|
||||
procStore.stop(false);
|
||||
|
||||
// Remove some byte from the log
|
||||
// (enough to cut the trailer and corrupt some entries)
|
||||
FileStatus[] logs = fs.listStatus(logDir);
|
||||
assertEquals(1, logs.length);
|
||||
corruptLog(logs[0], 1823);
|
||||
|
||||
int count = countProcedures(storeRestart());
|
||||
assertTrue(procStore.getCorruptedLogs() != null);
|
||||
assertEquals(1, procStore.getCorruptedLogs().size());
|
||||
assertEquals(85, count);
|
||||
}
|
||||
|
||||
private void corruptLog(final FileStatus logFile, final long dropBytes)
|
||||
throws IOException {
|
||||
assertTrue(logFile.getLen() > dropBytes);
|
||||
LOG.debug("corrupt log " + logFile.getPath() +
|
||||
" size=" + logFile.getLen() + " drop=" + dropBytes);
|
||||
Path tmpPath = new Path(testDir, "corrupted.log");
|
||||
InputStream in = fs.open(logFile.getPath());
|
||||
OutputStream out = fs.create(tmpPath);
|
||||
IOUtils.copyBytes(in, out, logFile.getLen() - dropBytes, true);
|
||||
fs.rename(tmpPath, logFile.getPath());
|
||||
}
|
||||
|
||||
private void verifyProcIdsOnRestart(final Set<Long> procIds) throws Exception {
|
||||
int count = 0;
|
||||
Iterator<Procedure> loader = storeRestart();
|
||||
while (loader.hasNext()) {
|
||||
Procedure proc = loader.next();
|
||||
LOG.debug("loading procId=" + proc.getProcId());
|
||||
assertTrue("procId=" + proc.getProcId() + " unexpected", procIds.contains(proc.getProcId()));
|
||||
count++;
|
||||
}
|
||||
assertEquals(procIds.size(), count);
|
||||
}
|
||||
|
||||
private void assertIsEmpty(Iterator<Procedure> iterator) {
|
||||
assertEquals(0, countProcedures(iterator));
|
||||
}
|
||||
|
||||
private int countProcedures(Iterator<Procedure> iterator) {
|
||||
int count = 0;
|
||||
while (iterator.hasNext()) {
|
||||
Procedure proc = iterator.next();
|
||||
LOG.trace("loading procId=" + proc.getProcId());
|
||||
count++;
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
private void assertEmptyLogDir() {
|
||||
try {
|
||||
FileStatus[] status = fs.listStatus(logDir);
|
||||
assertTrue("expected empty state-log dir", status == null || status.length == 0);
|
||||
} catch (FileNotFoundException e) {
|
||||
fail("expected the state-log dir to be present: " + logDir);
|
||||
} catch (IOException e) {
|
||||
fail("got en exception on state-log dir list: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestSequentialProcedure extends SequentialProcedure<Void> {
|
||||
private static long seqid = 0;
|
||||
|
||||
public TestSequentialProcedure() {
|
||||
setProcId(++seqid);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(Void env) { return null; }
|
||||
|
||||
@Override
|
||||
protected void rollback(Void env) { }
|
||||
|
||||
@Override
|
||||
protected boolean abort(Void env) { return false; }
|
||||
|
||||
@Override
|
||||
protected void serializeStateData(final OutputStream stream) throws IOException {
|
||||
long procId = getProcId();
|
||||
if (procId % 2 == 0) {
|
||||
stream.write(Bytes.toBytes(procId));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void deserializeStateData(InputStream stream) throws IOException {
|
||||
long procId = getProcId();
|
||||
if (procId % 2 == 0) {
|
||||
byte[] bProcId = new byte[8];
|
||||
assertEquals(8, stream.read(bProcId));
|
||||
assertEquals(procId, Bytes.toLong(bProcId));
|
||||
} else {
|
||||
assertEquals(0, stream.available());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,137 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
||||
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 TestTimeoutBlockingQueue {
|
||||
private static final Log LOG = LogFactory.getLog(TestTimeoutBlockingQueue.class);
|
||||
|
||||
static class TestObject {
|
||||
private long timeout;
|
||||
private int seqId;
|
||||
|
||||
public TestObject(int seqId, long timeout) {
|
||||
this.timeout = timeout;
|
||||
this.seqId = seqId;
|
||||
}
|
||||
|
||||
public long getTimeout() {
|
||||
return timeout;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("(%03d, %03d)", seqId, timeout);
|
||||
}
|
||||
}
|
||||
|
||||
static class TestObjectTimeoutRetriever implements TimeoutRetriever<TestObject> {
|
||||
@Override
|
||||
public long getTimeout(TestObject obj) {
|
||||
return obj.getTimeout();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeUnit getTimeUnit(TestObject obj) {
|
||||
return TimeUnit.MILLISECONDS;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOrder() {
|
||||
TimeoutBlockingQueue<TestObject> queue =
|
||||
new TimeoutBlockingQueue<TestObject>(8, new TestObjectTimeoutRetriever());
|
||||
|
||||
long[] timeouts = new long[] {500, 200, 700, 300, 600, 600, 200, 800, 500};
|
||||
|
||||
for (int i = 0; i < timeouts.length; ++i) {
|
||||
for (int j = 0; j <= i; ++j) {
|
||||
queue.add(new TestObject(j, timeouts[j]));
|
||||
queue.dump();
|
||||
}
|
||||
|
||||
long prev = 0;
|
||||
for (int j = 0; j <= i; ++j) {
|
||||
TestObject obj = queue.poll();
|
||||
assertTrue(obj.getTimeout() >= prev);
|
||||
prev = obj.getTimeout();
|
||||
queue.dump();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeoutBlockingQueue() {
|
||||
TimeoutBlockingQueue<TestObject> queue;
|
||||
|
||||
int[][] testArray = new int[][] {
|
||||
{200, 400, 600}, // append
|
||||
{200, 400, 100}, // prepend
|
||||
{200, 400, 300}, // insert
|
||||
};
|
||||
|
||||
for (int i = 0; i < testArray.length; ++i) {
|
||||
int[] sortedArray = Arrays.copyOf(testArray[i], testArray[i].length);
|
||||
Arrays.sort(sortedArray);
|
||||
|
||||
// test with head == 0
|
||||
queue = new TimeoutBlockingQueue<TestObject>(2, new TestObjectTimeoutRetriever());
|
||||
for (int j = 0; j < testArray[i].length; ++j) {
|
||||
queue.add(new TestObject(j, testArray[i][j]));
|
||||
queue.dump();
|
||||
}
|
||||
|
||||
for (int j = 0; !queue.isEmpty(); ++j) {
|
||||
assertEquals(sortedArray[j], queue.poll().getTimeout());
|
||||
}
|
||||
|
||||
queue = new TimeoutBlockingQueue<TestObject>(2, new TestObjectTimeoutRetriever());
|
||||
queue.add(new TestObject(0, 50));
|
||||
assertEquals(50, queue.poll().getTimeout());
|
||||
|
||||
// test with head > 0
|
||||
for (int j = 0; j < testArray[i].length; ++j) {
|
||||
queue.add(new TestObject(j, testArray[i][j]));
|
||||
queue.dump();
|
||||
}
|
||||
|
||||
for (int j = 0; !queue.isEmpty(); ++j) {
|
||||
assertEquals(sortedArray[j], queue.poll().getTimeout());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -176,6 +176,7 @@
|
|||
<include>MapReduce.proto</include>
|
||||
<include>Master.proto</include>
|
||||
<include>MultiRowMutation.proto</include>
|
||||
<include>Procedure.proto</include>
|
||||
<include>Quota.proto</include>
|
||||
<include>RegionServerStatus.proto</include>
|
||||
<include>RowProcessor.proto</include>
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,114 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "ProcedureProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
import "ErrorHandling.proto";
|
||||
|
||||
enum ProcedureState {
|
||||
INITIALIZING = 1; // Procedure in construction, not yet added to the executor
|
||||
RUNNABLE = 2; // Procedure added to the executor, and ready to be executed
|
||||
WAITING = 3; // The procedure is waiting on children to be completed
|
||||
WAITING_TIMEOUT = 4; // The procedure is waiting a timout or an external event
|
||||
ROLLEDBACK = 5; // The procedure failed and was rolledback
|
||||
FINISHED = 6; // The procedure execution is completed. may need a rollback if failed.
|
||||
}
|
||||
|
||||
/**
|
||||
* Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
|
||||
*/
|
||||
message Procedure {
|
||||
// internal "static" state
|
||||
required string class_name = 1; // full classname to be able to instantiate the procedure
|
||||
optional uint64 parent_id = 2; // parent if not a root-procedure otherwise not set
|
||||
required uint64 proc_id = 3;
|
||||
required uint64 start_time = 4;
|
||||
optional string owner = 5;
|
||||
|
||||
// internal "runtime" state
|
||||
required ProcedureState state = 6;
|
||||
repeated uint32 stack_id = 7; // stack indices in case the procedure was running
|
||||
required uint64 last_update = 8;
|
||||
optional uint32 timeout = 9;
|
||||
|
||||
// user state/results
|
||||
optional ForeignExceptionMessage exception = 10;
|
||||
optional bytes result = 11; // opaque (user) result structure
|
||||
optional bytes state_data = 12; // opaque (user) procedure internal-state
|
||||
}
|
||||
|
||||
/**
|
||||
* SequentialProcedure data
|
||||
*/
|
||||
message SequentialProcedureData {
|
||||
required bool executed = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* StateMachineProcedure data
|
||||
*/
|
||||
message StateMachineProcedureData {
|
||||
repeated uint32 state = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Procedure WAL header
|
||||
*/
|
||||
message ProcedureWALHeader {
|
||||
required uint32 version = 1;
|
||||
required uint32 type = 2;
|
||||
required uint64 log_id = 3;
|
||||
required uint64 min_proc_id = 4;
|
||||
}
|
||||
|
||||
/**
|
||||
* Procedure WAL trailer
|
||||
*/
|
||||
message ProcedureWALTrailer {
|
||||
required uint32 version = 1;
|
||||
required uint64 tracker_pos = 2;
|
||||
}
|
||||
|
||||
message ProcedureStoreTracker {
|
||||
message TrackerNode {
|
||||
required uint64 start_id = 1;
|
||||
repeated uint64 updated = 2;
|
||||
repeated uint64 deleted = 3;
|
||||
}
|
||||
|
||||
repeated TrackerNode node = 1;
|
||||
}
|
||||
|
||||
message ProcedureWALEntry {
|
||||
enum Type {
|
||||
EOF = 1;
|
||||
INIT = 2;
|
||||
INSERT = 3;
|
||||
UPDATE = 4;
|
||||
DELETE = 5;
|
||||
COMPACT = 6;
|
||||
}
|
||||
|
||||
required Type type = 1;
|
||||
repeated Procedure procedure = 2;
|
||||
optional uint64 proc_id = 3;
|
||||
}
|
21
pom.xml
21
pom.xml
|
@ -56,6 +56,7 @@
|
|||
<module>hbase-client</module>
|
||||
<module>hbase-hadoop-compat</module>
|
||||
<module>hbase-common</module>
|
||||
<module>hbase-procedure</module>
|
||||
<module>hbase-it</module>
|
||||
<module>hbase-examples</module>
|
||||
<module>hbase-prefix-tree</module>
|
||||
|
@ -871,7 +872,7 @@
|
|||
</fileMapper>
|
||||
</fileMappers>
|
||||
<outputDir>${basedir}/target/asciidoc</outputDir>
|
||||
</transformationSet>
|
||||
</transformationSet>
|
||||
</transformationSets>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
@ -1014,7 +1015,7 @@
|
|||
<plugin>
|
||||
<groupId>org.asciidoctor</groupId>
|
||||
<artifactId>asciidoctor-maven-plugin</artifactId>
|
||||
<version>1.5.2</version>
|
||||
<version>1.5.2</version>
|
||||
<inherited>false</inherited>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
|
@ -1034,10 +1035,10 @@
|
|||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>output-html</id>
|
||||
<id>output-html</id>
|
||||
<phase>site</phase>
|
||||
<goals>
|
||||
<goal>process-asciidoc</goal>
|
||||
<goal>process-asciidoc</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<attributes>
|
||||
|
@ -1191,6 +1192,7 @@
|
|||
Modules are pretty heavy-weight things, so doing this work isn't too bad. -->
|
||||
<server.test.jar>hbase-server-${project.version}-tests.jar</server.test.jar>
|
||||
<common.test.jar>hbase-common-${project.version}-tests.jar</common.test.jar>
|
||||
<procedure.test.jar>hbase-procedure-${project.version}-tests.jar</procedure.test.jar>
|
||||
<it.test.jar>hbase-it-${project.version}-tests.jar</it.test.jar>
|
||||
<annotations.test.jar>hbase-annotations-${project.version}-tests.jar</annotations.test.jar>
|
||||
<surefire.version>2.18</surefire.version>
|
||||
|
@ -1260,6 +1262,17 @@
|
|||
<artifactId>hbase-protocol</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-procedure</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-procedure</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-hadoop-compat</artifactId>
|
||||
|
|
Loading…
Reference in New Issue