You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org> on 2012/05/14 13:37:49 UTC

[jira] [Created] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Catalin Alexandru Zamfir created AVRO-1090:
----------------------------------------------

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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277867#comment-13277867 ] 

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

We're now catching the NullPointerException and retrying to "create" or "appendTo". This seems to fix it, but the NPE should require a check to see where it's coming from. It passed writing 125M per 6 threads on 6 nodes. This means over 1B records properly written via the patch to "appendTo". From our point of view, it passes all possible use-cases. Doug, please consider modifying SeekableFileInput to SeekableInput when you commit. Thanks again!
                
> 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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13283228#comment-13283228 ] 

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

Thanks! It's been working ever since.
                
> 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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277605#comment-13277605 ] 

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

Doug, patches are identical. Guess you uploaded the same patch. appendTo signature should be appendTo (SeekableInput, etc). in the second patch.
It's ok to commit with the "SeekableInput" change. In our tests it passed. Wrote 10M each thread, we were able to read the data 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
>            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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13274592#comment-13274592 ] 

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

Also, we're doing objRecordWriter.create (getHdfs ().append (objPath)) which should make a "DataFileWriter" on the FSDataOutputStream which respects the first sync marker of the written file. So: if thread #1 has already created the file, thread #2 can now "append" to the given path. But because it appends, it does not need to "generateSync" on the sync marker. Instead, it can read the sync marker from the already generated file and use it as it's own sync marker.

This does not happen. The fact that we get an "Invalid Sync" because of the fact we are creating multiple writers in different threads, even if one thread finishes first and create the given path, the next thread that should "append" to it does not seem to take in account the fact that it should first read the existing sync marker defined with the file. DataFileWriter should take in account that a file/path/stream written by it already contains a sync marker, and that there's no need to generate another one. It should get the existing sync marker and use that to append the data to the given HDFS path.

Thanks for your patience.
                
> 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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ 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

        

[jira] [Updated] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-1090:
-------------------------------

    Attachment: AVRO-1090.patch

Oops.  Yes, that should be SeekableInput not SeekableFileInput.

Here's a new version of the patch with that change.

I'll commit this soon unless someone objects.
                
> 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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ 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

        

[jira] [Updated] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Catalin Alexandru Zamfir updated AVRO-1090:
-------------------------------------------

    Description: 
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.

  was:
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.

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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13278157#comment-13278157 ] 

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

Confirmation: it works. Traversing means we're reading the data back. Tests have written about 700GB of data and are able to read it back.
{code}
22232086 [Main] INFO net.RnD.Hadoop.App - ## Traversed 1142M rows at: 2:35:59.897
22239836 [Main] INFO net.RnD.Hadoop.App - ## Traversed 1143M rows at: 2:36:07.647
22248107 [Main] INFO net.RnD.Hadoop.App - ## Traversed 1144M rows at: 2:36:15.918
{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

        

[jira] [Updated] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-1090:
-------------------------------

    Attachment: AVRO-1090.patch

Here's a patch that adds a method that should permit you to append to an HDFS file.  You'd call it with something like:

writer.appendTo(new FsInput(path, conf), fs.append(path));

Please let me know if this works for you.
                
> 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
>         Attachments: 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

        

[jira] [Updated] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-1090:
-------------------------------

    Resolution: Fixed
        Status: Resolved  (was: Patch Available)

I committed this.
                
> 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

        

[jira] [Commented] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Catalin Alexandru Zamfir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13276208#comment-13276208 ] 

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

Thanks Doug! Seems to work. We've written some tens of M (millions) of records, and read them back. Seems ok. The environment was: 6 nodes, 2 threads per node, writing 10M each thread. Reading was able to be done as soon as the first shard was written. It passed through almost all records that were written at the time.

I took your patch, promoted "SeekableFileInput" to "SeekableInput", because it's more generic. That way we can use SeekableHadoopInput from wibidata.github.com/odiago-avro and pass it to the "appendTo" patched method you've sent:

{code:title=Code.java|borderStyle=dashed}
// Append
bjRecordWriter.appendTo (new SeekableHadoopInput (App
.getObjConfiguration (), objKeyPath), App.getHdfs ()
.append (objKeyPath, objBufferSize));
{code}

I'll reset the tests, and go full-force for the 12B records we generated with our previous method and see the performance characteristics there. The environment is going to be 12 concurrent threads, via 6 nodes at 2 threads/node, writing for a few hours till it reaches the 1B records/thread mark. After that, we're going to read those records back, one at a time and count how many we've read.
                
> 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
>
>
> 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

        

[jira] [Updated] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-1090:
-------------------------------

    Status: Patch Available  (was: Open)
    
> 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

        

[jira] [Updated] (AVRO-1090) DataFileWriter should expose "sync marker" to allow concurrent writes to same .avro file

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-1090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-1090:
-------------------------------

    Fix Version/s: 1.7.0
         Assignee: Doug Cutting
    
> 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
>
>
> 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