[ 
https://issues.apache.org/jira/browse/FLINK-4702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15535142#comment-15535142
 ] 

ASF GitHub Bot commented on FLINK-4702:
---------------------------------------

Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2574#discussion_r81277067
  
    --- Diff: 
flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java
 ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.flink.streaming.connectors.kafka;
    +
    +import org.apache.flink.core.testutils.MultiShotLatch;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import 
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher;
    +import 
org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +import 
org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
    +import 
org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecords;
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +
    +import org.mockito.Mockito;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.stubbing.Answer;
    +import org.powermock.core.classloader.annotations.PrepareForTest;
    +import org.powermock.modules.junit4.PowerMockRunner;
    +
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Properties;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
    +
    +import static org.mockito.Mockito.any;
    +import static org.mockito.Mockito.anyLong;
    +import static org.powermock.api.mockito.PowerMockito.doAnswer;
    +import static org.powermock.api.mockito.PowerMockito.mock;
    +import static org.powermock.api.mockito.PowerMockito.when;
    +import static org.powermock.api.mockito.PowerMockito.whenNew;
    +
    +/**
    + * Unit tests for the {@link Kafka09Fetcher}.
    + */
    +@RunWith(PowerMockRunner.class)
    +@PrepareForTest(Kafka09Fetcher.class)
    +public class Kafka09FetcherTest {
    +
    +   @Test
    +   public void testCommitDoesNotBlock() throws Exception {
    +
    +           // test data
    +           final KafkaTopicPartition testPartition = new 
KafkaTopicPartition("test", 42);
    +           final Map<KafkaTopicPartition, Long> testCommitData = new 
HashMap<>();
    +           testCommitData.put(testPartition, 11L);
    +
    +           // to synchronize when the consumer is in its blocking method
    +           final OneShotLatch sync = new OneShotLatch();
    +
    +           // ----- the mock consumer with blocking poll calls ----
    +           final MultiShotLatch blockerLatch = new MultiShotLatch();
    +           
    +           KafkaConsumer<?, ?> mockConsumer = mock(KafkaConsumer.class);
    +           when(mockConsumer.poll(anyLong())).thenAnswer(new 
Answer<ConsumerRecords<?, ?>>() {
    +                   
    +                   @Override
    +                   public ConsumerRecords<?, ?> answer(InvocationOnMock 
invocation) throws InterruptedException {
    +                           sync.trigger();
    +                           blockerLatch.await();
    +                           return ConsumerRecords.empty();
    +                   }
    +           });
    +
    +           doAnswer(new Answer<Void>() {
    +                   @Override
    +                   public Void answer(InvocationOnMock invocation) {
    +                           blockerLatch.trigger();
    +                           return null;
    +                   }
    +           }).when(mockConsumer).wakeup();
    +
    +           // make sure the fetcher creates the mock consumer
    +           
whenNew(KafkaConsumer.class).withAnyArguments().thenReturn(mockConsumer);
    +
    +           // ----- create the test fetcher -----
    +
    +           @SuppressWarnings("unchecked")
    +           SourceContext<String> sourceContext = mock(SourceContext.class);
    +           List<KafkaTopicPartition> topics = 
Collections.singletonList(new KafkaTopicPartition("test", 42));
    +           KeyedDeserializationSchema<String> schema = new 
KeyedDeserializationSchemaWrapper<>(new SimpleStringSchema());
    +           StreamingRuntimeContext context = 
mock(StreamingRuntimeContext.class);
    +           
    +           final Kafka09Fetcher<String> fetcher = new Kafka09Fetcher<>(
    +                           sourceContext, topics, null, null, context, 
schema, new Properties(), 0L, false);
    +
    +           // ----- run the fetcher -----
    +
    +           final AtomicReference<Throwable> error = new 
AtomicReference<>();
    +           final Thread fetcherRunner = new Thread("fetcher runner") {
    +
    +                   @Override
    +                   public void run() {
    +                           try {
    +                                   fetcher.runFetchLoop();
    +                           } catch (Throwable t) {
    +                                   error.set(t);
    +                           }
    +                   }
    +           };
    +           fetcherRunner.start();
    +
    +           // wait until the fetcher has reached the method of interest
    +           sync.await();
    +
    +           // ----- trigger the offset commit -----
    +           
    +           final AtomicReference<Throwable> commitError = new 
AtomicReference<>();
    +           final Thread committer = new Thread("committer runner") {
    +                   @Override
    +                   public void run() {
    +                           try {
    +                                   
fetcher.commitSpecificOffsetsToKafka(testCommitData);
    +                           } catch (Throwable t) {
    +                                   commitError.set(t);
    +                           }
    +                   }
    +           };
    +           committer.start();
    +
    +           // ----- ensure that the committer finishes in time  -----
    +           committer.join(30000);
    +           assertFalse("The committer did not finish in time", 
committer.isAlive());
    +
    +           // ----- test done, wait till the fetcher is done for a clean 
shutdown -----
    +           fetcher.cancel();
    +           fetcherRunner.join();
    +
    +           // check that there were no errors in the fetcher
    +           final Throwable caughtError = error.get();
    +           if (caughtError != null) {
    +                   throw new Exception("Exception in the fetcher", 
caughtError);
    +           }
    --- End diff --
    
    Might as well also check `commitError`?


> Kafka consumer must commit offsets asynchronously
> -------------------------------------------------
>
>                 Key: FLINK-4702
>                 URL: https://issues.apache.org/jira/browse/FLINK-4702
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.1.2
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>            Priority: Blocker
>             Fix For: 1.2.0, 1.1.3
>
>
> The offset commit calls to Kafka may occasionally take very long.
> In that case, the {{notifyCheckpointComplete()}} method blocks for long and 
> the KafkaConsumer cannot make progress and cannot perform checkpoints.
> Kafka 0.9+ have methods to commit asynchronously.
> We should use those and make sure no more than one commit is concurrently in 
> progress, to that commit requests do not pile up.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to