Uploaded image for project: 'Phoenix'
  1. Phoenix
  2. PHOENIX-3986

UngroupedAggregateRegionObserver.commitBatch() should set the index metadata as an attribute on every mutation

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 4.9.0
    • 4.12.0
    • None

    Description

      We are seeing "unable to find cached index metadata" exceptions in production while running UPSERT SELECT queries on a table that has mutable indexes. These exceptions should not happen because PHOENIX-2935 changed the code to set the index metadata as an attribute.

      ERROR 2008 (INT10): ERROR 2008 (INT10): Unable to find cached index metadata.  key={} region={}.host={} Index update failed
       		at org.apache.phoenix.util.ServerUtil.createIOException(ServerUtil.java:81)
       		at org.apache.phoenix.util.ServerUtil.throwIOException(ServerUtil.java:55)
       		at org.apache.phoenix.index.PhoenixIndexMetaData.getIndexMetaData(PhoenixIndexMetaData.java:86)
       		at org.apache.phoenix.index.PhoenixIndexMetaData.<init>(PhoenixIndexMetaData.java:94)
       		at org.apache.phoenix.index.PhoenixIndexBuilder.getIndexMetaData(PhoenixIndexBuilder.java:93)
       		at org.apache.phoenix.hbase.index.builder.IndexBuildManager.getIndexUpdate(IndexBuildManager.java:122)
       		at org.apache.phoenix.hbase.index.Indexer.preBatchMutateWithExceptions(Indexer.java:324)
       		at org.apache.phoenix.hbase.index.Indexer.preBatchMutate(Indexer.java:249)
       		at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$35.call(RegionCoprocessorHost.java:996)
       		at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1621)
       		at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1697)
       		at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1653)
       		at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.preBatchMutate(RegionCoprocessorHost.java:992)
       		at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2650)
       		at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2410)
       		at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2365)
       		at org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver.commitBatch(UngroupedAggregateRegionObserver.java:216)
       		at org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver.commit(UngroupedAggregateRegionObserver.java:791)
       		at org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver.doPostScannerOpen(UngroupedAggregateRegionObserver.java:721)
       		at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$RegionScannerHolder.overrideDelegate(BaseScannerRegionObserver.java:236)
       		at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$RegionScannerHolder.nextRaw(BaseScannerRegionObserver.java:287)
       		at org.apache.hadoop.hbase.regionserver.HRegionServer.scan(HRegionServer.java:3361)
       		at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32492)
       		at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2210)
       		at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:104)
       		at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
       		at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
       		at java.lang.Thread.run(Thread.java:745)
      

      This is what I think is happening :
      In UngroupedAggregateRegionObserver.doPostScannerOpen we commit batches of mutations.
      In commitBatch() since the mutations are for the same region we only set the index maintainer as an attribute on the first mutation of the batch. We then call HRegion.batchMutate() which ends up calling HRegion.doMiniBatchMutation() in a loop.
      doMiniBatchMutation() loops throught the mutations one at a time and tries to acquire the each mutation's lock. If it cannot acquire a lock it processes the mutations for which it has acquired locks so far and then calls coprocessorHost.preBatchMutate which eventually ends up calling PhoenixIndexBuilder.getIndexMetaData() which set the index metadata using the attribute map of the first mutation in the mini batch.
      It then processes the next set of mutations for which it has acquired locks. The index metadata is not set as an attribute for this next mini batch, so we try and look it up in the cache which fails

      In PhoenixIndexBuilder

      @Override
          public IndexMetaData getIndexMetaData(MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
              return new PhoenixIndexMetaData(env, miniBatchOp.getOperation(0).getAttributesMap());
          }
      

      In MiniBatchOperationInProgress

      /**
         * @param index
         * @return The operation(Mutation) at the specified position.
         */
        public T getOperation(int index) {
          return operations[getAbsoluteIndex(index)];
        }
      
      
      private int getAbsoluteIndex(int index) {
          if (index < 0 || this.firstIndex + index >= this.lastIndexExclusive) {
            throw new ArrayIndexOutOfBoundsException(index);
          }
          return this.firstIndex + index;
        }
      

      ankit.singhal jamestaylor Does this make sense?

      Attachments

        1. PHOENIX-3986-v3.patch
          14 kB
          Thomas D'Silva
        2. PHOENIX-3986-v2.patch
          14 kB
          Thomas D'Silva
        3. PHOENIX-3986.patch
          14 kB
          Thomas D'Silva

        Issue Links

          Activity

            People

              tdsilva Thomas D'Silva
              tdsilva Thomas D'Silva
              Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: