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

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

Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1027#discussion_r151586853
  
    --- Diff: 
contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
 ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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.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;
    +    this.unionEnabled = 
context.getOptions().getOption(ExecConstants.ENABLE_UNION_TYPE);
    +    this.kafkaMsgReader = 
context.getOptions().getOption(ExecConstants.KAFKA_RECORD_READER).string_val;
    +    this.kafkaPollTimeOut = 
context.getOptions().getOption(ExecConstants.KAFKA_POLL_TIMEOUT).num_val;
    --- End diff --
    
    A recent PR has simplified the above. You can now do:
    
    ```
    kafkaPollTImeout = 
context.getOptions().getInt(ExecConstants.KAFKA_POLL_TIMEOUT));
    ```
    
    The `getInt()` method handles type checking and avoids NPEs if somehow the 
value type is stored incorrectly.
    
    Also, given the number of options, consider:
    
    ```
      OptionManager options = context.getOptions();
      foo = options.getSomething(...);
    ```


> 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