avro-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Thiruvalluvan M. G. (JIRA)" <j...@apache.org>
Subject [jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul
Date Sat, 13 Feb 2010 05:01:28 GMT

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

Thiruvalluvan M. G. commented on AVRO-392:
------------------------------------------

+1 to name the factory DecoderFactory.

I support adding isEnd() to Decoder. DirectBinaryDecoder would always return _false_. If someone
does not want buffering, he cannot rely on isEnd(). isEnd() will be called by "high-level"
functions and not by functions that work with a single Avro object corresponding to a schema.
If such a high-level function needs to mix Avro and non-avro data in the stream, it should
know when avro data ends, and so it would not need to call isEnd(). The high-level functions
that rely on isEnd() to recognize the end of avro data cannot mix avro data and non-avro data.

We can also add inputStream() to decoder. It would simply return the underlying InputStream()
for DirectBinaryDecoder.

+1 to keep the DirectBinaryDecoder, as you'd have guessed. :) One option is that we can add
a boolean to the factory methods that specifies if one wants buffering or not. Another option
is to have additional factory methods. I prefer the former, but okay with the other, too.



> Binary Decoder Performance and flexibility overhaul
> ---------------------------------------------------
>
>                 Key: AVRO-392
>                 URL: https://issues.apache.org/jira/browse/AVRO-392
>             Project: Avro
>          Issue Type: Improvement
>          Components: java
>            Reporter: Scott Carey
>            Assignee: Scott Carey
>            Priority: Critical
>             Fix For: 1.3.0
>
>         Attachments: AVRO-392-preview.patch, AVRO-392-with_DirectBinaryDecoder.patch,
AVRO-392.patch, AVRO-392.patch, AVRO-392.patch, AVRO-392.patch
>
>
> BinaryDecoder has room for significant performance improvement.  [AVRO-327|https://issues.apache.org/jira/browse/AVRO-327]
has some preliminary work here, but in order to satisfy some use cases there is much more
work to do.
> I am opening a new ticket because the scope of the changes needed to do this the right
way are larger.
> I have done a large bulk of a new implementation that abstracts a 'ByteSource' from the
BinaryDecoder.  Currently BinaryDecoder is tightly coupled to InputStream.  The ByteSource
can wrap an InputStream, FileChannel, or byte[] in this version, but could be extended to
support other channel types, sockets, etc.  This abstraction allows the BinaryDecoder to buffer
data from various sources while supporting interleaved access to the underlying data and greater
flexibility going forward.
> The performance of this abstraction has been heavily tuned so that maximum performance
can be achieved even for slower ByteSource implementations.
> For readers that must interleave reads on a stream with the decoder, this includes a
> {code}
> public InputStream inputStream();
> {code}
> method on the decoder that can serve interleaved reads.  
> Additionally it will be necessary to have a constructor on BinaryDecoder that allows
two BinaryDecoders to share a stream (and buffer).
> Performance results on this new version is better than previous prototypes:
> *current trunk BinaryDecoder*
> {noformat}
> ReadInt: 983 ms, 30.497877855999185 million entries/sec
> ReadLongSmall: 1058 ms, 28.336666040111496 million entries/sec
> ReadLong: 1518 ms, 19.75179889508437 million entries/sec
> ReadFloat: 657 ms, 45.61031157924184 million entries/sec
> ReadDouble: 761 ms, 39.387756709704355 million entries/sec
> ReadBoolean: 331 ms, 90.4268145647456 million entries/sec
> RepeaterTest: 7718 ms, 3.886725782038378 million entries/sec
> NestedRecordTest: 1884 ms, 15.91964611687992 million entries/sec
> ResolverTest: 8296 ms, 3.616055866616717 million entries/sec
> MigrationTest: 21216 ms, 1.4139999570144013 million entries/sec
> {noformat}
> *buffering BinaryDecoder*
> {noformat}
> ReadInt: 187 ms, 160.22131904871262 million entries/sec
> ReadLongSmall: 372 ms, 80.4863521975457 million entries/sec
> ReadLong: 613 ms, 48.882385721129246 million entries/sec
> ReadFloat: 253 ms, 118.16606270679061 million entries/sec
> ReadDouble: 275 ms, 108.94314257389068 million entries/sec
> ReadBoolean: 222 ms, 134.85327963176064 million entries/sec
> RepeaterTest: 3335 ms, 8.993007936329503 million entries/sec
> NestedRecordTest: 1152 ms, 26.0256943004597 million entries/sec
> ResolverTest: 4213 ms, 7.120659335077578 million entries/sec
> MigrationTest: 15310 ms, 1.9594884898992941 million entries/sec
> {noformat}
> Performance is 2x to 5x the throughput of trunk on most tests.  

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message