You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "Hong Shen (Jira)" <ji...@apache.org> on 2020/06/09 09:00:03 UTC

[jira] [Updated] (HUDI-1004) Failed to send metrics in HoodieDeltaStreamer

     [ https://issues.apache.org/jira/browse/HUDI-1004?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hong Shen updated HUDI-1004:
----------------------------
    Summary: Failed to send metrics in HoodieDeltaStreamer  (was: Failed to send metrics when HoodieDeltaStreamer)

> Failed to send metrics in HoodieDeltaStreamer
> ---------------------------------------------
>
>                 Key: HUDI-1004
>                 URL: https://issues.apache.org/jira/browse/HUDI-1004
>             Project: Apache Hudi
>          Issue Type: Sub-task
>          Components: Writer Core
>            Reporter: Hong Shen
>            Priority: Major
>
> When I run HoodieDeltaStreamer with hoodie.metrics.on=true and hoodie.metrics.reporter.type=GRAPHITE, The first batch run success, and metrics have been sent to GRAPHITE. But the second batch failed to send metrics with the following error.
> {code}
> 30820 [pool-17-thread-1] INFO  org.apache.hudi.metrics.HoodieMetrics  - Sending finalize write metrics (duration=9, numFilesFinalized=1)
> 30820 [pool-17-thread-1] ERROR org.apache.hudi.metrics.Metrics  - Failed to send metrics: 
> java.lang.IllegalArgumentException: A metric named test123.finalize.duration already exists
> 	at com.codahale.metrics.MetricRegistry.register(MetricRegistry.java:151)
> 	at org.apache.hudi.metrics.Metrics.registerGauge(Metrics.java:84)
> 	at org.apache.hudi.metrics.HoodieMetrics.updateFinalizeWriteMetrics(HoodieMetrics.java:177)
> 	at org.apache.hudi.client.AbstractHoodieWriteClient.lambda$finalizeWrite$0(AbstractHoodieWriteClient.java:170)
> 	at org.apache.hudi.common.util.Option.ifPresent(Option.java:96)
> 	at org.apache.hudi.client.AbstractHoodieWriteClient.finalizeWrite(AbstractHoodieWriteClient.java:168)
> 	at org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:111)
> 	at org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:94)
> 	at org.apache.hudi.utilities.deltastreamer.DeltaSync.writeToSink(DeltaSync.java:399)
> 	at org.apache.hudi.utilities.deltastreamer.DeltaSync.syncOnce(DeltaSync.java:232)
> 	at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.lambda$startService$0(HoodieDeltaStreamer.java:422)
> 	at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 	at java.lang.Thread.run(Thread.java:748)
> {code}
> Here is the hoodie properties.
> {code}
> hoodie.metrics.on=true
> hoodie.metrics.reporter.type=GRAPHITE
> hoodie.metrics.graphite.host=127.0.0.1
> hoodie.metrics.graphite.port=2003
> hoodie.metrics.graphite.metric.prefix=testHudiMetrics
> {code}
> Currently HoodieMetrics will call Metrics.registerGauge to update metrics, but if the same metricName has been registered, it will throw "Failed to send metrics" in the second time. 
> {code}
> public static void registerGauge(String metricName, final long value) {
>     try {
>       MetricRegistry registry = Metrics.getInstance().getRegistry();
>       registry.<Gauge<Long>>register(metricName, () -> value);
>     } catch (Exception e) {
>       // Here we catch all exception, so the major upsert pipeline will not be affected if the
>       // metrics system
>       // has some issues.
>       LOG.error("Failed to send metrics: ", e);
>     }
>   }
> {code}
> I'm not sure if it's a bug or my configuration error. If it's a bug, I'll try my best to fix it



--
This message was sent by Atlassian Jira
(v8.3.4#803005)