pawel-big-lebowski commented on code in PR #130:
URL: 
https://github.com/apache/flink-connector-kafka/pull/130#discussion_r1802454165


##########
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:
   The difficulty of this approach is that `KafkaFacet` properties are 
collected in different classes and this is currently done with 
`LineageFacetProvider` having a method `Collection<LineageDatasetFacet> 
getDatasetFacets()`. 
   
   A solution to this would be to create `KafkaFacetProvider` interfaces 
(instead of `LineageFacetProvider`) with a method:
   ```
   void buildKafkaFacet(KafkaFacetBuilder builder) 
   ```
   This would pass Kafka facet builder as an argument and let facet being 
enriched within the method calls. 
   
   @AHeise  Is this something you had on your mind? 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to