[
https://issues.apache.org/jira/browse/DRILL-4779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16256334#comment-16256334
]
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_r151582174
--- Diff:
contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaSubScan.java
---
@@ -0,0 +1,176 @@
+/*
+ * 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.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+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;
+
+@JsonTypeName("kafka-partition-scan")
+public class KafkaSubScan extends AbstractBase implements SubScan {
+
+ @JsonProperty
+ private final KafkaStoragePluginConfig kafkStoragePluginConfig;
+
+ @JsonIgnore
+ private final KafkaStoragePlugin kafkaStoragePlugin;
+ private final List<SchemaPath> coulmns;
+ private final List<KafkaSubScanSpec> partitionSubScanSpecList;
+
+ @JsonCreator
+ public KafkaSubScan(@JacksonInject StoragePluginRegistry registry,
@JsonProperty("userName") String userName,
+ @JsonProperty("kafkStoragePluginConfig") KafkaStoragePluginConfig
kafkStoragePluginConfig,
+ @JsonProperty("coulmns") List<SchemaPath> coulmns,
+ @JsonProperty("partitionSubScanSpecList")
LinkedList<KafkaSubScanSpec> partitionSubScanSpecList)
+ throws ExecutionSetupException {
+ super(userName);
+ this.kafkStoragePluginConfig = kafkStoragePluginConfig;
+ this.coulmns = coulmns;
+ this.partitionSubScanSpecList = partitionSubScanSpecList;
+ this.kafkaStoragePlugin = (KafkaStoragePlugin)
registry.getPlugin(kafkStoragePluginConfig);
+ }
+
+ public KafkaSubScan(String userName, KafkaStoragePlugin plugin,
KafkaStoragePluginConfig kafkStoragePluginConfig,
+ List<SchemaPath> coulmns, List<KafkaSubScanSpec>
partitionSubScanSpecList) {
+ super(userName);
+ this.coulmns = coulmns;
+ this.kafkStoragePluginConfig = kafkStoragePluginConfig;
+ this.kafkaStoragePlugin = plugin;
+ this.partitionSubScanSpecList = partitionSubScanSpecList;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E>
physicalVisitor, X value) throws E {
+ return physicalVisitor.visitSubScan(this, value);
+ }
+
+ @Override
+ public PhysicalOperator getNewWithChildren(List<PhysicalOperator>
children) throws ExecutionSetupException {
+ Preconditions.checkArgument(children.isEmpty());
+ return new KafkaSubScan(getUserName(), kafkaStoragePlugin,
kafkStoragePluginConfig, coulmns,
+ partitionSubScanSpecList);
+ }
+
+ @Override
+ public Iterator<PhysicalOperator> iterator() {
+ return Collections.emptyIterator();
+ }
+
+ @JsonIgnore
+ public KafkaStoragePluginConfig getKafkStoragePluginConfig() {
+ return kafkStoragePluginConfig;
+ }
+
+ @JsonIgnore
+ public KafkaStoragePlugin getKafkaStoragePlugin() {
+ return kafkaStoragePlugin;
+ }
+
+ public List<SchemaPath> getCoulmns() {
+ return coulmns;
+ }
+
+ public List<KafkaSubScanSpec> getPartitionSubScanSpecList() {
+ return partitionSubScanSpecList;
+ }
+
+ @Override
+ public int getOperatorType() {
+ return 0;
--- End diff --
Should add an operator code in `UserBitShared.proto`, something like:
```
KAFKA_SCAN = 38;
```
Then return that value here.
> 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)