You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Paul Weiss <pa...@gmail.com> on 2015/08/29 21:17:00 UTC

Tungsten off heap memory access for C++ libraries

Hi,

Would the benefits of project tungsten be available for access by non-JVM
programs directly into the off-heap memory?  Spark using dataframes w/ the
tungsten improvements will definitely help analytics within the JVM world
but accessing outside 3rd party c++ libraries is a challenge especially
when trying to do it with a zero copy.

Ideally the off heap memory would be accessible to a non JVM program and be
invoked in process using JNI per each partition.  The alternatives to this
involve additional costs of starting another process if using pipes as well
as the additional copy all the data.

In addition to read only non-JVM access in process would there be a way to
share the dataframe that is in memory out of process and across spark
contexts.  This way an expensive complicated initial build up of a
dataframe would not have to be replicated as well not having to pay the
penalty of the startup costs on failure.

thanks,

-paul

Re: Tungsten off heap memory access for C++ libraries

Posted by "jpivarski@gmail.com" <jp...@gmail.com>.
jpivarski@gmail.com wrote
> P.S. Concerning Java/C++ bindings, there are many. I tried JNI, JNA,
> BridJ, and JavaCPP personally, but in the end picked JNA because of its
> (comparatively) large user base. If Spark will be using Djinni, that could
> be a symmetry-breaking consideration and I'll start using it for
> consistency, maybe even interoperability.

I think I misunderstood what Djinni is. JNA, BridJ, and JavaCPP provide
access to untyped bytes (except for common cases like java.lang.String), but
it looks like Djinni goes further and provides a type mapping--- exactly the
"serialization format" or "layout of bytes" that I was asking about.

Is it safe to say that when Spark has off-heap caching, that it will be in
the format specified by Djinni? If I work to integrate ROOT with Djinni,
will this be a major step toward integrating it with Spark 2.0?

Even if the above answers my first question, I'd still like to know if the
new Spark API will allow RDDs to be /filled/ from the C++ side, as a data
source, rather than a derived dataset.




--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/Tungsten-off-heap-memory-access-for-C-libraries-tp13898p17388.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
For additional commands, e-mail: dev-help@spark.apache.org


Re: Tungsten off heap memory access for C++ libraries

Posted by "jpivarski@gmail.com" <jp...@gmail.com>.
Hi,

I'm coming from the particle physics community and I'm also very interested
in the development of this project. We have a huge C++ codebase and would
like to start using the higher-level abstractions of Spark in our data
analyses. To this end, I've been developing code that copies data from our
C++ framework, ROOT, into Scala:

https://github.com/diana-hep/rootconverter/tree/master/scaroot-reader
<https://github.com/diana-hep/rootconverter/tree/master/scaroot-reader>  

(Worth noting: the ROOT file format is too complex for a complete rewrite in
Java or Scala to be feasible. ROOT readers in Java and even Javascript
exist, but they only handle simple cases.)

I have a variety of options for how to lay out the bytes during this
transfer, and in all cases fill the constructor arguments of Scala classes
using macros. When I learned that you're moving the Spark data off-heap (at
the same time as I'm struggling to move it on-heap), I realized that you
must have chosen a serialization format for that data, and I should be using
/that/ serialization format.

Even though it's early, do you have any designs for that serialization
format? Have you picked a standard one? Most of the options, such as Avro,
don't make a lot of sense because they pack integers to minimize number of
bytes, rather than lay them out for efficient access (including any
byte-alignment considerations).

Also, are there any plans for an API that /fills/ an RDD or DataSet from the
C++ side, as I'm trying to do?

Thanks,
-- Jim


P.S. Concerning Java/C++ bindings, there are many. I tried JNI, JNA, BridJ,
and JavaCPP personally, but in the end picked JNA because of its
(comparatively) large user base. If Spark will be using Djinni, that could
be a symmetry-breaking consideration and I'll start using it for
consistency, maybe even interoperability.




--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/Tungsten-off-heap-memory-access-for-C-libraries-tp13898p17387.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
For additional commands, e-mail: dev-help@spark.apache.org


Re: Tungsten off heap memory access for C++ libraries

Posted by Paul Wais <pa...@gmail.com>.
Update for those who are still interested: djinni is a nice tool for
generating Java/C++ bindings.  Before today djinni's Java support was only
aimed at Android, but now djinni works with (at least) Debian, Ubuntu, and
CentOS.

djinni will help you run C++ code in-process with the caveat that djinni
only supports deep-copies of on-JVM-heap data (and no special off-heap
features yet).  However, you can in theory use Unsafe to get pointers to
off-heap memory and pass those (as ints) to native code.  

So if you need a solution *today*,  try checking out a small demo:
https://github.com/dropbox/djinni/tree/master/example/localhost

For the long deets, see:
 https://github.com/dropbox/djinni/pull/140



--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/Tungsten-off-heap-memory-access-for-C-libraries-tp13898p14427.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
For additional commands, e-mail: dev-help@spark.apache.org


Re: Tungsten off heap memory access for C++ libraries

Posted by Paul Weiss <pa...@gmail.com>.
https://issues.apache.org/jira/browse/SPARK-10399

Is the jira to track.
On Sep 1, 2015 5:32 PM, "Paul Wais" <pa...@gmail.com> wrote:

> Paul: I've worked on running C++ code on Spark at scale before (via JNA,
> ~200
> cores) and am working on something more contribution-oriented now (via
> JNI).
> A few comments:
>  * If you need something *today*, try JNA.  It can be slow (e.g. a short
> native function in a tight loop) but works if you have an existing C
> library.
>  * If you want true zero-copy nested data structures (with explicit
> schema),
> you probably want to look at Google Flatbuffers or Captain Proto.  Protobuf
> does copies; not sure about Avro.  However, if instances of your nested
> messages fit completely in CPU cache, there might not be much benefit to
> zero-copy.
>  * Tungsten numeric arrays and UTF-8 strings should be portable but likely
> need some special handling.  (A major benefit of Protobuf, Avro,
> Flatbuffers, Capnp, etc., is these libraries already handle endianness and
> UTF8 for C++).
>  * NB: Don't try to dive into messing with (standard) Java String <->
> std::string using JNI.  It's a very messy problem :)
>
> Was there indeed a JIRA started to track this issue?  Can't find it at the
> moment ...
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/Tungsten-off-heap-memory-access-for-C-libraries-tp13898p13929.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
> For additional commands, e-mail: dev-help@spark.apache.org
>
>

Re: Tungsten off heap memory access for C++ libraries

Posted by Paul Wais <pa...@gmail.com>.
Paul: I've worked on running C++ code on Spark at scale before (via JNA, ~200
cores) and am working on something more contribution-oriented now (via JNI). 
A few comments:
 * If you need something *today*, try JNA.  It can be slow (e.g. a short
native function in a tight loop) but works if you have an existing C
library.
 * If you want true zero-copy nested data structures (with explicit schema),
you probably want to look at Google Flatbuffers or Captain Proto.  Protobuf
does copies; not sure about Avro.  However, if instances of your nested
messages fit completely in CPU cache, there might not be much benefit to
zero-copy.
 * Tungsten numeric arrays and UTF-8 strings should be portable but likely
need some special handling.  (A major benefit of Protobuf, Avro,
Flatbuffers, Capnp, etc., is these libraries already handle endianness and
UTF8 for C++).  
 * NB: Don't try to dive into messing with (standard) Java String <->
std::string using JNI.  It's a very messy problem :)

Was there indeed a JIRA started to track this issue?  Can't find it at the
moment ...



--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/Tungsten-off-heap-memory-access-for-C-libraries-tp13898p13929.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
For additional commands, e-mail: dev-help@spark.apache.org


Re: Tungsten off heap memory access for C++ libraries

Posted by Reynold Xin <rx...@databricks.com>.
Please do. Thanks.

On Mon, Aug 31, 2015 at 5:00 AM, Paul Weiss <pa...@gmail.com> wrote:

> Sounds good, want me to create a jira and link it to SPARK-9697? Will put
> down some ideas to start.
> On Aug 31, 2015 4:14 AM, "Reynold Xin" <rx...@databricks.com> wrote:
>
>> BTW if you are interested in this, we could definitely get some help in
>> terms of prototyping the feasibility, i.e. how we can have a native (e.g.
>> C++) API for data access shipped with Spark. There are a lot of questions
>> (e.g. build, portability) that need to be answered.
>>
>> On Mon, Aug 31, 2015 at 1:12 AM, Reynold Xin <rx...@databricks.com> wrote:
>>
>>>
>>> On Sun, Aug 30, 2015 at 5:58 AM, Paul Weiss <pa...@gmail.com>
>>> wrote:
>>>
>>>>
>>>> Also, is this work being done on a branch I could look into further and
>>>> try out?
>>>>
>>>>
>>> We don't have a branch yet -- because there is no code nor design for
>>> this yet. As I said, it is one of the motivations behind Tungsten, but it
>>> is fairly early and we don't have anything yet. When we start doing it, I
>>> will shoot the dev list an email.
>>>
>>>
>>>
>>

Re: Tungsten off heap memory access for C++ libraries

Posted by Paul Weiss <pa...@gmail.com>.
Sounds good, want me to create a jira and link it to SPARK-9697? Will put
down some ideas to start.
On Aug 31, 2015 4:14 AM, "Reynold Xin" <rx...@databricks.com> wrote:

> BTW if you are interested in this, we could definitely get some help in
> terms of prototyping the feasibility, i.e. how we can have a native (e.g.
> C++) API for data access shipped with Spark. There are a lot of questions
> (e.g. build, portability) that need to be answered.
>
> On Mon, Aug 31, 2015 at 1:12 AM, Reynold Xin <rx...@databricks.com> wrote:
>
>>
>> On Sun, Aug 30, 2015 at 5:58 AM, Paul Weiss <pa...@gmail.com>
>> wrote:
>>
>>>
>>> Also, is this work being done on a branch I could look into further and
>>> try out?
>>>
>>>
>> We don't have a branch yet -- because there is no code nor design for
>> this yet. As I said, it is one of the motivations behind Tungsten, but it
>> is fairly early and we don't have anything yet. When we start doing it, I
>> will shoot the dev list an email.
>>
>>
>>
>

Re: Tungsten off heap memory access for C++ libraries

Posted by Reynold Xin <rx...@databricks.com>.
BTW if you are interested in this, we could definitely get some help in
terms of prototyping the feasibility, i.e. how we can have a native (e.g.
C++) API for data access shipped with Spark. There are a lot of questions
(e.g. build, portability) that need to be answered.

On Mon, Aug 31, 2015 at 1:12 AM, Reynold Xin <rx...@databricks.com> wrote:

>
> On Sun, Aug 30, 2015 at 5:58 AM, Paul Weiss <pa...@gmail.com>
> wrote:
>
>>
>> Also, is this work being done on a branch I could look into further and
>> try out?
>>
>>
> We don't have a branch yet -- because there is no code nor design for this
> yet. As I said, it is one of the motivations behind Tungsten, but it is
> fairly early and we don't have anything yet. When we start doing it, I will
> shoot the dev list an email.
>
>
>

Re: Tungsten off heap memory access for C++ libraries

Posted by Reynold Xin <rx...@databricks.com>.
On Sun, Aug 30, 2015 at 5:58 AM, Paul Weiss <pa...@gmail.com> wrote:

>
> Also, is this work being done on a branch I could look into further and
> try out?
>
>
We don't have a branch yet -- because there is no code nor design for this
yet. As I said, it is one of the motivations behind Tungsten, but it is
fairly early and we don't have anything yet. When we start doing it, I will
shoot the dev list an email.

Re: Tungsten off heap memory access for C++ libraries

Posted by Paul Weiss <pa...@gmail.com>.
Reynold,

That is great to hear.  Definitely interested in how 2. is being
implemented and how it will be exposed in C++.  One important aspect of
leveraging the off heap memory is how the data is organized as well as
being able to easily access it from the C++ side.  For example how would
you store a multi dimensional array of doubles and how would you specify
that?  Perhaps Avro or Protobuf could be used for storing complex nested
structures although making that a zero copy could be a challenge.
Regardless of how the internals lays the data out in memory the important
requirements are:

a) ensuring zero copy
b) providing a friendly api on the C++ side so folks don't have to deal
with raw bytes, serialization, and JNI
c) ability to specify a complex (multi type and nested) structure via a
schema for memory storage (compile time generated would be sufficient but
run time dynamically would be extremely flexible)

Perhaps a simple way to accomplish would be to enhance dataframes to have a
C++ api that can access the off-heap memory in a clean way from Spark (in
process and w/ zero copy).

Also, is this work being done on a branch I could look into further and try
out?

thanks,
-paul



On Sat, Aug 29, 2015 at 9:40 PM, Reynold Xin <rx...@databricks.com> wrote:

> Supporting non-JVM code without memory copying and serialization is
> actually one of the motivations behind Tungsten. We didn't talk much about
> it since it is not end-user-facing and it is still too early. There are a
> few challenges still:
>
> 1. Spark cannot run entirely in off-heap mode (by entirely here I'm
> referring to all the data-plane memory, not control-plane such as RPCs
> since those don't matter much). There is nothing fundamental. It just takes
> a while to make sure all code paths allocate/free memory using the proper
> allocators.
>
> 2. The memory layout of data is still in flux, since we are only 4 months
> into Tungsten. They will change pretty frequently for the foreseeable
> future, and as a result, the C++ side of things will have change as well.
>
>
>
> On Sat, Aug 29, 2015 at 12:29 PM, Timothy Chen <tn...@gmail.com> wrote:
>
>> I would also like to see data shared off-heap to a 3rd party C++
>> library with JNI, I think the complications would be how to memory
>> manage this and make sure the 3rd party libraries also adhere to the
>> access contracts as well.
>>
>> Tim
>>
>> On Sat, Aug 29, 2015 at 12:17 PM, Paul Weiss <pa...@gmail.com>
>> wrote:
>> > Hi,
>> >
>> > Would the benefits of project tungsten be available for access by
>> non-JVM
>> > programs directly into the off-heap memory?  Spark using dataframes w/
>> the
>> > tungsten improvements will definitely help analytics within the JVM
>> world
>> > but accessing outside 3rd party c++ libraries is a challenge especially
>> when
>> > trying to do it with a zero copy.
>> >
>> > Ideally the off heap memory would be accessible to a non JVM program
>> and be
>> > invoked in process using JNI per each partition.  The alternatives to
>> this
>> > involve additional costs of starting another process if using pipes as
>> well
>> > as the additional copy all the data.
>> >
>> > In addition to read only non-JVM access in process would there be a way
>> to
>> > share the dataframe that is in memory out of process and across spark
>> > contexts.  This way an expensive complicated initial build up of a
>> dataframe
>> > would not have to be replicated as well not having to pay the penalty
>> of the
>> > startup costs on failure.
>> >
>> > thanks,
>> >
>> > -paul
>> >
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
>> For additional commands, e-mail: dev-help@spark.apache.org
>>
>>
>

Re: Tungsten off heap memory access for C++ libraries

Posted by Reynold Xin <rx...@databricks.com>.
Supporting non-JVM code without memory copying and serialization is
actually one of the motivations behind Tungsten. We didn't talk much about
it since it is not end-user-facing and it is still too early. There are a
few challenges still:

1. Spark cannot run entirely in off-heap mode (by entirely here I'm
referring to all the data-plane memory, not control-plane such as RPCs
since those don't matter much). There is nothing fundamental. It just takes
a while to make sure all code paths allocate/free memory using the proper
allocators.

2. The memory layout of data is still in flux, since we are only 4 months
into Tungsten. They will change pretty frequently for the foreseeable
future, and as a result, the C++ side of things will have change as well.



On Sat, Aug 29, 2015 at 12:29 PM, Timothy Chen <tn...@gmail.com> wrote:

> I would also like to see data shared off-heap to a 3rd party C++
> library with JNI, I think the complications would be how to memory
> manage this and make sure the 3rd party libraries also adhere to the
> access contracts as well.
>
> Tim
>
> On Sat, Aug 29, 2015 at 12:17 PM, Paul Weiss <pa...@gmail.com>
> wrote:
> > Hi,
> >
> > Would the benefits of project tungsten be available for access by non-JVM
> > programs directly into the off-heap memory?  Spark using dataframes w/
> the
> > tungsten improvements will definitely help analytics within the JVM world
> > but accessing outside 3rd party c++ libraries is a challenge especially
> when
> > trying to do it with a zero copy.
> >
> > Ideally the off heap memory would be accessible to a non JVM program and
> be
> > invoked in process using JNI per each partition.  The alternatives to
> this
> > involve additional costs of starting another process if using pipes as
> well
> > as the additional copy all the data.
> >
> > In addition to read only non-JVM access in process would there be a way
> to
> > share the dataframe that is in memory out of process and across spark
> > contexts.  This way an expensive complicated initial build up of a
> dataframe
> > would not have to be replicated as well not having to pay the penalty of
> the
> > startup costs on failure.
> >
> > thanks,
> >
> > -paul
> >
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
> For additional commands, e-mail: dev-help@spark.apache.org
>
>

Re: Tungsten off heap memory access for C++ libraries

Posted by Timothy Chen <tn...@gmail.com>.
I would also like to see data shared off-heap to a 3rd party C++
library with JNI, I think the complications would be how to memory
manage this and make sure the 3rd party libraries also adhere to the
access contracts as well.

Tim

On Sat, Aug 29, 2015 at 12:17 PM, Paul Weiss <pa...@gmail.com> wrote:
> Hi,
>
> Would the benefits of project tungsten be available for access by non-JVM
> programs directly into the off-heap memory?  Spark using dataframes w/ the
> tungsten improvements will definitely help analytics within the JVM world
> but accessing outside 3rd party c++ libraries is a challenge especially when
> trying to do it with a zero copy.
>
> Ideally the off heap memory would be accessible to a non JVM program and be
> invoked in process using JNI per each partition.  The alternatives to this
> involve additional costs of starting another process if using pipes as well
> as the additional copy all the data.
>
> In addition to read only non-JVM access in process would there be a way to
> share the dataframe that is in memory out of process and across spark
> contexts.  This way an expensive complicated initial build up of a dataframe
> would not have to be replicated as well not having to pay the penalty of the
> startup costs on failure.
>
> thanks,
>
> -paul
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
For additional commands, e-mail: dev-help@spark.apache.org