Github user paul-rogers commented on a diff in the pull request: https://github.com/apache/drill/pull/1027#discussion_r150084784 --- Diff: contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java --- @@ -0,0 +1,178 @@ +/** + * 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.drill.exec.store.kafka; + +import static org.apache.drill.exec.store.kafka.DrillKafkaConfig.DRILL_KAFKA_POLL_TIMEOUT; + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.ops.FragmentContext; +import org.apache.drill.exec.ops.OperatorContext; +import org.apache.drill.exec.physical.impl.OutputMutator; +import org.apache.drill.exec.store.AbstractRecordReader; +import org.apache.drill.exec.store.kafka.KafkaSubScan.KafkaSubScanSpec; +import org.apache.drill.exec.store.kafka.decoders.MessageReader; +import org.apache.drill.exec.store.kafka.decoders.MessageReaderFactory; +import org.apache.drill.exec.util.Utilities; +import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter; +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.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +public class KafkaRecordReader extends AbstractRecordReader { + private static final Logger logger = LoggerFactory.getLogger(KafkaRecordReader.class); + public static final long DEFAULT_MESSAGES_PER_BATCH = 4000; + + private VectorContainerWriter writer; + private MessageReader messageReader; + + private boolean unionEnabled; + private KafkaConsumer<byte[], byte[]> kafkaConsumer; + private KafkaStoragePlugin plugin; + private KafkaSubScanSpec subScanSpec; + private long kafkaPollTimeOut; + private long endOffset; + + private long currentOffset; + private long totalFetchTime = 0; + + private List<TopicPartition> partitions; + private final boolean enableAllTextMode; + private final boolean readNumbersAsDouble; + + private Iterator<ConsumerRecord<byte[], byte[]>> messageIter; + + public KafkaRecordReader(KafkaSubScan.KafkaSubScanSpec subScanSpec, List<SchemaPath> projectedColumns, + FragmentContext context, KafkaStoragePlugin plugin) { + setColumns(projectedColumns); + this.enableAllTextMode = context.getOptions().getOption(ExecConstants.KAFKA_ALL_TEXT_MODE).bool_val; + this.readNumbersAsDouble = context.getOptions() + .getOption(ExecConstants.KAFKA_READER_READ_NUMBERS_AS_DOUBLE).bool_val; + this.unionEnabled = context.getOptions().getOption(ExecConstants.ENABLE_UNION_TYPE); + this.plugin = plugin; + this.subScanSpec = subScanSpec; + this.endOffset = subScanSpec.getEndOffset(); + this.kafkaPollTimeOut = Long.valueOf(plugin.getConfig().getDrillKafkaProps().getProperty(DRILL_KAFKA_POLL_TIMEOUT)); + } + + @Override + protected Collection<SchemaPath> transformColumns(Collection<SchemaPath> projectedColumns) { + Set<SchemaPath> transformed = Sets.newLinkedHashSet(); + if (!isStarQuery()) { + for (SchemaPath column : projectedColumns) { + transformed.add(column); + } + } else { + transformed.add(Utilities.STAR_COLUMN); + } + return transformed; + } + + @Override + public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { + this.writer = new VectorContainerWriter(output, unionEnabled); + messageReader = MessageReaderFactory.getMessageReader(plugin.getConfig().getDrillKafkaProps()); + messageReader.init(context.getManagedBuffer(), Lists.newArrayList(getColumns()), this.writer, + this.enableAllTextMode, false, this.readNumbersAsDouble); + kafkaConsumer = messageReader.getConsumer(plugin); + partitions = Lists.newArrayListWithCapacity(1); + TopicPartition topicPartition = new TopicPartition(subScanSpec.getTopicName(), subScanSpec.getPartitionId()); + partitions.add(topicPartition); + kafkaConsumer.assign(partitions); + currentOffset = subScanSpec.getStartOffset(); + kafkaConsumer.seek(topicPartition, currentOffset); + } + + private boolean hasNext() { + if (currentOffset > endOffset) { + logger.info("Read all the messages from Kafka. Start offset : {}, End offset : {}", currentOffset, endOffset); + return false; + } else if (messageIter != null && messageIter.hasNext()) { + return true; + } else { + return fetch(); + } + } + + private boolean fetch() { + long startTime = System.currentTimeMillis(); + ConsumerRecords<byte[], byte[]> consumerRecords = kafkaConsumer.poll(kafkaPollTimeOut); + + if (consumerRecords.isEmpty()) { + logger.warn("Not able to fetch messages within {}milliseconds. Consider increasing the value of {}", + DRILL_KAFKA_POLL_TIMEOUT, kafkaPollTimeOut); + } + + long lastFetchTime = (System.currentTimeMillis() - startTime); + logger.debug("Total number of messages fetched : " + consumerRecords.count()); + logger.debug("Time taken to fetch : " + (lastFetchTime / 1000) + " seconds"); + totalFetchTime += lastFetchTime; + + messageIter = consumerRecords.iterator(); + return messageIter.hasNext(); + } + + /** + * KafkaConsumer.poll will fetch 500 messages per poll call. So hasNext will + * take care of polling multiple times for this given batch next invocation + */ + @Override + public int next() { + writer.allocate(); + writer.reset(); + Stopwatch watch = Stopwatch.createStarted(); + + int messageCount = 0; + while (hasNext()) { + ConsumerRecord<byte[], byte[]> consumerRecord = messageIter.next(); + currentOffset = consumerRecord.offset(); + writer.setPosition(messageCount); + messageReader.readMessage(consumerRecord); + ++messageCount; + if (messageCount >= DEFAULT_MESSAGES_PER_BATCH) { + break; + } + } + + messageReader.ensureAtLeastOneField(); + writer.setValueCount(messageCount); + logger.info("Took {} ms to process {} records.", watch.elapsed(TimeUnit.MILLISECONDS), messageCount); + logger.info("Last offset consumed for {}:{} is {}", subScanSpec.getTopicName(), subScanSpec.getPartitionId(), + currentOffset); + return messageCount; + } + + @Override + public void close() throws Exception { --- End diff -- Should the `messageReader` be closed? Does it hold memory, an open socket or other resources? What about the iterator? Other resources to be released?
---