HBASE-17309 Fix connection leaks in TestAcidGuarantees (huaxiang sun)

This commit is contained in:
tedyu 2016-12-14 20:03:44 -08:00
parent 691f266fc2
commit a5a6036883
2 changed files with 41 additions and 5 deletions

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -143,12 +144,17 @@ public abstract class MultithreadedTestUtil {
}
public final void doWork() throws Exception {
try {
while (ctx.shouldRun() && !stopped) {
doAnAction();
}
} finally {
workDone();
}
}
public abstract void doAnAction() throws Exception;
public void workDone() throws IOException {}
}
/**

View File

@ -118,6 +118,7 @@ public class TestAcidGuarantees implements Tool {
byte data[] = new byte[10];
byte targetRows[][];
byte targetFamilies[][];
Connection connection;
Table table;
AtomicLong numWritten = new AtomicLong();
@ -126,7 +127,7 @@ public class TestAcidGuarantees implements Tool {
super(ctx);
this.targetRows = targetRows;
this.targetFamilies = targetFamilies;
Connection connection = ConnectionFactory.createConnection(ctx.getConf());
connection = ConnectionFactory.createConnection(ctx.getConf());
table = connection.getTable(TABLE_NAME);
}
public void doAnAction() throws Exception {
@ -144,6 +145,15 @@ public class TestAcidGuarantees implements Tool {
table.put(p);
numWritten.getAndIncrement();
}
@Override
public void workDone() throws IOException {
try {
table.close();
} finally {
connection.close();
}
}
}
/**
@ -153,6 +163,7 @@ public class TestAcidGuarantees implements Tool {
public static class AtomicGetReader extends RepeatingTestThread {
byte targetRow[];
byte targetFamilies[][];
Connection connection;
Table table;
int numVerified = 0;
AtomicLong numRead = new AtomicLong();
@ -162,7 +173,7 @@ public class TestAcidGuarantees implements Tool {
super(ctx);
this.targetRow = targetRow;
this.targetFamilies = targetFamilies;
Connection connection = ConnectionFactory.createConnection(ctx.getConf());
connection = ConnectionFactory.createConnection(ctx.getConf());
table = connection.getTable(TABLE_NAME);
}
@ -191,6 +202,15 @@ public class TestAcidGuarantees implements Tool {
numRead.getAndIncrement();
}
@Override
public void workDone() throws IOException {
try {
table.close();
} finally {
connection.close();
}
}
private void gotFailure(byte[] expected, Result res) {
StringBuilder msg = new StringBuilder();
msg.append("Failed after ").append(numVerified).append("!");
@ -213,6 +233,7 @@ public class TestAcidGuarantees implements Tool {
public static class AtomicScanReader extends RepeatingTestThread {
byte targetFamilies[][];
Table table;
Connection connection;
AtomicLong numScans = new AtomicLong();
AtomicLong numRowsScanned = new AtomicLong();
@ -220,7 +241,7 @@ public class TestAcidGuarantees implements Tool {
byte targetFamilies[][]) throws IOException {
super(ctx);
this.targetFamilies = targetFamilies;
Connection connection = ConnectionFactory.createConnection(ctx.getConf());
connection = ConnectionFactory.createConnection(ctx.getConf());
table = connection.getTable(TABLE_NAME);
}
@ -249,6 +270,15 @@ public class TestAcidGuarantees implements Tool {
numScans.getAndIncrement();
}
@Override
public void workDone() throws IOException {
try {
table.close();
} finally {
connection.close();
}
}
private void gotFailure(byte[] expected, Result res) {
StringBuilder msg = new StringBuilder();
msg.append("Failed after ").append(numRowsScanned).append("!");