HBASE-3120 [rest] Content transcoding; catch TNFE in scanTransformAttrs

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1029948 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Kyle Purtell 2010-11-02 04:11:04 +00:00
parent b9e98b72a3
commit b77e6f4319
1 changed files with 49 additions and 44 deletions

View File

@ -37,6 +37,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.rest.transform.NullTransform;
@ -131,58 +132,62 @@ public class TableResource extends ResourceBase {
* The attribute key must begin with the string "Transform$".
*/
void scanTransformAttrs() throws IOException {
HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
HTableDescriptor htd = admin.getTableDescriptor(Bytes.toBytes(table));
for (HColumnDescriptor hcd: htd.getFamilies()) {
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
hcd.getValues().entrySet()) {
// does the key start with the transform directive tag?
String key = Bytes.toString(e.getKey().get());
if (!key.startsWith(DIRECTIVE_KEY)) {
// no, skip
continue;
}
// match a comma separated list of one or more directives
byte[] value = e.getValue().get();
Matcher m = DIRECTIVE_PATTERN.matcher(Bytes.toString(value));
while (m.find()) {
byte[] qualifier = HConstants.EMPTY_BYTE_ARRAY;
String s = m.group(1);
if (s.length() > 0 && !s.equals("*")) {
qualifier = Bytes.toBytes(s);
try {
HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
HTableDescriptor htd = admin.getTableDescriptor(Bytes.toBytes(table));
for (HColumnDescriptor hcd: htd.getFamilies()) {
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
hcd.getValues().entrySet()) {
// does the key start with the transform directive tag?
String key = Bytes.toString(e.getKey().get());
if (!key.startsWith(DIRECTIVE_KEY)) {
// no, skip
continue;
}
boolean retry = false;
String className = m.group(2);
while (true) {
try {
// if a transform was previously configured for the qualifier,
// this will simply replace it
setTransform(table, hcd.getName(), qualifier,
(Transform)Class.forName(className).newInstance());
break;
} catch (InstantiationException ex) {
LOG.error(StringUtils.stringifyException(ex));
if (retry) {
// match a comma separated list of one or more directives
byte[] value = e.getValue().get();
Matcher m = DIRECTIVE_PATTERN.matcher(Bytes.toString(value));
while (m.find()) {
byte[] qualifier = HConstants.EMPTY_BYTE_ARRAY;
String s = m.group(1);
if (s.length() > 0 && !s.equals("*")) {
qualifier = Bytes.toBytes(s);
}
boolean retry = false;
String className = m.group(2);
while (true) {
try {
// if a transform was previously configured for the qualifier,
// this will simply replace it
setTransform(table, hcd.getName(), qualifier,
(Transform)Class.forName(className).newInstance());
break;
}
retry = true;
} catch (IllegalAccessException ex) {
LOG.error(StringUtils.stringifyException(ex));
if (retry) {
break;
}
retry = true;
} catch (ClassNotFoundException ex) {
if (retry) {
} catch (InstantiationException ex) {
LOG.error(StringUtils.stringifyException(ex));
break;
if (retry) {
break;
}
retry = true;
} catch (IllegalAccessException ex) {
LOG.error(StringUtils.stringifyException(ex));
if (retry) {
break;
}
retry = true;
} catch (ClassNotFoundException ex) {
if (retry) {
LOG.error(StringUtils.stringifyException(ex));
break;
}
className = "org.apache.hadoop.hbase.rest.transform." + className;
retry = true;
}
className = "org.apache.hadoop.hbase.rest.transform." + className;
retry = true;
}
}
}
}
} catch (TableNotFoundException e) {
// ignore
}
}