XMLWordPrintableJSON

    Details

    • Type: Sub-task
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 0.99.0
    • Component/s: Coprocessors, regionserver
    • Labels:
      None
    • Hadoop Flags:
      Reviewed
    • Release Note:
      Hide
      Removed the below 2 hooks from RegionObserver CP. These were deprecated since 0.96 and having replacement
      1. preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get, final List<KeyValue> result)
      2. postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get, final List<KeyValue> result)
      These are replaced with pre/postGetOp hooks
      Show
      Removed the below 2 hooks from RegionObserver CP. These were deprecated since 0.96 and having replacement 1. preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get, final List<KeyValue> result) 2. postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get, final List<KeyValue> result) These are replaced with pre/postGetOp hooks

      Description

      These hooks, dealing with List<KeyValue>, were deprecated since 0.96. We have 0.98, one more major version after that. Suggest this can be removed from 0.99 time.

      The impl in BaseRegionObserver is as below which can be very inefficient especially when we read from a DBE files. There we return not KeyValue but a new Cell impl (here by avoiding the need to copy value bytes) The KeyValueUtil.ensureKeyValue can kill this nice optimization if we come across this.

      public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
            final Get get, final List<Cell> results) throws IOException {
          // By default we are executing the deprecated preGet to support legacy RegionObservers
          // We may use the results coming in and we may return the results going out.
          List<KeyValue> kvs = new ArrayList<KeyValue>(results.size());
          for (Cell c : results) {
            kvs.add(KeyValueUtil.ensureKeyValue(c));
          }
          preGet(e, get, kvs);
          results.clear();
          results.addAll(kvs);
        }
      

        Attachments

        1. HBASE-11919.patch
          5 kB
          Anoop Sam John

          Activity

            People

            • Assignee:
              anoop.hbase Anoop Sam John
              Reporter:
              anoop.hbase Anoop Sam John
            • Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved: