fapaul commented on code in PR #152: URL: https://github.com/apache/flink-connector-kafka/pull/152#discussion_r1956378108
########## flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java: ########## @@ -359,14 +356,14 @@ void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exce assertThat(writer.getCurrentProducer() == firstProducer) .as("Expected same producer") .isTrue(); - assertThat(writer.getProducerPool()).hasSize(0); } } /** Test that producers are reused when committed. */ @Test void usePooledProducerForTransactional() throws Exception { - try (final KafkaWriter<Integer> writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + try (final ExactlyOnceKafkaWriter<Integer> writer = Review Comment: Nit: Now that we have a dedicated EOS writer we can also move the involved tests into a separate class. ########## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.sink.internal; + +import org.apache.flink.annotation.VisibleForTesting; + +import java.io.Closeable; +import java.util.function.Consumer; + +/** + * A backchannel for communication between the commiter -> writer. It's used to signal that certain + * transactions have been committed and respective producers are good to be reused. + * + * <p>The model closely follows the idea of statefun except that there is no need to checkpoint the + * state since the backchannel will fully recover on restart from the committer state. + * + * <p>Establishing a backchannel for Kafka sink works because there is only writer and committer and + * nothing in between these two operators. In most cases, these two are chained in live inside the + * same task thread. In rare cases, committer and writer are not chained, so writer and committer + * are in different tasks and threads. However, because of colocations of tasks, we still know that + * both instances will run inside the same JVM and we can establish a backchannel between them. The + * latter case requires some synchronization in the buffer. + * + * <p>Messages can be sent before the backchannel is established. They will be consumed once the + * backchannel is established. + */ +public interface Backchannel<T> extends Closeable { + /** + * Send a message to the other side of the backchannel. Currently, this is the transaction id of + * a committed transaction. + */ + void send(T message); + + /** + * Consume messages from the other side of the backchannel. This is used to signal that certain + * transactions have been committed, so that the writer can recycle the producer. + */ + void consume(Consumer<T> consumer); Review Comment: `drainTo` sounds good to me 👍 ########## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java: ########## @@ -0,0 +1,50 @@ +/* + * 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.sink.internal; + +import java.io.Closeable; + +/** + * The writable portion of a backchannel for communication between the commiter -> writer. It's used + * to signal that certain transactions have been committed and respective producers are good to be + * reused. + * + * <p>The model closely follows the idea of statefun except that there is no need to checkpoint the + * state since the backchannel will fully recover on restart from the committer state. + * + * <p>Establishing a backchannel for Kafka sink works because there is only writer and committer and + * nothing in between these two operators. In most cases, these two are chained in live inside the + * same task thread. In rare cases, committer and writer are not chained, so writer and committer + * are in different tasks and threads. However, because of colocations of sink tasks via {@link + * org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup}, we still know that both instances + * will run inside the same JVM and we can establish a backchannel between them. + * + * <p>Messages can be sent before the backchannel is established. They will be consumed once the + * backchannel is established. + */ +public interface WritableBackchannel<T> extends Closeable { Review Comment: Nit: WDYT about having one `BackChannel` interface that Readable and Writeable extend with `send` and poll ########## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java: ########## @@ -0,0 +1,123 @@ +/* + * 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.sink.internal; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple3; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +/** + * Creates and manages backchannels for the Kafka sink. The backchannels are used to communicate + * between the Kafka committer and writer. + * + * <p>Each backchannel is uniquely identified by the subtask id, attempt number, and transactional + * id prefix. The subtask id prevents concurrent subtasks of the same sink from conflicting. The + * attempt number prevents conflicts for task-local recovery. The transactional id prefix prevents + * conflicts between multiple Kafka sinks. + */ +@Internal +@ThreadSafe +public class BackchannelFactory { + private static final BackchannelFactory INSTANCE = new BackchannelFactory(); + + /** Gets the singleton instance of the {@link BackchannelFactory}. */ + public static BackchannelFactory getInstance() { + return INSTANCE; + } + + private BackchannelFactory() {} + + /** + * The map of backchannels, keyed by the subtask id and transactional id prefix to uniquely + * identify the backchannel while establishing the connection. + */ + private final Map<Tuple3<Integer, Integer, String>, BackchannelImpl<?>> backchannels = + new ConcurrentHashMap<>(); + + /** + * Gets a {@link ReadableBackchannel} for the given subtask, attempt, and transactional id + * prefix. + * + * <p>If this method is called twice with the same arguments, it will throw an exception as it + * indicates that the transactional id prefix is being reused for multiple Kafka sinks. + * + * <p>If the corresponding {@link #getWritableBackchannel(int, int, String)} is called, the + * {@link ReadableBackchannel#isEstablished()} will return true. + */ + @SuppressWarnings("unchecked") + public <T> ReadableBackchannel<T> getReadableBackchannel( + int subtaskId, int attemptNumber, String transactionalIdPrefix) { + return (ReadableBackchannel<T>) + getBackchannel( + subtaskId, + attemptNumber, Review Comment: Is the attempt idea used for the local recovery cases? ########## flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java: ########## @@ -0,0 +1,163 @@ +/* + * 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.sink.internal; + +import org.apache.flink.util.function.RunnableWithException; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinTask; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +class BackchannelImplTest { Review Comment: The duplicate test is definitely a good one but I was more referring to verifying the semantics of of `drainTo` that the Consumers are not registered with the backchannel but never invoked again once all messages for a single call are consumed. ########## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.sink.internal; + +import org.apache.flink.annotation.VisibleForTesting; + +import java.io.Closeable; +import java.util.function.Consumer; + +/** + * A backchannel for communication between the commiter -> writer. It's used to signal that certain + * transactions have been committed and respective producers are good to be reused. + * + * <p>The model closely follows the idea of statefun except that there is no need to checkpoint the + * state since the backchannel will fully recover on restart from the committer state. + * + * <p>Establishing a backchannel for Kafka sink works because there is only writer and committer and + * nothing in between these two operators. In most cases, these two are chained in live inside the + * same task thread. In rare cases, committer and writer are not chained, so writer and committer + * are in different tasks and threads. However, because of colocations of tasks, we still know that Review Comment: I understand the colocation implementation, but do we need to ensure that committer/writer are colocated, Afaict the colocation key is set on the transformation how does that work for the sink? -- 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