HBASE-2436 [stargate] update Jersey and JSON depedencies
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@933455 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
adbb2c58f8
commit
469b8a888b
|
@ -16,7 +16,7 @@
|
|||
<properties>
|
||||
<jsr311.version>1.1.1</jsr311.version>
|
||||
<protobuf.version>2.3.0</protobuf.version>
|
||||
<jersey.version>1.1.4.1</jersey.version>
|
||||
<jersey.version>1.1.5.1</jersey.version>
|
||||
<json.version>20090211</json.version>
|
||||
<hsqldb.version>1.8.0.10</hsqldb.version>
|
||||
<commons-httpclient.version>3.0.1</commons-httpclient.version>
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
|
||||
package org.apache.hadoop.hbase.stargate;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.util.ArrayList;
|
||||
|
@ -29,6 +30,11 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import javax.xml.bind.JAXBException;
|
||||
import javax.xml.bind.annotation.XmlAttribute;
|
||||
import javax.xml.bind.annotation.XmlElement;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
|
@ -49,6 +55,7 @@ import org.apache.hadoop.hbase.stargate.util.SoftUserData;
|
|||
import org.apache.hadoop.hbase.stargate.util.UserData;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
|
||||
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
@ -63,8 +70,8 @@ import org.apache.zookeeper.Watcher.Event.KeeperState;
|
|||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import org.json.JSONStringer;
|
||||
|
||||
import com.sun.jersey.api.json.JSONJAXBContext;
|
||||
import com.sun.jersey.api.json.JSONMarshaller;
|
||||
import com.sun.jersey.server.impl.container.servlet.ServletAdaptor;
|
||||
|
||||
/**
|
||||
|
@ -78,34 +85,46 @@ public class RESTServlet extends ServletAdaptor
|
|||
|
||||
private static RESTServlet instance;
|
||||
|
||||
@XmlRootElement(name="status")
|
||||
static class StatusModel {
|
||||
@XmlAttribute long requests;
|
||||
@XmlElement List<String> connectors = new ArrayList<String>();
|
||||
public void addConnector(String host, int port) {
|
||||
connectors.add(host + ":" + Integer.toString(port));
|
||||
}
|
||||
}
|
||||
|
||||
class StatusReporter extends Chore {
|
||||
|
||||
public StatusReporter(int period, AtomicBoolean stopping) {
|
||||
final JSONJAXBContext context;
|
||||
final JSONMarshaller marshaller;
|
||||
|
||||
public StatusReporter(int period, AtomicBoolean stopping)
|
||||
throws IOException {
|
||||
super(period, stopping);
|
||||
try {
|
||||
context = new JSONJAXBContext(StatusModel.class);
|
||||
marshaller = context.createJSONMarshaller();
|
||||
} catch (JAXBException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void chore() {
|
||||
if (wrapper != null) try {
|
||||
JSONStringer status = new JSONStringer();
|
||||
status.object();
|
||||
status.key("requests").value(metrics.getRequests());
|
||||
status.key("connectors").array();
|
||||
StatusModel model = new StatusModel();
|
||||
model.requests = (long)metrics.getRequests();
|
||||
for (Pair<String,Integer> e: connectors) {
|
||||
status.object()
|
||||
.key("host").value(e.getFirst())
|
||||
.key("port").value(e.getSecond())
|
||||
.endObject();
|
||||
model.addConnector(e.getFirst(), e.getSecond());
|
||||
}
|
||||
status.endArray();
|
||||
status.endObject();
|
||||
updateNode(wrapper, znode, CreateMode.EPHEMERAL,
|
||||
Bytes.toBytes(status.toString()));
|
||||
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||
marshaller.marshallToJSON(model, os);
|
||||
ensureExists(znode, CreateMode.EPHEMERAL, os.toByteArray());
|
||||
} catch (Exception e) {
|
||||
LOG.error(StringUtils.stringifyException(e));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
final String znode = INSTANCE_ZNODE_ROOT + "/" + System.currentTimeMillis();
|
||||
|
@ -133,49 +152,38 @@ public class RESTServlet extends ServletAdaptor
|
|||
return instance;
|
||||
}
|
||||
|
||||
static boolean ensureExists(final ZooKeeperWrapper zkw, final String znode,
|
||||
final CreateMode mode) throws IOException {
|
||||
ZooKeeper zk = zkw.getZooKeeper();
|
||||
private boolean ensureExists(final String znode, final CreateMode mode,
|
||||
final byte[] data) {
|
||||
try {
|
||||
ZooKeeper zk = wrapper.getZooKeeper();
|
||||
Stat stat = zk.exists(znode, false);
|
||||
if (stat != null) {
|
||||
zk.setData(znode, data, -1);
|
||||
return true;
|
||||
}
|
||||
zk.create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE, mode);
|
||||
LOG.debug("Created ZNode " + znode);
|
||||
zk.create(znode, data, Ids.OPEN_ACL_UNSAFE, mode);
|
||||
LOG.info("Created ZNode " + znode);
|
||||
return true;
|
||||
} catch (KeeperException.NodeExistsException e) {
|
||||
return true; // ok, move on.
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
return ensureParentExists(zkw, znode, mode) &&
|
||||
ensureExists(zkw, znode, mode);
|
||||
return ensureParentExists(znode, CreateMode.PERSISTENT, new byte[]{}) &&
|
||||
ensureExists(znode, mode, data);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
LOG.warn(StringUtils.stringifyException(e));
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException(e);
|
||||
LOG.warn(StringUtils.stringifyException(e));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
static boolean ensureParentExists(final ZooKeeperWrapper zkw,
|
||||
final String znode, final CreateMode mode) throws IOException {
|
||||
int index = znode.lastIndexOf("/");
|
||||
private boolean ensureParentExists(final String znode, final CreateMode mode,
|
||||
final byte[] data) {
|
||||
int index = znode.lastIndexOf('/');
|
||||
if (index <= 0) { // Parent is root, which always exists.
|
||||
return true;
|
||||
}
|
||||
return ensureExists(zkw, znode.substring(0, index), mode);
|
||||
}
|
||||
|
||||
static void updateNode(final ZooKeeperWrapper zkw, final String znode,
|
||||
final CreateMode mode, final byte[] data) throws IOException {
|
||||
ensureExists(zkw, znode, mode);
|
||||
ZooKeeper zk = zkw.getZooKeeper();
|
||||
try {
|
||||
zk.setData(znode, data, -1);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
return ensureExists(znode.substring(0, index), mode, data);
|
||||
}
|
||||
|
||||
ZooKeeperWrapper initZooKeeperWrapper() throws IOException {
|
||||
|
@ -191,7 +199,8 @@ public class RESTServlet extends ServletAdaptor
|
|||
this.pool = new HTablePool(conf, 10);
|
||||
this.wrapper = initZooKeeperWrapper();
|
||||
this.statusReporter = new StatusReporter(
|
||||
conf.getInt(STATUS_REPORT_PERIOD_KEY, 1000 * 60), stopping);
|
||||
conf.getInt(STATUS_REPORT_PERIOD_KEY, 1000 * 30), stopping);
|
||||
Threads.setDaemonThreadRunning(statusReporter, "Stargate.statusReporter");
|
||||
this.multiuser = conf.getBoolean("stargate.multiuser", false);
|
||||
if (this.multiuser) {
|
||||
LOG.info("multiuser mode enabled");
|
||||
|
@ -290,7 +299,7 @@ public class RESTServlet extends ServletAdaptor
|
|||
}
|
||||
|
||||
/**
|
||||
* @param flag true if the servlet should operate in multiuser mode
|
||||
* @param multiuser true if the servlet should operate in multiuser mode
|
||||
*/
|
||||
public void setMultiUser(boolean multiuser) {
|
||||
this.multiuser = multiuser;
|
||||
|
@ -308,11 +317,11 @@ public class RESTServlet extends ServletAdaptor
|
|||
if (className.endsWith(HBCAuthenticator.class.getName()) ||
|
||||
className.endsWith(HTableAuthenticator.class.getName()) ||
|
||||
className.endsWith(JDBCAuthenticator.class.getName())) {
|
||||
Constructor<?> cons = c.getConstructor(Configuration.class);
|
||||
Constructor<?> cons = c.getConstructor(HBaseConfiguration.class);
|
||||
authenticator = (Authenticator)
|
||||
cons.newInstance(new Object[] { conf });
|
||||
} else if (className.endsWith(ZooKeeperAuthenticator.class.getName())) {
|
||||
Constructor<?> cons = c.getConstructor(Configuration.class,
|
||||
Constructor<?> cons = c.getConstructor(HBaseConfiguration.class,
|
||||
ZooKeeperWrapper.class);
|
||||
authenticator = (Authenticator)
|
||||
cons.newInstance(new Object[] { conf, wrapper });
|
||||
|
@ -344,7 +353,7 @@ public class RESTServlet extends ServletAdaptor
|
|||
* @param want the number of tokens desired
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean userRequestLimit(final User user, int want)
|
||||
public boolean userRequestLimit(final User user, int want)
|
||||
throws IOException {
|
||||
if (multiuser) {
|
||||
UserData ud = SoftUserData.get(user);
|
||||
|
|
|
@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.stargate.model.ScannerModel;
|
||||
|
||||
import org.json.JSONObject;
|
||||
|
||||
public abstract class ResultGenerator implements Iterator<KeyValue> {
|
||||
|
||||
public static ResultGenerator fromRowSpec(final String table,
|
||||
|
@ -41,7 +39,7 @@ public abstract class ResultGenerator implements Iterator<KeyValue> {
|
|||
}
|
||||
|
||||
public static Filter buildFilter(final String filter) throws Exception {
|
||||
return ScannerModel.buildFilter(new JSONObject(filter));
|
||||
return ScannerModel.buildFilter(filter);
|
||||
}
|
||||
|
||||
public abstract void putBack(KeyValue kv);
|
||||
|
|
|
@ -20,11 +20,14 @@
|
|||
|
||||
package org.apache.hadoop.hbase.stargate.auth;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import javax.xml.bind.annotation.XmlAttribute;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.stargate.Constants;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.stargate.User;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
|
||||
|
||||
|
@ -36,7 +39,9 @@ import org.apache.zookeeper.ZooKeeper;
|
|||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import org.json.JSONObject;
|
||||
import com.sun.jersey.api.json.JSONConfiguration;
|
||||
import com.sun.jersey.api.json.JSONJAXBContext;
|
||||
import com.sun.jersey.api.json.JSONUnmarshaller;
|
||||
|
||||
/**
|
||||
* A simple authenticator module for ZooKeeper.
|
||||
|
@ -51,8 +56,17 @@ import org.json.JSONObject;
|
|||
public class ZooKeeperAuthenticator extends Authenticator
|
||||
implements Constants {
|
||||
|
||||
@XmlRootElement(name="user")
|
||||
static class UserModel {
|
||||
@XmlAttribute public String name;
|
||||
@XmlAttribute public boolean admin = false;
|
||||
@XmlAttribute public boolean disabled = false;
|
||||
}
|
||||
|
||||
final String usersZNode;
|
||||
ZooKeeperWrapper wrapper;
|
||||
final JSONJAXBContext context;
|
||||
final JSONUnmarshaller unmarshaller;
|
||||
|
||||
private boolean ensureParentExists(final String znode) {
|
||||
int index = znode.lastIndexOf("/");
|
||||
|
@ -98,11 +112,19 @@ public class ZooKeeperAuthenticator extends Authenticator
|
|||
* Constructor
|
||||
* @param conf
|
||||
* @param wrapper
|
||||
* @throws IOException
|
||||
*/
|
||||
public ZooKeeperAuthenticator(Configuration conf,
|
||||
ZooKeeperWrapper wrapper) {
|
||||
ZooKeeperWrapper wrapper) throws IOException {
|
||||
this.usersZNode = conf.get("stargate.auth.zk.users", USERS_ZNODE_ROOT);
|
||||
this.wrapper = wrapper;
|
||||
try {
|
||||
this.context = new JSONJAXBContext(JSONConfiguration.natural().build(),
|
||||
UserModel.class);
|
||||
this.unmarshaller = context.createJSONUnmarshaller();
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -113,16 +135,10 @@ public class ZooKeeperAuthenticator extends Authenticator
|
|||
if (data == null) {
|
||||
return null;
|
||||
}
|
||||
JSONObject o = new JSONObject(Bytes.toString(data));
|
||||
if (!o.has("name")) {
|
||||
throw new IOException("invalid record, missing 'name'");
|
||||
}
|
||||
String name = o.getString("name");
|
||||
boolean admin = false;
|
||||
if (o.has("admin")) { admin = o.getBoolean("admin"); }
|
||||
boolean disabled = false;
|
||||
if (o.has("disabled")) { disabled = o.getBoolean("disabled"); }
|
||||
return new User(name, token, admin, disabled);
|
||||
UserModel model =
|
||||
unmarshaller.unmarshalFromJSON(new ByteArrayInputStream(data),
|
||||
UserModel.class);
|
||||
return new User(model.name, token, model.admin, model.disabled);
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
|
|
|
@ -22,6 +22,8 @@ package org.apache.hadoop.hbase.stargate.model;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.io.StringReader;
|
||||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -56,12 +58,13 @@ import org.apache.hadoop.hbase.stargate.protobuf.generated.ScannerMessage.Scanne
|
|||
import org.apache.hadoop.hbase.util.Base64;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.json.JSONArray;
|
||||
import org.json.JSONObject;
|
||||
import org.json.JSONStringer;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
import com.sun.jersey.api.json.JSONConfiguration;
|
||||
import com.sun.jersey.api.json.JSONJAXBContext;
|
||||
import com.sun.jersey.api.json.JSONMarshaller;
|
||||
import com.sun.jersey.api.json.JSONUnmarshaller;
|
||||
|
||||
/**
|
||||
* A representation of Scanner parameters.
|
||||
*
|
||||
|
@ -83,28 +86,6 @@ import com.google.protobuf.ByteString;
|
|||
@XmlRootElement(name="Scanner")
|
||||
public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||
|
||||
static enum FilterType {
|
||||
ColumnCountGetFilter,
|
||||
FilterList,
|
||||
FirstKeyOnlyFilter,
|
||||
InclusiveStopFilter,
|
||||
PageFilter,
|
||||
PrefixFilter,
|
||||
QualifierFilter,
|
||||
RowFilter,
|
||||
SingleColumnValueFilter,
|
||||
SkipFilter,
|
||||
ValueFilter,
|
||||
WhileMatchFilter
|
||||
}
|
||||
|
||||
static enum ComparatorType {
|
||||
BinaryComparator,
|
||||
BinaryPrefixComparator,
|
||||
RegexStringComparator,
|
||||
SubstringComparator
|
||||
}
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private byte[] startRow = HConstants.EMPTY_START_ROW;
|
||||
|
@ -116,215 +97,248 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
|||
private String filter = null;
|
||||
private int maxVersions = Integer.MAX_VALUE;
|
||||
|
||||
/**
|
||||
* @param o the JSONObject under construction
|
||||
* @return the JSONObject under construction
|
||||
* @throws Exception
|
||||
*/
|
||||
public static WritableByteArrayComparable
|
||||
buildWritableByteArrayComparable(final JSONObject o) throws Exception {
|
||||
String type = o.getString("type");
|
||||
String value = o.getString("value");
|
||||
WritableByteArrayComparable comparator;
|
||||
switch (ComparatorType.valueOf(type)) {
|
||||
case BinaryComparator: {
|
||||
comparator = new BinaryComparator(Base64.decode(value));
|
||||
} break;
|
||||
case BinaryPrefixComparator: {
|
||||
comparator = new BinaryPrefixComparator(Base64.decode(value));
|
||||
} break;
|
||||
case RegexStringComparator: {
|
||||
comparator = new RegexStringComparator(value);
|
||||
} break;
|
||||
case SubstringComparator: {
|
||||
comparator = new SubstringComparator(value);
|
||||
} break;
|
||||
default: {
|
||||
throw new RuntimeException("unhandled comparator type: " + type);
|
||||
@XmlRootElement
|
||||
static class FilterModel {
|
||||
|
||||
@XmlRootElement
|
||||
static class WritableByteArrayComparableModel {
|
||||
@XmlAttribute public String type;
|
||||
@XmlAttribute public String value;
|
||||
|
||||
static enum ComparatorType {
|
||||
BinaryComparator,
|
||||
BinaryPrefixComparator,
|
||||
RegexStringComparator,
|
||||
SubstringComparator
|
||||
}
|
||||
|
||||
public WritableByteArrayComparableModel() { }
|
||||
|
||||
public WritableByteArrayComparableModel(
|
||||
WritableByteArrayComparable comparator) {
|
||||
String typeName = comparator.getClass().getSimpleName();
|
||||
ComparatorType type = ComparatorType.valueOf(typeName);
|
||||
this.type = typeName;
|
||||
switch (type) {
|
||||
case BinaryComparator:
|
||||
case BinaryPrefixComparator:
|
||||
this.value = Base64.encodeBytes(comparator.getValue());
|
||||
break;
|
||||
case RegexStringComparator:
|
||||
case SubstringComparator:
|
||||
this.value = Bytes.toString(comparator.getValue());
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("unhandled filter type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
public WritableByteArrayComparable build() {
|
||||
WritableByteArrayComparable comparator;
|
||||
switch (ComparatorType.valueOf(type)) {
|
||||
case BinaryComparator: {
|
||||
comparator = new BinaryComparator(Base64.decode(value));
|
||||
} break;
|
||||
case BinaryPrefixComparator: {
|
||||
comparator = new BinaryPrefixComparator(Base64.decode(value));
|
||||
} break;
|
||||
case RegexStringComparator: {
|
||||
comparator = new RegexStringComparator(value);
|
||||
} break;
|
||||
case SubstringComparator: {
|
||||
comparator = new SubstringComparator(value);
|
||||
} break;
|
||||
default: {
|
||||
throw new RuntimeException("unhandled comparator type: " + type);
|
||||
}
|
||||
}
|
||||
return comparator;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// a grab bag of fields, would have been a union if this were C
|
||||
@XmlAttribute public String type = null;
|
||||
@XmlAttribute public String op = null;
|
||||
@XmlElement WritableByteArrayComparableModel comparator = null;
|
||||
@XmlAttribute public String value = null;
|
||||
@XmlElement public List<FilterModel> filters = null;
|
||||
@XmlAttribute public Integer limit = null;
|
||||
@XmlAttribute public String family = null;
|
||||
@XmlAttribute public String qualifier = null;
|
||||
@XmlAttribute public Boolean ifMissing = null;
|
||||
@XmlAttribute public Boolean latestVersion = null;
|
||||
|
||||
static enum FilterType {
|
||||
ColumnCountGetFilter,
|
||||
FilterList,
|
||||
FirstKeyOnlyFilter,
|
||||
InclusiveStopFilter,
|
||||
PageFilter,
|
||||
PrefixFilter,
|
||||
QualifierFilter,
|
||||
RowFilter,
|
||||
SingleColumnValueFilter,
|
||||
SkipFilter,
|
||||
ValueFilter,
|
||||
WhileMatchFilter
|
||||
}
|
||||
|
||||
public FilterModel() { }
|
||||
|
||||
public FilterModel(Filter filter) {
|
||||
String typeName = filter.getClass().getSimpleName();
|
||||
FilterType type = FilterType.valueOf(typeName);
|
||||
this.type = typeName;
|
||||
switch (type) {
|
||||
case ColumnCountGetFilter:
|
||||
this.limit = ((ColumnCountGetFilter)filter).getLimit();
|
||||
break;
|
||||
case FilterList:
|
||||
this.op = ((FilterList)filter).getOperator().toString();
|
||||
this.filters = new ArrayList<FilterModel>();
|
||||
for (Filter child: ((FilterList)filter).getFilters()) {
|
||||
this.filters.add(new FilterModel(child));
|
||||
}
|
||||
break;
|
||||
case FirstKeyOnlyFilter:
|
||||
break;
|
||||
case InclusiveStopFilter:
|
||||
this.value =
|
||||
Base64.encodeBytes(((InclusiveStopFilter)filter).getStopRowKey());
|
||||
break;
|
||||
case PageFilter:
|
||||
this.value = Long.toString(((PageFilter)filter).getPageSize());
|
||||
break;
|
||||
case PrefixFilter:
|
||||
this.value = Base64.encodeBytes(((PrefixFilter)filter).getPrefix());
|
||||
break;
|
||||
case QualifierFilter:
|
||||
case RowFilter:
|
||||
case ValueFilter:
|
||||
this.op = ((CompareFilter)filter).getOperator().toString();
|
||||
this.comparator =
|
||||
new WritableByteArrayComparableModel(
|
||||
((CompareFilter)filter).getComparator());
|
||||
break;
|
||||
case SingleColumnValueFilter: {
|
||||
SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter;
|
||||
this.family = Base64.encodeBytes(scvf.getFamily());
|
||||
byte[] qualifier = scvf.getQualifier();
|
||||
if (qualifier != null) {
|
||||
this.qualifier = Base64.encodeBytes(qualifier);
|
||||
}
|
||||
this.op = scvf.getOperator().toString();
|
||||
this.comparator =
|
||||
new WritableByteArrayComparableModel(scvf.getComparator());
|
||||
if (scvf.getFilterIfMissing()) {
|
||||
this.ifMissing = true;
|
||||
}
|
||||
if (scvf.getLatestVersionOnly()) {
|
||||
this.latestVersion = true;
|
||||
}
|
||||
} break;
|
||||
case SkipFilter:
|
||||
this.filters = new ArrayList<FilterModel>();
|
||||
this.filters.add(new FilterModel(((SkipFilter)filter).getFilter()));
|
||||
break;
|
||||
case WhileMatchFilter:
|
||||
this.filters = new ArrayList<FilterModel>();
|
||||
this.filters.add(
|
||||
new FilterModel(((WhileMatchFilter)filter).getFilter()));
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("unhandled filter type " + type);
|
||||
}
|
||||
}
|
||||
return comparator;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param o the JSONObject under construction
|
||||
* @return the JSONObject under construction
|
||||
* @throws Exception
|
||||
*/
|
||||
public static Filter buildFilter(final JSONObject o) throws Exception {
|
||||
String type = o.getString("type");
|
||||
Filter filter;
|
||||
switch (FilterType.valueOf(type)) {
|
||||
public Filter build() {
|
||||
Filter filter;
|
||||
switch (FilterType.valueOf(type)) {
|
||||
case ColumnCountGetFilter: {
|
||||
filter = new ColumnCountGetFilter(o.getInt("limit"));
|
||||
filter = new ColumnCountGetFilter(limit);
|
||||
} break;
|
||||
case FilterList: {
|
||||
JSONArray arr = o.getJSONArray("filters");
|
||||
List<Filter> filters = new ArrayList<Filter>(arr.length());
|
||||
for (int i = 0; i < arr.length(); i++) {
|
||||
filters.add(buildFilter(arr.getJSONObject(i)));
|
||||
List<Filter> list = new ArrayList<Filter>();
|
||||
for (FilterModel model: filters) {
|
||||
list.add(model.build());
|
||||
}
|
||||
filter = new FilterList(
|
||||
FilterList.Operator.valueOf(o.getString("op")),
|
||||
filters);
|
||||
filter = new FilterList(FilterList.Operator.valueOf(op), list);
|
||||
} break;
|
||||
case FirstKeyOnlyFilter: {
|
||||
filter = new FirstKeyOnlyFilter();
|
||||
} break;
|
||||
case InclusiveStopFilter: {
|
||||
filter = new InclusiveStopFilter(Base64.decode(o.getString("value")));
|
||||
filter = new InclusiveStopFilter(Base64.decode(value));
|
||||
} break;
|
||||
case PageFilter: {
|
||||
filter = new PageFilter(o.getLong("value"));
|
||||
filter = new PageFilter(Long.valueOf(value));
|
||||
} break;
|
||||
case PrefixFilter: {
|
||||
filter = new PrefixFilter(Base64.decode(o.getString("value")));
|
||||
filter = new PrefixFilter(Base64.decode(value));
|
||||
} break;
|
||||
case QualifierFilter: {
|
||||
filter = new QualifierFilter(CompareOp.valueOf(o.getString("op")),
|
||||
buildWritableByteArrayComparable(o.getJSONObject("comparator")));
|
||||
filter = new QualifierFilter(CompareOp.valueOf(op), comparator.build());
|
||||
} break;
|
||||
case RowFilter: {
|
||||
filter = new RowFilter(CompareOp.valueOf(o.getString("op")),
|
||||
buildWritableByteArrayComparable(o.getJSONObject("comparator")));
|
||||
filter = new RowFilter(CompareOp.valueOf(op), comparator.build());
|
||||
} break;
|
||||
case SingleColumnValueFilter: {
|
||||
filter = new SingleColumnValueFilter(
|
||||
Base64.decode(o.getString("family")),
|
||||
o.has("qualifier") ? Base64.decode(o.getString("qualifier")) : null,
|
||||
CompareOp.valueOf(o.getString("op")),
|
||||
buildWritableByteArrayComparable(o.getJSONObject("comparator")));
|
||||
if (o.has("ifMissing")) {
|
||||
((SingleColumnValueFilter)filter)
|
||||
.setFilterIfMissing(o.getBoolean("ifMissing"));
|
||||
filter = new SingleColumnValueFilter(Base64.decode(family),
|
||||
qualifier != null ? Base64.decode(qualifier) : null,
|
||||
CompareOp.valueOf(op), comparator.build());
|
||||
if (ifMissing != null) {
|
||||
((SingleColumnValueFilter)filter).setFilterIfMissing(ifMissing);
|
||||
}
|
||||
if (o.has("latestVersion")) {
|
||||
((SingleColumnValueFilter)filter)
|
||||
.setLatestVersionOnly(o.getBoolean("latestVersion"));
|
||||
if (latestVersion != null) {
|
||||
((SingleColumnValueFilter)filter).setLatestVersionOnly(latestVersion);
|
||||
}
|
||||
} break;
|
||||
case SkipFilter: {
|
||||
filter = new SkipFilter(buildFilter(o.getJSONObject("filter")));
|
||||
filter = new SkipFilter(filters.get(0).build());
|
||||
} break;
|
||||
case ValueFilter: {
|
||||
filter = new ValueFilter(CompareOp.valueOf(o.getString("op")),
|
||||
buildWritableByteArrayComparable(o.getJSONObject("comparator")));
|
||||
filter = new ValueFilter(CompareOp.valueOf(op), comparator.build());
|
||||
} break;
|
||||
case WhileMatchFilter: {
|
||||
filter = new WhileMatchFilter(buildFilter(o.getJSONObject("filter")));
|
||||
filter = new WhileMatchFilter(filters.get(0).build());
|
||||
} break;
|
||||
default: {
|
||||
throw new RuntimeException("unhandled filter type: " + type);
|
||||
}
|
||||
}
|
||||
return filter;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param s the JSONStringer
|
||||
* @param comparator the comparator
|
||||
* @return the JSONStringer
|
||||
* @throws Exception
|
||||
*/
|
||||
public static JSONStringer stringifyComparator(final JSONStringer s,
|
||||
final WritableByteArrayComparable comparator) throws Exception {
|
||||
String typeName = comparator.getClass().getSimpleName();
|
||||
ComparatorType type = ComparatorType.valueOf(typeName);
|
||||
s.object();
|
||||
s.key("type").value(typeName);
|
||||
switch (type) {
|
||||
case BinaryComparator:
|
||||
case BinaryPrefixComparator:
|
||||
s.key("value").value(Base64.encodeBytes(comparator.getValue()));
|
||||
break;
|
||||
case RegexStringComparator:
|
||||
case SubstringComparator:
|
||||
s.key("value").value(Bytes.toString(comparator.getValue()));
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("unhandled filter type: " + type);
|
||||
}
|
||||
return filter;
|
||||
}
|
||||
s.endObject();
|
||||
return s;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* @param s the JSONStringer
|
||||
* @param filter the filter
|
||||
* @return the JSONStringer
|
||||
* @param s the JSON representation of the filter
|
||||
* @return the filter
|
||||
* @throws Exception
|
||||
*/
|
||||
public static JSONStringer stringifyFilter(final JSONStringer s,
|
||||
final Filter filter) throws Exception {
|
||||
String typeName = filter.getClass().getSimpleName();
|
||||
FilterType type;
|
||||
try {
|
||||
type = FilterType.valueOf(typeName);
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new RuntimeException("filter type " + typeName + " not supported");
|
||||
}
|
||||
s.object();
|
||||
s.key("type").value(typeName);
|
||||
switch (type) {
|
||||
case ColumnCountGetFilter:
|
||||
s.key("limit").value(((ColumnCountGetFilter)filter).getLimit());
|
||||
break;
|
||||
case FilterList:
|
||||
s.key("op").value(((FilterList)filter).getOperator().toString());
|
||||
s.key("filters").array();
|
||||
for (Filter child: ((FilterList)filter).getFilters()) {
|
||||
stringifyFilter(s, child);
|
||||
}
|
||||
s.endArray();
|
||||
break;
|
||||
case FirstKeyOnlyFilter:
|
||||
break;
|
||||
case InclusiveStopFilter:
|
||||
s.key("value").value(
|
||||
Base64.encodeBytes(((InclusiveStopFilter)filter).getStopRowKey()));
|
||||
break;
|
||||
case PageFilter:
|
||||
s.key("value").value(((PageFilter)filter).getPageSize());
|
||||
break;
|
||||
case PrefixFilter:
|
||||
s.key("value")
|
||||
.value(Base64.encodeBytes(((PrefixFilter)filter).getPrefix()));
|
||||
break;
|
||||
case QualifierFilter:
|
||||
case RowFilter:
|
||||
case ValueFilter:
|
||||
s.key("op").value(((CompareFilter)filter).getOperator().toString());
|
||||
s.key("comparator");
|
||||
stringifyComparator(s, ((CompareFilter)filter).getComparator());
|
||||
break;
|
||||
case SingleColumnValueFilter: {
|
||||
SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter;
|
||||
s.key("family").value(scvf.getFamily());
|
||||
byte[] qualifier = scvf.getQualifier();
|
||||
if (qualifier != null) {
|
||||
s.key("qualifier").value(qualifier);
|
||||
}
|
||||
s.key("op").value(scvf.getOperator().toString());
|
||||
s.key("comparator");
|
||||
stringifyComparator(s, scvf.getComparator());
|
||||
if (scvf.getFilterIfMissing()) {
|
||||
s.key("ifMissing").value(true);
|
||||
}
|
||||
if (scvf.getLatestVersionOnly()) {
|
||||
s.key("latestVersion").value(true);
|
||||
}
|
||||
} break;
|
||||
case SkipFilter:
|
||||
s.key("filter");
|
||||
stringifyFilter(s, ((SkipFilter)filter).getFilter());
|
||||
break;
|
||||
case WhileMatchFilter:
|
||||
s.key("filter");
|
||||
stringifyFilter(s, ((WhileMatchFilter)filter).getFilter());
|
||||
break;
|
||||
}
|
||||
s.endObject();
|
||||
return s;
|
||||
public static Filter buildFilter(String s) throws Exception {
|
||||
JSONJAXBContext context =
|
||||
new JSONJAXBContext(JSONConfiguration.natural().build(),
|
||||
FilterModel.class);
|
||||
JSONUnmarshaller unmarshaller = context.createJSONUnmarshaller();
|
||||
FilterModel model = unmarshaller.unmarshalFromJSON(new StringReader(s),
|
||||
FilterModel.class);
|
||||
return model.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param filter the filter
|
||||
* @return the JSON representation of the filter
|
||||
* @throws Exception
|
||||
*/
|
||||
public static String stringifyFilter(final Filter filter) throws Exception {
|
||||
JSONJAXBContext context =
|
||||
new JSONJAXBContext(JSONConfiguration.natural().build(),
|
||||
FilterModel.class);
|
||||
JSONMarshaller marshaller = context.createJSONMarshaller();
|
||||
StringWriter writer = new StringWriter();
|
||||
marshaller.marshallToJSON(new FilterModel(filter), writer);
|
||||
return writer.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -353,7 +367,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
|||
}
|
||||
Filter filter = scan.getFilter();
|
||||
if (filter != null) {
|
||||
model.setFilter(stringifyFilter(new JSONStringer(), filter).toString());
|
||||
model.setFilter(stringifyFilter(filter));
|
||||
}
|
||||
return model;
|
||||
}
|
||||
|
|
|
@ -20,15 +20,19 @@
|
|||
|
||||
package org.apache.hadoop.hbase.stargate.auth;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
|
||||
import org.apache.hadoop.hbase.stargate.MiniClusterTestBase;
|
||||
import org.apache.hadoop.hbase.stargate.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.stargate.auth.ZooKeeperAuthenticator.UserModel;
|
||||
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
|
||||
import org.json.JSONStringer;
|
||||
import com.sun.jersey.api.json.JSONConfiguration;
|
||||
import com.sun.jersey.api.json.JSONJAXBContext;
|
||||
import com.sun.jersey.api.json.JSONMarshaller;
|
||||
|
||||
public class TestZooKeeperAuthenticator extends MiniClusterTestBase {
|
||||
|
||||
|
@ -44,38 +48,41 @@ public class TestZooKeeperAuthenticator extends MiniClusterTestBase {
|
|||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
authenticator = new ZooKeeperAuthenticator(conf);
|
||||
ZooKeeper zk = authenticator.wrapper.getZooKeeper();
|
||||
JSONJAXBContext context =
|
||||
new JSONJAXBContext(JSONConfiguration.natural().build(),
|
||||
UserModel.class);
|
||||
JSONMarshaller marshaller = context.createJSONMarshaller();
|
||||
if (zk.exists(ZooKeeperAuthenticator.USERS_ZNODE_ROOT + "/" +
|
||||
ADMIN_TOKEN, null) == null) {
|
||||
UserModel model = new UserModel();
|
||||
model.name = ADMIN_USERNAME;
|
||||
model.admin = true;
|
||||
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||
marshaller.marshallToJSON(model, os);
|
||||
zk.create(ZooKeeperAuthenticator.USERS_ZNODE_ROOT + "/" + ADMIN_TOKEN,
|
||||
Bytes.toBytes(new JSONStringer()
|
||||
.object()
|
||||
.key("name").value(ADMIN_USERNAME)
|
||||
.key("admin").value(true)
|
||||
.endObject().toString()),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
os.toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
}
|
||||
if (zk.exists(ZooKeeperAuthenticator.USERS_ZNODE_ROOT + "/" +
|
||||
USER_TOKEN, null) == null) {
|
||||
UserModel model = new UserModel();
|
||||
model.name = USER_USERNAME;
|
||||
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||
marshaller.marshallToJSON(model, os);
|
||||
zk.create(ZooKeeperAuthenticator.USERS_ZNODE_ROOT + "/" + USER_TOKEN,
|
||||
Bytes.toBytes(new JSONStringer()
|
||||
.object()
|
||||
.key("name").value(USER_USERNAME)
|
||||
.key("disabled").value(false)
|
||||
.endObject().toString()),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
os.toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
}
|
||||
if (zk.exists(ZooKeeperAuthenticator.USERS_ZNODE_ROOT + "/" +
|
||||
DISABLED_TOKEN, null) == null) {
|
||||
UserModel model = new UserModel();
|
||||
model.name = DISABLED_USERNAME;
|
||||
model.disabled = true;
|
||||
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||
marshaller.marshallToJSON(model, os);
|
||||
zk.create(ZooKeeperAuthenticator.USERS_ZNODE_ROOT + "/" +DISABLED_TOKEN,
|
||||
Bytes.toBytes(new JSONStringer()
|
||||
.object()
|
||||
.key("name").value(DISABLED_USERNAME)
|
||||
.key("admin").value(false)
|
||||
.key("disabled").value(true)
|
||||
.endObject().toString()),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
os.toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue