accumulo-notifications mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Mike Drob (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ACCUMULO-2388) Continuous Ingest clients die
Date Mon, 14 Apr 2014 18:35:16 GMT

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

Mike Drob commented on ACCUMULO-2388:
-------------------------------------

By increasing the tserver rpc timeout, we're not really doing anything to address the issue.
It deals with the proximal cause of clients dying, but eventually somebody will reach the
point where 4 minutes is not enough due to the load on their cluster and up it goes again.
Does increasing the rpc timeout have adverse affects on anything else? Like causing servers
to hold on to more requests in memory?

I feel like we should build some retry capability into clients for this. Probably would be
good to do it inside of the [Batch]Scanner implementation so that users don't even need to
worry about it. I remember discussing the possibility of scanners that could advertise their
own timeout, did that ever get a JIRA filed? If not, it should.

> Continuous Ingest clients die
> -----------------------------
>
>                 Key: ACCUMULO-2388
>                 URL: https://issues.apache.org/jira/browse/ACCUMULO-2388
>             Project: Accumulo
>          Issue Type: Bug
>          Components: test, tserver
>         Environment: 1.6.0-SNAPSHOT (sha-1: 0da9a56)
> cdh4.5.0
>            Reporter: Mike Drob
>            Assignee: Mike Drob
>            Priority: Minor
>              Labels: 16_qa_bug
>             Fix For: 1.6.1
>
>         Attachments: tracer.debug.log, tserver1.log
>
>
> I was running continuous ingest on a 7 node cluster (5 slaves) and after enabling HDFS
agitation, my clients died.
> {code:title=ingest.err}
> Thread "org.apache.accumulo.test.continuous.ContinuousIngest" died java.lang.reflect.InvocationTargetException
> java.lang.reflect.InvocationTargetException
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> at java.lang.reflect.Method.invoke(Method.java:597)
> at org.apache.accumulo.start.Main$1.run(Main.java:137)
> at java.lang.Thread.run(Thread.java:662)
> Caused by: java.lang.reflect.UndeclaredThrowableException
> at $Proxy9.addMutation(Unknown Source)
> at org.apache.accumulo.test.continuous.ContinuousIngest.main(ContinuousIngest.java:212)
> ... 6 more
> Caused by: java.lang.reflect.InvocationTargetException
> at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> at java.lang.reflect.Method.invoke(Method.java:597)
> at org.apache.accumulo.trace.instrument.TraceProxy$2.invoke(TraceProxy.java:43)
> ... 8 more
> Caused by: org.apache.accumulo.core.client.MutationsRejectedException: # constraint violations
: 0 security codes: {} # server errors 1 # exceptions 0
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter.checkForFailures(TabletServerBatchWriter.java:537)
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter.addMutation(TabletServerBatchWriter.java:258)
> at org.apache.accumulo.core.client.impl.BatchWriterImpl.addMutation(BatchWriterImpl.java:43)
> ... 12 more
> {code}
> {code:title=ingest.out}
> UUID 1392844086463 f822a6a9-9592-4b3a-ab3b-1c172be20b96
> FLUSH 1392844135523 49047 6165 1000000 1000000
> FLUSH 1392844165594 30071 7787 2000000 1000000
> FLUSH 1392844195875 30281 7816 3000000 1000000
> FLUSH 1392844226787 30912 8086 4000000 1000000
> FLUSH 1392844257194 30407 7989 5000000 1000000
> FLUSH 1392844287518 30324 7743 6000000 1000000
> FLUSH 1392844325833 38315 10933 7000000 1000000
> FLUSH 1392844364708 38875 7916 8000000 1000000
> FLUSH 1392844395818 31110 8104 9000000 1000000
> 2014-02-19 13:16:57,444 [impl.TabletServerBatchWriter] ERROR: Server side error on tserver1:10011:
org.apache.thrift.TApplicationException: Internal error processing applyUpdates
> 2014-02-19 13:16:57,446 [impl.TabletServerBatchWriter] ERROR: Failed to send tablet server
tserver1:10011 its batch : Error on server tserver1:10011
> org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server tserver1:10011
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter.sendMutationsToTabletServer(TabletServerBatchWriter.java:937)
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter.access$1600(TabletServerBatchWriter.java:616)
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter$SendTask.send(TabletServerBatchWriter.java:801)
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter$SendTask.run(TabletServerBatchWriter.java:765)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 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:662)
> Caused by: org.apache.thrift.TApplicationException: Internal error processing applyUpdates
> at org.apache.thrift.TApplicationException.read(TApplicationException.java:108)
> at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
> at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_closeUpdate(TabletClientService.java:431)
> at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.closeUpdate(TabletClientService.java:417)
> at org.apache.accumulo.core.client.impl.TabletServerBatchWriter$MutationWriter.sendMutationsToTabletServer(TabletServerBatchWriter.java:899)
> ... 11 more
> {code}
> {code:title=tserver.log}
> 2014-02-19 13:16:56,156 [util.TServerUtils$THsHaServer] WARN : Got an IOException in
internalRead!
> java.io.IOException: Connection reset by peer
> at sun.nio.ch.FileDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:21)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:198)
> at sun.nio.ch.IOUtil.read(IOUtil.java:171)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:243)
> at org.apache.thrift.transport.TNonblockingSocket.read(TNonblockingSocket.java:141)
> at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.internalRead(AbstractNonblockingServer.java:515)
> at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.read(AbstractNonblockingServer.java:355)
> at org.apache.thrift.server.AbstractNonblockingServer$AbstractSelectThread.handleRead(AbstractNonblockingServer.java:202)
> at org.apache.thrift.server.TNonblockingServer$SelectAcceptThread.select(TNonblockingServer.java:198)
> at org.apache.thrift.server.TNonblockingServer$SelectAcceptThread.run(TNonblockingServer.java:154)
> {code}
> Note that this last message was not propagated to the monitor for some reason, but that
is likely a different issue. (I had been seeing other WARN messages show up earlier.)



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message