You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "Josh Elser (JIRA)" <ji...@apache.org> on 2015/06/11 02:28:00 UTC

[jira] [Commented] (ACCUMULO-3898) `admin stop` created deadlock in TabletServer being stopped

    [ https://issues.apache.org/jira/browse/ACCUMULO-3898?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14581277#comment-14581277 ] 

Josh Elser commented on ACCUMULO-3898:
--------------------------------------

Best guess at this point is that the master shouldn't have requested an unload of root yet. The UnloadTabletHandler should have been only invoked by master rpc (best as I can tell). The usual candidates (jvm pauses, network partitions) should really be relevant (in my head) since the master should have all of the user tablets offline before taking off meta and then root.

Sadly, the logging in the master is tremendously sparse around this area in the code. Might need some [~kturner] [~ecn] love.

> `admin stop` created deadlock in TabletServer being stopped
> -----------------------------------------------------------
>
>                 Key: ACCUMULO-3898
>                 URL: https://issues.apache.org/jira/browse/ACCUMULO-3898
>             Project: Accumulo
>          Issue Type: Bug
>          Components: tserver
>    Affects Versions: 1.7.0
>            Reporter: Josh Elser
>            Priority: Critical
>             Fix For: 1.7.1, 1.8.0
>
>
> Take a tabletserver, tserver1, one of a few.
> The other tabletservers have already been requested to stop and all have stopped (via {{admin stop}}).
> {noformat}
> 2015-06-10 21:07:40,485 [master.Master] DEBUG: Seeding FATE op to shutdown tserver1:9997[24ddf314d8c0010] with tid 9215266058859846639
> 2015-06-10 21:07:40,503 [master.EventCoordinator] INFO : Tablet Server shutdown requested for tserver1:9997[24ddf314d8c0010]
> {noformat}
> The master FATE op gets seeded to shutdown tserver1. As we can guess, since it's the last server, it's hosting {{r<<}}. It unloads it
> {noformat}
> 2015-06-10 21:07:40,581 [tablet.Tablet] DEBUG: initiateClose(saveState=true queueMinC=false disableWrites=false) +r<<
> 2015-06-10 21:07:40,581 [tablet.Tablet] DEBUG: completeClose(saveState=true completeClose=true) +r<<
> 2015-06-10 21:07:40,587 [tserver.TabletServer] DEBUG: MultiScanSess master:58287 1 entries in 0.00 secs (lookup_time:0.00 secs tablets:1 ranges:1)
> 2015-06-10 21:07:40,608 [tserver.NativeMap] DEBUG: Deallocating native map 0x0000000004df6fa0
> 2015-06-10 21:07:40,608 [tserver.NativeMap] DEBUG: Deallocating native map 0x0000000004df7040
> 2015-06-10 21:07:40,609 [tserver.NativeMap] DEBUG: Deallocating native map 0x00000000022b4da0
> 2015-06-10 21:07:40,609 [tablet.Tablet] TABLET_HIST: +r<< closed
> 2015-06-10 21:07:40,609 [tserver.TabletServer] DEBUG: Unassigning +r<<@(null,tserver1:9997[24ddf314d8c0010],null)
> 2015-06-10 21:07:40,610 [state.ZooStore] DEBUG: Removing /root_tablet/location
> 2015-06-10 21:07:40,613 [state.ZooStore] DEBUG: Removing /root_tablet/future_location
> 2015-06-10 21:07:40,614 [state.ZooTabletStateStore] DEBUG: unassign root tablet location
> 2015-06-10 21:07:40,619 [tserver.TabletServer] INFO : unloaded +r<<
> {noformat}
> After this, however, we can see it tries to run a minor compaction on a user tablet (1<<)
> {noformat}
> 2015-06-10 21:07:40,621 [tablet.Tablet] DEBUG: initiateClose(saveState=true queueMinC=false disableWrites=false) !0;~<
> 2015-06-10 21:07:40,621 [tablet.Tablet] DEBUG: completeClose(saveState=true completeClose=true) !0;~<
> 2015-06-10 21:07:40,622 [tablet.Tablet] DEBUG: initiateClose(saveState=true queueMinC=false disableWrites=false) 1<<
> 2015-06-10 21:07:40,623 [tserver.NativeMap] DEBUG: Allocated native map 0x0000000002e82e00
> 2015-06-10 21:07:40,624 [impl.ThriftScanner] DEBUG: Failed to locate tablet for table : +r row : !0;
> 2015-06-10 21:07:40,633 [tablet.MinorCompactor] DEBUG: Begin minor compaction hdfs://namenode:8020/apps/accumulo/data/tables/1/default_tablet/F0000007.rf_tmp 1<<
> 2015-06-10 21:07:40,759 [tablet.Compactor] DEBUG: Compaction 1<< 590 read | 590 written | 12,553 entries/sec |  0.047 secs
> 2015-06-10 21:07:40,767 [tablet.Tablet] DEBUG: Logs for memory compacted: 1<< tserver1:9997/hdfs://namenode:8020/apps/accumulo/data/wal/tserver1+9997/09dc5a75-b9a1-45e9-bd5b-118ef811e99f
> 2015-06-10 21:07:40,767 [tablet.Tablet] DEBUG: Logs to be destroyed: 1<< tserver1:9997/hdfs://namenode:8020/apps/accumulo/data/wal/tserver1+9997/09dc5a75-b9a1-45e9-bd5b-118ef811e99f
> 2015-06-10 21:07:40,777 [tabletserver.LargestFirstMemoryManager] DEBUG: BEFORE compactionThreshold = 0.550 maxObserved = 150,256
> 2015-06-10 21:07:40,777 [tabletserver.LargestFirstMemoryManager] DEBUG: AFTER compactionThreshold = 0.605
> {noformat}
> So, as we might expect, this is never going to finish. The close won't happen because root isn't assigned, and the master isn't going to reassign root unless it gets a new tserver (which it won't because we're trying to stop the entire instance).
> To verify this is what's gumming up the works:
> {noformat}
> "tablet migration 1" daemon prio=10 tid=0x0000000002e84800 nid=0x671 waiting on condition [0x00007fb80a03a000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at org.apache.accumulo.core.util.UtilWaitThread.sleep(UtilWaitThread.java:27)
> 	at org.apache.accumulo.core.client.impl.RootTabletLocator.locateTablet(RootTabletLocator.java:129)
> 	at org.apache.accumulo.core.client.impl.TabletLocatorImpl.lookupTabletLocation(TabletLocatorImpl.java:460)
> 	at org.apache.accumulo.core.client.impl.TabletLocatorImpl.lookupTabletLocationAndCheckLock(TabletLocatorImpl.java:634)
> 	at org.apache.accumulo.core.client.impl.TabletLocatorImpl._locateTablet(TabletLocatorImpl.java:620)
> 	at org.apache.accumulo.core.client.impl.TabletLocatorImpl.locateTablet(TabletLocatorImpl.java:439)
> 	at org.apache.accumulo.core.client.impl.Writer.update(Writer.java:88)
> 	at org.apache.accumulo.server.util.MetadataTableUtil.update(MetadataTableUtil.java:153)
> 	at org.apache.accumulo.server.util.MetadataTableUtil.update(MetadataTableUtil.java:145)
> 	at org.apache.accumulo.server.util.MasterMetadataUtil.updateTabletDataFile(MasterMetadataUtil.java:257)
> 	at org.apache.accumulo.tserver.tablet.Tablet.updateTabletDataFile(Tablet.java:2651)
> 	- locked <0x00000000ab6fa2d8> (a java.lang.Object)
> 	at org.apache.accumulo.tserver.tablet.DatafileManager.bringMinorCompactionOnline(DatafileManager.java:417)
> 	at org.apache.accumulo.tserver.tablet.Tablet.minorCompact(Tablet.java:956)
> 	at org.apache.accumulo.tserver.tablet.MinorCompactionTask.run(MinorCompactionTask.java:84)
> 	at org.apache.accumulo.tserver.tablet.Tablet.initiateClose(Tablet.java:1406)
> 	at org.apache.accumulo.tserver.tablet.Tablet.close(Tablet.java:1337)
> 	at org.apache.accumulo.tserver.TabletServer$UnloadTabletHandler.run(TabletServer.java:1999)
> 	at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
> 	at org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
> 	at java.lang.Thread.run(Thread.java:745)
>    Locked ownable synchronizers:
> 	- <0x00000000ab6f7bd8> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
> 	- <0x00000000ab6fb8f8> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
> 	- <0x00000000aba02208> (a java.util.concurrent.ThreadPoolExecutor$Worker)
> "metadata tablet migration 1" daemon prio=10 tid=0x0000000002e84000 nid=0x670 waiting on condition [0x00007fb809f3a000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at org.apache.accumulo.core.client.impl.ThriftScanner.pause(ThriftScanner.java:200)
> 	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:248)
> 	at org.apache.accumulo.core.client.impl.ScannerIterator$Reader.run(ScannerIterator.java:80)
> 	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:151)
> 	at org.apache.accumulo.server.util.MetadataTableUtil.getFileAndLogEntries(MetadataTableUtil.java:529)
> 	at org.apache.accumulo.tserver.tablet.Tablet.closeConsistencyCheck(Tablet.java:1505)
> 	at org.apache.accumulo.tserver.tablet.Tablet.completeClose(Tablet.java:1455)
> 	- locked <0x00000000ab6ff8a8> (a org.apache.accumulo.tserver.tablet.Tablet)
> 	at org.apache.accumulo.tserver.tablet.Tablet.close(Tablet.java:1338)
> 	at org.apache.accumulo.tserver.TabletServer$UnloadTabletHandler.run(TabletServer.java:1999)
> 	at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
> 	at org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
> 	at java.lang.Thread.run(Thread.java:745)
>    Locked ownable synchronizers:
> 	- <0x00000000ab9e8db0> (a java.util.concurrent.ThreadPoolExecutor$Worker)
> {noformat}
> We can see that a Tablet close, specfically a Tablet close due to a migration, is trying to find the root tablet so it can write the result of the compaction to the metadata table.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)