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

ASF GitHub Bot commented on DRILL-4779:
---------------------------------------

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<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;
    +    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<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(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<byte[], byte[]> 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.


> Kafka storage plugin support
> ----------------------------
>
>                 Key: DRILL-4779
>                 URL: https://issues.apache.org/jira/browse/DRILL-4779
>             Project: Apache Drill
>          Issue Type: New Feature
>          Components: Storage - Other
>    Affects Versions: 1.11.0
>            Reporter: B Anil Kumar
>            Assignee: B Anil Kumar
>              Labels: doc-impacting
>             Fix For: 1.12.0
>
>
> Implement Kafka storage plugin will enable the strong SQL support for Kafka.
> Initially implementation can target for supporting json and avro message types



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to