spark-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jeremy Freeman <freeman.jer...@gmail.com>
Subject Re: Storing large data for MLlib machine learning
Date Thu, 26 Mar 2015 22:01:12 GMT
Hi Ulvanov, great question, we’ve encountered it frequently with scientific data (e.g. time
series). Agreed text is inefficient for dense arrays, and we also found HDF5+Spark to be a
pain.
 
Our strategy has been flat binary files with fixed length records. Loading these is now supported
in Spark via the binaryRecords method, which wraps a custom Hadoop InputFormat we wrote.

An example (in python):

> # write data from an array
> from numpy import random
> dat = random.randn(100,5)
> f = open('test.bin', 'w')
> f.write(dat)
> f.close()

> # load the data back in
> from numpy import frombuffer
> nrecords = 5
> bytesize = 8
> recordsize = nrecords * bytesize
> data = sc.binaryRecords('test.bin', recordsize)
> parsed = data.map(lambda v: frombuffer(buffer(v, 0, recordsize), 'float'))

> # these should be equal
> parsed.first()
> dat[0,:]

Compared to something like Parquet, this is a little lighter-weight, and plays nicer with
non-distributed data science tools (e.g. numpy). It also scales great (we use it routinely
to process TBs of time series). And handles single files or directories. But it's extremely
simple!

-------------------------
jeremyfreeman.net
@thefreemanlab

On Mar 26, 2015, at 2:33 PM, Ulanov, Alexander <alexander.ulanov@hp.com> wrote:

> Thanks for suggestion, but libsvm is a format for sparse data storing in text file and
I have dense vectors. In my opinion, text format is not appropriate for storing large dense
vectors due to overhead related to parsing from string to digits and also storing digits as
strings is not efficient.
> 
> From: Stephen Boesch [mailto:javadba@gmail.com]
> Sent: Thursday, March 26, 2015 2:27 PM
> To: Ulanov, Alexander
> Cc: dev@spark.apache.org
> Subject: Re: Storing large data for MLlib machine learning
> 
> There are some convenience methods you might consider including:
> 
>           MLUtils.loadLibSVMFile
> 
> and   MLUtils.loadLabeledPoint
> 
> 2015-03-26 14:16 GMT-07:00 Ulanov, Alexander <alexander.ulanov@hp.com<mailto:alexander.ulanov@hp.com>>:
> Hi,
> 
> Could you suggest what would be the reasonable file format to store feature vector data
for machine learning in Spark MLlib? Are there any best practices for Spark?
> 
> My data is dense feature vectors with labels. Some of the requirements are that the format
should be easy loaded/serialized, randomly accessible, with a small footprint (binary). I
am considering Parquet, hdf5, protocol buffer (protobuf), but I have little to no experience
with them, so any suggestions would be really appreciated.
> 
> Best regards, Alexander
> 


Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message