You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/12/18 01:59:18 UTC

[GitHub] [kafka] satishd opened a new pull request, #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

satishd opened a new pull request, #13013:
URL: https://github.com/apache/kafka/pull/13013

   *KAFKA-14466 Move `ClassloaderAwareRemoteStorageManager` to storage module
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
satishd commented on PR #13013:
URL: https://github.com/apache/kafka/pull/13013#issuecomment-1357201725

   Thanks @ijuma for your review. Addressed them with the latest commit. 


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051966825


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAction.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.log.remote.storage;
+
+/**
+ * This interface is used to execute any remote storage/metadata related operation.
+ *
+ * @param <T> return type for execute operation
+ * @param <E> Exception type to be thrown
+ */
+@FunctionalInterface
+public interface ClassLoaderAction<T, E extends Exception> {

Review Comment:
   Yeah, my original suggestion assumed it would be used within one class. I didn't realize we had multiple "ClassLoaderAware" classes. Once it's a top level interface, it seems better to keep it general. Thanks.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051923726


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAction.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.log.remote.storage;
+
+/**
+ * This interface is used to execute any remote storage/metadata related operation.
+ *
+ * @param <T> return type for execute operation
+ * @param <E> Exception type to be thrown
+ */
+@FunctionalInterface
+public interface ClassLoaderAction<T, E extends Exception> {

Review Comment:
   I hadn't realised this is used by multiple classes. Since it is, maybe we can move this to the `internals` package and call it simply `Action` or something like that. I can then use it within one of the index classes top.



##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAction.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.log.remote.storage;
+
+/**
+ * This interface is used to execute any remote storage/metadata related operation.
+ *
+ * @param <T> return type for execute operation
+ * @param <E> Exception type to be thrown
+ */
+@FunctionalInterface
+public interface ClassLoaderAction<T, E extends Exception> {

Review Comment:
   I hadn't realised this is used by multiple classes. Since it is, maybe we can move this to the `internals` package and call it simply `Action` or something like that. I can then use it within one of the index classes too.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
satishd commented on PR #13013:
URL: https://github.com/apache/kafka/pull/13013#issuecomment-1356626291

   @junrao Please review when you get a chance. 


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051924558


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAction.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.log.remote.storage;
+
+/**
+ * This interface is used to execute any remote storage/metadata related operation.

Review Comment:
   Let's update this comment to make this general to any `storage` action.



##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAction.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.log.remote.storage;
+
+/**
+ * This interface is used to execute any remote storage/metadata related operation.
+ *
+ * @param <T> return type for execute operation
+ * @param <E> Exception type to be thrown
+ */
+@FunctionalInterface
+public interface ClassLoaderAction<T, E extends Exception> {

Review Comment:
   I hadn't realised this is used by multiple classes. Since it is, maybe we can move this to the `internals` package and call it simply `StorageAction` or something like that. I can then use it within one of the index classes too.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051520754


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAwareRemoteStorageManager.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.log.remote.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+/**
+ * A wrapper class of {@link RemoteStorageManager} that sets the context class loader when calling the respective
+ * methods.
+ */
+public class ClassLoaderAwareRemoteStorageManager implements RemoteStorageManager {
+
+    private final RemoteStorageManager delegate;
+    private final ClassLoader rsmClassLoader;
+
+    public ClassLoaderAwareRemoteStorageManager(RemoteStorageManager rsm, ClassLoader rsmClassLoader) {
+        this.delegate = rsm;
+        this.rsmClassLoader = rsmClassLoader;
+    }
+
+    public RemoteStorageManager delegate() {
+        return delegate;
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(rsmClassLoader);
+        try {
+            delegate.configure(configs);
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(rsmClassLoader);
+        try {
+            delegate.close();
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+
+    private <T> T withClassLoader(RemoteStorageAction<T> action) throws RemoteStorageException {

Review Comment:
   Perhaps a better option would be an interface like:
   
   ```java
   interface ClassLoaderAction<T, E extends Exception> {
       T execute() throws E;
   }
   ```
   
   Then you can use it from `close` too.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051520754


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAwareRemoteStorageManager.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.log.remote.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+/**
+ * A wrapper class of {@link RemoteStorageManager} that sets the context class loader when calling the respective
+ * methods.
+ */
+public class ClassLoaderAwareRemoteStorageManager implements RemoteStorageManager {
+
+    private final RemoteStorageManager delegate;
+    private final ClassLoader rsmClassLoader;
+
+    public ClassLoaderAwareRemoteStorageManager(RemoteStorageManager rsm, ClassLoader rsmClassLoader) {
+        this.delegate = rsm;
+        this.rsmClassLoader = rsmClassLoader;
+    }
+
+    public RemoteStorageManager delegate() {
+        return delegate;
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(rsmClassLoader);
+        try {
+            delegate.configure(configs);
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(rsmClassLoader);
+        try {
+            delegate.close();
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+
+    private <T> T withClassLoader(RemoteStorageAction<T> action) throws RemoteStorageException {

Review Comment:
   Perhaps a better option would be an interface like:
   
   ```java
   interface ClassLoaderAction<T, E extends Exception> {
       T act() throws E;
   }
   ```
   
   Then you can use it from `close` too.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13013:
URL: https://github.com/apache/kafka/pull/13013#issuecomment-1357765376

   Test failures are unrelated.


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma merged pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma merged PR #13013:
URL: https://github.com/apache/kafka/pull/13013


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
satishd commented on PR #13013:
URL: https://github.com/apache/kafka/pull/13013#issuecomment-1357351385

   Thanks @ijuma for your review. Addressed them with the latest commit.


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13013:
URL: https://github.com/apache/kafka/pull/13013#issuecomment-1357360605

   Thanks. I'll take a look soon.


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051962413


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAction.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.log.remote.storage;
+
+/**
+ * This interface is used to execute any remote storage/metadata related operation.
+ *
+ * @param <T> return type for execute operation
+ * @param <E> Exception type to be thrown
+ */
+@FunctionalInterface
+public interface ClassLoaderAction<T, E extends Exception> {

Review Comment:
   I set the name as `RemoteStorageAction` as it was meant for any remote storage actions. I though you wanted this to be classloader specific in your earlier comment. I am fine with renaming it to `StorageAction` so that it can be reused for any storage specific operations. 



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] ijuma commented on a diff in pull request #13013: KAFKA-14466 Move ClassloaderAwareRemoteStorageManager to storage module

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13013:
URL: https://github.com/apache/kafka/pull/13013#discussion_r1051520754


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAwareRemoteStorageManager.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.log.remote.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+/**
+ * A wrapper class of {@link RemoteStorageManager} that sets the context class loader when calling the respective
+ * methods.
+ */
+public class ClassLoaderAwareRemoteStorageManager implements RemoteStorageManager {
+
+    private final RemoteStorageManager delegate;
+    private final ClassLoader rsmClassLoader;
+
+    public ClassLoaderAwareRemoteStorageManager(RemoteStorageManager rsm, ClassLoader rsmClassLoader) {
+        this.delegate = rsm;
+        this.rsmClassLoader = rsmClassLoader;
+    }
+
+    public RemoteStorageManager delegate() {
+        return delegate;
+    }
+
+    @Override
+    public void configure(Map<String, ?> configs) {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(rsmClassLoader);
+        try {
+            delegate.configure(configs);
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(rsmClassLoader);
+        try {
+            delegate.close();
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+
+    private <T> T withClassLoader(RemoteStorageAction<T> action) throws RemoteStorageException {

Review Comment:
   Perhaps a better option would be an interface like:
   
   ```java
   interface ClassLoaderAction<T, E extends Exception> {
       T call() throws E;
   }
   ```
   
   Then you can use it from `close` too.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org