Skip to content

Commit 727327d

Browse files
[FLINK-34466] Lineage interfaces for kafka connector (#130)
Signed-off-by: Pawel Leszczynski <[email protected]>
1 parent 0fed445 commit 727327d

23 files changed

+1178
-12
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,65 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil;
5+
6+
import java.util.Objects;
7+
import java.util.Properties;
8+
9+
/** Default implementation of {@link KafkaDatasetFacet}. */
10+
@PublicEvolving
11+
public class DefaultKafkaDatasetFacet implements KafkaDatasetFacet {
12+
13+
public static final String KAFKA_FACET_NAME = "kafka";
14+
15+
private Properties properties;
16+
17+
private final KafkaDatasetIdentifier topicIdentifier;
18+
19+
public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier, Properties properties) {
20+
this(topicIdentifier);
21+
22+
this.properties = new Properties();
23+
KafkaPropertiesUtil.copyProperties(properties, this.properties);
24+
}
25+
26+
public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier) {
27+
this.topicIdentifier = topicIdentifier;
28+
}
29+
30+
public void setProperties(Properties properties) {
31+
this.properties = new Properties();
32+
KafkaPropertiesUtil.copyProperties(properties, this.properties);
33+
}
34+
35+
public Properties getProperties() {
36+
return properties;
37+
}
38+
39+
public KafkaDatasetIdentifier getTopicIdentifier() {
40+
return topicIdentifier;
41+
}
42+
43+
@Override
44+
public boolean equals(Object o) {
45+
if (this == o) {
46+
return true;
47+
}
48+
if (o == null || getClass() != o.getClass()) {
49+
return false;
50+
}
51+
DefaultKafkaDatasetFacet that = (DefaultKafkaDatasetFacet) o;
52+
return Objects.equals(properties, that.properties)
53+
&& Objects.equals(topicIdentifier, that.topicIdentifier);
54+
}
55+
56+
@Override
57+
public int hashCode() {
58+
return Objects.hash(properties, topicIdentifier);
59+
}
60+
61+
@Override
62+
public String name() {
63+
return KAFKA_FACET_NAME;
64+
}
65+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,59 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
5+
import javax.annotation.Nullable;
6+
7+
import java.util.List;
8+
import java.util.Objects;
9+
import java.util.regex.Pattern;
10+
11+
/** Default implementation of {@link KafkaDatasetIdentifier}. */
12+
@PublicEvolving
13+
public class DefaultKafkaDatasetIdentifier implements KafkaDatasetIdentifier {
14+
15+
@Nullable private final List<String> topics;
16+
@Nullable private final Pattern topicPattern;
17+
18+
private DefaultKafkaDatasetIdentifier(
19+
@Nullable List<String> fixedTopics, @Nullable Pattern topicPattern) {
20+
this.topics = fixedTopics;
21+
this.topicPattern = topicPattern;
22+
}
23+
24+
public static DefaultKafkaDatasetIdentifier ofPattern(Pattern pattern) {
25+
return new DefaultKafkaDatasetIdentifier(null, pattern);
26+
}
27+
28+
public static DefaultKafkaDatasetIdentifier ofTopics(List<String> fixedTopics) {
29+
return new DefaultKafkaDatasetIdentifier(fixedTopics, null);
30+
}
31+
32+
@Nullable
33+
public List<String> getTopics() {
34+
return topics;
35+
}
36+
37+
@Nullable
38+
public Pattern getTopicPattern() {
39+
return topicPattern;
40+
}
41+
42+
@Override
43+
public boolean equals(Object o) {
44+
if (this == o) {
45+
return true;
46+
}
47+
if (o == null || getClass() != o.getClass()) {
48+
return false;
49+
}
50+
DefaultKafkaDatasetIdentifier that = (DefaultKafkaDatasetIdentifier) o;
51+
return Objects.equals(topics, that.topics)
52+
&& Objects.equals(topicPattern, that.topicPattern);
53+
}
54+
55+
@Override
56+
public int hashCode() {
57+
return Objects.hash(topics, topicPattern);
58+
}
59+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,44 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
import org.apache.flink.api.common.typeinfo.TypeInformation;
5+
6+
import java.util.Objects;
7+
8+
/** Default implementation of {@link KafkaDatasetFacet}. */
9+
@PublicEvolving
10+
public class DefaultTypeDatasetFacet implements TypeDatasetFacet {
11+
12+
public static final String TYPE_FACET_NAME = "type";
13+
14+
private final TypeInformation typeInformation;
15+
16+
public DefaultTypeDatasetFacet(TypeInformation typeInformation) {
17+
this.typeInformation = typeInformation;
18+
}
19+
20+
public TypeInformation getTypeInformation() {
21+
return typeInformation;
22+
}
23+
24+
public boolean equals(Object o) {
25+
if (this == o) {
26+
return true;
27+
}
28+
if (o == null || getClass() != o.getClass()) {
29+
return false;
30+
}
31+
DefaultTypeDatasetFacet that = (DefaultTypeDatasetFacet) o;
32+
return Objects.equals(typeInformation, that.typeInformation);
33+
}
34+
35+
@Override
36+
public int hashCode() {
37+
return Objects.hash(typeInformation);
38+
}
39+
40+
@Override
41+
public String name() {
42+
return TYPE_FACET_NAME;
43+
}
44+
}
Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
import org.apache.flink.streaming.api.lineage.LineageDatasetFacet;
5+
6+
import java.util.Properties;
7+
8+
/** Facet definition to contain all Kafka specific information on Kafka sources and sinks. */
9+
@PublicEvolving
10+
public interface KafkaDatasetFacet extends LineageDatasetFacet {
11+
Properties getProperties();
12+
13+
KafkaDatasetIdentifier getTopicIdentifier();
14+
15+
void setProperties(Properties properties);
16+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
5+
import java.util.Optional;
6+
7+
/** Contains method to extract {@link KafkaDatasetFacet}. */
8+
@PublicEvolving
9+
public interface KafkaDatasetFacetProvider {
10+
11+
/**
12+
* Returns a Kafka dataset facet or empty in case an implementing class is not able to identify
13+
* a dataset.
14+
*/
15+
Optional<KafkaDatasetFacet> getKafkaDatasetFacet();
16+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,30 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
5+
import javax.annotation.Nullable;
6+
7+
import java.util.List;
8+
import java.util.Objects;
9+
import java.util.regex.Pattern;
10+
11+
/** Kafka dataset identifier which can contain either a list of topics or a topic pattern. */
12+
@PublicEvolving
13+
public interface KafkaDatasetIdentifier {
14+
@Nullable
15+
List<String> getTopics();
16+
17+
@Nullable
18+
Pattern getTopicPattern();
19+
20+
/**
21+
* Assigns lineage dataset's name which is topic pattern if it is present or comma separated
22+
* list of topics.
23+
*/
24+
default String toLineageName() {
25+
if (getTopicPattern() != null) {
26+
return getTopicPattern().toString();
27+
}
28+
return String.join(",", Objects.requireNonNull(getTopics()));
29+
}
30+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
5+
import java.util.Optional;
6+
7+
/** Contains method which allows extracting topic identifier. */
8+
@PublicEvolving
9+
public interface KafkaDatasetIdentifierProvider {
10+
11+
/**
12+
* Gets Kafka dataset identifier or empty in case a class implementing is not able to extract
13+
* dataset identifier.
14+
*/
15+
Optional<DefaultKafkaDatasetIdentifier> getDatasetIdentifier();
16+
}
Lines changed: 118 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,118 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*
18+
*/
19+
20+
package org.apache.flink.connector.kafka.lineage;
21+
22+
import org.apache.flink.api.connector.source.Boundedness;
23+
import org.apache.flink.streaming.api.lineage.LineageDataset;
24+
import org.apache.flink.streaming.api.lineage.LineageDatasetFacet;
25+
import org.apache.flink.streaming.api.lineage.LineageVertex;
26+
import org.apache.flink.streaming.api.lineage.SourceLineageVertex;
27+
28+
import java.util.Collection;
29+
import java.util.Collections;
30+
import java.util.HashMap;
31+
import java.util.List;
32+
import java.util.Map;
33+
import java.util.Properties;
34+
import java.util.stream.Collectors;
35+
36+
/** Utility class with useful methods for managing lineage objects. */
37+
public class LineageUtil {
38+
39+
private static final String KAFKA_DATASET_PREFIX = "kafka://";
40+
private static final String COMMA = ",";
41+
private static final String SEMICOLON = ";";
42+
43+
public static LineageDataset datasetOf(String namespace, KafkaDatasetFacet kafkaDatasetFacet) {
44+
return datasetOf(namespace, kafkaDatasetFacet, Collections.emptyList());
45+
}
46+
47+
public static LineageDataset datasetOf(
48+
String namespace, KafkaDatasetFacet kafkaDatasetFacet, TypeDatasetFacet typeFacet) {
49+
return datasetOf(namespace, kafkaDatasetFacet, Collections.singletonList(typeFacet));
50+
}
51+
52+
private static LineageDataset datasetOf(
53+
String namespace,
54+
KafkaDatasetFacet kafkaDatasetFacet,
55+
List<LineageDatasetFacet> facets) {
56+
return new LineageDataset() {
57+
@Override
58+
public String name() {
59+
return kafkaDatasetFacet.getTopicIdentifier().toLineageName();
60+
}
61+
62+
@Override
63+
public String namespace() {
64+
return namespace;
65+
}
66+
67+
@Override
68+
public Map<String, LineageDatasetFacet> facets() {
69+
Map<String, LineageDatasetFacet> facetMap = new HashMap<>();
70+
facetMap.put(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME, kafkaDatasetFacet);
71+
facetMap.putAll(
72+
facets.stream()
73+
.collect(
74+
Collectors.toMap(LineageDatasetFacet::name, item -> item)));
75+
return facetMap;
76+
}
77+
};
78+
}
79+
80+
public static String namespaceOf(Properties properties) {
81+
String bootstrapServers = properties.getProperty("bootstrap.servers");
82+
83+
if (bootstrapServers == null) {
84+
return KAFKA_DATASET_PREFIX;
85+
}
86+
87+
if (bootstrapServers.contains(COMMA)) {
88+
bootstrapServers = bootstrapServers.split(COMMA)[0];
89+
} else if (bootstrapServers.contains(SEMICOLON)) {
90+
bootstrapServers = bootstrapServers.split(SEMICOLON)[0];
91+
}
92+
93+
return String.format(KAFKA_DATASET_PREFIX + bootstrapServers);
94+
}
95+
96+
public static SourceLineageVertex sourceLineageVertexOf(Collection<LineageDataset> datasets) {
97+
return new SourceLineageVertex() {
98+
@Override
99+
public Boundedness boundedness() {
100+
return Boundedness.CONTINUOUS_UNBOUNDED;
101+
}
102+
103+
@Override
104+
public List<LineageDataset> datasets() {
105+
return datasets.stream().collect(Collectors.toList());
106+
}
107+
};
108+
}
109+
110+
public static LineageVertex lineageVertexOf(Collection<LineageDataset> datasets) {
111+
return new LineageVertex() {
112+
@Override
113+
public List<LineageDataset> datasets() {
114+
return datasets.stream().collect(Collectors.toList());
115+
}
116+
};
117+
}
118+
}
Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,11 @@
1+
package org.apache.flink.connector.kafka.lineage;
2+
3+
import org.apache.flink.annotation.PublicEvolving;
4+
import org.apache.flink.api.common.typeinfo.TypeInformation;
5+
import org.apache.flink.streaming.api.lineage.LineageDatasetFacet;
6+
7+
/** Facet definition to contain type information of source and sink. */
8+
@PublicEvolving
9+
public interface TypeDatasetFacet extends LineageDatasetFacet {
10+
TypeInformation getTypeInformation();
11+
}

0 commit comments

Comments
 (0)