-
Notifications
You must be signed in to change notification settings - Fork 5.5k
Extract pluggable interface for kafka cluster supplier #16272
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -15,27 +15,14 @@ | |
|
|
||
| import com.facebook.airlift.configuration.Config; | ||
| import com.facebook.presto.kafka.schema.file.FileTableDescriptionSupplier; | ||
| import com.facebook.presto.spi.HostAddress; | ||
| import com.google.common.base.Splitter; | ||
| import com.google.common.collect.ImmutableSet; | ||
| import com.facebook.presto.kafka.server.file.FileKafkaClusterMetadataSupplier; | ||
| import io.airlift.units.Duration; | ||
| import io.airlift.units.MinDuration; | ||
|
|
||
| import javax.validation.constraints.NotNull; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| import static com.google.common.collect.Iterables.transform; | ||
|
|
||
| public class KafkaConnectorConfig | ||
| { | ||
| private static final int KAFKA_DEFAULT_PORT = 9092; | ||
|
|
||
| /** | ||
| * Seed nodes for Kafka cluster. At least one must exist. | ||
| */ | ||
| private List<HostAddress> nodes; | ||
|
|
||
| /** | ||
| * Timeout to connect to Kafka. | ||
| */ | ||
|
|
@@ -66,6 +53,11 @@ public class KafkaConnectorConfig | |
| */ | ||
| private String tableDescriptionSupplier = FileTableDescriptionSupplier.NAME; | ||
|
|
||
| /** | ||
| * The kafka cluster metadata supplier to use, default is FILE | ||
| */ | ||
| private String clusterMetadataSupplier = FileKafkaClusterMetadataSupplier.NAME; | ||
|
|
||
| @NotNull | ||
| public String getDefaultSchema() | ||
| { | ||
|
|
@@ -79,18 +71,6 @@ public KafkaConnectorConfig setDefaultSchema(String defaultSchema) | |
| return this; | ||
| } | ||
|
|
||
| public List<HostAddress> getNodes() | ||
| { | ||
| return nodes; | ||
| } | ||
|
|
||
| @Config("kafka.nodes") | ||
| public KafkaConnectorConfig setNodes(String nodes) | ||
| { | ||
| this.nodes = (nodes == null) ? null : parseNodes(nodes).asList(); | ||
| return this; | ||
| } | ||
|
|
||
| @MinDuration("1s") | ||
| public Duration getKafkaConnectTimeout() | ||
| { | ||
|
|
@@ -141,26 +121,28 @@ public KafkaConnectorConfig setTableDescriptionSupplier(String tableDescriptionS | |
| return this; | ||
| } | ||
|
|
||
| public boolean isHideInternalColumns() | ||
| @NotNull | ||
| public String getClusterMetadataSupplier() | ||
| { | ||
| return hideInternalColumns; | ||
| return clusterMetadataSupplier; | ||
| } | ||
|
|
||
| @Config("kafka.hide-internal-columns") | ||
| public KafkaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) | ||
| @Config("kafka.cluster-metadata-supplier") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shall we document this new configuration? Thanks!
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. as for now, the only option for this configuration is the default option. do you recommend adding the document now or later when the new supplier is added?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see, we can document it later I think. I remember you also made some other pluggable interface before. Maybe you can document these new configuration together later. |
||
| public KafkaConnectorConfig setClusterMetadataSupplier(String clusterMetadataSupplier) | ||
| { | ||
| this.hideInternalColumns = hideInternalColumns; | ||
| this.clusterMetadataSupplier = clusterMetadataSupplier; | ||
| return this; | ||
| } | ||
|
|
||
| public static ImmutableSet<HostAddress> parseNodes(String nodes) | ||
| public boolean isHideInternalColumns() | ||
| { | ||
| Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); | ||
| return ImmutableSet.copyOf(transform(splitter.split(nodes), KafkaConnectorConfig::toHostAddress)); | ||
| return hideInternalColumns; | ||
| } | ||
|
|
||
| private static HostAddress toHostAddress(String value) | ||
| @Config("kafka.hide-internal-columns") | ||
| public KafkaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) | ||
| { | ||
| return HostAddress.fromString(value).withDefaultPort(KAFKA_DEFAULT_PORT); | ||
| this.hideInternalColumns = hideInternalColumns; | ||
| return this; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,31 @@ | ||
| /* | ||
| * Licensed 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 com.facebook.presto.kafka.server; | ||
|
|
||
| import com.facebook.presto.spi.HostAddress; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| /** | ||
| * This is mainly used to get Kafka cluster metadata such as broker list so that Kafka Connector can communicate with Kafka cluster | ||
| */ | ||
| public interface KafkaClusterMetadataSupplier | ||
| { | ||
| /** | ||
| * Gets kafka broker list for specified kafka cluster name | ||
| * @param clusterName the kafka cluster name | ||
| * @return kafka broker list | ||
| */ | ||
| List<HostAddress> getNodes(String clusterName); | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just wanna double confirm if this would break the existing config. is there any config migration required for this change?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The existing config can work as-is, no config migration is needed.
The reason is default cluster metadata supplier(FileKafkaClusterMetadataSupplier) will load FileKafkaClusterMetadataSupplierConfig which is the copy of the config that has been removed from this file.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Cool, sounds good to me.