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

Catalin Alexandru Zamfir commented on AVRO-1090:
------------------------------------------------

>From the source-code of DatumFileWriter, the "appendTo" method seems to do 
>what it's intended. But only accepts a (File) argument. In the case where one 
>writer connects over the network to Hadoop and needs to write a 
>"FSDataOutputStream" instead of a file, the advantages of the appendTo method 
>cannot be used. So it seems it is possible to retrieve the sync marker from an 
>existing .avro file and write forward with the same marker.

Can this be done here also? Can an appendTo (FSDataOutputStream) method be 
created? This would allow concurrent writers to create or append on the same 
output stream using the same marker, thus enabling the data to be read back.
                
> DataFileWriter should expose "sync marker" to allow concurrent writes to same 
> .avro file
> ----------------------------------------------------------------------------------------
>
>                 Key: AVRO-1090
>                 URL: https://issues.apache.org/jira/browse/AVRO-1090
>             Project: Avro
>          Issue Type: Bug
>    Affects Versions: 1.6.3
>            Reporter: Catalin Alexandru Zamfir
>
> We're writing to Hadoop via DataFileWriter (FSDataOutputStream). We're doing 
> this with two threads per node, on 8 nodes. Some of the nodes share the same 
> path. For example, our: TimestampedWriter class, takes a path argument and 
> appends the timestamp to it (ex: SomePath/2012/05/14). Thus, two threads or 
> two nodes can access the same path. The "race" condition when these streams 
> are written, is resolved with a check to see if the file exists (has been 
> created) by a faster thread. If that's so, it appends, instead of creating 
> the file on the HDFS.
> The problem is that DataFileWriter, generates a 16-byte, random string for 
> each instance. So, two threads with 2 different writer instances, have a 
> different sync marker. That means that data, when trying to read it back, 
> will get an IOException ("Invalid sync!").
> There's a big performance penalty here. Because only one writer can write at 
> once to one given path, it becomes a bottleneck. For 1B (billion) rows, it 
> took us 4 hours to generate & load. With 20 concurrent threads, it took only 
> 12.5 minutes. 
> If DataFileWriter would expose the "sync" marker, a developer could read that 
> and make sure that the next thread that appends to the file, uses the same 
> sync marker. Don't know if it's even possible to expose the sync marker so as 
> other instances of "DataFileWriter" can share the sync marker, from the file. 
> We have a fix for this, making sure each writer is an "unique" instance and 
> generating a path based on that uniqueness. But instead of having 
> "SomePath/2012/05/14/Shard.avro" we'd now have 
> "SomePath/2012/05/14/Shard-some-random-UUID.avro" for each of the writers 
> that write the data in.
> If it can be done, it would be a huge fix for a bottleneck problem. The 
> bottleneck being the single writer that can write to a single path.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to