Part of HBASE-1586 Bring back transactions and indexing for 0.20.. plus hbase-1588 Rig our build to support our new contribs -- stargate and THBase/ITHBase

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@789051 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-06-28 05:47:35 +00:00
parent 03e786b0a8
commit 61970dc11e
51 changed files with 6131 additions and 294 deletions

View File

@ -23,6 +23,7 @@
# The java implementation to use. Java 1.6 required.
# export JAVA_HOME=/usr/java/jdk1.6.0/
export JAVA_HOME=/usr/lib/jvm/java-6-sun
# Extra Java CLASSPATH elements. Optional.
# export HBASE_CLASSPATH=

View File

@ -0,0 +1,237 @@
<?xml version="1.0"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!-- Imported by contrib/*/build.xml files to share generic targets. -->
<project name="hadoopbuildcontrib">
<property name="name" value="${ant.project.name}"/>
<property name="root" value="${basedir}"/>
<!-- Load all the default properties, and any the user wants -->
<!-- to contribute (without having to type -D or edit this file -->
<property file="${user.home}/${name}.build.properties" />
<property file="${root}/build.properties" />
<property name="hadoop.root" location="${root}/../../../"/>
<property name="src.dir" location="${root}/src/java"/>
<property name="src.test" location="${root}/src/test"/>
<property name="src.examples" location="${root}/src/examples"/>
<available file="${src.examples}" type="dir" property="examples.available"/>
<available file="${src.test}" type="dir" property="test.available"/>
<property name="conf.dir" location="${hadoop.root}/conf"/>
<property name="test.junit.output.format" value="plain"/>
<property name="test.output" value="no"/>
<property name="test.timeout" value="900000"/>
<property name="build.dir" location="${hadoop.root}/build/contrib/${name}"/>
<property name="build.classes" location="${build.dir}/classes"/>
<property name="build.test" location="${build.dir}/test"/>
<property name="build.examples" location="${build.dir}/examples"/>
<property name="hadoop.log.dir" location="${build.dir}/test/logs"/>
<!-- all jars together -->
<property name="javac.deprecation" value="off"/>
<property name="javac.debug" value="on"/>
<property name="javadoc.link"
value="http://java.sun.com/j2se/1.4/docs/api/"/>
<property name="build.encoding" value="ISO-8859-1"/>
<fileset id="lib.jars" dir="${root}" includes="lib/*.jar"/>
<!-- the normal classpath -->
<path id="classpath">
<pathelement location="${build.classes}"/>
<fileset refid="lib.jars"/>
<pathelement location="${hadoop.root}/build/classes"/>
<fileset dir="${hadoop.root}/lib">
<include name="**/*.jar" />
</fileset>
</path>
<!-- the unit test classpath -->
<path id="test.classpath">
<pathelement location="${build.test}" />
<pathelement location="${hadoop.root}/build/test/classes"/>
<pathelement location="${hadoop.root}/src/contrib/test"/>
<pathelement location="${conf.dir}"/>
<pathelement location="${hadoop.root}/build"/>
<pathelement location="${build.examples}"/>
<path refid="classpath"/>
</path>
<!-- to be overridden by sub-projects -->
<target name="check-contrib"/>
<target name="init-contrib"/>
<!-- ====================================================== -->
<!-- Stuff needed by all targets -->
<!-- ====================================================== -->
<target name="init" depends="check-contrib" unless="skip.contrib">
<echo message="contrib: ${name}"/>
<mkdir dir="${build.dir}"/>
<mkdir dir="${build.classes}"/>
<mkdir dir="${build.test}"/>
<mkdir dir="${build.examples}"/>
<mkdir dir="${hadoop.log.dir}"/>
<antcall target="init-contrib"/>
</target>
<!-- ====================================================== -->
<!-- Compile a Hadoop contrib's files -->
<!-- ====================================================== -->
<target name="compile" depends="init" unless="skip.contrib">
<echo message="contrib: ${name}"/>
<javac
encoding="${build.encoding}"
srcdir="${src.dir}"
includes="**/*.java"
destdir="${build.classes}"
debug="${javac.debug}"
deprecation="${javac.deprecation}">
<classpath refid="classpath"/>
</javac>
</target>
<!-- ======================================================= -->
<!-- Compile a Hadoop contrib's example files (if available) -->
<!-- ======================================================= -->
<target name="compile-examples" depends="compile" if="examples.available">
<echo message="contrib: ${name}"/>
<javac
encoding="${build.encoding}"
srcdir="${src.examples}"
includes="**/*.java"
destdir="${build.examples}"
debug="${javac.debug}">
<classpath refid="classpath"/>
</javac>
</target>
<!-- ================================================================== -->
<!-- Compile test code -->
<!-- ================================================================== -->
<target name="compile-test" depends="compile-examples" if="test.available">
<echo message="contrib: ${name}"/>
<javac
encoding="${build.encoding}"
srcdir="${src.test}"
includes="**/*.java"
destdir="${build.test}"
debug="${javac.debug}">
<classpath refid="test.classpath"/>
</javac>
</target>
<!-- ====================================================== -->
<!-- Make a Hadoop contrib's jar -->
<!-- ====================================================== -->
<target name="jar" depends="compile" unless="skip.contrib">
<echo message="contrib: ${name}"/>
<jar
jarfile="${build.dir}/hadoop-${version}-${name}.jar"
basedir="${build.classes}"
/>
</target>
<!-- ====================================================== -->
<!-- Make a Hadoop contrib's examples jar -->
<!-- ====================================================== -->
<target name="jar-examples" depends="compile-examples"
if="examples.available" unless="skip.contrib">
<echo message="contrib: ${name}"/>
<jar jarfile="${build.dir}/hadoop-${version}-${name}-examples.jar">
<fileset dir="${build.classes}">
</fileset>
<fileset dir="${build.examples}">
</fileset>
</jar>
</target>
<!-- ====================================================== -->
<!-- Package a Hadoop contrib -->
<!-- ====================================================== -->
<target name="package" depends="jar, jar-examples" unless="skip.contrib">
<mkdir dir="${dist.dir}/contrib/${name}"/>
<copy todir="${dist.dir}/contrib/${name}" includeEmptyDirs="false" flatten="true">
<fileset dir="${build.dir}">
<include name="hadoop-${version}-${name}.jar" />
</fileset>
</copy>
</target>
<!-- ================================================================== -->
<!-- Run unit tests -->
<!-- ================================================================== -->
<target name="test" depends="compile-test, compile" if="test.available">
<echo message="contrib: ${name}"/>
<delete dir="${hadoop.log.dir}"/>
<mkdir dir="${hadoop.log.dir}"/>
<junit
printsummary="yes" showoutput="${test.output}"
haltonfailure="no" fork="yes" maxmemory="256m"
errorProperty="tests.failed" failureProperty="tests.failed"
timeout="${test.timeout}">
<sysproperty key="test.build.data" value="${build.test}/data"/>
<sysproperty key="build.test" value="${build.test}"/>
<sysproperty key="contrib.name" value="${name}"/>
<!-- requires fork=yes for:
relative File paths to use the specified user.dir
classpath to use build/contrib/*.jar
-->
<sysproperty key="user.dir" value="${build.test}/data"/>
<sysproperty key="fs.default.name" value="${fs.default.name}"/>
<sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
<sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/>
<classpath refid="test.classpath"/>
<formatter type="${test.junit.output.format}" />
<batchtest todir="${build.test}" unless="testcase">
<fileset dir="${src.test}"
includes="**/Test*.java" excludes="**/${test.exclude}.java" />
</batchtest>
<batchtest todir="${build.test}" if="testcase">
<fileset dir="${src.test}" includes="**/${testcase}.java"/>
</batchtest>
</junit>
<fail if="tests.failed">Tests failed!</fail>
</target>
<!-- ================================================================== -->
<!-- Clean. Delete the build files, and their directories -->
<!-- ================================================================== -->
<target name="clean">
<echo message="contrib: ${name}"/>
<delete dir="${build.dir}"/>
</target>
</project>

65
src/contrib/build.xml Normal file
View File

@ -0,0 +1,65 @@
<?xml version="1.0"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<project name="hadoopcontrib" default="compile" basedir=".">
<!-- In case one of the contrib subdirectories -->
<!-- fails the build or test targets and you cannot fix it: -->
<!-- Then add to fileset: excludes="badcontrib/build.xml" -->
<!-- ====================================================== -->
<!-- Compile contribs. -->
<!-- ====================================================== -->
<target name="compile">
<subant target="compile">
<fileset dir="." includes="*/build.xml"/>
</subant>
</target>
<!-- ====================================================== -->
<!-- Package contrib jars. -->
<!-- ====================================================== -->
<target name="package">
<subant target="package">
<fileset dir="." includes="*/build.xml"/>
</subant>
</target>
<!-- ====================================================== -->
<!-- Test all the contribs. -->
<!-- ====================================================== -->
<target name="test">
<subant target="test">
<fileset dir="." includes="streaming/build.xml"/>
<fileset dir="." includes="fairscheduler/build.xml"/>
<fileset dir="." includes="capacity-scheduler/build.xml"/>
</subant>
</target>
<!-- ====================================================== -->
<!-- Clean all the contribs. -->
<!-- ====================================================== -->
<target name="clean">
<subant target="clean">
<fileset dir="." includes="*/build.xml"/>
</subant>
</target>
</project>

View File

@ -0,0 +1,38 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.util.Map;
import org.apache.hadoop.io.Writable;
/**
* Interface for generating an index-row-key from a row in the base table.
*/
public interface IndexKeyGenerator extends Writable {
/** Create an index key from a base row.
*
* @param rowKey the row key of the base row
* @param columns the columns in the base row
* @return the row key in the indexed row.
*/
byte[] createIndexKey(byte[] rowKey, Map<byte[], byte[]> columns);
}

View File

@ -0,0 +1,47 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
/**
* Thrown when asking for an index that does not exist.
*/
public class IndexNotFoundException extends IOException {
private static final long serialVersionUID = 6533971528557000965L;
public IndexNotFoundException() {
super();
}
public IndexNotFoundException(String arg0) {
super(arg0);
}
public IndexNotFoundException(Throwable arg0) {
super(arg0.getMessage());
}
public IndexNotFoundException(String arg0, Throwable arg1) {
super(arg0+arg1.getMessage());
}
}

View File

@ -0,0 +1,199 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.ObjectWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
/** Holds the specification for a single secondary index. */
public class IndexSpecification implements Writable {
// Columns that are indexed (part of the indexRowKey)
private byte[][] indexedColumns;
// Constructs the
private IndexKeyGenerator keyGenerator;
// Additional columns mapped into the indexed row. These will be available for
// filters when scanning the index.
private byte[][] additionalColumns;
private byte[][] allColumns;
// Id of this index, unique within a table.
private String indexId;
/** Construct an "simple" index spec for a single column.
* @param indexId
* @param indexedColumn
*/
public IndexSpecification(String indexId, byte[] indexedColumn) {
this(indexId, new byte[][] { indexedColumn }, null,
new SimpleIndexKeyGenerator(indexedColumn));
}
/**
* Construct an index spec by specifying everything.
*
* @param indexId
* @param indexedColumns
* @param additionalColumns
* @param keyGenerator
* @param keyComparator
*/
public IndexSpecification(String indexId, byte[][] indexedColumns,
byte[][] additionalColumns, IndexKeyGenerator keyGenerator) {
this.indexId = indexId;
this.indexedColumns = indexedColumns;
this.additionalColumns = additionalColumns;
this.keyGenerator = keyGenerator;
this.makeAllColumns();
}
public IndexSpecification() {
// For writable
}
private void makeAllColumns() {
this.allColumns = new byte[indexedColumns.length
+ (additionalColumns == null ? 0 : additionalColumns.length)][];
System.arraycopy(indexedColumns, 0, allColumns, 0, indexedColumns.length);
if (additionalColumns != null) {
System.arraycopy(additionalColumns, 0, allColumns, indexedColumns.length,
additionalColumns.length);
}
}
/**
* Get the indexedColumns.
*
* @return Return the indexedColumns.
*/
public byte[][] getIndexedColumns() {
return indexedColumns;
}
/**
* Get the keyGenerator.
*
* @return Return the keyGenerator.
*/
public IndexKeyGenerator getKeyGenerator() {
return keyGenerator;
}
/**
* Get the additionalColumns.
*
* @return Return the additionalColumns.
*/
public byte[][] getAdditionalColumns() {
return additionalColumns;
}
/**
* Get the indexId.
*
* @return Return the indexId.
*/
public String getIndexId() {
return indexId;
}
public byte[][] getAllColumns() {
return allColumns;
}
public boolean containsColumn(byte[] column) {
for (byte[] col : allColumns) {
if (Bytes.equals(column, col)) {
return true;
}
}
return false;
}
public byte[] getIndexedTableName(byte[] baseTableName) {
return Bytes.add(baseTableName, Bytes.toBytes("-" + indexId));
}
private static final HBaseConfiguration CONF = new HBaseConfiguration();
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
indexId = in.readUTF();
int numIndexedCols = in.readInt();
indexedColumns = new byte[numIndexedCols][];
for (int i = 0; i < numIndexedCols; i++) {
indexedColumns[i] = Bytes.readByteArray(in);
}
int numAdditionalCols = in.readInt();
additionalColumns = new byte[numAdditionalCols][];
for (int i = 0; i < numAdditionalCols; i++) {
additionalColumns[i] = Bytes.readByteArray(in);
}
makeAllColumns();
keyGenerator = (IndexKeyGenerator) ObjectWritable.readObject(in, CONF);
// FIXME this is to read the deprecated comparator, in existing data
ObjectWritable.readObject(in, CONF);
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeUTF(indexId);
out.writeInt(indexedColumns.length);
for (byte[] col : indexedColumns) {
Bytes.writeByteArray(out, col);
}
if (additionalColumns != null) {
out.writeInt(additionalColumns.length);
for (byte[] col : additionalColumns) {
Bytes.writeByteArray(out, col);
}
} else {
out.writeInt(0);
}
ObjectWritable
.writeObject(out, keyGenerator, IndexKeyGenerator.class, CONF);
// FIXME need to maintain this for exisitng data
ObjectWritable.writeObject(out, null, WritableComparable.class,
CONF);
}
/** {@inheritDoc} */
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("ID => ");
sb.append(indexId);
return sb.toString();
}
}

View File

@ -0,0 +1,45 @@
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
/** Holds an array of index specifications.
*
*/
public class IndexSpecificationArray implements Writable {
private IndexSpecification [] indexSpecifications;
public IndexSpecificationArray() {
// FOr writable
}
public IndexSpecificationArray(IndexSpecification[] specs) {
this.indexSpecifications = specs;
}
public void readFields(DataInput in) throws IOException {
int size = in.readInt();
indexSpecifications = new IndexSpecification[size];
for (int i=0; i<size; i++) {
indexSpecifications[i] = new IndexSpecification();
indexSpecifications[i].readFields(in);
}
}
public void write(DataOutput out) throws IOException {
out.writeInt(indexSpecifications.length);
for (IndexSpecification indexSpec : indexSpecifications) {
indexSpec.write(out);
}
}
/** Get indexSpecifications.
* @return indexSpecifications
*/
public IndexSpecification[] getIndexSpecifications() {
return indexSpecifications;
}
}

View File

@ -0,0 +1,255 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.transactional.TransactionalTable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.util.Bytes;
/** HTable extended with indexed support. */
public class IndexedTable extends TransactionalTable {
// FIXME, these belong elsewhere
public static final byte[] INDEX_COL_FAMILY_NAME = Bytes.toBytes("__INDEX__");
public static final byte[] INDEX_COL_FAMILY = Bytes.add(
INDEX_COL_FAMILY_NAME, new byte[] { HStoreKey.COLUMN_FAMILY_DELIMITER });
public static final byte[] INDEX_BASE_ROW = Bytes.toBytes("ROW");
public static final byte[] INDEX_BASE_ROW_COLUMN = Bytes.add(
INDEX_COL_FAMILY, INDEX_BASE_ROW);
static final Log LOG = LogFactory.getLog(IndexedTable.class);
private final IndexedTableDescriptor indexedTableDescriptor;
private Map<String, HTable> indexIdToTable = new HashMap<String, HTable>();
public IndexedTable(final HBaseConfiguration conf, final byte[] tableName)
throws IOException {
super(conf, tableName);
this.indexedTableDescriptor = new IndexedTableDescriptor(super.getTableDescriptor());
for (IndexSpecification spec : this.indexedTableDescriptor.getIndexes()) {
indexIdToTable.put(spec.getIndexId(), new HTable(conf, spec
.getIndexedTableName(tableName)));
}
}
public IndexedTableDescriptor getIndexedTableDescriptor() {
return this.indexedTableDescriptor;
}
/**
* Open up an indexed scanner. Results will come back in the indexed order,
* but will contain RowResults from the original table.
*
* @param indexId the id of the index to use
* @param indexStartRow (created from the IndexKeyGenerator)
* @param indexColumns in the index table
* @param indexFilter filter to run on the index'ed table. This can only use
* columns that have been added to the index.
* @param baseColumns from the original table
* @return scanner
* @throws IOException
* @throws IndexNotFoundException
*/
public ResultScanner getIndexedScanner(String indexId, final byte[] indexStartRow,
byte[][] indexColumns, final Filter indexFilter,
final byte[][] baseColumns) throws IOException, IndexNotFoundException {
IndexSpecification indexSpec = this.indexedTableDescriptor.getIndex(indexId);
if (indexSpec == null) {
throw new IndexNotFoundException("Index " + indexId
+ " not defined in table "
+ super.getTableDescriptor().getNameAsString());
}
verifyIndexColumns(indexColumns, indexSpec);
// TODO, verify/remove index columns from baseColumns
HTable indexTable = indexIdToTable.get(indexId);
byte[][] allIndexColumns;
if (indexColumns != null) {
allIndexColumns = new byte[indexColumns.length + 1][];
System
.arraycopy(indexColumns, 0, allIndexColumns, 0, indexColumns.length);
allIndexColumns[indexColumns.length] = INDEX_BASE_ROW_COLUMN;
} else {
byte[][] allColumns = indexSpec.getAllColumns();
allIndexColumns = new byte[allColumns.length + 1][];
System.arraycopy(allColumns, 0, allIndexColumns, 0, allColumns.length);
allIndexColumns[allColumns.length] = INDEX_BASE_ROW_COLUMN;
}
Scan indexScan = new Scan(indexStartRow);
//indexScan.setFilter(filter); // FIXME
indexScan.addColumns(allIndexColumns);
ResultScanner indexScanner = indexTable.getScanner(indexScan);
return new ScannerWrapper(indexScanner, baseColumns);
}
private void verifyIndexColumns(byte[][] requestedColumns,
IndexSpecification indexSpec) {
if (requestedColumns == null) {
return;
}
for (byte[] requestedColumn : requestedColumns) {
boolean found = false;
for (byte[] indexColumn : indexSpec.getAllColumns()) {
if (Bytes.equals(requestedColumn, indexColumn)) {
found = true;
break;
}
}
if (!found) {
throw new RuntimeException("Column [" + Bytes.toString(requestedColumn)
+ "] not in index " + indexSpec.getIndexId());
}
}
}
private class ScannerWrapper implements ResultScanner {
private ResultScanner indexScanner;
private byte[][] columns;
public ScannerWrapper(ResultScanner indexScanner, byte[][] columns) {
this.indexScanner = indexScanner;
this.columns = columns;
}
/** {@inheritDoc} */
public Result next() throws IOException {
Result[] result = next(1);
if (result == null || result.length < 1)
return null;
return result[0];
}
/** {@inheritDoc} */
public Result[] next(int nbRows) throws IOException {
Result[] indexResult = indexScanner.next(nbRows);
if (indexResult == null) {
return null;
}
Result[] result = new Result[indexResult.length];
for (int i = 0; i < indexResult.length; i++) {
Result row = indexResult[i];
byte[] baseRow = row.getValue(INDEX_BASE_ROW_COLUMN);
LOG.debug("next index row [" + Bytes.toString(row.getRow())
+ "] -> base row [" + Bytes.toString(baseRow) + "]");
HbaseMapWritable<byte[], Cell> colValues =
new HbaseMapWritable<byte[], Cell>();
Result baseResult = null;
if (columns != null && columns.length > 0) {
LOG.debug("Going to base table for remaining columns");
Get baseGet = new Get(baseRow);
baseGet.addColumns(columns);
baseResult = IndexedTable.this.get(baseGet);
}
List<KeyValue> results = new ArrayList<KeyValue>();
for (KeyValue indexKV : row.list()) {
byte[] col = indexKV.getColumn();
if (HStoreKey.matchingFamily(INDEX_COL_FAMILY_NAME, col)) {
continue;
}
results.add(new KeyValue(baseRow, indexKV.getColumn(), indexKV.getTimestamp(), KeyValue.Type.Put, indexKV.getValue()));
}
if (baseResult != null) {
results.addAll(baseResult.list());
}
result[i] = new Result(results);
}
return result;
}
/** {@inheritDoc} */
public void close() {
indexScanner.close();
}
// Copied from HTable.ClientScanner.iterator()
public Iterator<Result> iterator() {
return new Iterator<Result>() {
// The next RowResult, possibly pre-read
Result next = null;
// return true if there is another item pending, false if there isn't.
// this method is where the actual advancing takes place, but you need
// to call next() to consume it. hasNext() will only advance if there
// isn't a pending next().
public boolean hasNext() {
if (next == null) {
try {
next = ScannerWrapper.this.next();
return next != null;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
return true;
}
// get the pending next item and advance the iterator. returns null if
// there is no next item.
public Result next() {
// since hasNext() does the real advancing, we call this to determine
// if there is a next before proceeding.
if (!hasNext()) {
return null;
}
// if we get to here, then hasNext() has given us an item to return.
// we want to return the item and then null out the next pointer, so
// we use a temporary variable.
Result temp = next;
next = null;
return temp;
}
public void remove() {
throw new UnsupportedOperationException();
}
};
}
}
}

View File

@ -0,0 +1,149 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ColumnNameParseException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.regionserver.tableindexed.IndexMaintenanceUtils;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Extension of HBaseAdmin that creates indexed tables.
*
*/
public class IndexedTableAdmin extends HBaseAdmin {
private static final Log LOG = LogFactory.getLog(IndexedTableAdmin.class);
/**
* Constructor
*
* @param conf Configuration object
* @throws MasterNotRunningException
*/
public IndexedTableAdmin(HBaseConfiguration conf)
throws MasterNotRunningException {
super(conf);
}
/**
* Creates a new indexed table
*
* @param desc table descriptor for table
*
* @throws IOException
*/
public void createIndexedTable(IndexedTableDescriptor desc) throws IOException {
super.createTable(desc.getBaseTableDescriptor());
this.createIndexTables(desc);
}
private void createIndexTables(IndexedTableDescriptor indexDesc) throws IOException {
byte[] baseTableName = indexDesc.getBaseTableDescriptor().getName();
for (IndexSpecification indexSpec : indexDesc.getIndexes()) {
HTableDescriptor indexTableDesc = createIndexTableDesc(baseTableName,
indexSpec);
super.createTable(indexTableDesc);
}
}
private HTableDescriptor createIndexTableDesc(byte[] baseTableName,
IndexSpecification indexSpec) throws ColumnNameParseException {
HTableDescriptor indexTableDesc = new HTableDescriptor(indexSpec
.getIndexedTableName(baseTableName));
Set<byte[]> families = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
families.add(IndexedTable.INDEX_COL_FAMILY);
for (byte[] column : indexSpec.getAllColumns()) {
families.add(Bytes.add(HStoreKey.getFamily(column),
new byte[] { HStoreKey.COLUMN_FAMILY_DELIMITER }));
}
for (byte[] colFamily : families) {
indexTableDesc.addFamily(new HColumnDescriptor(colFamily));
}
return indexTableDesc;
}
/** Remove an index for a table.
* @throws IOException
*
*/
public void removeIndex(byte[] baseTableName, String indexId) throws IOException {
super.disableTable(baseTableName);
HTableDescriptor desc = super.getTableDescriptor(baseTableName);
IndexedTableDescriptor indexDesc = new IndexedTableDescriptor(desc);
IndexSpecification spec = indexDesc.getIndex(indexId);
indexDesc.removeIndex(indexId);
this.disableTable(spec.getIndexedTableName(baseTableName));
this.deleteTable(spec.getIndexedTableName(baseTableName));
super.modifyTable(baseTableName, desc);
super.enableTable(baseTableName);
}
/** Add an index to a table. */
public void addIndex(byte []baseTableName, IndexSpecification indexSpec) throws IOException {
LOG.warn("Adding index to existing table ["+Bytes.toString(baseTableName)+"], this may take a long time");
// TODO, make table read-only
LOG.warn("Not putting table in readonly, if its being written to, the index may get out of sync");
HTableDescriptor indexTableDesc = createIndexTableDesc(baseTableName, indexSpec);
super.createTable(indexTableDesc);
super.disableTable(baseTableName);
IndexedTableDescriptor indexDesc = new IndexedTableDescriptor(super.getTableDescriptor(baseTableName));
indexDesc.addIndex(indexSpec);
super.modifyTable(baseTableName, indexDesc.getBaseTableDescriptor());
super.enableTable(baseTableName);
reIndexTable(baseTableName, indexSpec);
}
private void reIndexTable(byte[] baseTableName, IndexSpecification indexSpec) throws IOException {
HTable baseTable = new HTable(baseTableName);
HTable indexTable = new HTable(indexSpec.getIndexedTableName(baseTableName));
for (RowResult rowResult : baseTable.getScanner(indexSpec.getAllColumns())) {
SortedMap<byte[], byte[]> columnValues = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
for (Entry<byte[], Cell> entry : rowResult.entrySet()) {
columnValues.put(entry.getKey(), entry.getValue().getValue());
}
if (IndexMaintenanceUtils.doesApplyToIndex(indexSpec, columnValues)) {
Put indexUpdate = IndexMaintenanceUtils.createIndexUpdate(indexSpec, rowResult.getRow(), columnValues);
indexTable.put(indexUpdate);
}
}
}
}

View File

@ -0,0 +1,115 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.ObjectWritable;
import org.apache.hadoop.io.Writable;
public class IndexedTableDescriptor {
private static final byte[] INDEXES_KEY = Bytes.toBytes("INDEXES");
private final HTableDescriptor baseTableDescriptor;
// Key is indexId
private final Map<String, IndexSpecification> indexes = new HashMap<String, IndexSpecification>();
public IndexedTableDescriptor(HTableDescriptor baseTableDescriptor)
throws IOException {
this.baseTableDescriptor = baseTableDescriptor;
readFromTable();
}
public HTableDescriptor getBaseTableDescriptor() {
return this.baseTableDescriptor;
}
private void readFromTable() throws IOException {
byte [] bytes = baseTableDescriptor.getValue(INDEXES_KEY);
if (bytes == null) {
return;
}
ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
DataInputStream dis = new DataInputStream(bais);
IndexSpecificationArray indexArray = new IndexSpecificationArray();
indexArray.readFields(dis);
for (Writable index : indexArray.getIndexSpecifications()) {
IndexSpecification indexSpec = (IndexSpecification) index;
indexes.put(indexSpec.getIndexId(), indexSpec);
}
}
private void writeToTable() {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos);
IndexSpecificationArray indexArray = new IndexSpecificationArray(indexes.values().toArray(new IndexSpecification[0]));
try {
indexArray.write(dos);
dos.flush();
} catch (IOException e) {
throw new RuntimeException(e);
}
baseTableDescriptor.setValue(INDEXES_KEY, baos.toByteArray());
}
public Collection<IndexSpecification> getIndexes() {
return indexes.values();
}
public IndexSpecification getIndex(String indexId) {
return indexes.get(indexId);
}
public void addIndex(IndexSpecification index) {
indexes.put(index.getIndexId(), index);
writeToTable();
}
public void removeIndex(String indexId) {
indexes.remove(indexId);
writeToTable();
}
@Override
public String toString() {
StringBuilder s = new StringBuilder(baseTableDescriptor.toString());
if (!indexes.isEmpty()) {
s.append(", ");
s.append("INDEXES");
s.append(" => ");
s.append(indexes.values());
}
return s.toString();
}
}

View File

@ -0,0 +1,59 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Map;
import org.apache.hadoop.hbase.util.Bytes;
/** Creates indexed keys for a single column....
*
*/
public class SimpleIndexKeyGenerator implements IndexKeyGenerator {
private byte [] column;
public SimpleIndexKeyGenerator(byte [] column) {
this.column = column;
}
public SimpleIndexKeyGenerator() {
// For Writable
}
/** {@inheritDoc} */
public byte[] createIndexKey(byte[] rowKey, Map<byte[], byte[]> columns) {
return Bytes.add(columns.get(column), rowKey);
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
column = Bytes.readByteArray(in);
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, column);
}
}

View File

@ -0,0 +1,46 @@
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
<html>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<head />
<body bgcolor="white">
This package provides support for secondary indexing by maintaining a separate, "index", table for each index.
The IndexSpecification class provides the metadata for the index. This includes:
<li> the columns that contribute to the index key,
<li> additional columns to put in the index table (and are thus made available to filters on the index table),
<br> and
<li> an IndexKeyGenerator which constructs the index-row-key from the indexed column(s) and the original row.
IndexesSpecifications can be added to a table's metadata (HTableDescriptor) before the table is constructed.
Afterwards, updates and deletes to the original table will trigger the updates in the index, and
the indexes can be scanned using the API on IndexedTable.
For a simple example, look at the unit test in org.apache.hadoop.hbase.client.tableIndexed.
<p> To enable the indexing, modify hbase-site.xml to turn on the
IndexedRegionServer. This is done by setting
<i>hbase.regionserver.class</i> to
<i>org.apache.hadoop.hbase.ipc.IndexedRegionInterface</i> and
<i>hbase.regionserver.impl </i> to
<i>org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegionServer</i>
</body>
</html>

View File

@ -0,0 +1,56 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
/** Thrown when a transaction cannot be committed.
*
*/
public class CommitUnsuccessfulException extends Exception {
private static final long serialVersionUID = 7062921444531109202L;
/** Default Constructor */
public CommitUnsuccessfulException() {
super();
}
/**
* @param arg0 message
* @param arg1 cause
*/
public CommitUnsuccessfulException(String arg0, Throwable arg1) {
super(arg0, arg1);
}
/**
* @param arg0 message
*/
public CommitUnsuccessfulException(String arg0) {
super(arg0);
}
/**
* @param arg0 cause
*/
public CommitUnsuccessfulException(Throwable arg0) {
super(arg0);
}
}

View File

@ -0,0 +1,120 @@
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import java.util.Random;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
public class HBaseBackedTransactionLogger implements TransactionLogger {
/** The name of the transaction status table. */
public static final String TABLE_NAME = "__GLOBAL_TRX_LOG__";
private static final String INFO_FAMILY = "Info:";
/**
* Column which holds the transaction status.
*
*/
private static final String STATUS_COLUMN = INFO_FAMILY + "Status";
private static final byte[] STATUS_COLUMN_BYTES = Bytes
.toBytes(STATUS_COLUMN);
/**
* Create the table.
*
* @throws IOException
*
*/
public static void createTable() throws IOException {
HTableDescriptor tableDesc = new HTableDescriptor(TABLE_NAME);
tableDesc.addFamily(new HColumnDescriptor(INFO_FAMILY));
HBaseAdmin admin = new HBaseAdmin(new HBaseConfiguration());
admin.createTable(tableDesc);
}
private Random random = new Random();
private HTable table;
public HBaseBackedTransactionLogger() throws IOException {
initTable();
}
private void initTable() throws IOException {
HBaseAdmin admin = new HBaseAdmin(new HBaseConfiguration());
if (!admin.tableExists(TABLE_NAME)) {
throw new RuntimeException("Table not created. Call createTable() first");
}
this.table = new HTable(TABLE_NAME);
}
public long createNewTransactionLog() {
long id;
TransactionStatus existing;
do {
id = random.nextLong();
existing = getStatusForTransaction(id);
} while (existing != null);
setStatusForTransaction(id, TransactionStatus.PENDING);
return id;
}
public TransactionStatus getStatusForTransaction(long transactionId) {
try {
RowResult result = table.getRow(getRow(transactionId));
if (result == null || result.isEmpty()) {
return null;
}
Cell statusCell = result.get(STATUS_COLUMN_BYTES);
if (statusCell == null) {
throw new RuntimeException("No status cell for row " + transactionId);
}
String statusString = Bytes.toString(statusCell.getValue());
return TransactionStatus.valueOf(statusString);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
private byte [] getRow(long transactionId) {
return Bytes.toBytes(""+transactionId);
}
public void setStatusForTransaction(long transactionId,
TransactionStatus status) {
BatchUpdate update = new BatchUpdate(getRow(transactionId));
update.put(STATUS_COLUMN, Bytes.toBytes(status.name()));
try {
table.commit(update);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
public void forgetTransaction(long transactionId) {
BatchUpdate update = new BatchUpdate(getRow(transactionId));
update.delete(STATUS_COLUMN);
try {
table.commit(update);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}

View File

@ -0,0 +1,136 @@
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import javax.transaction.xa.XAException;
import javax.transaction.xa.XAResource;
import javax.transaction.xa.Xid;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
/**
* View hbase as a JTA transactional resource. This allows it to participate in
* transactions across multiple resources.
*
*
*/
public class JtaXAResource implements XAResource {
static final Log LOG = LogFactory.getLog(JtaXAResource.class);
private Map<Xid, TransactionState> xidToTransactionState = new HashMap<Xid, TransactionState>();
private final TransactionManager transactionManager;
private ThreadLocal<TransactionState> threadLocalTransactionState = new ThreadLocal<TransactionState>();
public JtaXAResource(TransactionManager transactionManager) {
this.transactionManager = transactionManager;
}
public void commit(Xid xid, boolean onePhase) throws XAException {
LOG.trace("commit [" + xid.toString() + "] "
+ (onePhase ? "one phase" : "two phase"));
TransactionState state = xidToTransactionState.remove(xid);
if (state == null) {
throw new XAException(XAException.XAER_NOTA);
}
try {
if (onePhase) {
transactionManager.tryCommit(state);
} else {
transactionManager.doCommit(state);
}
} catch (CommitUnsuccessfulException e) {
throw new XAException(XAException.XA_RBROLLBACK);
} catch (IOException e) {
throw new XAException(XAException.XA_RBPROTO); // FIXME correct code?
} finally {
threadLocalTransactionState.remove();
}
}
public void end(Xid xid, int flags) throws XAException {
LOG.trace("end [" + xid.toString() + "] ");
threadLocalTransactionState.remove();
}
public void forget(Xid xid) throws XAException {
LOG.trace("forget [" + xid.toString() + "] ");
threadLocalTransactionState.remove();
TransactionState state = xidToTransactionState.remove(xid);
if (state != null) {
try {
transactionManager.abort(state);
} catch (IOException e) {
throw new RuntimeException(e); // FIXME, should be an XAException?
}
}
}
public int getTransactionTimeout() throws XAException {
return 0;
}
public boolean isSameRM(XAResource xares) throws XAException {
if (xares instanceof JtaXAResource) {
return true;
}
return false;
}
public int prepare(Xid xid) throws XAException {
LOG.trace("prepare [" + xid.toString() + "] ");
TransactionState state = xidToTransactionState.get(xid);
int status;
try {
status = this.transactionManager.prepareCommit(state);
} catch (CommitUnsuccessfulException e) {
throw new XAException(XAException.XA_HEURRB); // FIXME correct code?
} catch (IOException e) {
throw new XAException(XAException.XA_RBPROTO); // FIXME correct code?
}
switch (status) {
case TransactionalRegionInterface.COMMIT_OK:
return XAResource.XA_OK;
case TransactionalRegionInterface.COMMIT_OK_READ_ONLY:
return XAResource.XA_RDONLY;
default:
throw new XAException(XAException.XA_RBPROTO); // FIXME correct code?
}
}
public Xid[] recover(int flag) throws XAException {
return xidToTransactionState.keySet().toArray(new Xid[] {});
}
public void rollback(Xid xid) throws XAException {
LOG.trace("rollback [" + xid.toString() + "] ");
forget(xid);
threadLocalTransactionState.remove();
}
public boolean setTransactionTimeout(int seconds) throws XAException {
return false; // Currently not supported. (Only global lease time)
}
public void start(Xid xid, int flags) throws XAException {
LOG.trace("start [" + xid.toString() + "] ");
// TODO, check flags
TransactionState state = this.transactionManager.beginTransaction();
threadLocalTransactionState.set(state);
xidToTransactionState.put(xid, state);
}
/**
* @return the threadLocalTransaction state.
*/
public TransactionState getThreadLocalTransactionState() {
return threadLocalTransactionState.get();
}
}

View File

@ -0,0 +1,79 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
/**
* A local, in-memory implementation of the transaction logger. Does not provide a global view, so
* it can't be relighed on by
*
*/
public class LocalTransactionLogger implements TransactionLogger {
private static LocalTransactionLogger instance;
/**
* Creates singleton if it does not exist
*
* @return reference to singleton
*/
public synchronized static LocalTransactionLogger getInstance() {
if (instance == null) {
instance = new LocalTransactionLogger();
}
return instance;
}
private Random random = new Random();
private Map<Long, TransactionStatus> transactionIdToStatusMap = Collections
.synchronizedMap(new HashMap<Long, TransactionStatus>());
private LocalTransactionLogger() {
// Enforce singlton
}
/** @return random longs to minimize possibility of collision */
public long createNewTransactionLog() {
long id;
do {
id = random.nextLong();
} while (transactionIdToStatusMap.containsKey(id));
transactionIdToStatusMap.put(id, TransactionStatus.PENDING);
return id;
}
public TransactionStatus getStatusForTransaction(final long transactionId) {
return transactionIdToStatusMap.get(transactionId);
}
public void setStatusForTransaction(final long transactionId,
final TransactionStatus status) {
transactionIdToStatusMap.put(transactionId, status);
}
public void forgetTransaction(long transactionId) {
transactionIdToStatusMap.remove(transactionId);
}
}

View File

@ -0,0 +1,59 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
/**
* Simple interface used to provide a log about transaction status. Written to
* by the client, and read by regionservers in case of failure.
*
*/
public interface TransactionLogger {
/** Transaction status values */
enum TransactionStatus {
/** Transaction is pending */
PENDING,
/** Transaction was committed */
COMMITTED,
/** Transaction was aborted */
ABORTED
}
/**
* Create a new transaction log. Return the transaction's globally unique id.
* Log's initial value should be PENDING
*
* @return transaction id
*/
long createNewTransactionLog();
/**
* @param transactionId
* @return transaction status
*/
TransactionStatus getStatusForTransaction(long transactionId);
/**
* @param transactionId
* @param status
*/
void setStatusForTransaction(long transactionId, TransactionStatus status);
}

View File

@ -0,0 +1,239 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
/**
* Transaction Manager. Responsible for committing transactions.
*
*/
public class TransactionManager {
static final Log LOG = LogFactory.getLog(TransactionManager.class);
private final HConnection connection;
private final TransactionLogger transactionLogger;
private JtaXAResource xAResource;
/**
* @param conf
*/
public TransactionManager(final HBaseConfiguration conf) {
this(LocalTransactionLogger.getInstance(), conf);
}
/**
* @param transactionLogger
* @param conf
*/
public TransactionManager(final TransactionLogger transactionLogger,
final HBaseConfiguration conf) {
this.transactionLogger = transactionLogger;
connection = HConnectionManager.getConnection(conf);
}
/**
* Called to start a transaction.
*
* @return new transaction state
*/
public TransactionState beginTransaction() {
long transactionId = transactionLogger.createNewTransactionLog();
LOG.debug("Begining transaction " + transactionId);
return new TransactionState(transactionId);
}
/**
* Prepare to commit a transaction.
*
* @param transactionState
* @return commitStatusCode (see {@link TransactionalRegionInterface})
* @throws IOException
* @throws CommitUnsuccessfulException
*/
public int prepareCommit(final TransactionState transactionState)
throws CommitUnsuccessfulException, IOException {
boolean allReadOnly = true;
try {
Iterator<HRegionLocation> locationIterator = transactionState.getParticipatingRegions().iterator();
while (locationIterator.hasNext()) {
HRegionLocation location = locationIterator.next();
TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
.getHRegionConnection(location.getServerAddress());
int commitStatus = transactionalRegionServer.commitRequest(location
.getRegionInfo().getRegionName(), transactionState
.getTransactionId());
boolean canCommit = true;
switch (commitStatus) {
case TransactionalRegionInterface.COMMIT_OK:
allReadOnly = false;
break;
case TransactionalRegionInterface.COMMIT_OK_READ_ONLY:
locationIterator.remove(); // No need to doCommit for read-onlys
break;
case TransactionalRegionInterface.COMMIT_UNSUCESSFUL:
canCommit = false;
break;
default:
throw new CommitUnsuccessfulException(
"Unexpected return code from prepareCommit: " + commitStatus);
}
if (LOG.isTraceEnabled()) {
LOG.trace("Region ["
+ location.getRegionInfo().getRegionNameAsString() + "] votes "
+ (canCommit ? "to commit" : "to abort") + " transaction "
+ transactionState.getTransactionId());
}
if (!canCommit) {
LOG.debug("Aborting [" + transactionState.getTransactionId() + "]");
abort(transactionState, location);
throw new CommitUnsuccessfulException();
}
}
} catch (Exception e) {
LOG.debug("Commit of transaction [" + transactionState.getTransactionId()
+ "] was unsucsessful", e);
// This happens on a NSRE that is triggered by a split
// FIXME, but then abort fails
try {
abort(transactionState);
} catch (Exception abortException) {
LOG.warn("Exeption durring abort", abortException);
}
throw new CommitUnsuccessfulException(e);
}
return allReadOnly ? TransactionalRegionInterface.COMMIT_OK_READ_ONLY : TransactionalRegionInterface.COMMIT_OK;
}
/**
* Try and commit a transaction. This does both phases of the 2-phase protocol: prepare and commit.
*
* @param transactionState
* @throws IOException
* @throws CommitUnsuccessfulException
*/
public void tryCommit(final TransactionState transactionState)
throws CommitUnsuccessfulException, IOException {
long startTime = System.currentTimeMillis();
LOG.debug("atempting to commit trasaction: " + transactionState.toString());
int status = prepareCommit(transactionState);
if (status == TransactionalRegionInterface.COMMIT_OK) {
doCommit(transactionState);
}
LOG.debug("Committed transaction ["+transactionState.getTransactionId()+"] in ["+((System.currentTimeMillis()-startTime))+"]ms");
}
/** Do the commit. This is the 2nd phase of the 2-phase protocol.
*
* @param transactionState
* @throws CommitUnsuccessfulException
*/
public void doCommit(final TransactionState transactionState)
throws CommitUnsuccessfulException{
try {
LOG.debug("Commiting [" + transactionState.getTransactionId() + "]");
transactionLogger.setStatusForTransaction(transactionState
.getTransactionId(), TransactionLogger.TransactionStatus.COMMITTED);
for (HRegionLocation location : transactionState
.getParticipatingRegions()) {
TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
.getHRegionConnection(location.getServerAddress());
transactionalRegionServer.commit(location.getRegionInfo()
.getRegionName(), transactionState.getTransactionId());
}
} catch (Exception e) {
LOG.debug("Commit of transaction [" + transactionState.getTransactionId()
+ "] was unsucsessful", e);
// This happens on a NSRE that is triggered by a split
// FIXME, but then abort fails
try {
abort(transactionState);
} catch (Exception abortException) {
LOG.warn("Exeption durring abort", abortException);
}
throw new CommitUnsuccessfulException(e);
}
// TODO: Transaction log can be deleted now ...
}
/**
* Abort a s transaction.
*
* @param transactionState
* @throws IOException
*/
public void abort(final TransactionState transactionState) throws IOException {
abort(transactionState, null);
}
private void abort(final TransactionState transactionState,
final HRegionLocation locationToIgnore) throws IOException {
transactionLogger.setStatusForTransaction(transactionState
.getTransactionId(), TransactionLogger.TransactionStatus.ABORTED);
for (HRegionLocation location : transactionState.getParticipatingRegions()) {
if (locationToIgnore != null && location.equals(locationToIgnore)) {
continue;
}
try {
TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
.getHRegionConnection(location.getServerAddress());
transactionalRegionServer.abort(location.getRegionInfo()
.getRegionName(), transactionState.getTransactionId());
} catch (UnknownTransactionException e) {
LOG
.debug("Got unknown transaciton exception durring abort. Transaction: ["
+ transactionState.getTransactionId()
+ "], region: ["
+ location.getRegionInfo().getRegionNameAsString()
+ "]. Ignoring.");
} catch (NotServingRegionException e) {
LOG
.debug("Got NSRE durring abort. Transaction: ["
+ transactionState.getTransactionId() + "], region: ["
+ location.getRegionInfo().getRegionNameAsString()
+ "]. Ignoring.");
}
}
}
public synchronized JtaXAResource getXAResource() {
if (xAResource == null){
xAResource = new JtaXAResource(this);
}
return xAResource;
}
}

View File

@ -0,0 +1,50 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
class TransactionScannerCallable extends ScannerCallable {
private TransactionState transactionState;
TransactionScannerCallable(final TransactionState transactionState,
final HConnection connection, final byte[] tableName,
final byte[] startRow, Scan scan) {
super(connection, tableName, startRow, scan);
this.transactionState = transactionState;
}
@Override
protected long openScanner() throws IOException {
if (transactionState.addRegion(location)) {
((TransactionalRegionInterface) server).beginTransaction(transactionState
.getTransactionId(), location.getRegionInfo().getRegionName());
}
return ((TransactionalRegionInterface) server).openScanner(transactionState
.getTransactionId(), this.location.getRegionInfo().getRegionName(),
getScan());
}
}

View File

@ -0,0 +1,78 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionLocation;
/**
* Holds client-side transaction information. Client's use them as opaque
* objects passed around to transaction operations.
*
*/
public class TransactionState {
static final Log LOG = LogFactory.getLog(TransactionState.class);
private final long transactionId;
private Set<HRegionLocation> participatingRegions = new HashSet<HRegionLocation>();
TransactionState(final long transactionId) {
this.transactionId = transactionId;
}
boolean addRegion(final HRegionLocation hregion) {
boolean added = participatingRegions.add(hregion);
if (added) {
LOG.debug("Adding new hregion ["
+ hregion.getRegionInfo().getRegionNameAsString()
+ "] to transaction [" + transactionId + "]");
}
return added;
}
Set<HRegionLocation> getParticipatingRegions() {
return participatingRegions;
}
/**
* Get the transactionId.
*
* @return Return the transactionId.
*/
public long getTransactionId() {
return transactionId;
}
/**
* @see java.lang.Object#toString()
*/
@Override
public String toString() {
return "id: " + transactionId + ", particpants: "
+ participatingRegions.size();
}
}

View File

@ -0,0 +1,190 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Table with transactional support.
*
*/
public class TransactionalTable extends HTable {
/**
* @param conf
* @param tableName
* @throws IOException
*/
public TransactionalTable(final HBaseConfiguration conf,
final String tableName) throws IOException {
this(conf, Bytes.toBytes(tableName));
}
/**
* @param conf
* @param tableName
* @throws IOException
*/
public TransactionalTable(final HBaseConfiguration conf,
final byte[] tableName) throws IOException {
super(conf, tableName);
}
private static abstract class TransactionalServerCallable<T> extends
ServerCallable<T> {
protected TransactionState transactionState;
protected TransactionalRegionInterface getTransactionServer() {
return (TransactionalRegionInterface) server;
}
protected void recordServer() throws IOException {
if (transactionState.addRegion(location)) {
getTransactionServer().beginTransaction(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName());
}
}
/**
* @param connection
* @param tableName
* @param row
* @param transactionState
*/
public TransactionalServerCallable(final HConnection connection,
final byte[] tableName, final byte[] row,
final TransactionState transactionState) {
super(connection, tableName, row);
this.transactionState = transactionState;
}
}
/**
* Method for getting data from a row
* @param get the Get to fetch
* @return the result
* @throws IOException
* @since 0.20.0
*/
public Result get(final TransactionState transactionState, final Get get) throws IOException {
return super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Result>(super.getConnection(), super
.getTableName(), get.getRow(), transactionState) {
public Result call() throws IOException {
recordServer();
return getTransactionServer().get(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), get);
}
});
}
/**
*
* @param delete
* @throws IOException
* @since 0.20.0
*/
public void delete(final TransactionState transactionState, final Delete delete)
throws IOException {
super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Object>(super.getConnection(), super
.getTableName(), delete.getRow(), transactionState) {
public Object call() throws IOException {
recordServer();
getTransactionServer().delete(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), delete);
return null;
}
});
}
/**
* Commit a Put to the table.
* <p>
* If autoFlush is false, the update is buffered.
* @param put
* @throws IOException
* @since 0.20.0
*/
public synchronized void put(TransactionState transactionState, final Put put) throws IOException {
//super.validatePut(put);
super.getConnection().getRegionServerWithRetries(
new TransactionalServerCallable<Object>(super.getConnection(), super
.getTableName(), put.getRow(), transactionState) {
public Object call() throws IOException {
recordServer();
getTransactionServer().put(
transactionState.getTransactionId(),
location.getRegionInfo().getRegionName(), put);
return null;
}
});
}
public ResultScanner getScanner(final TransactionState transactionState,
Scan scan) throws IOException {
ClientScanner scanner = new TransactionalClientScanner(transactionState, scan);
scanner.initialize();
return scanner;
}
protected class TransactionalClientScanner extends HTable.ClientScanner {
private TransactionState transactionState;
protected TransactionalClientScanner(
final TransactionState transactionState, Scan scan) {
super(scan);
this.transactionState = transactionState;
}
@Override
protected ScannerCallable getScannerCallable(
final byte[] localStartKey, int caching) {
TransactionScannerCallable t =
new TransactionScannerCallable(transactionState, getConnection(),
getTableName(), getScan().getStartRow(), getScan());
t.setCaching(caching);
return t;
}
}
}

View File

@ -0,0 +1,43 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Thrown if a region server is passed an unknown transaction id
*/
public class UnknownTransactionException extends DoNotRetryIOException {
private static final long serialVersionUID = 698575374929591099L;
/** constructor */
public UnknownTransactionException() {
super();
}
/**
* Constructor
* @param s message
*/
public UnknownTransactionException(String s) {
super(s);
}
}

View File

@ -0,0 +1,61 @@
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
<html>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<head />
<body bgcolor="white">
This package provides support for atomic transactions. Transactions can
span multiple regions. Transaction writes are applied when committing a
transaction. At commit time, the transaction is examined to see if it
can be applied while still maintaining atomicity. This is done by
looking for conflicts with the transactions that committed while the
current transaction was running. This technique is known as optimistic
concurrency control (OCC) because it relies on the assumption that
transactions will mostly not have conflicts with each other.
<p>
For more details on OCC, see the paper <i> On Optimistic Methods for Concurrency Control </i>
by Kung and Robinson available
<a href=http://www.seas.upenn.edu/~zives/cis650/papers/opt-cc.pdf> here </a>.
<p> To enable transactions, modify hbase-site.xml to turn on the
TransactionalRegionServer. This is done by setting
<i>hbase.regionserver.class</i> to
<i>org.apache.hadoop.hbase.ipc.TransactionalRegionInterface</i> and
<i>hbase.regionserver.impl </i> to
<i>org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer</i>
<p>
The read set claimed by a transactional scanner is determined from the start and
end keys which the scanner is opened with.
<h3> Known Issues </h3>
Recovery in the face of hregion server failure
is not fully implemented. Thus, you cannot rely on the transactional
properties in the face of node failure.
</body>
</html>

View File

@ -0,0 +1,151 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
/**
* Interface for transactional region servers.
*
* <p>
* NOTE: if you change the interface, you must change the RPC version number in
* HBaseRPCProtocolVersion
*
*/
public interface TransactionalRegionInterface extends HRegionInterface {
/** Status code representing a transaction that can be committed. */
int COMMIT_OK = 1;
/** Status code representing a read-only transaction that can be committed. */
int COMMIT_OK_READ_ONLY = 2;
/** Status code representing a transaction that cannot be committed. */
int COMMIT_UNSUCESSFUL = 3;
/**
* Sent to initiate a transaction.
*
* @param transactionId
* @param regionName name of region
* @throws IOException
*/
void beginTransaction(long transactionId, final byte[] regionName)
throws IOException;
/**
* Perform a transactional Get operation.
* @param regionName name of region to get from
* @param get Get operation
* @return Result
* @throws IOException
*/
public Result get(long transactionId, byte [] regionName, Get get) throws IOException;
/**
* Transactional put data into the specified region
* @param regionName
* @param put the data to be put
* @throws IOException
*/
public void put(long transactionId, final byte [] regionName, final Put put)
throws IOException;
/**
* Put an array of puts into the specified region
* @param regionName
* @param puts
* @return
* @throws IOException
*/
public int put(long transactionId, final byte[] regionName, final Put [] puts)
throws IOException;
/**
* Deletes all the KeyValues that match those found in the Delete object,
* if their ts <= to the Delete. In case of a delete with a specific ts it
* only deletes that specific KeyValue.
* @param regionName
* @param delete
* @throws IOException
*/
public void delete(long transactionId, final byte[] regionName, final Delete delete)
throws IOException;
//
// remote scanner interface
//
/**
* Opens a remote transactional scanner with a RowFilter.
*
* @param regionName name of region to scan
* @param scan configured scan object
* @return scannerId scanner identifier used in other calls
* @throws IOException
*/
public long openScanner(long transactionId, final byte [] regionName, final Scan scan)
throws IOException;
/**
* Ask if we can commit the given transaction.
*
* @param regionName
* @param transactionId
* @return status of COMMIT_OK, COMMIT_READ_ONLY, or COMMIT_UNSUSESSFULL
* @throws IOException
*/
int commitRequest(final byte[] regionName, long transactionId)
throws IOException;
/**
* Try to commit the given transaction. This is used when there is only one
* participating region.
*
* @param regionName
* @param transactionId
* @return true if committed
* @throws IOException
*/
boolean commitIfPossible(final byte[] regionName, long transactionId)
throws IOException;
/**
* Commit the transaction.
*
* @param regionName
* @param transactionId
* @throws IOException
*/
void commit(final byte[] regionName, long transactionId) throws IOException;
/**
* Abort the transaction.
*
* @param regionName
* @param transactionId
* @throws IOException
*/
void abort(final byte[] regionName, long transactionId) throws IOException;
}

View File

@ -0,0 +1,82 @@
/**
* Copyright 2008 The Apache Software Foundation Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this work for additional information regarding
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License. You may obtain a copy of the License at
* http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
* either express or implied. See the License for the specific language governing permissions and limitations under the
* License.
*/
package org.apache.hadoop.hbase.regionserver.tableindexed;
import java.util.SortedMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ColumnNameParseException;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
import org.apache.hadoop.hbase.client.tableindexed.IndexedTable;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Singleton class for index maintence logic.
*/
public class IndexMaintenanceUtils {
private static final Log LOG = LogFactory.getLog(IndexMaintenanceUtils.class);
public static Put createIndexUpdate(final IndexSpecification indexSpec, final byte[] row,
final SortedMap<byte[], byte[]> columnValues) {
byte[] indexRow = indexSpec.getKeyGenerator().createIndexKey(row, columnValues);
Put update = new Put(indexRow);
update.add(IndexedTable.INDEX_COL_FAMILY_NAME, IndexedTable.INDEX_BASE_ROW, row);
try {
for (byte[] col : indexSpec.getIndexedColumns()) {
byte[] val = columnValues.get(col);
if (val == null) {
throw new RuntimeException("Unexpected missing column value. [" + Bytes.toString(col) + "]");
}
byte [][] colSeperated = HStoreKey.parseColumn(col);
update.add(colSeperated[0], colSeperated[1], val);
}
for (byte[] col : indexSpec.getAdditionalColumns()) {
byte[] val = columnValues.get(col);
if (val != null) {
byte [][] colSeperated = HStoreKey.parseColumn(col);
update.add(colSeperated[0], colSeperated[1], val);
}
}
} catch (ColumnNameParseException e) {
throw new RuntimeException(e);
}
return update;
}
/**
* Ask if this update does apply to the index.
*
* @param indexSpec
* @param b
* @return true if possibly apply.
*/
public static boolean doesApplyToIndex(final IndexSpecification indexSpec,
final SortedMap<byte[], byte[]> columnValues) {
for (byte[] neededCol : indexSpec.getIndexedColumns()) {
if (!columnValues.containsKey(neededCol)) {
LOG.debug("Index [" + indexSpec.getIndexId() + "] can't be updated because ["
+ Bytes.toString(neededCol) + "] is missing");
return false;
}
}
return true;
}
}

View File

@ -0,0 +1,289 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.tableindexed;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.Map.Entry;
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.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Leases;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
import org.apache.hadoop.hbase.client.tableindexed.IndexedTableDescriptor;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegion;
import org.apache.hadoop.hbase.util.Bytes;
class IndexedRegion extends TransactionalRegion {
private static final Log LOG = LogFactory.getLog(IndexedRegion.class);
private final HBaseConfiguration conf;
private final IndexedTableDescriptor indexTableDescriptor;
private Map<IndexSpecification, HTable> indexSpecToTable = new HashMap<IndexSpecification, HTable>();
public IndexedRegion(final Path basedir, final HLog log, final FileSystem fs,
final HBaseConfiguration conf, final HRegionInfo regionInfo,
final FlushRequester flushListener, Leases trxLeases) throws IOException {
super(basedir, log, fs, conf, regionInfo, flushListener, trxLeases);
this.indexTableDescriptor = new IndexedTableDescriptor(regionInfo.getTableDesc());
this.conf = conf;
}
private synchronized HTable getIndexTable(IndexSpecification index)
throws IOException {
HTable indexTable = indexSpecToTable.get(index);
if (indexTable == null) {
indexTable = new HTable(conf, index.getIndexedTableName(super
.getRegionInfo().getTableDesc().getName()));
indexSpecToTable.put(index, indexTable);
}
return indexTable;
}
private Collection<IndexSpecification> getIndexes() {
return indexTableDescriptor.getIndexes();
}
/**
* @param batchUpdate
* @param lockid
* @param writeToWAL if true, then we write this update to the log
* @throws IOException
*/
@Override
public void put(Put put, Integer lockId, boolean writeToWAL)
throws IOException {
updateIndexes(put); // Do this first because will want to see the old row
super.put(put, lockId, writeToWAL);
}
private void updateIndexes(Put put) throws IOException {
List<IndexSpecification> indexesToUpdate = new LinkedList<IndexSpecification>();
// Find the indexes we need to update
for (IndexSpecification index : getIndexes()) {
if (possiblyAppliesToIndex(index, put)) {
indexesToUpdate.add(index);
}
}
if (indexesToUpdate.size() == 0) {
return;
}
NavigableSet<byte[]> neededColumns = getColumnsForIndexes(indexesToUpdate);
NavigableMap<byte[], byte[]> newColumnValues = getColumnsFromPut(put);
Get oldGet = new Get(put.getRow());
for (byte [] neededCol : neededColumns) {
oldGet.addColumn(neededCol);
}
Result oldResult = super.get(oldGet, null);
// Add the old values to the new if they are not there
if (oldResult != null && oldResult.raw() != null) {
for (KeyValue oldKV : oldResult.raw()) {
if (!newColumnValues.containsKey(oldKV.getColumn())) {
newColumnValues.put(oldKV.getColumn(), oldKV.getValue());
}
}
}
Iterator<IndexSpecification> indexIterator = indexesToUpdate.iterator();
while (indexIterator.hasNext()) {
IndexSpecification indexSpec = indexIterator.next();
if (!IndexMaintenanceUtils.doesApplyToIndex(indexSpec, newColumnValues)) {
indexIterator.remove();
}
}
SortedMap<byte[], byte[]> oldColumnValues = convertToValueMap(oldResult);
for (IndexSpecification indexSpec : indexesToUpdate) {
removeOldIndexEntry(indexSpec, put.getRow(), oldColumnValues);
updateIndex(indexSpec, put.getRow(), newColumnValues);
}
}
private void updateIndexes(Delete delete) {
// FIXME
// Handle delete batch updates. Go back and get the next older values
// for (BatchOperation op : batchUpdate) {
// if (!op.isPut()) {
// Cell current = oldColumnCells.get(op.getColumn());
// if (current != null) {
// // TODO: Fix this profligacy!!! St.Ack
// Cell [] older = Cell.createSingleCellArray(super.get(batchUpdate.getRow(),
// op.getColumn(), current.getTimestamp(), 1));
// if (older != null && older.length > 0) {
// newColumnValues.put(op.getColumn(), older[0].getValue());
// }
// }
// }
// }
}
/** Return the columns needed for the update. */
private NavigableSet<byte[]> getColumnsForIndexes(Collection<IndexSpecification> indexes) {
NavigableSet<byte[]> neededColumns = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
for (IndexSpecification indexSpec : indexes) {
for (byte[] col : indexSpec.getAllColumns()) {
neededColumns.add(col);
}
}
return neededColumns;
}
private void removeOldIndexEntry(IndexSpecification indexSpec, byte[] row,
SortedMap<byte[], byte[]> oldColumnValues) throws IOException {
for (byte[] indexedCol : indexSpec.getIndexedColumns()) {
if (!oldColumnValues.containsKey(indexedCol)) {
LOG.debug("Index [" + indexSpec.getIndexId()
+ "] not trying to remove old entry for row ["
+ Bytes.toString(row) + "] because col ["
+ Bytes.toString(indexedCol) + "] is missing");
return;
}
}
byte[] oldIndexRow = indexSpec.getKeyGenerator().createIndexKey(row,
oldColumnValues);
LOG.debug("Index [" + indexSpec.getIndexId() + "] removing old entry ["
+ Bytes.toString(oldIndexRow) + "]");
getIndexTable(indexSpec).delete(new Delete(oldIndexRow));
}
private NavigableMap<byte[], byte[]> getColumnsFromPut(Put put) {
NavigableMap<byte[], byte[]> columnValues = new TreeMap<byte[], byte[]>(
Bytes.BYTES_COMPARATOR);
for (List<KeyValue> familyPuts : put.getFamilyMap().values()) {
for (KeyValue kv : familyPuts) {
columnValues.put(kv.getColumn(), kv.getValue());
}
}
return columnValues;
}
/** Ask if this put *could* apply to the index. It may actually apply if some of the columns needed are missing.
*
* @param indexSpec
* @param put
* @return true if possibly apply.
*/
private boolean possiblyAppliesToIndex(IndexSpecification indexSpec, Put put) {
for (List<KeyValue> familyPuts : put.getFamilyMap().values()) {
for (KeyValue kv : familyPuts) {
if (indexSpec.containsColumn(kv.getColumn())) {
return true;
}
}
}
return false;
}
// FIXME: This call takes place in an RPC, and requires an RPC. This makes for
// a likely deadlock if the number of RPCs we are trying to serve is >= the
// number of handler threads.
private void updateIndex(IndexSpecification indexSpec, byte[] row,
SortedMap<byte[], byte[]> columnValues) throws IOException {
Put indexUpdate = IndexMaintenanceUtils.createIndexUpdate(indexSpec, row, columnValues);
getIndexTable(indexSpec).put(indexUpdate);
LOG.debug("Index [" + indexSpec.getIndexId() + "] adding new entry ["
+ Bytes.toString(indexUpdate.getRow()) + "] for row ["
+ Bytes.toString(row) + "]");
}
@Override
public void delete(Delete delete, final Integer lockid, boolean writeToWAL)
throws IOException {
if (!getIndexes().isEmpty()) {
// Need all columns
NavigableSet<byte[]> neededColumns = getColumnsForIndexes(getIndexes());
Get get = new Get(delete.getRow());
for (byte [] col : neededColumns) {
get.addColumn(col);
}
Result oldRow = super.get(get, null);
SortedMap<byte[], byte[]> oldColumnValues = convertToValueMap(oldRow);
for (IndexSpecification indexSpec : getIndexes()) {
removeOldIndexEntry(indexSpec, delete.getRow(), oldColumnValues);
}
// Handle if there is still a version visible.
if (delete.getTimeStamp() != HConstants.LATEST_TIMESTAMP) {
get.setTimeRange(1, delete.getTimeStamp());
oldRow = super.get(get, null);
SortedMap<byte[], byte[]> currentColumnValues = convertToValueMap(oldRow);
for (IndexSpecification indexSpec : getIndexes()) {
if (IndexMaintenanceUtils.doesApplyToIndex(indexSpec, currentColumnValues)) {
updateIndex(indexSpec, delete.getRow(), currentColumnValues);
}
}
}
}
super.delete(delete, lockid, writeToWAL);
}
private SortedMap<byte[], byte[]> convertToValueMap(Result result) {
SortedMap<byte[], byte[]> currentColumnValues = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
if (result == null || result.raw() == null) {
return currentColumnValues;
}
for(KeyValue kv : result.list()) {
currentColumnValues.put(kv.getColumn(), kv.getValue());
}
return currentColumnValues;
}
}

View File

@ -0,0 +1,67 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.tableindexed;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
import org.apache.hadoop.hbase.ipc.IndexedRegionInterface;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer;
import org.apache.hadoop.util.Progressable;
/**
* RegionServer which maintains secondary indexes.
*
**/
public class IndexedRegionServer extends TransactionalRegionServer implements
IndexedRegionInterface {
public IndexedRegionServer(HBaseConfiguration conf) throws IOException {
super(conf);
}
@Override
public long getProtocolVersion(final String protocol, final long clientVersion)
throws IOException {
if (protocol.equals(IndexedRegionInterface.class.getName())) {
return HBaseRPCProtocolVersion.versionID;
}
return super.getProtocolVersion(protocol, clientVersion);
}
@Override
protected HRegion instantiateRegion(final HRegionInfo regionInfo)
throws IOException {
HRegion r = new IndexedRegion(HTableDescriptor.getTableDir(super
.getRootDir(), regionInfo.getTableDesc().getName()), super.hlog, super
.getFileSystem(), super.conf, regionInfo, super.getFlushRequester(), super.getTransactionalLeases());
r.initialize(null, new Progressable() {
public void progress() {
addProcessingMessage(regionInfo);
}
});
return r;
}
}

View File

@ -0,0 +1,57 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.regionserver.HRegion;
/**
* Cleans up committed transactions when they are no longer needed to verify
* pending transactions.
*/
class CleanOldTransactionsChore extends Chore {
private static final String SLEEP_CONF = "hbase.transaction.clean.sleep";
private static final int DEFAULT_SLEEP = 60 * 1000;
private final TransactionalRegionServer regionServer;
/**
* @param regionServer
* @param stopRequest
*/
public CleanOldTransactionsChore(
final TransactionalRegionServer regionServer,
final AtomicBoolean stopRequest) {
super(regionServer.getConfiguration().getInt(SLEEP_CONF, DEFAULT_SLEEP),
stopRequest);
this.regionServer = regionServer;
}
@Override
protected void chore() {
for (HRegion region : regionServer.getOnlineRegions()) {
((TransactionalRegion) region).removeUnNeededCommitedTransactions();
}
}
}

View File

@ -0,0 +1,117 @@
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HLogKey;
import org.apache.hadoop.hbase.regionserver.LogRollListener;
import org.apache.hadoop.io.SequenceFile;
/**
* Add support for transactional operations to the regionserver's
* write-ahead-log.
*
*/
class THLog extends HLog {
public THLog(FileSystem fs, Path dir, Configuration conf,
LogRollListener listener) throws IOException {
super(fs, dir, conf, listener);
}
@Override
protected SequenceFile.Writer createWriter(Path path) throws IOException {
return super.createWriter(path, THLogKey.class, KeyValue.class);
}
@Override
protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqNum,
long now) {
return new THLogKey(regionName, tableName, seqNum, now);
}
public void writeStartToLog(HRegionInfo regionInfo, final long transactionId) throws IOException {
this.append(regionInfo, System.currentTimeMillis(),
THLogKey.TrxOp.START, transactionId);
}
public void writeUpdateToLog(HRegionInfo regionInfo, final long transactionId, final Put update)
throws IOException {
this.append(regionInfo, update, transactionId);
}
public void writeDeleteToLog(HRegionInfo regionInfo, final long transactionId, final Delete delete)
throws IOException {
// FIXME
}
public void writeCommitToLog(HRegionInfo regionInfo, final long transactionId) throws IOException {
this.append(regionInfo, System.currentTimeMillis(),
THLogKey.TrxOp.COMMIT, transactionId);
}
public void writeAbortToLog(HRegionInfo regionInfo, final long transactionId) throws IOException {
this.append(regionInfo, System.currentTimeMillis(),
THLogKey.TrxOp.ABORT, transactionId);
}
/**
* Write a general transaction op to the log. This covers: start, commit, and
* abort.
*
* @param regionInfo
* @param now
* @param txOp
* @param transactionId
* @throws IOException
*/
public void append(HRegionInfo regionInfo, long now, THLogKey.TrxOp txOp,
long transactionId) throws IOException {
THLogKey key = new THLogKey(regionInfo.getRegionName(), regionInfo
.getTableDesc().getName(), -1, now, txOp, transactionId);
super.append(regionInfo, key, new KeyValue(new byte [0], 0, 0)); // Empty KeyValue
}
/**
* Write a transactional update to the log.
*
* @param regionInfo
* @param now
* @param update
* @param transactionId
* @throws IOException
*/
public void append(HRegionInfo regionInfo, Put update, long transactionId)
throws IOException {
long commitTime = System.currentTimeMillis(); // FIXME ?
THLogKey key = new THLogKey(regionInfo.getRegionName(), regionInfo
.getTableDesc().getName(), -1, commitTime, THLogKey.TrxOp.OP,
transactionId);
for (KeyValue value : convertToKeyValues(update)) {
super.append(regionInfo, key, value);
}
}
private List<KeyValue> convertToKeyValues(Put update) {
List<KeyValue> edits = new ArrayList<KeyValue>();
for (List<KeyValue> kvs : update.getFamilyMap().values()) {
for (KeyValue kv : kvs) {
edits.add(kv);
}
}
return edits;
}
}

View File

@ -0,0 +1,79 @@
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.regionserver.HLogKey;
public class THLogKey extends HLogKey {
/** Type of Transactional op going into the HLot
*
*/
public enum TrxOp {
/** Start a transaction. */
START((byte)1),
/** A standard operation that is transactional. KV holds the op. */
OP((byte)2),
/** A transaction was committed. */
COMMIT((byte)3),
/** A transaction was aborted. */
ABORT((byte)4);
private final byte opCode;
private TrxOp(byte opCode) {
this.opCode = opCode;
}
public static TrxOp fromByte(byte opCode) {
for (TrxOp op : TrxOp.values()) {
if (op.opCode == opCode) {
return op;
}
}
return null;
}
}
private byte transactionOp = -1;
private long transactionId = -1;
public THLogKey() {
// For Writable
}
public THLogKey(byte[] regionName, byte[] tablename, long logSeqNum, long now) {
super(regionName, tablename, logSeqNum, now);
}
public THLogKey(byte[] regionName, byte[] tablename, long logSeqNum, long now, TrxOp op, long transactionId) {
super(regionName, tablename, logSeqNum, now);
this.transactionOp = op.opCode;
this.transactionId = transactionId;
}
public TrxOp getTrxOp() {
return TrxOp.fromByte(this.transactionOp);
}
public long getTransactionId() {
return this.transactionId;
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeByte(transactionOp);
out.writeLong(transactionId);
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.transactionOp = in.readByte();
this.transactionId = in.readLong();
}
}

View File

@ -0,0 +1,254 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.transactional.TransactionLogger;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.util.Progressable;
/**
* Responsible recovering transactional information from the HLog.
*/
class THLogRecoveryManager {
private static final Log LOG = LogFactory
.getLog(THLogRecoveryManager.class);
private final FileSystem fileSystem;
private final HRegionInfo regionInfo;
private final HBaseConfiguration conf;
/**
* @param region
*/
public THLogRecoveryManager(final TransactionalRegion region) {
this.fileSystem = region.getFilesystem();
this.regionInfo = region.getRegionInfo();
this.conf = region.getConf();
}
// For Testing
THLogRecoveryManager(final FileSystem fileSystem,
final HRegionInfo regionInfo, final HBaseConfiguration conf) {
this.fileSystem = fileSystem;
this.regionInfo = regionInfo;
this.conf = conf;
}
/**
* @param reconstructionLog
* @param maxSeqID
* @param reporter
* @return map of batch updates
* @throws UnsupportedEncodingException
* @throws IOException
*/
public Map<Long, List<KeyValue>> getCommitsFromLog(
final Path reconstructionLog, final long maxSeqID,
final Progressable reporter) throws UnsupportedEncodingException,
IOException {
if (reconstructionLog == null || !fileSystem.exists(reconstructionLog)) {
// Nothing to do.
return null;
}
// Check its not empty.
FileStatus[] stats = fileSystem.listStatus(reconstructionLog);
if (stats == null || stats.length == 0) {
LOG.warn("Passed reconstruction log " + reconstructionLog
+ " is zero-length");
return null;
}
SortedMap<Long, List<KeyValue>> pendingTransactionsById = new TreeMap<Long, List<KeyValue>>();
SortedMap<Long, List<KeyValue>> commitedTransactionsById = new TreeMap<Long, List<KeyValue>>();
Set<Long> abortedTransactions = new HashSet<Long>();
SequenceFile.Reader logReader = new SequenceFile.Reader(fileSystem,
reconstructionLog, conf);
try {
THLogKey key = new THLogKey();
KeyValue val = new KeyValue();
long skippedEdits = 0;
long totalEdits = 0;
long startCount = 0;
long writeCount = 0;
long abortCount = 0;
long commitCount = 0;
// How many edits to apply before we send a progress report.
int reportInterval = conf.getInt("hbase.hstore.report.interval.edits",
2000);
while (logReader.next(key, val)) {
LOG.debug("Processing edit: key: " + key.toString() + " val: "
+ val.toString());
if (key.getLogSeqNum() < maxSeqID) {
skippedEdits++;
continue;
}
if (key.getTrxOp() == null || !Bytes.equals(key.getRegionName(), regionInfo.getRegionName())) {
continue;
}
long transactionId = key.getTransactionId();
List<KeyValue> updates = pendingTransactionsById.get(transactionId);
switch (key.getTrxOp()) {
case START:
if (updates != null || abortedTransactions.contains(transactionId)
|| commitedTransactionsById.containsKey(transactionId)) {
LOG.error("Processing start for transaction: " + transactionId
+ ", but have already seen start message");
throw new IOException("Corrupted transaction log");
}
updates = new LinkedList<KeyValue>();
pendingTransactionsById.put(transactionId, updates);
startCount++;
break;
case OP:
if (updates == null) {
LOG.error("Processing edit for transaction: " + transactionId
+ ", but have not seen start message");
throw new IOException("Corrupted transaction log");
}
updates.add(val);
val = new KeyValue();
writeCount++;
break;
case ABORT:
if (updates == null) {
LOG.error("Processing abort for transaction: " + transactionId
+ ", but have not seen start message");
throw new IOException("Corrupted transaction log");
}
abortedTransactions.add(transactionId);
pendingTransactionsById.remove(transactionId);
abortCount++;
break;
case COMMIT:
if (updates == null) {
LOG.error("Processing commit for transaction: " + transactionId
+ ", but have not seen start message");
throw new IOException("Corrupted transaction log");
}
if (abortedTransactions.contains(transactionId)) {
LOG.error("Processing commit for transaction: " + transactionId
+ ", but also have abort message");
throw new IOException("Corrupted transaction log");
}
if (updates.size() == 0) {
LOG
.warn("Transaciton " + transactionId
+ " has no writes in log. ");
}
if (commitedTransactionsById.containsKey(transactionId)) {
LOG.error("Processing commit for transaction: " + transactionId
+ ", but have already commited transaction with that id");
throw new IOException("Corrupted transaction log");
}
pendingTransactionsById.remove(transactionId);
commitedTransactionsById.put(transactionId, updates);
commitCount++;
break;
default:
throw new IllegalStateException("Unexpected log entry type");
}
totalEdits++;
if (reporter != null && (totalEdits % reportInterval) == 0) {
reporter.progress();
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Read " + totalEdits + " tranasctional operations (skipped "
+ skippedEdits + " because sequence id <= " + maxSeqID + "): "
+ startCount + " starts, " + writeCount + " writes, " + abortCount
+ " aborts, and " + commitCount + " commits.");
}
} finally {
logReader.close();
}
if (pendingTransactionsById.size() > 0) {
resolvePendingTransaction(pendingTransactionsById, commitedTransactionsById);
}
return commitedTransactionsById;
}
private void resolvePendingTransaction(
SortedMap<Long, List<KeyValue>> pendingTransactionsById,
SortedMap<Long, List<KeyValue>> commitedTransactionsById) {
LOG.info("Region log has " + pendingTransactionsById.size()
+ " unfinished transactions. Going to the transaction log to resolve");
for (Entry<Long, List<KeyValue>> entry : pendingTransactionsById
.entrySet()) {
TransactionLogger.TransactionStatus transactionStatus = getGlobalTransactionLog()
.getStatusForTransaction(entry.getKey());
if (transactionStatus == null) {
throw new RuntimeException("Cannot resolve tranasction ["
+ entry.getKey() + "] from global tx log.");
}
switch (transactionStatus) {
case ABORTED:
break;
case COMMITTED:
commitedTransactionsById.put(entry.getKey(), entry.getValue());
break;
case PENDING:
break;
}
}
}
private TransactionLogger getGlobalTransactionLog() {
return null;
}
}

View File

@ -0,0 +1,342 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.NavigableSet;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Holds the state of a transaction.
*/
class TransactionState {
private static final Log LOG = LogFactory.getLog(TransactionState.class);
/** Current status. */
public enum Status {
/** Initial status, still performing operations. */
PENDING,
/**
* Checked if we can commit, and said yes. Still need to determine the
* global decision.
*/
COMMIT_PENDING,
/** Committed. */
COMMITED,
/** Aborted. */
ABORTED
}
/**
* Simple container of the range of the scanners we've opened. Used to check
* for conflicting writes.
*/
private static class ScanRange {
protected byte[] startRow;
protected byte[] endRow;
public ScanRange(byte[] startRow, byte[] endRow) {
this.startRow = startRow;
this.endRow = endRow;
}
/**
* Check if this scan range contains the given key.
*
* @param rowKey
* @return boolean
*/
public boolean contains(byte[] rowKey) {
if (startRow != null && Bytes.compareTo(rowKey, startRow) < 0) {
return false;
}
if (endRow != null && Bytes.compareTo(endRow, rowKey) < 0) {
return false;
}
return true;
}
@Override
public String toString() {
return "startRow: "
+ (startRow == null ? "null" : Bytes.toString(startRow))
+ ", endRow: " + (endRow == null ? "null" : Bytes.toString(endRow));
}
}
private final HRegionInfo regionInfo;
private final long hLogStartSequenceId;
private final long transactionId;
private Status status;
private SortedSet<byte[]> readSet = new TreeSet<byte[]>(
Bytes.BYTES_COMPARATOR);
private List<Put> writeSet = new LinkedList<Put>();
private List<ScanRange> scanSet = new LinkedList<ScanRange>();
private Set<TransactionState> transactionsToCheck = new HashSet<TransactionState>();
private int startSequenceNumber;
private Integer sequenceNumber;
private int commitPendingWaits = 0;
TransactionState(final long transactionId, final long rLogStartSequenceId,
HRegionInfo regionInfo) {
this.transactionId = transactionId;
this.hLogStartSequenceId = rLogStartSequenceId;
this.regionInfo = regionInfo;
this.status = Status.PENDING;
}
void addRead(final byte[] rowKey) {
readSet.add(rowKey);
}
Set<byte[]> getReadSet() {
return readSet;
}
void addWrite(final Put write) {
writeSet.add(write);
}
List<Put> getWriteSet() {
return writeSet;
}
void addDelete(final Delete delete) {
throw new UnsupportedOperationException("NYI");
}
/**
* GetFull from the writeSet.
*
* @param row
* @param columns
* @param timestamp
* @return
*/
Result localGet(Get get) {
List<KeyValue> localKVs = new LinkedList<KeyValue>();
for (Put put : writeSet) {
if (!Bytes.equals(get.getRow(), put.getRow())) {
continue;
}
if (put.getTimeStamp() > get.getTimeRange().getMax()) {
continue;
}
if (put.getTimeStamp() < get.getTimeRange().getMin()) {
continue;
}
for (Entry<byte [], NavigableSet<byte []>> getFamilyEntry : get.getFamilyMap().entrySet()) {
List<KeyValue> familyPuts = put.getFamilyMap().get(getFamilyEntry.getKey());
if (familyPuts == null) {
continue;
}
if (getFamilyEntry.getValue() == null){
localKVs.addAll(familyPuts);
} else {
for (KeyValue kv : familyPuts) {
if (getFamilyEntry.getValue().contains(kv.getQualifier())) {
localKVs.add(kv);
}
}
}
}
}
if (localKVs.isEmpty()) {
return null;
}
return new Result(localKVs);
}
void addTransactionToCheck(final TransactionState transaction) {
transactionsToCheck.add(transaction);
}
boolean hasConflict() {
for (TransactionState transactionState : transactionsToCheck) {
if (hasConflict(transactionState)) {
return true;
}
}
return false;
}
private boolean hasConflict(final TransactionState checkAgainst) {
if (checkAgainst.getStatus().equals(TransactionState.Status.ABORTED)) {
return false; // Cannot conflict with aborted transactions
}
for (Put otherUpdate : checkAgainst.getWriteSet()) {
if (this.getReadSet().contains(otherUpdate.getRow())) {
LOG.debug("Transaction [" + this.toString()
+ "] has read which conflicts with [" + checkAgainst.toString()
+ "]: region [" + regionInfo.getRegionNameAsString() + "], row["
+ Bytes.toString(otherUpdate.getRow()) + "]");
return true;
}
for (ScanRange scanRange : this.scanSet) {
if (scanRange.contains(otherUpdate.getRow())) {
LOG.debug("Transaction [" + this.toString()
+ "] has scan which conflicts with [" + checkAgainst.toString()
+ "]: region [" + regionInfo.getRegionNameAsString() + "], scanRange[" +
scanRange.toString()+"] ,row["
+ Bytes.toString(otherUpdate.getRow()) + "]");
return true;
}
}
}
return false;
}
/**
* Get the status.
*
* @return Return the status.
*/
Status getStatus() {
return status;
}
/**
* Set the status.
*
* @param status The status to set.
*/
void setStatus(final Status status) {
this.status = status;
}
/**
* Get the startSequenceNumber.
*
* @return Return the startSequenceNumber.
*/
int getStartSequenceNumber() {
return startSequenceNumber;
}
/**
* Set the startSequenceNumber.
*
* @param startSequenceNumber
*/
void setStartSequenceNumber(final int startSequenceNumber) {
this.startSequenceNumber = startSequenceNumber;
}
/**
* Get the sequenceNumber.
*
* @return Return the sequenceNumber.
*/
Integer getSequenceNumber() {
return sequenceNumber;
}
/**
* Set the sequenceNumber.
*
* @param sequenceNumber The sequenceNumber to set.
*/
void setSequenceNumber(final Integer sequenceNumber) {
this.sequenceNumber = sequenceNumber;
}
@Override
public String toString() {
StringBuilder result = new StringBuilder();
result.append("[transactionId: ");
result.append(transactionId);
result.append(" status: ");
result.append(status.name());
result.append(" read Size: ");
result.append(readSet.size());
result.append(" scan Size: ");
result.append(scanSet.size());
result.append(" write Size: ");
result.append(writeSet.size());
result.append(" startSQ: ");
result.append(startSequenceNumber);
if (sequenceNumber != null) {
result.append(" commitedSQ:");
result.append(sequenceNumber);
}
result.append("]");
return result.toString();
}
/**
* Get the transactionId.
*
* @return Return the transactionId.
*/
long getTransactionId() {
return transactionId;
}
/**
* Get the startSequenceId.
*
* @return Return the startSequenceId.
*/
long getHLogStartSequenceId() {
return hLogStartSequenceId;
}
void addScan(Scan scan) {
ScanRange scanRange = new ScanRange(scan.getStartRow(), scan.getStopRow());
LOG.trace(String.format(
"Adding scan for transcaction [%s], from [%s] to [%s]", transactionId,
scanRange.startRow == null ? "null" : Bytes
.toString(scanRange.startRow), scanRange.endRow == null ? "null"
: Bytes.toString(scanRange.endRow)));
scanSet.add(scanRange);
}
int getCommitPendingWaits() {
return commitPendingWaits;
}
void incrementCommitPendingWaits() {
this.commitPendingWaits++;
}
}

View File

@ -0,0 +1,733 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicInteger;
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.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LeaseException;
import org.apache.hadoop.hbase.LeaseListener;
import org.apache.hadoop.hbase.Leases;
import org.apache.hadoop.hbase.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.transactional.UnknownTransactionException;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.transactional.TransactionState.Status;
import org.apache.hadoop.util.Progressable;
/**
* Regionserver which provides transactional support for atomic transactions.
* This is achieved with optimistic concurrency control (see
* http://www.seas.upenn.edu/~zives/cis650/papers/opt-cc.pdf). We keep track
* read and write sets for each transaction, and hold off on processing the
* writes. To decide to commit a transaction we check its read sets with all
* transactions that have committed while it was running for overlaps.
* <p>
* Because transactions can span multiple regions, all regions must agree to
* commit a transactions. The client side of this commit protocol is encoded in
* org.apache.hadoop.hbase.client.transactional.TransactionManger
* <p>
* In the event of an failure of the client mid-commit, (after we voted yes), we
* will have to consult the transaction log to determine the final decision of
* the transaction. This is not yet implemented.
*/
public class TransactionalRegion extends HRegion {
private static final String OLD_TRANSACTION_FLUSH = "hbase.transaction.flush";
private static final int DEFAULT_OLD_TRANSACTION_FLUSH = 100; // Do a flush if
// we have this
// many old
// transactions..
static final Log LOG = LogFactory.getLog(TransactionalRegion.class);
// Collection of active transactions (PENDING) keyed by id.
protected Map<String, TransactionState> transactionsById = new HashMap<String, TransactionState>();
// Map of recent transactions that are COMMIT_PENDING or COMMITED keyed by
// their sequence number
private SortedMap<Integer, TransactionState> commitedTransactionsBySequenceNumber = Collections
.synchronizedSortedMap(new TreeMap<Integer, TransactionState>());
// Collection of transactions that are COMMIT_PENDING
private Set<TransactionState> commitPendingTransactions = Collections
.synchronizedSet(new HashSet<TransactionState>());
private AtomicInteger nextSequenceId = new AtomicInteger(0);
private Object commitCheckLock = new Object();
private THLog hlog;
private final int oldTransactionFlushTrigger;
private final Leases transactionLeases;
/**
* @param basedir
* @param log
* @param fs
* @param conf
* @param regionInfo
* @param flushListener
*/
public TransactionalRegion(final Path basedir, final HLog log,
final FileSystem fs, final HBaseConfiguration conf,
final HRegionInfo regionInfo, final FlushRequester flushListener,
final Leases transactionalLeases) {
super(basedir, log, fs, conf, regionInfo, flushListener);
this.hlog = (THLog) log;
oldTransactionFlushTrigger = conf.getInt(OLD_TRANSACTION_FLUSH,
DEFAULT_OLD_TRANSACTION_FLUSH);
this.transactionLeases = transactionalLeases;
}
@Override
protected void doReconstructionLog(final Path oldLogFile,
final long minSeqId, final long maxSeqId, final Progressable reporter)
throws UnsupportedEncodingException, IOException {
super.doReconstructionLog(oldLogFile, minSeqId, maxSeqId, reporter);
THLogRecoveryManager recoveryManager = new THLogRecoveryManager(this);
Map<Long, List<KeyValue>> commitedTransactionsById = recoveryManager
.getCommitsFromLog(oldLogFile, minSeqId, reporter);
if (commitedTransactionsById != null && commitedTransactionsById.size() > 0) {
LOG.debug("found " + commitedTransactionsById.size()
+ " COMMITED transactions");
for (Entry<Long, List<KeyValue>> entry : commitedTransactionsById
.entrySet()) {
LOG.debug("Writing " + entry.getValue().size()
+ " updates for transaction " + entry.getKey());
for (KeyValue b : entry.getValue()) {
Put put = new Put(b.getRow());
put.add(b);
super.put(put, true); // These are walled so they live forever
}
}
// LOG.debug("Flushing cache"); // We must trigger a cache flush,
// otherwise
// we will would ignore the log on subsequent failure
// if (!super.flushcache()) {
// LOG.warn("Did not flush cache");
// }
}
}
/**
* We need to make sure that we don't complete a cache flush between running
* transactions. If we did, then we would not find all log messages needed to
* restore the transaction, as some of them would be before the last
* "complete" flush id.
*/
@Override
protected long getCompleteCacheFlushSequenceId(final long currentSequenceId) {
LinkedList<TransactionState> transactionStates;
synchronized (transactionsById) {
transactionStates = new LinkedList<TransactionState>(transactionsById
.values());
}
long minPendingStartSequenceId = currentSequenceId;
for (TransactionState transactionState : transactionStates) {
minPendingStartSequenceId = Math.min(minPendingStartSequenceId,
transactionState.getHLogStartSequenceId());
}
return minPendingStartSequenceId;
}
/**
* @param transactionId
* @throws IOException
*/
public void beginTransaction(final long transactionId) throws IOException {
checkClosing();
String key = String.valueOf(transactionId);
if (transactionsById.get(key) != null) {
TransactionState alias = getTransactionState(transactionId);
if (alias != null) {
alias.setStatus(Status.ABORTED);
retireTransaction(alias);
}
LOG.error("Existing trasaction with id [" + key + "] in region ["
+ super.getRegionInfo().getRegionNameAsString() + "]");
throw new IOException("Already exiting transaction id: " + key);
}
TransactionState state = new TransactionState(transactionId, super.getLog()
.getSequenceNumber(), super.getRegionInfo());
// Order is important here ...
List<TransactionState> commitPendingCopy = new LinkedList<TransactionState>(
commitPendingTransactions);
for (TransactionState commitPending : commitPendingCopy) {
state.addTransactionToCheck(commitPending);
}
state.setStartSequenceNumber(nextSequenceId.get());
synchronized (transactionsById) {
transactionsById.put(key, state);
}
try {
transactionLeases.createLease(getLeaseId(transactionId),
new TransactionLeaseListener(key));
} catch (LeaseStillHeldException e) {
LOG.error("Lease still held for [" + key + "] in region ["
+ super.getRegionInfo().getRegionNameAsString() + "]");
throw new RuntimeException(e);
}
LOG.debug("Begining transaction " + key + " in region "
+ super.getRegionInfo().getRegionNameAsString());
this.hlog.writeStartToLog(super.getRegionInfo(), transactionId);
maybeTriggerOldTransactionFlush();
}
private String getLeaseId(long transactionId) {
return super.getRegionInfo().getRegionNameAsString() + transactionId;
}
public Result get(final long transactionId, Get get) throws IOException {
checkClosing();
TransactionState state = getTransactionState(transactionId);
state.addRead(get.getRow());
Result superGet = super.get(get, null);
Result localGet = state.localGet(get);
if (localGet != null) {
LOG
.trace("Transactional get of something we've written in the same transaction "
+ transactionId);
List<KeyValue> mergedGet = new ArrayList<KeyValue>(Arrays.asList(localGet
.raw()));
if (superGet != null && !superGet.isEmpty()) {
for (KeyValue kv : superGet.raw()) {
if (!localGet.containsColumn(kv.getFamily(), kv.getQualifier())) {
mergedGet.add(kv);
}
}
}
return new Result(mergedGet);
}
return superGet;
}
/**
* Get a transactional scanner.
*/
public InternalScanner getScanner(final long transactionId, Scan scan)
throws IOException {
checkClosing();
TransactionState state = getTransactionState(transactionId);
state.addScan(scan);
return new ScannerWrapper(transactionId, super.getScanner(scan));
}
/**
* Add a write to the transaction. Does not get applied until commit process.
*
* @param transactionId
* @param b
* @throws IOException
*/
public void put(final long transactionId, final Put put) throws IOException {
checkClosing();
TransactionState state = getTransactionState(transactionId);
state.addWrite(put);
this.hlog.writeUpdateToLog(super.getRegionInfo(), transactionId, put);
}
/**
* Add multiple writes to the transaction. Does not get applied until commit
* process.
*
* @param transactionId
* @param puts
* @throws IOException
*/
public void put(final long transactionId, final Put[] puts)
throws IOException {
checkClosing();
TransactionState state = getTransactionState(transactionId);
for (Put put : puts) {
state.addWrite(put);
this.hlog.writeUpdateToLog(super.getRegionInfo(), transactionId, put);
}
}
/**
* Add a delete to the transaction. Does not get applied until commit process.
* FIXME, not sure about this approach
*
* @param transactionId
* @param row
* @param timestamp
* @throws IOException
*/
public void delete(final long transactionId, Delete delete)
throws IOException {
checkClosing();
TransactionState state = getTransactionState(transactionId);
state.addDelete(delete);
this.hlog.writeDeleteToLog(super.getRegionInfo(), transactionId, delete);
}
/**
* @param transactionId
* @return TransactionRegionInterface commit code
* @throws IOException
*/
public int commitRequest(final long transactionId) throws IOException {
checkClosing();
synchronized (commitCheckLock) {
TransactionState state = getTransactionState(transactionId);
if (state == null) {
return TransactionalRegionInterface.COMMIT_UNSUCESSFUL;
}
if (hasConflict(state)) {
state.setStatus(Status.ABORTED);
retireTransaction(state);
return TransactionalRegionInterface.COMMIT_UNSUCESSFUL;
}
// No conflicts, we can commit.
LOG.trace("No conflicts for transaction " + transactionId
+ " found in region " + super.getRegionInfo().getRegionNameAsString()
+ ". Voting for commit");
// If there are writes we must keep record off the transaction
if (state.getWriteSet().size() > 0) {
// Order is important
state.setStatus(Status.COMMIT_PENDING);
commitPendingTransactions.add(state);
state.setSequenceNumber(nextSequenceId.getAndIncrement());
commitedTransactionsBySequenceNumber.put(state.getSequenceNumber(),
state);
return TransactionalRegionInterface.COMMIT_OK;
}
// Otherwise we were read-only and commitable, so we can forget it.
state.setStatus(Status.COMMITED);
retireTransaction(state);
return TransactionalRegionInterface.COMMIT_OK_READ_ONLY;
}
}
/**
* @param transactionId
* @return true if commit is successful
* @throws IOException
*/
public boolean commitIfPossible(final long transactionId) throws IOException {
int status = commitRequest(transactionId);
if (status == TransactionalRegionInterface.COMMIT_OK) {
commit(transactionId);
return true;
} else if (status == TransactionalRegionInterface.COMMIT_OK_READ_ONLY) {
return true;
}
return false;
}
private boolean hasConflict(final TransactionState state) {
// Check transactions that were committed while we were running
for (int i = state.getStartSequenceNumber(); i < nextSequenceId.get(); i++) {
TransactionState other = commitedTransactionsBySequenceNumber.get(i);
if (other == null) {
continue;
}
state.addTransactionToCheck(other);
}
return state.hasConflict();
}
/**
* Commit the transaction.
*
* @param transactionId
* @throws IOException
*/
public void commit(final long transactionId) throws IOException {
// Not checking closing...
TransactionState state;
try {
state = getTransactionState(transactionId);
} catch (UnknownTransactionException e) {
LOG.fatal("Asked to commit unknown transaction: " + transactionId
+ " in region " + super.getRegionInfo().getRegionNameAsString());
// FIXME Write to the transaction log that this transaction was corrupted
throw e;
}
if (!state.getStatus().equals(Status.COMMIT_PENDING)) {
LOG.fatal("Asked to commit a non pending transaction");
// FIXME Write to the transaction log that this transaction was corrupted
throw new IOException("commit failure");
}
commit(state);
}
/**
* Commit the transaction.
*
* @param transactionId
* @throws IOException
*/
public void abort(final long transactionId) throws IOException {
// Not checking closing...
TransactionState state;
try {
state = getTransactionState(transactionId);
} catch (UnknownTransactionException e) {
LOG.info("Asked to abort unknown transaction [" + transactionId
+ "] in region [" + getRegionInfo().getRegionNameAsString()
+ "], ignoring");
return;
}
state.setStatus(Status.ABORTED);
this.hlog.writeAbortToLog(super.getRegionInfo(), state.getTransactionId());
// Following removes needed if we have voted
if (state.getSequenceNumber() != null) {
commitedTransactionsBySequenceNumber.remove(state.getSequenceNumber());
}
commitPendingTransactions.remove(state);
retireTransaction(state);
}
private void commit(final TransactionState state) throws IOException {
LOG.debug("Commiting transaction: " + state.toString() + " to "
+ super.getRegionInfo().getRegionNameAsString());
this.hlog.writeCommitToLog(super.getRegionInfo(), state.getTransactionId());
for (Put update : state.getWriteSet()) {
this.put(update, false); // Don't need to WAL these
// FIME, maybe should be walled so we don't need to look so far back.
}
state.setStatus(Status.COMMITED);
if (state.getWriteSet().size() > 0
&& !commitPendingTransactions.remove(state)) {
LOG
.fatal("Commiting a non-query transaction that is not in commitPendingTransactions");
throw new IOException("commit failure"); // FIXME, how to handle?
}
retireTransaction(state);
}
@Override
public List<StoreFile> close(boolean abort) throws IOException {
prepareToClose();
if (!commitPendingTransactions.isEmpty()) {
// FIXME, better way to handle?
LOG.warn("Closing transactional region ["
+ getRegionInfo().getRegionNameAsString() + "], but still have ["
+ commitPendingTransactions.size()
+ "] transactions that are pending commit");
}
return super.close(abort);
}
@Override
protected void prepareToSplit() {
prepareToClose();
}
boolean closing = false;
/**
* Get ready to close.
*
*/
void prepareToClose() {
LOG.info("Preparing to close region "
+ getRegionInfo().getRegionNameAsString());
closing = true;
while (!commitPendingTransactions.isEmpty()) {
LOG.info("Preparing to closing transactional region ["
+ getRegionInfo().getRegionNameAsString() + "], but still have ["
+ commitPendingTransactions.size()
+ "] transactions that are pending commit. Sleeping");
for (TransactionState s : commitPendingTransactions) {
LOG.info(s.toString());
}
try {
Thread.sleep(200);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
}
private void checkClosing() throws IOException {
if (closing) {
throw new IOException("closing region, no more transaction allowed");
}
}
// Cancel leases, and removed from lease lookup. This transaction may still
// live in commitedTransactionsBySequenceNumber and commitPendingTransactions
private void retireTransaction(final TransactionState state) {
String key = String.valueOf(state.getTransactionId());
try {
transactionLeases.cancelLease(getLeaseId(state.getTransactionId()));
} catch (LeaseException e) {
// Ignore
}
transactionsById.remove(key);
}
protected TransactionState getTransactionState(final long transactionId)
throws UnknownTransactionException {
String key = String.valueOf(transactionId);
TransactionState state = null;
state = transactionsById.get(key);
if (state == null) {
LOG.debug("Unknown transaction: [" + key + "], region: ["
+ getRegionInfo().getRegionNameAsString() + "]");
throw new UnknownTransactionException("transaction: [" + key
+ "], region: [" + getRegionInfo().getRegionNameAsString() + "]");
}
try {
transactionLeases.renewLease(getLeaseId(transactionId));
} catch (LeaseException e) {
throw new RuntimeException(e);
}
return state;
}
private void maybeTriggerOldTransactionFlush() {
if (commitedTransactionsBySequenceNumber.size() > oldTransactionFlushTrigger) {
removeUnNeededCommitedTransactions();
}
}
/**
* Cleanup references to committed transactions that are no longer needed.
*
*/
synchronized void removeUnNeededCommitedTransactions() {
Integer minStartSeqNumber = getMinStartSequenceNumber();
if (minStartSeqNumber == null) {
minStartSeqNumber = Integer.MAX_VALUE; // Remove all
}
int numRemoved = 0;
// Copy list to avoid conc update exception
for (Entry<Integer, TransactionState> entry : new LinkedList<Entry<Integer, TransactionState>>(
commitedTransactionsBySequenceNumber.entrySet())) {
if (entry.getKey() >= minStartSeqNumber) {
break;
}
numRemoved = numRemoved
+ (commitedTransactionsBySequenceNumber.remove(entry.getKey()) == null ? 0
: 1);
numRemoved++;
}
if (LOG.isDebugEnabled()) {
StringBuilder debugMessage = new StringBuilder();
if (numRemoved > 0) {
debugMessage.append("Removed [").append(numRemoved).append(
"] commited transactions");
if (minStartSeqNumber == Integer.MAX_VALUE) {
debugMessage.append("with any sequence number");
} else {
debugMessage.append("with sequence lower than [").append(
minStartSeqNumber).append("].");
}
if (!commitedTransactionsBySequenceNumber.isEmpty()) {
debugMessage.append(" Still have [").append(
commitedTransactionsBySequenceNumber.size()).append("] left.");
} else {
debugMessage.append(" None left.");
}
LOG.debug(debugMessage.toString());
} else if (commitedTransactionsBySequenceNumber.size() > 0) {
debugMessage.append(
"Could not remove any transactions, and still have ").append(
commitedTransactionsBySequenceNumber.size()).append(" left");
LOG.debug(debugMessage.toString());
}
}
}
private Integer getMinStartSequenceNumber() {
LinkedList<TransactionState> transactionStates;
synchronized (transactionsById) {
transactionStates = new LinkedList<TransactionState>(transactionsById
.values());
}
Integer min = null;
for (TransactionState transactionState : transactionStates) {
if (min == null || transactionState.getStartSequenceNumber() < min) {
min = transactionState.getStartSequenceNumber();
}
}
return min;
}
private void resolveTransactionFromLog(final TransactionState transactionState)
throws IOException {
LOG
.error("Global transaction log is not Implemented. (Optimisticly) assuming transaction commit!");
commit(transactionState);
// throw new RuntimeException("Global transaction log is not Implemented");
}
private static final int MAX_COMMIT_PENDING_WAITS = 10;
private class TransactionLeaseListener implements LeaseListener {
private final String transactionName;
TransactionLeaseListener(final String n) {
this.transactionName = n;
}
public void leaseExpired() {
LOG.info("Transaction [" + this.transactionName + "] expired in region ["
+ getRegionInfo().getRegionNameAsString() + "]");
TransactionState s = null;
synchronized (transactionsById) {
s = transactionsById.remove(transactionName);
}
if (s == null) {
LOG.warn("Unknown transaction expired " + this.transactionName);
return;
}
switch (s.getStatus()) {
case PENDING:
s.setStatus(Status.ABORTED); // Other transactions may have a ref
break;
case COMMIT_PENDING:
LOG.info("Transaction " + s.getTransactionId()
+ " expired in COMMIT_PENDING state");
try {
if (s.getCommitPendingWaits() > MAX_COMMIT_PENDING_WAITS) {
LOG.info("Checking transaction status in transaction log");
resolveTransactionFromLog(s);
break;
}
LOG.info("renewing lease and hoping for commit");
s.incrementCommitPendingWaits();
String key = Long.toString(s.getTransactionId());
transactionsById.put(key, s);
try {
transactionLeases.createLease(getLeaseId(s.getTransactionId()),
this);
} catch (LeaseStillHeldException e) {
transactionLeases.renewLease(getLeaseId(s.getTransactionId()));
}
} catch (IOException e) {
throw new RuntimeException(e);
}
break;
default:
LOG.warn("Unexpected status on expired lease");
}
}
}
/** Wrapper which keeps track of rows returned by scanner. */
private class ScannerWrapper implements InternalScanner {
private long transactionId;
private InternalScanner scanner;
/**
* @param transactionId
* @param scanner
* @throws UnknownTransactionException
*/
public ScannerWrapper(final long transactionId,
final InternalScanner scanner) throws UnknownTransactionException {
this.transactionId = transactionId;
this.scanner = scanner;
}
public void close() throws IOException {
scanner.close();
}
public boolean next(List<KeyValue> results) throws IOException {
boolean result = scanner.next(results);
TransactionState state = getTransactionState(transactionId);
// FIXME need to weave in new stuff from this transaction too.
return result;
}
}
}

View File

@ -0,0 +1,246 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Leases;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.MapWritable;
import org.apache.hadoop.util.Progressable;
/**
* RegionServer with support for transactions. Transactional logic is at the
* region level, so we mostly just delegate to the appropriate
* TransactionalRegion.
*/
public class TransactionalRegionServer extends HRegionServer implements
TransactionalRegionInterface {
private static final String LEASE_TIME = "hbase.transaction.leasetime";
private static final int DEFAULT_LEASE_TIME = 60 * 1000;
private static final int LEASE_CHECK_FREQUENCY = 1000;
static final Log LOG = LogFactory.getLog(TransactionalRegionServer.class);
private final Leases transactionLeases;
private final CleanOldTransactionsChore cleanOldTransactionsThread;
/**
* @param address
* @param conf
* @throws IOException
*/
public TransactionalRegionServer(final HBaseConfiguration conf)
throws IOException {
super(conf);
cleanOldTransactionsThread = new CleanOldTransactionsChore(this,
super.stopRequested);
transactionLeases = new Leases(conf.getInt(LEASE_TIME, DEFAULT_LEASE_TIME),
LEASE_CHECK_FREQUENCY);
LOG.error("leases time:"+conf.getInt(LEASE_TIME, DEFAULT_LEASE_TIME));
}
@Override
public long getProtocolVersion(final String protocol, final long clientVersion)
throws IOException {
if (protocol.equals(TransactionalRegionInterface.class.getName())) {
return HBaseRPCProtocolVersion.versionID;
}
return super.getProtocolVersion(protocol, clientVersion);
}
@Override
protected void init(final MapWritable c) throws IOException {
super.init(c);
String n = Thread.currentThread().getName();
UncaughtExceptionHandler handler = new UncaughtExceptionHandler() {
public void uncaughtException(final Thread t, final Throwable e) {
abort();
LOG.fatal("Set stop flag in " + t.getName(), e);
}
};
Threads.setDaemonThreadRunning(this.cleanOldTransactionsThread, n
+ ".oldTransactionCleaner", handler);
Threads.setDaemonThreadRunning(this.transactionLeases, "Transactional leases");
}
@Override
protected HLog instantiateHLog(Path logdir) throws IOException {
HLog newlog = new THLog(super.getFileSystem(), logdir, conf, super.getLogRoller());
return newlog;
}
@Override
protected HRegion instantiateRegion(final HRegionInfo regionInfo)
throws IOException {
HRegion r = new TransactionalRegion(HTableDescriptor.getTableDir(super
.getRootDir(), regionInfo.getTableDesc().getName()), super.hlog, super
.getFileSystem(), super.conf, regionInfo, super.getFlushRequester(), this.getTransactionalLeases());
r.initialize(null, new Progressable() {
public void progress() {
addProcessingMessage(regionInfo);
}
});
return r;
}
protected TransactionalRegion getTransactionalRegion(final byte[] regionName)
throws NotServingRegionException {
return (TransactionalRegion) super.getRegion(regionName);
}
protected Leases getTransactionalLeases() {
return this.transactionLeases;
}
/** We want to delay the close region for a bit if we have commit pending transactions.
*
*/
@Override
protected void closeRegion(final HRegionInfo hri, final boolean reportWhenCompleted)
throws IOException {
getTransactionalRegion(hri.getRegionName()).prepareToClose();
super.closeRegion(hri, reportWhenCompleted);
}
public void abort(final byte[] regionName, final long transactionId)
throws IOException {
checkOpen();
super.getRequestCount().incrementAndGet();
try {
getTransactionalRegion(regionName).abort(transactionId);
} catch(NotServingRegionException e) {
LOG.info("Got not serving region durring abort. Ignoring.");
} catch (IOException e) {
checkFileSystem();
throw e;
}
}
public void commit(final byte[] regionName, final long transactionId)
throws IOException {
checkOpen();
super.getRequestCount().incrementAndGet();
try {
getTransactionalRegion(regionName).commit(transactionId);
} catch (IOException e) {
checkFileSystem();
throw e;
}
}
public int commitRequest(final byte[] regionName, final long transactionId)
throws IOException {
checkOpen();
super.getRequestCount().incrementAndGet();
try {
return getTransactionalRegion(regionName).commitRequest(transactionId);
} catch (IOException e) {
checkFileSystem();
throw e;
}
}
public boolean commitIfPossible(byte[] regionName, long transactionId)
throws IOException {
checkOpen();
super.getRequestCount().incrementAndGet();
try {
return getTransactionalRegion(regionName).commitIfPossible(transactionId);
} catch (IOException e) {
checkFileSystem();
throw e;
}
}
public long openScanner(final long transactionId, byte [] regionName, Scan scan)
throws IOException {
checkOpen();
NullPointerException npe = null;
if (regionName == null) {
npe = new NullPointerException("regionName is null");
} else if (scan == null) {
npe = new NullPointerException("scan is null");
}
if (npe != null) {
throw new IOException("Invalid arguments to openScanner", npe);
}
super.getRequestCount().incrementAndGet();
try {
TransactionalRegion r = getTransactionalRegion(regionName);
InternalScanner s = r.getScanner(transactionId, scan);
long scannerId = addScanner(s);
return scannerId;
} catch (IOException e) {
LOG.error("Error opening scanner (fsOk: " + this.fsOk + ")",
RemoteExceptionHandler.checkIOException(e));
checkFileSystem();
throw e;
}
}
public void beginTransaction(final long transactionId, final byte[] regionName)
throws IOException {
getTransactionalRegion(regionName).beginTransaction(transactionId);
}
public void delete(long transactionId, byte[] regionName, Delete delete)
throws IOException {
getTransactionalRegion(regionName).delete(transactionId, delete);
}
public Result get(long transactionId, byte[] regionName, Get get)
throws IOException {
return getTransactionalRegion(regionName).get(transactionId, get);
}
public void put(long transactionId, byte[] regionName, Put put)
throws IOException {
getTransactionalRegion(regionName).put(transactionId, put);
}
public int put(long transactionId, byte[] regionName, Put[] puts)
throws IOException {
getTransactionalRegion(regionName).put(transactionId, puts);
return puts.length; // ??
}
}

View File

@ -0,0 +1,135 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.tableindexed;
import java.io.IOException;
import java.util.Random;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.PerformanceEvaluation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
public class TestIndexedTable extends HBaseClusterTestCase {
private static final Log LOG = LogFactory.getLog(TestIndexedTable.class);
private static final String TABLE_NAME = "table1";
private static final byte[] FAMILY_COLON = Bytes.toBytes("family:");
private static final byte[] FAMILY = Bytes.toBytes("family");
private static final byte[] QUAL_A = Bytes.toBytes("a");
private static final byte[] COL_A = Bytes.toBytes("family:a");
private static final String INDEX_COL_A = "A";
private static final int NUM_ROWS = 10;
private static final int MAX_VAL = 10000;
private IndexedTableAdmin admin;
private IndexedTable table;
private Random random = new Random();
/** constructor */
public TestIndexedTable() {
conf
.set(HConstants.REGION_SERVER_IMPL, IndexedRegionServer.class.getName());
conf.setInt("hbase.master.info.port", -1);
conf.setInt("hbase.regionserver.info.port", -1);
}
@Override
protected void setUp() throws Exception {
super.setUp();
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
desc.addFamily(new HColumnDescriptor(FAMILY_COLON));
IndexedTableDescriptor indexDesc = new IndexedTableDescriptor(desc);
// Create a new index that does lexicographic ordering on COL_A
IndexSpecification colAIndex = new IndexSpecification(INDEX_COL_A,
COL_A);
indexDesc.addIndex(colAIndex);
admin = new IndexedTableAdmin(conf);
admin.createIndexedTable(indexDesc);
table = new IndexedTable(conf, desc.getName());
}
private void writeInitalRows() throws IOException {
for (int i = 0; i < NUM_ROWS; i++) {
Put update = new Put(PerformanceEvaluation.format(i));
byte[] valueA = PerformanceEvaluation.format(random.nextInt(MAX_VAL));
update.add(FAMILY, QUAL_A, valueA);
table.put(update);
LOG.info("Inserted row [" + Bytes.toString(update.getRow()) + "] val: ["
+ Bytes.toString(valueA) + "]");
}
}
public void testInitialWrites() throws IOException {
writeInitalRows();
assertRowsInOrder(NUM_ROWS);
}
private void assertRowsInOrder(int numRowsExpected) throws IndexNotFoundException, IOException {
ResultScanner scanner = table.getIndexedScanner(INDEX_COL_A,
HConstants.EMPTY_START_ROW, null, null, null);
int numRows = 0;
byte[] lastColA = null;
for (Result rowResult : scanner) {
byte[] colA = rowResult.getValue(COL_A);
LOG.info("index scan : row [" + Bytes.toString(rowResult.getRow())
+ "] value [" + Bytes.toString(colA) + "]");
if (lastColA != null) {
Assert.assertTrue(Bytes.compareTo(lastColA, colA) <= 0);
}
lastColA = colA;
numRows++;
}
scanner.close();
Assert.assertEquals(numRowsExpected, numRows);
}
public void testMultipleWrites() throws IOException {
writeInitalRows();
writeInitalRows(); // Update the rows.
assertRowsInOrder(NUM_ROWS);
}
public void testDelete() throws IOException {
writeInitalRows();
// Delete the first row;
table.deleteAll(PerformanceEvaluation.format(0));
assertRowsInOrder(NUM_ROWS - 1);
}
}

View File

@ -0,0 +1,416 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicBoolean;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Stress Test the transaction functionality. This requires to run an
* {@link TransactionalRegionServer}. We run many threads doing reads/writes
* which may conflict with each other. We have two types of transactions, those
* which operate on rows of a single table, and those which operate on rows
* across multiple tables. Each transaction type has a modification operation
* which changes two values while maintaining the sum. Also each transaction
* type has a consistency-check operation which sums all rows and verifies that
* the sum is as expected.
*/
public class StressTestTransactions extends HBaseClusterTestCase {
protected static final Log LOG = LogFactory
.getLog(StressTestTransactions.class);
private static final int NUM_TABLES = 3;
private static final int NUM_ST_ROWS = 3;
private static final int NUM_MT_ROWS = 3;
private static final int NUM_TRANSACTIONS_PER_THREAD = 100;
private static final int NUM_SINGLE_TABLE_THREADS = 6;
private static final int NUM_MULTI_TABLE_THREADS = 6;
private static final int PRE_COMMIT_SLEEP = 10;
protected static final Random RAND = new Random();
private static final byte[] FAMILY_COLON = Bytes.toBytes("family:");
private static final byte[] FAMILY = Bytes.toBytes("family");
private static final byte[] QUAL_A = Bytes.toBytes("a");
static final byte[] COL = Bytes.toBytes("family:a");
private HBaseAdmin admin;
protected TransactionalTable[] tables;
protected TransactionManager transactionManager;
/** constructor */
public StressTestTransactions() {
conf.set(HConstants.REGION_SERVER_CLASS, TransactionalRegionInterface.class
.getName());
conf.set(HConstants.REGION_SERVER_IMPL, TransactionalRegionServer.class
.getName());
}
@Override
protected void setUp() throws Exception {
super.setUp();
tables = new TransactionalTable[NUM_TABLES];
for (int i = 0; i < tables.length; i++) {
HTableDescriptor desc = new HTableDescriptor(makeTableName(i));
desc.addFamily(new HColumnDescriptor(FAMILY_COLON));
admin = new HBaseAdmin(conf);
admin.createTable(desc);
tables[i] = new TransactionalTable(conf, desc.getName());
}
transactionManager = new TransactionManager(conf);
}
private String makeTableName(final int i) {
return "table" + i;
}
private void writeInitalValues() throws IOException {
for (TransactionalTable table : tables) {
for (int i = 0; i < NUM_ST_ROWS; i++) {
table.put(new Put(makeSTRow(i)).add(FAMILY, QUAL_A, Bytes
.toBytes(SingleTableTransactionThread.INITIAL_VALUE)));
}
for (int i = 0; i < NUM_MT_ROWS; i++) {
table.put(new Put(makeMTRow(i)).add(FAMILY, QUAL_A, Bytes
.toBytes(MultiTableTransactionThread.INITIAL_VALUE)));
}
}
}
protected byte[] makeSTRow(final int i) {
return Bytes.toBytes("st" + i);
}
protected byte[] makeMTRow(final int i) {
return Bytes.toBytes("mt" + i);
}
static int nextThreadNum = 1;
protected static final AtomicBoolean stopRequest = new AtomicBoolean(false);
static final AtomicBoolean consistencyFailure = new AtomicBoolean(false);
// Thread which runs transactions
abstract class TransactionThread extends Thread {
private int numRuns = 0;
private int numAborts = 0;
private int numUnknowns = 0;
public TransactionThread(final String namePrefix) {
super.setName(namePrefix + "transaction " + nextThreadNum++);
}
@Override
public void run() {
for (int i = 0; i < NUM_TRANSACTIONS_PER_THREAD; i++) {
if (stopRequest.get()) {
return;
}
try {
numRuns++;
transaction();
} catch (UnknownTransactionException e) {
numUnknowns++;
} catch (IOException e) {
throw new RuntimeException(e);
} catch (CommitUnsuccessfulException e) {
numAborts++;
}
}
}
protected abstract void transaction() throws IOException,
CommitUnsuccessfulException;
public int getNumAborts() {
return numAborts;
}
public int getNumUnknowns() {
return numUnknowns;
}
protected void preCommitSleep() {
try {
Thread.sleep(PRE_COMMIT_SLEEP);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
protected void consistencyFailure() {
LOG.fatal("Consistency failure");
stopRequest.set(true);
consistencyFailure.set(true);
}
/**
* Get the numRuns.
*
* @return Return the numRuns.
*/
public int getNumRuns() {
return numRuns;
}
}
// Atomically change the value of two rows rows while maintaining the sum.
// This should preserve the global sum of the rows, which is also checked
// with a transaction.
private class SingleTableTransactionThread extends TransactionThread {
private static final int INITIAL_VALUE = 10;
public static final int TOTAL_SUM = INITIAL_VALUE * NUM_ST_ROWS;
private static final int MAX_TRANSFER_AMT = 100;
private TransactionalTable table;
boolean doCheck = false;
public SingleTableTransactionThread() {
super("single table ");
}
@Override
protected void transaction() throws IOException,
CommitUnsuccessfulException {
if (doCheck) {
checkTotalSum();
} else {
doSingleRowChange();
}
doCheck = !doCheck;
}
private void doSingleRowChange() throws IOException,
CommitUnsuccessfulException {
table = tables[RAND.nextInt(NUM_TABLES)];
int transferAmount = RAND.nextInt(MAX_TRANSFER_AMT * 2)
- MAX_TRANSFER_AMT;
int row1Index = RAND.nextInt(NUM_ST_ROWS);
int row2Index;
do {
row2Index = RAND.nextInt(NUM_ST_ROWS);
} while (row2Index == row1Index);
byte[] row1 = makeSTRow(row1Index);
byte[] row2 = makeSTRow(row2Index);
TransactionState transactionState = transactionManager.beginTransaction();
int row1Amount = Bytes.toInt(table.get(transactionState,
new Get(row1).addColumn(COL)).getValue(COL));
int row2Amount = Bytes.toInt(table.get(transactionState,
new Get(row2).addColumn(COL)).getValue(COL));
row1Amount -= transferAmount;
row2Amount += transferAmount;
table.put(transactionState, new Put(row1).add(FAMILY, QUAL_A, Bytes
.toBytes(row1Amount)));
table.put(transactionState, new Put(row2).add(FAMILY, QUAL_A, Bytes
.toBytes(row2Amount)));
super.preCommitSleep();
transactionManager.tryCommit(transactionState);
LOG.debug("Commited");
}
// Check the table we last mutated
private void checkTotalSum() throws IOException,
CommitUnsuccessfulException {
TransactionState transactionState = transactionManager.beginTransaction();
int totalSum = 0;
for (int i = 0; i < NUM_ST_ROWS; i++) {
totalSum += Bytes.toInt(table.get(transactionState,
new Get(makeSTRow(i)).addColumn(COL)).getValue(COL));
}
transactionManager.tryCommit(transactionState);
if (TOTAL_SUM != totalSum) {
super.consistencyFailure();
}
}
}
// Similar to SingleTable, but this time we maintain consistency across tables
// rather than rows
private class MultiTableTransactionThread extends TransactionThread {
private static final int INITIAL_VALUE = 1000;
public static final int TOTAL_SUM = INITIAL_VALUE * NUM_TABLES;
private static final int MAX_TRANSFER_AMT = 100;
private byte[] row;
boolean doCheck = false;
public MultiTableTransactionThread() {
super("multi table");
}
@Override
protected void transaction() throws IOException,
CommitUnsuccessfulException {
if (doCheck) {
checkTotalSum();
} else {
doSingleRowChange();
}
doCheck = !doCheck;
}
private void doSingleRowChange() throws IOException,
CommitUnsuccessfulException {
row = makeMTRow(RAND.nextInt(NUM_MT_ROWS));
int transferAmount = RAND.nextInt(MAX_TRANSFER_AMT * 2)
- MAX_TRANSFER_AMT;
int table1Index = RAND.nextInt(tables.length);
int table2Index;
do {
table2Index = RAND.nextInt(tables.length);
} while (table2Index == table1Index);
TransactionalTable table1 = tables[table1Index];
TransactionalTable table2 = tables[table2Index];
TransactionState transactionState = transactionManager.beginTransaction();
int table1Amount = Bytes.toInt(table1.get(transactionState,
new Get(row).addColumn(COL)).getValue(COL));
int table2Amount = Bytes.toInt(table2.get(transactionState,
new Get(row).addColumn(COL)).getValue(COL));
table1Amount -= transferAmount;
table2Amount += transferAmount;
table1.put(transactionState, new Put(row).add(FAMILY, QUAL_A, Bytes
.toBytes(table1Amount)));
table2.put(transactionState, new Put(row).add(FAMILY, QUAL_A, Bytes
.toBytes(table2Amount)));
super.preCommitSleep();
transactionManager.tryCommit(transactionState);
LOG.trace(Bytes.toString(table1.getTableName()) + ": " + table1Amount);
LOG.trace(Bytes.toString(table2.getTableName()) + ": " + table2Amount);
}
private void checkTotalSum() throws IOException,
CommitUnsuccessfulException {
TransactionState transactionState = transactionManager.beginTransaction();
int totalSum = 0;
int[] amounts = new int[tables.length];
for (int i = 0; i < tables.length; i++) {
int amount = Bytes.toInt(tables[i].get(transactionState,
new Get(row).addColumn(COL)).getValue(COL));
amounts[i] = amount;
totalSum += amount;
}
transactionManager.tryCommit(transactionState);
for (int i = 0; i < tables.length; i++) {
LOG.trace(Bytes.toString(tables[i].getTableName()) + ": " + amounts[i]);
}
if (TOTAL_SUM != totalSum) {
super.consistencyFailure();
}
}
}
public void testStressTransactions() throws IOException, InterruptedException {
writeInitalValues();
List<TransactionThread> transactionThreads = new LinkedList<TransactionThread>();
for (int i = 0; i < NUM_SINGLE_TABLE_THREADS; i++) {
TransactionThread transactionThread = new SingleTableTransactionThread();
transactionThread.start();
transactionThreads.add(transactionThread);
}
for (int i = 0; i < NUM_MULTI_TABLE_THREADS; i++) {
TransactionThread transactionThread = new MultiTableTransactionThread();
transactionThread.start();
transactionThreads.add(transactionThread);
}
for (TransactionThread transactionThread : transactionThreads) {
transactionThread.join();
}
for (TransactionThread transactionThread : transactionThreads) {
LOG.info(transactionThread.getName() + " done with "
+ transactionThread.getNumAborts() + " aborts, and "
+ transactionThread.getNumUnknowns() + " unknown transactions of "
+ transactionThread.getNumRuns());
}
doFinalConsistencyChecks();
}
private void doFinalConsistencyChecks() throws IOException {
int[] mtSums = new int[NUM_MT_ROWS];
for (int i = 0; i < mtSums.length; i++) {
mtSums[i] = 0;
}
for (TransactionalTable table : tables) {
int thisTableSum = 0;
for (int i = 0; i < NUM_ST_ROWS; i++) {
byte[] row = makeSTRow(i);
thisTableSum += Bytes.toInt(table.get(new Get(row).addColumn(COL))
.getValue(COL));
}
Assert.assertEquals(SingleTableTransactionThread.TOTAL_SUM, thisTableSum);
for (int i = 0; i < NUM_MT_ROWS; i++) {
byte[] row = makeMTRow(i);
mtSums[i] += Bytes.toInt(table.get(new Get(row).addColumn(COL))
.getValue(COL));
}
}
for (int mtSum : mtSums) {
Assert.assertEquals(MultiTableTransactionThread.TOTAL_SUM, mtSum);
}
}
}

View File

@ -0,0 +1,137 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.transactional;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Test the transaction functionality. This requires to run an
* {@link TransactionalRegionServer}.
*/
public class TestTransactions extends HBaseClusterTestCase {
private static final String TABLE_NAME = "table1";
private static final byte[] FAMILY_COLON = Bytes.toBytes("family:");
private static final byte[] FAMILY = Bytes.toBytes("family");
private static final byte[] QUAL_A = Bytes.toBytes("a");
private static final byte[] COL_A = Bytes.toBytes("family:a");
private static final byte[] ROW1 = Bytes.toBytes("row1");
private static final byte[] ROW2 = Bytes.toBytes("row2");
private static final byte[] ROW3 = Bytes.toBytes("row3");
private HBaseAdmin admin;
private TransactionalTable table;
private TransactionManager transactionManager;
/** constructor */
public TestTransactions() {
conf.set(HConstants.REGION_SERVER_CLASS, TransactionalRegionInterface.class
.getName());
conf.set(HConstants.REGION_SERVER_IMPL, TransactionalRegionServer.class
.getName());
}
@Override
protected void setUp() throws Exception {
super.setUp();
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
desc.addFamily(new HColumnDescriptor(FAMILY_COLON));
admin = new HBaseAdmin(conf);
admin.createTable(desc);
table = new TransactionalTable(conf, desc.getName());
transactionManager = new TransactionManager(conf);
writeInitalRow();
}
private void writeInitalRow() throws IOException {
table.put(new Put(ROW1).add(FAMILY, QUAL_A, Bytes.toBytes(1)));
}
public void testSimpleTransaction() throws IOException,
CommitUnsuccessfulException {
TransactionState transactionState = makeTransaction1();
transactionManager.tryCommit(transactionState);
}
public void testTwoTransactionsWithoutConflict() throws IOException,
CommitUnsuccessfulException {
TransactionState transactionState1 = makeTransaction1();
TransactionState transactionState2 = makeTransaction2();
transactionManager.tryCommit(transactionState1);
transactionManager.tryCommit(transactionState2);
}
public void TestTwoTransactionsWithConflict() throws IOException,
CommitUnsuccessfulException {
TransactionState transactionState1 = makeTransaction1();
TransactionState transactionState2 = makeTransaction2();
transactionManager.tryCommit(transactionState2);
try {
transactionManager.tryCommit(transactionState1);
fail();
} catch (CommitUnsuccessfulException e) {
// Good
}
}
// Read from ROW1,COL_A and put it in ROW2_COLA and ROW3_COLA
private TransactionState makeTransaction1() throws IOException {
TransactionState transactionState = transactionManager.beginTransaction();
Result row1_A = table.get(transactionState, new Get(ROW1).addColumn(COL_A));
table.put(new Put(ROW2).add(FAMILY, QUAL_A, row1_A.getValue(COL_A)));
table.put(new Put(ROW3).add(FAMILY, QUAL_A, row1_A.getValue(COL_A)));
return transactionState;
}
// Read ROW1,COL_A, increment its (integer) value, write back
private TransactionState makeTransaction2() throws IOException {
TransactionState transactionState = transactionManager.beginTransaction();
Result row1_A = table.get(transactionState, new Get(ROW1).addColumn(COL_A));
int value = Bytes.toInt(row1_A.getValue(COL_A));
table.put(new Put(ROW1).add(FAMILY, QUAL_A, Bytes.toBytes(value + 1)));
return transactionState;
}
}

View File

@ -0,0 +1,284 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
/** JUnit test case for HLog */
public class TestTHLog extends HBaseTestCase implements
HConstants {
private Path dir;
private MiniDFSCluster cluster;
final byte[] tableName = Bytes.toBytes("tablename");
final HTableDescriptor tableDesc = new HTableDescriptor(tableName);
final HRegionInfo regionInfo = new HRegionInfo(tableDesc,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
final byte[] row1 = Bytes.toBytes("row1");
final byte[] val1 = Bytes.toBytes("val1");
final byte[] row2 = Bytes.toBytes("row2");
final byte[] val2 = Bytes.toBytes("val2");
final byte[] row3 = Bytes.toBytes("row3");
final byte[] val3 = Bytes.toBytes("val3");
final byte[] family = Bytes.toBytes("family");
final byte[] column = Bytes.toBytes("a");
@Override
public void setUp() throws Exception {
cluster = new MiniDFSCluster(conf, 2, true, (String[]) null);
// Set the hbase.rootdir to be the home directory in mini dfs.
this.conf.set(HConstants.HBASE_DIR, this.cluster.getFileSystem()
.getHomeDirectory().toString());
super.setUp();
this.dir = new Path("/hbase", getName());
if (fs.exists(dir)) {
fs.delete(dir, true);
}
}
@Override
public void tearDown() throws Exception {
if (this.fs.exists(this.dir)) {
this.fs.delete(this.dir, true);
}
shutdownDfs(cluster);
super.tearDown();
}
/**
* @throws IOException
*/
public void testSingleCommit() throws IOException {
THLog log = new THLog(fs, dir, this.conf, null);
THLogRecoveryManager logRecoveryMangaer = new THLogRecoveryManager(fs,
regionInfo, conf);
// Write columns named 1, 2, 3, etc. and then values of single byte
// 1, 2, 3...
long transactionId = 1;
log.writeStartToLog(regionInfo, transactionId);
log.writeUpdateToLog(regionInfo, transactionId, new Put(row1).add(family,
column, val1));
log.writeUpdateToLog(regionInfo, transactionId, new Put(row2).add(family,
column, val2));
log.writeUpdateToLog(regionInfo, transactionId, new Put(row3).add(family,
column, val3));
log.writeCommitToLog(regionInfo, transactionId);
// log.completeCacheFlush(regionName, tableName, logSeqId);
log.close();
Path filename = log.computeFilename(log.getFilenum());
Map<Long, List<KeyValue>> commits = logRecoveryMangaer.getCommitsFromLog(
filename, -1, null);
assertEquals(1, commits.size());
assertTrue(commits.containsKey(transactionId));
assertEquals(3, commits.get(transactionId).size());
List<KeyValue> updates = commits.get(transactionId);
KeyValue update1 = updates.get(0);
assertTrue(Bytes.equals(row1, update1.getRow()));
assertTrue(Bytes.equals(val1, update1.getValue()));
KeyValue update2 = updates.get(1);
assertTrue(Bytes.equals(row2, update2.getRow()));
assertTrue(Bytes.equals(val2, update2.getValue()));
KeyValue update3 = updates.get(2);
assertTrue(Bytes.equals(row3, update3.getRow()));
assertTrue(Bytes.equals(val3, update3.getValue()));
}
/**
* @throws IOException
*/
public void testSingleAbort() throws IOException {
THLog log = new THLog(fs, dir, this.conf, null);
THLogRecoveryManager logRecoveryMangaer = new THLogRecoveryManager(fs,
regionInfo, conf);
long transactionId = 1;
log.writeStartToLog(regionInfo, transactionId);
log.writeUpdateToLog(regionInfo, transactionId, new Put(row1).add(family,
column, val1));
log.writeUpdateToLog(regionInfo, transactionId, new Put(row2).add(family,
column, val2));
log.writeUpdateToLog(regionInfo, transactionId, new Put(row3).add(family,
column, val3));
log.writeAbortToLog(regionInfo, transactionId);
// log.completeCacheFlush(regionName, tableName, logSeqId);
log.close();
Path filename = log.computeFilename(log.getFilenum());
Map<Long, List<KeyValue>> commits = logRecoveryMangaer.getCommitsFromLog(
filename, -1, null);
assertEquals(0, commits.size());
}
/**
* @throws IOException
*/
public void testInterlievedCommits() throws IOException {
THLog log = new THLog(fs, dir, this.conf, null);
THLogRecoveryManager logMangaer = new THLogRecoveryManager(fs, regionInfo,
conf);
long transaction1Id = 1;
long transaction2Id = 2;
log.writeStartToLog(regionInfo, transaction1Id);
log.writeUpdateToLog(regionInfo, transaction1Id, new Put(row1).add(family,
column, val1));
log.writeStartToLog(regionInfo, transaction2Id);
log.writeUpdateToLog(regionInfo, transaction2Id, new Put(row2).add(family,
column, val2));
log.writeUpdateToLog(regionInfo, transaction1Id, new Put(row3).add(family,
column, val3));
log.writeCommitToLog(regionInfo, transaction1Id);
log.writeCommitToLog(regionInfo, transaction2Id);
// log.completeCacheFlush(regionName, tableName, logSeqId);
log.close();
Path filename = log.computeFilename(log.getFilenum());
Map<Long, List<KeyValue>> commits = logMangaer.getCommitsFromLog(filename,
-1, null);
assertEquals(2, commits.size());
assertEquals(2, commits.get(transaction1Id).size());
assertEquals(1, commits.get(transaction2Id).size());
}
/**
* @throws IOException
*/
public void testInterlievedAbortCommit() throws IOException {
THLog log = new THLog(fs, dir, this.conf, null);
THLogRecoveryManager logMangaer = new THLogRecoveryManager(fs, regionInfo,
conf);
long transaction1Id = 1;
long transaction2Id = 2;
log.writeStartToLog(regionInfo, transaction1Id);
log.writeUpdateToLog(regionInfo, transaction1Id, new Put(row1).add(family,
column, val1));
log.writeStartToLog(regionInfo, transaction2Id);
log.writeUpdateToLog(regionInfo, transaction2Id, new Put(row2).add(family,
column, val2));
log.writeAbortToLog(regionInfo, transaction2Id);
log.writeUpdateToLog(regionInfo, transaction1Id, new Put(row3).add(family,
column, val3));
log.writeCommitToLog(regionInfo, transaction1Id);
// log.completeCacheFlush(regionName, tableName, logSeqId);
log.close();
Path filename = log.computeFilename(log.getFilenum());
Map<Long, List<KeyValue>> commits = logMangaer.getCommitsFromLog(filename,
-1, null);
assertEquals(1, commits.size());
assertEquals(2, commits.get(transaction1Id).size());
}
/**
* @throws IOException
*/
public void testInterlievedCommitAbort() throws IOException {
THLog log = new THLog(fs, dir, this.conf, null);
THLogRecoveryManager logMangaer = new THLogRecoveryManager(fs, regionInfo,
conf);
long transaction1Id = 1;
long transaction2Id = 2;
log.writeStartToLog(regionInfo, transaction1Id);
log.writeUpdateToLog(regionInfo, transaction1Id, new Put(row1).add(family,
column, val1));
log.writeStartToLog(regionInfo, transaction2Id);
log.writeUpdateToLog(regionInfo, transaction2Id, new Put(row2).add(family,
column, val2));
log.writeCommitToLog(regionInfo, transaction2Id);
log.writeUpdateToLog(regionInfo, transaction1Id, new Put(row3).add(family,
column, val3));
log.writeAbortToLog(regionInfo, transaction1Id);
// log.completeCacheFlush(regionName, tableName, logSeqId);
log.close();
Path filename = log.computeFilename(log.getFilenum());
Map<Long, List<KeyValue>> commits = logMangaer.getCommitsFromLog(filename,
-1, null);
assertEquals(1, commits.size());
assertEquals(1, commits.get(transaction2Id).size());
}
// FIXME Cannot do this test without a global transacton manager
// public void testMissingCommit() {
// fail();
// }
// FIXME Cannot do this test without a global transacton manager
// public void testMissingAbort() {
// fail();
// }
}

View File

@ -0,0 +1,290 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.transactional;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
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.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.transactional.CommitUnsuccessfulException;
import org.apache.hadoop.hbase.client.transactional.HBaseBackedTransactionLogger;
import org.apache.hadoop.hbase.client.transactional.TransactionManager;
import org.apache.hadoop.hbase.client.transactional.TransactionState;
import org.apache.hadoop.hbase.client.transactional.TransactionalTable;
import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
public class TestTHLogRecovery extends HBaseClusterTestCase {
private static final Log LOG = LogFactory.getLog(TestTHLogRecovery.class);
private static final String TABLE_NAME = "table1";
private static final byte[] FAMILY_COLON = Bytes.toBytes("family:");
private static final byte[] FAMILY = Bytes.toBytes("family");
private static final byte[] QUAL_A = Bytes.toBytes("a");
private static final byte[] COL_A = Bytes.toBytes("family:a");
private static final byte[] ROW1 = Bytes.toBytes("row1");
private static final byte[] ROW2 = Bytes.toBytes("row2");
private static final byte[] ROW3 = Bytes.toBytes("row3");
private static final int TOTAL_VALUE = 10;
private HBaseAdmin admin;
private TransactionManager transactionManager;
private TransactionalTable table;
/** constructor */
public TestTHLogRecovery() {
super(2, false);
conf.set(HConstants.REGION_SERVER_CLASS, TransactionalRegionInterface.class
.getName());
conf.set(HConstants.REGION_SERVER_IMPL, TransactionalRegionServer.class
.getName());
// Set flush params so we don't get any
// FIXME (defaults are probably fine)
// Copied from TestRegionServerExit
conf.setInt("ipc.client.connect.max.retries", 5); // reduce ipc retries
conf.setInt("ipc.client.timeout", 10000); // and ipc timeout
conf.setInt("hbase.client.pause", 10000); // increase client timeout
conf.setInt("hbase.client.retries.number", 10); // increase HBase retries
}
@Override
protected void setUp() throws Exception {
FileSystem.getLocal(conf).delete(new Path(conf.get(HConstants.HBASE_DIR)),
true);
super.setUp();
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
desc.addFamily(new HColumnDescriptor(FAMILY_COLON));
admin = new HBaseAdmin(conf);
admin.createTable(desc);
table = new TransactionalTable(conf, desc.getName());
HBaseBackedTransactionLogger.createTable();
transactionManager = new TransactionManager(
new HBaseBackedTransactionLogger(), conf);
writeInitalRows();
}
private void writeInitalRows() throws IOException {
table.put(new Put(ROW1).add(FAMILY, QUAL_A, Bytes.toBytes(TOTAL_VALUE)));
table.put(new Put(ROW2).add(FAMILY, QUAL_A, Bytes.toBytes(0)));
table.put(new Put(ROW3).add(FAMILY, QUAL_A, Bytes.toBytes(0)));
}
public void testWithoutFlush() throws IOException,
CommitUnsuccessfulException {
writeInitalRows();
TransactionState state1 = makeTransaction(false);
transactionManager.tryCommit(state1);
stopOrAbortRegionServer(true);
Thread t = startVerificationThread(1);
t.start();
threadDumpingJoin(t);
}
public void testWithFlushBeforeCommit() throws IOException,
CommitUnsuccessfulException {
writeInitalRows();
TransactionState state1 = makeTransaction(false);
flushRegionServer();
transactionManager.tryCommit(state1);
stopOrAbortRegionServer(true);
Thread t = startVerificationThread(1);
t.start();
threadDumpingJoin(t);
}
// FIXME, TODO
// public void testWithFlushBetweenTransactionWrites() {
// fail();
// }
private void flushRegionServer() {
List<LocalHBaseCluster.RegionServerThread> regionThreads = cluster
.getRegionThreads();
HRegion region = null;
int server = -1;
for (int i = 0; i < regionThreads.size() && server == -1; i++) {
HRegionServer s = regionThreads.get(i).getRegionServer();
Collection<HRegion> regions = s.getOnlineRegions();
for (HRegion r : regions) {
if (Bytes.equals(r.getTableDesc().getName(), Bytes.toBytes(TABLE_NAME))) {
server = i;
region = r;
}
}
}
if (server == -1) {
LOG.fatal("could not find region server serving table region");
fail();
}
((TransactionalRegionServer) regionThreads.get(server).getRegionServer())
.getFlushRequester().request(region);
}
/**
* Stop the region server serving TABLE_NAME.
*
* @param abort set to true if region server should be aborted, if false it is
* just shut down.
*/
private void stopOrAbortRegionServer(final boolean abort) {
List<LocalHBaseCluster.RegionServerThread> regionThreads = cluster
.getRegionThreads();
int server = -1;
for (int i = 0; i < regionThreads.size(); i++) {
HRegionServer s = regionThreads.get(i).getRegionServer();
Collection<HRegion> regions = s.getOnlineRegions();
LOG.info("server: " + regionThreads.get(i).getName());
for (HRegion r : regions) {
LOG.info("region: " + r.getRegionInfo().getRegionNameAsString());
if (Bytes.equals(r.getTableDesc().getName(), Bytes.toBytes(TABLE_NAME))) {
server = i;
}
}
}
if (server == -1) {
LOG.fatal("could not find region server serving table region");
fail();
}
if (abort) {
this.cluster.abortRegionServer(server);
} else {
this.cluster.stopRegionServer(server, false);
}
LOG.info(this.cluster.waitOnRegionServer(server) + " has been "
+ (abort ? "aborted" : "shut down"));
}
private void verify(final int numRuns) throws IOException {
// Reads
int row1 = Bytes.toInt(table.get(new Get(ROW1).addColumn(COL_A)).getValue(
COL_A));
int row2 = Bytes.toInt(table.get(new Get(ROW2).addColumn(COL_A)).getValue(
COL_A));
int row3 = Bytes.toInt(table.get(new Get(ROW3).addColumn(COL_A)).getValue(
COL_A));
assertEquals(TOTAL_VALUE - 2 * numRuns, row1);
assertEquals(numRuns, row2);
assertEquals(numRuns, row3);
}
// Move 2 out of ROW1 and 1 into ROW2 and 1 into ROW3
private TransactionState makeTransaction(final boolean flushMidWay)
throws IOException {
TransactionState transactionState = transactionManager.beginTransaction();
// Reads
int row1 = Bytes.toInt(table.get(transactionState,
new Get(ROW1).addColumn(COL_A)).getValue(COL_A));
int row2 = Bytes.toInt(table.get(transactionState,
new Get(ROW2).addColumn(COL_A)).getValue(COL_A));
int row3 = Bytes.toInt(table.get(transactionState,
new Get(ROW3).addColumn(COL_A)).getValue(COL_A));
row1 -= 2;
row2 += 1;
row3 += 1;
if (flushMidWay) {
flushRegionServer();
}
// Writes
Put write = new Put(ROW1);
write.add(FAMILY, QUAL_A, Bytes.toBytes(row1));
table.put(transactionState, write);
write = new Put(ROW2);
write.add(FAMILY, QUAL_A, Bytes.toBytes(row2));
table.put(transactionState, write);
write = new Put(ROW3);
write.add(FAMILY, QUAL_A, Bytes.toBytes(row3));
table.put(transactionState, write);
return transactionState;
}
/*
* Run verification in a thread so I can concurrently run a thread-dumper
* while we're waiting (because in this test sometimes the meta scanner looks
* to be be stuck). @param tableName Name of table to find. @param row Row we
* expect to find. @return Verification thread. Caller needs to calls start on
* it.
*/
private Thread startVerificationThread(final int numRuns) {
Runnable runnable = new Runnable() {
public void run() {
try {
// Now try to open a scanner on the meta table. Should stall until
// meta server comes back up.
HTable t = new HTable(conf, TABLE_NAME);
Scan s = new Scan();
s.addColumn(FAMILY, QUAL_A);
ResultScanner scanner = t.getScanner(s);
scanner.close();
} catch (IOException e) {
LOG.fatal("could not re-open meta table because", e);
fail();
}
try {
verify(numRuns);
LOG.info("Success!");
} catch (Exception e) {
e.printStackTrace();
fail();
}
}
};
return new Thread(runnable);
}
}

View File

@ -31,7 +31,6 @@ import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.fs.Path;
//import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
@ -52,6 +51,7 @@ ISerializable {
// Changes prior to version 3 were not recorded here.
// Version 3 adds metadata as a map where keys and values are byte[].
// Version 4 adds indexes
// FIXME version 5 should remove indexes
public static final byte TABLE_DESCRIPTOR_VERSION = 4;
private byte [] name = HConstants.EMPTY_BYTE_ARRAY;
@ -104,13 +104,7 @@ ISerializable {
// Key is hash of the family name.
public final Map<byte [], HColumnDescriptor> families =
new TreeMap<byte [], HColumnDescriptor>(KeyValue.FAMILY_COMPARATOR);
// private final Map<byte [], HColumnDescriptor> families =
// new TreeMap<byte [], HColumnDescriptor>(KeyValue.FAMILY_COMPARATOR);
// Key is indexId
// private final Map<String, IndexSpecification> indexes =
// new HashMap<String, IndexSpecification>();
/**
* Private constructor used internally creating table descriptors for
* catalog tables: e.g. .META. and -ROOT-.
@ -129,23 +123,6 @@ ISerializable {
* Private constructor used internally creating table descriptors for
* catalog tables: e.g. .META. and -ROOT-.
*/
// protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families,
// Collection<IndexSpecification> indexes,
// Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
// this.name = name.clone();
// this.nameAsString = Bytes.toString(this.name);
// setMetaFlags(name);
// for(HColumnDescriptor descriptor : families) {
// this.families.put(descriptor.getName(), descriptor);
// }
// for(IndexSpecification index : indexes) {
// this.indexes.put(index.getIndexId(), index);
// }
// for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> entry:
// values.entrySet()) {
// this.values.put(entry.getKey(), entry.getValue());
// }
// }
protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families,
Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
this.name = name.clone();
@ -216,7 +193,6 @@ ISerializable {
desc.values.entrySet()) {
this.values.put(e.getKey(), e.getValue());
}
// this.indexes.putAll(desc.indexes);
}
/*
@ -454,18 +430,6 @@ ISerializable {
setValue(MEMSTORE_FLUSHSIZE_KEY,
Bytes.toBytes(Integer.toString(memstoreFlushSize)));
}
// public Collection<IndexSpecification> getIndexes() {
// return indexes.values();
// }
//
// public IndexSpecification getIndex(String indexId) {
// return indexes.get(indexId);
// }
//
// public void addIndex(IndexSpecification index) {
// indexes.put(index.getIndexId(), index);
// }
/**
* Adds a column family.
@ -524,13 +488,6 @@ ISerializable {
s.append(FAMILIES);
s.append(" => ");
s.append(families.values());
// if (!indexes.isEmpty()) {
// // Don't emit if empty. Has to do w/ transactional hbase.
// s.append(", ");
// s.append("INDEXES");
// s.append(" => ");
// s.append(indexes.values());
// }
s.append('}');
return s.toString();
}
@ -595,16 +552,9 @@ ISerializable {
c.readFields(in);
families.put(c.getName(), c);
}
// indexes.clear();
if (version < 4) {
return;
}
// int numIndexes = in.readInt();
// for (int i = 0; i < numIndexes; i++) {
// IndexSpecification index = new IndexSpecification();
// index.readFields(in);
// addIndex(index);
// }
}
public void write(DataOutput out) throws IOException {
@ -624,10 +574,6 @@ ISerializable {
HColumnDescriptor family = it.next();
family.write(out);
}
// out.writeInt(indexes.size());
// for(IndexSpecification index : indexes.values()) {
// index.write(out);
// }
}
// Comparable

View File

@ -194,18 +194,10 @@ public class KeyValue implements Writable, HeapSize {
* @return Type associated with passed code.
*/
public static Type codeToType(final byte b) {
// This is messy repeating each type here below but no way around it; we
// can't use the enum ordinal.
if (b == Put.getCode()) {
return Put;
} else if (b == Delete.getCode()) {
return Delete;
} else if (b == DeleteColumn.getCode()) {
return DeleteColumn;
} else if (b == DeleteFamily.getCode()) {
return DeleteFamily;
} else if (b == Maximum.getCode()) {
return Maximum;
for (Type t : Type.values()) {
if (t.getCode() == b) {
return t;
}
}
throw new RuntimeException("Unknown code " + b);
}

View File

@ -1805,7 +1805,7 @@ public class HTable {
// respect.
}
protected void initialize() throws IOException {
public void initialize() throws IOException {
nextScanner(this.scannerCaching);
}

View File

@ -37,9 +37,6 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor {
* Create an unmodifyable copy of an HTableDescriptor
* @param desc
*/
// UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
// super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues());
// }
UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
super(desc.getName(), getUnmodifyableFamilies(desc), desc.getValues());
}

View File

@ -1,199 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
/**
* Interface for transactional region servers.
*
* <p>NOTE: if you change the interface, you must change the RPC version
* number in HBaseRPCProtocolVersion
*
*/
public interface TransactionalRegionInterface extends HRegionInterface {
/**
* Sent to initiate a transaction.
*
* @param transactionId
* @param regionName name of region
* @throws IOException
*/
public void beginTransaction(long transactionId, final byte[] regionName)
throws IOException;
/**
* Retrieve a single value from the specified region for the specified row and
* column keys
*
* @param transactionId
* @param regionName name of region
* @param row row key
* @param column column key
* @return alue for that region/row/column
* @throws IOException
*/
public Cell get(long transactionId, final byte[] regionName,
final byte[] row, final byte[] column) throws IOException;
/**
* Get the specified number of versions of the specified row and column
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param column column key
* @param numVersions number of versions to return
* @return array of values
* @throws IOException
*/
public Cell[] get(long transactionId, final byte[] regionName,
final byte[] row, final byte[] column, final int numVersions)
throws IOException;
/**
* Get the specified number of versions of the specified row and column with
* the specified timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param column column key
* @param timestamp timestamp
* @param numVersions number of versions to return
* @return array of values
* @throws IOException
*/
public Cell[] get(long transactionId, final byte[] regionName,
final byte[] row, final byte[] column, final long timestamp,
final int numVersions) throws IOException;
/**
* Get all the data for the specified row at a given timestamp
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param ts timestamp
* @return map of values
* @throws IOException
*/
public RowResult getRow(long transactionId, final byte[] regionName,
final byte[] row, final long ts) throws IOException;
/**
* Get selected columns for the specified row at a given timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param columns colums to get
* @param ts timestamp
* @return map of values
* @throws IOException
*/
public RowResult getRow(long transactionId, final byte[] regionName,
final byte[] row, final byte[][] columns, final long ts)
throws IOException;
/**
* Get selected columns for the specified row at the latest timestamp.
*
* @param transactionId
* @param regionName region name
* @param row row key
* @param columns columns to get
* @return map of values
* @throws IOException
*/
public RowResult getRow(long transactionId, final byte[] regionName,
final byte[] row, final byte[][] columns) throws IOException;
/**
* Delete all cells that match the passed row and whose timestamp is equal-to
* or older than the passed timestamp.
*
* @param transactionId
* @param regionName region name
* @param delete
* @throws IOException
*/
public void delete(long transactionId, byte [] regionName, Delete delete)
throws IOException;
/**
* Opens a remote scanner with a RowFilter.
*
* @param transactionId
* @param regionName name of region to scan
* @param scan
* @return scannerId scanner identifier used in other calls
* @throws IOException
*/
public long openScanner(final long transactionId, final byte[] regionName,
Scan scan) throws IOException;
/**
* Applies a batch of updates via one RPC
*
* @param transactionId
* @param regionName name of the region to update
* @param b BatchUpdate
* @throws IOException
*/
public void batchUpdate(long transactionId, final byte[] regionName,
final BatchUpdate b) throws IOException;
/**
* Ask if we can commit the given transaction.
*
* @param regionName
* @param transactionId
* @return true if we can commit
* @throws IOException
*/
public boolean commitRequest(final byte[] regionName, long transactionId)
throws IOException;
/**
* Commit the transaction.
*
* @param regionName
* @param transactionId
* @throws IOException
*/
public void commit(final byte[] regionName, long transactionId)
throws IOException;
/**
* Abort the transaction.
*
* @param regionName
* @param transactionId
* @throws IOException
*/
public void abort(final byte[] regionName, long transactionId)
throws IOException;
}

View File

@ -290,12 +290,7 @@ public class HLog implements HConstants, Syncable {
Path oldFile = cleanupCurrentWriter(this.filenum);
this.filenum = System.currentTimeMillis();
Path newPath = computeFilename(this.filenum);
this.writer = SequenceFile.createWriter(this.fs, this.conf, newPath,
HLogKey.class, KeyValue.class,
fs.getConf().getInt("io.file.buffer.size", 4096),
fs.getDefaultReplication(), this.blocksize,
SequenceFile.CompressionType.NONE, new DefaultCodec(), null,
new Metadata());
this.writer = createWriter(newPath);
LOG.info((oldFile != null?
"Roll " + FSUtils.getPath(oldFile) + ", entries=" +
this.numEntries.get() +
@ -326,6 +321,20 @@ public class HLog implements HConstants, Syncable {
}
return regionToFlush;
}
protected SequenceFile.Writer createWriter(Path path) throws IOException {
return createWriter(path, HLogKey.class, KeyValue.class);
}
protected SequenceFile.Writer createWriter(Path path,
Class<? extends HLogKey> keyClass, Class<? extends KeyValue> valueClass)
throws IOException {
return SequenceFile.createWriter(this.fs, this.conf, path, keyClass,
valueClass, fs.getConf().getInt("io.file.buffer.size", 4096), fs
.getDefaultReplication(), this.blocksize,
SequenceFile.CompressionType.NONE, new DefaultCodec(), null,
new Metadata());
}
/*
* Clean up old commit logs.
@ -463,19 +472,35 @@ public class HLog implements HConstants, Syncable {
}
}
/** Append an entry without a row to the log.
/** Append an entry to the log.
*
* @param regionInfo
* @param row
* @param logEdit
* @param now
* @param now Time of this edit write.
* @throws IOException
*/
public void append(HRegionInfo regionInfo, KeyValue logEdit, final long now)
public void append(HRegionInfo regionInfo, KeyValue logEdit,
final long now)
throws IOException {
this.append(regionInfo, new byte[0], logEdit, now);
byte [] regionName = regionInfo.getRegionName();
byte [] tableName = regionInfo.getTableDesc().getName();
this.append(regionInfo, makeKey(regionName, tableName, -1, now), logEdit);
}
/** Construct a new log key.
*
* @param now
* @param regionName
* @param tableName
* @return
*/
protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqnum, long now) {
return new HLogKey(regionName, tableName, seqnum, now);
}
/** Append an entry to the log.
*
* @param regionInfo
@ -484,24 +509,22 @@ public class HLog implements HConstants, Syncable {
* @param now Time of this edit write.
* @throws IOException
*/
public void append(HRegionInfo regionInfo, byte [] row, KeyValue logEdit,
final long now)
public void append(HRegionInfo regionInfo, HLogKey logKey, KeyValue logEdit)
throws IOException {
if (this.closed) {
throw new IOException("Cannot append; log is closed");
}
byte [] regionName = regionInfo.getRegionName();
byte [] tableName = regionInfo.getTableDesc().getName();
synchronized (updateLock) {
long seqNum = obtainSeqNum();
logKey.setLogSeqNum(seqNum);
// The 'lastSeqWritten' map holds the sequence number of the oldest
// write for each region. When the cache is flushed, the entry for the
// region being flushed is removed if the sequence number of the flush
// is greater than or equal to the value in lastSeqWritten.
this.lastSeqWritten.putIfAbsent(regionName, Long.valueOf(seqNum));
HLogKey logKey = new HLogKey(regionName, tableName, seqNum, now);
boolean sync = regionInfo.isMetaRegion() || regionInfo.isRootRegion();
doWrite(logKey, logEdit, sync, now);
doWrite(logKey, logEdit, sync, logKey.getWriteTime());
this.numEntries.incrementAndGet();
updateLock.notifyAll();
}
@ -536,7 +559,7 @@ public class HLog implements HConstants, Syncable {
* @param now
* @throws IOException
*/
void append(byte [] regionName, byte [] tableName, List<KeyValue> edits,
public void append(byte [] regionName, byte [] tableName, List<KeyValue> edits,
boolean sync, final long now)
throws IOException {
if (this.closed) {
@ -551,8 +574,7 @@ public class HLog implements HConstants, Syncable {
this.lastSeqWritten.putIfAbsent(regionName, Long.valueOf(seqNum[0]));
int counter = 0;
for (KeyValue kv: edits) {
HLogKey logKey =
new HLogKey(regionName, tableName, seqNum[counter++], now);
HLogKey logKey = makeKey(regionName, tableName, seqNum[counter++], now);
doWrite(logKey, kv, sync, now);
this.numEntries.incrementAndGet();
}
@ -686,8 +708,8 @@ public class HLog implements HConstants, Syncable {
return;
}
synchronized (updateLock) {
this.writer.append(new HLogKey(regionName, tableName, logSeqId,
System.currentTimeMillis()), completeCacheFlushLogEdit());
this.writer.append(makeKey(regionName, tableName, logSeqId, System.currentTimeMillis()),
completeCacheFlushLogEdit());
this.numEntries.incrementAndGet();
Long seq = this.lastSeqWritten.get(regionName);
if (seq != null && logSeqId >= seq.longValue()) {

View File

@ -87,6 +87,10 @@ public class HLogKey implements WritableComparable<HLogKey>, HeapSize {
public long getLogSeqNum() {
return logSeqNum;
}
void setLogSeqNum(long logSeqNum) {
this.logSeqNum = logSeqNum;
}
/**
* @return the write time

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@ -311,6 +312,8 @@ public class HRegion implements HConstants { // , Writable{
}
}
// Play log if one. Delete when done.
doReconstructionLog(oldLogFile, minSeqId, maxSeqId, reporter);
if (fs.exists(oldLogFile)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Deleting old log file: " + oldLogFile);
@ -419,7 +422,7 @@ public class HRegion implements HConstants { // , Writable{
*
* @throws IOException
*/
List<StoreFile> close(final boolean abort) throws IOException {
public List<StoreFile> close(final boolean abort) throws IOException {
if (isClosed()) {
LOG.warn("region " + this + " already closed");
return null;
@ -578,6 +581,7 @@ public class HRegion implements HConstants { // , Writable{
* @throws IOException
*/
HRegion [] splitRegion(final byte [] splitRow) throws IOException {
prepareToSplit();
synchronized (splitLock) {
if (closed.get()) {
return null;
@ -666,6 +670,10 @@ public class HRegion implements HConstants { // , Writable{
}
}
protected void prepareToSplit() {
// nothing
}
/*
* @param dir
* @return compaction directory for the passed in <code>dir</code>
@ -1476,6 +1484,13 @@ public class HRegion implements HConstants { // , Writable{
private boolean isFlushSize(final long size) {
return size > this.memstoreFlushSize;
}
// Do any reconstruction needed from the log
protected void doReconstructionLog(Path oldLogFile, long minSeqId, long maxSeqId,
Progressable reporter)
throws UnsupportedEncodingException, IOException {
// Nothing to do (Replaying is done in HStores)
}
protected Store instantiateHStore(Path baseDir,
HColumnDescriptor c, Path oldLogFile, Progressable reporter)

View File

@ -1046,10 +1046,24 @@ public class HRegionServer implements HConstants, HRegionInterface,
"running at " + this.serverInfo.getServerAddress().toString() +
" because logdir " + logdir.toString() + " exists");
}
HLog newlog = instantiateHLog(logdir);
return newlog;
}
// instantiate
protected HLog instantiateHLog(Path logdir) throws IOException {
HLog newlog = new HLog(fs, logdir, conf, hlogRoller);
return newlog;
}
protected LogRoller getLogRoller() {
return hlogRoller;
}
/*
* @param interval Interval since last time metrics were called.
*/
protected void doMetrics() {
try {
metrics();
@ -1580,7 +1594,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
getOutboundMsgs().add(new HMsg(HMsg.Type.MSG_REPORT_PROCESS_OPEN, hri));
}
void closeRegion(final HRegionInfo hri, final boolean reportWhenCompleted)
protected void closeRegion(final HRegionInfo hri, final boolean reportWhenCompleted)
throws IOException {
HRegion region = this.removeFromOnlineRegions(hri);
if (region != null) {