Uploaded image for project: 'Accumulo'
  1. Accumulo
  2. ACCUMULO-3597

Metadata table load prevented by flush

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.6.0, 1.6.1, 1.6.2
    • 1.5.3, 1.6.3, 1.7.0
    • None
    • None

    Description

      Was running random walk test against 1.6.2 RC5 on a 20 node EC2 cluster. Everything hung because a metadata table was not loading. I think the problem was a flush message.

      On this cluster the master was 10.1.2.10 and the tserver that was supposed to load a metadata tablet was 10.1.2.13.

      Below is the root tablet entries for the problem metadata tablet showing it has a future location of 10.1.2.13.

      !0< file:hdfs://ip-10-1-2-11:9000/accumulo/tables/!0/default_tablet/A0000xs5.rf []    59542,7512
      !0< file:hdfs://ip-10-1-2-11:9000/accumulo/tables/!0/default_tablet/F0000xs8.rf []    8596,927
      !0< file:hdfs://ip-10-1-2-11:9000/accumulo/tables/!0/default_tablet/F0000xs9.rf []    1735,70
      !0< future:24b7ebf8cba00c3 []    ip-10-1-2-13:9997
      !0< last:24b7ebf8cba00f4 []    ip-10-1-2-22:9997
      !0< srv:compact []    39
      !0< srv:dir []    hdfs://ip-10-1-2-11:9000/accumulo/tables/!0/default_tablet
      !0< srv:flush []    39
      !0< srv:lock []    tservers/ip-10-1-2-22:9997/zlock-0000000001$24b7ebf8cba00f4
      !0< srv:time []    L193895
      !0< ~tab:~pr []    \x0179dd555cc928f80d
      

      Below shows grepping the tserver logs, nothing about loading the tablet.

      $ grep 79dd555cc928f80d tserver_ip-10-1-2-13.ec2.internal.debug.log 
      2015-02-12 20:24:49,526 [impl.ThriftScanner] DEBUG: Scan failed, not serving tablet (!0<;79dd555cc928f80d,ip-10-1-2-22:9997,24b7ebf8cba00f4)
      

      Below netstat -nape run on the tserver shows alot of backed up data from master to tserver. I suspect the tablet load messages are in this backed up data.

      tcp   471408      0 10.1.2.13:9997              10.1.2.10:51271             ESTABLISHED 500        659703     30785/java
      

      Below is a flush thread on the tserver stuck waiting to update the problem metadata tablet.

      "ClientPool 420" daemon prio=10 tid=0x0000000038b72000 nid=0x5f4e waiting on condition [0x00007fea175c8000]
         java.lang.Thread.State: TIMED_WAITING (sleeping)
              at java.lang.Thread.sleep(Native Method)
              at org.apache.accumulo.core.util.UtilWaitThread.sleep(UtilWaitThread.java:26)
              at org.apache.accumulo.core.client.impl.TabletLocatorImpl.locateTablet(TabletLocatorImpl.java:442)
              at org.apache.accumulo.core.client.impl.Writer.update(Writer.java:85)
              at org.apache.accumulo.server.util.MetadataTableUtil.update(MetadataTableUtil.java:143)
              at org.apache.accumulo.server.util.MetadataTableUtil.update(MetadataTableUtil.java:135)
              at org.apache.accumulo.server.util.MetadataTableUtil.updateTabletFlushID(MetadataTableUtil.java:164)
              at org.apache.accumulo.tserver.Tablet.flush(Tablet.java:2227)
              at org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.flush(TabletServer.java:2380)
              at sun.reflect.GeneratedMethodAccessor22.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:606)
              at org.apache.accumulo.trace.instrument.thrift.RpcServerInvocationHandler.invoke(RpcServerInvocationHandler.java:46)
              at org.apache.accumulo.server.util.RpcWrapper$1.invoke(RpcWrapper.java:47)
              at com.sun.proxy.$Proxy22.flush(Unknown Source)
              at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$flush.getResult(TabletClientService.java:2595)
              at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$flush.getResult(TabletClientService.java:2581)
              at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
              at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
              at org.apache.accumulo.server.util.TServerUtils$TimedProcessor.process(TServerUtils.java:168)
              at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:516)
              at org.apache.accumulo.server.util.CustomNonBlockingServer$1.run(CustomNonBlockingServer.java:77)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
              at org.apache.accumulo.trace.instrument.TraceRunnable.run(TraceRunnable.java:47)
              at org.apache.accumulo.core.util.LoggingRunnable.run(LoggingRunnable.java:34)
              at java.lang.Thread.run(Thread.java:744)
      

      Below are the loadTablet and flush messages from thrift. I think the master sent a oneway flush call, then a one way load tablet call over the same connection. I think the flush blocked waiting for the tablet to load and the flush was preventing the tablet from loading.

        oneway void loadTablet(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:string lock, 2:data.TKeyExtent extent),
        oneway void flush(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 3:string lock, 2:string tableId, 5:binary startRow, 6:binary endRow),
      

      Attachments

        1. ACCUMULO-3597-1.patch
          2 kB
          Keith Turner
        2. ACCUMULO-3597-2.patch
          2 kB
          Keith Turner

        Issue Links

          Activity

            People

              kturner Keith Turner
              kturner Keith Turner
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: