flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-3102) Allow reading from multiple topics with one FlinkKafkaConsumer
Date Mon, 07 Dec 2015 16:30:10 GMT

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

ASF GitHub Bot commented on FLINK-3102:
---------------------------------------

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

    https://github.com/apache/flink/pull/1437#discussion_r46841698
  
    --- Diff: flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
---
    @@ -0,0 +1,117 @@
    +/*
    + * 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.flink.streaming.connectors.kafka.internals;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * A serializable representation of a kafka topic and a partition.
    + * Used as an operator state for the Kafka consumer
    + */
    +public class KafkaTopicPartition implements Serializable {
    +
    +	private static final long serialVersionUID = 722083576322742325L;
    +
    +	private String topic;
    +	private int partition;
    +
    +	public KafkaTopicPartition(String topic, int partition) {
    +		this.topic = topic;
    +		this.partition = partition;
    +	}
    +
    +	public String getTopic() {
    +		return topic;
    +	}
    +
    +	public int getPartition() {
    +		return partition;
    +	}
    +
    +	public void setTopic(String topic) {
    +		this.topic = topic;
    +	}
    +
    +	public void setPartition(int partition) {
    +		this.partition = partition;
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "KafkaTopicPartition{" +
    +				"topic='" + topic + '\'' +
    +				", partition=" + partition +
    +				'}';
    +	}
    +
    +	@Override
    +	public boolean equals(Object o) {
    +		if (this == o) {
    +			return true;
    +		}
    +		if (o == null || !(o instanceof KafkaTopicPartition)) {
    +			return false;
    +		}
    +
    +		KafkaTopicPartition that = (KafkaTopicPartition) o;
    +
    +		if (partition != that.partition) {
    +			return false;
    +		}
    +		return !(topic != null ? !topic.equals(that.topic) : that.topic != null);
    --- End diff --
    
    Make topic a "non null" fields (check in the constructor) and save a few cycles here...


> Allow reading from multiple topics with one FlinkKafkaConsumer
> --------------------------------------------------------------
>
>                 Key: FLINK-3102
>                 URL: https://issues.apache.org/jira/browse/FLINK-3102
>             Project: Flink
>          Issue Type: New Feature
>          Components: Kafka Connector
>            Reporter: Robert Metzger
>            Assignee: Robert Metzger
>
> Currently, a Kafka consumer allows to read from only one topic.
> For cases where multiple topics contain messages with the same schema, it is useful to
allow to subscribe to many topics using one FlinkKafkaConsumer instance.



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

Mime
View raw message