You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Scott Carey (JIRA)" <ji...@apache.org> on 2010/02/03 11:49:28 UTC

[jira] Created: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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


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.


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835906#action_12835906 ] 

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

bq. Nobody seems to use the constructor BinaryDecoder(InputStream), which is marked depricated. I think we can get rid of it.

It is public, 'someone' might be using it.  If others think this is an OK API compatibility break, I'll remove it.  Its much cleaner to funnel everything through the factory.

bq. Scott, do you agree with Thiru's latest recommendations & mine? If so, will you be able to make a new patch today?

Yes, I can get to this in a couple hours.  The javadoc cleanup is useful (and with all the variations of this patch, a likely place where I missed something).

bq. BinaryDecoder#vin should be package-private, not protected, no? I don't think this should appear in end-user javadoc.
Ahh, javadoc.   Protected is more access restricted than package level, so from a code point of view preferred (no other classes in the package should see it).  
But its more open on the javadoc side that way, which we don't want.

I'll change that for now so its hidden.  If we ever design the file reader to be extended by users we can re-visit that.


> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.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.


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835915#action_12835915 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

> If others think this is an OK API compatibility break, I'll remove it.

I'm okay leaving this deprecated and removing it in the next release.

> Yes, I can get to this in a couple hours.

Great!

> Protected is more access restricted than package level [ ... ]

Protected can be accessed from a subclass in another package, while package-private cannot.  Packages can be sealed to prevent new classes from being added to them.


> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.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.


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Thiruvalluvan M. G. updated AVRO-392:
-------------------------------------

    Attachment:     (was: AVRO-392-with_DirectBinaryDecoder-2.patch)

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12833490#action_12833490 ] 

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

{code}
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.
{code}

That isn't quite enough.  For example, if we did that, then the file format would break if implementations switched, since it throws an exception if isEnd() doesn't return true at the end of a data block.
I strongly prefer APIs that aren't wish-washy.  So a client needs to know if isEnd is guaranteed to work, or if not.  We could do an isEnd() if we also did a supportsIsEnd(), so that a client could enforce the use of the feature.  This is acceptable, and jives with the configurable factory ("give me an instance that supports isEnd please" --> (buffered impl).  "give me one that supports isEnd and is also direct" --> sorry!

We could generalize this, and have one method on Decoder:
boolean isFeatureSupported(Feature f);
Where Feature is an enum in Decoder, and would currently have IS_END, INPUTSTREAM_PROXY, and NO_READAHEAD, or something like that.

Putting inputStream() on Decoder has similar issues, JsonDecoder for example can't always support that.

{quote}
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.
{quote}
The way i'll build this would be like Jackson's Factory API.  Configure the factory to make instances with certain configuration settings, and then there would be even fewer (smarter) factory methods. Otherwise, we'll find a few more configuration parameters, and end up with 5 booleans and two ints on each method signature.

bq. BinaryData class seems to know the internals of BinaryDecoder. Is there a way to avoid it?

Probably not, but i'll have another look.  They are intimately related classes.  

bq.There are significant code that is common between . . .

Yeah, my original version had a parent BinaryEncoder with the readahead and direct ones as children.  But I got frustrated with isEnd() -- The DataFileStream class would have to know for sure that it had an instance with a functioning isEnd().  If we do want to have both decoders and maintain them, this is an option to merge that shared code.  

However it could be worth it to wait.  Merging them implies standardizing on an API that has to have optionally supported parts, like isEnd() + supportsIsEnd(), forever.  I suspect that these optional parts will grow over time, and all client code will have to check for them, specify them at factory configuration, etc.  All that adds up to a non-trivial set of baggage to manage the decoders with different semantics.  I still think that it would be easier to add features to the Decoder/Encoder APIs for interleaving non-Avro data.

bq. Can we not make ByteArrayByteSource.compactAndFill(), by simply setting the minPos to pos, instead of creating a new array? Am I missing something?

It used to do an arraycopy() to the front of the array, but that mutated the data which made it impossible to for example, read the same array twice.  Since the purpose of compactAndFill() is for use by ensureBounds(), it has to make a new (usually very small) array to deal with the leftovers so that readLong() won't overflow bounds.  Well, I suppose we could try catching IndexOutOfBounds instead, or something else.  
Since this wasn't clear, I should add code comments or organize the code so that it is more clear what is going on there.

bq. I think ByteArrayByteSource.close() should set the eof flag and ba.setPos(ba.limit()) so that all subsequent reads will fail.

Good idea!

bq. The indentation isn't consistent. The most common problem is that the close brace is not indented properly (sometimes too much, sometimes too little), but there are other lines too.

I was worried that would happen :(  I used the 'ignore whitespace' option of svn diff.  This is a side effect -- less consistent whitespace for recipients of the patch.  The benefit is for those who wanted to make the patch easier to read by minimizing line changes that only affected whitespace, another solution to that is to use a diff viewing tool that makes it clear what changes are whitespace-only.
For a patch this big, I'm going to go for including whitespace and producing consistently indented final output for files where half or more lines are already changing.  I get it that one shouldn't change whitespace or formatting in a small patch, but this is an overhaul of a couple classes and the final result should be self-consistent.

{quote}
The fields position and max in ByteArrayByteSource are used in attach(). After that they are merely tracking the the corresponding fields in ba, with apparent reason. Since a single ByteSource gets attached only once, that too immediately after constuction, we get get rid of the attach() and move the logic to the constructors. This way, (1) we can avoid passing the unused bufferSize parameter to ByteArrayBufferSource, (2) get rid of the fields position and max in ByteArrayByteSource.
{quote}
Sounds like a good simplification.  Position and max were there for the detach() use case, the buffer accessor might be enough.  I'll look more deeply at this.



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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392-preview.patch

Partial patch -- no tests, does not pass tests without changes elsewhere.

This is intended to:

* Show the performance gains of this approach.
* Invite criticism on the API changes and approach to abstracting the decoding from the fetching of bytes.  

I expect that there will be minor changes in many places to accommodate these changes (as noted in AVRO-327).  Additionally, there are optimizations and cleanup in the Avro file reader code that can be done if this approach is agreed upon.

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392.patch

Nearly complete patch.  BinaryDecoder replaced with a buffering optimized one.  All tests pass.  Main changes are in BinaryDecoder, BinaryData, DataFileReader, DataFileStream, and Codecs.  Need to add more tests and decide on final changes to Decoder API.

This patch adds an isEnd() method to BinaryDecoder, that is used by the DataFileReader.  This changes the API of the Codec to depend on BinaryDecoder instead of Decoder.  I have not looked into how one would implement isEnd() for the JsonDecoder which would be required to move that to Decoder.  Do we expect to write JsonDecoder data to the file format?



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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829230#action_12829230 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

> FileChannel has position() and size().

Sure, then perhaps the API should accept FileChannel instead of File.  Applications need to be able to seek.  So either one must be able to construct a Decoder using a pre-positioned file handle, or one must be able to reposition the file handle that the decoder wraps.  In the current patch, neither is possible so far as I can tell.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Philip Zeyliger (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12830462#action_12830462 ] 

Philip Zeyliger commented on AVRO-392:
--------------------------------------

Took a quick pass here.  Excellent speed improvements.

I'm a bit concerned that we haven't tested all of the edges cases here.  Introducing buffering into the Decoder has certainly increased the code complexity.  We could run a code coverage tool to see if all the new code is touched by the existing TestBinaryDecoder code.  I didn't see (though I only had a chance to do a quick pass) tests that detect EOFExceptions on the tricky ensureBounds() codepath.  Likewise, I'd be more comfortable if the two variants of ByteSource had their own unit tests.

{noformat}
if (pos + len > limit) {                                                                
  throw new EOFException();                                                             
 }                  
{noformat]
Do we distinguish between malformed data (a varint that hasn't finished) and a genuine EOF?  I think we don't need to (that's the job of the level high up), but just want to make sure.

bq. static class BufferAccessor

Why is BufferAccessor static?  It seems that it attaches itself only ever to a BinaryDecoder object.  I don't know that it makes a difference; might be just a style preference.


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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392.patch

Addresses points brought up by Thiru and Doug. 

BinaryDecoder(InputStream in) 
remains deprecated, we expect to remove this in a later version.

> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.patch, AVRO-392.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.


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835776#action_12835776 ] 

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

+1 for the patch.

A couple of nits:

   - In ResolvingDecoder, we can use the simpler createBinaryDecoder(byte[], BinaryDecoder).
   - Nobody seems to use the constructor BinaryDecoder(InputStream), which is marked depricated. I think we can get rid of it.

> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.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.


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392.patch

This patch changes several things:

The Decoder API is now unchanged.  All new constructor types and variants are hidden behind AvroFactory.java.

I have removed the use of these old constructors and init methods from all tests and the rest of the code and replaced it with the factory.
We can decide to deprecate the old constructors and and the init(InputStream) method on Decoder and put that in another patch if we wish.  I had included that deprecation at first, but have several other things dependant on this patch and wish to get it committed soon and leave details like that for another ticket.

Unit tests in TestBinaryDecoder were significantly enhanced.  There is 92.5% instruction coverage on BinaryDecoder from this class (with trivial items left) as measured by EclEmma code coverage plugin in Eclipse.  This single unit test class by itself covers 11% of all of org.apache.avro.*.*  

This patch does not have a "DirectBinaryDecoder", but is build to easily bolt that on.  I have a separate patch with that in it, or we can commit this and do that addition in another ticket.   Adding it is as easy as adding a factory method, and svn cp'ing an old BinaryDecoder to a new DirectBinaryDecoder and then applying a patch to change its name and some signatures.

The Performance test has been modified to be more consistent and balance time between tests better. The first test was too fast because it had not compiled other variations of Decoder ant Test and was able to inline the decoder calls into the test -- once the polymorphism of the Decoder interface is seen by the JIT, it has to deoptimize some.  
Results on my laptop are:
{noformat}
ReadInt: 475 ms, 105.22084803794685 million entries/sec.  264.7735331687679 million bytes/sec
ReadLongSmall: 742 ms, 67.30978926651177 million entries/sec.  169.37566131867953 million bytes/sec
ReadLong: 964 ms, 51.819754312180855 million entries/sec.  226.4181253063843 million bytes/sec
ReadFloat: 408 ms, 122.33463415827654 million entries/sec.  489.34343001847253 million bytes/sec
ReadDouble: 412 ms, 121.10613499458655 million entries/sec.  968.8539242020922 million bytes/sec
ReadBoolean: 369 ms, 135.35646124067733 million entries/sec.  135.36187549912694 million bytes/sec
RepeaterTest: 1039 ms, 8.017210110940177 million entries/sec.  311.1342336699734 million bytes/sec
NestedRecordTest: 1766 ms, 28.312137923146135 million entries/sec.  71.243531384288 million bytes/sec
ResolverTest: 1193 ms, 6.9835422054685 million entries/sec.  271.0193474205395 million bytes/sec
MigrationTest: 3785 ms, 2.2012176566554804 million entries/sec.  85.4254983051707 million bytes/sec
GenericReaderTest: 3243 ms, 2.5692238249338963 million entries/sec.  99.70718926358578 million bytes/sec
GenericReaderTestWithDefaultTest: 6494 ms, 1.283021087397709 million entries/sec.  49.791857427457266 million bytes/sec
GenericReaderTestWithOutOfOrderTest: 3257 ms, 2.5579396506737884 million entries/sec.  99.26926973017773 million bytes/sec
GenericReaderTestWithPromotionTest: 3415 ms, 2.439907076813885 million entries/sec.  94.68862709916043 million bytes/sec
{noformat}

Results with the old decoder:
{noformat}
ReadInt: 2025 ms, 24.68210680539985 million entries/sec.  62.109066280835975 million bytes/sec
ReadLongSmall: 1519 ms, 32.91266558362055 million entries/sec.  82.8201151679994 million bytes/sec
ReadLong: 2455 ms, 20.360469903356993 million entries/sec.  88.96181556753385 million bytes/sec
ReadFloat: 1268 ms, 39.41542199333249 million entries/sec.  157.66326459020968 million bytes/sec
ReadDouble: 1266 ms, 39.472365002538076 million entries/sec.  315.7804989149047 million bytes/sec
ReadBoolean: 555 ms, 89.94391097711467 million entries/sec.  89.94750873355376 million bytes/sec
RepeaterTest: 2387 ms, 3.490798376127346 million entries/sec.  135.4719238517502 million bytes/sec
NestedRecordTest: 3346 ms, 14.942269049301121 million entries/sec.  37.600128144899365 million bytes/sec
ResolverTest: 2498 ms, 3.3354787991155557 million entries/sec.  129.4442363022417 million bytes/sec
MigrationTest: 5160 ms, 1.6148558208740609 million entries/sec.  62.66979677001705 million bytes/sec
{noformat} 
(Generic tests are harder to replace the decoder in).

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392-preview.patch

This patch includes the prototype BinaryDecoder replacement:  BufferedBinaryDecoder.

Perf.java can be configured to run tests against this by changing the Decoder initialized in the newDecoder() method, line 164 of Perf.java after the patch.

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Doug Cutting updated AVRO-392:
------------------------------

      Resolution: Fixed
    Hadoop Flags: [Reviewed]
          Status: Resolved  (was: Patch Available)

I just committed this.  Thanks, Scott!

> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.patch, AVRO-392.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.


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829562#action_12829562 ] 

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

Perf.java results with the new BinaryDecoder:

{noformat}
ReadInt: 183 ms, 163.80731891100893 million entries/sec
ReadLongSmall: 374 ms, 80.17681660622226 million entries/sec
ReadLong: 621 ms, 48.24873186249755 million entries/sec
ReadFloat: 277 ms, 108.25010103342763 million entries/sec
ReadDouble: 301 ms, 99.60291637339141 million entries/sec
ReadBoolean: 222 ms, 134.65897614291805 million entries/sec
RepeaterTest: 3334 ms, 8.997873802420488 million entries/sec
NestedRecordTest: 1164 ms, 25.764828947300618 million entries/sec
ResolverTest: 4106 ms, 7.306147806779666 million entries/sec
{noformat}

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835109#action_12835109 ] 

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

I'm nearly done with the updated patch.  As for supportsIsEnd() this is what it looks like with and without it for the unit tests:


without supportsIsEnd() the unit tests have blocks like this:

{code}
      try {
        Assert.assertTrue(bd.isEnd());
      } catch (UnsupportedOperationException e) {
        if (bd.getClass() != DirectBinaryDecoder.class) {
          throw e;
        }
      }
{code}

with supportsIsEnd() it is:

{code}
if (supportsIsEnd()) {
  Assert.assertTrue(bd.isEnd());
}
{code}

The latter will break only if the implementations tested are themselves broken.  The former depends on the tests having knowledge of which types support what, and will break if the test is not kept in sync with that.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12833344#action_12833344 ] 

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

BinaryData class seems to know the internals of BinaryDecoder. Is there a way to avoid it?

There are significant code that is common between BinaryEncoder and DirectBinaryEncoder, like skipBytes, skipString, readBytes, readString, all array and map functions etc. I think refactroring them into a common abstract base class will be good. I suppose the performance impact will be near zero if a program uses only one of the implementations (Direct or Buffered).

Can we not make ByteArrayByteSource.compactAndFill(), by simply setting the minPos to pos, instead of creating a new array? Am I missing something?

The fields _position_ and _max_ in ByteArrayByteSource are used in attach(). After that they are merely tracking the the corresponding fields in _ba_, with apparent reason. Since a single ByteSource gets attached only once, that too immediately after constuction, we get get rid of the attach() and move the logic to the constructors. This way, (1) we can avoid passing the unused bufferSize parameter to ByteArrayBufferSource, (2) get rid of the fields _position_ and _max_ in ByteArrayByteSource.

I think ByteArrayByteSource.close() should set the eof flag and ba.setPos(ba.limit()) so that all subsequent reads will fail.

The indentation isn't consistent. The most common problem is that the close brace is not indented properly (sometimes too much, sometimes too little), but there are other lines too.

[nit] In ByteSource.skip "if (remaining > n)" could be "if (remaining >= n)".
[nit] There is a useless return statement in ByteArrayByteSource.attach()

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835889#action_12835889 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

I'd like to get this committed today and roll a new 1.3 release candidate.

A few other nits I've noticed:
 - many javadoc comments still mention AvroFactory.  These should instead link to DecoderFactory now.  Using links helps check class names.
 - the class javadoc for BinaryDecoder should link to DecoderFactory.  it should also be broken into multiple paragraphs.  and, when buffering is mentioned, it should link to the DecoderFactory method that creates a non-buffering BinaryDecoder.
 - the class javadoc for DirectBinaryDecoder should just be something like, "a non-buffering version of BinaryDecoder".
 - BinaryDecoder#vin should be package-private, not protected, no?  I don't think this should appear in end-user javadoc.

Scott, do you agree with Thiru's latest recommendations & mine?  If so, will you be able to make a new patch today?

> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.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.


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392.patch

Cleaned up patch.  Incorporates all of Doug's concerns.  Found a few more placed where I lost the Javadoc in BinaryDecoder and corrected those.

I have left the FileChannelByteSource in BinaryDecoder, but it is not used or exposed yet.  Remove it if you wish.

I have additionally expanded TestBinaryDecoder to cover both types of BinaryDecoder sources.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829215#action_12829215 ] 

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

bq. It's hard to tell whether even this is required without a more detailed patch.

Oh, I see the confusion.  The prototype version of a new BinaryDecoder is the BufferedBinaryDecoder that I failed to include in the patch.   

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12830353#action_12830353 ] 

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

I am unsure about splitting BinaryDecoder up into two.  Should the old version implement all of the same constructors/initializers?  Or only maintain the InputStream one?   One option is to make a "DirectBinaryDecoder" class that has the same capabilities as the old one, and supports only InputStream construction.   This implementation is not buffered if given a byte[], buffering is optional and depends on the ByteSource, so BufferedBinaryDecoder might not be the right name.  This would mean that clients that cannot allow any read-ahead can switch to that, but all others will by default get the new one.
A second class also raises some important testing questions.  Do we need to duplicate a lot of tests to run against both?  How many clients actually need a guarantee that it does not read more bytes than absolutely necessary? 
Can anyone come up with a use case where inputStream() combined with remaining() to detect how far ahead it read and the means to retrieve those bytes, is not sufficient?

Splitting it up with an abstract parent may hurt performance a bit, enough implementations of a method eventually trigger the JIT to de-optimize and prevent method inlining.  For most things this doesn't matter, but for something in a tight loop like ReadInt() it might.

Another problem is that there will likely be features going forward that will depend on the buffering, or only be needed by the buffering, which will further branch the two implementations.   I would also prefer not to maintain two versions of every readType() method if possible.
What use cases are there where inputStream() is not sufficient?  Can we add to the new BinaryDecoder to address those rather than keep another variation? 

*Issues*
{quote}
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.
{quote}
Good catch.  Fixed to add the result of tryReadRaw() to reamaining, and check that total for 0 for the -1 return.

{quote}
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?
{quote}
That is what tryReadRaw() is for.  It should always return how many bytes are actually read, up to MIN(maximum available, requested).  A client that must know how much data is copied should use tryReadRaw and treat a 0 return as EOF.  I tried removing readRaw() and only having tryReadRaw(), but that hurt performance since the clients had to have more conditionals in 'common path' code -- that is most of the time is not EOF, so checking for it on every read hurts tight loops.  This is essentially what is wrong with InputStream -- checks for EOF happen on both the client and internal side of the read() methods all the time even though the common case is not EOF.

{quote}
Is there a need to have mark() with empty body in ByteSource. InputStream already has an identical implementation.
{quote}
Removed.

{quote}
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().
{quote}
That was the first implementation, but ByteBufferInputStream and InflaterInputStream both do not adhere to the InputStream contract and throw EOFException instead of returning -1 in some cases.  If you remove the catch, tests do not pass (in fact, its worse, the SocketServer test spins in an infinite loop forever).  For example:
{noformat}
java.io.EOFException
	at org.apache.avro.ipc.ByteBufferInputStream.getBuffer(ByteBufferInputStream.java:84)
	at org.apache.avro.ipc.ByteBufferInputStream.read(ByteBufferInputStream.java:46)
	at org.apache.avro.io.BinaryDecoder$InputStreamByteSource.tryReadRaw(BinaryDecoder.java:683)
{noformat}
Because the declared contract of skip() and read() can throw IOException, I have to be defensive and catch the EOF subtype EOFExeption since the InputStream javadoc contract is not enforced by the compiler, and trySkipBytes and tryReadRaw both must never throw EOFException.

{quote}
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.
{quote}
This started out as an optimization that was worth about 10% in decoding performance when _pos_ was incremented with each read.  But, by removing the false dependency between the check and the read it doesn't matter as much as in the past.  Placing the most common case first and the unusual case (EOF) second tends to help performance.  For smaller buffer sizes, or less aggressive compilers this helps.
The check and the load are not dependent.  Although the code looks at first glance that one part is 'first' and one part is 'second', the compiler and CPU are free to reorder these and speculatively execute since one operation does not depend on the other.  Placing the conditional second is a sort of compiler 'hint' that we want this to be a speculative load -- load the bytes and deal with the consequences of the conditional simultaneously if possible; avoid waiting on the conditional before initiating the loads and do both concurrently if possible.

{quote}
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?
{quote}

Good catch!  I enhanced a test in TestBinaryDecoder to detect this bug.  This test reliably breaks without minPos and limit set properly.  It found an additional bug one line below as well:  limit did not account for minPos.

{quote}
# 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.
{quote}
I'll clean those up.

{quote}
# 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.
{quote}
Yeah, its a matter of taste.  I prefer separating complex statements into multiple lines since it is easier to debug when stepping through code line by line and watching the variables change state.  Additionally, if the code changes, and the statements are less condensed, the diff better represents the change.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12836290#action_12836290 ] 

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

A local  setting leaked into the trunk. Fixed it.

{code}
Index: lang/java/src/test/java/org/apache/avro/TestDataFile.java
===================================================================
--- lang/java/src/test/java/org/apache/avro/TestDataFile.java   (revision 912267
)
+++ lang/java/src/test/java/org/apache/avro/TestDataFile.java   (working copy)
@@ -69,7 +69,7 @@
   private static final File DIR
     = new File(System.getProperty("test.dir", "/tmp"));
   private static final File DATAFILE_DIR
-    = new File(System.getProperty("test.dir", "/Users/scottc/Documents/workspac
e/avro2/build/interop/data"));
+    = new File(System.getProperty("test.dir", "/tmp"));
   private static final long SEED = System.currentTimeMillis();
   private static final String SCHEMA_JSON =
     "{\"type\": \"record\", \"name\": \"Test\", \"fields\": ["
{code}

Committed revision 912268.

> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.patch, AVRO-392.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.


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829140#action_12829140 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

This patch doesn't include the new implementations of #readInt(), etc., just some high-level API changes.

Instead of Decoder#init(File), perhaps we should have Decoder#init(ReadableByteChannel)?  It's hard to tell whether even this is required without a more detailed patch.

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Thiruvalluvan M. G. updated AVRO-392:
-------------------------------------

    Attachment: AVRO-392-with_DirectBinaryDecoder-2.patch

This is modification over Scott's latest patch. The primary motivation for this modification was to share the tests for BinaryDecoder and DirectBinaryDecoder. I thought it'd be easier to do it than to describe. 

The other changes are:
   - Fixed the indentation for BinaryDecoder.java
   - Made DirectBinaryDecoder package private. Consequently its init() is no longer available outside so, it is no longer depricated.
   - Added a couple of more factory methods for ease of use with DirectBinaryDecoder. Since DirectBinaryDecoder does not add anything to the interface, I have returned and accepted Decoder in the factory methods.
   - Replaced a couple of uses of BinaryDecoder with Decoder as BinaryDecoder functionality wasn't needed there

Scott, I hope I've not stomped over any of your local changes. If you like, please consider accepting a Decoder in the factory methods for BinaryDecoder the way I've done for DirectBinaryDecoder. This will allow users like Requestor work with just Decoder. Thanks.


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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829274#action_12829274 ] 

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

I'd like to focus on the core of the BufferedBinaryDecoder change and the ByteSource concept within first.  The other changes are mostly a consequence of that or use cases in the file writer and reader.
If we get that right, then the API details around the edges will be easy to adjust. 

The current patch has an initialization from a File because for simplicity I wanted to align it with the Jackson API as a starting point.  If it was a FileChannel, it would be difficult to have the JsonDecoder operate on it.

As far as construction and initialization go, we have many options to consider.  I am delaying decisions on this until it is more clear what is needed to allow BufferedBinaryDecoder to replace BinaryDecoder and for two decoders to share one input/buffer.

Before the number of constructors/initializes publicly available grows too far, a factory might be a useful abstraction to use instead for decoders and encoders.  Especially as more variations come up, some that will share concepts and some that will not.  



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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12833239#action_12833239 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

Some concerns with the current patch:
 - the javadoc for DirectBinaryDecoder should be updated to describe that it is a non-buffering variant.
 - AvroFactory is not a good name.  If the intent is to eventually support encoders to, then might better be named CodecFactory, were not the term codec already used for compression codecs.  We could perhaps rename those Compressors or somesuch.  Or we could just name this DecoderFactory, since that's all it does at present.
 - Is there a reason we can't add #isEnd() to Decoder?  That would make it easier to write Decoder-agnostic programs.
 - BinaryDecoder might become an abstract class, like a tag interface, with concrete subclasses BufferedBinaryDecoder and DirectBinaryDecoder.  That would enable applications to accept a BinaryDecoder that's either buffered or not.


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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392.patch

This patch incorporates a few changes:

DirectBinaryDecoder now extends BinaryDecoder. 

BinaryDecoder has isEnd() and inputStream() , but Decoder does not.  It is now trivial to pull either of these to Decoder if desired -- with the default implementation there throwing an exception.  isEnd is more broadly applicable to resolving decoders, etc, but will frequently be unusable since JsonDecoder and DirectBinaryDecoder can't support it.

Factory methods return BinaryDecoder instances, and based on the factory configuration, return different implementations.
The factory in this form is easy to use as a parameterized input for a unit tests or otherwise.  



> 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-2.patch, AVRO-392-with_DirectBinaryDecoder.patch, AVRO-392.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.


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Thiruvalluvan M. G. updated AVRO-392:
-------------------------------------

    Attachment: AVRO-392-with_DirectBinaryDecoder-2.patch

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829891#action_12829891 ] 

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

bq.  If ByteSource needs to be shared with codecs, files, decoders, etc. then it should probably be a public, documented class. But we don't need to go there yet, do we?

Nope!  But getting a sense of the future direction that code will take always helps minimize thrashing later.

bq. It would be great to have a public Decoder#isEnd() method. We could then clean up loops to call this instead of catching EOFException.

Ok, lets plan on doing that later.  I'm not sure how do do it on the JSON side right now, and the patch I've got now is looking good and cleaned up.

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment:     (was: AVRO-392-preview.patch)

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829201#action_12829201 ] 

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

I started with ReadableByteChannel, but it does not have the ability to skip bytes naturally (one has to read them to skip them).  It has close(), isOpen() and read().   FileChannel has position() and size(). For a file source, skipping long distances by reading instead of seeking would be bad.
A ByteSource implementation for a generic ReadableByteChannel could be added, but I suspect specific implementations that take advantage of each channel's capability will be preferred.

The performance of the decoding -- though highly sensitive to the details of the readX() and ensureBounds() methods -- is not very sensitive to the ByteSource method implementation details.  The JIT inlines most of the methods, but leaves the ByteSource a virtual call.  Because these happen infrequently this is faster -- the method body for the common path is smaller.

There are changes to readInt() and other read methods included.  All have improved performance with some tweaks.  ReadInt's fastest form I could find is:

{code}
@Override
  public int readInt() throws IOException {
    ensureBounds(5); // won't throw index out of bounds
    int len = 1;
    int b = buf[pos] & 0xff;
    int n = b & 0x7f;
    if (b > 0x7f) {
      b = buf[pos + len++] & 0xff;
      n ^= (b & 0x7f) << 7;
      if (b > 0x7f) {
        b = buf[pos + len++] & 0xff;
        n ^= (b & 0x7f) << 14;
        if (b > 0x7f) {
          b = buf[pos + len++] & 0xff;
          n ^= (b & 0x7f) << 21;
          if (b > 0x7f) {
            b = buf[pos + len++] & 0xff;
            n ^= (b & 0x7f) << 28;
          }
        }
      }
    }
    if (pos + len > limit) {
      throw new EOFException();
    }
    pos += len;
    return (n >>> 1) ^ -(n & 1); // back to two's-complement
  }
{code}

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829857#action_12829857 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

> What about the isEnd() method on BinaryDecoder? Should this be moved to Decoder?

It would be great to have a public Decoder#isEnd() method.  We could then clean up loops to call this instead of catching EOFException.

If ByteSource needs to be shared with codecs, files, decoders, etc.  then it should probably be a public, documented class.  But we don't need to go there yet, do we?

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12834899#action_12834899 ] 

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

bq. Instead of having a state for the factory, can we have a boolean argument to the factory method that takes an InputStream? If we have the state, we need to have another method returns the state etc.

I'm concerned about the future flexibility of a boolean flag.  It is essentially exposing the implementation detail that the current BinaryDecoder is not 'direct' for InputStreams, but is for the other stuff.

If the mapping of which factory methods return different instances for 'direct' and non-direct changes for some reason, it affects the API.
For example, maybe in the future, we add support for a factory method that operates on an Iterator<ByteBuffer>.  The first implementation might 'read-ahead' past the minimum ByteBuffer necessary and thus not be applicable when 'direct' is used.
But, a later implementation might fix this and ensure that the iterator never advances past the minimum necessary position and become applicable for 'direct' needs.  With a state flag on the factory, this all remains hidden.  With a flag on the factory method, it is exposed.

Maybe the flexibility is a bit of overkill for this one use case, but if we end up with more and more configuration state flags over time, the factory methods don't have to change.  If we push all that to the method signatures, it is harder to evolve.
We currently have one state variable in the factory -- the buffer size.  It is only applicable to the InputStream factory at the moment.  
The factory could be static if we put that on the factory method, but that would have the same drawbacks -- bufferSize may be applicable to some implementations, and not others, and which implementations are returned from each factory may change too.


bq. But instead of adding supported() methods, can we have these methods throw OperationNotSupportedException(). If the clients encounter, for example isEndSupported() == false, they have to somehow manage the situation, They can now handle the exception to achieve the same. at least it makes the interface simpler. What do you think?

I think that implementations that don't support these operations should definitely throw OperationNotSupportedException (or equivalent).  But a client should be able to test for support proactively.  
I suppose that can be done with a try / catch instead of an if(), but I guess I belong to the "exceptions are for exceptional situations" school.  Its not exceptional for a user to want to know if a method is going to work, so I feel that 

{code}
if (!isEndSupported()) {
  logger.error("Decoder must support isEnd());
  throw ... // or return, or whatever the client needs to do
}
{code}
is cleaner than
{code}
try {
  isEnd();
} catch (OperationNotSupportedException e) {
  logger.error("Decoder must support isEnd());
  throw e;  // or return, or whatever the client needs to do
}
{code}

Also, there may be cases where a client could have
{code}
if (isEndSupported()) {
  // do one thing
} else {
  // do something else
}
{code}
Which is much uglier (and significantly slower) in exception form.

But that is definitely a style thing.  I can go either way here.

We agree on the core of all this so I'll start working on that right now.  What is left will be a decision on whether we should remove the supportsXX on Decoder and whether to put the 'direct' part of the factory as a method flag or a state variable.  Some input/opinions from some others would help resolve that quickly.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12832166#action_12832166 ] 

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

bq. Do we distinguish between malformed data (a varint that hasn't finished) and a genuine EOF? I think we don't need to (that's the job of the level high up), but just want to make sure.

We might want to introduce something like a 'BinaryFormatException' to deal with the "this can't be an encoded int" -like cases. but we can do that in another ticket.

bq. Why is BufferAccessor static? It seems that it attaches itself only ever to a BinaryDecoder object. I don't know that it makes a difference; might be just a style preference.
2 things.  
* Style -- Since a non-static private class has a hidden private member 'this' that points to the parent object, it has more overhead.  So by default I start with static and remove it if needed.
* Flexibility -- I wasn't sure if I'd want to share a source or 'reattach' one.  For example, two decoders could share one source.  Once it was coded to work as static, I left it that way.  With the current use cases, it doesn't matter much.

{quote}I'm a bit concerned that we haven't tested all of the edges cases here. Introducing buffering into the Decoder has certainly increased the code complexity. We could run a code coverage tool to see if all the new code is touched by the existing TestBinaryDecoder code. I didn't see (though I only had a chance to do a quick pass) tests that detect EOFExceptions on the tricky ensureBounds() codepath. Likewise, I'd be more comfortable if the two variants of ByteSource had their own unit tests.{quote}

I've used EclEmma in eclipse to generate code coverage reports.  The main parts missing (counting coverage from TestBinaryDecoder and TestDataFile) are:
* Invalid Int/Long encoding.
* Most skipXX methods.
* BufferAccessor when detached.
* .inputStream() read and skip methods.

I'll add some tests to cover these.  Several of the above are covered in other tests (for example, the skipXXX methods).


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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829818#action_12829818 ] 

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

Thanks for the feedback Doug.

One at a time:

bq. please don't make whitespace-only changes, reorder imports or make other extraneous changes that just make the patch harder to read.

I wish there was a way to back out of it, but I experimented with a few dozen variations on almost every bit of BinaryDecoder, so its almost impossible to track down the white space changes without a lot of trial and error.  I can fix the import changes.   DataFileReader is also heavily modified.  The other classes I can clean up.

{quote}
# the new JsonDecoder constuctors have bad javadoc.
# the BinaryDecoder javadoc has been lost
{quote}

I'll clean up the javadoc.  Are there any places that could be better documented or explained?

bq. why is DataFileStream#vin now protected? can't we keep this package protected?

Not sure.  I'll change it back.

bq. does ByteSource need to be public? I'd prefer it were not.

Good catch.

{quote}
# finally, you've added new Decoder constructors and init() methods for File and byte[], but, as far as I can see, these are not yet used. can we separate this from the rest of this patch to be considered in a separate issue? the different ByteSource implementations could be removed for now too. if i'm not mistaken, you're getting the above speedups with InputStream-based methods only. i'd prefer if this issue could focus on the minimal API changes needed to get these performance improvements. if further performance improvements can be had with API changes, let's do that separately.
{quote}

The byte[] initializer and ByteSource is used for optimizations on file reading with NullCodec, and for BinaryData$Decoders.set().  The File initializer and FileChannelByteSource is more of a prototype.   I can move out for now since it is intended for a future optimization for the DataFileReader, it might be best as a FileChannel anyway.    I did need to prototype the other ByteSource variants to make sure the concept was flexible enough for future evolution.  I'll move all these things to another patch for a different issue.


What about the isEnd() method on BinaryDecoder?  Should this be moved to Decoder?  Its public, but it could be package protected if we move the classes in org.apache.avro.file into o.a.a.io.  This would also allow ByteSource to be shared between file types, codecs, and decoders.  I suspect there will be more things we want to share between classes in o.a.a.file and o.a.a.io since these are so tightly related.


A heads-up:  the same approach will help BinaryEncoder, but we need a good set of tests first.  That side of things should be easier because Encoder already has buffering semantics.


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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12834761#action_12834761 ] 

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

@Scott
bq. This isn't necessary, there is no point in using a DirectBinaryDecoder on a set of bytes, BinaryDecoder doesn't buffer/copy/alter the bytes, so there is no reason to be 'direct'.

Good point. Please get rid of it.

bq. There would be a configureDirect(true) method on the factory, and the factory in this state would return a direct instance only when needed (when initialized on an InputStream for now). But this requires that the return type be a shared superclass. So inputStream() and isEnd() have to either go on Decoder() or DirectBinaryDecoder has to extend whatever type has those two methods.

Instead of having a state for the factory, can we have a boolean argument to the factory method that takes an InputStream? If we have the state, we need to have another method returns the state etc.

I like your approach of having only the decoder visible outside the package and having the isEnd() and inputStream() in it. But instead of adding supported() methods, can we have these methods throw OperationNotSupportedException(). If the clients encounter, for example isEndSupported() == false, they have to somehow manage the situation, They can now handle the exception to achieve the same. at least it makes the interface simpler. What do you think?

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Philip Zeyliger (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12830453#action_12830453 ] 

Philip Zeyliger commented on AVRO-392:
--------------------------------------

{quote}
doug> please don't make whitespace-only changes, reorder imports or make other extraneous changes that just make the patch harder to read.

scott> I wish there was a way to back out of it, but I experimented with a few dozen variations on almost every bit of BinaryDecoder, so its almost impossible to track down the white space changes without a lot of trial and error. I can fix the import changes. DataFileReader is also heavily modified. The other classes I can clean up.
{quote}

Both git and svn diffs seem to have an ignore whitespace option.  To be honest, I haven't tried them, but they might be worth a shot.
{noformat}
  -x [--extensions] ARG    : Default: '-u'. When Subversion is invoking an
                             external diff program, ARG is simply passed along
                             to the program. But when Subversion is using its
                             default internal diff implementation, or when
                             Subversion is displaying blame annotations, ARG
                             could be any of the following:
                                -u (--unified):
                                   Output 3 lines of unified context.
                                -b (--ignore-space-change):
                                   Ignore changes in the amount of white space.
                                -w (--ignore-all-space):
                                   Ignore all white space.
                                --ignore-eol-style:
                                   Ignore changes in EOL style.
{noformat}

-- Philip

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12833283#action_12833283 ] 

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

bq. AvroFactory is not a good name. If the intent is to eventually support encoders to, then might better be named CodecFactory, were not the term codec already used for compression codecs. We could perhaps rename those Compressors or somesuch. Or we could just name this DecoderFactory, since that's all it does at present.

I think it should work with encoders too.  Configuration might overlap, and one central place to deal with these is probably better than two.
The compression codec name collision is an issue, but that is in the file spec.  There already is a class named CodecFactory.  Maybe renaming them should go in a different ticket?  Also, CodecFactory is not the best name either, since Codec means to do both Decoding and Encoding, and these do one or the other. Hmm.  Maybe two is better just so the naming is clean.  Opinions?

bq. Is there a reason we can't add #isEnd() to Decoder? That would make it easier to write Decoder-agnostic programs.
A DirectBinaryDecoder could never implement it othter than by always returning the same value or throwing an exception.  It must actually attempt a read in order to figure out if it is at the end, which means it might 'overshoot' its input, the whole point of its existence is not to do that.  
If we want isEnd() to be universally useable without side-effects for all decoders (or binary decoders), we can't have DirectBinaryDecoder.  It will be significantly easier to maintain a nice common interface if the semantics don't differ too much.   What semantics for read-ahead and 'isEnd' from Jacson on the Json side?

bq. BinaryDecoder might become an abstract class, like a tag interface, with concrete subclasses BufferedBinaryDecoder and DirectBinaryDecoder. That would enable applications to accept a BinaryDecoder that's either buffered or not.
That is how I started out.  The factory also created the abstract BinaryDecoder type and the implementations were hidden, and the 'best' type given the factory configuration was returned (even if 'direct' is requested, the buffered one doesn't buffer when accessing a byte array so it is preferred).  But the DataFileReader needed isEnd(), and couldn't operate on this type, which broke that factory paradigm.  In short, the parent type became no more useful than Decoder. 
I thought about it more, and realized that the semantic differences between one implementation that _must_ never read ahead and _must_ work with InputStreams and another that may read ahead and can work with a variety of data sources is going to continue to cause very different client API needs.

A user who has the need to use the 'direct' form and also for whatever reason can't use the .inputStream() proxy is going to have peculiar special needs that I don't think belong on a supertype.
A user who has simpler needs and will primarily be streaming data from various sources where all data is Avro will have their own unique needs.

So I'm guessing that isEnd(), is only the beginning -- of things we can do easily on one but not on the other that are useful for some and not for others.

I think that the first use case is extremely rare.

bq. the javadoc for DirectBinaryDecoder should be updated to describe that it is a non-buffering variant.
If we decide to keep that class, I'll clean it up.


So in summary:
* Change the factory name to DecoderFactory unless someone comes up with a good name for a factory that is both a DecoderFactory and an EncoderFactory
* Decide if we really want a DirectBinaryDecoder and how much of an API it shares with a read-ahead one.  Clean up javadoc for DirectBinaryDecoder if it is kept.
* Consider a new ticket to rename file block Codec's to something much more distinct from Decoder and Encoder. 

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835115#action_12835115 ] 

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

bq. The fields position and max in ByteArrayByteSource are used in attach(). After that they are merely tracking the the corresponding fields in ba, with apparent reason. Since a single ByteSource gets attached only once, that too immediately after constuction, we get get rid of the attach() and move the logic to the constructors. This way, (1) we can avoid passing the unused bufferSize parameter to ByteArrayBufferSource, (2) get rid of the fields position and max in ByteArrayByteSource.

That is unsafe because the old source must be detached before the new one is attached.  
If we attach in the source constructor, we would have to detach the old before the source is constructed.   That is a little opaque.
This would prevent us from validating the source during/after construction before detaching the old one.

The ByteSource API separates the construction step from the attachment so that it is more flexible and can deal with a wider variety of possible future input types. 
The consequence is that the buffer size gets passed to all source types, regardless of whether that type buffers or not., and that simple sources have to separate the attach from the construction.


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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392.patch

New patch incorporating changes from Thiru's review.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12834565#action_12834565 ] 

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

Ok, I've got some time to make another patch for tomorrow.   

@Thiru -- 
For the factory, I think we should consider simplification:
{code}
  /**
   * Equivalent to createDirectBinaryDecoder(
   *    new ByteArrayInputStream(bytes, start, length), reuse)
   */
  public Decoder createDirectBinaryDecoder(byte[] bytes,
      int start, int length, Decoder reuse) {
    return createDirectBinaryDecoder(
        new ByteArrayInputStream(bytes, start, length), reuse);
  }
{code}
This isn't necessary, there is no point in using a DirectBinaryDecoder on a set of bytes, BinaryDecoder doesn't buffer/copy/alter the bytes, so there is no reason to be 'direct'.  

My earlier Idea for the factory would be to not even have a createDirectBinaryDecoder() method at all.  There would be a configureDirect(true) method on the factory, and the factory in this state
would return a direct instance only when needed (when initialized on an InputStream for now).  But this requires that the return type be a shared superclass.  So inputStream() and isEnd() have to either
go on Decoder() or DirectBinaryDecoder has to extend whatever type has those two methods.

I think I'll just propose putting inputStream() and isEnd() in Decder.  
This is the simplest thing.  However, in order to allow those methods to be strict, the contract will be that

boolean supportsIsEnd()
and
boolean supportsInputStream()

be also added, and that if these return true, their corresponding methods have strict behavior -- IsEnd() never lies, and inputStream() always returns a valid object that can read bytes interleaved with  the decoder.
If false, these methods throw IllegalStateException.

Does that approach sound like a good compromise?  If so I'll get started tomorrow and produce a patch with the above design.

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Status: Patch Available  (was: Open)

AVRO-392.patch  "2010-02-05 02:02 AM" 

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829804#action_12829804 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

This is fast!  Some comments
 - please don't make whitespace-only changes, reorder imports or make other extraneous changes that just make the patch harder to read.
 - the new JsonDecoder constuctors have bad javadoc.
 - the BinaryDecoder javadoc has been lost
 - why is DataFileStream#vin now protected?  can't we keep this package protected?
 - does ByteSource need to be public?  I'd prefer it were not.
 - finally, you've added new Decoder constructors and init() methods for File and byte[], but, as far as I can see, these are not yet used.  can we separate this from the rest of this patch to be considered in a separate issue? the different ByteSource implementations could be removed for now too.  if i'm not mistaken, you're getting the above speedups with InputStream-based methods only.  i'd prefer if this issue could focus on the minimal API changes needed to get these performance improvements.  if further performance improvements can be had with API changes, let's do that separately.

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


[jira] Updated: (AVRO-392) Binary Decoder Performance and flexibility overhaul

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

Scott Carey updated AVRO-392:
-----------------------------

    Attachment: AVRO-392-with_DirectBinaryDecoder.patch

A version of the patch with a DirectBinaryDecoder.java class.

One would want to 
{noformat}
svn cp lang/java/src/java/org/apache/avro/io/BinaryDecoder.java lang/java/src/java/org/apache/avro/io/DirectBinaryDecoder.java
rm lang/java/src/java/org/apache/avro/io/DirectBinaryDecoder.java
patch -p0 AVRO-392-with_DirectBinaryDecoder.patch
{noformat}

In order to make sure that svn knows that DirectBinaryDecoder is a branch of the original BinaryDecoder.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12835007#action_12835007 ] 

Doug Cutting commented on AVRO-392:
-----------------------------------

I personally am not a fan of isXSupported() methods and would rather see an exception used here.

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


[jira] Commented: (AVRO-392) Binary Decoder Performance and flexibility overhaul

Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12830449#action_12830449 ] 

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

bq. Can anyone come up with a use case where inputStream() combined with remaining() to detect how far ahead it read and the means to retrieve those bytes, is not sufficient?

{code}
private GenericDatumReader<X> reader;
int f(InputStream in) {
    if (some_condition) {
         BinaryDecoder d = new BinaryDecoder(in);
         X x = reder.read(null, d);
         return f2(x);
    } else {
        // Do something else with in.
        return 100;
    }
}

void f2(InputStream in) {
    if (some_condition) {
       f(in);
    }
    // dome something more with in.
}
{code}

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.

If the original InputStream is a field in a class, it cannot be final.

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.

If Avro is a small part of project, it would pollute the design for small performance improvement for them. With the old BinaryDecoder, they have a choice - clean interface or performance.

Agreed InputStream is bad for performance and people should not use it, if they can. But it is the most common binary input interface in Java and is not going to go away anytime soon.

Bq. One option is to make a "DirectBinaryDecoder" class that has the same capabilities as the old one, and supports only InputStream construction.
Name doesn't matter to me. You can trivially support init(byte[], int, int) with a ByteArrayInputStream. We could move out the inputStream() method out of Decoder.

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