Eric,

Apologies for the delayed reply, but do i really need to expand my cluster. The ingest rate is not so high( around 100 - 500 docs/sec) and I hope, Accumulo can handle it easily with 3 Tservers.

The swap memory problem is not longer exists. Servers are not using any swap memory now. And after diagnosing the clients, the mutations rejected exception also does not appear till now. I reduced the number of threads running on clients to ingest data and tuned some JVM parameters. Thanks

But I ran into a new problem

org.apache.accumulo.tserver.HoldTimeoutException: Commits are held

due to this two if my tservers died or killed by master. This is serious

2016-01-01 00:16:34,829 [master.Master] WARN : Tablet server 
orkash1:9997[1517fe14295029c] exceeded maximum hold time: attempting to kill it
2016-01-01 00:16:34,829 [master.LiveTServerSet] INFO : Removing zookeeper lock 
for orkash1:9997[1517fe14295029c]
2015-12-31 23:27:39,973 [master.Master] WARN : Tablet server 
orkash3:9997[351edd882900000] exceeded maximum hold time: attempting to kill it
2015-12-31 23:27:39,992 [master.LiveTServerSet] INFO : Removing zookeeper lock 
for orkash3:9997[351edd882900000]


Further, I found other exceptions in the monitor logs related to writer, flushes and some more .

(1) Problem flushing traces, resetting writer. Set log level to DEBUG to see 
stacktrace cause: org.apache.accumulo.core.client.MutationsRejectedException: # 
constraint violations : 0  security codes: {}  # server errors 1 # exceptions 0
(2) Server side error on orkash2:9997: org.apache.thrift.TApplicationException: 
Internal error processing closeUpdate
(3) Unable to write mutation to table; discarding span.Set log level to DEBUG 
to see stacktrace cause: 
org.apache.accumulo.core.client.MutationsRejectedException: # constraint 
violations : 0  security codes: {}  # server errors 1 # exceptions 1
(4) Problem closing batch writer. Set log level to DEBUG to see stacktrace. 
cause: org.apache.accumulo.core.client.MutationsRejectedException: # constraint 
violations : 0  security codes: {}  # server errors 1 # exceptions 1
(5) Commits are held
        org.apache.accumulo.tserver.HoldTimeoutException: Commits are held
                at 
org.apache.accumulo.tserver.TabletServerResourceManager.waitUntilCommitsAreEnabled(TabletServerResourceManager.java:493)
                at 
org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.flush(TabletServer.java:795)
                at 
org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.closeUpdate(TabletServer.java:952)
                at sun.reflect.GeneratedMethodAccessor13.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:47)
                at com.sun.proxy.$Proxy19.closeUpdate(Unknown Source)
                at 
org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$closeUpdate.getResult(TabletClientService.java:2446)
                at 
org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$closeUpdate.getResult(TabletClientService.java:2430)
                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)
(6) Assignment for 4<;5684390f has been running for at least 822922ms
        java.lang.Exception: Assignment of 4<;5684390f
                at java.lang.Object.wait(Native Method)
                at java.lang.Object.wait(Object.java:503)
                at 
org.apache.hadoop.hdfs.DFSOutputStream.waitAndQueueCurrentPacket(DFSOutputStream.java:1755)
                at 
org.apache.hadoop.hdfs.DFSOutputStream.writeChunkImpl(DFSOutputStream.java:1839)
                at 
org.apache.hadoop.hdfs.DFSOutputStream.writeChunk(DFSOutputStream.java:1788)
                at 
org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:206)
                at 
org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:124)
                at 
org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:110)
                at 
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
                at java.io.DataOutputStream.write(DataOutputStream.java:107)
                at 
org.apache.accumulo.core.file.rfile.bcfile.SimpleBufferedOutputStream.flushBuffer(SimpleBufferedOutputStream.java:39)
                at 
org.apache.accumulo.core.file.rfile.bcfile.SimpleBufferedOutputStream.flush(SimpleBufferedOutputStream.java:68)
                at 
org.apache.hadoop.io.compress.CompressionOutputStream.flush(CompressionOutputStream.java:69)
                at 
org.apache.accumulo.core.file.rfile.bcfile.Compression$FinishOnFlushCompressionStream.flush(Compression.java:66)
                at 
java.io.BufferedOutputStream.flush(BufferedOutputStream.java:141)
                at 
org.apache.accumulo.core.file.rfile.bcfile.BCFile$Writer$WBlockState.finish(BCFile.java:233)
                at 
org.apache.accumulo.core.file.rfile.bcfile.BCFile$Writer$BlockAppender.close(BCFile.java:320)
                at 
org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile$BlockWrite.close(CachableBlockFile.java:121)
                at 
org.apache.accumulo.core.file.rfile.RFile$Writer.closeBlock(RFile.java:398)
                at 
org.apache.accumulo.core.file.rfile.RFile$Writer.append(RFile.java:382)
                at 
org.apache.accumulo.tserver.tablet.Compactor.compactLocalityGroup(Compactor.java:356)
                at 
org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:208)
                at 
org.apache.accumulo.tserver.tablet.MinorCompactor.call(MinorCompactor.java:97)
                at 
org.apache.accumulo.tserver.tablet.Tablet.minorCompact(Tablet.java:949)
                at 
org.apache.accumulo.tserver.tablet.MinorCompactionTask.run(MinorCompactionTask.java:84)
                at 
org.apache.accumulo.tserver.tablet.Tablet.minorCompactNow(Tablet.java:1078)
                at 
org.apache.accumulo.tserver.TabletServer$AssignmentHandler.run(TabletServer.java:2165)
                at 
org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
                at 
org.apache.accumulo.tserver.ActiveAssignmentRunnable.run(ActiveAssignmentRunnable.java:61)
                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)


*I expected that all the tablets hosted by these tservers will be moved to the only alive tserver and the mutations going to the tablets will all be moved to the alive tserver but this wasn't the case* more than 100 tablets were left unassigned and all the mutations going to those servers got rejected, Strange? why the tablets did not move to other tserver? I need to now but no info in logs. And why the hold time exceeded maximum limit? can it be a network issue? Please provide your inputs and help me to handle this.
/
//*org.apache.accumulo.core.client.MutationsRejectedException: # constraint violations : 0 security codes: {} # server errors 0 # exceptions 17*// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter.checkForFailures(TabletServerBatchWriter.java:537)// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter.addMutation(TabletServerBatchWriter.java:249)// // at org.apache.accumulo.core.client.impl.MultiTableBatchWriterImpl$TableBatchWriter.addMutation(MultiTableBatchWriterImpl.java:64)// // at com.orkash.accumulo.IngestionWithoutServiceOnCondition.main(IngestionWithoutServiceOnCondition.java:246)//
//    at com.orkash.db.DBQuery.insertDB(DBQuery.java:326)//
//    at com.orkash.Crawling.CrawlerThread.run(CrawlerThread.java:143)//
//    at java.lang.Thread.run(Thread.java:745)//
////*Caused by: org.apache.accumulo.core.client.TimedOutException: Failed to obtain metadata*// // at org.apache.accumulo.core.client.impl.TimeoutTabletLocator.failed(TimeoutTabletLocator.java:46)// // at org.apache.accumulo.core.client.impl.TimeoutTabletLocator.binMutations(TimeoutTabletLocator.java:85)// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter.binMutations(TabletServerBatchWriter.java:653)// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter.addMutations(TabletServerBatchWriter.java:694)// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter.startProcessing(TabletServerBatchWriter.java:233)// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter.access$200(TabletServerBatchWriter.java:101)// // at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$1.run(TabletServerBatchWriter.java:219)//
//    at java.util.TimerThread.mainLoop(Timer.java:555)//
//    at java.util.TimerThread.run(Timer.java:505)/

I can post more details, if you want and start a new thread for this, if needed.

Thanks
-Mohit Kaushik


On 12/26/2015 06:24 PM, Eric Newton wrote:
Generally speaking, rejected mutations due to resource contention is considered a system failure, requiring a re-examination of system resources.

That requires re-architecting your ingest or adding significant resources.

You could do some substantial pre-processing of your ingest and bulk-load the result. It will increase latency of the incoming information, but it will reduce the pressure on accumulo.

Or, as I suggested, you could increase your processing/storage by an order of magnitude. That is why the software is built to handle hundreds (or more) nodes.

3-5G of swap out of 32G is not a lot. But why is it using any at all? Pulling 3G from disk is not going to be very fast. If you must, reduce the size of your tserver. Focus on keeping your system at zero swap.

I suggest, again, that you consider expanding your system to many more nodes. Accumulo is not written in hand-tuned assembler. It was written with the knowledge that more hardware is pretty cheap, and scaling up is better than small inefficiencies.



On Thu, Dec 24, 2015 at 5:49 AM, mohit.kaushik <mohit.kaus...@orkash.com <mailto:mohit.kaus...@orkash.com>> wrote:


    @ Eric:  yes I have notices 3GB to 5GB swap uses out of 32GB on
    servers. And if I will resend the mutations rejected explicitly
    then this may create a loop for mutations getting rejected again
    and again. Then how can I handle it? How did you? Am i getting it
    right?
    @ Josh: For one of the zookeeper host I was sharing the same drive
    to store zookeeper data and hadoop datanode. I have changed it to
    the same drive as others have. I hope this will resolve zookeeper
    issue. lets see

    BTW, here is my zoo.cfg
    clientPort=2181
    dataDir=/usr/local/zookeeper/data/
    syncLimit=5
    tickTime=2000
    initLimit=10
    maxClientCnxn=100
    server.1=orkash1:2888:3888
    server.2=orkash2:2888:3888
    server.3=orkash3:2888:3888

    Thanks a lot
    Mohit Kaushik



    On 12/24/2015 12:47 AM, Josh Elser wrote:
    Eric Newton wrote:

    Failure to talk to zookeeper is *really* unexpected.

    Have you noticed your nodes using any significant swap?

    Emphasis on this. Failing to connect to ZooKeeper for 60s (2*30)
    is a very long time (although, I think I have seen JVM GC pauses
    longer before).

    A couple of generic ZooKeeper questions:

    1. Can you share your zoo.cfg?

    2. Make sure that ZooKeeper has a "dedicated" drive for it's
    dataDir. HDFS DataNodes using the same drive as ZooKeeper for its
    transaction log can cause ZooKeeper to be starved for I/O
    throughput. A normal "spinning" disk is also better for ZK over
    SSDs (last I read).

    3. Check OS/host level metrics on these ZooKeeper hosts during
    the times you see these failures.

    4. Consider moving your ZooKeeper hosts to "less busy" nodes if
    you can. You can consider adding more ZooKeeper hosts to the
    quorum, but keep in mind that this will increase the minimum
    latency for ZooKeeper operations (as more nodes need to
    acknowledge updates n/2 + 1)




Reply via email to