-
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 ea71a84
Showing
25 changed files
with
530 additions
and
674 deletions.
There are no files selected for viewing
16 changes: 16 additions & 0 deletions
16
...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,16 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import org.apache.flink.connector.kafka.lineage.facets.KafkaDatasetFacet; | ||
|
||
import java.util.Optional; | ||
|
||
/** Contains method to extract {@link KafkaDatasetFacet}. */ | ||
public interface KafkaDatasetFacetProvider { | ||
|
||
/** | ||
* List of lineage dataset facets. | ||
* | ||
* @return | ||
*/ | ||
Optional<KafkaDatasetFacet> getKafkaDatasetFacet(); | ||
} |
16 changes: 16 additions & 0 deletions
16
...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,16 @@ | ||
package org.apache.flink.connector.kafka.lineage; | ||
|
||
import org.apache.flink.connector.kafka.lineage.facets.KafkaDatasetFacet.KafkaDatasetIdentifier; | ||
|
||
import java.util.Optional; | ||
|
||
/** Contains method which allows extracting topic identifier. */ | ||
public interface KafkaDatasetIdentifierProvider { | ||
|
||
/** | ||
* List of lineage dataset facets. | ||
* | ||
* @return | ||
*/ | ||
Optional<KafkaDatasetIdentifier> 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
97 changes: 97 additions & 0 deletions
97
...afka/src/main/java/org/apache/flink/connector/kafka/lineage/facets/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,97 @@ | ||
package org.apache.flink.connector.kafka.lineage.facets; | ||
|
||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Objects; | ||
import java.util.Properties; | ||
import java.util.regex.Pattern; | ||
|
||
/** Facet containing all information related to sources and sinks on Kafka. */ | ||
public class KafkaDatasetFacet implements LineageDatasetFacet { | ||
|
||
public static final String KAFKA_FACET_NAME = "kafka"; | ||
|
||
public final Properties properties; | ||
public final TypeInformation typeInformation; | ||
public final KafkaDatasetIdentifier topicIdentifier; | ||
|
||
public KafkaDatasetFacet( | ||
KafkaDatasetIdentifier topicIdentifier, | ||
Properties properties, | ||
TypeInformation typeInformation) { | ||
this.topicIdentifier = topicIdentifier; | ||
this.properties = properties; | ||
this.typeInformation = typeInformation; | ||
} | ||
|
||
public void addProperties(Properties properties) { | ||
this.properties.putAll(properties); | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
KafkaDatasetFacet that = (KafkaDatasetFacet) o; | ||
return Objects.equals(properties, that.properties) | ||
&& Objects.equals(typeInformation, that.typeInformation) | ||
&& Objects.equals(topicIdentifier, that.topicIdentifier); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(properties, typeInformation, topicIdentifier); | ||
} | ||
|
||
@Override | ||
public String name() { | ||
return KAFKA_FACET_NAME; | ||
} | ||
|
||
/** | ||
* Record class to contain topics' identifier information which can be either a list of topics | ||
* or a topic pattern. | ||
*/ | ||
public static class KafkaDatasetIdentifier { | ||
public final List<String> topics; | ||
public final Pattern topicPattern; | ||
|
||
public KafkaDatasetIdentifier(List<String> fixedTopics, Pattern topicPattern) { | ||
this.topics = fixedTopics; | ||
this.topicPattern = topicPattern; | ||
} | ||
|
||
public static KafkaDatasetIdentifier of(Pattern pattern) { | ||
return new KafkaDatasetIdentifier(Collections.emptyList(), pattern); | ||
} | ||
|
||
public static KafkaDatasetIdentifier of(List<String> fixedTopics) { | ||
return new KafkaDatasetIdentifier(fixedTopics, null); | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
KafkaDatasetIdentifier that = (KafkaDatasetIdentifier) o; | ||
return Objects.equals(topics, that.topics) | ||
&& Objects.equals(topicPattern, that.topicPattern); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(topics, topicPattern); | ||
} | ||
} | ||
} |
59 changes: 0 additions & 59 deletions
59
...a/src/main/java/org/apache/flink/connector/kafka/lineage/facets/KafkaPropertiesFacet.java
This file was deleted.
Oops, something went wrong.
59 changes: 0 additions & 59 deletions
59
...ka/src/main/java/org/apache/flink/connector/kafka/lineage/facets/KafkaTopicListFacet.java
This file was deleted.
Oops, something went wrong.
Oops, something went wrong.