Skip to content

Commit 0293b45

Browse files
authored
range relevant filter (#80)
1 parent e6d6da6 commit 0293b45

File tree

3 files changed

+192
-0
lines changed

3 files changed

+192
-0
lines changed

src/main/java/com/alipay/oceanbase/hbase/filter/HBaseFilterUtils.java

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import java.io.ByteArrayOutputStream;
2727
import java.io.IOException;
2828
import java.util.List;
29+
import java.util.Set;
2930

3031
@InterfaceAudience.Private
3132
public class HBaseFilterUtils {
@@ -49,6 +50,8 @@ private static void toParseableByteArray(ByteArrayOutputStream byteStream, Filte
4950
toParseableByteArray(byteStream, (PageFilter) filter);
5051
} else if (filter instanceof ColumnCountGetFilter) {
5152
toParseableByteArray(byteStream, (ColumnCountGetFilter) filter);
53+
} else if (filter instanceof FirstKeyValueMatchingQualifiersFilter) {
54+
toParseableByteArray(byteStream, (FirstKeyValueMatchingQualifiersFilter) filter);
5255
} else if (filter instanceof PrefixFilter) {
5356
toParseableByteArray(byteStream, (PrefixFilter) filter);
5457
} else if (filter instanceof FilterList) {
@@ -243,6 +246,32 @@ private static void toParseableByteArray(ByteArrayOutputStream byteStream,
243246
byteStream.write(')');
244247
}
245248

249+
// FirstKeyValueMatchingQualifiersFilter('q1','q2')
250+
private static void toParseableByteArray(ByteArrayOutputStream byteStream,
251+
FirstKeyValueMatchingQualifiersFilter filter) throws IOException {
252+
Set<byte[]> qualifiers;
253+
try {
254+
Field field = filter.getClass().getDeclaredField("qualifiers");
255+
field.setAccessible(true);
256+
qualifiers = (Set<byte[]>)field.get(filter);
257+
} catch (NoSuchFieldException | IllegalAccessException e) {
258+
throw new RuntimeException(e);
259+
}
260+
byteStream.write(filter.getClass().getSimpleName().getBytes());
261+
byteStream.write('(');
262+
int i = 0;
263+
for (byte[] qualifier: qualifiers) {
264+
byteStream.write("'".getBytes());
265+
byteStream.write(qualifier);
266+
byteStream.write("'".getBytes());
267+
if (i < qualifiers.size() - 1) {
268+
byteStream.write(',');
269+
}
270+
i++;
271+
}
272+
byteStream.write(')');
273+
}
274+
246275
// PrefixFilter('prefix');
247276
private static void toParseableByteArray(ByteArrayOutputStream byteStream, PrefixFilter filter)
248277
throws IOException {

src/test/java/com/alipay/oceanbase/hbase/HTableTestBase.java

Lines changed: 152 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1124,6 +1124,158 @@ public void testFilter2() throws Exception {
11241124
scanner.close();
11251125
}
11261126

1127+
@Test
1128+
public void testFirstKeyValueMatchingQualifiersFilter() throws Exception {
1129+
String key1 = "getKey1";
1130+
String key2 = "getKey2";
1131+
String column1 = "c1";
1132+
String column2 = "c2";
1133+
String column3 = "c3";
1134+
String column4 = "c4";
1135+
String column5 = "c5";
1136+
String value1 = "value1";
1137+
String value2 = "value2";
1138+
String value3 = "value3";
1139+
String family = "family1";
1140+
Delete deleteKey1Family = new Delete(toBytes(key1));
1141+
deleteKey1Family.deleteFamily(toBytes(family));
1142+
1143+
Delete deleteKey2Family = new Delete(toBytes(key2));
1144+
deleteKey2Family.deleteFamily(toBytes(family));
1145+
1146+
hTable.delete(deleteKey1Family);
1147+
hTable.delete(deleteKey2Family);
1148+
1149+
Put putKey1Column1Value1 = new Put(toBytes(key1));
1150+
putKey1Column1Value1.add(toBytes(family), toBytes(column1), toBytes(value1));
1151+
1152+
Put putKey1Column1Value2 = new Put(toBytes(key1));
1153+
putKey1Column1Value2.add(toBytes(family), toBytes(column1), toBytes(value2));
1154+
1155+
Put putKey1Column2Value2 = new Put(toBytes(key1));
1156+
putKey1Column2Value2.add(toBytes(family), toBytes(column2), toBytes(value2));
1157+
1158+
Put putKey1Column2Value1 = new Put(toBytes(key1));
1159+
putKey1Column2Value1.add(toBytes(family), toBytes(column2), toBytes(value1));
1160+
1161+
Put putKey1Column3Value1 = new Put(toBytes(key1));
1162+
putKey1Column3Value1.add(toBytes(family), toBytes(column3), toBytes(value1));
1163+
1164+
Put putKey1Column4Value1 = new Put(toBytes(key1));
1165+
putKey1Column4Value1.add(toBytes(family), toBytes(column4), toBytes(value1));
1166+
1167+
Put putKey1Column5Value1 = new Put(toBytes(key1));
1168+
putKey1Column5Value1.add(toBytes(family), toBytes(column5), toBytes(value1));
1169+
1170+
Put putKey2Column1Value1 = new Put(toBytes(key2));
1171+
putKey2Column1Value1.add(toBytes(family), toBytes(column1), toBytes(value1));
1172+
1173+
Put putKey2Column1Value2 = new Put(toBytes(key2));
1174+
putKey2Column1Value2.add(toBytes(family), toBytes(column1), toBytes(value2));
1175+
1176+
Put putKey2Column2Value2 = new Put(toBytes(key2));
1177+
putKey2Column2Value2.add(toBytes(family), toBytes(column2), toBytes(value2));
1178+
1179+
Put putKey2Column2Value1 = new Put(toBytes(key2));
1180+
putKey2Column2Value1.add(toBytes(family), toBytes(column2), toBytes(value1));
1181+
1182+
hTable.delete(deleteKey1Family);
1183+
hTable.delete(deleteKey2Family);
1184+
tryPut(hTable, putKey1Column1Value1);
1185+
tryPut(hTable, putKey1Column1Value2);
1186+
tryPut(hTable, putKey1Column1Value1);
1187+
tryPut(hTable, putKey1Column2Value1);
1188+
tryPut(hTable, putKey1Column2Value2);
1189+
tryPut(hTable, putKey1Column2Value1);
1190+
tryPut(hTable, putKey1Column2Value2);
1191+
tryPut(hTable, putKey1Column3Value1);
1192+
tryPut(hTable, putKey1Column4Value1);
1193+
tryPut(hTable, putKey1Column5Value1);
1194+
tryPut(hTable, putKey2Column2Value1);
1195+
tryPut(hTable, putKey2Column2Value2);
1196+
1197+
Scan scan;
1198+
scan = new Scan();
1199+
scan.addFamily(family.getBytes());
1200+
scan.setMaxVersions(10);
1201+
TreeSet<byte []> qualifiers = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1202+
qualifiers.add(Bytes.toBytes("c11"));
1203+
qualifiers.add(Bytes.toBytes("c2"));
1204+
FirstKeyValueMatchingQualifiersFilter filter = new FirstKeyValueMatchingQualifiersFilter(qualifiers);
1205+
scan.setFilter(filter);
1206+
ResultScanner scanner = hTable.getScanner(scan);
1207+
1208+
int res_count = 0;
1209+
for (Result result : scanner) {
1210+
for (KeyValue keyValue : result.raw()) {
1211+
System.out.printf("Rowkey: %s, Column Family: %s, Column Qualifier: %s, Timestamp: %d, Value: %s%n",
1212+
Bytes.toString(result.getRow()),
1213+
Bytes.toString(keyValue.getFamily()),
1214+
Bytes.toString(keyValue.getQualifier()),
1215+
keyValue.getTimestamp(),
1216+
Bytes.toString(keyValue.getValue())
1217+
);
1218+
res_count += 1;
1219+
}
1220+
}
1221+
Assert.assertEquals(res_count, 5);
1222+
scanner.close();
1223+
1224+
scan = new Scan();
1225+
scan.addFamily(family.getBytes());
1226+
scan.setMaxVersions(10);
1227+
scan.setReversed(true);
1228+
qualifiers = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1229+
qualifiers.add(Bytes.toBytes("c22"));
1230+
qualifiers.add(Bytes.toBytes("c4"));
1231+
filter = new FirstKeyValueMatchingQualifiersFilter(qualifiers);
1232+
scan.setFilter(filter);
1233+
scanner = hTable.getScanner(scan);
1234+
1235+
res_count = 0;
1236+
for (Result result : scanner) {
1237+
for (KeyValue keyValue : result.raw()) {
1238+
System.out.printf("Rowkey: %s, Column Family: %s, Column Qualifier: %s, Timestamp: %d, Value: %s%n",
1239+
Bytes.toString(result.getRow()),
1240+
Bytes.toString(keyValue.getFamily()),
1241+
Bytes.toString(keyValue.getQualifier()),
1242+
keyValue.getTimestamp(),
1243+
Bytes.toString(keyValue.getValue())
1244+
);
1245+
res_count += 1;
1246+
}
1247+
}
1248+
Assert.assertEquals(res_count, 11);
1249+
scanner.close();
1250+
1251+
scan = new Scan();
1252+
scan.addFamily(family.getBytes());
1253+
scan.setMaxVersions(10);
1254+
scan.setReversed(true);
1255+
qualifiers = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1256+
qualifiers.add(Bytes.toBytes("c22"));
1257+
qualifiers.add(Bytes.toBytes("a"));
1258+
filter = new FirstKeyValueMatchingQualifiersFilter(qualifiers);
1259+
scan.setFilter(filter);
1260+
scanner = hTable.getScanner(scan);
1261+
1262+
res_count = 0;
1263+
for (Result result : scanner) {
1264+
for (KeyValue keyValue : result.raw()) {
1265+
System.out.printf("Rowkey: %s, Column Family: %s, Column Qualifier: %s, Timestamp: %d, Value: %s%n",
1266+
Bytes.toString(result.getRow()),
1267+
Bytes.toString(keyValue.getFamily()),
1268+
Bytes.toString(keyValue.getQualifier()),
1269+
keyValue.getTimestamp(),
1270+
Bytes.toString(keyValue.getValue())
1271+
);
1272+
res_count += 1;
1273+
}
1274+
}
1275+
Assert.assertEquals(res_count, 12);
1276+
scanner.close();
1277+
}
1278+
11271279
@Test
11281280
public void testGetFilter() throws Exception {
11291281
String key1 = "getKey1";

src/test/java/com/alipay/oceanbase/hbase/filter/HBaseFilterUtilsTest.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.junit.Test;
2525

2626
import java.io.IOException;
27+
import java.util.TreeSet;
2728

2829
public class HBaseFilterUtilsTest {
2930
private static final CompareFilter.CompareOp[] ops = { CompareFilter.CompareOp.LESS,
@@ -166,6 +167,16 @@ public void testColumnCountGetFilter() throws IOException {
166167
HBaseFilterUtils.toParseableByteArray(filter));
167168
}
168169

170+
@Test
171+
public void testFirstKeyValueMatchingQualifiersFilter() throws IOException {
172+
TreeSet<byte []> qualifiers = new TreeSet<>(Bytes.BYTES_COMPARATOR);
173+
qualifiers.add(Bytes.toBytes("q1"));
174+
qualifiers.add(Bytes.toBytes("q2"));
175+
FirstKeyValueMatchingQualifiersFilter filter = new FirstKeyValueMatchingQualifiersFilter(qualifiers);
176+
Assert.assertArrayEquals("FirstKeyValueMatchingQualifiersFilter('q1','q2')".getBytes(),
177+
HBaseFilterUtils.toParseableByteArray(filter));
178+
}
179+
169180
@Test
170181
public void testPrefixFilter() throws IOException {
171182
PrefixFilter filter = new PrefixFilter("prefix".getBytes());

0 commit comments

Comments
 (0)