Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id E13F0200498 for ; Tue, 29 Aug 2017 21:57:17 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id D8BB3166A23; Tue, 29 Aug 2017 19:57:17 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 337081669AE for ; Tue, 29 Aug 2017 21:57:17 +0200 (CEST) Received: (qmail 64297 invoked by uid 500); 29 Aug 2017 19:57:14 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list user@flink.apache.org Received: (qmail 64288 invoked by uid 99); 29 Aug 2017 19:57:14 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Aug 2017 19:57:14 +0000 Received: from [192.168.2.104] (p57A8B382.dip0.t-ipconnect.de [87.168.179.130]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id ACD551A0029 for ; Tue, 29 Aug 2017 19:57:12 +0000 (UTC) Subject: Re: metrics for Flink sinks To: user@flink.apache.org References: From: Chesnay Schepler Message-ID: Date: Tue, 29 Aug 2017 21:57:09 +0200 User-Agent: Mozilla/5.0 (Windows NT 10.0; WOW64; rv:52.0) Gecko/20100101 Thunderbird/52.2.1 MIME-Version: 1.0 In-Reply-To: Content-Type: multipart/alternative; boundary="------------93805F126D638414FC1A397A" Content-Language: en-US archived-at: Tue, 29 Aug 2017 19:57:18 -0000 This is a multi-part message in MIME format. --------------93805F126D638414FC1A397A Content-Type: text/plain; charset=utf-8; format=flowed Content-Transfer-Encoding: 7bit Hello, 1. Because no one found time to fix it. In contrast to the remaining byte/record metrics, input metrics for sources / output metrics for sinks have to be implemented for every single implementation with their respective semantics. In contrast, the output metrics are gathered in the intersection between operators, independent of the actual operator implementation. Furthermore, this requires system metrics (i.e. metrics that Flink itself creates) to be exposed (and be mutable!) to user-defined functions, which is something i /generally /wanted to avoid, but it appears to be a big enough pain point to make an exception here. 2. Due to the above it is currently not possible without modifications of the code to know how many reads/writes were made. 3. Do you mean aggregated metrics? The web UI allows the aggregation of record/byte metrics on the task level. Beyond that we defer aggregation to actual time-series databases that specialize in these things. On 28.08.2017 19:08, Martin Eden wrote: > Hi all, > > Just 3 quick questions both related to Flink metrics, especially > around sinks: > > 1. In the Flink UI Sources always have 0 input records / bytes and > Sinks always have 0 output records / bytes? Why is it like that? > > 2. What is the best practice for instrumenting off the shelf Flink sinks? > > Currently the only metrics available are num records/bytes in and out > at the operator and task scope. For the task scope there are extra > buffer metrics. However the output metrics are always zero (see > question 1). How can one know the actual number of successful writes > done by an off the shelf Flink sink? Or the latency of the write > operation? > > 3. Is it possible to configure Flink to get global job metrics for all > subtasks of an operator? Or are there any best practices around that? > > Thanks, > M --------------93805F126D638414FC1A397A Content-Type: text/html; charset=utf-8 Content-Transfer-Encoding: 7bit
Hello,

1. Because no one found time to fix it. In contrast to the remaining byte/record metrics, input metrics for sources / output metrics for sinks have to be implemented for every single implementation with their respective semantics. In contrast, the output metrics are gathered in the intersection between operators, independent of the actual operator implementation. Furthermore, this requires system metrics (i.e. metrics that Flink itself creates) to be exposed (and be mutable!) to user-defined functions, which is something i generally wanted to avoid, but it appears to be a big enough pain point to make an exception here.

2. Due to the above it is currently not possible without modifications of the code to know how many reads/writes were made.

3. Do you mean aggregated metrics? The web UI allows the aggregation of record/byte metrics on the task level. Beyond that we defer aggregation to actual time-series databases that specialize in these things.

On 28.08.2017 19:08, Martin Eden wrote:
Hi all,

Just 3 quick questions both related to Flink metrics, especially around sinks:

1. In the Flink UI Sources always have 0 input records / bytes and Sinks always have 0 output records / bytes? Why is it like that?

2. What is the best practice for instrumenting off the shelf Flink sinks?

Currently the only metrics available are num records/bytes in and out at the operator and task scope. For the task scope there are extra buffer metrics. However the output metrics are always zero (see question 1). How can one know the actual number of successful writes done by an off the shelf Flink sink? Or the latency of the write operation?

3. Is it possible to configure Flink to get global job metrics for all subtasks of an operator? Or are there any best practices around that?

Thanks,
M


--------------93805F126D638414FC1A397A--