AHeise commented on code in PR #130:
URL: 
https://github.com/apache/flink-connector-kafka/pull/130#discussion_r1803045472


##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.connector.kafka.lineage;
+
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.connector.kafka.lineage.facets.KafkaTopicListFacet;
+import org.apache.flink.connector.kafka.lineage.facets.KafkaTopicPatternFacet;
+import org.apache.flink.streaming.api.lineage.LineageDataset;
+import org.apache.flink.streaming.api.lineage.LineageDatasetFacet;
+import org.apache.flink.streaming.api.lineage.LineageVertex;
+import org.apache.flink.streaming.api.lineage.SourceLineageVertex;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/** Utility class with useful methods for managing dataset facets. */
+public class LineageUtil {
+
+    private static final String KAFKA_DATASET_PREFIX = "kafka://";
+    private static final String COMMA = ",";
+    private static final String SEMICOLON = ";";
+
+    /**
+     * Loads facet from any object implementing @link{DatasetFacetProvider} 
interface.
+     *
+     * @param object
+     * @return
+     */
+    public static List<LineageDatasetFacet> facetsFrom(Object object) {
+        return Optional.of(object)
+                .filter(LineageFacetProvider.class::isInstance)
+                .map(LineageFacetProvider.class::cast)
+                .map(LineageFacetProvider::getDatasetFacets)
+                .orElse(Collections.emptyList());
+    }
+
+    /**
+     * Creates dataset from a list of facets. Uses {@link KafkaTopicListFacet} 
to extract dataset
+     * name from. Dataset per each element of topic list is created
+     *
+     * @param facets
+     * @return
+     */
+    public static List<LineageDataset> datasetsFrom(

Review Comment:
   Hm I have not fully understood from which classes we actually need to 
extract the various data points. Could we recap here?
   - Source/Sink gives us the properties directly
   - Source gives us the type information directly but we also try to extract 
it from the deserialization schema (why?).
   - KafkaSubscriber of the source either gives us a topicPattern or a 
topicList.
   - SerializationSchema of the sink gives us the topicList
   
   In the end, we emit a lineageVertex that has facets per topic (pattern) in 
some cross-product fashion. I have not fully understood how a given input looks 
fully expanded after `datasetsFrom`. Maybe you could summarize that.
   
   Anyways, it feels like the KafkaFacet contains a list of topics that is 
filled through polymorphism and some parts that are filled statically. Can we 
maybe separate that? Would we be able to say that the topic selector/subscriber 
just return a list of facet names and we use them to create the facets with the 
statically set properties and type information?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to