Answered inline.
On Thu, Nov 14, 2019 at 10:30 AM Zhengxiang Pan <[email protected]> wrote: > . > Great, this solve my 2nd issue. Followup question: HUDI internal columns ( > _hoodie_record_key, _hoodie_partition_path), now duplicate my existing > columns(keys), my concern is that this will increase the data size, or I > should not worry about it. These are fields Hudi requires and shouldnt be a cause fo concern wrt to data size. > secondly, how do you control the partitions? I > am most concerned about too many small files. > Hudi provides config knobs to control small files. You can leverage the config - compactionSmallFileSize <http://localhost:4000/configurations.html#compactionSmallFileSize> to do so. Thanks > Pan > > On Thu, Nov 14, 2019 at 12:42 AM Bhavani Sudha <[email protected]> > wrote: > > > It should be possible by using ComplexKeyGenerator > > < > > > https://github.com/apache/incubator-hudi/blob/master/hudi-spark/src/main/java/org/apache/hudi/ComplexKeyGenerator.java > > > > > class for the config - KEYGENERATOR_CLASS_OPT_KEY > > <https://hudi.apache.org/configurations.html#KEYGENERATOR_CLASS_OPT_KEY > >. > > I > > ll try to search for examples for this. > > > > Thanks, > > Sudha > > > > On Wed, Nov 13, 2019 at 3:59 PM Zhengxiang Pan <[email protected]> > wrote: > > > > > Hi Sudha, > > > Your example clears my understanding. that was the happening as my > > row_key > > > is not unique. > > > > > > Is there a way to define multiple (or composite) columns as row_key? > > > > > > Thanks > > > Pan > > > > > > On Wed, Nov 13, 2019 at 5:46 PM Bhavani Sudha <[email protected] > > > > > wrote: > > > > > > > Hi Zhengxiang, > > > > > > > > precombine works like this. If there are several rows with the same > > > row_key > > > > in an insert/update batch of records, the precombine key will be used > > to > > > > pick the latest value of the same row_key. Taking a really simple > > > example, > > > > assume these are the 6 records in the original dataset. > > > > |row_key|precombine_key|other columns|...| > > > > |abc|1|...|...| > > > > |abc|2|...|...| > > > > |def|3|...|...| > > > > |abc|4|...|...| > > > > |def|5|...|...| > > > > |ghi|6|...|...| > > > > > > > > On applying the precombine the hudi dataset becomes: > > > > |abc|4|...|...| > > > > |def|5|...|...| > > > > |ghi|6|...|...| > > > > > > > > In this case you will not see all 6 records. It will be reduced to 1 > > per > > > > distinct row_key after applying the precombine logic. I think this is > > > what > > > > is happening in your case. I noticed that the precombine key is a > > string > > > > from the snippet. String.compareTo would be used to determine the > > latest > > > > value of strings. > > > > Please note that in the above example, I assumed default values for > the > > > > configs "PAYLOAD_CLASS_OPT_KEY > > > > <https://hudi.apache.org/configurations.html#PAYLOAD_CLASS_OPT_KEY > >", > > " > > > > PRECOMBINE_FIELD_OPT_KEY > > > > < > https://hudi.apache.org/configurations.html#PRECOMBINE_FIELD_OPT_KEY > > >", > > > > etc. You can change these configs based on your needs. > > > > > > > > Can you please verify if this is the case? > > > > > > > > Thanks, > > > > Sudha > > > > > > > > > > > > On Wed, Nov 13, 2019 at 2:11 PM Zhengxiang Pan <[email protected]> > > > wrote: > > > > > > > > > Hi Sudha, > > > > > Yes, I did check, the number of distinct row_key matches. My > > > > understanding > > > > > is that row_key is not the key to do de-dup. My row_key is not > > unique, > > > > > meaning several rows might have the same row_key, but pre-combine > key > > > for > > > > > sure is unique. > > > > > > > > > > Thanks, > > > > > Pan > > > > > > > > > > On Wed, Nov 13, 2019 at 2:54 PM Bhavani Sudha < > > [email protected] > > > > > > > > > wrote: > > > > > > > > > > > Hi Zhengxiang, > > > > > > > > > > > > regarding issue 2, were you able to confirm if the number of > > distinct > > > > > > row_key in your original df and the distinct row_key in Hudi > > dataset > > > > > > matches? If that matches, then we can dig into the precombine > > logic > > > to > > > > > see > > > > > > whats happening. > > > > > > > > > > > > Thanks, > > > > > > Sudha > > > > > > > > > > > > On Tue, Nov 12, 2019 at 9:42 AM Zhengxiang Pan < > [email protected]> > > > > > wrote: > > > > > > > > > > > > > Hi Balaji.V, > > > > > > > W.r.t issue 1), same issue occurs with spark 2.3.4. > > > > > > > > > > > > > > Pan > > > > > > > > > > > > > > > > > > > > > > > > > > > >
