[ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12832148#action_12832148 ]
Scott Carey commented on AVRO-392: ---------------------------------- {quote} Now the decision to do use Avro is local to f(). But to make it work with the overshoot-Binary decoder, it must somehow return a new InputStream. For the same reason, f2, which is not even aware of Avro, must return an InputStream. If the InputStream gets passed for n levels, all n levels should somehow return the new InputStream, which I think is a pain. {quote} Generally, its not a good design to pass anything around other than the 'top-most' stream since one layer might buffer. The semantics are the same as a BufferedInputStream, for example. What if we just made BinaryDecoder a BufferedInputStream? In the use case above, the user would pass that around and construct it external to the two methods, and the f() method can then choose to use the InputStream methods or the Decoder methods depending on some "use_avro" flag. This would require that Decoder become an interface instead of an Abstract Class however. Alternatively, a separate DirectBinaryDecoder class could be fine. But at this point It would make more sense to have a Factory class to deal with all the Decoder variations than proliferate classes and constructors. This leaves Decoder as an abstract class and removes inputStream() from it, and helps abstract out future changes to these classes -- a factory getDirectBinaryDecoder() might return a concrete instance that is DirectBinaryDecoder today, and some other class tomorrow, without causing an API change. {quote} Assume that one is willing to return InputStream all the way. This InputStream is actually InputStreamByteSource. The further users of this inputstream will go though additional function call overhead due to InputStreamByteSource for the rest of its lifetime. With the current implementation, it means single-byte IO.{quote} The performance impact will be small to zero, and often improved if the underlying stream was not buffered. The current implementation reads from the buffer until it is empty and then delegates. It could always buffer if the performance impact of the InputStream wrapper is of concern. It only does single-byte-IO if the underlying Source does single-byte-IO and the client is requesting single-byte-IO, otherwise it does array-range-IO. ---- Unless there are objections, I am going to work on a variation that centralizes construction of Decoders to an AvroFactory. This way, we can have a DirectBinaryDecoder implementation, a BufferedBinaryDecoder implementation, and more, without having to worry about breaking APIs later by switching implementations. This will be modeled after Jackson's JsonFactory -- An instance stores configuration parameters (such as buffer size, direct or buffered decoders, etc) and a limited set of methods create Decoder instances. The factory is thread safe and can be kept around for the life of the application. Multiple factory instances can exist with different configurations. I envision this same factory will likely be used for Encoders eventually, and we could consider it for other things as well. > 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.