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 Fri, 05 Feb 2010 16:17:27 GMT

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

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

The performance improvement is impressive. Great!

I was looking at the new BinaryDecoder. I'm sorry I am coming up with my comments so late.
I hope they are still useful.

As Doug suggested earlier, can we call this BufferedBinaryReader and encourage everyone to
use it? If someone needs to have the no-overshoot feature and does not want to (or cannot,
for whatever reason) use our inputStream() method, they can use the (original) BinaryDecoder.
It should be trivial to support the new functions of Decoder in the original BinaryDecoder.
If we are starting from scratch today, I wouldn't ask for it. But since we already have it,
why throw it away? If agree on this proposal, we can refactor the common portions of BinaryDecoder
and BufferedBinaryDecoder into an abstract base class.

*Spec/correctness issues*
   - I think ByteSource.read(byte[], int, int) does not conform to the InputStream spec under
a specific condition. If the buffer has some bytes less than the number of desired bytes it
copies that data into the destination buffer. Then it calls tryReadRaw(). If tryReadRaw()
returns 0, it returns -1. It should return the number of bytes copied so far.
   - If the underlying stream has fewer than desired length, readRaw would throw EOFException.
Wouldn't be useful to somehow let the client know the actual number of bytes read into the
buffer passed?
   - Is there a need to have mark() with empty body in ByteSource. InputStream already has
an identical implementation.
   - InputStream.skipSourceBytes() catches EOFException and does something. The only method
called in that function is InputStream.skip(). InputStream.skip() newer throws EOFException.
It indicates EOF thorough a return value of 0. The similar observation applies to InputStream.read()
within InputStreamByteSource.tryReadRaw().
   - In BinaryDecoder.readFloat() and readDouble(), we check if we have enough number of bytes
in the buffer after converting bytes into int's. It will be better if we check before converting.
This logic is sound for readInt() and readLong() since we don't know the number of bytes needed.
But for readFloat() and readDouble(), we know the number of bytes needed, so we check early.
   - I think minpos is not used correctly in BinaryDecoder.ensureBounds(). The remaining bytes
in the buffer are moved to location starting at minPos, but pos is set to 0. It should be
set to minPos, right?

*Nit*
   - Indentation for continuation lines is the same as the first line in a small number of
places. E.g the definition of the method ByteSource.skipSourceBytes().
   - Some documentation around ByteSource and BufferAccessor classes will be useful.
   - In InputStreamByteSource.readRaw(), the variable _read_ is only used within the while
loop. So, the declaration can move inside the while loop.
   - There are many occasions like _if (c) return x; esle return y;_. I feel it'll be more
readable if we have _return c ? x : y_. Of course, it is a matter of taste.
   - There are many occassions with code _pos += x; setPos(pos); return xx_. Will it be better
if we write _setPos(pos + x); return xx;_? Again, it's a matter of taste.



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