HBASE-5824 addendum fixes coprocessor tests (Jimmy)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1328166 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a09cea8e44
commit
85842f019e
|
@ -852,7 +852,8 @@ public class HTable implements HTableInterface {
|
|||
writeBuffer.add(put);
|
||||
currentWriteBufferSize += put.heapSize();
|
||||
|
||||
// we need to periodically see if the writebuffer is full instead of waiting until the end of the List
|
||||
// we need to periodically see if the writebuffer is full
|
||||
// instead of waiting until the end of the List
|
||||
n++;
|
||||
if (n % DOPUT_WB_CHECK == 0 && currentWriteBufferSize > writeBufferSize) {
|
||||
flushCommits();
|
||||
|
|
|
@ -17,13 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.constraint;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
|
@ -32,7 +29,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -118,13 +114,8 @@ public class TestConstraint {
|
|||
try {
|
||||
table.put(put);
|
||||
fail("This put should not have suceeded - AllFailConstraint was not run!");
|
||||
} catch (RetriesExhaustedWithDetailsException e) {
|
||||
List<Throwable> causes = e.getCauses();
|
||||
assertEquals(
|
||||
"More than one failure cause - should only be the failure constraint exception",
|
||||
1, causes.size());
|
||||
Throwable t = causes.get(0);
|
||||
assertEquals(ConstraintException.class, t.getClass());
|
||||
} catch (Throwable t) {
|
||||
assertTrue(t instanceof ConstraintException);
|
||||
}
|
||||
table.close();
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration;
|
|||
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.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
@ -112,12 +111,12 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
|
|||
Put put = new Put(ROW);
|
||||
put.add(TEST_FAMILY, ROW, ROW);
|
||||
table.put(put);
|
||||
} catch (RetriesExhaustedWithDetailsException e) {
|
||||
} catch (Throwable t) {
|
||||
// below, could call instead :
|
||||
// startsWith("Failed 1 action: DoNotRetryIOException.")
|
||||
// But that might be too brittle if client-side
|
||||
// DoNotRetryIOException-handler changes its message.
|
||||
assertTrue(e.getMessage().contains("DoNotRetryIOException"));
|
||||
assertTrue(t instanceof DoNotRetryIOException);
|
||||
threwDNRE = true;
|
||||
} finally {
|
||||
assertTrue(threwDNRE);
|
||||
|
|
Loading…
Reference in New Issue