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

Onur Karaman resolved KAFKA-4747.
---------------------------------
    Resolution: Won't Fix

[~junrao] pointed out that the distinction between tim-in-poll and 
time-in-application can be effectively computed as 1 - (io-ratio) - 
(io-wait-ratio). If this value is close to 1, then time is mostly being spent 
on the application-side. Otherwise if this value is close to 0, then time is 
mostly being spent on the client-side.

Here's a simple experiment I ran to verify:
{code}
/**
 * Licensed to the Apache Software Foundation (ASF) under one or more 
contributor license agreements. See the NOTICE
 * file distributed with this work for additional information regarding 
copyright ownership. The ASF licenses this file
 * to You under the Apache License, Version 2.0 (the "License"); you may not 
use this file except in compliance with the
 * License. You may obtain a copy of the License at
 *
 * http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software 
distributed under the License is distributed on
 * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either 
express or implied. See the License for the
 * specific language governing permissions and limitations under the License.
 */
package org.apache.kafka.clients.consumer;

import org.apache.kafka.common.TopicPartition;

import java.util.Collections;
import java.util.Properties;

public class SlowKafkaConsumer {
    public static void main(String[] args) throws InterruptedException {
        long pollTimeout = Long.valueOf(args[0]);
        long sleepDuration = Long.valueOf(args[1]);
        Properties props = new Properties();
        props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9090");
        props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "onur");
        props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
"org.apache.kafka.common.serialization.ByteArrayDeserializer");
        props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
"org.apache.kafka.common.serialization.ByteArrayDeserializer");
        KafkaConsumer<byte[], byte[]> kafkaConsumer = new 
KafkaConsumer<>(props);
        kafkaConsumer.assign(Collections.singletonList(new TopicPartition("t", 
0)));
        kafkaConsumer.seekToBeginning(Collections.singletonList(new 
TopicPartition("t", 0)));
        while (true) {
            kafkaConsumer.poll(pollTimeout);
            Thread.sleep(sleepDuration);
        }
    }
}
{code}

{code}
no data
=======
> ./bin/kafka-run-class.sh org.apache.kafka.clients.consumer.SlowKafkaConsumer 
> 2000 0
io-ratio ~ 0
io-wait-ratio ~ 0.99

> ./bin/kafka-run-class.sh org.apache.kafka.clients.consumer.SlowKafkaConsumer 
> 2000 10000
io-ratio ~ 0
io-wait-ratio ~ [0.1, 0.2]

> ./bin/kafka-run-class.sh org.apache.kafka.clients.consumer.SlowKafkaConsumer 
> 2000 20000
io-ratio ~ 0
io-wait-ratio ~ [0.05, 0.12]

with data
=========
> ./bin/kafka-producer-perf-test.sh --producer-props 
> bootstrap.servers=localhost:9090 --topic t --throughput -1 --num-records 
> 100000000 --record-size 1000

> ./bin/kafka-run-class.sh org.apache.kafka.clients.consumer.SlowKafkaConsumer 
> 2000 0
io-ratio ~ 0.06
io-wait-ratio ~ 0.8

> ./bin/kafka-run-class.sh org.apache.kafka.clients.consumer.SlowKafkaConsumer 
> 2000 10000
io-ratio ~ 0
io-wait-ratio ~ [0.05, 0.1]

> ./bin/kafka-run-class.sh org.apache.kafka.clients.consumer.SlowKafkaConsumer 
> 2000 20000
io-ratio ~ 0
io-wait-ratio ~ [0, 0.03]
{code}

> add metrics for KafkaConsumer.poll
> ----------------------------------
>
>                 Key: KAFKA-4747
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4747
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Onur Karaman
>            Assignee: Onur Karaman
>
> KafkaConsumer heavily depends on KafkaConsumer.poll yet we don't have metrics 
> directly associated with it.
> We probably want to add two metrics:
> 1. time spent in KafkaConsumer.poll
> 2. time since last KafkaConsumer.poll (measured as now - endTimeOfLastPoll)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to