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


##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.KafkaDatasetFacet;
+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.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+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 = ";";
+
+    public static LineageDataset datasetOf(String namespace, KafkaDatasetFacet 
kafkaDatasetFacet) {
+        return new LineageDataset() {
+            @Override
+            public String name() {
+                if (kafkaDatasetFacet.topicIdentifier.topicPattern != null) {
+                    return kafkaDatasetFacet.topicIdentifier.toString();
+                }
+
+                return String.join(",", 
kafkaDatasetFacet.topicIdentifier.topics);
+            }
+
+            @Override
+            public String namespace() {
+                return namespace;
+            }
+
+            @Override
+            public Map<String, LineageDatasetFacet> facets() {
+                return Collections.singletonMap(
+                        KafkaDatasetFacet.KAFKA_FACET_NAME, kafkaDatasetFacet);

Review Comment:
   Duplicates in metadata aint that bad thing. Kafka facet contains topic 
pattern or topics' list which are necessary to identify datasets and name them 
properly. On the other hand, the name field is obligatory, which makes sense 
for other datasets where naming is more straight-forward. Having only the name 
field wouldn't allow distinction between those scenarios.
   
   A separate `TypeInformationFacet` makes sense as it clearly describes some 
aspect of a dataset. On the other hand, I feel Kafka properties should be a 
part of `KafkaDatasetFacet` bcz it's Kafka specific and bcz the properties 
allow resolving topics' pattern, so it's worth to have them both in a single 
facet.
   
   What would make most sense to me now, is having two facets: 
`TypeInformationFacet` for type information and `KafkaDatasetFacet` to contain: 
Kafka properties, topics list and topic pattern in case it's present. Does it 
make sense to you as well?



-- 
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