HBASE-3904 Addendum that fixes number of retries (Ita Pai)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1145847 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2011-07-13 03:13:10 +00:00
parent 16a366b5cd
commit 1f3a4fefb8
1 changed files with 45 additions and 41 deletions

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.SocketTimeoutException; import java.net.SocketTimeoutException;
import java.util.Arrays; import java.util.Arrays;
import java.util.LinkedList; import java.util.LinkedList;
@ -333,49 +334,52 @@ public class HBaseAdmin implements Abortable, Closeable {
} catch (SocketTimeoutException ste) { } catch (SocketTimeoutException ste) {
LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste); LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste);
} }
int numRegs = splitKeys == null ? 1 : splitKeys.length+1; int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
for (int tries = 0; tries < numRetries; tries++) { int prevRegCount = 0;
try { for (int tries = 0; tries < numRetries; ++tries) {
// Wait for new table to come on-line // Wait for new table to come on-line
final AtomicInteger actualRegCount = new AtomicInteger(0); final AtomicInteger actualRegCount = new AtomicInteger(0);
MetaScannerVisitor visitor = new MetaScannerVisitor() {
MetaScannerVisitor visitor = new MetaScannerVisitor() { @Override
public boolean processRow(Result rowResult) throws IOException { public boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo( HRegionInfo info = Writables.getHRegionInfo(
rowResult.getValue(HConstants.CATALOG_FAMILY, rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER)); HConstants.REGIONINFO_QUALIFIER));
if (!(Bytes.equals(info.getTableDesc().getName(), desc.getName()))) {
if (!(Bytes.equals(info.getTableName(), desc.getName()))) { return false;
return false;
}
String hostAndPort = null;
byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
hostAndPort = Bytes.toString(value);
}
if (!(info.isOffline() || info.isSplit()) && hostAndPort != null) {
actualRegCount.incrementAndGet();
}
return true;
} }
String hostAndPort = null;
}; byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
MetaScanner.metaScan(conf, visitor, desc.getName()); HConstants.SERVER_QUALIFIER);
if (actualRegCount.get() == numRegs) // Make sure that regions are assigned to server
break; if (value != null && value.length > 0) {
} catch (RegionException e) { hostAndPort = Bytes.toString(value);
if (tries == numRetries - 1) { }
// Ran out of tries if (!(info.isOffline() || info.isSplit()) && hostAndPort != null) {
throw e; actualRegCount.incrementAndGet();
}
return true;
} }
} };
try { MetaScanner.metaScan(conf, visitor, desc.getName());
Thread.sleep(getPauseTime(tries)); if (actualRegCount.get() != numRegs) {
} catch (InterruptedException e) { if (tries == numRetries - 1) {
// Just continue; ignore the interruption. throw new RegionOfflineException("Only " + actualRegCount.get() +
" of " + numRegs + " regions are online; retries exhausted.");
}
try { // Sleep
Thread.sleep(getPauseTime(tries));
} catch (InterruptedException e) {
throw new InterruptedIOException("Interrupted when opening" +
" regions; " + actualRegCount.get() + " of " + numRegs +
" regions processed so far");
}
if (actualRegCount.get() > prevRegCount) { // Making progress
prevRegCount = actualRegCount.get();
tries = -1;
}
} else {
return;
} }
} }
} }