Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 649C0200D43 for ; Tue, 21 Nov 2017 15:13:41 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 632F1160BFC; Tue, 21 Nov 2017 14:13:41 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 80B36160BED for ; Tue, 21 Nov 2017 15:13:40 +0100 (CET) Received: (qmail 21503 invoked by uid 500); 21 Nov 2017 14:13:39 -0000 Mailing-List: contact dev-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@drill.apache.org Delivered-To: mailing list dev@drill.apache.org Received: (qmail 21491 invoked by uid 99); 21 Nov 2017 14:13:39 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 21 Nov 2017 14:13:39 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 3A39DDFCFC; Tue, 21 Nov 2017 14:13:39 +0000 (UTC) From: kameshb To: dev@drill.apache.org Reply-To: dev@drill.apache.org References: In-Reply-To: Subject: [GitHub] drill pull request #1027: DRILL-4779 : Kafka storage plugin Content-Type: text/plain Message-Id: <20171121141339.3A39DDFCFC@git1-us-west.apache.org> Date: Tue, 21 Nov 2017 14:13:39 +0000 (UTC) archived-at: Tue, 21 Nov 2017 14:13:41 -0000 Github user kameshb commented on a diff in the pull request: https://github.com/apache/drill/pull/1027#discussion_r152285907 --- Diff: contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java --- @@ -0,0 +1,145 @@ +/* + * 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 java.util.Collection; +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.exceptions.UserException; +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.server.options.OptionManager; +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.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 final boolean unionEnabled; + private final KafkaStoragePlugin plugin; + private final KafkaSubScanSpec subScanSpec; + private final long kafkaPollTimeOut; + + private long currentOffset; + private MessageIterator msgItr; + + private final boolean enableAllTextMode; + private final boolean readNumbersAsDouble; + private final String kafkaMsgReader; + + public KafkaRecordReader(KafkaSubScan.KafkaSubScanSpec subScanSpec, List 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; + OptionManager options = context.getOptions(); + this.unionEnabled = options.getOption(ExecConstants.ENABLE_UNION_TYPE); + this.kafkaMsgReader = options.getOption(ExecConstants.KAFKA_RECORD_READER).string_val; + this.kafkaPollTimeOut = options.getOption(ExecConstants.KAFKA_POLL_TIMEOUT).num_val; + this.plugin = plugin; + this.subScanSpec = subScanSpec; + } + + @Override + protected Collection transformColumns(Collection projectedColumns) { + Set 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(kafkaMsgReader); + messageReader.init(context.getManagedBuffer(), Lists.newArrayList(getColumns()), this.writer, + this.enableAllTextMode, this.readNumbersAsDouble); + msgItr = new MessageIterator(messageReader.getConsumer(plugin), subScanSpec, kafkaPollTimeOut); + } + + /** + * 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; + + try { + while (currentOffset < subScanSpec.getEndOffset() - 1 && msgItr.hasNext()) { + ConsumerRecord consumerRecord = msgItr.next(); + currentOffset = consumerRecord.offset(); + writer.setPosition(messageCount); + messageReader.readMessage(consumerRecord); + if (++messageCount >= DEFAULT_MESSAGES_PER_BATCH) { + break; + } + } + + messageReader.ensureAtLeastOneField(); + writer.setValueCount(messageCount); + logger.debug("Took {} ms to process {} records.", watch.elapsed(TimeUnit.MILLISECONDS), messageCount); + logger.debug("Last offset consumed for {}:{} is {}", subScanSpec.getTopicName(), subScanSpec.getPartitionId(), + currentOffset); + return messageCount; + } catch (Exception e) { + String msg = "Failure while reading messages from kafka. Recordreader was at record: " + (messageCount + 1); + throw UserException.dataReadError(e).message(msg).addContext(e.getMessage()).build(logger); + } + } + + @Override + public void close() throws Exception { + logger.info("Last offset processed for {}:{} is - {}", subScanSpec.getTopicName(), subScanSpec.getPartitionId(), --- End diff -- This would help while debugging if there any issues. That is the reason why made it info and most of other are logged as debug. ---