You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "Dima Spivak (JIRA)" <ji...@apache.org> on 2016/10/04 18:10:21 UTC

[jira] [Created] (ACCUMULO-4485) Missing entries and dying tservers follow replication with fault injection on source cluster

Dima Spivak created ACCUMULO-4485:
-------------------------------------

             Summary: Missing entries and dying tservers follow replication with fault injection on source cluster
                 Key: ACCUMULO-4485
                 URL: https://issues.apache.org/jira/browse/ACCUMULO-4485
             Project: Accumulo
          Issue Type: Bug
          Components: replication
    Affects Versions: 1.7.2
            Reporter: Dima Spivak
            Assignee: Josh Elser
            Priority: Critical


In running some of the [existing testing around replication|https://github.com/apache/accumulo/tree/master/test/system/merkle-replication] (you da man, [~elserj]), I've hit what looks like a serious data loss problem. 

To provide some background: I'm testing Accumulo 1.7.2 using 2 6-node clusterdock-based clusters (source cluster nodes are named {{node-1.cluster}} through to {{node-6.cluster}} and destination cluster nodes are named {{node-7.cluster}} through {{node-12.cluster}}). After starting up the clusters, I first established that the test itself works (and, again, it's really nifty) by modifying {{merkle-env.sh}} to point to the correct ZK and Accumulo instances and setting {{NUM_RECORDS=100000000}} to shorten run times by an order of magnitude. I then run {{configure-replication.sh}}, {{ingest-data.sh}}, and then when the ingest script returns (and the UI shows ingest counts down to 0), I restart the source cluster to ensure WALs are available to replicate. I then wait and see that the replication UI on the source cluster says it has no more files to replicate (and that ingest on the destination cluster is also complete), and then run {{verify-data.sh}} to confirm that all went well. This goes off without a hitch and I've run it a dozen times without a single failure. Good.

Now, just to throw a spanner in the works, I enable fault-injection on the source cluster while the data is being written with {{ingest-data.sh}} and after restarting the Accumulo cluster on the source cluster. Because I'm running testing on clusters that use Cloudera Manager, I use an in-house fault injection tool that allows me to specify policies for killing any cluster services or roles using the CM REST API. In my case, I simply have it kill datanodes, the Accumulo master, tservers, tracers, and monitor at random intervals. Starting this up and then starting the data ingest doesn't seem to be a problem and my source cluster eventually gets all its data written and the script returns (and, it's worth noting, I get the same root hash for this source cluster across repeated runs). On the destination cluster, though, things don't seem to be behaving. In particular, I've observed that tservers appear to die with log messages ending with this ([full logs on Gist|https://gist.github.com/dimaspivak/2200ab10e0e00a4a7b81a5f6d83e3d85]):
{code}
2016-10-04 15:46:54,458 [replication.BatchWriterReplicationReplayer] INFO : Applying 11048 mutations to table replicationDestination as part of batch
2016-10-04 15:46:55,473 [zookeeper.ClientCnxn] WARN : Session 0x157904c6753001c for server node-7.cluster/192.168.124.8:2181, unexpected error, closing socket connection and attempting reconnect
java.io.IOException: Broken pipe
	at sun.nio.ch.FileDispatcherImpl.write0(Native Method)
	at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47)
	at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93)
	at sun.nio.ch.IOUtil.write(IOUtil.java:65)
	at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:487)
	at org.apache.zookeeper.ClientCnxnSocketNIO.doIO(ClientCnxnSocketNIO.java:117)
	at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:355)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)
2016-10-04 15:46:55,573 [zookeeper.ZooReader] WARN : Saw (possibly) transient exception communicating with ZooKeeper
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/tservers/node-8.cluster:10011
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1041)
	at org.apache.accumulo.fate.zookeeper.ZooReader.getStatus(ZooReader.java:132)
	at org.apache.accumulo.fate.zookeeper.ZooLock.process(ZooLock.java:383)
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:522)
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
2016-10-04 15:46:55,574 [zookeeper.ZooReader] WARN : Saw (possibly) transient exception communicating with ZooKeeper
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/replication/workqueue
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1468)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1496)
	at org.apache.accumulo.fate.zookeeper.ZooReader.getChildren(ZooReader.java:151)
	at org.apache.accumulo.server.zookeeper.DistributedWorkQueue$3.run(DistributedWorkQueue.java:210)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
2016-10-04 15:46:55,575 [zookeeper.ZooCache] WARN : Saw (possibly) transient exception communicating with ZooKeeper, will retry
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/tables/7/namespace
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1041)
	at org.apache.accumulo.fate.zookeeper.ZooCache$2.run(ZooCache.java:317)
	at org.apache.accumulo.fate.zookeeper.ZooCache$2.run(ZooCache.java:293)
	at org.apache.accumulo.fate.zookeeper.ZooCache$ZooRunnable.retry(ZooCache.java:189)
	at org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:345)
	at org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:280)
	at org.apache.accumulo.core.client.impl.Tables.getNamespaceId(Tables.java:235)
	at org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.setUpdateTablet(TabletServer.java:724)
	at org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.applyUpdates(TabletServer.java:769)
	at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.apache.accumulo.core.trace.wrappers.RpcServerInvocationHandler.invoke(RpcServerInvocationHandler.java:46)
	at org.apache.accumulo.server.rpc.RpcWrapper$1.invoke(RpcWrapper.java:74)
	at com.sun.proxy.$Proxy22.applyUpdates(Unknown Source)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$applyUpdates.getResult(TabletClientService.java:2425)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$applyUpdates.getResult(TabletClientService.java:2411)
	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
	at org.apache.accumulo.server.rpc.TimedProcessor.process(TimedProcessor.java:63)
	at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:516)
	at org.apache.accumulo.server.rpc.CustomNonBlockingServer$1.run(CustomNonBlockingServer.java:78)
	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)
2016-10-04 15:46:57,086 [zookeeper.ZooReader] WARN : Saw (possibly) transient exception communicating with ZooKeeper
org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/tservers/node-8.cluster:10011
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1041)
	at org.apache.accumulo.fate.zookeeper.ZooReader.getStatus(ZooReader.java:132)
	at org.apache.accumulo.fate.zookeeper.ZooLock.process(ZooLock.java:383)
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:522)
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
2016-10-04 15:46:57,086 [zookeeper.ZooCache] WARN : Saw (possibly) transient exception communicating with ZooKeeper, will retry
org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/tables
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1468)
	at org.apache.accumulo.fate.zookeeper.ZooCache$1.run(ZooCache.java:249)
	at org.apache.accumulo.fate.zookeeper.ZooCache$1.run(ZooCache.java:228)
	at org.apache.accumulo.fate.zookeeper.ZooCache$ZooRunnable.retry(ZooCache.java:189)
	at org.apache.accumulo.fate.zookeeper.ZooCache.getChildren(ZooCache.java:264)
	at org.apache.accumulo.core.client.impl.Tables.exists(Tables.java:142)
	at org.apache.accumulo.server.tabletserver.LargestFirstMemoryManager.tableExists(LargestFirstMemoryManager.java:150)
	at org.apache.accumulo.server.tabletserver.LargestFirstMemoryManager.getMemoryManagementActions(LargestFirstMemoryManager.java:176)
	at org.apache.accumulo.tserver.TabletServerResourceManager$MemoryManagementFramework.manageMemory(TabletServerResourceManager.java:414)
	at org.apache.accumulo.tserver.TabletServerResourceManager$MemoryManagementFramework.access$400(TabletServerResourceManager.java:324)
	at org.apache.accumulo.tserver.TabletServerResourceManager$MemoryManagementFramework$2.run(TabletServerResourceManager.java:352)
	at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
	at java.lang.Thread.run(Thread.java:745)
2016-10-04 15:46:57,086 [zookeeper.ZooReader] WARN : Saw (possibly) transient exception communicating with ZooKeeper
org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/replication/workqueue
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1468)
	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1496)
	at org.apache.accumulo.fate.zookeeper.ZooReader.getChildren(ZooReader.java:151)
	at org.apache.accumulo.server.zookeeper.DistributedWorkQueue$3.run(DistributedWorkQueue.java:210)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
2016-10-04 15:46:57,690 [watcher.MonitorLog4jWatcher] INFO : Changing monitor log4j address to node-7.cluster:4560
2016-10-04 15:46:57,690 [watcher.MonitorLog4jWatcher] INFO : Enabled log-forwarding
2016-10-04 15:46:57,698 [zookeeper.DistributedWorkQueue] INFO : Got unexpected zookeeper event: None for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/replication/workqueue
2016-10-04 15:46:57,698 [zookeeper.DistributedWorkQueue] INFO : Got unexpected zookeeper event: None for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/bulk_failed_copyq
2016-10-04 15:46:57,698 [zookeeper.DistributedWorkQueue] INFO : Got unexpected zookeeper event: None for /accumulo/75b12d57-3962-4427-8de7-1586b7cf7be5/recovery
2016-10-04 15:46:57,708 [tserver.TabletServer] ERROR: Lost tablet server lock (reason = SESSION_EXPIRED), exiting.
{code}
The Accumulo monitor on the destination cluster just notes {{unexpected failure}} as the event that led to the dead tablet server (in this case, {{node-8.cluster}}). 

While running the {{verify-data.sh}} script, I also see stack traces calling out tservers on the destination cluster (again, the one that doesn't have any fault injection being run against it):
{code}
2016-10-04 17:36:32,928 [cli.GenerateHashes] INFO : Using provided split points
Exception in thread "pool-1-thread-6" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-10.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-10.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
Exception in thread "pool-1-thread-8" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-9.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-9.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
Exception in thread "pool-1-thread-4" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-12.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-12.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
Exception in thread "pool-1-thread-10" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-12.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-12.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
Exception in thread "pool-1-thread-2" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-11.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-11.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
Exception in thread "pool-1-thread-12" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-11.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-11.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
Exception in thread "pool-1-thread-14" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-11.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-11.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
2016-10-04 17:36:33,118 [cli.GenerateHashes] INFO : pool-1-thread-1 computed digest for (-inf,1%00; : [] 9223372036854775807 false) of 7d55aa124671d95426aefee0ef5c8340
Exception in thread "pool-1-thread-1" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-9.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-9.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
2016-10-04 17:37:06,718 [cli.GenerateHashes] INFO : pool-1-thread-3 computed digest for [1%00; : [] 9223372036854775807 false,2%00; : [] 9223372036854775807 false) of 6cb4dfdfa15869cec69f4e29e296bbf5
Exception in thread "pool-1-thread-3" java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-12.cluster:10011
	at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161)
	at org.apache.accumulo.test.replication.merkle.cli.GenerateHashes$1.run(GenerateHashes.java:202)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server node-12.cluster:10011
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:293)
	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)
	... 4 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing startScan
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232)
	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410)
	at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285)
	... 6 more
2016-10-04 17:37:07,208 [cli.GenerateHashes] INFO : pool-1-thread-13 computed digest for [6%00; : [] 9223372036854775807 false,7%00; : [] 9223372036854775807 false) of 8126e795004a32fae39bf7b92ed8227d
2016-10-04 17:37:09,078 [cli.GenerateHashes] INFO : pool-1-thread-5 computed digest for [2%00; : [] 9223372036854775807 false,3%00; : [] 9223372036854775807 false) of 988f21ab0944b9646fce2c2a2ff1dc93
2016-10-04 17:37:11,748 [cli.GenerateHashes] INFO : pool-1-thread-15 computed digest for [7%00; : [] 9223372036854775807 false,8%00; : [] 9223372036854775807 false) of 6ac0817f132ba7c6cdc3f1bdeebe0f84
2016-10-04 17:37:12,223 [cli.GenerateHashes] INFO : pool-1-thread-11 computed digest for [5%00; : [] 9223372036854775807 false,6%00; : [] 9223372036854775807 false) of 86cdf23dce4398e7a4b1a24854553c51
2016-10-04 17:37:12,518 [cli.GenerateHashes] INFO : pool-1-thread-7 computed digest for [3%00; : [] 9223372036854775807 false,4%00; : [] 9223372036854775807 false) of d7920d8a683185060f16fe0eb0976857
2016-10-04 17:37:13,890 [cli.GenerateHashes] INFO : pool-1-thread-16 computed digest for [8%00; : [] 9223372036854775807 false,9%00; : [] 9223372036854775807 false) of 61606a9be65b436c87bbbc811af9ec82
2016-10-04 17:37:15,561 [cli.GenerateHashes] INFO : pool-1-thread-9 computed digest for [4%00; : [] 9223372036854775807 false,5%00; : [] 9223372036854775807 false) of 81a5f91bc5802860fb768fee541b01b4
2016-10-04 17:37:28,855 [cli.GenerateHashes] INFO : pool-1-thread-17 computed digest for [9%00; : [] 9223372036854775807 false,+inf) of fb435652798c75523850abc42fbc9a38

Computing root hash:
2016-10-04 17:37:30,108 [conf.ConfigSanityCheck] WARN : Use of instance.dfs.uri and instance.dfs.dir are deprecated. Consider using instance.volumes instead.
2016-10-04 17:37:30,313 [client.ClientConfiguration] WARN : Found no client.conf in default paths. Using default client configuration values.
2016-10-04 17:37:30,317 [client.ClientConfiguration] WARN : Found no client.conf in default paths. Using default client configuration values.
c5aa865d2d3e8ff0b12d4a25df9e50ab
2016-10-04 17:37:31,713 [conf.ConfigSanityCheck] WARN : Use of instance.dfs.uri and instance.dfs.dir are deprecated. Consider using instance.volumes instead.
2016-10-04 17:37:31,955 [client.ClientConfiguration] WARN : Found no client.conf in default paths. Using default client configuration values.
2016-10-04 17:37:31,959 [client.ClientConfiguration] WARN : Found no client.conf in default paths. Using default client configuration values.
8a24ae4e9233611ef4b9eb72d9ff2f1f
{code}
I'm still relatively new to the Accumulo scene and I'm planning on running some more testing to see if killing any particular Accumulo daemon triggers this, but wanted to open a JIRA to see if anyone else has encountered anything like this in the past.



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