HBASE-7803 [REST] Support caching on scan
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1458576 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1d6a98257a
commit
d5b03c3648
|
@ -89,7 +89,7 @@ public class ScannerResource extends ResourceBase {
|
||||||
Filter filter = ScannerResultGenerator.buildFilterFromModel(model);
|
Filter filter = ScannerResultGenerator.buildFilterFromModel(model);
|
||||||
String tableName = tableResource.getName();
|
String tableName = tableResource.getName();
|
||||||
ScannerResultGenerator gen =
|
ScannerResultGenerator gen =
|
||||||
new ScannerResultGenerator(tableName, spec, filter);
|
new ScannerResultGenerator(tableName, spec, filter, model.getCaching());
|
||||||
String id = gen.getID();
|
String id = gen.getID();
|
||||||
ScannerInstanceResource instance =
|
ScannerInstanceResource instance =
|
||||||
new ScannerInstanceResource(tableName, id, gen, model.getBatch());
|
new ScannerInstanceResource(tableName, id, gen, model.getBatch());
|
||||||
|
|
|
@ -59,6 +59,11 @@ public class ScannerResultGenerator extends ResultGenerator {
|
||||||
|
|
||||||
public ScannerResultGenerator(final String tableName, final RowSpec rowspec,
|
public ScannerResultGenerator(final String tableName, final RowSpec rowspec,
|
||||||
final Filter filter) throws IllegalArgumentException, IOException {
|
final Filter filter) throws IllegalArgumentException, IOException {
|
||||||
|
this(tableName, rowspec, filter, -1);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ScannerResultGenerator(final String tableName, final RowSpec rowspec,
|
||||||
|
final Filter filter, final int caching) throws IllegalArgumentException, IOException {
|
||||||
HTablePool pool = RESTServlet.getInstance().getTablePool();
|
HTablePool pool = RESTServlet.getInstance().getTablePool();
|
||||||
HTableInterface table = pool.getTable(tableName);
|
HTableInterface table = pool.getTable(tableName);
|
||||||
try {
|
try {
|
||||||
|
@ -86,6 +91,9 @@ public class ScannerResultGenerator extends ResultGenerator {
|
||||||
}
|
}
|
||||||
// always disable block caching on the cluster when scanning
|
// always disable block caching on the cluster when scanning
|
||||||
scan.setCacheBlocks(false);
|
scan.setCacheBlocks(false);
|
||||||
|
if (caching > 0 ) {
|
||||||
|
scan.setCaching(caching);
|
||||||
|
}
|
||||||
scanner = table.getScanner(scan);
|
scanner = table.getScanner(scan);
|
||||||
cached = null;
|
cached = null;
|
||||||
id = Long.toString(System.currentTimeMillis()) +
|
id = Long.toString(System.currentTimeMillis()) +
|
||||||
|
|
|
@ -61,6 +61,7 @@ import com.sun.jersey.api.json.JSONUnmarshaller;
|
||||||
* <attribute name="startRow" type="base64Binary"></attribute>
|
* <attribute name="startRow" type="base64Binary"></attribute>
|
||||||
* <attribute name="endRow" type="base64Binary"></attribute>
|
* <attribute name="endRow" type="base64Binary"></attribute>
|
||||||
* <attribute name="batch" type="int"></attribute>
|
* <attribute name="batch" type="int"></attribute>
|
||||||
|
* <attribute name="caching" type="int"></attribute>
|
||||||
* <attribute name="startTime" type="int"></attribute>
|
* <attribute name="startTime" type="int"></attribute>
|
||||||
* <attribute name="endTime" type="int"></attribute>
|
* <attribute name="endTime" type="int"></attribute>
|
||||||
* <attribute name="maxVersions" type="int"></attribute>
|
* <attribute name="maxVersions" type="int"></attribute>
|
||||||
|
@ -81,6 +82,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
private long endTime = Long.MAX_VALUE;
|
private long endTime = Long.MAX_VALUE;
|
||||||
private String filter = null;
|
private String filter = null;
|
||||||
private int maxVersions = Integer.MAX_VALUE;
|
private int maxVersions = Integer.MAX_VALUE;
|
||||||
|
private int caching = -1;
|
||||||
|
|
||||||
@XmlRootElement
|
@XmlRootElement
|
||||||
static class FilterModel {
|
static class FilterModel {
|
||||||
|
@ -472,7 +474,11 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
model.setEndTime(scan.getTimeRange().getMax());
|
model.setEndTime(scan.getTimeRange().getMax());
|
||||||
int caching = scan.getCaching();
|
int caching = scan.getCaching();
|
||||||
if (caching > 0) {
|
if (caching > 0) {
|
||||||
model.setBatch(caching);
|
model.setCaching(caching);
|
||||||
|
}
|
||||||
|
int batch = scan.getBatch();
|
||||||
|
if (batch > 0) {
|
||||||
|
model.setBatch(batch);
|
||||||
}
|
}
|
||||||
int maxVersions = scan.getMaxVersions();
|
int maxVersions = scan.getMaxVersions();
|
||||||
if (maxVersions > 0) {
|
if (maxVersions > 0) {
|
||||||
|
@ -496,18 +502,20 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
* @param endRow the end key of the row-range
|
* @param endRow the end key of the row-range
|
||||||
* @param columns the columns to scan
|
* @param columns the columns to scan
|
||||||
* @param batch the number of values to return in batch
|
* @param batch the number of values to return in batch
|
||||||
|
* @param caching the number of rows that the scanner will fetch at once
|
||||||
* @param endTime the upper bound on timestamps of values of interest
|
* @param endTime the upper bound on timestamps of values of interest
|
||||||
* @param maxVersions the maximum number of versions to return
|
* @param maxVersions the maximum number of versions to return
|
||||||
* @param filter a filter specification
|
* @param filter a filter specification
|
||||||
* (values with timestamps later than this are excluded)
|
* (values with timestamps later than this are excluded)
|
||||||
*/
|
*/
|
||||||
public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns,
|
public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns,
|
||||||
int batch, long endTime, int maxVersions, String filter) {
|
int batch, int caching, long endTime, int maxVersions, String filter) {
|
||||||
super();
|
super();
|
||||||
this.startRow = startRow;
|
this.startRow = startRow;
|
||||||
this.endRow = endRow;
|
this.endRow = endRow;
|
||||||
this.columns = columns;
|
this.columns = columns;
|
||||||
this.batch = batch;
|
this.batch = batch;
|
||||||
|
this.caching = caching;
|
||||||
this.endTime = endTime;
|
this.endTime = endTime;
|
||||||
this.maxVersions = maxVersions;
|
this.maxVersions = maxVersions;
|
||||||
this.filter = filter;
|
this.filter = filter;
|
||||||
|
@ -519,6 +527,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
* @param endRow the end key of the row-range
|
* @param endRow the end key of the row-range
|
||||||
* @param columns the columns to scan
|
* @param columns the columns to scan
|
||||||
* @param batch the number of values to return in batch
|
* @param batch the number of values to return in batch
|
||||||
|
* @param caching the number of rows that the scanner will fetch at once
|
||||||
* @param startTime the lower bound on timestamps of values of interest
|
* @param startTime the lower bound on timestamps of values of interest
|
||||||
* (values with timestamps earlier than this are excluded)
|
* (values with timestamps earlier than this are excluded)
|
||||||
* @param endTime the upper bound on timestamps of values of interest
|
* @param endTime the upper bound on timestamps of values of interest
|
||||||
|
@ -526,12 +535,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
* @param filter a filter specification
|
* @param filter a filter specification
|
||||||
*/
|
*/
|
||||||
public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns,
|
public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns,
|
||||||
int batch, long startTime, long endTime, String filter) {
|
int batch, int caching, long startTime, long endTime, String filter) {
|
||||||
super();
|
super();
|
||||||
this.startRow = startRow;
|
this.startRow = startRow;
|
||||||
this.endRow = endRow;
|
this.endRow = endRow;
|
||||||
this.columns = columns;
|
this.columns = columns;
|
||||||
this.batch = batch;
|
this.batch = batch;
|
||||||
|
this.caching = caching;
|
||||||
this.startTime = startTime;
|
this.startTime = startTime;
|
||||||
this.endTime = endTime;
|
this.endTime = endTime;
|
||||||
this.filter = filter;
|
this.filter = filter;
|
||||||
|
@ -591,6 +601,14 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
return batch;
|
return batch;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the number of rows that the scanner to fetch at once
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public int getCaching() {
|
||||||
|
return caching;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the lower bound on timestamps of items of interest
|
* @return the lower bound on timestamps of items of interest
|
||||||
*/
|
*/
|
||||||
|
@ -651,6 +669,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
this.batch = batch;
|
this.batch = batch;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param caching the number of rows to fetch at once
|
||||||
|
*/
|
||||||
|
public void setCaching(int caching) {
|
||||||
|
this.caching = caching;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param maxVersions maximum number of versions to return
|
* @param maxVersions maximum number of versions to return
|
||||||
*/
|
*/
|
||||||
|
@ -698,6 +723,9 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
builder.setEndTime(endTime);
|
builder.setEndTime(endTime);
|
||||||
}
|
}
|
||||||
builder.setBatch(getBatch());
|
builder.setBatch(getBatch());
|
||||||
|
if (caching > 0) {
|
||||||
|
builder.setCaching(caching);
|
||||||
|
}
|
||||||
builder.setMaxVersions(maxVersions);
|
builder.setMaxVersions(maxVersions);
|
||||||
if (filter != null) {
|
if (filter != null) {
|
||||||
builder.setFilter(filter);
|
builder.setFilter(filter);
|
||||||
|
@ -722,6 +750,9 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
||||||
if (builder.hasBatch()) {
|
if (builder.hasBatch()) {
|
||||||
batch = builder.getBatch();
|
batch = builder.getBatch();
|
||||||
}
|
}
|
||||||
|
if (builder.hasCaching()) {
|
||||||
|
caching = builder.getCaching();
|
||||||
|
}
|
||||||
if (builder.hasStartTime()) {
|
if (builder.hasStartTime()) {
|
||||||
startTime = builder.getStartTime();
|
startTime = builder.getStartTime();
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,6 +43,10 @@ public final class ScannerMessage {
|
||||||
// optional string filter = 8;
|
// optional string filter = 8;
|
||||||
boolean hasFilter();
|
boolean hasFilter();
|
||||||
String getFilter();
|
String getFilter();
|
||||||
|
|
||||||
|
// optional int32 caching = 9;
|
||||||
|
boolean hasCaching();
|
||||||
|
int getCaching();
|
||||||
}
|
}
|
||||||
public static final class Scanner extends
|
public static final class Scanner extends
|
||||||
com.google.protobuf.GeneratedMessage
|
com.google.protobuf.GeneratedMessage
|
||||||
|
@ -179,6 +183,16 @@ public final class ScannerMessage {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// optional int32 caching = 9;
|
||||||
|
public static final int CACHING_FIELD_NUMBER = 9;
|
||||||
|
private int caching_;
|
||||||
|
public boolean hasCaching() {
|
||||||
|
return ((bitField0_ & 0x00000080) == 0x00000080);
|
||||||
|
}
|
||||||
|
public int getCaching() {
|
||||||
|
return caching_;
|
||||||
|
}
|
||||||
|
|
||||||
private void initFields() {
|
private void initFields() {
|
||||||
startRow_ = com.google.protobuf.ByteString.EMPTY;
|
startRow_ = com.google.protobuf.ByteString.EMPTY;
|
||||||
endRow_ = com.google.protobuf.ByteString.EMPTY;
|
endRow_ = com.google.protobuf.ByteString.EMPTY;
|
||||||
|
@ -188,6 +202,7 @@ public final class ScannerMessage {
|
||||||
endTime_ = 0L;
|
endTime_ = 0L;
|
||||||
maxVersions_ = 0;
|
maxVersions_ = 0;
|
||||||
filter_ = "";
|
filter_ = "";
|
||||||
|
caching_ = 0;
|
||||||
}
|
}
|
||||||
private byte memoizedIsInitialized = -1;
|
private byte memoizedIsInitialized = -1;
|
||||||
public final boolean isInitialized() {
|
public final boolean isInitialized() {
|
||||||
|
@ -225,6 +240,9 @@ public final class ScannerMessage {
|
||||||
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||||
output.writeBytes(8, getFilterBytes());
|
output.writeBytes(8, getFilterBytes());
|
||||||
}
|
}
|
||||||
|
if (((bitField0_ & 0x00000080) == 0x00000080)) {
|
||||||
|
output.writeInt32(9, caching_);
|
||||||
|
}
|
||||||
getUnknownFields().writeTo(output);
|
getUnknownFields().writeTo(output);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -271,6 +289,10 @@ public final class ScannerMessage {
|
||||||
size += com.google.protobuf.CodedOutputStream
|
size += com.google.protobuf.CodedOutputStream
|
||||||
.computeBytesSize(8, getFilterBytes());
|
.computeBytesSize(8, getFilterBytes());
|
||||||
}
|
}
|
||||||
|
if (((bitField0_ & 0x00000080) == 0x00000080)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt32Size(9, caching_);
|
||||||
|
}
|
||||||
size += getUnknownFields().getSerializedSize();
|
size += getUnknownFields().getSerializedSize();
|
||||||
memoizedSerializedSize = size;
|
memoizedSerializedSize = size;
|
||||||
return size;
|
return size;
|
||||||
|
@ -411,6 +433,8 @@ public final class ScannerMessage {
|
||||||
bitField0_ = (bitField0_ & ~0x00000040);
|
bitField0_ = (bitField0_ & ~0x00000040);
|
||||||
filter_ = "";
|
filter_ = "";
|
||||||
bitField0_ = (bitField0_ & ~0x00000080);
|
bitField0_ = (bitField0_ & ~0x00000080);
|
||||||
|
caching_ = 0;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000100);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -482,6 +506,10 @@ public final class ScannerMessage {
|
||||||
to_bitField0_ |= 0x00000040;
|
to_bitField0_ |= 0x00000040;
|
||||||
}
|
}
|
||||||
result.filter_ = filter_;
|
result.filter_ = filter_;
|
||||||
|
if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
|
||||||
|
to_bitField0_ |= 0x00000080;
|
||||||
|
}
|
||||||
|
result.caching_ = caching_;
|
||||||
result.bitField0_ = to_bitField0_;
|
result.bitField0_ = to_bitField0_;
|
||||||
onBuilt();
|
onBuilt();
|
||||||
return result;
|
return result;
|
||||||
|
@ -529,6 +557,9 @@ public final class ScannerMessage {
|
||||||
if (other.hasFilter()) {
|
if (other.hasFilter()) {
|
||||||
setFilter(other.getFilter());
|
setFilter(other.getFilter());
|
||||||
}
|
}
|
||||||
|
if (other.hasCaching()) {
|
||||||
|
setCaching(other.getCaching());
|
||||||
|
}
|
||||||
this.mergeUnknownFields(other.getUnknownFields());
|
this.mergeUnknownFields(other.getUnknownFields());
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -600,6 +631,11 @@ public final class ScannerMessage {
|
||||||
filter_ = input.readBytes();
|
filter_ = input.readBytes();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
case 72: {
|
||||||
|
bitField0_ |= 0x00000100;
|
||||||
|
caching_ = input.readInt32();
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -825,6 +861,27 @@ public final class ScannerMessage {
|
||||||
onChanged();
|
onChanged();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// optional int32 caching = 9;
|
||||||
|
private int caching_ ;
|
||||||
|
public boolean hasCaching() {
|
||||||
|
return ((bitField0_ & 0x00000100) == 0x00000100);
|
||||||
|
}
|
||||||
|
public int getCaching() {
|
||||||
|
return caching_;
|
||||||
|
}
|
||||||
|
public Builder setCaching(int value) {
|
||||||
|
bitField0_ |= 0x00000100;
|
||||||
|
caching_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearCaching() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000100);
|
||||||
|
caching_ = 0;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
// @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Scanner)
|
// @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Scanner)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -851,11 +908,11 @@ public final class ScannerMessage {
|
||||||
static {
|
static {
|
||||||
java.lang.String[] descriptorData = {
|
java.lang.String[] descriptorData = {
|
||||||
"\n\024ScannerMessage.proto\022/org.apache.hadoo" +
|
"\n\024ScannerMessage.proto\022/org.apache.hadoo" +
|
||||||
"p.hbase.rest.protobuf.generated\"\224\001\n\007Scan" +
|
"p.hbase.rest.protobuf.generated\"\245\001\n\007Scan" +
|
||||||
"ner\022\020\n\010startRow\030\001 \001(\014\022\016\n\006endRow\030\002 \001(\014\022\017\n" +
|
"ner\022\020\n\010startRow\030\001 \001(\014\022\016\n\006endRow\030\002 \001(\014\022\017\n" +
|
||||||
"\007columns\030\003 \003(\014\022\r\n\005batch\030\004 \001(\005\022\021\n\tstartTi" +
|
"\007columns\030\003 \003(\014\022\r\n\005batch\030\004 \001(\005\022\021\n\tstartTi" +
|
||||||
"me\030\005 \001(\003\022\017\n\007endTime\030\006 \001(\003\022\023\n\013maxVersions" +
|
"me\030\005 \001(\003\022\017\n\007endTime\030\006 \001(\003\022\023\n\013maxVersions" +
|
||||||
"\030\007 \001(\005\022\016\n\006filter\030\010 \001(\t"
|
"\030\007 \001(\005\022\016\n\006filter\030\010 \001(\t\022\017\n\007caching\030\t \001(\005"
|
||||||
};
|
};
|
||||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||||
|
@ -867,7 +924,7 @@ public final class ScannerMessage {
|
||||||
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Scanner_fieldAccessorTable = new
|
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Scanner_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Scanner_descriptor,
|
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Scanner_descriptor,
|
||||||
new java.lang.String[] { "StartRow", "EndRow", "Columns", "Batch", "StartTime", "EndTime", "MaxVersions", "Filter", },
|
new java.lang.String[] { "StartRow", "EndRow", "Columns", "Batch", "StartTime", "EndTime", "MaxVersions", "Filter", "Caching", },
|
||||||
org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner.class,
|
org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner.class,
|
||||||
org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner.Builder.class);
|
org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner.Builder.class);
|
||||||
return null;
|
return null;
|
||||||
|
|
|
@ -26,4 +26,5 @@ message Scanner {
|
||||||
optional int64 endTime = 6;
|
optional int64 endTime = 6;
|
||||||
optional int32 maxVersions = 7;
|
optional int32 maxVersions = 7;
|
||||||
optional string filter = 8;
|
optional string filter = 8;
|
||||||
|
optional int32 caching = 9;
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,6 +41,7 @@ public class TestScannerModel extends TestCase {
|
||||||
private static final byte[] COLUMN2 = Bytes.toBytes("column2:foo");
|
private static final byte[] COLUMN2 = Bytes.toBytes("column2:foo");
|
||||||
private static final long START_TIME = 1245219839331L;
|
private static final long START_TIME = 1245219839331L;
|
||||||
private static final long END_TIME = 1245393318192L;
|
private static final long END_TIME = 1245393318192L;
|
||||||
|
private static final int CACHING = 1000;
|
||||||
private static final int BATCH = 100;
|
private static final int BATCH = 100;
|
||||||
|
|
||||||
private static final String AS_XML =
|
private static final String AS_XML =
|
||||||
|
@ -48,14 +49,15 @@ public class TestScannerModel extends TestCase {
|
||||||
" startRow=\"YWJyYWNhZGFicmE=\"" +
|
" startRow=\"YWJyYWNhZGFicmE=\"" +
|
||||||
" endTime=\"1245393318192\"" +
|
" endTime=\"1245393318192\"" +
|
||||||
" endRow=\"enp5eng=\"" +
|
" endRow=\"enp5eng=\"" +
|
||||||
" batch=\"100\">" +
|
" batch=\"100\"" +
|
||||||
|
" caching=\"1000\">" +
|
||||||
"<column>Y29sdW1uMQ==</column>" +
|
"<column>Y29sdW1uMQ==</column>" +
|
||||||
"<column>Y29sdW1uMjpmb28=</column>" +
|
"<column>Y29sdW1uMjpmb28=</column>" +
|
||||||
"</Scanner>";
|
"</Scanner>";
|
||||||
|
|
||||||
private static final String AS_PB =
|
private static final String AS_PB =
|
||||||
"CgthYnJhY2FkYWJyYRIFenp5engaB2NvbHVtbjEaC2NvbHVtbjI6Zm9vIGQo47qL554kMLDi57mf" +
|
"CgthYnJhY2FkYWJyYRIFenp5engaB2NvbHVtbjEaC2NvbHVtbjI6Zm9" +
|
||||||
"JA==";
|
"vIGQo47qL554kMLDi57mfJDj/////B0joBw==";
|
||||||
|
|
||||||
private JAXBContext context;
|
private JAXBContext context;
|
||||||
|
|
||||||
|
@ -73,6 +75,7 @@ public class TestScannerModel extends TestCase {
|
||||||
model.setStartTime(START_TIME);
|
model.setStartTime(START_TIME);
|
||||||
model.setEndTime(END_TIME);
|
model.setEndTime(END_TIME);
|
||||||
model.setBatch(BATCH);
|
model.setBatch(BATCH);
|
||||||
|
model.setCaching(CACHING);
|
||||||
return model;
|
return model;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -114,6 +117,7 @@ public class TestScannerModel extends TestCase {
|
||||||
assertEquals(model.getStartTime(), START_TIME);
|
assertEquals(model.getStartTime(), START_TIME);
|
||||||
assertEquals(model.getEndTime(), END_TIME);
|
assertEquals(model.getEndTime(), END_TIME);
|
||||||
assertEquals(model.getBatch(), BATCH);
|
assertEquals(model.getBatch(), BATCH);
|
||||||
|
assertEquals(model.getCaching(), CACHING);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testBuildModel() throws Exception {
|
public void testBuildModel() throws Exception {
|
||||||
|
@ -127,6 +131,5 @@ public class TestScannerModel extends TestCase {
|
||||||
public void testFromPB() throws Exception {
|
public void testFromPB() throws Exception {
|
||||||
checkModel(fromPB(AS_PB));
|
checkModel(fromPB(AS_PB));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue