accumulo-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Josh Elser <josh.el...@gmail.com>
Subject Re: Reduce task failing on job with error java.lang.IllegalStateException: Keys appended out-of-order
Date Fri, 07 Dec 2012 03:33:23 GMT
Yup, the AccumuloOutputFormat essentially uses a BatchWriter.

I'm all in favor of any for fixing any inconsistencies or ambiguous areas
that might exist in the manual.

"high-speed" is a very subjective term, I do agree. It falls back to the
common assumption that a MapReduce job can sort resulting KeyValue pairs
quicker than Accumulo can for a given table (which is usually the case).

Please feel free to make a ticket, submit a patch with some wording
changes, etc. Making the end-user experience better is the ultimate goal.


On Thu, Dec 6, 2012 at 1:35 PM, Chris Burrell <chris@burrell.me.uk> wrote:

> Thanks. By limitation, I wasn't meaning drawback. I completely agree that
> this is a very useful feature. I was just trying to understand the
> requirements for the various "high-speed ingest" methods outlines on the
> Accumulo User manual.
>
> Can I suggest that we add a bit more detail to the Accumulo User Manual?
> In particular, the two points mentioned above.
>
>    - The AccumuloOutputFileFormat is the format used internally by
>    Accumulo to store the files.
>    - As a results of the above, the MapReduce job is required to create
>    the keys (rowId, columnFamily, columnQualifier, timestamp) in
>    lexicographical order.
>
> Presumably the MapReduce Ingest (AccumuloOutputFormat) uses the
> BatchWriters in the background?
> Chris
>
>
>
> On 6 December 2012 15:15, Josh Elser <josh.elser@gmail.com> wrote:
>
>>  The point of bulk-ingest is that you can perform this work "out of band"
>> from Accumulo. You can perform the work "somewhere else" and just tell
>> Accumulo to bring files online. The only potential work Accumulo has to do
>> at that point is maintain the internal tree of files (merging and splitting
>> as the table is configured). Given that we have this massively popular tool
>> for performing distributed sorting (cough MapReduce cough), I don't agree
>> with your assertion.
>>
>> If you don't want to be burdened with sorting output during the ingest
>> task, use live ingest (BatchWriters). For reasonable data flows, live
>> ingest tends to be faster; however, bulk ingest provides the ability to
>> scale to much larger flows of data while not tanking Accumulo.
>>
>>
>> On 12/6/12 9:15 AM, Chris Burrell wrote:
>>
>> 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 <wilhelm.von.cloud@accumulo.net>wrote:
>>
>>> 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@0014efca-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    …             DataN
>>>>>
>>>>>
>>>>>
>>>>> The map parses the data, for each row, into a Map<String, String>.
>>>>> This will contain the following:
>>>>>
>>>>> Col1       Data1
>>>>>
>>>>> Col2       Data2
>>>>>
>>>>> Col3       Data3
>>>>>
>>>>> …
>>>>>
>>>>> ColN      DataN
>>>>>
>>>>>
>>>>>  An outputKey is then generated for this row in the format *
>>>>> client@timeStamp@randomUUID*
>>>>>
>>>>> Then for each entry in Map<String, String> 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<Text,Text,Key,Value>
>>>>> {
>>>>>
>>>>>          *public* *void* reduce(Text key, Iterable<Text> 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 = keyValue.toString().split("\\|"
>>>>> );
>>>>>
>>>>>
>>>>>
>>>>>                      // Generate key
>>>>>
>>>>>                      Key outputKey = *new* Key(key, *new* Text("foo"
>>>>> ), *new* Text(values[0]), *new* Text("myVis"));
>>>>>
>>>>>
>>>>>
>>>>>                      // Generate value
>>>>>
>>>>>                      Value outputValue = *new* Value(values[1].getBytes(),
>>>>> 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
>>>>>
>>>>> …
>>>>>
>>>>> client@20121206123059@0014efca-d8e8-492e-83cb-e5b6b7c49f7a foo:ColN [
>>>>> myVis] DataN
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> Thanks,
>>>>>
>>>>> Andrew
>>>>>
>>>>
>>>>
>>>
>>
>>
>

Mime
View raw message