Hello Nan, What you described seems to be a broker-side latency spike, not a client (either producer, or consumer, or streams)-side latency spike. There's a number of possible reasons for such spikes: disk flushing (though async, it can also cause the processing thread to halt), GC, page faults (in case the thread needs to access a cold page), etc. It is hard to tell which one is the actual root cause.
For example, you can take a look at this slides (starting at 14), for a concrete example of such an investigation: https://www.slideshare.net/kawamuray/multitenancy-kafka-clusters-for-everyone-at-line My point is that it is not really easy via email discussion and by looking at your experiment code to tell exactly what is the root cause: the community can share with your some past experience and a few quick hinters, but most likely the issue varies case by case and hence can only be fully understandable by yourself. Guozhang On Sat, Aug 25, 2018 at 6:58 PM, Nan Xu <nanxu1...@gmail.com> wrote: > maybe easier to use github. > > https://github.com/angelfox123/kperf > > > On Sat, Aug 25, 2018 at 8:43 PM Nan Xu <nanxu1...@gmail.com> wrote: > > > so I did upgrade to 2.0.0 and still seeing the same result. below is the > > program I am using. I am running everything on a single server. (centos > 7, > > 24 core, 32G ram , 1 broker, 1 zookeeper, single harddrive), I understand > > the single hard drive is less ideal. but still don't expect it can over 3 > > seconds. > > > > > > case 1. > > I create 1 parittions for input and 1 partition for output. message size > > 10K > > producer give parameter (3600, 1000, 2) // 2 message per 1000 micro > > second for 3600 seconds, that translate to 2,000 message/s, I still see > > latency, sometime can reach to 3 seconds. > > > > case 2 > > 50 partitions for input, and 50 partitions for output. message size 10K > > producer give parameter (3600, 1000, 20) // 20 message per 1000 micro > > second for 3600 seconds, that translate to 20,000 message/s,latency not > > only high, and happen more often. > > > > > > Any suggestion is appreciated. target is per partition handle 1,000 -- > > 2,000 message/s and all latency lower than 100ms. > > > > ====build.gradle====== > > plugins { > > id 'application' > > id 'java' > > } > > group 'com.bofa' > > version '1.0-SNAPSHOT' > > sourceCompatibility = 1.8 > > mainClassName="main.StreamApp" > > > > repositories { > > mavenCentral() > > } > > > > dependencies { > > compile group: 'org.apache.kafka', name: 'kafka-clients', version: > > '2.0.0' > > compile group: "org.apache.kafka", name: "kafka-streams", version: > > "2.0.0" > > compile group: 'io.dropwizard.metrics', name: 'metrics-core', > > version:'3.2.6' > > testCompile group: 'junit', name: 'junit', version: '4.12' > > } > > > > ========producer======== > > package main; > > > > import java.util.Properties; > > import java.util.concurrent.atomic.AtomicInteger; > > > > import Util.BusyTimer; > > import org.apache.kafka.clients.producer.KafkaProducer; > > import org.apache.kafka.clients.producer.ProducerConfig; > > import org.apache.kafka.clients.producer.ProducerRecord; > > import org.apache.kafka.common.serialization.Serde; > > import org.apache.kafka.common.serialization.Serdes; > > import org.apache.kafka.common.serialization.StringSerializer; > > > > public class SimpleProducer { > > public static void main(String[] args) { > > final int time =Integer.valueOf(args[0]); > > final long interval = Integer.valueOf(args[1]); > > final int batch =Integer.valueOf(args[2]); > > Properties props = new Properties(); > > props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, > > "localhost:9092"); > > props.put(ProducerConfig.CLIENT_ID_CONFIG, > > "kafka-perf-test-producer"); > > props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, > > StringSerializer.class); > > props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, > > StringSerializer.class); > > > > KafkaProducer<String,String> kafkaProducer = new > > KafkaProducer(props); > > > > StringBuffer buffer = new StringBuffer(); > > for(int i=0; i<10240; i++) { > > buffer.append('a'); > > } > > String value = buffer.toString(); > > > > final long speed = 1000000/interval; > > Runnable task = new Runnable() { > > int sendNum=0; > > @Override > > public void run() { > > > > for(int i=0; i<batch; i++) { > > ProducerRecord<String, String> record = new > > ProducerRecord<>("input", System.nanoTime() + "-" + value); > > kafkaProducer.send(record); > > sendNum++; > > } > > > > if(sendNum % (speed * batch) == 0){ > > System.out.println(System.currentTimeMillis() + " : > " > > + sendNum); > > } > > } > > }; > > > > BusyTimer timer = new BusyTimer(interval,time, task); > > timer.spaceMessageWithInterval(); > > } > > } > > > > > > ============kafka stream============= > > package main; > > > > import java.util.Properties; > > > > import org.apache.kafka.clients.producer.ProducerConfig; > > import org.apache.kafka.common.serialization.Serdes; > > import org.apache.kafka.streams.Consumed; > > import org.apache.kafka.streams.KafkaStreams; > > import org.apache.kafka.streams.StreamsBuilder; > > import org.apache.kafka.streams.StreamsConfig; > > import org.apache.kafka.streams.kstream.KStream; > > import org.apache.kafka.streams.kstream.Produced; > > > > public class StreamApp { > > public static void main(String[] args) { > > final Properties streamsConfiguration = new Properties(); > > streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, > > "simple-stream"); > > streamsConfiguration.put(StreamsConfig.CLIENT_ID_CONFIG, > > "simple_stream_1"); > > streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, > > "localhost:9092"); > > > > streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, > > Serdes.String() > > .getClass().getName()); > > streamsConfiguration.put(StreamsConfig.PRODUCER_PREFIX > > + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 30); > > streamsConfiguration.put(StreamsConfig.PRODUCER_PREFIX > > + ProducerConfig.LINGER_MS_CONFIG,"5"); > > > > streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG,2); > > > > > > StreamsBuilder builder = new StreamsBuilder(); > > final KStream<String, String> inputStream = builder.stream( > > "input", > > Consumed.with( > > new Serdes.StringSerde(), > > new Serdes.StringSerde() > > ) > > ); > > > > inputStream.to( > > "output", > > Produced.with(new Serdes.StringSerde(), new > > Serdes.StringSerde()) > > ); > > > > KafkaStreams streams = new KafkaStreams(builder.build(), > > streamsConfiguration); > > streams.start(); > > } > > } > > > > =============consumer============================ > > package main; > > > > import java.util.Collections; > > import java.util.Properties; > > > > import com.codahale.metrics.Reservoir; > > import com.codahale.metrics.UniformReservoir; > > import org.apache.kafka.clients.consumer.ConsumerConfig; > > import org.apache.kafka.clients.consumer.ConsumerRecord; > > import org.apache.kafka.clients.consumer.ConsumerRecords; > > import org.apache.kafka.clients.consumer.KafkaConsumer; > > import org.apache.kafka.common.serialization.StringDeserializer; > > > > public class SimpleConsumer { > > public static void main(String[] args) { > > int expectedSpeed = args[0]; > > Properties props = new Properties(); > > props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, > > "localhost:9092"); > > props.put(ConsumerConfig.GROUP_ID_CONFIG, > > "kafka-perf-consumer-group"); > > props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, > > StringDeserializer.class.getName()); > > props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); > > props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true); > > > > props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, > > StringDeserializer.class.getName()); > > > > KafkaConsumer consumer = new KafkaConsumer<String, > String>(props); > > consumer.subscribe(Collections.singletonList("output")); > > > > consumer.poll(0); > > int recNum=0; > > > > Reservoir totalRes = new UniformReservoir(); > > > > while (true) { > > ConsumerRecords<String, String> records = consumer.poll(10); > > for(ConsumerRecord<String,String> record : records){ > > long sendTime = Long.valueOf(record.value(). > split("-")[0]); > > long takeTime = System.nanoTime() - sendTime; > > if(recNum> 20000) { > > totalRes.update(takeTime); > > } > > recNum++; > > > > if(recNum % expectedSpeed == 0){ > > System.out.println("==============="+ recNum + > > "============"); > > System.out.println(" mean: " + > > totalRes.getSnapshot().getMean()/1000000); > > System.out.println(" 75%: " + > > totalRes.getSnapshot().get75thPercentile()/1000000); > > System.out.println(" 99%: " + > > totalRes.getSnapshot().get99thPercentile()/1000000); > > System.out.println(" 99.9%: " + > > totalRes.getSnapshot().get999thPercentile()/1000000); > > System.out.println(" Max: " + > > totalRes.getSnapshot().getMax()/1000000); > > > > System.out.println("========================================"); > > totalRes = new UniformReservoir(); > > } > > }; > > } > > } > > } > > > > ==========busy timer===================== > > //idea is space the message at a fixed time.(as thread.sleep, but sleep > is > > less accurate) > > package Util; > > > > import java.util.ArrayList; > > import java.util.concurrent.ExecutorService; > > import java.util.concurrent.Executors; > > import java.util.concurrent.atomic.AtomicInteger; > > import java.util.concurrent.atomic.AtomicLong; > > > > public class BusyTimer { > > long interval; > > long duration; > > ArrayList<Long> pubTime; > > ExecutorService ex = Executors.newSingleThreadExecutor(); > > Runnable task; > > > > > > public BusyTimer(long microInterval, long exDurationInSeconds, > > Runnable task){ > > pubTime = new ArrayList<Long>((int)(exDurationInSeconds * 1000 * > > 1000 / microInterval+1)); > > > > this.interval = microInterval * 1000; > > this.duration = exDurationInSeconds * 1000000000; > > this.task = task; > > > > } > > > > private void busywaitUntil(long nano){ > > while(System.nanoTime() < nano){ > > > > } > > } > > > > public void spaceMessageWithInterval(){ > > int i =0 ; > > long baseTime = System.nanoTime(); > > long doneTime = baseTime + duration; > > while(true) { > > task.run(); > > pubTime.add(System.nanoTime()); > > long targetTime = System.nanoTime() + interval; > > if(System.nanoTime() > doneTime ){ > > break; > > } > > busywaitUntil(targetTime); > > } > > } > > } > > > > > > > > On Fri, Aug 24, 2018 at 3:37 PM Nan Xu <nanxu1...@gmail.com> wrote: > > > >> Looks really promising but after upgrade, still show the same result. I > >> will post the program soon. Maybe you can see where the problem could > be. > >> > >> Nan > >> > >> On Thu, Aug 23, 2018, 7:34 PM Guozhang Wang <wangg...@gmail.com> wrote: > >> > >>> Hello Nan, > >>> > >>> Kafka does not tie up the processing thread to do disk flushing. > However, > >>> since you are on an older version of Kafka I suspect you're bumping > into > >>> some old issues that have been resolved in later versions. e.g. > >>> > >>> https://issues.apache.org/jira/browse/KAFKA-4614 > >>> > >>> I'd suggest you upgrading to latest version (2.0.0) and try again to > see > >>> if > >>> you observe the same pattern. > >>> > >>> Guozhang > >>> > >>> On Thu, Aug 23, 2018 at 3:22 PM, Sudhir Babu Pothineni < > >>> sbpothin...@gmail.com> wrote: > >>> > >>> > I will wait for the expert’s opinion: > >>> > > >>> > Did the Transparent Huge Pages(THP) disabled on the broker machine? > >>> it’s a > >>> > Linux kernel parameter. > >>> > > >>> > -Sudhir > >>> > > >>> > > On Aug 23, 2018, at 4:46 PM, Nan Xu <nanxu1...@gmail.com> wrote: > >>> > > > >>> > > I think I found where the problem is, how to solve and why, still > not > >>> > sure. > >>> > > > >>> > > it related to disk (maybe flushing?). I did a single machine, > single > >>> > node, > >>> > > single topic and single partition setup. producer pub as 2000 > >>> message/s, > >>> > > 10K size message size. and single key. > >>> > > > >>> > > when I save kafka log to the memory based partition, I don't see a > >>> > latency > >>> > > over 100ms. top around 70ms. > >>> > > when I save to a ssd hard drive. I do see latency spike, sometime > >>> over > >>> > 1s. > >>> > > > >>> > > adjust the log.flush.inteval.message / log.flush.intefval.ms has > >>> impact, > >>> > > but only to make thing worse... need suggestion. > >>> > > > >>> > > I think log flushing is totally async and done by OS in the default > >>> > > setting. does kafka has to wait when flushing data to disk? > >>> > > > >>> > > Thanks, > >>> > > Nan > >>> > > > >>> > > > >>> > > > >>> > >> On Wed, Aug 22, 2018 at 11:37 PM Guozhang Wang < > wangg...@gmail.com> > >>> > wrote: > >>> > >> > >>> > >> Given your application code: > >>> > >> > >>> > >> ---------------------------- > >>> > >> > >>> > >> final KStream<String, NodeMutation> localDeltaStream = > >>> builder.stream( > >>> > >> > >>> > >> localDeltaTopic, > >>> > >> > >>> > >> Consumed.with( > >>> > >> > >>> > >> new Serdes.StringSerde(), > >>> > >> > >>> > >> new NodeMutationSerde<>() > >>> > >> > >>> > >> ) > >>> > >> > >>> > >> ); > >>> > >> > >>> > >> KStream<String, NodeState> localHistStream = > >>> > localDeltaStream.mapValues( > >>> > >> > >>> > >> (mutation) -> NodeState > >>> > >> > >>> > >> .newBuilder() > >>> > >> > >>> > >> .setMeta( > >>> > >> > >>> > >> mutation.getMetaMutation().getMeta() > >>> > >> > >>> > >> ) > >>> > >> > >>> > >> .setValue( > >>> > >> > >>> > >> mutation.getValueMutation().getValue() > >>> > >> > >>> > >> ) > >>> > >> > >>> > >> .build() > >>> > >> > >>> > >> ); > >>> > >> > >>> > >> localHistStream.to( > >>> > >> > >>> > >> localHistTopic, > >>> > >> > >>> > >> Produced.with(new Serdes.StringSerde(), new > >>> > NodeStateSerde<>()) > >>> > >> > >>> > >> ); > >>> > >> > >>> > >> ---------------------------- > >>> > >> > >>> > >> which is pure stateless, committing will not touch on an state > >>> > directory at > >>> > >> all. Hence committing only involves committing offsets to Kafka. > >>> > >> > >>> > >> > >>> > >> Guozhang > >>> > >> > >>> > >> > >>> > >>> On Wed, Aug 22, 2018 at 8:11 PM, Nan Xu <nanxu1...@gmail.com> > >>> wrote: > >>> > >>> > >>> > >>> I was suspecting that too, but I also noticed the spike is not > >>> spaced > >>> > >>> around 10s. to further prove it. I put kafka data directory in a > >>> memory > >>> > >>> based directory. it still has such latency spikes. I am going > to > >>> test > >>> > >> it > >>> > >>> on a single broker, single partition env. will report back soon. > >>> > >>> > >>> > >>> On Wed, Aug 22, 2018 at 5:14 PM Guozhang Wang < > wangg...@gmail.com> > >>> > >> wrote: > >>> > >>> > >>> > >>>> Hello Nan, > >>> > >>>> > >>> > >>>> Thanks for the detailed information you shared. When Kafka > >>> Streams is > >>> > >>>> normally running, no rebalances should be triggered unless some > >>> of the > >>> > >>>> instances (in your case, docker containers) have soft failures. > >>> > >>>> > >>> > >>>> I suspect the latency spike is due to the commit intervals: > >>> streams > >>> > >> will > >>> > >>>> try to commit its offset at a regular paces, which may increase > >>> > >> latency. > >>> > >>> It > >>> > >>>> is controlled by the "commit.interval.ms" config value. I saw > >>> that in > >>> > >>> your > >>> > >>>> original email you've set it to 10 * 1000 (10 seconds). Is that > >>> > aligned > >>> > >>>> with the frequency you observe latency spikes? > >>> > >>>> > >>> > >>>> > >>> > >>>> Guozhang > >>> > >>>> > >>> > >>>> > >>> > >>>>> On Sun, Aug 19, 2018 at 10:41 PM, Nan Xu <nanxu1...@gmail.com> > >>> > wrote: > >>> > >>>>> > >>> > >>>>> did more test and and make the test case simple. > >>> > >>>>> all the setup now is a single physical machine. running 3 > docker > >>> > >>>> instance. > >>> > >>>>> a1, a2, a3 > >>> > >>>>> > >>> > >>>>> kafka + zookeeper running on all of those docker containers. > >>> > >>>>> producer running on a1, send a single key, update speed 2000 > >>> > >>> message/s, > >>> > >>>>> each message is 10K size. > >>> > >>>>> 3 consumer(different group) are running. one on each docker. > >>> > >>>>> all topics are pre-created. > >>> > >>>>> in startup, I do see some latency greater than 100ms, which is > >>> fine. > >>> > >>> and > >>> > >>>>> then everything is good. latency is low and consumer don't see > >>> > >> anything > >>> > >>>>> over 100ms for a while. > >>> > >>>>> then I see a few messages have latency over 100ms. then back to > >>> > >> normal, > >>> > >>>>> then happen again..... do seems like gc problem. but I check > the > >>> gc > >>> > >>>> log. I > >>> > >>>>> don't think it can cause over 100ms. (both are G1 collector) > >>> > >>>>> > >>> > >>>>> after the stream stable running( exclude the startup), the > first > >>> > >>> message > >>> > >>>>> over 100ms take 179ms and the gc ( it has a 30ms pause, but > >>> should > >>> > >> not > >>> > >>>>> cause a 179ms end to end). > >>> > >>>>> > >>> > >>>>> FROM APP > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:28:47.118-0500: 406.838: [GC (Allocation Failure) > >>> > >>>>> 3184739K->84018K(5947904K), 0.0093730 secs] > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:28:56.094-0500: 415.814: [GC (Allocation Failure) > >>> > >>>>> 3184690K->84280K(6053888K), 0.0087473 secs] > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:29:05.069-0500: 424.789: [GC (Allocation Failure) > >>> > >>>>> 3301176K->84342K(6061056K), 0.0127339 secs] > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:29:14.234-0500: 433.954: [GC (Allocation Failure) > >>> > >>>>> 3301238K->84624K(6143488K), 0.0140844 secs] > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:29:24.523-0500: 444.243: [GC (Allocation Failure) > >>> > >>>>> 3386000K->89949K(6144000K), 0.0108118 secs] > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> kafka a1 > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:29:14.306-0500: 7982.657: [GC pause (G1 > Evacuation > >>> > >> Pause) > >>> > >>>>> (young), 0.0214200 secs] > >>> > >>>>> > >>> > >>>>> [Parallel Time: 17.2 ms, GC Workers: 8] > >>> > >>>>> > >>> > >>>>> [GC Worker Start (ms): Min: 7982657.5, Avg: 7982666.9, > Max: > >>> > >>>>> 7982673.8, Diff: 16.3] > >>> > >>>>> > >>> > >>>>> [Ext Root Scanning (ms): Min: 0.0, Avg: 0.2, Max: 1.5, > Diff: > >>> > >> 1.5, > >>> > >>>>> Sum: 1.5] > >>> > >>>>> > >>> > >>>>> [Update RS (ms): Min: 0.0, Avg: 1.0, Max: 6.5, Diff: 6.5, > >>> Sum: > >>> > >>> 8.4] > >>> > >>>>> > >>> > >>>>> [Processed Buffers: Min: 0, Avg: 4.6, Max: 13, Diff: > 13, > >>> > >> Sum: > >>> > >>>> 37] > >>> > >>>>> > >>> > >>>>> [Scan RS (ms): Min: 0.0, Avg: 0.9, Max: 2.0, Diff: 2.0, > Sum: > >>> > >> 7.1] > >>> > >>>>> > >>> > >>>>> [Code Root Scanning (ms): Min: 0.0, Avg: 0.0, Max: 0.0, > >>> Diff: > >>> > >>> 0.0, > >>> > >>>>> Sum: 0.0] > >>> > >>>>> > >>> > >>>>> [Object Copy (ms): Min: 0.0, Avg: 4.6, Max: 6.5, Diff: > 6.5, > >>> > >> Sum: > >>> > >>>>> 36.5] > >>> > >>>>> > >>> > >>>>> [Termination (ms): Min: 0.0, Avg: 0.4, Max: 0.9, Diff: > 0.9, > >>> > >> Sum: > >>> > >>>> 2.9] > >>> > >>>>> > >>> > >>>>> [Termination Attempts: Min: 1, Avg: 10.4, Max: 25, > Diff: > >>> 24, > >>> > >>>> Sum: > >>> > >>>>> 83] > >>> > >>>>> > >>> > >>>>> [GC Worker Other (ms): Min: 0.0, Avg: 0.0, Max: 0.0, Diff: > >>> 0.0, > >>> > >>>> Sum: > >>> > >>>>> 0.1] > >>> > >>>>> > >>> > >>>>> [GC Worker Total (ms): Min: 0.2, Avg: 7.1, Max: 16.4, > Diff: > >>> > >> 16.2, > >>> > >>>>> Sum: 56.5] > >>> > >>>>> > >>> > >>>>> [GC Worker End (ms): Min: 7982673.9, Avg: 7982674.0, Max: > >>> > >>>> 7982674.5, > >>> > >>>>> Diff: 0.6] > >>> > >>>>> > >>> > >>>>> [Code Root Fixup: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Code Root Purge: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Clear CT: 1.0 ms] > >>> > >>>>> > >>> > >>>>> [Other: 3.2 ms] > >>> > >>>>> > >>> > >>>>> [Choose CSet: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Ref Proc: 1.9 ms] > >>> > >>>>> > >>> > >>>>> [Ref Enq: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Redirty Cards: 0.8 ms] > >>> > >>>>> > >>> > >>>>> [Humongous Register: 0.1 ms] > >>> > >>>>> > >>> > >>>>> [Humongous Reclaim: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Free CSet: 0.2 ms] > >>> > >>>>> > >>> > >>>>> [Eden: 48.0M(48.0M)->0.0B(48.0M) Survivors: 3072.0K->3072.0K > >>> Heap: > >>> > >>>>> 265.5M(1024.0M)->217.9M(1024.0M)] > >>> > >>>>> > >>> > >>>>> [Times: user=0.05 sys=0.00, real=0.03 secs] > >>> > >>>>> > >>> > >>>>> 2018-08-20T00:29:16.074-0500: 7984.426: [GC pause (G1 > Evacuation > >>> > >> Pause) > >>> > >>>>> (young), 0.0310004 secs] > >>> > >>>>> > >>> > >>>>> [Parallel Time: 24.4 ms, GC Workers: 8] > >>> > >>>>> > >>> > >>>>> [GC Worker Start (ms): Min: 7984426.1, Avg: 7984436.8, > Max: > >>> > >>>>> 7984444.7, Diff: 18.6] > >>> > >>>>> > >>> > >>>>> [Ext Root Scanning (ms): Min: 0.0, Avg: 0.3, Max: 1.9, > Diff: > >>> > >> 1.9, > >>> > >>>>> Sum: 2.0] > >>> > >>>>> > >>> > >>>>> [Update RS (ms): Min: 0.0, Avg: 4.1, Max: 11.8, Diff: > 11.8, > >>> > >> Sum: > >>> > >>>>> 32.9] > >>> > >>>>> > >>> > >>>>> [Processed Buffers: Min: 0, Avg: 5.4, Max: 25, Diff: > 25, > >>> > >> Sum: > >>> > >>>> 43] > >>> > >>>>> > >>> > >>>>> [Scan RS (ms): Min: 0.1, Avg: 3.2, Max: 11.3, Diff: 11.2, > >>> Sum: > >>> > >>>> 25.5] > >>> > >>>>> > >>> > >>>>> [Code Root Scanning (ms): Min: 0.0, Avg: 0.0, Max: 0.0, > >>> Diff: > >>> > >>> 0.0, > >>> > >>>>> Sum: 0.0] > >>> > >>>>> > >>> > >>>>> [Object Copy (ms): Min: 0.0, Avg: 4.1, Max: 6.9, Diff: > 6.9, > >>> > >> Sum: > >>> > >>>>> 32.7] > >>> > >>>>> > >>> > >>>>> [Termination (ms): Min: 0.0, Avg: 0.8, Max: 1.6, Diff: > 1.6, > >>> > >> Sum: > >>> > >>>> 6.8] > >>> > >>>>> > >>> > >>>>> [Termination Attempts: Min: 1, Avg: 5.4, Max: 11, Diff: > >>> 10, > >>> > >>> Sum: > >>> > >>>>> 43] > >>> > >>>>> > >>> > >>>>> [GC Worker Other (ms): Min: 0.0, Avg: 0.0, Max: 0.0, Diff: > >>> 0.0, > >>> > >>>> Sum: > >>> > >>>>> 0.1] > >>> > >>>>> > >>> > >>>>> [GC Worker Total (ms): Min: 4.4, Avg: 12.5, Max: 23.6, > Diff: > >>> > >>> 19.1, > >>> > >>>>> Sum: 100.1] > >>> > >>>>> > >>> > >>>>> [GC Worker End (ms): Min: 7984449.1, Avg: 7984449.3, Max: > >>> > >>>> 7984449.9, > >>> > >>>>> Diff: 0.8] > >>> > >>>>> > >>> > >>>>> [Code Root Fixup: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Code Root Purge: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Clear CT: 1.1 ms] > >>> > >>>>> > >>> > >>>>> [Other: 5.5 ms] > >>> > >>>>> > >>> > >>>>> [Choose CSet: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Ref Proc: 2.2 ms] > >>> > >>>>> > >>> > >>>>> [Ref Enq: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Redirty Cards: 2.8 ms] > >>> > >>>>> > >>> > >>>>> [Humongous Register: 0.1 ms] > >>> > >>>>> > >>> > >>>>> [Humongous Reclaim: 0.0 ms] > >>> > >>>>> > >>> > >>>>> [Free CSet: 0.1 ms] > >>> > >>>>> > >>> > >>>>> [Eden: 48.0M(48.0M)->0.0B(48.0M) Survivors: 3072.0K->3072.0K > >>> Heap: > >>> > >>>>> 265.9M(1024.0M)->218.4M(1024.0M)] > >>> > >>>>> > >>> > >>>>> [Times: user=0.05 sys=0.00, real=0.03 secs] > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> so when kafka stream running, is there any trying to rebalance? > >>> > >> either > >>> > >>>>> broker rebalance or client rebalance? > >>> > >>>>> any kind of test to see what cause the trouble? > >>> > >>>>> > >>> > >>>>> Thanks, > >>> > >>>>> Nan > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> > >>> > >>>>> On Sun, Aug 19, 2018 at 10:34 PM Guozhang Wang < > >>> wangg...@gmail.com> > >>> > >>>> wrote: > >>> > >>>>> > >>> > >>>>>> Okay, so you're measuring end-to-end time from producer -> > >>> broker > >>> > >> -> > >>> > >>>>>> streams' consumer client, there are multiple phases that can > >>> > >>> contribute > >>> > >>>>> to > >>> > >>>>>> the 100ms latency, and I cannot tell if stream's consumer > phase > >>> is > >>> > >>> the > >>> > >>>>>> major contributor. For example, if the topic was not created > >>> > >> before, > >>> > >>>> then > >>> > >>>>>> when the broker first received a produce request it may need > to > >>> > >>> create > >>> > >>>>> the > >>> > >>>>>> topic, which involves multiple steps including writes to ZK > >>> which > >>> > >>> could > >>> > >>>>>> take time. > >>> > >>>>>> > >>> > >>>>>> There are some confusions from your description: you mentioned > >>> > >> "Kafka > >>> > >>>>>> cluster is already up and running", but I think you are > >>> referring > >>> > >> to > >>> > >>>>> "Kafka > >>> > >>>>>> Streams application instances are already up and running", > >>> right? > >>> > >>> Since > >>> > >>>>>> only the latter has rebalance process, while the Kafak brokers > >>> do > >>> > >> not > >>> > >>>>>> really have "rebalances" except balancing load by migrating > >>> > >>> partitions. > >>> > >>>>>> > >>> > >>>>>> Guozhang > >>> > >>>>>> > >>> > >>>>>> > >>> > >>>>>> > >>> > >>>>>> On Sun, Aug 19, 2018 at 7:47 PM, Nan Xu <nanxu1...@gmail.com> > >>> > >> wrote: > >>> > >>>>>> > >>> > >>>>>>> right, so my kafka cluster is already up and running for a > >>> while, > >>> > >>>> and I > >>> > >>>>>> can > >>> > >>>>>>> see from the log all broker instance already change from > >>> > >> rebalance > >>> > >>> to > >>> > >>>>>>> running. > >>> > >>>>>>> > >>> > >>>>>>> I did a another test. > >>> > >>>>>>> from producer, right before the message get send to the > >>> broker, I > >>> > >>>> put a > >>> > >>>>>>> timestamp in the message. and from the consumer side which is > >>> > >> after > >>> > >>>>>> stream > >>> > >>>>>>> processing, I compare this timestamp with current time. I can > >>> see > >>> > >>>> some > >>> > >>>>>>> message processing time is above 100ms on some real powerful > >>> > >>>> hardware. > >>> > >>>>>> and > >>> > >>>>>>> from my application gc, all the gc time is below 1ms, kafka > gc > >>> > >> only > >>> > >>>>>> happen > >>> > >>>>>>> once and below 1ms too. > >>> > >>>>>>> > >>> > >>>>>>> very puzzled. is there any communication to zookeeper, if not > >>> get > >>> > >>>>>> response, > >>> > >>>>>>> will cause the broker to pause? I don't think that's the case > >>> but > >>> > >>> at > >>> > >>>>> this > >>> > >>>>>>> time don't know what else can be suspected. > >>> > >>>>>>> > >>> > >>>>>>> Nan > >>> > >>>>>>> > >>> > >>>>>>> On Sun, Aug 19, 2018 at 1:08 PM Guozhang Wang < > >>> > >> wangg...@gmail.com> > >>> > >>>>>> wrote: > >>> > >>>>>>> > >>> > >>>>>>>> Hello Nan, > >>> > >>>>>>>> > >>> > >>>>>>>> Note that Streams may need some time to rebalance and assign > >>> > >>> tasks > >>> > >>>>> even > >>> > >>>>>>> if > >>> > >>>>>>>> you only starts with one instance. > >>> > >>>>>>>> > >>> > >>>>>>>> I'd suggest you register your state listener in Kafka > Streams > >>> > >> via > >>> > >>>>>>>> KafkaStreams#setStateListener, and your customized > >>> > >> StateListener > >>> > >>>>> should > >>> > >>>>>>>> record when the state transits from REBALANCING to RUNNING > >>> > >> since > >>> > >>>> only > >>> > >>>>>>> after > >>> > >>>>>>>> that the streams client will start to process the first > >>> record. > >>> > >>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>>>> Guozhang > >>> > >>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>>>> On Sat, Aug 18, 2018 at 8:52 PM, Nan Xu < > nanxu1...@gmail.com> > >>> > >>>> wrote: > >>> > >>>>>>>> > >>> > >>>>>>>>> thanks, which JMX properties indicate "processing latency > >>> > >>>>> spikes" / > >>> > >>>>>>>>> "throughput" > >>> > >>>>>>>>> > >>> > >>>>>>>>> > >>> > >>>>>>>>> On Sat, Aug 18, 2018 at 5:45 PM Matthias J. Sax < > >>> > >>>>>> matth...@confluent.io > >>> > >>>>>>>> > >>> > >>>>>>>>> wrote: > >>> > >>>>>>>>> > >>> > >>>>>>>>>> I cannot spot any obvious reasons. > >>> > >>>>>>>>>> > >>> > >>>>>>>>>> As you consume from the result topic for verification, we > >>> > >>>> should > >>> > >>>>>>> verify > >>> > >>>>>>>>>> that the latency spikes original on write and not on read: > >>> > >>> you > >>> > >>>>>> might > >>> > >>>>>>>>>> want to have a look into Kafka Streams JMX metric to see > if > >>> > >>>>>>> processing > >>> > >>>>>>>>>> latency spikes or throughput drops. > >>> > >>>>>>>>>> > >>> > >>>>>>>>>> Also watch for GC pauses in the JVM. > >>> > >>>>>>>>>> > >>> > >>>>>>>>>> Hope this helps. > >>> > >>>>>>>>>> > >>> > >>>>>>>>>> > >>> > >>>>>>>>>> -Matthias > >>> > >>>>>>>>>> > >>> > >>>>>>>>>>> On 8/17/18 12:13 PM, Nan Xu wrote: > >>> > >>>>>>>>>>> btw, I am using version 0.10.2.0 > >>> > >>>>>>>>>>> > >>> > >>>>>>>>>>> On Fri, Aug 17, 2018 at 2:04 PM Nan Xu < > >>> > >>> nanxu1...@gmail.com> > >>> > >>>>>>> wrote: > >>> > >>>>>>>>>>> > >>> > >>>>>>>>>>>> I am working on a kafka stream app, and see huge latency > >>> > >>>>>> variance, > >>> > >>>>>>>>>>>> wondering what can cause this? > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> the processing is very simple and don't have state, > >>> > >>>> linger.ms > >>> > >>>>>>>> already > >>> > >>>>>>>>>>>> change to 5ms. the message size is around 10K byes and > >>> > >>>>> published > >>> > >>>>>>> as > >>> > >>>>>>>>> 2000 > >>> > >>>>>>>>>>>> messages/s, network is 10G. using a regular consumer > >>> > >>> watch > >>> > >>>>> the > >>> > >>>>>>>>>>>> localHistTopic topic and just every 2000 message print > >>> > >>> out > >>> > >>>> a > >>> > >>>>>>>> counter, > >>> > >>>>>>>>>> it > >>> > >>>>>>>>>>>> usually every second I get a count 2000 as the publish > >>> > >>>> speed, > >>> > >>>>>> but > >>> > >>>>>>>>>> sometime > >>> > >>>>>>>>>>>> I see it stall for 3 or more seconds and then print out > >>> > >> a > >>> > >>>> few > >>> > >>>>>>> count. > >>> > >>>>>>>>>> like > >>> > >>>>>>>>>>>> cpu is paused during that time or message being > >>> > >>> cache/batch > >>> > >>>>> then > >>> > >>>>>>>>>> processed. > >>> > >>>>>>>>>>>> any suggestion? > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> final Properties streamsConfiguration = new > >>> > >>> Properties(); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>> streamsConfiguration.put(StreamsConfig.APPLICATION_ID_ > CONFIG, > >>> > >>>>>>>>>>>> applicationId); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >> streamsConfiguration.put(StreamsConfig.CLIENT_ID_ > >>> > >>>>> CONFIG, > >>> > >>>>>>>>>> clientId); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >> streamsConfiguration.put(StreamsConfig.BOOTSTRAP_ > >>> > >>>>>>>>> SERVERS_CONFIG, > >>> > >>>>>>>>>>>> bootstrapServers); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_ > >>> > >>>>>>>>> SERDE_CLASS_CONFIG, > >>> > >>>>>>>>>>>> Serdes.String() > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> .getClass().getName()); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>> streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_ > >>> > >>>>> MS_CONFIG, > >>> > >>>>>>>>>>>> 10 * 1000); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> // > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>> > >>> > >>>> > >>> > >> streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_ > >>> > BUFFERING_CONFIG, > >>> > >>>>>>>>> 0); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.BATCH_SIZE_CONFIG,163840); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.BUFFER_MEMORY_CONFIG, > >>> > >>>>> 335544320); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.MAX_IN_FLIGHT_ > >>> > >>>>>>> REQUESTS_PER_CONNECTION, > >>> > >>>>>>>>> 30); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.LINGER_MS_CONFIG,"5"); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.consumerPrefix( > >>> > >>>>>>>>>>>> ConsumerConfig.MAX_PARTITION_ > >>> > >>>>> FETCH_BYTES_CONFIG),20 > >>> > >>>>>> * > >>> > >>>>>>>>> 1024 * > >>> > >>>>>>>>>>>> 1024);MS_CONFIG, 10 * 1000); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> // > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>> > >>> > >>>> > >>> > >> streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_ > >>> > BUFFERING_CONFIG, > >>> > >>>>>>>>> 0); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.BATCH_SIZE_CONFIG,163840); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.BUFFER_MEMORY_CONFIG, > >>> > >>>>> 335544320); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.MAX_IN_FLIGHT_ > >>> > >>>>>>> REQUESTS_PER_CONNECTION, > >>> > >>>>>>>>> 30); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.PRODUCER_PREFIX > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> + ProducerConfig.LINGER_MS_CONFIG,"5"); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streamsConfiguration.put( > >>> > >>>>> StreamsConfig.consumerPrefix( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ConsumerConfig. MAX_PARTITION_FETCH_BYTES_ > >>> > >>> CONFIG > >>> > >>>>> , > >>> > >>>>>>> 20 * > >>> > >>>>>>>>>> 1024 * > >>> > >>>>>>>>>>>> 1024); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> final StreamsBuilder builder = new StreamsBuilder(); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> final KStream<String, NodeMutation> localDeltaStream = > >>> > >>>>>>>>> builder.stream( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> localDeltaTopic, > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> Consumed.with( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> new Serdes.StringSerde(), > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> new NodeMutationSerde<>() > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ) > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> KStream<String, NodeState> localHistStream = > >>> > >>>>>>>>>> localDeltaStream.mapValues( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> (mutation) -> NodeState > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> .newBuilder() > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> .setMeta( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> mutation.getMetaMutation().getMeta() > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ) > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> .setValue( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> mutation.getValueMutation(). > >>> > >>> getValue() > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ) > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> .build() > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> localHistStream.to( > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> localHistTopic, > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> Produced.with(new Serdes.StringSerde(), new > >>> > >>>>>>>>>> NodeStateSerde<>()) > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> ); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streams = new KafkaStreams(builder.build(), > >>> > >>>>>>> streamsConfiguration); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streams.cleanUp(); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> streams.start(); > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>>> > >>> > >>>>>>>>>>> > >>> > >>>>>>>>>> > >>> > >>>>>>>>>> > >>> > >>>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>>>> > >>> > >>>>>>>> -- > >>> > >>>>>>>> -- Guozhang > >>> > >>>>>>>> > >>> > >>>>>>> > >>> > >>>>>> > >>> > >>>>>> > >>> > >>>>>> > >>> > >>>>>> -- > >>> > >>>>>> -- Guozhang > >>> > >>>>>> > >>> > >>>>> > >>> > >>>> > >>> > >>>> > >>> > >>>> > >>> > >>>> -- > >>> > >>>> -- Guozhang > >>> > >>>> > >>> > >>> > >>> > >> > >>> > >> > >>> > >> > >>> > >> -- > >>> > >> -- Guozhang > >>> > >> > >>> > > >>> > >>> > >>> > >>> -- > >>> -- Guozhang > >>> > >> > -- -- Guozhang