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 BB9C1D151 for ; Thu, 6 Dec 2012 14:16:24 +0000 (UTC) Received: (qmail 73042 invoked by uid 500); 6 Dec 2012 14:16:24 -0000 Delivered-To: apmail-accumulo-user-archive@accumulo.apache.org Received: (qmail 73000 invoked by uid 500); 6 Dec 2012 14:16:24 -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 72988 invoked by uid 99); 6 Dec 2012 14:16:24 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 06 Dec 2012 14:16:24 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of christopher.burrell@gmail.com designates 209.85.223.178 as permitted sender) Received: from [209.85.223.178] (HELO mail-ie0-f178.google.com) (209.85.223.178) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 06 Dec 2012 14:16:17 +0000 Received: by mail-ie0-f178.google.com with SMTP id c12so10034611ieb.9 for ; Thu, 06 Dec 2012 06:15:56 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:reply-to:sender:in-reply-to:references:date :x-google-sender-auth:message-id:subject:from:to:content-type; bh=79o1OSmo1B1aCcgpJkNW9v5OaOD3W+A6xgrv0+yT3Wk=; b=LIfkRKxuejQZjUfci96QmYP4r5dLOsdXAepALymrvC2ABxMkTLCRuatkbyYoBu8oOU uoKRmbiUokUCMDYXjCagDZSaGhVTAolFwX4J0vZH9N6Cl/whCGVNDrcS+0GOmQ4m5ra9 knTAYqB1YIhMd5h/i3BcbTvutUMz8aXbzQM/BiNp/vOmQ07rSWVd8J8NPIqxWua1SjRM l1RxSnd5GiUc3gVd/CVTX0ahe88iSZ7TCe64J0dr1f9qvWxtQZ7/x8lGhPfVa1eQj9ND vMtbKwTIHRcuNuufnV5s3uuSBNuuK3Pc7a84CVAvKdbqRbhPmuZMfyh6UH/bVMFfVw8Q 7zxQ== MIME-Version: 1.0 Received: by 10.50.40.137 with SMTP id x9mr6032741igk.1.1354803356199; Thu, 06 Dec 2012 06:15:56 -0800 (PST) Reply-To: chris@burrell.me.uk Sender: christopher.burrell@gmail.com Received: by 10.64.14.166 with HTTP; Thu, 6 Dec 2012 06:15:56 -0800 (PST) In-Reply-To: References: Date: Thu, 6 Dec 2012 14:15:56 +0000 X-Google-Sender-Auth: A-Tzq7dPSkGMB0-g2qUuDR6afeY Message-ID: Subject: Re: Reduce task failing on job with error java.lang.IllegalStateException: Keys appended out-of-order From: Chris Burrell To: user@accumulo.apache.org Content-Type: multipart/alternative; boundary=14dae93404b9c9694604d02fbcf6 X-Virus-Checked: Checked by ClamAV on apache.org --14dae93404b9c9694604d02fbcf6 Content-Type: text/plain; charset=windows-1252 Content-Transfer-Encoding: quoted-printable Is this a limitation of the bulk ingest approach? Does the MapReduce job need to give the data to the AccumuloOutputFileFormat in a lexicographically-sorted manner? If so, is this not a rather big limitation of this approach, as you need to ensure your data comes in from your various data sources in a form such that the accumulo keys are then sorted. This seems to suggest that although the bulk ingest would be very quick, you would lose most of the time trying to sort and adapt the source files themselves in the MR job. Chris On 6 December 2012 14:08, William Slacum wr= ote: > Excuse me, 'col3' sorts lexicographically *after* 'col16'. > > > On Thu, Dec 6, 2012 at 9:07 AM, William Slacum < > wilhelm.von.cloud@accumulo.net> wrote: > >> 'col3' sorts lexicographically before 'col16'. you'll either need to >> encode your numerics or zero pad them. >> >> >> On Thu, Dec 6, 2012 at 9:03 AM, Andrew Catterall < >> catterallandrew@googlemail.com> wrote: >> >>> Hi, >>> >>> >>> I am trying to run a bulk ingest to import data into Accumulo but it is >>> failing at the reduce task with the below error: >>> >>> >>> >>> java.lang.IllegalStateException: Keys appended out-of-order. New key >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:col3 [ >>> myVis] 9223372036854775807 false, previous key client@20121206123059@00= 14efca-d8e8-492e-83cb-e5b6b7c49f7a >>> foo:col16 [myVis] 9223372036854775807 false >>> >>> at >>> org.apache.accumulo.core.file.rfile.RFile$Writer.append(RFile.java:378) >>> >>> >>> >>> Could this be caused by the order at which the writes are being done? >>> >>> >>> *-- Background* >>> >>> * >>> * >>> >>> The input file is a tab separated file. A sample row would look like: >>> >>> Data1 Data2 Data3 Data4 Data5 =85 DataN >>> >>> >>> >>> The map parses the data, for each row, into a Map. >>> This will contain the following: >>> >>> Col1 Data1 >>> >>> Col2 Data2 >>> >>> Col3 Data3 >>> >>> =85 >>> >>> ColN DataN >>> >>> >>> An outputKey is then generated for this row in the format * >>> client@timeStamp@randomUUID* >>> >>> Then for each entry in Map a outputValue is generated >>> in the format *ColN|DataN* >>> >>> The outputKey and outputValue are written to Context. >>> >>> >>> >>> This completes successfully, however, the reduce task fails. >>> >>> >>> My ReduceClass is as follows: >>> >>> >>> >>> *public* *static* *class* ReduceClass *extends* Reducer >>> { >>> >>> *public* *void* reduce(Text key, Iterable keyValues, >>> Context output) *throws* IOException, InterruptedException { >>> >>> >>> >>> // for each value belonging to the key >>> >>> *for* (Text keyValue : keyValues) { >>> >>> >>> >>> //split the keyValue into *Col* and Data >>> >>> String[] values =3D keyValue.toString().split("\\|= "); >>> >>> >>> >>> // Generate key >>> >>> Key outputKey =3D *new* Key(key, *new* Text("foo")= , * >>> new* Text(values[0]), *new* Text("myVis")); >>> >>> >>> >>> // Generate value >>> >>> Value outputValue =3D *new* Value(values[1].getByt= es(), >>> 0, values[1].length()); >>> >>> >>> >>> // Write to context >>> >>> output.write(outputKey, outputValue); >>> >>> } >>> >>> } >>> >>> } >>> >>> >>> >>> >>> *-- Expected output* >>> >>> >>> >>> I am expecting the contents of the Accumulo table to be as follows: >>> >>> >>> >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col1 [ >>> myVis] Data1 >>> >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col2 [ >>> myVis] Data2 >>> >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col3 [ >>> myVis] Data3 >>> >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col4 [ >>> myVis] Data4 >>> >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col5 [ >>> myVis] Data5 >>> >>> =85 >>> >>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:ColN [ >>> myVis] DataN >>> >>> >>> >>> >>> >>> Thanks, >>> >>> Andrew >>> >> >> > --14dae93404b9c9694604d02fbcf6 Content-Type: text/html; charset=windows-1252 Content-Transfer-Encoding: quoted-printable Is this a limitation of the bulk ingest approach? Does the MapReduce job ne= ed to give the data to the AccumuloOutputFileFormat in a=A0lexicographicall= y-sorted manner? If so, is this not a rather big limitation of this approac= h, as you need to ensure your data comes in from your various data sources = in a form such that the accumulo keys are then sorted.

This seems to suggest that although the bulk ingest would be= very quick, you would lose most of the time trying to sort and adapt the s= ource files themselves in the MR job.

Chris



On 6 December 2012 14:08, William Slacum <wilhelm.von.cloud= @accumulo.net> wrote:
Excuse me, 'col3' sorts lexicographi= cally *after* 'col16'.

<= br>
On Thu, Dec 6, 2012 at 9:07 AM, William Slacum <wilhelm.von.c= loud@accumulo.net> wrote:
'col3' sorts lexicographically befor= e 'col16'. you'll either need to encode your numerics or zero p= ad them.


On Thu, Dec 6, 2012 at 9:03 AM, And= rew Catterall <catterallandrew@googlemail.com> = wrote:

Hi,


I am trying to= run a bulk ingest to import data into Accumulo but it is failing at the re= duce task with the below error:

=A0

java.lang.IllegalStateException: Keys appende= d out-of-order.=A0 New key client@20121206123059@0014efca-d8e8-492e-83cb-e5= b6b7c49f7a foo:col3 [myVis] 922= 3372036854775807 false, previous key client@20121206123059@0014efca-d8e8-49= 2e-83cb-e5b6b7c49f7a foo:col16 [myVis<= /span>] 9223372036854775807 false

<= span style=3D"font-family:'Courier New'">=A0=A0=A0=A0=A0=A0=A0 at o= rg.apache.accumulo.core.file.rfile.RFile$Writer.append(RFile.java:378)

=A0

Could this = be caused by the order at which the writes are being done?


-- Background


The input file is a tab separated file.=A0 A sample row would look like:=

Data1=A0=A0=A0= Data2=A0=A0=A0 Data3=A0=A0=A0 Data4=A0=A0=A0 Data5=A0=A0=A0 =85=A0=A0=A0= =A0=A0=A0=A0=A0=A0=A0=A0=A0 DataN

=A0

The map parses the= data, for each row, into a=A0Map<String, String>.=A0 This will contain the following:<= /p>

Col1=A0=A0=A0=A0=A0=A0 Data1

Col2=A0=A0=A0=A0=A0=A0 Data2

Col3=A0=A0=A0=A0=A0=A0 Data3

=85

ColN=A0=A0=A0=A0=A0 Da= taN


An=A0outputKey=A0is then generated for this = row in the format=A0client@timeStamp@randomUUID

Then for each entr= y in=A0Map<String, Str= ing>=A0a=A0outp= utValue=A0is generated in the format=A0ColN|DataN

The=A0outputKey=A0and=A0outputValue=A0are written to= =A0Context.

=A0

This completes successfully, = however, the reduce task fails.


My=A0ReduceClass=A0is as fol= lows:

=A0

=A0=A0=A0=A0=A0=A0public=A0<= /span>static= =A0class=A0ReduceClass=A0extends=A0Reducer<Text,Text,Key,Value> {

=A0=A0=A0 =A0=A0 =A0=A0public=A0void=A0reduce(Text = key, Iterable<Text> keyValues, Context output)=A0throws= =A0IOException, Int= erruptedException {

=A0=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0 =A0=A0// for each value belonging to the key=

=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0=A0=A0 =A0= =A0for=A0(Text keyValue : keyValues) {

=A0=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0 =A0= =A0

=A0=A0=A0=A0=A0=A0 =A0= =A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0 =A0=A0//split the keyValue into= =A0Col=A0and Data =A0=A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0=A0String[] values =3D keyValue.toString().split(<= span style=3D"font-size:10pt;font-family:Consolas;color:rgb(42,0,255)">&quo= t;\\|");

=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0 =A0= =A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0// Generate key

=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0 Key outputKey =3D=A0new=A0Key(key,=A0new=A0Text("foo"= ;),=A0= new= =A0Text(valu= es[0]),=A0new=A0Text("myVis"));

=A0=A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0 =A0= =A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0// Generate value

=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0 Value outputValue =3D=A0new=A0Value(values[1].getBytes(), 0, = values[1].length());=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0 =A0= =A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0// Write to context

=A0=A0=A0=A0=A0=A0 =A0=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0 output.write(outputKey, outputValue);=A0=A0=A0=A0=A0=A0= =A0=A0=A0=A0=A0=A0=A0

=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0=A0 = =A0=A0}

=A0=A0=A0 =A0=A0 =A0=A0= }

=A0=A0=A0=A0=A0 }

=A0


-- Expected output

=A0

I am expect= ing the contents of the Accumulo table to be as follows:

=A0

client@20121206123059@001= 4efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col1 [myVis] Data1

client@20121206123059@001= 4efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col2 [myVis] Data2

client@20121206123059@001= 4efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col3 [myVis] Data3

client@20121206123059@001= 4efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col4 [myVis] Data4

client@20121206123059@001= 4efca-d8e8-492e-83cb-e5b6b7c49f7a foo:Col5 [myVis] Data5

=85

client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c4= 9f7a foo:ColN [myVis] DataN

=A0

=A0

Thanks,

Andrew




--14dae93404b9c9694604d02fbcf6--