HBASE-936 REST Interface: enable to get numbers of rows from scanner interface

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@708631 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2008-10-28 18:51:46 +00:00
parent c18a713726
commit cb41078c8d
3 changed files with 53 additions and 23 deletions

View File

@ -78,6 +78,8 @@ Release 0.19.0 - Unreleased
its used checking batch size its used checking batch size
HBASE-959 Be able to get multiple RowResult at one time from client side HBASE-959 Be able to get multiple RowResult at one time from client side
(Sishen Freecity via Stack) (Sishen Freecity via Stack)
HBASE-936 REST Interface: enable get number of rows from scanner interface
(Sishen Freecity via Stack)
NEW FEATURES NEW FEATURES
HBASE-875 Use MurmurHash instead of JenkinsHash [in bloomfilters] HBASE-875 Use MurmurHash instead of JenkinsHash [in bloomfilters]

View File

@ -25,6 +25,7 @@ import java.io.UnsupportedEncodingException;
import java.net.URLDecoder; import java.net.URLDecoder;
import java.util.Map; import java.util.Map;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse; import javax.servlet.http.HttpServletResponse;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -56,6 +57,8 @@ public abstract class GenericHandler {
protected static final String ROW = "row"; protected static final String ROW = "row";
protected static final String REGIONS = "regions"; protected static final String REGIONS = "regions";
protected static final String VERSION = "version"; protected static final String VERSION = "version";
protected static final String OFFSET = "offset";
protected static final String LIMIT = "limit";
protected final Log LOG = LogFactory.getLog(this.getClass()); protected final Log LOG = LogFactory.getLog(this.getClass());

View File

@ -21,7 +21,9 @@ package org.apache.hadoop.hbase.rest;
import java.io.IOException; import java.io.IOException;
import java.net.URLDecoder; import java.net.URLDecoder;
import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import javax.servlet.ServletException; import javax.servlet.ServletException;
@ -37,14 +39,14 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JenkinsHash; import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.RowResult;
import org.mortbay.servlet.MultiPartResponse;
import org.znerd.xmlenc.XMLOutputter; import org.znerd.xmlenc.XMLOutputter;
/** /**
* ScannderHandler fields all scanner related requests. * ScannderHandler fields all scanner related requests.
*/ */
public class ScannerHandler extends GenericHandler { public class ScannerHandler extends GenericHandler {
private static final String ROWS = "rows";
public ScannerHandler(HBaseConfiguration conf, HBaseAdmin admin) public ScannerHandler(HBaseConfiguration conf, HBaseAdmin admin)
throws ServletException{ throws ServletException{
super(conf, admin); super(conf, admin);
@ -52,20 +54,24 @@ public class ScannerHandler extends GenericHandler {
private class ScannerRecord { private class ScannerRecord {
private final Scanner scanner; private final Scanner scanner;
private RowResult next; private List<RowResult> nextRows;
ScannerRecord(final Scanner s) { ScannerRecord(final Scanner s) {
this.scanner = s; this.scanner = s;
nextRows = new ArrayList<RowResult>();
} }
public Scanner getScanner() { public Scanner getScanner() {
return this.scanner; return this.scanner;
} }
public boolean hasNext() throws IOException { public boolean hasNext(int nbRows) throws IOException {
if (next == null) { if (nextRows.size() < nbRows) {
next = scanner.next(); RowResult[] results = scanner.next(nbRows - nextRows.size());
return next != null; for (RowResult result : results) {
nextRows.add(result);
}
return nextRows.size() > 0;
} else { } else {
return true; return true;
} }
@ -76,12 +82,12 @@ public class ScannerHandler extends GenericHandler {
* @return Null if finished, RowResult otherwise * @return Null if finished, RowResult otherwise
* @throws IOException * @throws IOException
*/ */
public RowResult next() throws IOException { public RowResult[] next(int nbRows) throws IOException {
if (!hasNext()) { if (!hasNext(nbRows)) {
return null; return null;
} }
RowResult temp = next; RowResult[] temp = nextRows.toArray(new RowResult[nextRows.size()]);
next = null; nextRows.clear();
return temp; return temp;
} }
} }
@ -142,10 +148,15 @@ public class ScannerHandler extends GenericHandler {
return; return;
} }
if (sr.hasNext()) { String limitString = request.getParameter(LIMIT);
int limit = 1;
if (limitString != null && limitString.length() > 0) {
limit = Integer.valueOf(limitString);
}
if (sr.hasNext(limit)) {
switch (ContentType.getContentType(request.getHeader(ACCEPT))) { switch (ContentType.getContentType(request.getHeader(ACCEPT))) {
case XML: case XML:
outputScannerEntryXML(response, sr); outputScannerEntryXML(response, sr, limit);
break; break;
case MIME: case MIME:
/* outputScannerEntryMime(response, sr);*/ /* outputScannerEntryMime(response, sr);*/
@ -162,24 +173,38 @@ public class ScannerHandler extends GenericHandler {
} }
private void outputScannerEntryXML(final HttpServletResponse response, private void outputScannerEntryXML(final HttpServletResponse response,
final ScannerRecord sr) final ScannerRecord sr, int limit)
throws IOException { throws IOException {
RowResult rowResult = sr.next();
// respond with a 200 and Content-type: text/xml // respond with a 200 and Content-type: text/xml
setResponseHeader(response, 200, ContentType.XML.toString()); setResponseHeader(response, 200, ContentType.XML.toString());
// setup an xml outputter // setup an xml outputter
XMLOutputter outputter = getXMLOutputter(response.getWriter()); XMLOutputter outputter = getXMLOutputter(response.getWriter());
outputter.startTag(ROW); boolean rows = false;
// write the row key if (limit > 1) {
doElement(outputter, "name", outputter.startTag(ROWS);
org.apache.hadoop.hbase.util.Base64.encodeBytes(rowResult.getRow())); rows = true;
}
RowResult[] rowResults = sr.next(limit);
for (RowResult rowResult: rowResults) {
outputter.startTag(ROW);
// write the row key
doElement(outputter, "name",
org.apache.hadoop.hbase.util.Base64.encodeBytes(rowResult.getRow()));
outputColumnsXml(outputter, rowResult);
outputter.endTag();
}
if (rows) {
outputter.endTag();
}
outputColumnsXml(outputter, rowResult);
outputter.endTag();
outputter.endDocument(); outputter.endDocument();
outputter.getWriter().close(); outputter.getWriter().close();
} }