You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2011/07/06 13:18:16 UTC

[jira] [Commented] (CASSANDRA-2820) Re-introduce FastByteArrayInputStream (and Output equivalent)

    [ https://issues.apache.org/jira/browse/CASSANDRA-2820?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13060484#comment-13060484 ] 

Sylvain Lebresne commented on CASSANDRA-2820:
---------------------------------------------

I'm actually not too much of a fan of extending ByteArray*Stream but redeclaring all the fields, giving "fake" argument to the super constructor. It's a bit ugly and not too optimal in memory size (the space argument is not a huge one I agree). I think we should extend and reuse the protected field of the super class. Those are part of the public API, so there is not so much liberty over what a future Harmony implementation (or any implementation for that matter) can do with them in the future and more importantly, if we override all the methods, we don't care (expect for the call to the super constructor actually, but it's easy enough to assert the validity of the field assignation after the call to super() if we really want to). And talking about that, we need to add an implementation of read(byte[]), flush() and write(byte[]) because we can't be sure that the implementation we extend will not override the ones inherited from InputStream/OutputStream.

Now I'll admit that I made my earlier comment a bit quickly and that in the end it may be simpler/cleaner to make FastByteArray*Stream extends Input/OutputStream directly and use
{noformat}
InputStream buffer = new FastByteArrayInputStream(bytes);
OutputStream buffer = new FastByteArrayOutputStream(bytes);
{noformat}
We cannot always do that for the OutputStream side because OutputStream doesn't have the toByteArray() method, but we can use FastByteArrayOutputStream then. Really my earlier comment was more about avoiding to use FastByteArray*Stream in more places than strictly necessary.


> Re-introduce FastByteArrayInputStream (and Output equivalent)
> -------------------------------------------------------------
>
>                 Key: CASSANDRA-2820
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2820
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>    Affects Versions: 0.8.0
>         Environment: n/a
>            Reporter: Paul Loy
>            Priority: Minor
>              Labels: bytearrayinputstream, bytearrayoutputstream, license, synchronized
>             Fix For: 1.0
>
>         Attachments: fast_bytearray_iostreams_harmony-patch-2.txt, fast_bytearray_iostreams_harmony-patch-3.txt
>
>
> In https://issues.apache.org/jira/browse/CASSANDRA-37 FastByteArrayInputStream and FastByteArrayOutputStream were removed due to being code copied from the JDK and then subsequently modified. The JDK license is incompatible with Apache 2 license so the code had to go.
> I have since had a look at the performance of the JDK ByteArrayInputStream and a FastByteArrayInputStream (i.e. one with synchronized methods made un-synchronized) and seen the difference is significant.
> After a warmup-period of >10000 loops I get the following for 10000 loops through a 128000 byte array:
> bais : 3513ms
> fbais: 72ms
> This varies depending on the OS, machine and Java version, but it's always in favour of the FastByteArrayInputStream as you might expect.
> Then, at Jonathan Ellis' suggestion, I tried this using a modified Apache Harmony ByteArrayInputStream - i.e. one whose license is compatible - and the results were the same. A significant boost.
> I will attach a patch with changes for the 0.8.0 tag.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira