Re: [Discuss][Java] 64-bit lengths for ValueVectors

2019-08-07 Thread Gonzalo Ortiz Jaureguizar
I would recommend to take care with this kind of changes.

I didn't try Arrow in more than one year, but by then the performance was
quite bad in comparison with plain byte buffer access
(see http://git.net/apache-arrow-development/msg02353.html *) and
there are several optimizations that the JVM (specifically, C2) does not
apply when dealing with int instead of longs. One of the
most commons is the loop unrolling and vectorization.

* It doesn't seem the best way to reference an old email on the list, but
it is the only result shown by Google

El mié., 7 ago. 2019 a las 11:42, Fan Liya ()
escribió:

> Hi Micah,
>
> Thanks for your effort. The performance result looks good.
>
> As you indicated, ArrowBuf will take additional 12 bytes (4 bytes for each
> of length, write index, and read index).
> Similar overheads also exist for vectors like BaseFixedWidthVector,
> BaseVariableWidthVector, etc.
>
> IMO, such overheads are small enough to justify the change.
> Let's check if there are other overheads.
>
> Best,
> Liya Fan
>
> On Wed, Aug 7, 2019 at 3:30 PM Micah Kornfield 
> wrote:
>
> > Hi Liya Fan,
> > Based on the Float8Benchmark there does not seem to be any meaningful
> > performance difference on my machine.  At least for me, the benchmarks
> are
> > not stable enough to say one is faster than the other (I've pasted
> results
> > below).  That being said my machine isn't necessarily the most reliable
> for
> > benchmarking.
> >
> > On an intuitive level, this makes sense to me,  for the most part it
> seems
> > like the change just moves casting from "int" to "long" further up the
> > stack  for  "PlatformDepdendent" operations.  If there are other
> benchmarks
> > that you think are worth running let me know.
> >
> > One downside performance wise I think for his change is it increases the
> > size of ArrowBuf objects, which I suppose could influence cache misses at
> > some level or increase the size of call-stacks, but this doesn't seem to
> > show up in the benchmark..
> >
> > Thanks,
> > Micah
> >
> > Sample benchmark numbers:
> >
> > [New Code]
> > BenchmarkMode  Cnt   Score   Error  Units
> > Float8Benchmarks.copyFromBenchmark   avgt5  15.441 ± 0.469  us/op
> > Float8Benchmarks.readWriteBenchmark  avgt5  14.057 ± 0.115  us/op
> >
> >
> > [Old code]
> > BenchmarkMode  Cnt   Score   Error  Units
> > Float8Benchmarks.copyFromBenchmark   avgt5  16.248 ± 1.409  us/op
> > Float8Benchmarks.readWriteBenchmark  avgt5  14.150 ± 0.084  us/op
> >
> > On Tue, Aug 6, 2019 at 1:18 AM Fan Liya  wrote:
> >
> >> Hi Micah,
> >>
> >> Thanks a lot for doing this.
> >>
> >> I am a little concerned about if there is any negative performance
> impact
> >> on the current 32-bit-length based applications.
> >> Can we do some performance comparison on our existing benchmarks?
> >>
> >> Best,
> >> Liya Fan
> >>
> >>
> >> On Tue, Aug 6, 2019 at 3:35 PM Micah Kornfield 
> >> wrote:
> >>
> >>> There have been some previous discussions on the mailing about
> supporting
> >>> 64-bit lengths for  Java ValueVectors (this is what the IPC
> specification
> >>> and C++ support).  I created a PR [1] that changes all APIs that I
> could
> >>> find that take an index to take an "long" instead of an "int" (and
> >>> similarly change "size/rowcount" APIs).
> >>>
> >>> It is a big change, so I think it is worth discussing if it is
> something
> >>> we
> >>> still want to move forward with.  It would be nice to come to a
> >>> conclusion
> >>> quickly, ideally in the next few days, to avoid a lot of merge
> conflicts.
> >>>
> >>> The reason I did this work now is the C++ implementation has added
> >>> support
> >>> for LargeList, LargeBinary and LargeString arrays and based on prior
> >>> discussions we need to have similar support in Java before our next
> >>> release. Support 64-bit indexes means we can have full compatibility
> and
> >>> make the most use of the types in Java.
> >>>
> >>> Look forward to hearing feedback.
> >>>
> >>> Thanks,
> >>> Micah
> >>>
> >>> [1] https://github.com/apache/arrow/pull/5020
> >>>
> >>
>


Re: A simple benchmark on Java implementation

2017-09-15 Thread Gonzalo Ortiz Jaureguizar
Yeah... I said "you can found it here" but forgot to add the link. My bad.
You can found it here <https://github.com/gortiz/arrow-jmh>.

2017-09-15 15:33 GMT+02:00 Wes McKinney <wesmck...@gmail.com>:

> hi Gonzalo,
>
> This is interesting, thank you. Do you have code available to reproduce
> these results?
>
> - Wes
>
> On Fri, Sep 15, 2017 at 9:28 AM, Gonzalo Ortiz Jaureguizar <
> golthir...@gmail.com> wrote:
>
> > I forgot to say that test were executed on my Ubuntu 17.04 laptop on
> > Oracle JDK 1.8.0_144-b01.
> >
> > 2017-09-15 13:21 GMT+02:00 Gonzalo Ortiz Jaureguizar <
> golthir...@gmail.com
> > >:
> >
> >> Hi there,
> >>
> >> I have created a little JMH test to check the Arrow performance. You can
> >> found it here. The idea is to test an API with implementations on heap
> >> arrays, nio buffers (that follow the arrow format) and Arrow. At this
> >> moment the API only supports nullable int buffers and contains read only
> >> methods.
> >>
> >> The benchmark run on automatically generated vectors of 2^10, 2^20 and
> >> 2^26 never-null integers and it tests three different access patterns:
> >>
> >>- Random access: Where a random element is read
> >>- Sequential access: Where a random index is chosen and then the
> >>following 32 elements are read
> >>- Sum access: Similar to sequential, but instead of simply read them,
> >>they are added into a long.
> >>
> >> Disclaimer: Microbenchmars are error prone and I'm not an expert on JMH
> >> and this benchmark has been done in a couple of hours.
> >>
> >> Results
> >> On all charts the Y axis is the relation between the throughput of the
> >> offheap versions with the heap version (so the higher the better).
> >>
> >> TD;LR: It seems that the complex structures of Arrow are preventing some
> >> optimizations on the JVM.
> >>
> >> Random
> >> The random access is quite good. The heap version is a little bit
> better,
> >> but both offheap solutions seems pretty similar.
> >>
> >> 1K  1M  64M
> >> Array   75.139  53.025  10.872
> >> Arrow   67.399  43.491  10.42
> >> Buf 82.877  38.092  10.753
> >> [image: Imágenes integradas 1]
> >>
> >> Sequential
> >> If you see the absolute values, it is clear that JMH's blackhole is
> >> preventing any JVM optimization on the loop. I think thats fine, as it
> >> simulates several calls to the vector on a *not omptimized* scenario.
> >> It seems that the JVM is not smart enough to optimize offheap sequential
> >> as much as it does with heap structures. Although both offheap
> >> implementations are worse than the heap version, the one that uses
> Arrow is
> >> sensible worse than the one that directly uses ByteBuffers:
> >> 1K  1M  64M
> >> Array   6.335   4.563   3.145
> >> Arrow   2.664   2.453   1.989
> >> Buf 4.456   3.971   3.018
> >> [image: Imágenes integradas 2]
> >>
> >> Sum
> >> The result is awful. It seems that the JVM is able to optimize (I guess
> >> vectorizing) the heap and ByteBuffer implementation (at least with small
> >> vectors), but not in the case with the Arrow version. I guess it is due
> to
> >> the indirections and deeper stack required to execute the same code on
> >> Arrow.
> >>
> >> 1K  1M  64M
> >> Array   44.833  26.617  9.787
> >> Arrow   3.426   3.265   2.521
> >> Buf 38.288  19.295  5.668
> >> [image: Imágenes integradas 4]
> >>
> >>
> >
>


Re: A simple benchmark on Java implementation

2017-09-15 Thread Gonzalo Ortiz Jaureguizar
I forgot to say that test were executed on my Ubuntu 17.04 laptop on Oracle
JDK 1.8.0_144-b01.

2017-09-15 13:21 GMT+02:00 Gonzalo Ortiz Jaureguizar <golthir...@gmail.com>:

> Hi there,
>
> I have created a little JMH test to check the Arrow performance. You can
> found it here. The idea is to test an API with implementations on heap
> arrays, nio buffers (that follow the arrow format) and Arrow. At this
> moment the API only supports nullable int buffers and contains read only
> methods.
>
> The benchmark run on automatically generated vectors of 2^10, 2^20 and
> 2^26 never-null integers and it tests three different access patterns:
>
>- Random access: Where a random element is read
>- Sequential access: Where a random index is chosen and then the
>following 32 elements are read
>- Sum access: Similar to sequential, but instead of simply read them,
>they are added into a long.
>
> Disclaimer: Microbenchmars are error prone and I'm not an expert on JMH
> and this benchmark has been done in a couple of hours.
>
> Results
> On all charts the Y axis is the relation between the throughput of the
> offheap versions with the heap version (so the higher the better).
>
> TD;LR: It seems that the complex structures of Arrow are preventing some
> optimizations on the JVM.
>
> Random
> The random access is quite good. The heap version is a little bit better,
> but both offheap solutions seems pretty similar.
>
> 1K  1M  64M
> Array   75.139  53.025  10.872
> Arrow   67.399  43.491  10.42
> Buf 82.877  38.092  10.753
> [image: Imágenes integradas 1]
>
> Sequential
> If you see the absolute values, it is clear that JMH's blackhole is
> preventing any JVM optimization on the loop. I think thats fine, as it
> simulates several calls to the vector on a *not omptimized* scenario.
> It seems that the JVM is not smart enough to optimize offheap sequential
> as much as it does with heap structures. Although both offheap
> implementations are worse than the heap version, the one that uses Arrow is
> sensible worse than the one that directly uses ByteBuffers:
> 1K  1M  64M
> Array   6.335   4.563   3.145
> Arrow   2.664   2.453   1.989
> Buf 4.456   3.971   3.018
> [image: Imágenes integradas 2]
>
> Sum
> The result is awful. It seems that the JVM is able to optimize (I guess
> vectorizing) the heap and ByteBuffer implementation (at least with small
> vectors), but not in the case with the Arrow version. I guess it is due to
> the indirections and deeper stack required to execute the same code on
> Arrow.
>
> 1K  1M  64M
> Array   44.833  26.617  9.787
> Arrow   3.426   3.265   2.521
> Buf 38.288  19.295  5.668
> [image: Imágenes integradas 4]
>
>


A simple benchmark on Java implementation

2017-09-15 Thread Gonzalo Ortiz Jaureguizar
Hi there,

I have created a little JMH test to check the Arrow performance. You can
found it here. The idea is to test an API with implementations on heap
arrays, nio buffers (that follow the arrow format) and Arrow. At this
moment the API only supports nullable int buffers and contains read only
methods.

The benchmark run on automatically generated vectors of 2^10, 2^20 and 2^26
never-null integers and it tests three different access patterns:

   - Random access: Where a random element is read
   - Sequential access: Where a random index is chosen and then the
   following 32 elements are read
   - Sum access: Similar to sequential, but instead of simply read them,
   they are added into a long.

Disclaimer: Microbenchmars are error prone and I'm not an expert on JMH and
this benchmark has been done in a couple of hours.

Results
On all charts the Y axis is the relation between the throughput of the
offheap versions with the heap version (so the higher the better).

TD;LR: It seems that the complex structures of Arrow are preventing some
optimizations on the JVM.

Random
The random access is quite good. The heap version is a little bit better,
but both offheap solutions seems pretty similar.

1K  1M  64M
Array   75.139  53.025  10.872
Arrow   67.399  43.491  10.42
Buf 82.877  38.092  10.753
[image: Imágenes integradas 1]

Sequential
If you see the absolute values, it is clear that JMH's blackhole is
preventing any JVM optimization on the loop. I think thats fine, as it
simulates several calls to the vector on a *not omptimized* scenario.
It seems that the JVM is not smart enough to optimize offheap sequential as
much as it does with heap structures. Although both offheap implementations
are worse than the heap version, the one that uses Arrow is sensible worse
than the one that directly uses ByteBuffers:
1K  1M  64M
Array   6.335   4.563   3.145
Arrow   2.664   2.453   1.989
Buf 4.456   3.971   3.018
[image: Imágenes integradas 2]

Sum
The result is awful. It seems that the JVM is able to optimize (I guess
vectorizing) the heap and ByteBuffer implementation (at least with small
vectors), but not in the case with the Arrow version. I guess it is due to
the indirections and deeper stack required to execute the same code on
Arrow.

1K  1M  64M
Array   44.833  26.617  9.787
Arrow   3.426   3.265   2.521
Buf 38.288  19.295  5.668
[image: Imágenes integradas 4]


Re: Apache Arrow at JupyterCon

2017-09-07 Thread Gonzalo Ortiz Jaureguizar
On a library like Arrow it also is very important to have the less dynamic
methods call on the critical paths (get/puts). If it is decided to supports
other memory systems, it is important to try to minimize that as much as
possible. If there is a single vector class that supports both systems (by
calling an interface, for example), the JVM will try to optimize the
dynamic calls by heuristic. If, on a JVM, only one implementation of both
is used (lets say a program only uses the Netty implementation), then the
impact should be negligible. Contrary if more than one is used, then there
is going to be problems.

Ideally we would like to have an abstract vector that doesn't know about
the memory buffer and then N implementations with specific methods to talk
with the buffer. And that should be repeated for each vector type. For
example, if there is a IntVector extended by NettyIntVector and a
MnemonicIntVector and NullableIntVector delegates on IntVector, there
should be a NettyNullableIntVector that delegates on NettyIntVector and the
same for a MnemonicNullableIntVector. This may sound cumbersome, but by
doing that, clients that really care about performance can use the specific
class on their code to be sure that methods calls are not dynamic.

2017-09-07 6:11 GMT+02:00 Jacques Nadeau <jacq...@apache.org>:

> This is a interesting problem but also pretty complex. Arrow's Java memory
> management model is complex on purpose (see
> https://github.com/apache/arrow/blob/master/java/memory/
> src/main/java/org/apache/arrow/memory/README.md
> for more info). It is designed to reserve and share memory in multiple
> hierarchical domains (with reservations and limits) while providing
> transfer semantics across those domains with minimal contention and
> locking. An opaque (and potentially easy starting point would be to
> optionally allow AllocationManager to use something other than the
> PooledByteBufAllocatorL and UnsafeDirectLittleEndian for memory allocation.
> This wouldn't expose movement between different memory tiers but that could
> be managed underneath the Arrow system. At the end of the day, the whole
> hierarchy is basically a collection of memory addresses, accounting and
> reference counting.
>
> A phase two could be a proposal which allows movement between memory
> domains and could be generified across systems like Mnemonic as well
> GPU/Device memory domains.
>
>
> On Wed, Sep 6, 2017 at 4:45 PM, Wes McKinney <wesmck...@gmail.com> wrote:
>
> > Thanks Gary, that is helpful context. In light if this, it might be
> > worth writing some kind of a proposal for how to enable the Java
> > vector classes to be backed by some other kind of byte buffers. It
> > might be that an alternative version of portions of the Arrow Java
> > library (i.e. decoupled from Netty) might need to be created.
> >
> > If it cannot be reconciled with the Netty AbstractByteBuf class then
> > this would be useful to know so that Arrow developers can plan
> > accordingly for the future.
> >
> > On Wed, Sep 6, 2017 at 2:15 PM, Gary Wong <qich...@gmail.com> wrote:
> > > The ArrowBuf is inherited from AbstractByteBuf, the AbstractByteBuf is
> > > defined in the Netty library, it does more like a memory pool not a
> pure
> > > buffer so that's why ArrowBuf is not backed by ByteBuffer as now.
> > >
> > > I have ever tried to make ArrowBuf build on top of DurableBuffer of
> > > Mnemonic, but looks it is not very easy to decouple the refcount from
> > other
> > > parts because the lifecycle of DurableBuffer could also be managed by
> > > JVM automatically instead of using refcount.
> > >
> > > I still want to figure out how gracefully to migrate the backend of
> > > ArrowBuf from Netty to Mnemonic. In addition, DurableBuffer could bring
> > > other benefits for Arrow e.g. persistent on any kind of memory service
> > that
> > > could make use of SSD, NVMe, Memory and NAS and more. in this way,
> Arrow
> > is
> > > able to break through the capacity limitation of system memory, avoid
> the
> > > SerDe for storage and link other durable objects with ease and etc.
> > >
> > >
> > >
> > >
> > > On Wed, Sep 6, 2017 at 10:40 AM, Wes McKinney <wesmck...@gmail.com>
> > wrote:
> > >
> > >> It should be possible to have an ArrowBuf backed by a
> > >> MappedByteBuffer. Anyone reading is welcome to dig in and write a
> > >> patch for this.
> > >>
> > >> Semantically this is what we have done in C++ -- a memory map inherits
> > >> from arrow::Buffer, so we can slice and dice a memory map as we would
&

Re: Apache Arrow at JupyterCon

2017-09-04 Thread Gonzalo Ortiz Jaureguizar
This is a very interesting feature. It's very surprising that there is no
ByteBuffer implementation backed on a MappedByteBuffer. As far as I
understand, it should be trivial to implement (maybe not to pool) as
usually ByteBuf is backed on a ByteBuffer and MappedByteBuffer extends
that. But I didn't find implementations when I goggled for it.

2017-09-03 16:12 GMT+02:00 Wes McKinney :

> I think ideally we would have a Java interface that would support all of:
>
> - Memory mapped files
> - Anonymous shared memory segments (e.g. POSIX shm)
> - NVM / Mnemonic
>
> We already have the ability to do zero-copy reads from buffer-like
> objects in C++ and IO interfaces that support zero copy (like memory
> mapped files). We can do zero-copy reads from ArrowBuf in Java but we
> are missing the interfaces to shared memory sources
>
> - Wes
>
> On Thu, Aug 31, 2017 at 5:09 PM, Gang(Gary) Wang  wrote:
> > Hi Wes,
> >
> > Thank you for the explanation. the usage of
> > https://issues.apache.org/jira/browse/ARROW-721 could be directly
> supported
> > by Mnemonic through DurableBuffer and DurableChunk, the DurableChunk
> makes
> > use of unsafe to expose a plain memory space for Arrow to use without
> > performance penalties. that's why most of the big data frameworks take
> the
> > advantage of unsafe, please refer to
> > https://mnemonic.apache.org/docs/domusecases.html for the use cases. we
> > could work on this ticket if you think that's exactly what you want.
> >
> > Regarding the NVM tech., that is what Mnemonic created for. it could be
> > used to directly persist Java generic objects and collection on NVM with
> no
> > SerDe. so what kind of basic tools you mentioned? probably,  we can help
> > also identify the gaps for Mnemonic as well. Thanks!
> >
> > Very truly yours,
> > Gary
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > On Thu, Aug 31, 2017 at 12:32 PM, Wes McKinney 
> wrote:
> >
> >> hi Gary,
> >>
> >> The Java libraries are not yet capable of writing or zero-copy reads
> >> of Arrow datasets to/from shared memory or memory-mapped files:
> >> https://issues.apache.org/jira/browse/ARROW-721. We've developed quite
> >> a bit of technology on the C++ side for dealing with shared memory IPC
> >> but we need someone to help with that on the Java side.
> >>
> >> In the context of NVM technologies, it would be nice to be able to
> >> persist a dataset to NVM and continue to do analytics on it, while
> >> retaining a "handle" so that the dataset can be easily recovered in
> >> the event of process failure. We may arrive at new use cases once some
> >> of the basic tools exist.
> >>
> >> - Wes
> >>
> >> On Wed, Aug 30, 2017 at 6:19 PM, Gang(Gary) Wang 
> wrote:
> >> > Thank you for sharing the videos. We are very interested in how to
> >> support
> >> > Arrow data format and collection very closely, could you please help
> to
> >> > point out which interfaces to allow Mnemonic act as a memory provider
> for
> >> > the user to store and access Arrow managed datasets ? Thanks!
> >> >
> >> > Very truly yours,
> >> > Gary.
> >> >
> >> >
> >> > On Wed, Aug 30, 2017 at 2:11 PM, Ivan Sadikov  >
> >> > wrote:
> >> >
> >> >> Great presentation! Thank you for sharing.
> >> >>
> >> >>
> >> >> On Thu, 31 Aug 2017 at 8:02 AM, Wes McKinney 
> >> wrote:
> >> >>
> >> >> > Absolutely. I will do that now
> >> >> >
> >> >> > On Wed, Aug 30, 2017 at 3:33 PM, Julian Hyde 
> >> wrote:
> >> >> > > Thanks for sharing. Can we tweet those videos as well? I see that
> >> >> > https://twitter.com/apachearrow 
> >> only
> >> >> > tweeted your slides.
> >> >> > >
> >> >> > >> On Aug 26, 2017, at 1:11 PM, Wes McKinney 
> >> >> wrote:
> >> >> > >>
> >> >> > >> hi all,
> >> >> > >>
> >> >> > >> In case folks here are interested, I gave a keynote this week at
> >> >> > >> JupyterCon explaining my motivations for being involved in
> Apache
> >> >> > >> Arrow and how I see it fitting in with the data science
> ecosystem
> >> long
> >> >> > >> term:
> >> >> > >>
> >> >> > >> https://www.youtube.com/watch?v=wdmf1msbtVs
> >> >> > >>
> >> >> > >> I also gave an interview going a little deeper into some of the
> >> topics
> >> >> > >> from the talk:
> >> >> > >>
> >> >> > >> https://www.youtube.com/watch?v=Q7y9l-L8yiU
> >> >> > >>
> >> >> > >> I believe we have an exciting journey ahead of us, but it's
> >> certainly
> >> >> > >> going to take a lot of collaboration and community development.
> >> >> > >>
> >> >> > >> - Wes
> >> >> > >
> >> >> >
> >> >>
> >>
>