From db05432b4997a926344e6f14bfaf549b482654a2 Mon Sep 17 00:00:00 2001 From: chandrasekhar-188k <154109917+chandrasekhar-188k@users.noreply.github.com> Date: Thu, 28 Nov 2024 21:26:32 +0530 Subject: [PATCH] HBASE-28627 REST ScannerModel doesn't support includeStartRow/includeStopRow (addendum) --- .../hadoop/hbase/rest/model/ScannerModel.java | 12 ++++ .../hbase/rest/model/TestScannerModel.java | 60 ++++++++++++++++++- 2 files changed, 70 insertions(+), 2 deletions(-) diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java index d68e36ad6ff4..a050dc9e54f9 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java @@ -121,8 +121,10 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable { private boolean cacheBlocks = true; private int limit = -1; + @JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = IncludeStartRowFilter.class) private boolean includeStartRow = true; + @JsonInclude(value = JsonInclude.Include.NON_DEFAULT) private boolean includeStopRow = false; @XmlAttribute @@ -143,6 +145,16 @@ public void setIncludeStartRow(boolean includeStartRow) { this.includeStartRow = includeStartRow; } + @edu.umd.cs.findbugs.annotations.SuppressWarnings( + value = { "EQ_CHECK_FOR_OPERAND_NOT_COMPATIBLE_WITH_THIS", "HE_EQUALS_NO_HASHCODE" }, + justification = "1.The supplied value from the JSON Value Filter is of Type Boolean, hence supressing the check, 2.hashCode method will not be invoked, hence supressing the check") + private static class IncludeStartRowFilter { + @Override + public boolean equals(Object value) { + return Boolean.TRUE.equals(value); + } + } + /** * Implement lazily-instantiated singleton as per recipe here: * http://literatejava.com/jvm/fastest-threadsafe-singleton-jvm/ diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java index 79984af62dfc..f68358de634f 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestScannerModel.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.rest.ScannerResultGenerator; import org.apache.hadoop.hbase.testclassification.RestTests; @@ -62,8 +63,7 @@ public TestScannerModel() throws Exception { AS_JSON = "{\"batch\":100,\"caching\":1000,\"cacheBlocks\":false,\"endRow\":\"enp5eng=\"," + "\"endTime\":1245393318192,\"maxVersions\":2147483647,\"startRow\":\"YWJyYWNhZGFicmE=\"," + "\"startTime\":1245219839331,\"column\":[\"Y29sdW1uMQ==\",\"Y29sdW1uMjpmb28=\"]," - + "\"labels\":[\"private\",\"public\"]," + "\"limit\":10000," - + "\"includeStartRow\":true,\"includeStopRow\":false}"; + + "\"labels\":[\"private\",\"public\"]," + "\"limit\":10000}"; AS_PB = "CgthYnJhY2FkYWJyYRIFenp5engaB2NvbHVtbjEaC2NvbHVtbjI6Zm9vIGQo47qL554kMLDi57mfJDj" + "/////B0joB1IHcHJpdmF0ZVIGcHVibGljWABgkE4="; @@ -151,4 +151,60 @@ private void verifyException(final String FILTER) throws Exception { model.setFilter(FILTER); ScannerResultGenerator.buildFilterFromModel(model); } + + @Test() + public void testToJsonWithIncludeStartRowAndIncludeStopRow() throws Exception { + String jsonStr = + "{\"batch\":100,\"caching\":1000,\"cacheBlocks\":false,\"endRow\":\"enp5eng=\"," + + "\"endTime\":1245393318192,\"maxVersions\":2147483647,\"startRow\":\"YWJyYWNhZGFicmE=\"," + + "\"startTime\":1245219839331,\"column\":[\"Y29sdW1uMQ==\",\"Y29sdW1uMjpmb28=\"]," + + "\"labels\":[\"private\",\"public\"]," + "\"limit\":10000," + + "\"includeStartRow\":false,\"includeStopRow\":true}"; + + ObjectNode expObj = mapper.readValue(jsonStr, ObjectNode.class); + ObjectNode actObj = mapper.readValue( + toJSON(buildTestModelWithIncludeStartRowAndIncludeStopRow(false, true)), ObjectNode.class); + assertEquals(expObj, actObj); + + jsonStr = "{\"batch\":100,\"caching\":1000,\"cacheBlocks\":false,\"endRow\":\"enp5eng=\"," + + "\"endTime\":1245393318192,\"maxVersions\":2147483647,\"startRow\":\"YWJyYWNhZGFicmE=\"," + + "\"startTime\":1245219839331,\"column\":[\"Y29sdW1uMQ==\",\"Y29sdW1uMjpmb28=\"]," + + "\"labels\":[\"private\",\"public\"]," + "\"limit\":10000," + "\"includeStopRow\":true}"; + + expObj = mapper.readValue(jsonStr, ObjectNode.class); + actObj = mapper.readValue( + toJSON(buildTestModelWithIncludeStartRowAndIncludeStopRow(true, true)), ObjectNode.class); + assertEquals(expObj, actObj); + + jsonStr = "{\"batch\":100,\"caching\":1000,\"cacheBlocks\":false,\"endRow\":\"enp5eng=\"," + + "\"endTime\":1245393318192,\"maxVersions\":2147483647,\"startRow\":\"YWJyYWNhZGFicmE=\"," + + "\"startTime\":1245219839331,\"column\":[\"Y29sdW1uMQ==\",\"Y29sdW1uMjpmb28=\"]," + + "\"labels\":[\"private\",\"public\"]," + "\"limit\":10000," + "\"includeStartRow\":false}"; + + expObj = mapper.readValue(jsonStr, ObjectNode.class); + actObj = mapper.readValue( + toJSON(buildTestModelWithIncludeStartRowAndIncludeStopRow(false, false)), ObjectNode.class); + assertEquals(expObj, actObj); + + } + + protected ScannerModel buildTestModelWithIncludeStartRowAndIncludeStopRow(boolean includeStartRow, + boolean includeStopRow) { + ScannerModel model = new ScannerModel(); + model.setStartRow(START_ROW); + model.setEndRow(END_ROW); + model.addColumn(COLUMN1); + model.addColumn(COLUMN2); + model.setStartTime(START_TIME); + model.setEndTime(END_TIME); + model.setBatch(BATCH); + model.setCaching(CACHING); + model.addLabel(PRIVATE); + model.addLabel(PUBLIC); + model.setCacheBlocks(CACHE_BLOCKS); + model.setLimit(LIMIT); + model.setIncludeStartRow(includeStartRow); + model.setIncludeStopRow(includeStopRow); + return model; + } }