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

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

akumarb2010 commented on a change in pull request #1272: DRILL-5977: Filter 
Pushdown in Drill-Kafka plugin
URL: https://github.com/apache/drill/pull/1272#discussion_r194247012
 
 

 ##########
 File path: 
contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaPartitionScanSpec.java
 ##########
 @@ -0,0 +1,103 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class KafkaPartitionScanSpec {
+  private String topicName;
+  private int partitionId;
+  private long startOffset;
+  private long endOffset;
+
+  @JsonCreator
+  public KafkaPartitionScanSpec(@JsonProperty("topicName") String topicName,
+                                @JsonProperty("partitionId") int partitionId,
+                                @JsonProperty("startOffset") long startOffset,
+                                @JsonProperty("endOffset") long endOffset) {
+    this.topicName = topicName;
+    this.partitionId = partitionId;
+    this.startOffset = startOffset;
+    this.endOffset = endOffset;
+  }
+
+  public String getTopicName() {
+    return topicName;
+  }
+
+  public int getPartitionId() {
+    return partitionId;
+  }
+
+  public long getStartOffset() {
+    return startOffset;
+  }
+
+  public long getEndOffset() {
+    return endOffset;
+  }
+
+  public void mergeScanSpec(String functionName, KafkaPartitionScanSpec 
scanSpec) {
+    switch (functionName) {
+      case "booleanAnd":
+        //Reduce the scan range
+        if(startOffset < scanSpec.startOffset) {
+          startOffset = scanSpec.startOffset;
+        }
+
+        if(endOffset > scanSpec.endOffset) {
+          endOffset = scanSpec.endOffset;
+        }
+        break;
+      case "booleanOr":
+        //Increase the scan range
+        if(scanSpec.startOffset < startOffset) {
+          startOffset = scanSpec.startOffset;
+        }
+
+        if(scanSpec.endOffset > endOffset) {
+          endOffset = scanSpec.endOffset;
+        }
+
+        break;
+      default:
+        assert(false);
+    }
 
 Review comment:
   Here we no need to fail for default. In other plugin's default, it will 
create empty ScanSpec. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> predicate pushdown support kafkaMsgOffset
> -----------------------------------------
>
>                 Key: DRILL-5977
>                 URL: https://issues.apache.org/jira/browse/DRILL-5977
>             Project: Apache Drill
>          Issue Type: Improvement
>            Reporter: B Anil Kumar
>            Assignee: Abhishek Ravi
>            Priority: Major
>             Fix For: 1.14.0
>
>
> As part of Kafka storage plugin review, below is the suggestion from Paul.
> {noformat}
> Does it make sense to provide a way to select a range of messages: a starting 
> point or a count? Perhaps I want to run my query every five minutes, scanning 
> only those messages since the previous scan. Or, I want to limit my take to, 
> say, the next 1000 messages. Could we use a pseudo-column such as 
> "kafkaMsgOffset" for that purpose? Maybe
> SELECT * FROM <some topic> WHERE kafkaMsgOffset > 12345
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to