-
Notifications
You must be signed in to change notification settings - Fork 134
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-34466] create KafkaDatasetFacet
Signed-off-by: Pawel Leszczynski <[email protected]>
- Loading branch information
1 parent
ca14634
commit 4bbff17
Showing
31 changed files
with
837 additions
and
684 deletions.
There are no files selected for viewing
63 changes: 63 additions & 0 deletions
63
...afka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; | ||
|
||
import java.util.Objects; | ||
import java.util.Properties; | ||
|
||
/** Default implementation of {@link KafkaDatasetFacet}. */ | ||
public class DefaultKafkaDatasetFacet implements KafkaDatasetFacet { | ||
|
||
public static final String KAFKA_FACET_NAME = "kafka"; | ||
|
||
private Properties properties; | ||
|
||
private final KafkaDatasetIdentifier topicIdentifier; | ||
|
||
public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier, Properties properties) { | ||
this(topicIdentifier); | ||
|
||
this.properties = new Properties(); | ||
KafkaPropertiesUtil.copyProperties(properties, this.properties); | ||
} | ||
|
||
public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier) { | ||
this.topicIdentifier = topicIdentifier; | ||
} | ||
|
||
public void setProperties(Properties properties) { | ||
this.properties = new Properties(); | ||
KafkaPropertiesUtil.copyProperties(properties, this.properties); | ||
} | ||
|
||
public Properties getProperties() { | ||
return properties; | ||
} | ||
|
||
public KafkaDatasetIdentifier getTopicIdentifier() { | ||
return topicIdentifier; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
DefaultKafkaDatasetFacet that = (DefaultKafkaDatasetFacet) o; | ||
return Objects.equals(properties, that.properties) | ||
&& Objects.equals(topicIdentifier, that.topicIdentifier); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(properties, topicIdentifier); | ||
} | ||
|
||
@Override | ||
public String name() { | ||
return KAFKA_FACET_NAME; | ||
} | ||
} |
56 changes: 56 additions & 0 deletions
56
...src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,56 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Objects; | ||
import java.util.regex.Pattern; | ||
|
||
/** Default implementation of {@link KafkaDatasetIdentifier}. */ | ||
public class DefaultKafkaDatasetIdentifier implements KafkaDatasetIdentifier { | ||
|
||
@Nullable private final List<String> topics; | ||
@Nullable private final Pattern topicPattern; | ||
|
||
public DefaultKafkaDatasetIdentifier(List<String> fixedTopics, Pattern topicPattern) { | ||
this.topics = fixedTopics; | ||
this.topicPattern = topicPattern; | ||
} | ||
|
||
public static DefaultKafkaDatasetIdentifier ofPattern(Pattern pattern) { | ||
return new DefaultKafkaDatasetIdentifier(Collections.emptyList(), pattern); | ||
} | ||
|
||
public static DefaultKafkaDatasetIdentifier ofTopics(List<String> fixedTopics) { | ||
return new DefaultKafkaDatasetIdentifier(fixedTopics, null); | ||
} | ||
|
||
@Nullable | ||
public List<String> getTopics() { | ||
return topics; | ||
} | ||
|
||
@Nullable | ||
public Pattern getTopicPattern() { | ||
return topicPattern; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
DefaultKafkaDatasetIdentifier that = (DefaultKafkaDatasetIdentifier) o; | ||
return Objects.equals(topics, that.topics) | ||
&& Objects.equals(topicPattern, that.topicPattern); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(topics, topicPattern); | ||
} | ||
} |
42 changes: 42 additions & 0 deletions
42
...kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,42 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import java.util.Objects; | ||
|
||
/** Default implementation of {@link KafkaDatasetFacet}. */ | ||
public class DefaultTypeDatasetFacet implements TypeDatasetFacet { | ||
|
||
public static final String TYPE_FACET_NAME = "type"; | ||
|
||
private final TypeInformation typeInformation; | ||
|
||
public DefaultTypeDatasetFacet(TypeInformation typeInformation) { | ||
this.typeInformation = typeInformation; | ||
} | ||
|
||
public TypeInformation getTypeInformation() { | ||
return typeInformation; | ||
} | ||
|
||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
DefaultTypeDatasetFacet that = (DefaultTypeDatasetFacet) o; | ||
return Objects.equals(typeInformation, that.typeInformation); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(typeInformation); | ||
} | ||
|
||
@Override | ||
public String name() { | ||
return TYPE_FACET_NAME; | ||
} | ||
} |
14 changes: 14 additions & 0 deletions
14
...ector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,14 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; | ||
|
||
import java.util.Properties; | ||
|
||
/** Facet definition to contain all Kafka specific information on Kafka sources and sinks. */ | ||
public interface KafkaDatasetFacet extends LineageDatasetFacet { | ||
Properties getProperties(); | ||
|
||
KafkaDatasetIdentifier getTopicIdentifier(); | ||
|
||
void setProperties(Properties properties); | ||
} |
15 changes: 15 additions & 0 deletions
15
...fka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,15 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import java.util.Optional; | ||
|
||
/** Contains method to extract {@link KafkaDatasetFacet}. */ | ||
public interface KafkaDatasetFacetProvider { | ||
|
||
/** | ||
* Returns a Kafka dataset facet or `Optional.empty` in case an implementing class is not able | ||
* to identify a dataset. | ||
* | ||
* @return | ||
*/ | ||
Optional<KafkaDatasetFacet> getKafkaDatasetFacet(); | ||
} |
28 changes: 28 additions & 0 deletions
28
...-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,28 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.util.List; | ||
import java.util.regex.Pattern; | ||
|
||
/** Kafka dataset identifier which can contain either a list of topics or a topic pattern. */ | ||
public interface KafkaDatasetIdentifier { | ||
@Nullable | ||
List<String> getTopics(); | ||
|
||
@Nullable | ||
Pattern getTopicPattern(); | ||
|
||
/** | ||
* Assigns lineage dataset's name which is topic pattern if it is present or comma separated | ||
* list of topics. | ||
* | ||
* @return | ||
*/ | ||
default String toLineageName() { | ||
if (getTopicPattern() != null) { | ||
return getTopicPattern().toString(); | ||
} | ||
return String.join(",", getTopics()); | ||
} | ||
} |
15 changes: 15 additions & 0 deletions
15
...rc/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,15 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import java.util.Optional; | ||
|
||
/** Contains method which allows extracting topic identifier. */ | ||
public interface KafkaDatasetIdentifierProvider { | ||
|
||
/** | ||
* Gets Kafka dataset identifier or empty in case a class implementing is not able to extract | ||
* dataset identifier. | ||
* | ||
* @return | ||
*/ | ||
Optional<DefaultKafkaDatasetIdentifier> getDatasetIdentifier(); | ||
} |
19 changes: 0 additions & 19 deletions
19
...or-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageFacetProvider.java
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
9 changes: 9 additions & 0 deletions
9
...nector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,9 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; | ||
|
||
/** Facet definition to contain type information of source and sink. */ | ||
public interface TypeDatasetFacet extends LineageDatasetFacet { | ||
TypeInformation getTypeInformation(); | ||
} |
Oops, something went wrong.