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

Reply via email to