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

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_r149763028
  
    --- Diff: 
contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
 ---
    @@ -0,0 +1,301 @@
    +/**
    + * 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.Arrays;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.physical.EndpointAffinity;
    +import org.apache.drill.exec.physical.base.AbstractGroupScan;
    +import org.apache.drill.exec.physical.base.GroupScan;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.base.ScanStats;
    +import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
    +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
    +import org.apache.drill.exec.store.StoragePluginRegistry;
    +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.store.schedule.AffinityCreator;
    +import org.apache.drill.exec.store.schedule.AssignmentCreator;
    +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.KafkaConsumer;
    +import org.apache.kafka.common.Node;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.fasterxml.jackson.annotation.JacksonInject;
    +import com.fasterxml.jackson.annotation.JsonCreator;
    +import com.fasterxml.jackson.annotation.JsonIgnore;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ListMultimap;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +
    +@JsonTypeName("kafka-scan")
    +public class KafkaGroupScan extends AbstractGroupScan {
    +
    +  static final Logger logger = 
LoggerFactory.getLogger(KafkaGroupScan.class);
    +  private static final long MSG_SIZE = 1024;
    +
    +  private KafkaStoragePlugin kafkaStoragePlugin;
    +  private KafkaStoragePluginConfig kafkaStoragePluginConfig;
    +  private List<SchemaPath> columns;
    +  private KafkaScanSpec kafkaScanSpec;
    +
    +  private List<PartitionScanWork> partitionWorkList;
    +  private ListMultimap<Integer, PartitionScanWork> assignments;
    +  private List<EndpointAffinity> affinities;
    +
    +  @JsonCreator
    +  public KafkaGroupScan(@JsonProperty("userName") String userName,
    +      @JsonProperty("kafkaStoragePluginConfig") KafkaStoragePluginConfig 
kafkaStoragePluginConfig,
    +      @JsonProperty("columns") List<SchemaPath> columns, 
@JsonProperty("scanSpec") KafkaScanSpec scanSpec,
    +      @JacksonInject StoragePluginRegistry pluginRegistry) {
    +    this(userName, kafkaStoragePluginConfig, columns, scanSpec, 
(KafkaStoragePlugin) pluginRegistry);
    +  }
    +
    +  public KafkaGroupScan(KafkaStoragePlugin kafkaStoragePlugin, 
KafkaScanSpec kafkaScanSpec, List<SchemaPath> columns) {
    +    super(StringUtils.EMPTY);
    +    this.kafkaStoragePlugin = kafkaStoragePlugin;
    +    this.kafkaStoragePluginConfig = (KafkaStoragePluginConfig) 
kafkaStoragePlugin.getConfig();
    +    this.columns = columns;
    +    this.kafkaScanSpec = kafkaScanSpec;
    +    init();
    +  }
    +
    +  public KafkaGroupScan(String userName, KafkaStoragePluginConfig 
kafkaStoragePluginConfig, List<SchemaPath> columns,
    +      KafkaScanSpec kafkaScanSpec, KafkaStoragePlugin pluginRegistry) {
    +    super(userName);
    +    this.kafkaStoragePluginConfig = kafkaStoragePluginConfig;
    +    this.columns = columns;
    +    this.kafkaScanSpec = kafkaScanSpec;
    +    this.kafkaStoragePlugin = pluginRegistry;
    +    init();
    +  }
    +
    +  public KafkaGroupScan(KafkaGroupScan that) {
    +    super(that);
    +    this.kafkaStoragePluginConfig = that.kafkaStoragePluginConfig;
    +    this.columns = that.columns;
    +    this.kafkaScanSpec = that.kafkaScanSpec;
    +    this.kafkaStoragePlugin = that.kafkaStoragePlugin;
    +    this.partitionWorkList = that.partitionWorkList;
    +    this.assignments = that.assignments;
    +  }
    +
    +  private static class PartitionScanWork implements CompleteWork {
    +
    +    private EndpointByteMapImpl byteMap = new EndpointByteMapImpl();
    +
    +    private TopicPartition topicPartition;
    +    private long beginOffset;
    +    private long latestOffset;
    +
    +    public PartitionScanWork(TopicPartition topicPartition, long 
beginOffset, long latestOffset) {
    +      this.topicPartition = topicPartition;
    +      this.beginOffset = beginOffset;
    +      this.latestOffset = latestOffset;
    +    }
    +
    +    public TopicPartition getTopicPartition() {
    +      return topicPartition;
    +    }
    +
    +    public long getBeginOffset() {
    +      return beginOffset;
    +    }
    +
    +    public long getLatestOffset() {
    +      return latestOffset;
    +    }
    +
    +    @Override
    +    public int compareTo(CompleteWork o) {
    +      return Long.compare(getTotalBytes(), o.getTotalBytes());
    +    }
    +
    +    @Override
    +    public long getTotalBytes() {
    +      return (latestOffset - beginOffset) * MSG_SIZE;
    +    }
    +
    +    @Override
    +    public EndpointByteMap getByteMap() {
    +      return byteMap;
    +    }
    +
    +  }
    +
    +  /**
    +   * Computes work per topic partition, based on start and end offset of 
each
    +   * corresponding topicPartition
    +   */
    +  private void init() {
    +    partitionWorkList = Lists.newArrayList();
    +    Collection<DrillbitEndpoint> endpoints = 
kafkaStoragePlugin.getContext().getBits();
    +    Map<String, DrillbitEndpoint> endpointMap = Maps.newHashMap();
    +    for (DrillbitEndpoint endpoint : endpoints) {
    +      endpointMap.put(endpoint.getAddress(), endpoint);
    +    }
    +
    +    String topicName = kafkaScanSpec.getTopicName();
    +    MessageReader messageReader = MessageReaderFactory
    +        
.getMessageReader(kafkaStoragePlugin.getConfig().getDrillKafkaProps());
    +    KafkaConsumer<?, ?> kafkaConsumer = 
messageReader.getConsumer(kafkaStoragePlugin);
    +
    +    Map<TopicPartition, Long> startOffsetsMap = Maps.newHashMap();
    +    Map<TopicPartition, Long> endOffsetsMap = Maps.newHashMap();
    +    kafkaConsumer.subscribe(Arrays.asList(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);
    --- End diff --
    
    Drill is used in production systems where all manner of things can go 
wrong. Over the years, we've found that good error messages are vital when, 
say, we want to diagnose a user problem on the user mailing list, or a 
commercial support person needs to track down a problem.
    
    The [Kafka 
docs](https://kafka.apache.org/0101/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html)
 suggest a number of exceptions can be thrown from this API call: 
`InvalidOffsetException`, `AuthorizationException`, `KafkaException` and so on. 
All seem to be unchecked.
    
    Drill prefers unchecked exceptions, so this seems fine. But, if the Kafka 
exceptions bubble up to the Scan operator, the scan operator won't have any 
context and the resulting user message will be vague, of the form "something 
went wrong."
    
    All this is a lead-in to suggesting wrapping every Kafka call that might 
fail in a try/catch block, then translating the exception to a Drill 
`UserException`, perhaps using the `dataReadError` form (or another form, if 
that fits better.) Provide context to help pinpoint problems.
    
    Think of it this way: if you got a user note that "the Kafka plugin doesn't 
work for me", what would you want to see in the log file to help diagnose the 
problems?



> 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