[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940892#comment-16940892 ] Vinoth Chandar commented on HUDI-269: - [~XingXPan] You should nt have to choose based on this.. > if I don't have throttle, delta streamer will send too many requests even no > input comming) Throttle is a useful feature. Each run delta streamer will try to list the target directory for obtaining the previous checkpoint. We had always been testing with large volume input streams I think :) >But if I have kafka input streaming like 10 records per second, I found that >even if I set the 5 seconds throttle, writing hudi with delta stream will >cause 10 times more request than do it in data source writer way. We will fix the delta streamer and get it inline with the data source writer. Conceptually cant think of anything here that will make it 10x more calls. Are you running the datasource every 5 seconds as well? Also is there a way for you to tell us what these addtional requests are going to at the s3 level i.e what file/objects does it access in both cases.. Then one of us can try reproducing this and fixinf.. once we identify, it should be a simple fix if any. Thanks for working thru this with us. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: Balaji Varadarajan >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png, image-2019-09-26-09-02-24-761.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16939797#comment-16939797 ] Xing Pan commented on HUDI-269: --- [~vinoth] I thought it's because of that delta streamer too aggressive too, so I add the throttle param to control this, and it helped. now the issue is: I do significantly reduce requests count by set delta streamer *throttle to 5 seconds*. * when there is *nothing* coming from kafka, request matrix looks acceptable both in data source writer way and delta streamer way. * But if I have kafka input streaming like 10 records per second, I found that even if I set the 5 seconds throttle, writing hudi with delta stream will cause 10 times more request than do it in data source writer way. so I would choose data source writer anyway since in this way I always save requests count. so now I am wondering what are the pro and cons for me to choose between spark datasource and delta streamer. as far as I can see, in my scenario, if I use delta streamer: * Delta streamer can help ingested data from kafka * It have a self managed checkpoint. * I can set the compaction job weight and if I use spark data source writer: * I have more control of my code, I can have my own kafka ingestion implementation * I will save money :) (still costs lower if delta streamer have a throttle control for now) so if currently if I can't make two request matrix on the same level, I'd use data source writer. Any more suggestions on choosing delta streamer? > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: Balaji Varadarajan >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png, image-2019-09-26-09-02-24-761.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16939647#comment-16939647 ] Vinoth Chandar commented on HUDI-269: - When you are writing your own spark app, specifying hoodie options using --conf may not be taking effect? the ones below? {code} --conf hoodie.embed.timeline.server=true \ --conf hoodie.filesystem.view.incr.timeline.sync.enable=true \ {code} I am wondering if there requests are just due to delta streamer being way too aggressively listing the partition over and over. (unlike the spark streaming code, which has a sane 5s processing window? Was deltastreamer generating many commits/second without teh throttle? > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: Balaji Varadarajan >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png, image-2019-09-26-09-02-24-761.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16939642#comment-16939642 ] Vinoth Chandar commented on HUDI-269: - [~uditme] as fyi, in case you can quickly spot anything.. :) > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: Balaji Varadarajan >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png, image-2019-09-26-09-02-24-761.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16938543#comment-16938543 ] Balaji Varadarajan commented on HUDI-269: - cc [~vinoth]. We will look into this > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: Balaji Varadarajan >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png, image-2019-09-26-09-02-24-761.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16938163#comment-16938163 ] Xing Pan commented on HUDI-269: --- I tried to run the same hudi app via hudi spark datasource writer: {code:java} spark .readStream .format("kafka") .option("kafka.bootstrap.servers", KAFKA_SERVER) .option("subscribe", DEMO_11_TOPIC) .load() .select(from_confluent_avro(col("value"), SCHEMA_REGISTRY_CONF) as 'data).select("data.*") .writeStream.format("org.apache.hudi") .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, tableType) .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "id") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "dateStr") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts") .option(HoodieWriteConfig.TABLE_NAME, DEMO_11_TABLE_NAME) .option("checkpointLocation", checkpointPath) .option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY, DEMO_11_TABLE_NAME) .option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY, "default") .option(DataSourceWriteOptions.HIVE_URL_OPT_KEY, HIVE_URL) .option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "dateStr") .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName) .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true") .outputMode(OutputMode.Append) .trigger(Trigger.ProcessingTime(5000)) .start(outputPath) .awaitTermination() {code} {code:java} spark-submit --class xxx.HudiSpark \ --jars \ xxx/hudi-spark-bundle-0.5.1-SNAPSHOT.jar,\ xxx/abris_2.11-3.0.1.jar,\ xxx/common-utils-5.3.0.jar,xxx/kafka-schema-registry-client-5.3.0.jar,xxx/kafka-avro-serializer-5.3.0.jar,xxx/common-config-5.3.0.jar \ --packages org.apache.spark:spark-sql-kafka-0-10_2.11:2.4.3,org.apache.spark:spark-avro_2.11:2.4.3 \ --conf spark.hadoop.fs.s3a.endpoint=s3-ap-east-1.amazonaws.com \ --conf spark.dynamicAllocation.executorIdleTimeout=10s \ --conf hoodie.embed.timeline.server=true \ --conf hoodie.filesystem.view.incr.timeline.sync.enable=true \ --conf hoodie.upsert.shuffle.parallelism=2 \ --executor-memory 1g \ my_test.jar {code} and push 300 records for every second, and the S3 request count is fairly low: !image-2019-09-26-09-02-24-761.png! I am not quite sure about the difference between datasource writer and delta streamer, as far as I know, when there is no data coming, request count is about the same, but if I push some record every second, *datasource writer* costs about 10 times lower request count than delta streamer. [~vinoth] > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: Balaji Varadarajan >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png, image-2019-09-26-09-02-24-761.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937825#comment-16937825 ] Xing Pan commented on HUDI-269: --- [~vinoth] , I'm planing to use hudi in our data lake project and happy to contribute. Since this naive throttle feature in this ticket will not actually solve the request issue completely, I will do some deeper investigation on this. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937760#comment-16937760 ] Vinoth Chandar commented on HUDI-269: - This is super useful [~XingXPan]. I am currently looking into performance more holistically.. Will add this to one of items to consider. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937363#comment-16937363 ] Xing Pan commented on HUDI-269: --- [~vbalaji] ic, I was just trying to observe the request count change in a simplest way, so I just generate thousands of records in one partition. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937360#comment-16937360 ] BALAJI VARADARAJAN commented on HUDI-269: - ok, looks like there is only one file and one partition in the whole dataset. So, its understandable why you did not see any benefits with timeline server. With larger dataset, you should observe savings. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: hudi_request_test.tar.gz, > image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937358#comment-16937358 ] Xing Pan commented on HUDI-269: --- [~vbalaji] yea, these strange 5K requests are mainly head requests, and cause a lot of s3 4xx error, which is defined as "client side error". I only have one partition "1100/01/01", and attached pleas find the *hudi_request_test.tar.gz* {code:java} aws s3 ls s3://xxx/output/1100/01/01/ 2019-09-25 01:56:57 93 .hoodie_partition_metadata 2019-09-25 02:12:18 535993 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-111-99_20190925021213.parquet 2019-09-25 02:50:30 679546 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-126-108_20190925025025.parquet 2019-09-25 02:32:27 597943 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-135-117_20190925023222.parquet 2019-09-25 02:38:03 623372 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-135-117_20190925023758.parquet 2019-09-25 02:12:48 537971 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-149-130_20190925021243.parquet 2019-09-25 02:50:39 680323 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-161-136_20190925025033.parquet 2019-09-25 02:32:57 599788 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-173-148_20190925023252.parquet 2019-09-25 02:38:33 625295 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-173-148_20190925023828.parquet 2019-09-25 02:13:18 540308 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-187-161_20190925021313.parquet 2019-09-25 02:50:47 681076 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-196-164_20190925025042.parquet 2019-09-25 02:31:07 591207 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-21-24_20190925023057.parquet 2019-09-25 02:36:48 615894 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-21-24_20190925023637.parquet 2019-09-25 02:50:01 675036 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-21-24_20190925024946.parquet 2019-09-25 02:33:27 602011 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-211-179_20190925023322.parquet 2019-09-25 02:39:03 627524 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-211-179_20190925023858.parquet 2019-09-25 02:13:48 542690 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-225-192_20190925021343.parquet 2019-09-25 02:50:55 681495 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-231-192_20190925025049.parquet 2019-09-25 02:33:57 604273 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-249-210_20190925023352.parquet 2019-09-25 02:39:33 629743 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-249-210_20190925023928.parquet 2019-09-25 02:14:18 545021 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-263-223_20190925021413.parquet 2019-09-25 02:51:03 682267 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-266-220_20190925025058.parquet 2019-09-25 02:34:27 606495 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-287-241_20190925023422.parquet 2019-09-25 02:40:03 632018 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-287-241_20190925023958.parquet 2019-09-25 02:51:11 682667 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-301-248_20190925025106.parquet 2019-09-25 02:14:48 547294 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-301-254_20190925021443.parquet 2019-09-25 02:34:57 608770 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-325-272_20190925023452.parquet 2019-09-25 02:40:33 634280 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-325-272_20190925024028.parquet 2019-09-25 02:51:18 683418 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-336-276_20190925025113.parquet 2019-09-25 02:15:18 549588 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-339-285_20190925021513.parquet 2019-09-25 01:56:59 533148 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-35-37_20190925015651.parquet 2019-09-25 02:35:27 610998 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-363-303_20190925023522.parquet 2019-09-25 02:41:04 636524 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-363-303_20190925024058.parquet 2019-09-25 02:51:26 683833 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-371-304_20190925025121.parquet 2019-09-25 02:15:48 551902 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-377-316_20190925021543.parquet 2019-09-25 02:35:57 613259 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-401-334_20190925023552.parquet 2019-09-25 02:41:33 638757 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-401-334_20190925024128.parquet 2019-09-25 02:51:34 684572 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-406-332_20190925025130.parquet 2019-09-25 02:16:18 553820 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-415-347_20190925021613.parquet 2019-09-25 02:42:03 641007 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-439-365_20190925024158.parquet 2019-09-25 02:51:42 684965 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-441-360_20190925025137.parquet 2019-09-25 02:16:48 556070 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-453-378_20190925021643.parquet 2019-09-25 02:51:49 685729 68d656cc-65a5-47f7-bf28-961315e718bc-0_0-476-388_20190925025144.parquet 2019-09-25 02:42:33 643281
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937346#comment-16937346 ] BALAJI VARADARAJAN commented on HUDI-269: - [~XingXPan]: I cannot understand why we are seeing close to 5K requests for such a setup. Can you list the partition directory and .hoodie folder ? If it is fine with you, can you attach the .hoodie folder as tarball. Anyways, as you have a knob now to control throttling, I guess you are not blocked. We will look more into this when we get some time free from release related work. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937336#comment-16937336 ] Xing Pan commented on HUDI-269: --- and I ran delta streamer like: {code:java} spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer \ --executor-memory 1g \ --executor-cores 1 \ --conf spark.dynamicAllocation.executorIdleTimeout=10s \ --conf spark.dynamicAllocation.maxExecutors=3 \ xxx/hudi-utilities-bundle-0.5.1-SNAPSHOT.jar \ --target-base-path s3a://xxx \ --target-table default.xxx \ --storage-type MERGE_ON_READ \ --props s3a:///kafka-source.properties \ --source-class org.apache.hudi.utilities.sources.AvroKafkaSource \ --source-ordering-field ts \ --schemaprovider-class org.apache.hudi.utilities.schema.SchemaRegistryProvider \ --op UPSERT \ --enable-hive-sync \ --continuous \ --min-sync-interval-seconds 0 \ --hoodie-conf hoodie.embed.timeline.server=true \ --hoodie-conf hoodie.filesystem.view.incr.timeline.sync.enable=true {code} > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937334#comment-16937334 ] Xing Pan commented on HUDI-269: --- [~vbalaji] : yeah, this is just a single test hudi app in my sandbox emr, so I'm pretty sure only this one hudi spark job is writing to this bucket. I just have one partition for this table. and I've test it on two scenarios: # nothing come from kafka at all. # 10 records come in each 5 seconds. and I didn't do any query on this table during delta sync, do you expect request count decrease from query end? I've set sync interval = 0 in some tests, so I think it've ran of several iterations. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937330#comment-16937330 ] BALAJI VARADARAJAN commented on HUDI-269: - [~XingXPan] : Thank you for sharing the S3 metrics Can you confirm if all these requests is for writing to 1 table and no other write happened on that bucket. The incremental timeline sync should see benefits if you are running for several iterations. Wondering if this test was done for only one iteration. Regarding the embedded timeline-server only mode, you should see reductions approximately in the order of (Number of Files updated)/(Number of partitions touched) How many partitions do the dataset have ? If the number of partitions are large, cleaner operations could have produce more directory listing calls when trying to find all partitions. Just for testing this hypothesis, Can you try disabling cleaner for testing by setting hoodie.clean.automatic=false Thanks, Balaji.V > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: image-2019-09-25-08-51-19-686.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937302#comment-16937302 ] Xing Pan commented on HUDI-269: --- [~vbalaji] [~vinoth] I just did some simple test on these configs. basically in my scenario, we will get cdc data from a slow change table and sync cdc from kafka to hudi dataset. and as plot in the graph above, throttle DeltaStreamer sync runs will significantly decrease get request per min. but `embed timeline server` and `incr timeline sync` didn't help reduce requests count too much. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: image-2019-09-25-08-51-19-686.png, request_histogram.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16937296#comment-16937296 ] Xing Pan commented on HUDI-269: --- !image-2019-09-25-08-51-19-686.png! > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Attachments: request_histogram.png > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16936933#comment-16936933 ] BALAJI VARADARAJAN commented on HUDI-269: - Agree. [~XingXPan] : You should try the embedded timeline server and see how much reduction you get. The data partitions will be cached as part of this. Also, There is a feature called "Incremental Timeline syncing". You need to use this config : hoodie.filesystem.view.incr.timeline.sync.enable = true. Please note that this is an experimental feature. You would need to run with this in your test setup and validate before you feel comfortable enabling it. > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16936830#comment-16936830 ] Vinoth Chandar commented on HUDI-269: - I am thinking if [~XingXPan] can try this out and confirm. we can flip the defaults to true? > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16936827#comment-16936827 ] Vinoth Chandar commented on HUDI-269: - [~XingXPan] can you try turning on the embedded timeline service and see if it helps in general reduce calls to s3? In this mode, esp when running delta streamer in continuous, we simply keep the filesystem metadata cached in the driver using rocksdb and every commit simply updates this metadata.. (it atleast helps a lot with hdfs).. So curious about how it plays out for you. [~vbalaji] thoughts? anything else we should turn on? is there a config for incremental syncing? {code:java} private static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server"; private static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "false"; {code} > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Assignee: Xing Pan >Priority: Major > Labels: pull-request-available > Fix For: 0.5.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (HUDI-269) Provide ability to throttle DeltaStreamer sync runs
[ https://issues.apache.org/jira/browse/HUDI-269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16936542#comment-16936542 ] Xing Pan commented on HUDI-269: --- just update pull request > Provide ability to throttle DeltaStreamer sync runs > --- > > Key: HUDI-269 > URL: https://issues.apache.org/jira/browse/HUDI-269 > Project: Apache Hudi (incubating) > Issue Type: Improvement > Components: deltastreamer >Reporter: BALAJI VARADARAJAN >Priority: Major > Fix For: 0.5.0 > > > Copied from [https://github.com/apache/incubator-hudi/issues/922] > In some scenario in our cluster, we may want delta streamer to slow down a > bit. > so it's nice to have a parameter to control the min sync interval of each > sync in continuous mode. > this param is default to 0, so this should not affect current logic. > minor pr: [#921|https://github.com/apache/incubator-hudi/pull/921] > the main reason we want to slow it down is that aws s3 is charged by s3 > get/put/list requests. we don't want to pay for too many requests for a > really slow change table. -- This message was sent by Atlassian Jira (v8.3.4#803005)