Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.store.kafka;

import java.time.Duration;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
Expand All @@ -26,6 +27,7 @@
import java.util.function.Function;
import java.util.stream.Collectors;

import org.apache.drill.exec.ExecConstants;
import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
import org.apache.commons.lang3.StringUtils;
import org.apache.drill.common.exceptions.ExecutionSetupException;
Expand All @@ -44,6 +46,7 @@
import org.apache.drill.exec.store.schedule.CompleteWork;
import org.apache.drill.exec.store.schedule.EndpointByteMap;
import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
Expand Down Expand Up @@ -176,14 +179,13 @@ private void init() {
.message("Table '%s' does not exist", topicName)
.build(logger);
}

kafkaConsumer.subscribe(Collections.singletonList(topicName));
// based on KafkaConsumer JavaDoc, seekToBeginning/seekToEnd functions
// evaluates lazily, seeking to the first/last offset in all partitions only
// when poll(long) or
// position(TopicPartition) are called
kafkaConsumer.poll(0);
Set<TopicPartition> assignments = kafkaConsumer.assignment();
kafkaConsumer.poll(Duration.ofSeconds(5));
Set<TopicPartition> assignments = waitForConsumerAssignment(kafkaConsumer);
topicPartitions = kafkaConsumer.partitionsFor(topicName);

// fetch start offsets for each topicPartition
Expand Down Expand Up @@ -227,6 +229,34 @@ private void init() {
}
}


/** Workaround for Kafka > 2.0 version due to KIP-505.
* It can be replaced with Kafka implementation once it will be introduced.
* @param consumer Kafka consumer whom need to get assignments
* @return
* @throws InterruptedException
*/
private Set<TopicPartition> waitForConsumerAssignment(Consumer consumer) throws InterruptedException {
Set<TopicPartition> assignments = consumer.assignment();

long waitingForAssigmentTimeout = kafkaStoragePlugin.getContext().getOptionManager().getLong(ExecConstants.KAFKA_POLL_TIMEOUT);
long timeout = 0;

while (assignments.isEmpty() && timeout < waitingForAssigmentTimeout) {
Thread.sleep(500);
timeout += 500;
assignments = consumer.assignment();
}

if (timeout >= waitingForAssigmentTimeout) {
throw UserException.dataReadError()
.message("Consumer assignment wasn't completed within the timeout %s", waitingForAssigmentTimeout)
.build(logger);
}

return assignments;
}

@Override
public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
assignments = AssignmentCreator.getMappings(incomingEndpoints, Lists.newArrayList(partitionWorkMap.values()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.store.kafka;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
Expand Down Expand Up @@ -79,7 +80,7 @@ public boolean hasNext() {
ConsumerRecords<byte[], byte[]> consumerRecords;
Stopwatch stopwatch = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
try {
consumerRecords = kafkaConsumer.poll(kafkaPollTimeOut);
consumerRecords = kafkaConsumer.poll(Duration.ofMillis(kafkaPollTimeOut));
} catch (KafkaException ke) {
throw UserException.dataReadError(ke).message(ke.getMessage()).build(logger);
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.junit.experimental.categories.Category;
import org.junit.runners.MethodSorters;

import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
Expand Down Expand Up @@ -120,7 +121,7 @@ public void testInformationSchema() throws Exception {
queryBuilder().sql(query).run();
}

private Map<TopicPartition, Long> fetchOffsets(int flag) {
private Map<TopicPartition, Long> fetchOffsets(int flag) throws InterruptedException {
Consumer<byte[], byte[]> kafkaConsumer = null;
try {
kafkaConsumer = new KafkaConsumer<>(storagePluginConfig.getKafkaConsumerProps(),
Expand All @@ -132,8 +133,8 @@ private Map<TopicPartition, Long> fetchOffsets(int flag) {
// evaluates lazily, seeking to the
// first/last offset in all partitions only when poll(long) or
// position(TopicPartition) are called
kafkaConsumer.poll(0);
Set<TopicPartition> assignments = kafkaConsumer.assignment();
kafkaConsumer.poll(Duration.ofSeconds(5));
Set<TopicPartition> assignments = waitForConsumerAssignment(kafkaConsumer);

if (flag == -2) {
// fetch start offsets for each topicPartition
Expand All @@ -156,6 +157,25 @@ private Map<TopicPartition, Long> fetchOffsets(int flag) {
}
}

private Set<TopicPartition> waitForConsumerAssignment(Consumer consumer) throws InterruptedException {
Set<TopicPartition> assignments = consumer.assignment();

long waitingForAssigmentTimeout = 5000;
long timeout = 0;

while (assignments.isEmpty() && timeout < waitingForAssigmentTimeout) {
Thread.sleep(500);
timeout += 500;
assignments = consumer.assignment();
}

if (timeout >= waitingForAssigmentTimeout) {
fail("Consumer assignment wasn't completed within the timeout " + waitingForAssigmentTimeout);
}

return assignments;
}

@Test
public void testPhysicalPlanSubmission() throws Exception {
String query = String.format(TestQueryConstants.MSG_SELECT_QUERY, TestQueryConstants.JSON_TOPIC);
Expand Down Expand Up @@ -281,4 +301,4 @@ public void testEscapeAnyChar() throws Exception {
client.resetSession(ExecConstants.KAFKA_READER_ESCAPE_ANY_CHAR);
}
}
}
}