HBASE-5824 revert changes to single Put case, preserving deprecation for ICV

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1328457 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2012-04-20 17:36:02 +00:00
parent 85842f019e
commit e05c21c0cb
3 changed files with 25 additions and 34 deletions

View File

@ -829,40 +829,21 @@ public class HTable implements HTableInterface {
}
private void doPut(final List<Put> puts) throws IOException {
if (autoFlush && puts.size() == 1) {
final Put put = puts.get(0);
int n = 0;
for (Put put : puts) {
validatePut(put);
new ServerCallable<Void>(connection, tableName, put.getRow(),
operationTimeout) {
public Void call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), put);
server.mutate(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
return null;
}
}.withRetries();
} else {
int n = 0;
for (Put put : puts) {
validatePut(put);
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
n++;
if (n % DOPUT_WB_CHECK == 0 && currentWriteBufferSize > writeBufferSize) {
flushCommits();
}
}
if (autoFlush || currentWriteBufferSize > writeBufferSize) {
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
n++;
if (n % DOPUT_WB_CHECK == 0 && currentWriteBufferSize > writeBufferSize) {
flushCommits();
}
}
if (autoFlush || currentWriteBufferSize > writeBufferSize) {
flushCommits();
}
}
/**

View File

@ -17,10 +17,13 @@
*/
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;
@ -29,6 +32,7 @@ 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;
@ -114,8 +118,13 @@ public class TestConstraint {
try {
table.put(put);
fail("This put should not have suceeded - AllFailConstraint was not run!");
} catch (Throwable t) {
assertTrue(t instanceof ConstraintException);
} 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());
}
table.close();
}

View File

@ -26,6 +26,7 @@ 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;
@ -111,12 +112,12 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
Put put = new Put(ROW);
put.add(TEST_FAMILY, ROW, ROW);
table.put(put);
} catch (Throwable t) {
} catch (RetriesExhaustedWithDetailsException e) {
// below, could call instead :
// startsWith("Failed 1 action: DoNotRetryIOException.")
// But that might be too brittle if client-side
// DoNotRetryIOException-handler changes its message.
assertTrue(t instanceof DoNotRetryIOException);
assertTrue(e.getMessage().contains("DoNotRetryIOException"));
threwDNRE = true;
} finally {
assertTrue(threwDNRE);