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


##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java:
##########
@@ -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 {

Review Comment:
   Make sure to tag all public API with `@PublicEvolving`. It needs to be 
clearly visible if a user is supposed to touch the class or not (the easiest 
way is to not use public unless needed).



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java:
##########
@@ -0,0 +1,15 @@
+package org.apache.flink.connector.kafka.lineage;
+
+import java.util.Optional;
+
+/** Contains method to extract {@link TypeDatasetFacet}. */
+public interface TypeDatasetFacetProvider {
+
+    /**
+     * Returns a type dataset facet or `Optional.empty` in case an 
implementing class is not able to
+     * resolve type.
+     *
+     * @return

Review Comment:
   Please remove all empty javadoc tags or let Copilot help you ;)



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java:
##########
@@ -26,13 +29,14 @@
 
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 
 import static 
org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata;
 
 /** A subscriber for a partition set. */
-class PartitionSetSubscriber implements KafkaSubscriber {
+class PartitionSetSubscriber implements KafkaDatasetIdentifierProvider, 
KafkaSubscriber {

Review Comment:
   ```suggestion
   class PartitionSetSubscriber implements KafkaSubscriber, 
KafkaDatasetIdentifierProvider {
   ```
   
   keep it consistent



##########
flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.source;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet;
+import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier;
+import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet;
+import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider;
+import 
org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber;
+import 
org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
+import org.apache.flink.streaming.api.lineage.LineageVertex;
+import org.apache.flink.util.Collector;
+
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link KafkaSource}. */
+public class KafkaSourceTest {
+    Properties kafkaProperties;
+
+    private interface TestingKafkaSubscriber
+            extends KafkaSubscriber, KafkaDatasetIdentifierProvider {}
+
+    @BeforeEach
+    void setup() {
+        kafkaProperties = new Properties();
+        kafkaProperties.put("bootstrap.servers", "host1;host2");
+    }
+
+    @Test
+    public void 
testGetLineageVertexWhenSubscriberNotAnKafkaDatasetFacetProvider() {
+        KafkaSource source =
+                new KafkaSource(
+                        new KafkaSubscriber() {
+                            @Override
+                            public Set<TopicPartition> 
getSubscribedTopicPartitions(
+                                    AdminClient adminClient) {
+                                return null;
+                            }
+                        },
+                        null,
+                        null,
+                        Boundedness.CONTINUOUS_UNBOUNDED,
+                        null,
+                        kafkaProperties,
+                        null);
+        assertThat(source.getLineageVertex().datasets()).isEmpty();

Review Comment:
   Can you use the builder instead? That should also be less verbose.



##########
flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java:
##########
@@ -256,6 +271,132 @@ public void testSerializeRecordWithTimestamp() {
         assertThat(recordWithInvalidTimestamp.timestamp()).isNull();
     }
 
+    @Test
+    public void 
testGetLineageDatasetFacetsWhenTopicSelectorNotKafkaTopicsIdentifierProvider() {
+        SerializationSchema<String> serializationSchema = new 
SimpleStringSchema();
+        KafkaRecordSerializationSchema<String> schema =
+                KafkaRecordSerializationSchema.builder()
+                        .setTopicSelector((TopicSelector<Object>) o -> 
DEFAULT_TOPIC)
+                        .setValueSerializationSchema(serializationSchema)
+                        .setKeySerializationSchema(serializationSchema)
+                        .build();
+
+        assertThat(((KafkaDatasetFacetProvider) 
schema).getKafkaDatasetFacet()).isEmpty();

Review Comment:
   A bit more assertj-ish would be
   
   ```
           assertThat(schema)
                   
.asInstanceOf(InstanceOfAssertFactories.type(KafkaDatasetFacetProvider.class))
                   .returns(List.of(), 
KafkaDatasetFacetProvider::getKafkaDatasetFacet);
   ```
   
   That would result in an assertion error instead of runtime error if the 
Schema does not implement the interface.



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java:
##########
@@ -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) {

Review Comment:
   ```suggestion
       public DefaultKafkaDatasetIdentifier(@Nullable List<String> fixedTopics, 
@Nullable Pattern topicPattern) {
   ```
   
   Just try to be as consistent as possible.



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/** Utility class with useful methods for managing lineage objects. */
+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 datasetOf(namespace, kafkaDatasetFacet, 
Collections.emptyList());
+    }
+
+    public static LineageDataset datasetOf(
+            String namespace, KafkaDatasetFacet kafkaDatasetFacet, 
TypeDatasetFacet typeFacet) {
+        return datasetOf(namespace, kafkaDatasetFacet, 
Collections.singletonList(typeFacet));
+    }
+
+    private static LineageDataset datasetOf(
+            String namespace,
+            KafkaDatasetFacet kafkaDatasetFacet,
+            List<LineageDatasetFacet> facets) {
+        return new LineageDataset() {
+            @Override
+            public String name() {
+                return kafkaDatasetFacet.getTopicIdentifier().toLineageName();
+            }
+
+            @Override
+            public String namespace() {
+                return namespace;
+            }
+
+            @Override
+            public Map<String, LineageDatasetFacet> facets() {
+                Map facetMap = new HashMap<String, LineageDatasetFacet>();
+                facetMap.put(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME, 
kafkaDatasetFacet);
+                facetMap.putAll(
+                        facets.stream()
+                                .collect(
+                                        
Collectors.toMap(LineageDatasetFacet::name, item -> item)));
+                return facetMap;
+            }
+        };
+    }
+
+    public static String namespaceOf(Properties properties) {
+        String bootstrapServers = properties.getProperty("bootstrap.servers");
+
+        if (bootstrapServers == null) {
+            return KAFKA_DATASET_PREFIX;
+        }
+
+        if (bootstrapServers.contains(COMMA)) {
+            bootstrapServers = bootstrapServers.split(COMMA)[0];
+        } else if (bootstrapServers.contains(SEMICOLON)) {
+            bootstrapServers = bootstrapServers.split(SEMICOLON)[0];
+        }

Review Comment:
   Can you check if there is already some util in kafka that does that? If not, 
leave as is.



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/** Utility class with useful methods for managing lineage objects. */
+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 datasetOf(namespace, kafkaDatasetFacet, 
Collections.emptyList());
+    }
+
+    public static LineageDataset datasetOf(
+            String namespace, KafkaDatasetFacet kafkaDatasetFacet, 
TypeDatasetFacet typeFacet) {
+        return datasetOf(namespace, kafkaDatasetFacet, 
Collections.singletonList(typeFacet));
+    }
+
+    private static LineageDataset datasetOf(
+            String namespace,
+            KafkaDatasetFacet kafkaDatasetFacet,
+            List<LineageDatasetFacet> facets) {
+        return new LineageDataset() {
+            @Override
+            public String name() {
+                return kafkaDatasetFacet.getTopicIdentifier().toLineageName();
+            }
+
+            @Override
+            public String namespace() {
+                return namespace;
+            }
+
+            @Override
+            public Map<String, LineageDatasetFacet> facets() {
+                Map facetMap = new HashMap<String, LineageDatasetFacet>();
+                facetMap.put(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME, 
kafkaDatasetFacet);
+                facetMap.putAll(
+                        facets.stream()
+                                .collect(
+                                        
Collectors.toMap(LineageDatasetFacet::name, item -> item)));
+                return facetMap;
+            }
+        };
+    }
+
+    public static String namespaceOf(Properties properties) {
+        String bootstrapServers = properties.getProperty("bootstrap.servers");
+
+        if (bootstrapServers == null) {
+            return KAFKA_DATASET_PREFIX;
+        }
+
+        if (bootstrapServers.contains(COMMA)) {
+            bootstrapServers = bootstrapServers.split(COMMA)[0];
+        } else if (bootstrapServers.contains(SEMICOLON)) {
+            bootstrapServers = bootstrapServers.split(SEMICOLON)[0];
+        }
+
+        return String.format(KAFKA_DATASET_PREFIX + bootstrapServers);
+    }
+
+    public static SourceLineageVertex 
sourceLineageVertexOf(Collection<LineageDataset> datasets) {
+        return new SourceLineageVertex() {
+            @Override
+            public Boundedness boundedness() {
+                return Boundedness.CONTINUOUS_UNBOUNDED;
+            }
+
+            @Override
+            public List<LineageDataset> datasets() {
+                return datasets.stream().collect(Collectors.toList());

Review Comment:
   ```suggestion
                   return List.copyOf(datasets);
   ```



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java:
##########
@@ -79,6 +95,7 @@
  */
 @PublicEvolving
 public class KafkaRecordSerializationSchemaBuilder<IN> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KafkaSource.class);

Review Comment:
   Not the correct place, but please update the docs of the 
KafkaRecordSerializationSchema to point to the FacetProvider interface. Same to 
all other APIs where you hope that optional interfaces are implemented.



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