HBASE-4737 Categorize the tests into small/medium/large; allow small tests to be run

in parallel within a single JVM (N Keywal)


git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1199604 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2011-11-09 03:38:10 +00:00
parent 007e7dc183
commit 1f2aadb80d
128 changed files with 882 additions and 443 deletions

View File

@ -34,6 +34,8 @@ Release 0.93.0 - Unreleased
HBASE-1744 Thrift server to match the new java api (Tim Sell)
HBASE-4746 Use a random ZK client port in unit tests so we can run them in parallel
(Mikhail Bautin)
HBASE-4737 Categorize the tests into small/medium/large; allow small tests to be run
in parallel within a single JVM (N Keywal)
BUG FIXES
HBASE-4488 Store could miss rows during flush (Lars H via jgray)

400
dev-support/hbasetests.sh Normal file
View File

@ -0,0 +1,400 @@
#!/usr/bin/env bash
#
# This script:
# - analyse the content of the .java test file to split them between
# small/medium/large
# - launch the small tests in a single maven, with surefire
# parallelisation activated
# - launch the medium & large in two maven, parallelized
# - the flaky tests are run at the end, not parallelized
# - present a small report of the global results
# - copy the failed test reports with prefix 'fail_' and a timestamp
# to protect them from a later deletion by maven
# - if configured for, relaunch the tests in errors
#
#
# Caveats:
# - multiple maven are launch, hence there can be recompilation
# between the tests if a file is modified. For non flaky tests and
# parallelization, the frame is the time to execute the small tests,
# so it's around 4 minutes.
# - Note that surefire is buggy, and the results presented while
# running may be wrong. For example, it can says that a class tests
# have 5 errors. When you look at the file it wrote, it says that the
# 2 tests are ok, and in the class there are actually two tests
# methods, not five. If you generate the report at the end with
# surefire-report it's fine however.
#
######################################### parameters
#set to 0 to run only developpers tests (small & medium categories)
runAllTests=0
#set to 1 to replay the failed tests. Previous reports are kept in
# fail_ files
replayFailed=0
#set to 0 to run all medium & large tests in a single maven operation
# instead of two
parallelMaven=1
#harcoded list of tests that often fail. We don't want to add any
# complexity around then so there are not run in parallel but after
# the others
#The ',' at the end is mandatory
flakyTests=
#org.apache.hadoop.hbase.mapreduce.TestTableInputFormatScan,org.apache.hadoop.hbase.catalog.TestMetaReaderEditorNoCluster,org.apache.hadoop.hbase.catalog.TestMetaReaderEditor,org.apache.hadoop.hbase.mapreduce.TestHFileOutputFormat,org.apache.hadoop.hbase.mapred.TestTableMapReduce,org.apache.hadoop.hbase.coprocessor.TestMasterCoprocessorExceptionWithAbort,org.apache.hadoop.hbase.coprocessor.TestMasterCoprocessorExceptionWithRemove,org.apache.hadoop.hbase.client.TestAdmin,org.apache.hadoop.hbase.master.TestMasterFailover,org.apache.hadoop.hbase.regionserver.wal.TestLogRolling,org.apache.hadoop.hbase.master.TestDistributedLogSplitting,org.apache.hadoop.hbase.master.TestMasterRestartAfterDisablingTable,org.apache.hadoop.hbase.TestGlobalMemStoreSize,
######################################### Internal parameters
#directory used for surefire & the source code.
#They should not need to be modified
#The final / is mandatory
rootTestClassDirectory="./src/test/java/"
surefireReportDirectory="./target/surefire-reports/"
#variable to use to debug the script without launching the tests
mvnCommand=mvn
#mvnCommand=echo
######################################### Functions
#get the list of the process considered as dead
# i.e.: in the same group as the script and with a ppid of 1
# We do this because surefire can leave some dead process, so
# we will jstack them and kill them
function createListDeadProcess {
id=$$
listDeadProcess=""
#list of the process with a ppid of 1
sonProcess=`ps -o pid= --ppid 1`
#then the process with a pgid of the script
for pId in $sonProcess
do
pgid=`ps -o pgid= --pid $pId | sed 's/ //g'`
if [ "$pgid" == "$id" ]
then
listDeadProcess="$pId $listDeadProcess"
fi
done
}
#kill the java sub process, if any, with a kill and a kill -9
#When maven/surefire fails, it lefts some process with a ppid==1
#we're going to find them with the pgid, print the stack and kill them.
function cleanProcess {
id=$$
createListDeadProcess
for pId in $listDeadProcess
do
echo "$pId survived, I will kill if it's a java process. 'ps' says:"
ps -fj --pid $pId
name=`ps -o comm= --pid $pId`
if [ "$name" == "java" ]
then
echo "$pId, java sub process of $id, is still running, killing it with a standard kill"
echo "Stack for $pId before kill:"
jstack -F -l $pId
kill $pId
echo "kill sent, waiting for 30 seconds"
sleep 30
son=`ps -o pid= --pid $pId | wc -l`
if (test $son -gt 0)
then
echo "$pId, java sub process of $id, is still running after a nice kill, using kill -9 now"
echo "Stack for $pId before kill -9:"
jstack -F -l $pId
kill -9 $pId
echo "Process $pId killed by kill -9"
else
echo "Process $pId killed by standard kill -9"
fi
else
echo "$pId is not a java process (it's $name), I don't kill it."
fi
done
createListDeadProcess
if (test ${#listDeadProcess} -gt 0)
then
echo "There are still $sonProcess for process $id left."
else
echo "Process $id clean, no son process left"
fi
}
#count the number of ',' in a string
# used to calculate the number of class
#write $count
function countClasses {
cars=`echo $1 | sed 's/[^,]//g' | wc -c `
count=$((cars - 1))
}
######################################### script
echo "Starting Script. Possible parameters are: runAllTests, replayFailed, nonParallelMaven"
echo "Other parameters are sent to maven"
#We will use this value at the end to calculate the execution time
startTime=`date +%s`
#look in the arguments if we override default values
for arg in "$@"
do
if [ $arg == "runAllTests" ]
then
runAllTests=1
else
if [ $arg == "replayFailed" ]
then
replayFailed=1
else
if [ $arg == "nonParallelMaven" ]
then
parallelMaven=0
else
args=$args" $arg"
fi
fi
fi
done
testsList=$(find $rootTestClassDirectory -name "Test*.java")
#for all java test files, let see if they contain the pattern
# to recognize the category
for testFile in $testsList
do
lenPath=$((${#rootTestClassDirectory}))
len=$((${#testFile} - $lenPath - 5)) # len(".java") == 5
shortTestFile=${testFile:lenPath:$len}
testName=$(echo $shortTestFile | sed 's/\//\./g')
#The ',' is used in the grep pattern as we don't want to catch
# partial name
isFlaky=$((`echo $flakyTests | grep "$testName," | wc -l`))
if (test $isFlaky -eq 0)
then
isSmall=0
isMedium=0
isLarge=0
# determine the category of the test by greping into the source code
isMedium=`grep "@Category" $testFile | grep "MediumTests.class" | wc -l`
if (test $isMedium -eq 0)
then
isLarge=`grep "@Category" $testFile | grep "LargeTests.class" | wc -l`
if (test $isLarge -eq 0)
then
isSmall=1
#sanity check on small tests
isStrange=`grep "\.startMini" $testFile | wc -l`
if (test $isStrange -gt 0)
then
echo "$testFile is categorized as 'small' but contains a .startMini string. Keep it as small anyway"
fi
fi
fi
#put the test in the right list
if (test $isSmall -gt 0)
then
smallList="$smallList,$testName"
fi
if (test $isMedium -gt 0)
then
mediumList="$mediumList,$testName"
fi
if (test $isLarge -gt 0)
then
largeList="$largeList,$testName"
fi
fi
done
#remove the ',' at the beginning
smallList=${smallList:1:${#smallList}}
mediumList=${mediumList:1:${#mediumList}}
largeList=${largeList:1:${#largeList}}
countClasses $smallList
echo "There are $count small tests"
countClasses $mediumList
echo "There are $count medium tests"
countClasses $largeList
echo "There are $count large tests"
#do we launch only dev or all tests?
if (test $runAllTests -eq 1)
then
echo "Running all tests, small, medium and large"
longList="$mediumList,$largeList"
else
echo "Running developper tests only, small and medium categories"
longList=$mediumList
fi
#medium and large test can be run in //, so we're
#going to create two lists
nextList=1
for testClass in `echo $longList | sed 's/,/ /g'`
do
if (test $nextList -eq 1)
then
nextList=2
runList1=$runList1,$testClass
else
nextList=1
runList2=$runList2,$testClass
fi
done
#remove the ',' at the beginning
runList1=${runList1:1:${#runList1}}
runList2=${runList2:1:${#runList2}}
#now we can run the tests, at last!
echo "Running small tests with one maven instance, in parallel"
$mvnCommand -P parallelTests test -Dtest=$smallList $args
cleanProcess
exeTime=$(((`date +%s` - $startTime)/60))
echo "Small tests executed after $exeTime minutes"
if (test $parallelMaven -gt 0)
then
echo "Running tests with two maven instances in parallel"
$mvnCommand -P nonParallelTests test -Dtest=$runList1 $args &
#give some time to the fist process if there is anything to compile
sleep 30
$mvnCommand -P nonParallelTests test -Dtest=$runList2 $args
#wait for forked process to finish
wait
cleanProcess
exeTime=$(((`date +%s` - $startTime)/60))
echo "Medium and large (if selected) tests executed after $exeTime minutes"
#now the flaky tests, alone, if the list is not empty
# we test on size greater then 5 to remove any "," effect
if (test $runAllTests -eq 1 && test ${#flakyTests} -gt 5)
then
echo "Running flaky tests"
$mvnCommand -P nonParallelTests test -Dtest=$flakyTests $args
cleanProcess
exeTime=$(((`date +%s` - $startTime)/60))
echo "Flaky tests executed after $exeTime minutes"
fi
else
echo "Running tests with a single maven instance, no parallelization"
$mvnCommand -P nonParallelTests test -Dtest=$runList1,$runList2,$flakyTests $args
cleanProcess
exeTime=$(((`date +%s` - $startTime)/60))
echo "Single maven instance tests executed after $exeTime minutes"
fi
#let's analyze the results
fullRunList="$smallList,$longList"
if (test $runAllTests -eq 1)
then
fullRunList="$fullRunList,$flakyTests"
fi
#single timestamp to ensure files uniquess.
timestamp=`date +%s`
#some counters, initialized because they may not be touched
# in the loop
errorCounter=0
sucessCounter=0
notFinishedCounter=0
for testClass in `echo $fullRunList | sed 's/,/ /g'`
do
reportFile=$surefireReportDirectory/$testClass.txt
outputReportFile=$surefireReportDirectory/$testClass-output.txt
if [ -s $reportFile ];
then
isError=`grep FAILURE $reportFile | wc -l`
if (test $isError -gt 0)
then
errorList="$errorList,$testClass"
errorCounter=$(($errorCounter + 1))
#let's copy the files if we want to use it later
cp $reportFile "$surefireReportDirectory/fail_$timestamp.$testClass.txt"
if [ -s $reportFile ];
then
cp $outputReportFile "$surefireReportDirectory/fail_$timestamp.$testClass"-output.txt""
fi
else
sucessCounter=$(($sucessCounter +1))
fi
else
#report file does not exist or is empty => the test didn't finish
notFinishedCounter=$(($notFinishedCounter + 1))
notFinishedList="$notFinishedList,$testClass"
fi
done
#list of all tests that failed
replayList="$notFinishedList""$errorList"
#remove the ',' at the beginning
notFinishedList=${notFinishedList:1:${#notFinishedList}}
errorList=${errorList:1:${#errorList}}
replayList=${replayList:1:${#replayList}}
#make it simpler to read by removing the org.* stuff from the name
notFinishedPresList=`echo $notFinishedList | sed 's/org.apache.hadoop.hbase.//g' | sed 's/,/, /g'`
errorPresList=`echo $errorList | sed 's/org.apache.hadoop.hbase.//g' | sed 's/,/, /g'`
#calculate the execution time
curTime=`date +%s`
exeTime=$((($curTime - $startTime)/60))
echo "##########################"
echo "$sucessCounter tests executed successfully"
echo "$errorCounter tests are in error"
echo "$notFinishedCounter tests didn't finish"
echo
echo "Tests in error are: $errorPresList"
echo "Tests that didn't finish are: $notFinishedPresList"
echo
echo "Execution time in minutes: $exeTime"
echo "##########################"
if (test ${#replayList} -gt 0)
then
if (test $replayFailed -gt 0)
then
echo "Replaying all tests that failed"
$mvnCommand -P nonParallelTests test -Dtest=$replayList $args
echo "Replaying done"
fi
fi
exit

View File

@ -0,0 +1,38 @@
/*
* Copyright 2011 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;
/**
* Tag a test as 'large', meaning that the test class has the following
* characteristics:
* - executed in an isolated JVM. Tests can however be executed in different
* JVM on the same machine simultaneously.
* - will not have to be executed by the developer before submitting a bug
* - ideally, last less than 2 minutes to help parallelization
*
* It the worst case compared to small or medium, use it only for tests that
* you cannot put in the other categories
*
* @see SmallTests
* @see MediumTests
*/
public interface LargeTests {
}

View File

@ -0,0 +1,37 @@
/*
* Copyright 2011 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;
/**
* Tag a test as 'Medium', meaning that the test class has the following
* characteristics:
* - executed in an isolated JVM. Tests can however be executed in different
* JVM on the same machine simultaneously.
* - will have to be executed by the developer before submitting a bug
* - ideally, last less than 1 minutes to help parallelization
*
* Use it for tests that cannot be tagged as 'Small'.
*
* @see SmallTests
* @see LargeTests
*/
public interface MediumTests {
}

View File

@ -0,0 +1,34 @@
/*
* Copyright 2011 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;
/**
* Tag a test as 'small', meaning that the test class has the following
* characteristics:
* - can be run simultaneously with other small tests in the same JVM
* - ideally, last less than 15 seconds
* - does not use a cluster
*
* @see MediumTests
* @see LargeTests
*/
public interface SmallTests {
}

View File

@ -40,6 +40,7 @@ import org.junit.Ignore;
import org.junit.Test;
import com.google.common.collect.Lists;
import org.junit.experimental.categories.Category;
/**
* Test case that uses multiple threads to read and write multifamily rows
@ -48,6 +49,7 @@ import com.google.common.collect.Lists;
* This can run as a junit test, or with a main() function which runs against
* a real cluster (eg for testing with failures, region movement, etc)
*/
@Category(MediumTests.class)
public class TestAcidGuarantees {
protected static final Log LOG = LogFactory.getLog(TestAcidGuarantees.class);
public static final byte [] TABLE_NAME = Bytes.toBytes("TestAcidGuarantees");

View File

@ -39,11 +39,13 @@ import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test the draining servers feature.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-4298">HBASE-4298</a>
*/
@Category(MediumTests.class)
public class TestDrainingServer {
private static final Log LOG = LogFactory.getLog(TestDrainingServer.class);
private static final HBaseTestingUtility TEST_UTIL =

View File

@ -33,7 +33,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestFullLogReconstruction {
private final static HBaseTestingUtility

View File

@ -34,11 +34,13 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test HBASE-3694 whether the GlobalMemStoreSize is the same as the summary
* of all the online region's MemStoreSize
*/
@Category(MediumTests.class)
public class TestGlobalMemStoreSize {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
private static int regionServerNum = 4;

View File

@ -43,10 +43,12 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test our testing utility class
*/
@Category(LargeTests.class)
public class TestHBaseTestingUtility {
private final Log LOG = LogFactory.getLog(this.getClass());

View File

@ -29,12 +29,15 @@ import org.apache.hadoop.hbase.client.HTable;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertTrue;
/**
* Testing, info servers are disabled. This test enables then and checks that
* they serve pages.
*/
@Category(MediumTests.class)
public class TestInfoServers {
static final Log LOG = LogFactory.getLog(TestInfoServers.class);
private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -46,11 +46,13 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Port of old TestScanMultipleVersions, TestTimestamp and TestGetRowVersions
* from old testing framework to {@link HBaseTestingUtility}.
*/
@Category(MediumTests.class)
public class TestMultiVersions {
private static final Log LOG = LogFactory.getLog(TestMultiVersions.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@ -318,4 +320,4 @@ public class TestMultiVersions {
s.close();
}
}
}
}

View File

@ -39,10 +39,12 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test whether region rebalancing works. (HBASE-71)
*/
@Category(LargeTests.class)
public class TestRegionRebalancing {
final Log LOG = LogFactory.getLog(this.getClass().getName());
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@ -214,4 +216,4 @@ public class TestRegionRebalancing {
} catch (InterruptedException e) {}
}
}
}
}

View File

@ -50,7 +50,9 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestZooKeeper {
private final Log LOG = LogFactory.getLog(this.getClass());

View File

@ -27,6 +27,7 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericData;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.avro.generated.AColumn;
import org.apache.hadoop.hbase.avro.generated.AColumnValue;
import org.apache.hadoop.hbase.avro.generated.AFamilyDescriptor;
@ -40,11 +41,13 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Unit testing for AvroServer.HBaseImpl, a part of the
* org.apache.hadoop.hbase.avro package.
*/
@Category(MediumTests.class)
public class TestAvroServer {
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -31,14 +31,7 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -58,12 +51,14 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Matchers;
import org.mockito.Mockito;
/**
* Test {@link CatalogTracker}
*/
@Category(MediumTests.class)
public class TestCatalogTracker {
private static final Log LOG = LogFactory.getLog(TestCatalogTracker.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@ -410,4 +405,4 @@ public class TestCatalogTracker {
this.ct.waitForRoot();
}
}
}
}

View File

@ -21,15 +21,15 @@ package org.apache.hadoop.hbase.catalog;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Do {@link CatalogTracker} tests on running cluster.
*/
@Category(LargeTests.class)
public class TestCatalogTrackerOnCluster {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final Log LOG =

View File

@ -31,11 +31,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.Bytes;
@ -45,10 +41,12 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test {@link MetaReader}, {@link MetaEditor}, and {@link RootLocationEditor}.
*/
@Category(MediumTests.class)
public class TestMetaReaderEditor {
private static final Log LOG = LogFactory.getLog(TestMetaReaderEditor.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -26,13 +26,7 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
@ -45,12 +39,14 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
/**
* Test MetaReader/Editor but without spinning up a cluster.
* We mock regionserver back and forth (we do spin up a zk cluster).
*/
@Category(MediumTests.class)
public class TestMetaReaderEditorNoCluster {
private static final Log LOG = LogFactory.getLog(TestMetaReaderEditorNoCluster.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -39,17 +39,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -62,6 +52,7 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
@ -69,6 +60,7 @@ import org.junit.Test;
* Spins up the minicluster once at test start and then takes it down afterward.
* Add any testing of HBaseAdmin functionality here.
*/
@Category(LargeTests.class)
public class TestAdmin {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -48,13 +48,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
@ -80,12 +74,14 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Run tests that use the HBase clients; {@link HTable} and {@link HTablePool}.
* Sets up the HBase mini cluster once at start and runs through all client tests.
* Each creates a table named for the method and does its stuff against that.
*/
@Category(LargeTests.class)
public class TestFromClientSide {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -34,20 +34,18 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* This class is for testing HCM features
*/
@Category(MediumTests.class)
public class TestHCM {
private static final Log LOG = LogFactory.getLog(TestHCM.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -25,13 +25,11 @@ import junit.framework.Assert;
import junit.framework.TestCase;
import junit.framework.TestSuite;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PoolMap.PoolType;
import org.junit.*;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Suite;
@ -40,6 +38,7 @@ import org.junit.runners.Suite;
*/
@RunWith(Suite.class)
@Suite.SuiteClasses({TestHTablePool.TestHTableReusablePool.class, TestHTablePool.TestHTableThreadLocalPool.class})
@Category(MediumTests.class)
public class TestHTablePool {
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static byte[] TABLENAME = Bytes.toBytes("TestHTablePool");

View File

@ -27,15 +27,18 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* This class provides tests for the {@link HTableUtil} class
*
*/
@Category(MediumTests.class)
public class TestHTableUtil {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -38,11 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD;
import org.apache.hadoop.hbase.catalog.MetaReader;
@ -52,11 +48,13 @@ import org.apache.hadoop.hbase.util.Writables;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test migration that removes HTableDescriptor from HRegionInfo moving the
* meta version from no version to {@link MetaReader#META_VERSION}.
*/
@Category(MediumTests.class)
public class TestMetaMigrationRemovingHTD {
static final Log LOG = LogFactory.getLog(TestMetaMigrationRemovingHTD.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -22,15 +22,16 @@ package org.apache.hadoop.hbase.client;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.mockito.Mockito.*;
@Category(MediumTests.class)
public class TestMetaScanner {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -27,8 +27,7 @@ import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.AfterClass;
@ -36,9 +35,11 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@Category(MediumTests.class)
public class TestMultiParallel {
private static final Log LOG = LogFactory.getLog(TestMultiParallel.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -28,20 +28,21 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Run tests related to {@link TimestampsFilter} using HBase client APIs.
* Sets up the HBase mini cluster once at start. Each creates a table
* named for the method and does its stuff against that.
*/
@Category(LargeTests.class)
public class TestMultipleTimestamps {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -25,8 +25,7 @@ import static org.junit.Assert.fail;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
@ -34,10 +33,12 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test various scanner timeout issues.
*/
@Category(LargeTests.class)
public class TestScannerTimeout {
private final static HBaseTestingUtility

View File

@ -27,17 +27,16 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.jruby.embed.ScriptingContainer;
import org.jruby.embed.PathType;
import org.junit.experimental.categories.Category;
/**
*
* @author scoundrel
*/
@Category(LargeTests.class)
public class TestShell {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -28,8 +28,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.TimestampsFilter;
import org.apache.hadoop.hbase.util.Bytes;
@ -38,12 +37,14 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Run tests related to {@link TimestampsFilter} using HBase client APIs.
* Sets up the HBase mini cluster once at start. Each creates a table
* named for the method and does its stuff against that.
*/
@Category(MediumTests.class)
public class TestTimestampsFilter {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -24,12 +24,11 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertEquals;
@ -37,6 +36,7 @@ import static org.junit.Assert.assertEquals;
/**
* Unit testing of ReplicationAdmin
*/
@Category(MediumTests.class)
public class TestReplicationAdmin {
private static final Log LOG =

View File

@ -24,9 +24,7 @@ import static org.junit.Assert.assertEquals;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
@ -38,11 +36,13 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* A test class to cover aggregate functions, that can be implemented using
* Coprocessors.
*/
@Category(MediumTests.class)
public class TestAggregateProtocol {
protected static Log myLog = LogFactory.getLog(TestAggregateProtocol.class);

View File

@ -23,17 +23,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -45,6 +37,7 @@ import java.util.Arrays;
import java.util.jar.*;
import org.junit.*;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -53,6 +46,7 @@ import static org.junit.Assert.assertFalse;
/**
* Test coprocessors class loading.
*/
@Category(MediumTests.class)
public class TestClassLoading {
private static final Log LOG = LogFactory.getLog(TestClassLoading.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -26,9 +26,7 @@ import java.io.IOException;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
@ -38,10 +36,12 @@ import org.apache.hadoop.io.Text;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* TestEndpoint: test cases to verify coprocessor Endpoint
*/
@Category(MediumTests.class)
public class TestCoprocessorEndpoint {
private static final byte[] TEST_TABLE = Bytes.toBytes("TestTable");

View File

@ -24,14 +24,8 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes;
@ -40,6 +34,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
/**
@ -48,6 +44,7 @@ import static org.junit.Assert.*;
* error message describing the set of its loaded coprocessors for crash diagnosis.
* (HBASE-4014).
*/
@Category(MediumTests.class)
public class TestMasterCoprocessorExceptionWithAbort {
public static class MasterTracker extends ZooKeeperNodeTracker {

View File

@ -24,14 +24,8 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes;
@ -40,6 +34,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
/**
@ -49,6 +45,7 @@ import static org.junit.Assert.*;
* back to the client.
* (HBASE-4014).
*/
@Category(MediumTests.class)
public class TestMasterCoprocessorExceptionWithRemove {
public static class MasterTracker extends ZooKeeperNodeTracker {

View File

@ -34,14 +34,7 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -52,11 +45,13 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver}
* interface hooks at all appropriate times during normal HMaster operations.
*/
@Category(MediumTests.class)
public class TestMasterObserver {
private static final Log LOG = LogFactory.getLog(TestMasterObserver.class);

View File

@ -25,9 +25,7 @@ import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
@ -42,9 +40,11 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
@Category(MediumTests.class)
public class TestRegionObserverBypass {
private static HBaseTestingUtility util;
private static final byte[] tableName = Bytes.toBytes("test");

View File

@ -29,13 +29,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.*;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@ -49,9 +43,11 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@Category(MediumTests.class)
public class TestRegionObserverInterface {
static final Log LOG = LogFactory.getLog(TestRegionObserverInterface.class);
static final String DIR = "test/build/data/TestRegionObserver/";

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@ -41,6 +42,7 @@ import static org.junit.Assert.*;
* error message describing the set of its loaded coprocessors for crash
* diagnosis. (HBASE-4014).
*/
@Category(MediumTests.class)
public class TestRegionServerCoprocessorExceptionWithAbort {
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@ -43,6 +44,7 @@ import static org.junit.Assert.*;
* back to the client.
* (HBASE-4014).
*/
@Category(MediumTests.class)
public class TestRegionServerCoprocessorExceptionWithRemove {
public static class BuggyRegionObserver extends SimpleRegionObserver {
@SuppressWarnings("null")

View File

@ -25,15 +25,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
@ -49,6 +42,7 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
@ -62,6 +56,7 @@ import static org.junit.Assert.*;
* Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver}
* interface hooks at all appropriate times during normal HMaster operations.
*/
@Category(MediumTests.class)
public class TestWALObserver {
private static final Log LOG = LogFactory.getLog(TestWALObserver.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -29,12 +29,7 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -48,6 +43,8 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
class StringRange {
private String start = null;
@ -118,6 +115,7 @@ class StringRange {
}
}
@Category(MediumTests.class)
public class TestColumnRangeFilter {
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -32,13 +32,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
@ -49,6 +49,7 @@ import static org.junit.Assert.*;
* types: data blocks, non-root index blocks, and Bloom filter blocks.
*/
@RunWith(Parameterized.class)
@Category(MediumTests.class)
public class TestCacheOnWrite {
private static final Log LOG = LogFactory.getLog(TestCacheOnWrite.class);

View File

@ -28,8 +28,10 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.hbase.*;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
@ -42,8 +44,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
@RunWith(Parameterized.class)
public class TestFixedFileTrailer {

View File

@ -30,13 +30,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
import org.junit.experimental.categories.Category;
/**
* test hfile features.

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.io.compress.Compressor;
@ -51,7 +52,9 @@ import org.apache.hadoop.io.compress.Compressor;
import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestHFileBlock {
// change this value to activate more logs
private static final boolean detailedLogging = false;

View File

@ -39,8 +39,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader;
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk;
import org.apache.hadoop.hbase.util.Bytes;
@ -48,6 +47,7 @@ import org.apache.hadoop.hbase.util.ClassSize;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;

View File

@ -31,10 +31,12 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.junit.experimental.categories.Category;
/**
* Set of long-running tests to measure performance of HFile.

View File

@ -22,10 +22,12 @@ package org.apache.hadoop.hbase.io.hfile;
import java.nio.ByteBuffer;
import java.util.Random;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.ClassSize;
import junit.framework.TestCase;
import org.junit.experimental.categories.Category;
/**
* Tests the concurrent LruBlockCache.<p>
@ -34,6 +36,7 @@ import junit.framework.TestCase;
* evictions run when they're supposed to and do what they should,
* and that cached blocks are accessible when expected to be.
*/
@Category(MediumTests.class)
public class TestLruBlockCache extends TestCase {
public void testBackgroundEvictionThread() throws Exception {

View File

@ -19,10 +19,12 @@
*/
package org.apache.hadoop.hbase.io.hfile.slab;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests SingleSlabCache.
@ -31,6 +33,7 @@ import org.junit.Test;
* Tests will ensure that evictions operate when they're supposed to and do what
* they should, and that cached blocks are accessible when expected to be.
*/
@Category(MediumTests.class)
public class TestSingleSizeCache {
SingleSizeCache cache;
final int CACHE_SIZE = 1000000;

View File

@ -35,9 +35,7 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -49,6 +47,7 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@ -56,6 +55,7 @@ import org.mockito.stubbing.Answer;
* This tests the TableInputFormat and its recovery semantics
*
*/
@Category(LargeTests.class)
public class TestTableInputFormat {
private static final Log LOG = LogFactory.getLog(TestTableInputFormat.class);

View File

@ -27,11 +27,7 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -48,12 +44,14 @@ import org.apache.hadoop.mapred.Reporter;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test Map/Reduce job over HBase tables. The map/reduce process we're testing
* on our tables is simple - take every row in the table, reverse the value of
* a particular cell, and write it back to the table.
*/
@Category(LargeTests.class)
public class TestTableMapReduce {
private static final Log LOG =
LogFactory.getLog(TestTableMapReduce.class.getName());
@ -255,4 +253,4 @@ public class TestTableMapReduce {
scanner.close();
}
}
}
}

View File

@ -42,13 +42,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.PerformanceEvaluation;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
@ -73,6 +67,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.common.collect.Lists;
@ -83,6 +78,7 @@ import com.google.common.collect.Lists;
* Creates a few inner classes to implement splits and an inputformat that
* emits keys and values like those of {@link PerformanceEvaluation}.
*/
@Category(LargeTests.class)
public class TestHFileOutputFormat {
private final static int ROWSPERSPLIT = 1024;

View File

@ -23,6 +23,7 @@ import java.io.UnsupportedEncodingException;
import java.util.List;
import java.util.ArrayList;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
@ -30,20 +31,14 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser;
import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.ParsedLine;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.client.Result;
import org.junit.Test;
@ -51,9 +46,11 @@ import org.junit.Test;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
import com.google.common.collect.Iterables;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@Category(MediumTests.class)
public class TestImportTsv {
@Test

View File

@ -29,10 +29,7 @@ import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -42,12 +39,14 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.*;
import org.junit.experimental.categories.Category;
/**
* Test cases for the "load" half of the HFileOutputFormat bulk load
* functionality. These tests run faster than the full MR cluster
* tests in TestHFileOutputFormat
*/
@Category(LargeTests.class)
public class TestLoadIncrementalHFiles {
private static final byte[] QUALIFIER = Bytes.toBytes("myqual");
private static final byte[] FAMILY = Bytes.toBytes("myfam");

View File

@ -37,11 +37,7 @@ 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
@ -58,10 +54,12 @@ import org.junit.BeforeClass;
import org.junit.Test;
import com.google.common.collect.Multimap;
import org.junit.experimental.categories.Category;
/**
* Test cases for the atomic load error handling of the bulk load functionality.
*/
@Category(LargeTests.class)
public class TestLoadIncrementalHFilesSplitRecovery {
final static Log LOG = LogFactory.getLog(TestHRegionServerBulkLoad.class);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@ -44,6 +45,7 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -53,6 +55,7 @@ import static org.junit.Assert.assertTrue;
* tested in a MapReduce job to see if that is handed over and done properly
* too.
*/
@Category(LargeTests.class)
public class TestTableInputFormatScan {
static final Log LOG = LogFactory.getLog(TestTableInputFormatScan.class);
@ -341,4 +344,4 @@ public class TestTableInputFormatScan {
assertTrue(job.isComplete());
LOG.info("After map/reduce completion - job " + jobName);
}
}
}

View File

@ -29,11 +29,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -47,6 +43,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
@ -56,6 +54,7 @@ import static org.junit.Assert.assertFalse;
* on our tables is simple - take every row in the table, reverse the value of
* a particular cell, and write it back to the table.
*/
@Category(LargeTests.class)
public class TestTableMapReduce {
private static final Log LOG = LogFactory.getLog(TestTableMapReduce.class);
private static final HBaseTestingUtility UTIL =

View File

@ -32,13 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -56,7 +50,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestTimeRangeMapRed {
private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class);
private static final HBaseTestingUtility UTIL =
@ -214,4 +210,4 @@ public class TestTimeRangeMapRed {
}
scanner.close();
}
}
}

View File

@ -29,9 +29,7 @@ import java.util.concurrent.Semaphore;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.util.Bytes;
@ -42,11 +40,13 @@ import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
/**
* Test the {@link ActiveMasterManager}.
*/
@Category(MediumTests.class)
public class TestActiveMasterManager {
private final static Log LOG = LogFactory.getLog(TestActiveMasterManager.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -39,12 +39,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
@ -67,7 +62,9 @@ import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestDistributedLogSplitting {
private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
static {

View File

@ -26,14 +26,14 @@ import static org.junit.Assert.fail;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.ipc.RemoteException;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestHMasterRPCException {
@Test

View File

@ -29,10 +29,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
@ -40,7 +37,9 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestLogsCleaner {
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -21,11 +21,7 @@ package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -45,9 +41,11 @@ import org.junit.BeforeClass;
import org.junit.Test;
import com.google.common.base.Joiner;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@Category(MediumTests.class)
public class TestMaster {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final Log LOG = LogFactory.getLog(TestMaster.class);
@ -143,4 +141,4 @@ public class TestMaster {
public void beforeProcess(EventHandler event) {
}
}
}
}

View File

@ -33,15 +33,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
@ -57,7 +49,9 @@ import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestMasterFailover {
private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
@ -907,7 +901,7 @@ public class TestMasterFailover {
Thread.sleep(100);
}
LOG.debug("\n\nRegion of disabled table was open at steady-state on dead RS"
+ "\n" + region + "\n\n");
+ "\n" + region + "\n\n");
/*
* DONE MOCKING
@ -1018,4 +1012,4 @@ public class TestMasterFailover {
private void log(String string) {
LOG.info("\n\n" + string + " \n\n");
}
}
}

View File

@ -29,10 +29,7 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@ -41,7 +38,9 @@ import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestMasterRestartAfterDisablingTable {
private static final Log LOG = LogFactory.getLog(TestMasterRestartAfterDisablingTable.class);

View File

@ -23,9 +23,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -39,11 +37,13 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test transitions of state across the master. Sets up the cluster once and
* then runs a couple of tests.
*/
@Category(LargeTests.class)
public class TestMasterTransitions {
private static final Log LOG = LogFactory.getLog(TestMasterTransitions.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -30,13 +30,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
@ -53,8 +47,10 @@ import org.apache.zookeeper.data.Stat;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
@Category(MediumTests.class)
public class TestOpenedRegionHandler {
private static final Log LOG = LogFactory

View File

@ -28,10 +28,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.util.Bytes;
@ -41,7 +38,9 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestRestartCluster {
private static final Log LOG = LogFactory.getLog(TestRestartCluster.class);
private HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -30,11 +30,7 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@ -43,10 +39,12 @@ import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests the restarting of everything as done during rolling restarts.
*/
@Category(LargeTests.class)
public class TestRollingRestart {
private static final Log LOG = LogFactory.getLog(TestRollingRestart.class);

View File

@ -32,8 +32,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
import org.apache.hadoop.hbase.regionserver.TestMasterAddressManager.NodeCreationListener;
@ -51,8 +50,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestSplitLogManager {
private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
static {

View File

@ -27,10 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -49,6 +46,7 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -56,6 +54,7 @@ import static org.junit.Assert.assertTrue;
/**
* Test open and close of regions using zk.
*/
@Category(MediumTests.class)
public class TestZKBasedOpenCloseRegion {
private static final Log LOG = LogFactory.getLog(TestZKBasedOpenCloseRegion.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -28,13 +28,7 @@ import java.util.List;
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.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
@ -46,7 +40,9 @@ import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestBlocksRead extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestBlocksRead.class);
static final String[] BLOOM_TYPE = new String[] { "ROWCOL", "ROW", "NONE" };

View File

@ -19,8 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -31,10 +30,12 @@ import org.apache.hadoop.hbase.util.PairOfSameType;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
@Category(LargeTests.class)
public class TestEndToEndSplitTransaction {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -38,11 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PositionedReadable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -50,12 +46,14 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test cases that ensure that file system level errors are bubbled up
* appropriately to clients, rather than swallowed.
*/
@Category(MediumTests.class)
public class TestFSErrorsExposed {
private static final Log LOG = LogFactory.getLog(TestFSErrorsExposed.class);

View File

@ -35,20 +35,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MultithreadedTestUtil;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
@ -76,6 +65,7 @@ import org.apache.hadoop.hbase.util.Threads;
import org.junit.Test;
import com.google.common.collect.Lists;
import org.junit.experimental.categories.Category;
/**
@ -84,6 +74,7 @@ import com.google.common.collect.Lists;
* A lot of the meta information for an HRegion now lives inside other
* HRegions or in the HBaseMaster, so only basic testing is possible.
*/
@Category(MediumTests.class)
public class TestHRegion extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestHRegion.class);

View File

@ -27,14 +27,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
@ -50,11 +45,13 @@ import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test;
import com.google.common.collect.Lists;
import org.junit.experimental.categories.Category;
/**
* Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
* the region server's bullkLoad functionality.
*/
@Category(LargeTests.class)
public class TestHRegionServerBulkLoad {
final static Log LOG = LogFactory.getLog(TestHRegionServerBulkLoad.class);
private static HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -26,16 +26,16 @@ import java.util.concurrent.Semaphore;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MasterAddressTracker;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestMasterAddressManager {
private static final Log LOG = LogFactory.getLog(TestMasterAddressManager.class);

View File

@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
import org.apache.hadoop.metrics.ContextFactory;
import org.apache.hadoop.metrics.MetricsContext;
@ -38,9 +39,11 @@ import org.apache.hadoop.metrics.spi.OutputRecord;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
@Category(MediumTests.class)
public class TestRpcMetrics {
/**
* Defines test methods to register with HBaseRpcMetrics

View File

@ -37,9 +37,7 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
@ -48,6 +46,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
@ -57,6 +56,7 @@ import org.junit.runners.Parameterized.Parameters;
* actually saving I/O operations.
*/
@RunWith(Parameterized.class)
@Category(MediumTests.class)
public class TestSeekOptimizations {
private static final Log LOG =

View File

@ -25,13 +25,7 @@ import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -50,7 +44,9 @@ import org.junit.BeforeClass;
import org.junit.Test;
import com.google.common.collect.Lists;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestServerCustomProtocol {
/* Test protocol */
public static interface PingProtocol extends CoprocessorProtocol {

View File

@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
@ -47,9 +48,9 @@ import org.apache.zookeeper.ZooDefs.Ids;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestSplitLogWorker {
private static final Log LOG = LogFactory.getLog(TestSplitLogWorker.class);
static {

View File

@ -28,14 +28,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -53,12 +46,14 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Like {@link TestSplitTransaction} in that we're testing {@link SplitTransaction}
* only the below tests are against a running cluster where {@link TestSplitTransaction}
* is tests against a bare {@link HRegion}.
*/
@Category(LargeTests.class)
public class TestSplitTransactionOnCluster {
private static final Log LOG =
LogFactory.getLog(TestSplitTransactionOnCluster.class);

View File

@ -28,6 +28,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableUtil;
import org.apache.hadoop.hbase.client.Put;
@ -41,12 +42,14 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests the block cache summary functionality in StoreFile,
* which contains the BlockCache
*
*/
@Category(MediumTests.class)
public class TestStoreFileBlockCacheSummary {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -25,10 +25,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -45,10 +42,12 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test of the {@link OpenRegionHandler}.
*/
@Category(MediumTests.class)
public class TestOpenRegionHandler {
static final Log LOG = LogFactory.getLog(TestOpenRegionHandler.class);
private final static HBaseTestingUtility HTU = new HBaseTestingUtility();

View File

@ -59,8 +59,10 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/** JUnit test case for HLog */
@Category(LargeTests.class)
public class TestHLog {
private static final Log LOG = LogFactory.getLog(TestHLog.class);
{

View File

@ -39,14 +39,9 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
@ -60,6 +55,7 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@ -70,6 +66,7 @@ import com.google.common.collect.ImmutableList;
/**
* Testing {@link HLog} splitting code.
*/
@Category(LargeTests.class)
public class TestHLogSplit {
private final static Log LOG = LogFactory.getLog(TestHLogSplit.class);

View File

@ -26,11 +26,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -47,11 +43,13 @@ import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests for conditions that should trigger RegionServer aborts when
* rolling the current HLog fails.
*/
@Category(MediumTests.class)
public class TestLogRollAbort {
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
private static MiniDFSCluster dfsCluster;

View File

@ -39,13 +39,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -68,10 +62,12 @@ import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test log deletion as logs are rolled.
*/
@Category(LargeTests.class)
public class TestLogRolling {
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
private HRegionServer server;

View File

@ -33,13 +33,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -58,11 +52,13 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
/**
* Test replay of edits out of a WAL split.
*/
@Category(MediumTests.class)
public class TestWALReplay {
public static final Log LOG = LogFactory.getLog(TestWALReplay.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

View File

@ -29,12 +29,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -53,7 +48,9 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestMasterReplication {
private static final Log LOG = LogFactory.getLog(TestReplication.class);

View File

@ -28,12 +28,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -47,7 +42,9 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestMultiSlaveReplication {
private static final Log LOG = LogFactory.getLog(TestReplication.class);

View File

@ -26,13 +26,7 @@ import static org.junit.Assert.fail;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -52,7 +46,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestReplication {
private static final Log LOG = LogFactory.getLog(TestReplication.class);

View File

@ -22,12 +22,13 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException.SessionExpiredException;
import org.junit.*;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestReplicationPeer {
private static final Log LOG = LogFactory.getLog(TestReplicationPeer.class);

View File

@ -24,10 +24,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -35,10 +32,12 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
@Category(MediumTests.class)
public class TestReplicationSource {
private static final Log LOG =

View File

@ -27,10 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
@ -44,7 +41,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestReplicationSink {
private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
private static final int BATCH_SIZE = 10;

View File

@ -30,15 +30,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@ -53,7 +45,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestReplicationSourceManager {
private static final Log LOG =

View File

@ -26,9 +26,7 @@ import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
import org.apache.commons.httpclient.Header;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -43,7 +41,9 @@ import static org.junit.Assert.*;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestGzipFilter {
private static final String TABLE = "TestGzipFilter";
private static final String CFA = "a";
@ -149,4 +149,4 @@ public class TestGzipFilter {
response = client.get(scannerUrl);
assertEquals(response.getCode(), 204);
}
}
}

Some files were not shown because too many files have changed in this diff Show More