HADOOP-2017 TestRegionServerAbort failure in patch build #903 and

nightly #266


git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@583309 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2007-10-09 21:48:11 +00:00
parent 5233c28aaa
commit c9f105acd5
7 changed files with 200 additions and 172 deletions

View File

@ -72,6 +72,8 @@ Trunk (unreleased changes)
HADOOP-2004 webapp hql formatting bugs
HADOOP_2011 Make hbase daemon scripts take args in same order as hadoop
daemon scripts
HADOOP-2017 TestRegionServerAbort failure in patch build #903 and
nightly #266
IMPROVEMENTS
HADOOP-1737 Make HColumnDescriptor data publically members settable

View File

@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Threads;
/**
* Chore is a task performed on a period in hbase. The chore is run in its own

View File

@ -205,7 +205,7 @@ HMasterRegionInterface {
HRegionInterface regionServer = null;
long scannerId = -1L;
LOG.info(Thread.currentThread().getName() + " scanning meta region " +
region.getRegionName() + " on " + region.getServer().toString());
region.toString());
// Array to hold list of split parents found. Scan adds to list. After
// scan we go check if parents can be removed.
@ -247,7 +247,7 @@ HMasterRegionInterface {
}
numberOfRegionsFound += 1;
}
if (rootRegion) {
if (this.rootRegion) {
numberOfMetaRegions.set(numberOfRegionsFound);
}
} catch (IOException e) {
@ -282,7 +282,7 @@ HMasterRegionInterface {
}
}
LOG.info(Thread.currentThread().getName() + " scan of meta region " +
region.getRegionName() + " complete");
region.toString() + " complete");
}
/*
@ -405,8 +405,8 @@ HMasterRegionInterface {
}
protected void checkAssigned(final HRegionInfo info,
final String serverName, final long startCode) throws IOException {
final String serverName, final long startCode)
throws IOException {
// Skip region - if ...
if(info.offLine // offline
|| killedRegions.contains(info.regionName) // queued for offline
@ -415,7 +415,6 @@ HMasterRegionInterface {
assignAttempts.remove(info.regionName);
return;
}
HServerInfo storedInfo = null;
if (serverName.length() != 0) {
Map<Text, HRegionInfo> regionsToKill = killList.get(serverName);
@ -432,14 +431,17 @@ HMasterRegionInterface {
storedInfo = serversToServerInfo.get(serverName);
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Checking " + info.regionName + " is assigned");
}
if (!(unassignedRegions.containsKey(info.regionName) ||
pendingRegions.contains(info.regionName))
pendingRegions.contains(info.regionName))
&& (storedInfo == null || storedInfo.getStartCode() != startCode)) {
// The current assignment is no good
if (LOG.isDebugEnabled()) {
LOG.debug("Current assignment of " + info.regionName + " is no good");
}
// Recover the region server's log if there is one.
if (serverName.length() != 0) {
StringBuilder dirName = new StringBuilder("log_");
dirName.append(serverName.replace(":", "_"));
@ -449,20 +451,19 @@ HMasterRegionInterface {
splitLogLock.lock();
try {
HLog.splitLog(dir, logDir, fs, conf);
} finally {
splitLogLock.unlock();
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Split " + logDir.toString());
}
} catch (IOException e) {
LOG.warn("unable to split region server log because: ", e);
throw e;
}
}
// Now get the region assigned
unassignedRegions.put(info.regionName, info);
assignAttempts.put(info.regionName, Long.valueOf(0L));
}
@ -479,6 +480,7 @@ HMasterRegionInterface {
}
private void scanRoot() {
boolean succeeded = false;
int tries = 0;
while (!closed.get() && tries < numRetries) {
synchronized (rootRegionLocation) {
@ -503,6 +505,7 @@ HMasterRegionInterface {
scanRegion(new MetaRegion(rootRegionLocation.get(),
HGlobals.rootRegionInfo.regionName, null));
}
succeeded = true;
break;
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
@ -511,8 +514,9 @@ HMasterRegionInterface {
LOG.warn("Scan ROOT region", e);
} else {
LOG.error("Scan ROOT region", e);
if (tries == numRetries - 1) {
// We ran out of tries. Make sure the file system is still available
if (tries == numRetries - 1) {
// We ran out of tries. Make sure the file system is still
// available
if (!checkFileSystem()) {
continue; // Avoid sleeping.
}
@ -524,7 +528,13 @@ HMasterRegionInterface {
LOG.error("Unexpected exception", e);
}
sleeper.sleep();
}
}
if (!succeeded) {
// We tried numretries to reach root and failed. Is it gone.
// Currently we just flounder. Should we reallocate root?
// This would be catastrophic?
// unassignRootRegion();
}
}
@Override
@ -564,6 +574,12 @@ HMasterRegionInterface {
this.startKey.set(startKey);
}
}
@Override
public String toString() {
return "regionname: " + this.regionName.toString() + ", startKey: <" +
this.startKey.toString() + ">, server: " + this.server.toString() + "}";
}
/** @return the regionName */
public Text getRegionName() {
@ -599,23 +615,28 @@ HMasterRegionInterface {
/** {@inheritDoc} */
public int compareTo(Object o) {
MetaRegion other = (MetaRegion)o;
int result = this.regionName.compareTo(other.getRegionName());
if(result == 0) {
result = this.startKey.compareTo(other.getStartKey());
if (result == 0) {
// Might be on different host?
result = this.server.compareTo(other.server);
}
}
return result;
}
}
/** Set by root scanner to indicate the number of meta regions */
AtomicInteger numberOfMetaRegions;
final AtomicInteger numberOfMetaRegions = new AtomicInteger();
/** Work for the meta scanner is queued up here */
BlockingQueue<MetaRegion> metaRegionsToScan;
final BlockingQueue<MetaRegion> metaRegionsToScan =
new LinkedBlockingQueue<MetaRegion>();
/** These are the online meta regions */
SortedMap<Text, MetaRegion> onlineMetaRegions;
final SortedMap<Text, MetaRegion> onlineMetaRegions =
Collections.synchronizedSortedMap(new TreeMap<Text, MetaRegion>());
/** Set by meta scanner after initial scan */
volatile boolean initialMetaScanComplete;
@ -657,15 +678,24 @@ HMasterRegionInterface {
e = RemoteExceptionHandler.checkIOException(e);
tries += 1;
if (tries == 1) {
LOG.warn("Scan one META region", e);
LOG.warn("Scan one META region: " + region.toString(), e);
} else {
LOG.error("Scan one META region", e);
if (tries == numRetries - 1) {
// We ran out of tries. Make sure the file system is still
// available
if (!checkFileSystem()) {
continue; // avoid sleeping
}
LOG.error("Scan one META region: " + region.toString(), e);
}
// The region may have moved (TestRegionServerAbort, etc.). If
// so, either it won't be in the onlineMetaRegions list or its host
// address has changed and the containsValue will fail. If not
// found, best thing to do here is probably break.
if (!onlineMetaRegions.containsValue(region)) {
LOG.debug("Scanned region is no longer in map of online " +
"regions or its value has changed");
break;
}
if (tries == numRetries - 1) {
// We ran out of tries. Make sure the file system is still
// available
if (!checkFileSystem()) {
continue; // avoid sleeping
}
}
} catch (Exception e) {
@ -673,6 +703,7 @@ HMasterRegionInterface {
// at least log it rather than go out silently.
LOG.error("Unexpected exception", e);
}
// Sleep before going around again.
sleeper.sleep();
}
}
@ -710,9 +741,8 @@ HMasterRegionInterface {
* regions. This wakes up any threads that were waiting for this to happen.
*/
private synchronized boolean metaRegionsScanned() {
if (!rootScanned ||
if (!rootScanned ||
numberOfMetaRegions.get() != onlineMetaRegions.size()) {
return false;
}
LOG.info("all meta regions scanned");
@ -754,14 +784,16 @@ HMasterRegionInterface {
* <p>Items are removed from this list when a region server reports in that
* the region has been deployed.
*/
SortedMap<Text, HRegionInfo> unassignedRegions;
final SortedMap<Text, HRegionInfo> unassignedRegions =
Collections.synchronizedSortedMap(new TreeMap<Text, HRegionInfo>());
/**
* The 'assignAttempts' table maps from regions to a timestamp that indicates
* the last time we *tried* to assign the region to a RegionServer. If the
* timestamp is out of date, then we can try to reassign it.
*/
Map<Text, Long> assignAttempts;
final Map<Text, Long> assignAttempts =
Collections.synchronizedMap(new HashMap<Text, Long>());
/**
* Regions that have been assigned, and the server has reported that it has
@ -897,28 +929,11 @@ HMasterRegionInterface {
this.rootScannerThread = new RootScanner();
// Scans the meta table
this.numberOfMetaRegions = new AtomicInteger();
this.metaRegionsToScan = new LinkedBlockingQueue<MetaRegion>();
this.onlineMetaRegions =
Collections.synchronizedSortedMap(new TreeMap<Text, MetaRegion>());
this.initialMetaScanComplete = false;
this.metaScannerThread = new MetaScanner();
this.unassignedRegions =
Collections.synchronizedSortedMap(new TreeMap<Text, HRegionInfo>());
this.unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
HGlobals.rootRegionInfo);
this.assignAttempts =
Collections.synchronizedMap(new HashMap<Text, Long>());
this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
Long.valueOf(0L));
unassignRootRegion();
this.pendingRegions =
Collections.synchronizedSet(new HashSet<Text>());
@ -943,6 +958,22 @@ HMasterRegionInterface {
this.closed.set(false);
LOG.info("HMaster initialized on " + this.address.toString());
}
/*
* Unassign the root region.
* This method would be used in case where root region server had died
* without reporting in. Currently, we just flounder and never recover. We
* could 'notice' dead region server in root scanner -- if we failed access
* multiple times -- but reassigning root is catastrophic.
*/
void unassignRootRegion() {
this.rootRegionLocation.set(null);
this.unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
HGlobals.rootRegionInfo);
this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
Long.valueOf(0L));
// TODO: If the old root region server had a log, it needs splitting.
}
/**
* Checks to see if the file system is still accessible.
@ -1271,17 +1302,13 @@ HMasterRegionInterface {
LOG.info("Region server " + serverName +
": MSG_REPORT_EXITING -- lease cancelled");
// Get all the regions the server was serving reassigned
// (if we are not shutting down).
if (!closed.get()) {
for (int i = 1; i < msgs.length; i++) {
HRegionInfo info = msgs[i].getRegionInfo();
if (info.tableDesc.getName().equals(ROOT_TABLE_NAME)) {
rootRegionLocation.set(null);
} else if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
onlineMetaRegions.remove(info.getStartKey());
}
@ -1448,14 +1475,11 @@ HMasterRegionInterface {
} else {
LOG.info(info.getServerAddress().toString() + " serving " +
region.regionName);
// Remove from unassigned list so we don't assign it to someone else
this.unassignedRegions.remove(region.regionName);
this.assignAttempts.remove(region.regionName);
if (region.regionName.compareTo(
HGlobals.rootRegionInfo.regionName) == 0) {
// Store the Root Region location (in memory)
synchronized (rootRegionLocation) {
this.rootRegionLocation.
@ -2226,30 +2250,22 @@ HMasterRegionInterface {
* root region which is handled specially.
*/
private class PendingOpenReport extends PendingOperation {
private boolean rootRegion;
private HRegionInfo region;
private HServerAddress serverAddress;
private byte [] startCode;
private final boolean rootRegion;
private final HRegionInfo region;
private final HServerAddress serverAddress;
private final byte [] startCode;
PendingOpenReport(HServerInfo info, HRegionInfo region)
throws IOException {
if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
// The region which just came on-line is a META region.
// We need to look in the ROOT region for its information.
this.rootRegion = true;
} else {
// Just an ordinary region. Look for it in the META table.
this.rootRegion = false;
}
// If true, the region which just came on-line is a META region.
// We need to look in the ROOT region for its information. Otherwise,
// its just an ordinary region. Look for it in the META table.
this.rootRegion = region.tableDesc.getName().equals(META_TABLE_NAME);
this.region = region;
this.serverAddress = info.getServerAddress();
this.startCode = Writables.longToBytes(info.getStartCode());
}
/** {@inheritDoc} */
@Override
public String toString() {
return "PendingOpenOperation from " + serverAddress.toString();
@ -2261,15 +2277,15 @@ HMasterRegionInterface {
if (closed.get()) {
return true;
}
LOG.info(region.getRegionName() + " open on " +
LOG.info(region.toString() + " open on " +
this.serverAddress.toString());
// Register the newly-available Region's location.
Text metaRegionName;
HRegionInterface server;
if (rootRegion) {
if (this.rootRegion) {
if (rootRegionLocation.get() == null || !rootScanned) {
// We can't proceed until the root region is online and has been scanned
// We can't proceed until root region is online and scanned
if (LOG.isDebugEnabled()) {
LOG.debug("root region: " +
((rootRegionLocation.get() != null)?
@ -2283,12 +2299,10 @@ HMasterRegionInterface {
} else {
if (!rootScanned ||
numberOfMetaRegions.get() != onlineMetaRegions.size()) {
// We can't proceed because not all of the meta regions are online.
// We can't block either because that would prevent the meta region
// online message from being processed. So return false to have this
// operation requeued.
if (LOG.isDebugEnabled()) {
LOG.debug("Requeuing open because rootScanned: " +
rootScanned + ", numberOfMetaRegions: " +
@ -2298,21 +2312,18 @@ HMasterRegionInterface {
return false;
}
MetaRegion r = null;
if (onlineMetaRegions.containsKey(region.getRegionName())) {
r = onlineMetaRegions.get(region.getRegionName());
} else {
r = onlineMetaRegions.get(onlineMetaRegions.headMap(
region.getRegionName()).lastKey());
}
MetaRegion r = onlineMetaRegions.containsKey(region.getRegionName())?
onlineMetaRegions.get(region.getRegionName()):
onlineMetaRegions.get(onlineMetaRegions.
headMap(region.getRegionName()).lastKey());
metaRegionName = r.getRegionName();
server = connection.getHRegionConnection(r.getServer());
}
LOG.info("updating row " + region.getRegionName() + " in table " +
metaRegionName + " with startcode " +
Writables.bytesToLong(this.startCode) + " and server "+
serverAddress.toString());
try {
BatchUpdate b = new BatchUpdate(rand.nextLong());
long lockid = b.startUpdate(region.getRegionName());
@ -2320,34 +2331,28 @@ HMasterRegionInterface {
Writables.stringToBytes(serverAddress.toString()));
b.put(lockid, COL_STARTCODE, startCode);
server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
// It's a meta region.
MetaRegion m =
new MetaRegion(serverAddress, region.regionName, region.startKey);
MetaRegion m = new MetaRegion(this.serverAddress,
this.region.regionName, this.region.startKey);
if (!initialMetaScanComplete) {
// Put it on the queue to be scanned for the first time.
try {
LOG.debug("Adding " + m.toString() + " to regions to scan");
metaRegionsToScan.put(m);
} catch (InterruptedException e) {
throw new RuntimeException(
"Putting into metaRegionsToScan was interrupted.", e);
}
} else {
// Add it to the online meta regions
onlineMetaRegions.put(region.startKey, m);
LOG.debug("Adding to onlineMetaRegions: " + m.toString());
onlineMetaRegions.put(this.region.startKey, m);
}
}
// If updated successfully, remove from pending list.
pendingRegions.remove(region.getRegionName());
break;
} catch (IOException e) {
if (tries == numRetries - 1) {
throw RemoteExceptionHandler.checkIOException(e);

View File

@ -68,41 +68,12 @@ public class DFSAbort extends HBaseClusterTestCase {
cluster.getDFSCluster().shutdown();
// Now wait for Mini HBase Cluster to shut down
// cluster.join();
join();
threadDumpingJoin();
} catch (Exception e) {
e.printStackTrace();
throw e;
}
}
private void join() {
if (this.cluster.regionThreads != null) {
synchronized(this.cluster.regionThreads) {
for(Thread t: this.cluster.regionThreads) {
join(t);
}
}
}
join(this.cluster.getMasterThread());
}
private void join(final Thread t) {
if (t == null) {
return;
}
for (int i = 0; t.isAlive(); i++) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
LOG.info("Continuing...", e);
}
if (i != 0 && i % 30 == 0) {
ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
"Automatic Stack Trace every 30 seconds waiting on " +
t.getName());
}
}
}
/**
* @param args unused

View File

@ -19,11 +19,19 @@
*/
package org.apache.hadoop.hbase;
import java.io.PrintWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.ReflectionUtils;
/**
* Abstract base class for HBase cluster junit tests. Spins up cluster on
* {@link #setUp()} and takes it down again in {@link #tearDown()}.
*/
public abstract class HBaseClusterTestCase extends HBaseTestCase {
private static final Log LOG =
LogFactory.getLog(HBaseClusterTestCase.class.getName());
protected MiniHBaseCluster cluster;
final boolean miniHdfs;
int regionServers;
@ -69,7 +77,6 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
this.regionServers = 1;
}
/** {@inheritDoc} */
@Override
protected void setUp() throws Exception {
super.setUp();
@ -77,7 +84,6 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
new MiniHBaseCluster(this.conf, this.regionServers, this.miniHdfs);
}
/** {@inheritDoc} */
@Override
protected void tearDown() throws Exception {
super.tearDown();
@ -86,4 +92,41 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
}
HConnectionManager.deleteConnection(conf);
}
}
/**
* Use this utility method debugging why cluster won't go down. On a
* period it throws a thread dump. Method ends when all cluster
* regionservers and master threads are no long alive.
*/
public void threadDumpingJoin() {
if (this.cluster.regionThreads != null) {
synchronized(this.cluster.regionThreads) {
for(Thread t: this.cluster.regionThreads) {
threadDumpingJoin(t);
}
}
}
threadDumpingJoin(this.cluster.getMasterThread());
}
public void threadDumpingJoin(final Thread t) {
if (t == null) {
return;
}
long startTime = System.currentTimeMillis();
while (t.isAlive()) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
LOG.info("Continuing...", e);
}
if (System.currentTimeMillis() - startTime > 60000) {
startTime = System.currentTimeMillis();
ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
"Automatic Stack Trace every 60 seconds waiting on " +
t.getName());
}
}
}
}

View File

@ -21,11 +21,8 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.dfs.MiniDFSCluster;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -38,22 +35,17 @@ import org.apache.hadoop.io.WritableComparable;
/**
* Test HStoreFile
*/
public class TestHStoreFile extends TestCase {
public class TestHStoreFile extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
private static String DIR = "/";
private static final char FIRST_CHAR = 'a';
private static final char LAST_CHAR = 'z';
private MiniDFSCluster cluster;
private FileSystem fs;
private Configuration conf;
private Path dir = null;
/** {@inheritDoc} */
@Override
public void setUp() throws Exception {
super.setUp();
this.conf = new HBaseConfiguration();
this.cluster = null;
this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
this.fs = cluster.getFileSystem();
this.dir = new Path(DIR, getName());

View File

@ -30,15 +30,15 @@ import org.apache.hadoop.io.Text;
* Tests region server failover when a region server exits.
*/
public class TestRegionServerAbort extends HBaseClusterTestCase {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
private HTable table;
final Log LOG = LogFactory.getLog(this.getClass().getName());
HTable table;
/** constructor */
public TestRegionServerAbort() {
super(2);
conf.setInt("ipc.client.timeout", 5000); // reduce client timeout
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
conf.setInt("hbase.client.retries.number", 5); // reduce HBase retries
conf.setInt("ipc.client.timeout", 10000); // reduce client timeout
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
conf.setInt("hbase.client.retries.number", 5); // reduce HBase retries
}
/**
@ -50,14 +50,14 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
@SuppressWarnings("unused")
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
// Put something into the meta table.
String tableName = getName();
final String tableName = getName();
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
// put some values in the table
this.table = new HTable(conf, new Text(tableName));
Text row = new Text("row1");
final Text row = new Text("row1");
long lockid = table.startUpdate(row);
table.put(lockid, HConstants.COLUMN_FAMILY,
tableName.getBytes(HConstants.UTF8_ENCODING));
@ -68,29 +68,45 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
// Now shutdown the region server and wait for it to go down.
this.cluster.abortRegionServer(0);
LOG.info(this.cluster.waitOnRegionServer(0) + " has been shutdown");
HScannerInterface scanner = null;
try {
// Verify that the client can find the data after the region has moved
// to a different server
scanner =
table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
LOG.info("Obtained scanner " + scanner);
HStoreKey key = new HStoreKey();
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
while (scanner.next(key, results)) {
assertTrue(key.getRow().equals(row));
assertEquals(1, results.size());
byte[] bytes = results.get(HConstants.COLUMN_FAMILY);
assertNotNull(bytes);
assertTrue(tableName.equals(new String(bytes,
HConstants.UTF8_ENCODING)));
// Run verification in a thread so I can concurrently run a thread-dumper
// while we're waiting (because in this test sometimes the meta scanner
// looks to be be stuck).
Runnable runnable = new Runnable() {
public void run() {
HScannerInterface scanner = null;
try {
// Verify that the client can find the data after the region has moved
// to a different server
scanner =
table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
LOG.info("Obtained scanner " + scanner);
HStoreKey key = new HStoreKey();
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
while (scanner.next(key, results)) {
assertTrue(key.getRow().equals(row));
assertEquals(1, results.size());
byte[] bytes = results.get(HConstants.COLUMN_FAMILY);
assertNotNull(bytes);
assertTrue(tableName.equals(new String(bytes,
HConstants.UTF8_ENCODING)));
}
LOG.info("Success!");
} catch (IOException e) {
e.printStackTrace();
} finally {
if (scanner != null) {
LOG.info("Closing scanner " + scanner);
try {
scanner.close();
} catch (IOException e) {
e.printStackTrace();
}
}
}
}
LOG.info("Success!");
} finally {
if (scanner != null) {
LOG.info("Closing scanner " + scanner);
scanner.close();
}
}
};
Thread t = new Thread(runnable);
t.start();
threadDumpingJoin(t);
}
}