On 2019/03/13 08:42:13, Vinoth Chandar <[email protected]> wrote:
> Hi Rahul,
>
> Good to know. Yes for copy_on_write please turn off inline compaction.
> (Probably explains why the default was false).
>
> Thanks
> Vinoth
>
> On Wed, Mar 13, 2019 at 12:51 AM [email protected] <
> [email protected]> wrote:
>
> >
> >
> > On 2019/03/12 23:04:43, Vinoth Chandar <[email protected]> wrote:
> > > Opened up https://github.com/uber/hudi/pull/599/files to improve this
> > > out-of-box
> > >
> > > On Tue, Mar 12, 2019 at 1:27 PM Vinoth Chandar <[email protected]>
> > wrote:
> > >
> > > > Hi Rahul,
> > > >
> > > > The files you shared all belong to same file group (they share the same
> > > > prefix if you notice) (
> > https://hudi.apache.org/concepts.html#terminologies
> > > > ).
> > > > Given its not creating new file groups every run, means the feature is
> > > > kicking in.
> > > >
> > > > During each insert, Hudi will find the latest file in each file group
> > (I,e
> > > > the one with largest instant time, timestamp) and rewrite/expand that
> > with
> > > > the new inserts. Hudi does not clean up the old files immediately,
> > since
> > > > that can cause running queries to fail, since they could have started
> > even
> > > > hours ago (e.g Hive).
> > > >
> > > > If you want to reduce the number of files you see, you can lower
> > number of
> > > > commits retained
> > > > https://hudi.apache.org/configurations.html#retainCommits
> > > > We retain 24 by default.. i.e after the 25th file, the first one will
> > be
> > > > automatically cleaned..
> > > >
> > > > Does that make sense? Are you able to query this data and find the
> > > > expected records?
> > > >
> > > > Thanks
> > > > Vinoth
> > > >
> > > > On Tue, Mar 12, 2019 at 12:23 PM [email protected] <
> > > > [email protected]> wrote:
> > > >
> > > >>
> > > >>
> > > >> On 2019/03/11 18:25:46, Vinoth Chandar <[email protected]> wrote:
> > > >> > Hi Rahul,
> > > >> >
> > > >> > Hudi/Copy-on-write storage would keep expanding your existing
> > parquet
> > > >> files
> > > >> > to reach the configured file size, once you set the small file size
> > > >> > config..
> > > >> >
> > > >> > For e.g: we at uber, write 1GB files this way.. to do that, you
> > could
> > > >> set
> > > >> > something like this.
> > > >> > http://hudi.apache.org/configurations.html#limitFileSize = 1 *
> > 1024 *
> > > >> 1024
> > > >> > * 1024
> > > >> > http://hudi.apache.org/configurations.html#compactionSmallFileSize
> > =
> > > >> 900 *
> > > >> > 1024 * 1024
> > > >> >
> > > >> >
> > > >> > Please let me know if you have trouble achieving this. Also please
> > use
> > > >> the
> > > >> > insert operation (not bulk_insert) for this to work
> > > >> >
> > > >> >
> > > >> > Thanks
> > > >> > Vinoth
> > > >> >
> > > >> > On Mon, Mar 11, 2019 at 12:32 AM [email protected] <
> > > >> > [email protected]> wrote:
> > > >> >
> > > >> > >
> > > >> > >
> > > >> > > On 2019/03/08 13:43:52, Vinoth Chandar <[email protected]> wrote:
> > > >> > > > Hi Rahul,
> > > >> > > >
> > > >> > > > you can try adding hoodie.parquet.small.file.limit=104857600, to
> > > >> your
> > > >> > > > property file to specify 100MB files. Note that this works only
> > if
> > > >> you
> > > >> > > are
> > > >> > > > using insert (not bulk_insert) operation. Hudi will enforce file
> > > >> sizing
> > > >> > > on
> > > >> > > > ingest time. As of now, there is no support for collapsing these
> > > >> file
> > > >> > > > groups (parquet + related log files) into a large file group
> > > >> (HIP/Design
> > > >> > > > may come soon). Does that help?
> > > >> > > >
> > > >> > > > Also on the compaction in general, since you don't have any
> > updates.
> > > >> > > > I think you can simply use the copy_on_write storage? inserts
> > will
> > > >> go to
> > > >> > > > the parquet file anyway on MOR..(but if you like to be able to
> > deal
> > > >> with
> > > >> > > > updates later, understand where you are going)
> > > >> > > >
> > > >> > > > Thanks
> > > >> > > > Vinoth
> > > >> > > >
> > > >> > > > On Fri, Mar 8, 2019 at 3:25 AM [email protected] <
> > > >> > > > [email protected]> wrote:
> > > >> > > >
> > > >> > > > > Dear All
> > > >> > > > >
> > > >> > > > > I am using DeltaStreamer to stream the data from kafka topic
> > and
> > > >> to
> > > >> > > write
> > > >> > > > > it into the hudi data set.
> > > >> > > > > For this use case I am not doing any upsert all are insert
> > only
> > > >> so each
> > > >> > > > > job creates new parquet file after the inject job. So large
> > > >> number of
> > > >> > > > > small files are creating. how can i merge these files from
> > > >> > > deltastreamer
> > > >> > > > > job using the available configurations.
> > > >> > > > >
> > > >> > > > > I think compactionSmallFileSize may useful for this case,
> > but i
> > > >> am not
> > > >> > > > > sure whether it is for deltastreamer or not. I tried it in
> > > >> > > deltastreamer
> > > >> > > > > but it did't worked. Please assist on this. If possible give
> > one
> > > >> > > example
> > > >> > > > > for the same
> > > >> > > > >
> > > >> > > > > Thanks & Regards
> > > >> > > > > Rahul
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> > >
> > > >> > > Dear Vinoth
> > > >> > >
> > > >> > > For one of my use case , I doing only inserts.For testing i am
> > > >> inserting
> > > >> > > data which have 5-10 records only. I am continuously pushing
> > data to
> > > >> hudi
> > > >> > > dataset. As it is insert only for every insert it's creating new
> > > >> small
> > > >> > > files to the dataset.
> > > >> > >
> > > >> > > If my insertion interval is less and i am planning for data to
> > keep
> > > >> for
> > > >> > > years, this flow will create lots of small files.
> > > >> > > I just want to know whether hudi can merge these small files in
> > any
> > > >> ways.
> > > >> > >
> > > >> > >
> > > >> > > Thanks & Regards
> > > >> > > Rahul P
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > > >> Dear Vinoth
> > > >>
> > > >> I tried below configurations.
> > > >>
> > > >> hoodie.parquet.max.file.size=1073741824
> > > >> hoodie.parquet.small.file.limit=943718400
> > > >>
> > > >> I am using below code for inserting data from json kafka source.
> > > >>
> > > >> spark-submit --class
> > > >> com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer
> > > >> hoodie-utilities-0.4.5.jar --storage-type COPY_ON_WRITE --source-class
> > > >> com.uber.hoodie.utilities.sources.JsonKafkaSource
> > --source-ordering-field
> > > >> stype --target-base-path /MERGE --target-table MERGE --props
> > > >> /hudi/kafka-source.properties --schemaprovider-class
> > > >> com.uber.hoodie.utilities.schema.FilebasedSchemaProvider --op insert
> > > >>
> > > >> But for each insert job it's creating new parquet file. It's not
> > touching
> > > >> old parquet files.
> > > >>
> > > >> For reference i am sharing some of the parquet files of hudi dataset
> > > >> which are generating as part of DeltaStreamer data insertion.
> > > >>
> > > >> 93 /MERGE/2019/03/06/.hoodie_partition_metadata
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002655.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002733.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002754.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002815.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002837.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002859.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002921.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312002942.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312003003.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312003024.parquet
> > > >> 424.0 K
> > > >>
> > /MERGE/2019/03/06/1e9735d2-2057-40c6-a4df-078eb297a298_0_20190312003045.parquet
> > > >>
> > > >> Each job it's creating files of 424K & it's not merging any. Can you
> > > >> please confirm whether hudi can achieve the use case which i
> > mentioned. If
> > > >> this merging/compacting feature is there, kindly tell what i am
> > missing
> > > >> here.
> > > >>
> > > >> Thanks & Regards
> > > >> Rahul
> > > >>
> > > >>
> > >
> >
> > Dear Vinoth
> >
> > I too verified that the feature is kicking in.
> > I am using below properties and my insert job is running with 10S interval.
> >
> > hoodie.cleaner.commits.retained=6
> > hoodie.keep.max.commits=6
> > hoodie.keep.min.commits=3
> > hoodie.parquet.small.file.limit=943718400
> > hoodie.parquet.max.file.size=1073741824
> > hoodie.compact.inline=false
> >
> > Now i can see about 180 files in the hudi data set with
> > hoodie.compact.inline=false.
> > hadoop fs -ls /MERGE/2019/03/14/* | wc -l
> > 181
> >
> > if set hoodie.compact.inline=true
> > i am getiing below error
> >
> > Loaded instants [[20190313131254__clean__COMPLETED],
> > [20190313131254__commit__COMPLETED], [20190313131316__clean__COMPLETED],
> > [20190313131316__commit__COMPLETED], [20190313131339__clean__COMPLETED],
> > [20190313131339__commit__COMPLETED], [20190313131401__clean__COMPLETED],
> > [20190313131401__commit__COMPLETED], [20190313131423__clean__COMPLETED],
> > [20190313131423__commit__COMPLETED], [20190313131445__clean__COMPLETED],
> > [20190313131445__commit__COMPLETED], [20190313131512__commit__COMPLETED]]
> > Exception in thread "main"
> > com.uber.hoodie.exception.HoodieNotSupportedException: Compaction is not
> > supported from a CopyOnWrite table
> >
> > at
> > com.uber.hoodie.table.HoodieCopyOnWriteTable.scheduleCompaction(HoodieCopyOnWriteTable.java:168)
> >
> >
> > please assist on this.
> >
> > Thanks & Regards
> > Rahul
> >
> >
>
Dear Vinod
Previous mail i already mentioned i am seeing more than 180 parquet files.
hadoop fs -ls /MERGE/2019/03/14/* | wc -l
181
I given commit to retain as 6(hoodie.cleaner.commits.retained=6) only. why then
181 files are coming. I am facing problem at this point.
Thanks & Regards
Rahul