Re: Migrate Existing DataFrame to Hudi DataSet

2019-11-14 Thread Bhavani Sudha
Answered inline.



On Thu, Nov 14, 2019 at 10:30 AM Zhengxiang Pan  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
 to do
so.

Thanks
> Pan
>
> On Thu, Nov 14, 2019 at 12:42 AM Bhavani Sudha 
> 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
> >  >.
> > I
> > ll try to search for examples for this.
> >
> > Thanks,
> > Sudha
> >
> > On Wed, Nov 13, 2019 at 3:59 PM Zhengxiang Pan 
> 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  >
> > > 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
> > > >  >",
> > "
> > > > 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 
> > > 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 <
> > bhavanisud...@gmail.com
> > > >
> > > > > 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 <
> zx...@alphonso.tv>
> > > > > wrote:
> > > > > >
> > > > > > > Hi Balaji.V,
> > > > > > > W.r.t issue 1), same issue occurs with spark 2.3.4.
> > > > > > >
> > > > > > > Pan
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] RFC-10: Restructuring and auto-generation of docs

2019-11-14 Thread Y Ethan Guo
Hey Gurudatt,

Thanks for the great feedback!  Comments inlined...

On Wed, Nov 13, 2019 at 10:57 PM Gurudatt Kulkarni 
wrote:

> Hi Ethan,
>
> Thanks for the RFC. I have a few observations about the docs. I saw the
> diagram for the docs, asf-site and master are kept as separate branches,
> currently.
>
> (1)  I suggest two approaches for maintaining the docs looking at how other
> popular Apache projects do,
>
>-  Apache Flink   -
>Keeping docs part of the master.
>
>
>- Keeping docs part of the master will help in updating the docs and
>   code simultaneously when the releases are cut. (I prefer this).
>
>
Agree.  I also suggest putting docs along with the code on master in the
RFC.


>
>- Apache Kafka  - Keeping a
>separate repository for documentation.
>
>
>- A separate repo will give much more flexibility and independence in
>   terms of deployment, but will also be a bit inconvenient in terms of
>   switching between the repository while making changes, like the
> current
>   case of different branches. I can't think of any other
> advantages of doing
>   this.
>

Yeah, in the current RFC I suggest maintaining the site-level documentation
(more of high-level information, not specific to a release) in the asf-site
branch as what we have now, to continue the tradition.  Yet I'm also fine
with a separate repo for maintaining the documentation site.  Apache Spark
uses this approach as well (https://github.com/apache/spark-website).

@vinoth  wdyt?


>
>
> (2)  Introduce a new section for listing PMC members and Committers  -
>
> This will help in getting in touch with core contributors and give them
> visibility as well.
>
>
Good idea.  Can we have this in the "Community" page?


>
> (3)  For improving new contributor experience,
>
> Add a little more detail regarding the project structure and what each
> module is designed for. A section on describing the Hudi metadata structure
> (A good reference https://kafka.apache.org/protocol)
>
>
Yes, this is handy.  We can also explain the information of Hudi metadata
structure in the user documentation.


>
> (4) Regarding the comparison table with other alternatives
>
> Not sure if we should keep this. Since all projects are developing at a
> good pace, keeping tabs on what each project is doing or has implemented
> can be taxing, since people here are majorly working on Hudi. It shouldn't
> happen that we present a misleading or old comparison regarding the
> features of other projects. I feel benchmarks and comparisons make a good
> topic for blog posts since they are always dated.
>

Agree that the comparison can get outdated if not updated frequently.  So
may be add the time when the comparison is done (either in blog or a
separate page), e.g., "as of Nov 2019"?

I think it's still beneficial even for ourselves to compare Hudi with other
solutions periodically so we prioritize our work based on the state of the
art and keep Hudi relevant in the ecosystem.  Users may want to use this
comparison to choose among Hudi and others.  Maybe we can update the
comparison every Hudi release (a month or two) or quarter?


> Regards,
> Gurudatt
>
>
> On Thu, Nov 14, 2019 at 6:21 AM Y Ethan Guo 
> wrote:
>
> > Hey folks,
> >
> > I put my thoughts around the topic in this RFC:
> >
> > RFC-10: Restructuring and auto-generation of docs
> >
> >
> https://cwiki.apache.org/confluence/display/HUDI/RFC-10%3A+Restructuring+and+auto-generation+of+docs
> >
> > Feel free to provide feedback there or here.
> >
> > Thanks,
> > - Ethan
> >
>


Re: Migrate Existing DataFrame to Hudi DataSet

2019-11-14 Thread Zhengxiang Pan
.
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.  secondly, how do you control the partitions? I
am most concerned about too many small files.

Thanks
Pan

On Thu, Nov 14, 2019 at 12:42 AM Bhavani Sudha 
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
> .
> I
> ll try to search for examples for this.
>
> Thanks,
> Sudha
>
> On Wed, Nov 13, 2019 at 3:59 PM Zhengxiang Pan  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 
> > 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
> > > ",
> "
> > > 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 
> > 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 <
> bhavanisud...@gmail.com
> > >
> > > > 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 
> > > > wrote:
> > > > >
> > > > > > Hi Balaji.V,
> > > > > > W.r.t issue 1), same issue occurs with spark 2.3.4.
> > > > > >
> > > > > > Pan
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: DISCUSS RFC 6 - Add indexing support to the log file

2019-11-14 Thread Vinoth Chandar
Since attachments don't really work on the mailing list, Can you may be
attach them to comments on the RFC itself?

In this scenario, we will get a larger range than is probably in the newly
compacted base file, correct? Current thinking is, yes it will lead to less
efficient pruning by ranges, but should still be correct? Do you have
correctness concerns? If so, I would really like to understand since there
is a good chance its not fixable :)

We could have the next delta commit log a "corrected" rangeInfo down the
line as an optimization. May be leave the initial design simpler, as this
adds to complexity around compatibility handling and so on? Good point, we
can think about it more.

On Thu, Nov 14, 2019 at 6:37 AM Sivabalan  wrote:

> I have s doubt on the design. I guess this is the right place to discuss.
>
> I want to understand how compaction interplays with this new scheme.
> Let's assume all log block are of new format only. Once compaction
> completes, those log blocks/files not compacted will have range info
> pertaining to compacted ones right? When this will get fixed? Won't the
> look up return true for those keys from compacted log files. I have
> attached two diagrams depicting before and after compaction. If you look at
> 2nd pic (after compaction), ideally min and max should have been 6 and 11.
>
> In general, when does the key range pruning will happen? And will the
> bloom filter also be adjusted?
>
>
> On Wed, Oct 30, 2019 at 10:09 PM Nishith  wrote:
>
>> Thanks for the detailed design write up Vinoth. I concur with the others
>> on option 2, default indexing as off and enable it when we have enough
>> confidence on stability & performance. Although, I do think practically it
>> might be good to have the code in place for users who might revert to an
>> older build as part of some build rollback mechanisms that they may have in
>> place (for reasons not even related to hudi). The latest data block
>> (denoted by the latest version) being a new block as suggested by Balaji
>> sounds like one option - not sure how the complicated the code will become
>> though...
>> Will comment on the RFC about some doubts/concerns regarding first
>> migration customers from canIndexLogFiles = false to true and then rollback
>> to ensure my understand is correct.
>>
>> -Nishith
>>
>> Sent from my iPhone
>>
>> > On Oct 30, 2019, at 4:00 PM, Balaji Varadarajan
>>  wrote:
>> >
>> > Thanks Vinoth for proposing a clean and extendable design. The overall
>> design looks great. Another rollout option is to only use consolidated log
>> index for index lookup if latest "valid" log block has been written in new
>> format. If that is not the case, we can revert to scanning previous log
>> blocks for index lookup.
>> > Balaji.VOn Tuesday, October 29, 2019, 07:52:00 PM PDT, Bhavani
>> Sudha  wrote:
>> >
>> > I vote for the second option. Also it can give time to analyze on how to
>> > deal with backwards compatibility. I ll take a look at the RFC later
>> > tonight and get back.
>> >
>> >
>> >> On Sun, Oct 27, 2019 at 10:24 AM Vinoth Chandar 
>> wrote:
>> >>
>> >> One issue I have some open questions myself
>> >>
>> >> Is it ok to assume log will have old data block versions, followed by
>> new
>> >> data block versions. For e.g, if rollout new code, then revert back
>> then
>> >> there could be an arbitrary mix of new and old data blocks. Handling
>> this
>> >> might make design/code fairly complex. Alternatively we can keep it
>> simple
>> >> for now, disable by default and only advise to enable for new tables or
>> >> when hudi version is stable
>> >>
>> >>
>> >>> On Sun, Oct 27, 2019 at 12:13 AM Vinoth Chandar 
>> wrote:
>> >>>
>> >>>
>> >>>
>> >>
>> https://cwiki.apache.org/confluence/display/HUDI/RFC-6+Add+indexing+support+to+the+log+file
>> >>>
>> >>>
>> >>> Feedback welcome, on this RFC tackling HUDI-86
>> >>>
>> >>
>>
>
>
> --
> Regards,
> -Sivabalan
>


Re: aws dependencies not working for writing for S3 Write access

2019-11-14 Thread Vinoth Chandar
Hi,

You might want to subscribe the mailing list, so that the replies actually
make it to the list automatically.

This seems like a class version mismatch between jars, since you. are
getting NoSuchMethodError (and not NoClassDefFound..)
We don't bundle either hadoop or aws or spark jars. There is no special
config here, except
So, are you able to access s3 via regular spark.read.parquet(..)
successfully?

- Vinoth

On Thu, Nov 14, 2019 at 8:43 AM Sudharshan Rajendhiran <
sudhars...@sensehq.com> wrote:

> Hello, can anyone point me to the right dependencies to configure Hudi
> with to write to S3
>
> I start the Spark shell with aws sdk and hadoop-aws libs as per the S3
> guide
> with hudi.conf consists of spark Kryo  serialiser  and S3 keys.
>
> spark-shell --jars
> $HUDI_SPARK_BUNDLE_PATH,/tmp/hudi/hadoop-aws-2.7.3.jar,/tmp/hudi/aws-java-sdk-1.7.4.jar
> --properties-file ~/hudi.conf
>
> import org.apache.hudi.QuickstartUtils._
> import scala.collection.JavaConversions._
> import org.apache.spark.sql.SaveMode._
> import org.apache.hudi.DataSourceReadOptions._
> import org.apache.hudi.DataSourceWriteOptions._
> import org.apache.hudi.config.HoodieWriteConfig._
>
> val tableName = "hudi_cow_table"
> val s3path = "s3a://bucket-name/hudi-test/"
> val dataGen = new DataGenerator
>
> val inserts = convertToStringList(dataGen.generateInserts(10))
> val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
> df.write.format("org.apache.hudi").
> options(getQuickstartWriteConfigs).
> option(PRECOMBINE_FIELD_OPT_KEY, "ts").
> option(RECORDKEY_FIELD_OPT_KEY, "uuid").
> option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
> option(TABLE_NAME, tableName).
> mode(Overwrite).
> save(s3path);
>
>
> results in Error
>
> java.lang.NoSuchMethodError:
> com.amazonaws.services.s3.transfer.TransferManager.(Lcom/amazonaws/services/s3/AmazonS3;Ljava/util/concurrent/ThreadPoolExecutor;)V
>   at
> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:287)
>   at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669)
>   at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
>   at
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
>   at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
>   at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
>   at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
>   at
> org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:98)
>   at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:91)
>   at
> org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
>   at
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
>   at
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
>   at
> org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:86)
>   at
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
>   at
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
>   at
> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
>   at
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>   at
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
>   at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
>   at
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
>   at
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
>   at
> org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
>   at
> org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
>   at
> org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
>   at
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
>   at
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
>   at
> org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
>   at
> org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:285)
>   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
>   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
>   ... 68 elided
>
>


aws dependencies not working for writing for S3 Write access

2019-11-14 Thread Sudharshan Rajendhiran
Hello, can anyone point me to the right dependencies to configure Hudi with to 
write to S3 

I start the Spark shell with aws sdk and hadoop-aws libs as per the S3 guide 
with hudi.conf consists of spark Kryo  serialiser  and S3 keys. 

spark-shell --jars 
$HUDI_SPARK_BUNDLE_PATH,/tmp/hudi/hadoop-aws-2.7.3.jar,/tmp/hudi/aws-java-sdk-1.7.4.jar
  --properties-file ~/hudi.conf

import org.apache.hudi.QuickstartUtils._
import scala.collection.JavaConversions._
import org.apache.spark.sql.SaveMode._
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.config.HoodieWriteConfig._

val tableName = "hudi_cow_table"
val s3path = "s3a://bucket-name/hudi-test/"
val dataGen = new DataGenerator

val inserts = convertToStringList(dataGen.generateInserts(10))
val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD_OPT_KEY, "ts").
option(RECORDKEY_FIELD_OPT_KEY, "uuid").
option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
option(TABLE_NAME, tableName).
mode(Overwrite).
save(s3path);


results in Error

java.lang.NoSuchMethodError: 
com.amazonaws.services.s3.transfer.TransferManager.(Lcom/amazonaws/services/s3/AmazonS3;Ljava/util/concurrent/ThreadPoolExecutor;)V
  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:287)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
  at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
  at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
  at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:98)
  at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:91)
  at 
org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
  at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:86)
  at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
  at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
  at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
  at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
  at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
  at 
org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
  at 
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
  at 
org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
  at 
org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
  at 
org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
  at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
  at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
  at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
  at 
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:285)
  at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
  at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
  ... 68 elided



Re: DISCUSS RFC 6 - Add indexing support to the log file

2019-11-14 Thread Sivabalan
I have s doubt on the design. I guess this is the right place to discuss.

I want to understand how compaction interplays with this new scheme.
Let's assume all log block are of new format only. Once compaction
completes, those log blocks/files not compacted will have range info
pertaining to compacted ones right? When this will get fixed? Won't the
look up return true for those keys from compacted log files. I have
attached two diagrams depicting before and after compaction. If you look at
2nd pic (after compaction), ideally min and max should have been 6 and 11.

In general, when does the key range pruning will happen? And will the bloom
filter also be adjusted?


On Wed, Oct 30, 2019 at 10:09 PM Nishith  wrote:

> Thanks for the detailed design write up Vinoth. I concur with the others
> on option 2, default indexing as off and enable it when we have enough
> confidence on stability & performance. Although, I do think practically it
> might be good to have the code in place for users who might revert to an
> older build as part of some build rollback mechanisms that they may have in
> place (for reasons not even related to hudi). The latest data block
> (denoted by the latest version) being a new block as suggested by Balaji
> sounds like one option - not sure how the complicated the code will become
> though...
> Will comment on the RFC about some doubts/concerns regarding first
> migration customers from canIndexLogFiles = false to true and then rollback
> to ensure my understand is correct.
>
> -Nishith
>
> Sent from my iPhone
>
> > On Oct 30, 2019, at 4:00 PM, Balaji Varadarajan
>  wrote:
> >
> > Thanks Vinoth for proposing a clean and extendable design. The overall
> design looks great. Another rollout option is to only use consolidated log
> index for index lookup if latest "valid" log block has been written in new
> format. If that is not the case, we can revert to scanning previous log
> blocks for index lookup.
> > Balaji.VOn Tuesday, October 29, 2019, 07:52:00 PM PDT, Bhavani Sudha
>  wrote:
> >
> > I vote for the second option. Also it can give time to analyze on how to
> > deal with backwards compatibility. I ll take a look at the RFC later
> > tonight and get back.
> >
> >
> >> On Sun, Oct 27, 2019 at 10:24 AM Vinoth Chandar 
> wrote:
> >>
> >> One issue I have some open questions myself
> >>
> >> Is it ok to assume log will have old data block versions, followed by
> new
> >> data block versions. For e.g, if rollout new code, then revert back then
> >> there could be an arbitrary mix of new and old data blocks. Handling
> this
> >> might make design/code fairly complex. Alternatively we can keep it
> simple
> >> for now, disable by default and only advise to enable for new tables or
> >> when hudi version is stable
> >>
> >>
> >>> On Sun, Oct 27, 2019 at 12:13 AM Vinoth Chandar 
> wrote:
> >>>
> >>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/HUDI/RFC-6+Add+indexing+support+to+the+log+file
> >>>
> >>>
> >>> Feedback welcome, on this RFC tackling HUDI-86
> >>>
> >>
>


-- 
Regards,
-Sivabalan