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

Scott Carey commented on AVRO-392:
----------------------------------

I'm not sure it is a good idea to support both "buffered" and non-buffered 
binary decoders anymore.  

I spent a lot of time coming up with a factory scheme that works well, and then 
to have both implementations share what they could from an abstract parent. 

But no matter what, theier API's won't be the same, besides the read-ahead 
versus read-minimal semantics:

The 'direct' one cannot have an 'isEnd()' method.  Because the InputStream API 
is what it is, the only way to find out if one is at the end is to try and read 
-- and only the buffered implementation can do that without losing data.  This 
means the simple, abstract factory pattern won't work:  
{code}
 BinaryDecoder createBinaryDecoder(InputStream in, BinaryDecoder reuse);
{code}
where the above returns a direct or buffered implementation depending on the 
factory configuration and the concrete type of 'reuse'.  This pattern got rid 
of all the public "init" methods and is fairly clean, but if the BinaryDecoder 
supertype can't have an isEnd() method, it all breaks down.  Factory methods 
would have to be specific to a type, rather than generic, exposing the concrete 
types and defeating half of the purpose of the factory (unless we go with an 
interface per concrete class and make a more brittle API which I think is a bad 
idea at this stage).

So, there are a few options:

1. The buffered and direct implementations have separate instantiation 
semantics, and are essentially distinct.  This will solidify having two 
distinct implementations in the API.  Clients will access two specific _types_ 
-- one that is 'direct' and one that is 'buffered' than access one type that is 
configured to behave either way (but behind the scenes is currently two 
implementations).
2. Go back to my last patch or something close, and don't have a direct 
version. (we can add one back later if it turns out to be a big issue).
3. get rid of isEnd, and make any consumer that wants to write an iterator 
interface around a Decoder force an EOF exception to check for the end.

I don't like the third option because it maps very poorly to anything but 
InputStream data sources.  Also, the more I think about it, the less that 
having the two classes share too 
much of a common parent type for the Factory to expose as a primary type makes 
sense.  The semantics are rather different, and if we maintain both, the API of 
each type will likely diverge due to that, isEnd() isn't the only thing that 
buffering can facilitate.   There is some basic overlap however.

So, if we do maintain a 'direct' binary decoder, we will probably always have 
to maintain that, and expose it as a _distinct_ api from the buffered one.  
They won't be able to share identical APIs and simply have slightly different 
semantics.

Is that an acceptable cost to the project in order to deal with a few corner 
cases where the buffered version's '.inputStream()' method is not convinient? 
(I am not worried about the performance cost of the wrapper, that can be 
improved).


Thoughts on what the right approach is?




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

Reply via email to