Return-Path: X-Original-To: apmail-accumulo-user-archive@www.apache.org Delivered-To: apmail-accumulo-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 56A9F11A7F for ; Wed, 18 Jun 2014 16:49:17 +0000 (UTC) Received: (qmail 52754 invoked by uid 500); 18 Jun 2014 16:49:17 -0000 Delivered-To: apmail-accumulo-user-archive@accumulo.apache.org Received: (qmail 52705 invoked by uid 500); 18 Jun 2014 16:49:17 -0000 Mailing-List: contact user-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@accumulo.apache.org Delivered-To: mailing list user@accumulo.apache.org Received: (qmail 52694 invoked by uid 99); 18 Jun 2014 16:49:17 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 18 Jun 2014 16:49:17 +0000 X-ASF-Spam-Status: No, hits=-0.7 required=5.0 tests=RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of josh.elser@gmail.com designates 209.85.220.51 as permitted sender) Received: from [209.85.220.51] (HELO mail-pa0-f51.google.com) (209.85.220.51) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 18 Jun 2014 16:49:14 +0000 Received: by mail-pa0-f51.google.com with SMTP id hz1so929325pad.38 for ; Wed, 18 Jun 2014 09:48:49 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=message-id:date:from:user-agent:mime-version:to:subject:references :in-reply-to:content-type:content-transfer-encoding; bh=Uw3jzNTEtIPZr4H8XDcoegufd6XZ2F4prNynoL0l58U=; b=uCEEhsjqKxWAoqieusowlibYiJ3KjY7Uh5I5/GVsvKycmfEvcKW2E6rDGSRGntsuwR HWWzzmcTy7nwzVPbkGzViCvvQfCZI4GiVT7ab2w2ilIhRtJzvWOlhUzHF0LV6YFoKxMp 2WHT6ToE5+Ki+m7FnX9vaU+KpLtyKGUhveqsgDWTTcGmRLrxd4Q0tl0e5L6fz5bibvVE ok0pJkMlBNBCHzKcj2Ex4uCTr1ODYZ1o2qW8/h0ksSI/0BL+R6Xc+MYDCciZGgxeQwnB +T+RofXh/riPcVWy7BG+PtTe9Z1pe/JyybFJVP1w5QAvvHN9heeLvYqkVQhuxBNQEmfh ampA== X-Received: by 10.68.245.100 with SMTP id xn4mr3945442pbc.152.1403110128909; Wed, 18 Jun 2014 09:48:48 -0700 (PDT) Received: from HW10447.local ([192.175.27.2]) by mx.google.com with ESMTPSA id su8sm4303239pbc.72.2014.06.18.09.48.47 for (version=TLSv1 cipher=ECDHE-RSA-RC4-SHA bits=128/128); Wed, 18 Jun 2014 09:48:48 -0700 (PDT) Message-ID: <53A1C2EF.7070504@gmail.com> Date: Wed, 18 Jun 2014 09:48:47 -0700 From: Josh Elser User-Agent: Mozilla/5.0 (Macintosh; Intel Mac OS X 10.9; rv:24.0) Gecko/20100101 Thunderbird/24.6.0 MIME-Version: 1.0 To: user@accumulo.apache.org Subject: Re: MutationsRejectedException and TApplicationException References: <53A11987.5080003@gmail.com> In-Reply-To: Content-Type: text/plain; charset=UTF-8; format=flowed Content-Transfer-Encoding: 7bit X-Virus-Checked: Checked by ClamAV on apache.org Which memory size? :) JVM max heap size? In-memory map size? Either (really, both) would be good to increase. If the situation is as Eric posited, it's very likely that increasing the resources that Accumulo has available would make it more responsive and able to keep up with your ingest load. - Josh On 6/18/14, 9:44 AM, Jianshi Huang wrote: > Does memory size important to reduce this sort of errors? I used the > default settings (1GB/per tserver), this might be too small. > > I increased it to 20GB, and I saw no errors after that. > > Jianshi > > > On Thu, Jun 19, 2014 at 12:39 AM, Eric Newton > wrote: > > This error is often a result of overwhelming your server resources. > > It basically says "an update came in that was so old, the id used to > identify the sender has already aged off." > > What is your expected ingest rate during the job? What sort of > resources does accumulo have? > > > On Wed, Jun 18, 2014 at 7:09 AM, Jianshi Huang > > wrote: > > Here's the error message I got from the tserver_xxx.log > > 2014-06-18 01:06:06,816 [tserver.TabletServer] INFO : Adding 1 > logs for extent g;cust:2072821;cust:20700111 as alias 37 > 2014-06-18 01:06:16,286 [thrift.ProcessFunction] ERROR: Internal > error processing applyUpdates > java.lang.RuntimeException: No Such SessionID > at > org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.applyUpdates(TabletServer.java:1522) > at sun.reflect.GeneratedMethodAccessor4.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.TraceWrap$1.invoke(TraceWrap.java:63) > at com.sun.proxy.$Proxy23.applyUpdates(Unknown Source) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$applyUpdates.getResult(TabletClientService.java:2347\ > ) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$applyUpdates.getResult(TabletClientService.java:2333\ > ) > 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:171) > at > org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:478) > at > org.apache.accumulo.server.util.TServerUtils$THsHaServer$Invocation.run(TServerUtils.java:231) > 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:724) > 2014-06-18 01:06:16,287 [thrift.ProcessFunction] ERROR: Internal > error processing applyUpdates > java.lang.RuntimeException: No Such SessionID > at > org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.applyUpdates(TabletServer.java:1522) > at sun.reflect.GeneratedMethodAccessor4.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.TraceWrap$1.invoke(TraceWrap.java:63) > at com.sun.proxy.$Proxy23.applyUpdates(Unknown Source) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$applyUpdates.getResult(TabletClientService.java:2347\ > ) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$applyUpdates.getResult(TabletClientService.java:2333\ > ) > 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:171) > at > org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:478) > at > org.apache.accumulo.server.util.TServerUtils$THsHaServer$Invocation.run(TServerUtils.java:231) > 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:724) > 2014-06-18 01:06:16,287 [util.TServerUtils$THsHaServer] WARN : > Got an IOException during write! > java.io.IOException: Connection reset by peer > 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.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:164) > at > org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.write(AbstractNonblockingServer.java:381) > at > org.apache.thrift.server.AbstractNonblockingServer$AbstractSelectThread.handleWrite(AbstractNonblockingServer.java:220) > at > org.apache.thrift.server.TNonblockingServer$SelectAcceptThread.select(TNonblockingServer.java:201) > > > Jianshi > > > On Wed, Jun 18, 2014 at 2:54 PM, Jianshi Huang > > wrote: > > I see. I'll check the tablet server log and paste the error > message in later thread. > > BTW, looks like the AccumuloOutputFormat is the cause, I'm > currently using BatchWriter and it works well. > > My code looks like this (it's in Scala as I'm using Spark): > > AccumuloOutputFormat.setZooKeeperInstance(job, > Conf.getString("accumulo.instance"), > Conf.getString("accumulo.zookeeper.servers")) > AccumuloOutputFormat.setConnectorInfo(job, > Conf.getString("accumulo.user"), new > PasswordToken(Conf.getString("accumulo.password"))) > AccumuloOutputFormat.setDefaultTableName(job, > Conf.getString("accumulo.table")) > > val paymentRDD: RDD[(Text, Mutation)] = payment.flatMap > { payment => > // val key = new Text(Conf.getString("accumulo.table")) > paymentMutations(payment).map((null, _)) > } > > > paymentRDD.saveAsNewAPIHadoopFile(Conf.getString("accumulo.instance"), > classOf[Void], classOf[Mutation], > classOf[AccumuloOutputFormat], job.getConfiguration) > > > It's also possible that saveAsNewAPIHadoopFile doesn't work > well with AccumuloOutputFormat. > > > > Jianshi > > > > > On Wed, Jun 18, 2014 at 12:45 PM, Josh Elser > > wrote: > > Check the TabletServer logs. This Exception is telling > you that there was an error on the server. You should > look there for what the real problem was. You can do > this one of two ways. > > 1) Use the "Recent Logs" page on the Accumulo monitor > (http://accumulo_monitor_host:__50095 > ). Unless you > cleared the logs, or restarted the monitor process since > you got this error, you should be able to see a nice > HTML view of any errors > > 2) Check the debug log, e.g. > $ACCUMULO_HOME/logs/tserver_$__host.debug.log. If you're > running tservers on more than one node, be sure that you > check the log files on all nodes. > > - Josh > > > On 6/17/14, 9:33 PM, Jianshi Huang wrote: > > Hi, > > I got the following errors during MapReduce > ingestion, are they serious > errors? > > java.io.IOException: > org.apache.accumulo.core.__client.__MutationsRejectedException: > # constraint > violations : 0 security codes: {} # server\ errors > 1 # exceptions 0 > at > org.apache.accumulo.core.__client.mapreduce.__AccumuloOutputFormat$__AccumuloRecordWriter.write(__AccumuloOutputFormat.java:437) > at > org.apache.accumulo.core.__client.mapreduce.__AccumuloOutputFormat$__AccumuloRecordWriter.write(__AccumuloOutputFormat.java:373) > at > org.apache.spark.rdd.__PairRDDFunctions.org > > >$apache$__spark$rdd$PairRDDFunctions$$__writeShard$1(PairRDDFunctions.__scala:716) > > at > org.apache.spark.rdd.__PairRDDFunctions$$anonfun$__saveAsNewAPIHadoopDataset$1.__apply(PairRDDFunctions.scala:__730) > at > org.apache.spark.rdd.__PairRDDFunctions$$anonfun$__saveAsNewAPIHadoopDataset$1.__apply(PairRDDFunctions.scala:__730) > at > org.apache.spark.scheduler.__ResultTask.runTask(ResultTask.__scala:111) > at > org.apache.spark.scheduler.__Task.run(Task.scala:51) > at > org.apache.spark.executor.__Executor$TaskRunner.run(__Executor.scala:187) > 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:724) > > > And > > java.io.IOException: > org.apache.accumulo.core.__client.AccumuloException: > org.apache.thrift.__TApplicationException: Internal > error processing\ > applyUpdates > at > org.apache.accumulo.core.__client.mapreduce.__AccumuloOutputFormat.__getRecordWriter(__AccumuloOutputFormat.java:558) > at > org.apache.spark.rdd.__PairRDDFunctions.org > > >$apache$__spark$rdd$PairRDDFunctions$$__writeShard$1(PairRDDFunctions.__scala:712) > > at > org.apache.spark.rdd.__PairRDDFunctions$$anonfun$__saveAsNewAPIHadoopDataset$1.__apply(PairRDDFunctions.scala:__730) > at > org.apache.spark.rdd.__PairRDDFunctions$$anonfun$__saveAsNewAPIHadoopDataset$1.__apply(PairRDDFunctions.scala:__730) > at > org.apache.spark.scheduler.__ResultTask.runTask(ResultTask.__scala:111) > at > org.apache.spark.scheduler.__Task.run(Task.scala:51) > at > org.apache.spark.executor.__Executor$TaskRunner.run(__Executor.scala:187) > 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:724) > > > Cheers, > -- > Jianshi Huang > > LinkedIn: jianshi > Twitter: @jshuang > Github & Blog: http://huangjs.github.com/ > > > > > -- > Jianshi Huang > > LinkedIn: jianshi > Twitter: @jshuang > Github & Blog: http://huangjs.github.com/ > > > > > -- > Jianshi Huang > > LinkedIn: jianshi > Twitter: @jshuang > Github & Blog: http://huangjs.github.com/ > > > > > > -- > Jianshi Huang > > LinkedIn: jianshi > Twitter: @jshuang > Github & Blog: http://huangjs.github.com/