Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,14 @@ public class Topic {
public static final String TRANSACTION_STATE_TOPIC_NAME = "__transaction_state";
public static final String SHARE_GROUP_STATE_TOPIC_NAME = "__share_group_state";
public static final String CLUSTER_METADATA_TOPIC_NAME = "__cluster_metadata";
public static final String REMOTE_LOG_METADATA_TOPIC_NAME = "__remote_log_metadata";
public static final TopicPartition CLUSTER_METADATA_TOPIC_PARTITION = new TopicPartition(
CLUSTER_METADATA_TOPIC_NAME,
0
);
public static final String LEGAL_CHARS = "[a-zA-Z0-9._-]";

private static final Set<String> INTERNAL_TOPICS = Set.of(GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME);
private static final Set<String> INTERNAL_TOPICS = Set.of(GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME, REMOTE_LOG_METADATA_TOPIC_NAME);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe internal topics are those fundamental to kafka's core and are handled uniquely, not via standard producer/consumer logic. Therefore, REMOTE_LOG_METADATA_TOPIC_NAME is similar to connector topics rather than an internal topic

@AndrewJSchofield @jiafu1115 WDYT?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You may be correct, but I think the main difference is that internal topics are filtered out of listing topics by default. I do think that we need to tread carefully here. If there is any doubt about making this an internal topic, we should not do it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AndrewJSchofield @chia7712 I agree. I just created this PR as an example to show what the code changes would look like if we decided to proceed. I think we can leave it here for future discussion: It involve how we define the internal topic: not allow to see or not allow to be write by standard producer.
Thanks


private static final int MAX_NAME_LENGTH = 249;

Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/KafkaApis.scala
Original file line number Diff line number Diff line change
Expand Up @@ -529,7 +529,7 @@ class KafkaApis(val requestChannel: RequestChannel,
if (authorizedRequestInfo.isEmpty)
sendResponseCallback(Map.empty)
else {
val internalTopicsAllowed = request.header.clientId == "__admin_client"
val internalTopicsAllowed = request.header.clientId != null && request.header.clientId.startsWith("__")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should not touch this "backdoor" - I have posted a discussion on the https://issues.apache.org/jira/browse/KAFKA-5246

val transactionSupportedOperation = AddPartitionsToTxnManager.produceRequestVersionToTransactionSupportedOperation(request.header.apiVersion())
// call the replica manager to append messages to the replicas
replicaManager.handleProduceAppend(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;

Expand All @@ -41,7 +42,7 @@
*/
public final class TopicBasedRemoteLogMetadataManagerConfig {

public static final String REMOTE_LOG_METADATA_TOPIC_NAME = "__remote_log_metadata";
public static final String REMOTE_LOG_METADATA_TOPIC_NAME = Topic.REMOTE_LOG_METADATA_TOPIC_NAME;

public static final String REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP = "remote.log.metadata.topic.replication.factor";
public static final String REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP = "remote.log.metadata.topic.num.partitions";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2634,7 +2634,6 @@ public static boolean isRemoteLogEnabled(boolean remoteStorageSystemEnable, LogC
// Remote log is enabled only for non-compact and non-internal topics
return remoteStorageSystemEnable &&
!(config.compact || Topic.isInternal(topic)
|| TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME.equals(topic)
|| Topic.CLUSTER_METADATA_TOPIC_NAME.equals(topic)) &&
config.remoteStorageEnable();
}
Expand Down