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

Salted local index failure is causing region server to abort

    XMLWordPrintableJSON

Details

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

    Description

      If you run just this case

      { false, true, true, true, false, null}
      

      in MutableIndexFailureIT on the 4.x-HBase-1.2 branch, rajeshbabu, you will see the following NPE in logs:

      2017-09-11 00:27:08,119 WARN  [B.defaultRpcServer.handler=2,queue=0,port=63436] org.apache.phoenix.index.PhoenixIndexFailurePolicy(143): handleFailure failed
      java.lang.NullPointerException
      	at org.apache.phoenix.util.SchemaUtil.getTableKeyFromFullName(SchemaUtil.java:707)
      	at org.apache.phoenix.util.IndexUtil.updateIndexState(IndexUtil.java:717)
      	at org.apache.phoenix.index.PhoenixIndexFailurePolicy.handleFailureWithExceptions(PhoenixIndexFailurePolicy.java:221)
      	at org.apache.phoenix.index.PhoenixIndexFailurePolicy.handleFailure(PhoenixIndexFailurePolicy.java:140)
      	at org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:155)
      	at org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:139)
      	at org.apache.phoenix.hbase.index.Indexer.doPostWithExceptions(Indexer.java:651)
      	at org.apache.phoenix.hbase.index.Indexer.doPost(Indexer.java:608)
      	at org.apache.phoenix.hbase.index.Indexer.postBatchMutateIndispensably(Indexer.java:591)
      	at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$37.call(RegionCoprocessorHost.java:1034)
      	at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1673)
      	at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1749)
      	at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1705)
      	at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.postBatchMutateIndispensably(RegionCoprocessorHost.java:1030)
      	at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:3322)
      	at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2881)
      	at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2823)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(RSRpcServices.java:758)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(RSRpcServices.java:720)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(RSRpcServices.java:2168)
      	at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33656)
      	at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2188)
      	at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
      	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 happens only for salted local indexes. If I remove the SALT_BUCKETS from the table DDL, then the test passes fine. On looking closely at the code, it seems like something is wrong with the computation of offset and subsequent parsing of the index id from the row key here (in PhoenixIndexFailurePolicy):

      int offset =
                          regionInfo.getStartKey().length == 0 ? regionInfo.getEndKey().length
                                  : regionInfo.getStartKey().length;
                  byte[] viewId = null;
                  for (Mutation mutation : mutations) {
                      viewId =
                              indexMaintainer.getViewIndexIdFromIndexRowKey(
                                      new ImmutableBytesWritable(mutation.getRow(), offset,
                                              mutation.getRow().length - offset));
                      String indexTableName = localIndexNames.get(new ImmutableBytesWritable(viewId));
                      indexTableNames.add(indexTableName);
                  }
      

      Because of this NPE in PhoenixIndexFailurePolicy, we end up triggering the KillServerOnFailurePolicy which ends up causing the region server to abort.
      This region server abort is also the reason why our builds against the 4.x-HBase-1.2 branch are hanging. I also believe once we fix this, we can hopefully reenable back the parameters which were testing out rebuild of local indexes for the 4.x-HBase-0.98, 4.x-HBase-1.1 and 4.x-HBase-1.2 branches. On the master branch, because local index update is transactional with data table update, we won' run into such failure scenarios (I think).

      jamestaylor - A bit orthogonal, but it seems like we can do better here. Wouldn't a better option here would be to let HBase black list the Indexer co-processor in cases of such bugs? Else, we run the risk of shutting down the entire HBase cluster which is what happened here.

      Attachments

        1. PHOENIX-4190.patch
          3 kB
          James R. Taylor

        Activity

          People

            jamestaylor James R. Taylor
            samarthjain Samarth Jain
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: