Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-28842

TestRequestAttributes should fail when expected

    XMLWordPrintableJSON

Details

    Description

      Problem:

      The tests in the TestRequestAttributes class pass even when they should fail. I've included an example of a test that should fail but does not below.

      Fix:
      Throw an IOException in the AttributesCoprocessor when the map of expected request attributes does not match the map of given request attributes.
       
      Test:
      We set 2+ request attributes on the Get request but always return 0 request attributes from AttributesCoprocessor::getRequestAttributesForRowKey method. Yet the test passes even though the map of expected request attributes never matches the map of given request attributes.

      @Category({ ClientTests.class, MediumTests.class })
      public class TestRequestAttributes {
      
        @ClassRule
        public static final HBaseClassTestRule CLASS_RULE =
          HBaseClassTestRule.forClass(TestRequestAttributes.class);
      
        private static final byte[] ROW_KEY1 = Bytes.toBytes("1");
        private static final Map<byte[], Map<String, byte[]>> ROW_KEY_TO_REQUEST_ATTRIBUTES =
          new HashMap<>();
        static {
          CONNECTION_ATTRIBUTES.put("clientId", Bytes.toBytes("foo"));
          ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY1, addRandomRequestAttributes());
        }
        private static final ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(100);
        private static final byte[] FAMILY = Bytes.toBytes("0");
        private static final TableName TABLE_NAME = TableName.valueOf("testRequestAttributes");
      
        private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
        private static SingleProcessHBaseCluster cluster;
      
        @BeforeClass
        public static void setUp() throws Exception {
          cluster = TEST_UTIL.startMiniCluster(1);
          Table table = TEST_UTIL.createTable(TABLE_NAME, new byte[][] { FAMILY }, 1,
            HConstants.DEFAULT_BLOCKSIZE, AttributesCoprocessor.class.getName());
          table.close();
        }
      
        @AfterClass
        public static void afterClass() throws Exception {
          cluster.close();
          TEST_UTIL.shutdownMiniCluster();
        }
      @Test
      public void testRequestAttributesGet() throws IOException {
        Configuration conf = TEST_UTIL.getConfiguration();
        try (
          Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf),
            CONNECTION_ATTRIBUTES);
          Table table = configureRequestAttributes(conn.getTableBuilder(TABLE_NAME, EXECUTOR_SERVICE),
            ROW_KEY_TO_REQUEST_ATTRIBUTES.get(ROW_KEY1)).build()) {
      
          table.get(new Get(ROW_KEY1));
        }
      }
      
      private static Map<String, byte[]> addRandomRequestAttributes() {
        Map<String, byte[]> requestAttributes = new HashMap<>();
        int j = Math.max(2, (int) (10 * Math.random()));
        for (int i = 0; i < j; i++) {
          requestAttributes.put(String.valueOf(i), Bytes.toBytes(UUID.randomUUID().toString()));
        }
        return requestAttributes;
      }
      
      public static class AttributesCoprocessor implements RegionObserver, RegionCoprocessor {
      
          @Override
          public Optional<RegionObserver> getRegionObserver() {
            return Optional.of(this);
          }
      
          @Override
          public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
            List<Cell> result) throws IOException {
             validateRequestAttributes(getRequestAttributesForRowKey(get.getRow())));   
          }
      
          private Map<String, byte[]> getRequestAttributesForRowKey(byte[] rowKey) {
            return Collections.emptyMap(); // This line helps demonstrate the bug
          }
      
          private boolean validateRequestAttributes(Map<String, byte[]> requestAttributes) {
            RpcCall rpcCall = RpcServer.getCurrentCall().get();
            Map<String, byte[]> attrs = rpcCall.getRequestAttributes();
            if (attrs.size() != requestAttributes.size()) {
              return;
            }
            for (Map.Entry<String, byte[]> attr : attrs.entrySet()) {
              if (!requestAttributes.containsKey(attr.getKey())) {
                return;
              }
              if (!Arrays.equals(requestAttributes.get(attr.getKey()), attr.getValue())) {
                return;
              }
            }
            return;
          }
        }
      } 

       

       

      Attachments

        Activity

          People

            eboland148 Evelyn Boland
            eboland148 Evelyn Boland
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: