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

ASF GitHub Bot commented on TAJO-1487:
--------------------------------------

Github user jinossy commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/1043#discussion_r75110466
  
    --- Diff: 
tajo-storage/tajo-storage-kafka/src/main/java/org/apache/tajo/storage/kafka/KafkaFragment.java
 ---
    @@ -0,0 +1,199 @@
    +/**
    + * 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.tajo.storage.kafka;
    +
    +import org.apache.tajo.storage.fragment.BuiltinFragmentKinds;
    +import org.apache.tajo.storage.fragment.Fragment;
    +import org.apache.tajo.storage.kafka.KafkaFragment.KafkaFragmentKey;
    +
    +import java.net.URI;
    +
    +import com.google.common.base.Objects;
    +
    +/**
    + * Fragment for Kafka
    + */
    +public class KafkaFragment extends Fragment<KafkaFragmentKey> {
    +  private String topicName;
    +  private boolean last;
    +
    +  public KafkaFragment(URI uri, String tableName, String topicName, long 
startOffset, long lastOffset,
    +      int partitionId, String leaderHost) {
    +    super(BuiltinFragmentKinds.KAFKA, uri, tableName, new 
KafkaFragmentKey(partitionId, startOffset),
    +        new KafkaFragmentKey(partitionId, lastOffset), lastOffset - 
startOffset, new String[] { leaderHost });
    +
    +    this.topicName = topicName;
    +    this.last = false;
    +  }
    +
    +  public KafkaFragment(URI uri, String tableName, String topicName, long 
startOffset, long lastOffset,
    +      int partitionId, String leaderHost, boolean last) {
    +    this(uri, tableName, topicName, startOffset, lastOffset, partitionId, 
leaderHost);
    +    this.last = last;
    +  }
    +
    +  @Override
    +  public boolean isEmpty() {
    +    return startKey.isEmpty() || endKey.isEmpty();
    +  }
    +
    +  public Object clone() throws CloneNotSupportedException {
    +    KafkaFragment frag = (KafkaFragment) super.clone();
    +    frag.topicName = topicName;
    +    frag.last = last;
    +    return frag;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (o instanceof KafkaFragment) {
    +      KafkaFragment t = (KafkaFragment) o;
    +      if (inputSourceId.equals(t.inputSourceId) && 
topicName.equals(t.topicName)
    +        && getStartKey().equals(t.getStartKey()) && 
getEndKey().equals(t.getEndKey())) {
    +        return true;
    +      }
    +    }
    +    return false;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    return Objects.hashCode(inputSourceId, topicName, getStartKey(), 
getEndKey());
    +  }
    +
    +  @Override
    +  public String toString() {
    +    StringBuilder builder = new StringBuilder();
    +    builder.append("\"fragment\": {\"topicName\":");
    +    builder.append(topicName);
    +    builder.append(", \"uri\":");
    +    builder.append(uri);
    +    builder.append(", \"inputSourceId\":");
    +    builder.append(inputSourceId);
    +    builder.append(", \"startKey\":");
    +    builder.append(startKey);
    +    builder.append(", \"endKey\":");
    +    builder.append(endKey);
    +    builder.append(", \"length\":");
    +    builder.append(length);
    +    builder.append("}");
    +    return builder.toString();
    +  }
    +
    +  public boolean isLast() {
    +    return last;
    +  }
    +
    +  public void setLast(boolean last) {
    +    this.last = last;
    +  }
    +
    +  public String getTopicName() {
    +    return this.topicName;
    +  }
    +
    +  public void setStartKey(int partitionId, long startOffset) {
    +    this.startKey = new KafkaFragmentKey(partitionId, startOffset);
    +  }
    +
    +  public void setEndKey(int partitionId, long lastOffset) {
    +    this.endKey = new KafkaFragmentKey(partitionId, lastOffset);
    +  }
    +
    +  public static class KafkaFragmentKey implements 
Comparable<KafkaFragmentKey> {
    +    private final Integer partitionId;
    --- End diff --
    
    partitionId and offset should be primitive


> Kafka Scanner for kafka strage.
> -------------------------------
>
>                 Key: TAJO-1487
>                 URL: https://issues.apache.org/jira/browse/TAJO-1487
>             Project: Tajo
>          Issue Type: Sub-task
>          Components: Storage
>            Reporter: YeonSu Han
>            Assignee: Byunghwa Yun
>              Labels: kafka_storage
>
> Have to implement Scanner interface for Kafka storage.
> The scan by split into many fragments. Scanner and Fragment are 1 to 1 
> mapping.
> For this, need to feature like this,
> - Create Fragment class
> - Implement 'Tuple next()' method in Scanner.
> - etc..
> The issue is related to TAJO-1480, TAJO-1502. So first, must be commit 
> TAJO-1480 and TAJO-1502.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to