Hi Shuai,
It should certainly be possible to do it that way, but I would recommend
against it. If you look at HadoopRDD, its doing all sorts of little
book-keeping that you would most likely want to mimic. eg., tracking the
number of bytes records that are read, setting up all the hadoop
configuration, splits, readers, scheduling tasks for locality, etc. Thats
why I suggested that really you want to just create a small variant of
HadoopRDD.
hope that helps,
Imran
On Sat, Mar 14, 2015 at 11:10 AM, Shawn Zheng szheng.c...@gmail.com wrote:
Sorry for reply late.
But I just think of one solution: if I load all the file name itself (not
the contain of the file), so I have a RDD[key, iterable[filename]], then I
run mapPartitionsToPair on it with preservesPartitioning=true
Do you think it is a right solution? I am not sure whether it has
potential issue if I try to fake/enforce the partition in my own way.
Regards,
Shuai
On Wed, Mar 11, 2015 at 8:09 PM, Imran Rashid iras...@cloudera.com
wrote:
It should be *possible* to do what you want ... but if I understand you
right, there isn't really any very easy way to do it. I think you would
need to write your own subclass of RDD, which has its own logic on how the
input files get put divided among partitions. You can probably subclass
HadoopRDD and just modify getPartitions(). your logic could look at the
day of each filename to decide which partition it goes into. You'd need to
make corresponding changes for HadoopPartition the compute() method.
(or if you can't subclass HadoopRDD directly you can use it for
inspiration.)
On Mon, Mar 9, 2015 at 11:18 AM, Shuai Zheng szheng.c...@gmail.com
wrote:
Hi All,
If I have a set of time series data files, they are in parquet format
and the data for each day are store in naming convention, but I will not
know how many files for one day.
20150101a.parq
20150101b.parq
20150102a.parq
20150102b.parq
20150102c.parq
…
201501010a.parq
…
Now I try to write a program to process the data. And I want to make
sure each day’s data into one partition, of course I can load all into one
big RDD to do partition but it will be very slow. As I already know the
time series of the file name, is it possible for me to load the data into
the RDD also preserve the partition? I know I can preserve the partition by
each file, but is it anyway for me to load the RDD and preserve partition
based on a set of files: one partition multiple files?
I think it is possible because when I load a RDD from 100 files (assume
cross 100 days), I will have 100 partitions (if I disable file split when
load file). Then I can use a special coalesce to repartition the RDD? But I
don’t know is it possible to do that in current Spark now?
Regards,
Shuai