HBASE-8141. Remove accidental uses of org.mortbay.log.Log

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1458001 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Kyle Purtell 2013-03-18 21:11:33 +00:00
parent 84d0115d44
commit a9f2ff5374
3 changed files with 10 additions and 6 deletions

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.util.ByteRange;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.mortbay.log.Log;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -74,7 +73,6 @@ public class TestTreeDepth {
} }
protected void testInternal(List<String> inputs, int expectedTreeDepth) { protected void testInternal(List<String> inputs, int expectedTreeDepth) {
Log.warn("init logger");
Tokenizer builder = new Tokenizer(); Tokenizer builder = new Tokenizer();
for (String s : inputs) { for (String s : inputs) {
ByteRange b = new ByteRange(Bytes.toBytes(s)); ByteRange b = new ByteRange(Bytes.toBytes(s));

View File

@ -32,6 +32,8 @@ import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options; import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException; import org.apache.commons.cli.ParseException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
@ -44,7 +46,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer; import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.mortbay.log.Log;
/** /**
* test the performance for seek. * test the performance for seek.
@ -66,6 +67,8 @@ public class TestHFileSeek extends TestCase {
private RandomDistribution.DiscreteRNG keyLenGen; private RandomDistribution.DiscreteRNG keyLenGen;
private KVGenerator kvGen; private KVGenerator kvGen;
private static final Log LOG = LogFactory.getLog(TestHFileSeek.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@Override @Override
@ -531,7 +534,7 @@ public class TestHFileSeek extends TestCase {
testCase.options = options; testCase.options = options;
for (int i = 0; i < options.trialCount; i++) { for (int i = 0; i < options.trialCount; i++) {
Log.info("Beginning trial " + (i+1)); LOG.info("Beginning trial " + (i+1));
testCase.setUp(); testCase.setUp();
testCase.testSeeks(); testCase.testSeeks();
testCase.tearDown(); testCase.tearDown();

View File

@ -28,6 +28,8 @@ import java.net.InetSocketAddress;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; 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.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -41,7 +43,6 @@ import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent; import org.apache.log4j.spi.LoggingEvent;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.mortbay.log.Log;
/** /**
* Test that delayed RPCs work. Fire up three calls, the first of which should * Test that delayed RPCs work. Fire up three calls, the first of which should
@ -50,6 +51,8 @@ import org.mortbay.log.Log;
*/ */
@Category(MediumTests.class) // Fails sometimes with small tests @Category(MediumTests.class) // Fails sometimes with small tests
public class TestDelayedRpc { public class TestDelayedRpc {
private static final Log LOG = LogFactory.getLog(TestDelayedRpc.class);
public static RpcServer rpcServer; public static RpcServer rpcServer;
public static final int UNDELAYED = 0; public static final int UNDELAYED = 0;
@ -287,7 +290,7 @@ public class TestDelayedRpc {
if (e.getCause().getMessage().contains( if (e.getCause().getMessage().contains(
"java.lang.Exception: Something went wrong")) "java.lang.Exception: Something went wrong"))
caughtException = true; caughtException = true;
Log.warn(e); LOG.warn(e);
} }
assertTrue(caughtException); assertTrue(caughtException);
} finally { } finally {