AndrewJSchofield commented on code in PR #22209:
URL: https://github.com/apache/kafka/pull/22209#discussion_r3189227514


##########
server-common/src/main/java/org/apache/kafka/server/share/dlq/DefaultShareGroupDLQManager.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.kafka.server.share.dlq;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The default share group DLQ manager responsible for processing
+ * incoming messages and writing them to the appropriate dlq topic.
+ */
+public class DefaultShareGroupDLQManager implements ShareGroupDLQ {

Review Comment:
   I wonder whether `ShareGroupDLQ` should actually be called 
`ShareGroupDLQManager`.



##########
server-common/src/main/java/org/apache/kafka/server/share/dlq/DefaultShareGroupDLQManager.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.kafka.server.share.dlq;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The default share group DLQ manager responsible for processing
+ * incoming messages and writing them to the appropriate dlq topic.
+ */
+public class DefaultShareGroupDLQManager implements ShareGroupDLQ {
+    /**
+     * Reference to state manager responsible for actually sending
+     * the relevant RPCs and writing records.
+     */
+    private final ShareGroupDLQStateManager stateManager;
+
+    private static final Logger log = 
LoggerFactory.getLogger(DefaultShareGroupDLQManager.class);
+
+    public DefaultShareGroupDLQManager(ShareGroupDLQStateManager stateManager) 
{
+        this.stateManager = stateManager;
+        this.stateManager.start();

Review Comment:
   Generally I prefer not to start another component during the constructor. 
You're heading in the direction of a this-escape I feel.



##########
server-common/src/main/java/org/apache/kafka/server/share/dlq/ShareGroupDLQStateManager.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.kafka.server.share.dlq;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.RequestCompletionHandler;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import 
org.apache.kafka.server.share.persister.ShareCoordinatorMetadataCacheHelper;
+import org.apache.kafka.server.util.InterBrokerSendThread;
+import org.apache.kafka.server.util.RequestAndCompletionHandler;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Core implementation of RPC send logic for the dlq manager.
+ * This class allows for enqueuing records meant to be DLQ'ed
+ * and manages various RPC which are to be sent to the KafkaApis.
+ * These RPCs include PRODUCE, CREATE_TOPIC.
+ */
+public class ShareGroupDLQStateManager {
+    private final AtomicBoolean isStarted = new AtomicBoolean(false);
+    private final SendThread sender;
+    private final Time time;
+    private final Timer timer;
+    private final ShareCoordinatorMetadataCacheHelper cacheHelper;
+
+    public enum RPCType {
+        PRODUCE,
+        CREATE_TOPIC
+    }
+
+    public ShareGroupDLQStateManager(KafkaClient client, 
ShareCoordinatorMetadataCacheHelper cacheHelper, Time time, Timer timer) {
+        if (client == null) {
+            throw new IllegalArgumentException("Kafkaclient must not be 
null.");
+        }
+
+        if (cacheHelper == null) {
+            throw new IllegalArgumentException("Cache helper must not be 
null.");
+        }
+
+        if (time == null) {
+            throw new IllegalArgumentException("Time must not be null.");
+        }
+
+        if (timer == null) {
+            throw new IllegalArgumentException("Timer must not be null.");
+        }
+
+        this.time = time;
+        this.timer = timer;
+        this.cacheHelper = cacheHelper;
+        this.sender = new SendThread(
+            "ShareGroupDLQSendThread",
+            client,
+            
Math.toIntExact(CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS),
  //30 seconds
+            this.time,
+            true,
+            new Random(this.time.milliseconds())
+        );
+    }
+
+    public void start() {
+        if (isStarted.compareAndSet(false, true)) {
+            this.sender.start();
+            isStarted.set(true);
+        }
+    }
+
+    public void stop() throws Exception {
+        if (isStarted.compareAndSet(true, false)) {
+            this.sender.shutdown();
+        }
+    }
+
+    /**
+     * The main method which enqueues a method to be DLQ'ed.

Review Comment:
   "Enqueues a method" seems a bit strange. I'm not sure a method is being 
enqueued really.



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