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

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_r151584935
  
    --- Diff: 
contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/decoders/JsonMessageReader.java
 ---
    @@ -0,0 +1,104 @@
    +/*
    + * 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.decoders;
    +
    +import static 
org.apache.drill.exec.store.kafka.MetaDataField.KAFKA_MSG_KEY;
    +import static org.apache.drill.exec.store.kafka.MetaDataField.KAFKA_OFFSET;
    +import static 
org.apache.drill.exec.store.kafka.MetaDataField.KAFKA_PARTITION_ID;
    +import static 
org.apache.drill.exec.store.kafka.MetaDataField.KAFKA_TIMESTAMP;
    +import static org.apache.drill.exec.store.kafka.MetaDataField.KAFKA_TOPIC;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.store.kafka.KafkaStoragePlugin;
    +import org.apache.drill.exec.vector.complex.fn.JsonReader;
    +import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Charsets;
    +import com.google.gson.JsonObject;
    +import com.google.gson.JsonParser;
    +
    +import io.netty.buffer.DrillBuf;
    +
    +/**
    + * MessageReader class which will convert ConsumerRecord into JSON and 
writes to
    + * VectorContainerWriter of JsonReader
    + *
    + */
    +public class JsonMessageReader implements MessageReader {
    +
    +  private static final Logger logger = 
LoggerFactory.getLogger(JsonMessageReader.class);
    +  private JsonReader jsonReader;
    +  private VectorContainerWriter writer;
    +
    +  @Override
    +  public void init(DrillBuf buf, List<SchemaPath> columns, 
VectorContainerWriter writer, boolean allTextMode,
    +      boolean skipOuterList, boolean readNumbersAsDouble) {
    +    this.jsonReader = new JsonReader(buf, columns, allTextMode, 
skipOuterList, readNumbersAsDouble);
    +    this.writer = writer;
    +  }
    +
    +  @Override
    +  public void readMessage(ConsumerRecord<?, ?> record) {
    +    try {
    +      byte[] recordArray = (byte[]) record.value();
    +      JsonObject jsonObj = (new JsonParser()).parse(new 
String(recordArray, Charsets.UTF_8)).getAsJsonObject();
    +      jsonObj.addProperty(KAFKA_TOPIC.getFieldName(), record.topic());
    +      jsonObj.addProperty(KAFKA_PARTITION_ID.getFieldName(), 
record.partition());
    +      jsonObj.addProperty(KAFKA_OFFSET.getFieldName(), record.offset());
    +      jsonObj.addProperty(KAFKA_TIMESTAMP.getFieldName(), 
record.timestamp());
    +      jsonObj.addProperty(KAFKA_MSG_KEY.getFieldName(), record.key() != 
null ? record.key().toString() : null);
    +      jsonReader.setSource(jsonObj.toString().getBytes(Charsets.UTF_8));
    --- End diff --
    
    Actually, no need for all this complexity. This code:
    
    * Gets UTF-8 bytes encoding JSON
    * Converts that to a string
    * Parses that into a JSON object
    * Serializes that into a string
    * Converts that to a byte array.
    * The "JSON reader" parses the bytes back into JSON tokens.
    
    Any reason we can't just pass in the byte array directly? Another 
alternative is to create an `InputStream` that reads the byte array. The key 
thing to realize here is that the `jsonReader` incorporates the Jackson JSON 
parser, and is perfectly capable of parsing any form of serialized JSON.
    
    So, the above can be reduced to:
    
    * Pass in the `recordArray` to the `jsonReader`.
    * The "JSON reader" parses the bytes into JSON tokens.



> 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