bbeaudreault commented on code in PR #5488:
URL: https://github.com/apache/hbase/pull/5488#discussion_r1383408208


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java:
##########
@@ -99,4 +129,51 @@ public void testBigDecimalComparator() throws Exception {
       
ProtobufUtil.toComparator(ProtobufUtil.toComparator(bigDecimalComparator))));
   }
 
+  /**
+   * Test that we can load and deserialize custom comparators. Good to have 
generally, but also
+   * proves that this still works after HBASE-27276 despite not going through 
our fast function
+   * caches.
+   */
+  @Test
+  public void testCustomComparator() throws Exception {
+    ByteArrayComparable baseFilter = new BinaryComparator("foo".getBytes());
+    ComparatorProtos.Comparator proto = ProtobufUtil.toComparator(baseFilter);
+    String className = "CustomLoadedComparator" + 
allowFastReflectionFallthrough;
+    proto = proto.toBuilder().setName(className).build();
+
+    Configuration conf = HBaseConfiguration.create();
+    HBaseTestingUtil testUtil = new HBaseTestingUtil();
+    String dataTestDir = testUtil.getDataTestDir().toString();
+
+    // First make sure the test bed is clean, delete any pre-existing class.
+    // Below toComparator call is expected to fail because the comparator is 
not loaded now
+    ClassLoaderTestHelper.deleteClass(className, dataTestDir, conf);
+    try {
+      ProtobufUtil.toComparator(proto);
+      fail("expected to fail");
+    } catch (IOException e) {
+      // do nothing, this is expected
+    }
+
+    // Write a jar to be loaded into the classloader
+    String code = StringSubstitutor.replace(
+      
IOUtils.toString(getClass().getResourceAsStream("/CustomLoadedComparator.java"),
+        Charset.defaultCharset()),
+      Collections.singletonMap("suffix", allowFastReflectionFallthrough));
+    ClassLoaderTestHelper.buildJar(dataTestDir, className, code,
+      ClassLoaderTestHelper.localDirPath(conf));
+
+    // Disallow fallthrough at first, we expect below to fail
+    ProtobufUtil.setAllowFastReflectionFallthrough(false);

Review Comment:
   > Why disable fallthrough will lead to a failure here? It is because the 
class is not loaded when initializing, so it is not in the 
ReflectedFunctionCache?
   
   Correct
   
   > So we do not add special test for testing the scenario where 
ReflectedFunctionCache can work? As it is enabled by default so testing filter 
usage is enough?
   
   Correct, we try the cache first. All of the tests in this class will run 
with setAllowFastReflectionFallthrough false and true. The false case is what 
matters -- If ReflectedFunctionCache were not working, the tests would fail 
because they can't fallback on the old reflection. So with disallowing 
fallback, we know that all of our filters and comparators can work with 
ReflectedFunctionCache.
   
   I added this extra test to specially handle user custom filters, to ensure 
that they would still work despite not being supported by 
ReflectionFunctionCache.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to