Skip to content

Commit

Permalink
[admin-tool] Add support for dumping topic switch message details (#1341
Browse files Browse the repository at this point in the history
)

Add support to dump details of TopicSwitch messages via the admin tool.  
This will help in debugging and root cause analysis of ingestion issues.
  • Loading branch information
sushantmane authored Nov 25, 2024
1 parent 1735a63 commit aa1ccf7
Show file tree
Hide file tree
Showing 5 changed files with 127 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1661,6 +1661,7 @@ private static void dumpKafkaTopic(CommandLine cmd, PubSubClientsFactory pubSubC
boolean logMetadata = cmd.hasOption(Arg.LOG_METADATA.toString());
boolean logDataRecord = cmd.hasOption(Arg.LOG_DATA_RECORD.toString());
boolean logRmdRecord = cmd.hasOption(Arg.LOG_RMD_RECORD.toString());
boolean logTsRecord = cmd.hasOption(Arg.LOG_TS_RECORD.toString());
try (PubSubConsumerAdapter consumer = getConsumer(consumerProps, pubSubClientsFactory)) {
try (KafkaTopicDumper ktd = new KafkaTopicDumper(
controllerClient,
Expand All @@ -1673,7 +1674,8 @@ private static void dumpKafkaTopic(CommandLine cmd, PubSubClientsFactory pubSubC
maxConsumeAttempts,
logMetadata,
logDataRecord,
logRmdRecord)) {
logRmdRecord,
logTsRecord)) {
ktd.fetchAndProcess();
} catch (Exception e) {
System.err.println("Something went wrong during topic dump");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@ public enum Arg {
LOG_METADATA("log-metadata", "lm", false, "Log the metadata for each kafka message on console"),
LOG_DATA_RECORD("log-data-record", "ldr", false, "Log the data record for each kafka message on console"),
LOG_RMD_RECORD("log-rmd-record", "lrr", false, "Log the RMD record for each kafka message on console"),
LOG_TS_RECORD("log-ts-record", "lts", false, "Log the topic switch message on console"),
NATIVE_REPLICATION_SOURCE_FABRIC(
"native-replication-source-fabric", "nrsf", true,
"The source fabric name to be used in native replication. Remote consumption will happen from kafka in this fabric."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.linkedin.venice.kafka.protocol.LeaderMetadata;
import com.linkedin.venice.kafka.protocol.ProducerMetadata;
import com.linkedin.venice.kafka.protocol.Put;
import com.linkedin.venice.kafka.protocol.TopicSwitch;
import com.linkedin.venice.kafka.protocol.Update;
import com.linkedin.venice.kafka.protocol.enums.ControlMessageType;
import com.linkedin.venice.kafka.protocol.enums.MessageType;
Expand Down Expand Up @@ -109,6 +110,7 @@ public class KafkaTopicDumper implements AutoCloseable {
private final boolean logMetadata;
private final boolean logDataRecord;
private final boolean logRmdRecord;
private final boolean logTsRecord;

private final ChunkKeyValueTransformer chunkKeyValueTransformer;
private final AvroSpecificDeserializer<ChunkedKeySuffix> chunkedKeySuffixDeserializer;
Expand All @@ -132,7 +134,8 @@ public KafkaTopicDumper(
int maxConsumeAttempts,
boolean logMetadata,
boolean logDataRecord,
boolean logRmdRecord) {
boolean logRmdRecord,
boolean logTsRecord) {
this.consumer = consumer;
this.maxConsumeAttempts = maxConsumeAttempts;

Expand Down Expand Up @@ -186,6 +189,7 @@ public KafkaTopicDumper(
this.logMetadata = logMetadata;
this.logDataRecord = logDataRecord;
this.logRmdRecord = logRmdRecord;
this.logTsRecord = logTsRecord;

if (logMetadata && !logDataRecord) {
this.latestValueSchemaStr = null;
Expand Down Expand Up @@ -347,7 +351,7 @@ private void logRecordMetadata(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Lon
final String chunkMetadata = getChunkMetadataLog(record);

LOGGER.info(
"[Record Metadata] Offset:{}; {}; {}; ProducerMd=(guid:{},seg:{},seq:{},mts:{},lts:{}); LeaderMd=(host:{},uo:{},ukcId:{}){}",
"Offset:{}; {}; {}; ProducerMd=(guid:{},seg:{},seq:{},mts:{},lts:{}); LeaderMd=(host:{},uo:{},ukcId:{}){}",
record.getOffset(),
kafkaKey.isControlMessage() ? CONTROL_REC : REGULAR_REC,
msgType,
Expand Down Expand Up @@ -454,7 +458,9 @@ String buildDataRecordLog(
}

private void processRecord(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record) {
if (logDataRecord) {
if (logTsRecord) {
logIfTopicSwitchMessage(record);
} else if (logDataRecord) {
logDataRecord(record, logMetadata, logRmdRecord);
} else if (logMetadata) {
logRecordMetadata(record);
Expand All @@ -464,6 +470,53 @@ private void processRecord(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> r
}
}

static void logIfTopicSwitchMessage(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record) {
KafkaKey kafkaKey = record.getKey();
if (!kafkaKey.isControlMessage()) {
// TS message is a control message, so we only care about control messages.
return;
}

ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion;
if (controlMessage.controlMessageType != ControlMessageType.TOPIC_SWITCH.getValue()) {
return;
}

String logMessage = constructTopicSwitchLog(record);
LOGGER.info(logMessage);
}

/**
* Constructs the log message for a TopicSwitch message.
*
* @param record The PubSubMessage containing the TopicSwitch message.
* @return A formatted string representing the log message.
*/
static String constructTopicSwitchLog(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record) {
KafkaMessageEnvelope kafkaMessageEnvelope = record.getValue();
ProducerMetadata producerMetadata = kafkaMessageEnvelope.producerMetadata;
LeaderMetadata leaderMetadata = kafkaMessageEnvelope.leaderMetadataFooter;
ControlMessage controlMessage = (ControlMessage) kafkaMessageEnvelope.payloadUnion;
TopicSwitch topicSwitch = (TopicSwitch) controlMessage.controlMessageUnion;

return String.format(
"Offset:%s; %s; SourceKafkaServers: %s; SourceTopicName: %s; RewindStartTimestamp: %s; "
+ "ProducerMd=(guid:%s,seg:%s,seq:%s,mts:%s,lts:%s); LeaderMd=(host:%s,uo:%s,ukcId:%s)",
record.getOffset(),
ControlMessageType.TOPIC_SWITCH.name(),
topicSwitch.sourceKafkaServers,
topicSwitch.sourceTopicName,
topicSwitch.rewindStartTimestamp,
GuidUtils.getHexFromGuid(producerMetadata.producerGUID),
producerMetadata.segmentNumber,
producerMetadata.messageSequenceNumber,
producerMetadata.messageTimestamp,
producerMetadata.logicalTimestamp,
leaderMetadata == null ? "-" : leaderMetadata.hostName,
leaderMetadata == null ? "-" : leaderMetadata.upstreamOffset,
leaderMetadata == null ? "-" : leaderMetadata.upstreamKafkaClusterId);
}

private void writeToFile(PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> record) {
try {
KafkaKey kafkaKey = record.getKey();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,7 @@ public void testAdminToolConsumption() {
3,
true,
false,
false,
false);
Assert.assertEquals(kafkaTopicDumper.fetchAndProcess(), consumedMessageCount);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,16 +2,26 @@

import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;

import com.linkedin.venice.compression.CompressionStrategy;
import com.linkedin.venice.controllerapi.ControllerClient;
import com.linkedin.venice.controllerapi.MultiSchemaResponse;
import com.linkedin.venice.controllerapi.SchemaResponse;
import com.linkedin.venice.controllerapi.StoreResponse;
import com.linkedin.venice.kafka.protocol.ControlMessage;
import com.linkedin.venice.kafka.protocol.GUID;
import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope;
import com.linkedin.venice.kafka.protocol.ProducerMetadata;
import com.linkedin.venice.kafka.protocol.StartOfSegment;
import com.linkedin.venice.kafka.protocol.TopicSwitch;
import com.linkedin.venice.kafka.protocol.enums.ControlMessageType;
import com.linkedin.venice.kafka.protocol.enums.MessageType;
import com.linkedin.venice.message.KafkaKey;
import com.linkedin.venice.meta.StoreInfo;
import com.linkedin.venice.meta.Version;
import com.linkedin.venice.pubsub.ImmutablePubSubMessage;
import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl;
import com.linkedin.venice.pubsub.PubSubTopicRepository;
import com.linkedin.venice.pubsub.adapter.kafka.consumer.ApacheKafkaConsumerAdapter;
Expand All @@ -23,9 +33,12 @@
import com.linkedin.venice.serializer.SerializerDeserializerFactory;
import com.linkedin.venice.utils.ChunkingTestUtils;
import com.linkedin.venice.utils.TestWriteUtils;
import com.linkedin.venice.utils.Utils;
import com.linkedin.venice.writer.update.UpdateBuilderImpl;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
Expand All @@ -35,6 +48,8 @@


public class TestKafkaTopicDumper {
private final PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository();

@Test
public void testAdminToolConsumptionForChunkedData() throws IOException {
String schemaStr = "\"string\"";
Expand All @@ -57,7 +72,6 @@ public void testAdminToolConsumptionForChunkedData() throws IOException {
when(controllerClient.getStore(storeName)).thenReturn(storeResponse);
when(storeResponse.getStore()).thenReturn(storeInfo);

PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository();
int assignedPartition = 0;
long startOffset = 0;
long endOffset = 4;
Expand All @@ -84,6 +98,7 @@ public void testAdminToolConsumptionForChunkedData() throws IOException {
3,
true,
false,
false,
false);

int numChunks = 3;
Expand Down Expand Up @@ -165,7 +180,6 @@ public void testDumpDataRecord() throws IOException {
when(controllerClient.getStore(storeName)).thenReturn(storeResponse);
when(storeResponse.getStore()).thenReturn(storeInfo);

PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository();
int assignedPartition = 0;
long startOffset = 0;
long endOffset = 4;
Expand All @@ -192,6 +206,7 @@ public void testDumpDataRecord() throws IOException {
3,
true,
true,
false,
false);

// Test different message type.
Expand Down Expand Up @@ -236,4 +251,53 @@ public void testDumpDataRecord() throws IOException {
expectedLog = String.format("Key: %s; Value: %s; Schema: %d; RMD: %s", keyString, null, 1, rmdRecord);
Assert.assertEquals(returnedLog, expectedLog);
}

@Test
public void testTopicSwitchMessageLogging() {
// Case 1: TopicSwitch message with non-null sourceKafkaServers
List<CharSequence> sourceKafkaServers = Arrays.asList("source1", "source2");
ControlMessage controlMessage = new ControlMessage();
controlMessage.controlMessageType = ControlMessageType.TOPIC_SWITCH.getValue();
TopicSwitch topicSwitch = new TopicSwitch();
topicSwitch.sourceKafkaServers = sourceKafkaServers;
topicSwitch.sourceTopicName = "test_topic_rt";
topicSwitch.rewindStartTimestamp = 123456789L;
controlMessage.controlMessageUnion = topicSwitch;
KafkaKey kafkaKey = new KafkaKey(MessageType.CONTROL_MESSAGE, Utils.getUniqueString("key-").getBytes());
KafkaMessageEnvelope messageEnvelope = new KafkaMessageEnvelope();
messageEnvelope.producerMetadata = new ProducerMetadata();
messageEnvelope.producerMetadata.messageTimestamp = 0;
messageEnvelope.producerMetadata.messageSequenceNumber = 0;
messageEnvelope.producerMetadata.segmentNumber = 0;
messageEnvelope.producerMetadata.producerGUID = new GUID();
messageEnvelope.payloadUnion = controlMessage;

PubSubTopicPartition pubSubTopicPartition =
new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic("test_topic_rt"), 0);

PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> message =
new ImmutablePubSubMessage<>(kafkaKey, messageEnvelope, pubSubTopicPartition, 120, 0, 0, null);

String actualLog = KafkaTopicDumper.constructTopicSwitchLog(message);
assertNotNull(actualLog);
assertTrue(actualLog.contains("[source1, source2]"));
assertTrue(actualLog.contains("test_topic_rt"));
assertTrue(actualLog.contains("123456789"));

// Case 2: Non TS Control message
controlMessage = new ControlMessage();
controlMessage.controlMessageType = ControlMessageType.START_OF_SEGMENT.getValue();
controlMessage.controlMessageUnion = new StartOfSegment();
messageEnvelope.payloadUnion = controlMessage;

PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> nonTsCtrlMsg =
new ImmutablePubSubMessage<>(kafkaKey, messageEnvelope, pubSubTopicPartition, 120, 0, 0, null);
KafkaTopicDumper.logIfTopicSwitchMessage(nonTsCtrlMsg); // Should not throw any exception

// Case 3: Non-control message
KafkaKey regularMsgKey = new KafkaKey(MessageType.PUT, Utils.getUniqueString("key-").getBytes());
PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> regularMessage =
new ImmutablePubSubMessage<>(regularMsgKey, null, pubSubTopicPartition, 120, 0, 0, null);
KafkaTopicDumper.logIfTopicSwitchMessage(regularMessage); // Should not throw any exception
}
}

0 comments on commit aa1ccf7

Please sign in to comment.