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

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

We were doing the 12B tests, an uncovered a NPE (NullPointerException) at:
{code}
716754 [Main] INFO net.RnD.Hadoop.App - ## At: 0:11:54.332 # W: 21000000
Exception in thread "Main" java.lang.reflect.InvocationTargetException
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:616)
        at 
org.eclipse.jdt.internal.jarinjarloader.JarRsrcLoader.main(JarRsrcLoader.java:58)
Caused by: java.lang.NullPointerException
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.updateBlockInfo(DFSClient.java:1882)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.openInfo(DFSClient.java:1855)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.<init>(DFSClient.java:1831)
        at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:578)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:154)
        at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:427)
        at 
org.apache.avro.file.SeekableHadoopInput.<init>(SeekableHadoopInput.java:56)
        at 
net.RnD.FileUtils.BufferedTimestampedWriter.spawnConcurrentWriteThreads(BufferedTimestampedWriter.java:251)
        at 
net.RnD.FileUtils.BufferedTimestampedWriter.syncRecord(BufferedTimestampedWriter.java:159)
        at net.RnD.Hadoop.App.write1BAvros(App.java:270)
        at net.RnD.Hadoop.App.executeCode(App.java:535)
        at net.RnD.Hadoop.App.main(App.java:455)
        ... 5 more
{code}
                
> 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
>            Assignee: Doug Cutting
>             Fix For: 1.7.0
>
>         Attachments: AVRO-1090.patch, AVRO-1090.patch
>
>
> 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 HAS ALSO been requested on the avro-user thread: 
> http://grokbase.com/t/avro/user/122m4sjm1y/is-it-possible-to-append-to-an-already-existing-avro-file
> I just could not find the JIRA ticket for this request.

--
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