Add km module kafka

This commit is contained in:
leewei
2023-02-14 16:27:47 +08:00
parent 229140f067
commit 0b8160a714
4039 changed files with 718112 additions and 46204 deletions

View File

@@ -0,0 +1,338 @@
/*
* 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.log4jappender;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.helpers.LogLog;
import org.apache.log4j.spi.LoggingEvent;
import java.nio.charset.StandardCharsets;
import java.util.Date;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import static org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG;
import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG;
import static org.apache.kafka.common.config.SaslConfigs.SASL_MECHANISM;
import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_TYPE_CONFIG;
import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG;
import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG;
import static org.apache.kafka.common.config.SaslConfigs.SASL_KERBEROS_SERVICE_NAME;
/**
* A log4j appender that produces log messages to Kafka
*/
public class KafkaLog4jAppender extends AppenderSkeleton {
private String brokerList;
private String topic;
private String compressionType;
private String securityProtocol;
private String sslTruststoreLocation;
private String sslTruststorePassword;
private String sslKeystoreType;
private String sslKeystoreLocation;
private String sslKeystorePassword;
private String saslKerberosServiceName;
private String saslMechanism;
private String clientJaasConfPath;
private String clientJaasConf;
private String kerb5ConfPath;
private Integer maxBlockMs;
private int retries = Integer.MAX_VALUE;
private int requiredNumAcks = 1;
private int deliveryTimeoutMs = 120000;
private boolean ignoreExceptions = true;
private boolean syncSend;
private Producer<byte[], byte[]> producer;
public Producer<byte[], byte[]> getProducer() {
return producer;
}
public String getBrokerList() {
return brokerList;
}
public void setBrokerList(String brokerList) {
this.brokerList = brokerList;
}
public int getRequiredNumAcks() {
return requiredNumAcks;
}
public void setRequiredNumAcks(int requiredNumAcks) {
this.requiredNumAcks = requiredNumAcks;
}
public int getRetries() {
return retries;
}
public void setRetries(int retries) {
this.retries = retries;
}
public int getDeliveryTimeoutMs() {
return deliveryTimeoutMs;
}
public void setDeliveryTimeoutMs(int deliveryTimeoutMs) {
this.deliveryTimeoutMs = deliveryTimeoutMs;
}
public String getCompressionType() {
return compressionType;
}
public void setCompressionType(String compressionType) {
this.compressionType = compressionType;
}
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
public boolean getIgnoreExceptions() {
return ignoreExceptions;
}
public void setIgnoreExceptions(boolean ignoreExceptions) {
this.ignoreExceptions = ignoreExceptions;
}
public boolean getSyncSend() {
return syncSend;
}
public void setSyncSend(boolean syncSend) {
this.syncSend = syncSend;
}
public String getSslTruststorePassword() {
return sslTruststorePassword;
}
public String getSslTruststoreLocation() {
return sslTruststoreLocation;
}
public String getSecurityProtocol() {
return securityProtocol;
}
public void setSecurityProtocol(String securityProtocol) {
this.securityProtocol = securityProtocol;
}
public void setSslTruststoreLocation(String sslTruststoreLocation) {
this.sslTruststoreLocation = sslTruststoreLocation;
}
public void setSslTruststorePassword(String sslTruststorePassword) {
this.sslTruststorePassword = sslTruststorePassword;
}
public void setSslKeystorePassword(String sslKeystorePassword) {
this.sslKeystorePassword = sslKeystorePassword;
}
public void setSslKeystoreType(String sslKeystoreType) {
this.sslKeystoreType = sslKeystoreType;
}
public void setSslKeystoreLocation(String sslKeystoreLocation) {
this.sslKeystoreLocation = sslKeystoreLocation;
}
public void setSaslKerberosServiceName(String saslKerberosServiceName) {
this.saslKerberosServiceName = saslKerberosServiceName;
}
public void setClientJaasConfPath(String clientJaasConfPath) {
this.clientJaasConfPath = clientJaasConfPath;
}
public void setKerb5ConfPath(String kerb5ConfPath) {
this.kerb5ConfPath = kerb5ConfPath;
}
public String getSslKeystoreLocation() {
return sslKeystoreLocation;
}
public String getSslKeystoreType() {
return sslKeystoreType;
}
public String getSslKeystorePassword() {
return sslKeystorePassword;
}
public String getSaslKerberosServiceName() {
return saslKerberosServiceName;
}
public String getClientJaasConfPath() {
return clientJaasConfPath;
}
public void setSaslMechanism(String saslMechanism) {
this.saslMechanism = saslMechanism;
}
public String getSaslMechanism() {
return this.saslMechanism;
}
public void setClientJaasConf(final String clientJaasConf) {
this.clientJaasConf = clientJaasConf;
}
public String getClientJaasConf() {
return this.clientJaasConf;
}
public String getKerb5ConfPath() {
return kerb5ConfPath;
}
public int getMaxBlockMs() {
return maxBlockMs;
}
public void setMaxBlockMs(int maxBlockMs) {
this.maxBlockMs = maxBlockMs;
}
@Override
public void activateOptions() {
// check for config parameter validity
Properties props = new Properties();
if (brokerList != null)
props.put(BOOTSTRAP_SERVERS_CONFIG, brokerList);
if (props.isEmpty())
throw new ConfigException("The bootstrap servers property should be specified");
if (topic == null)
throw new ConfigException("Topic must be specified by the Kafka log4j appender");
if (compressionType != null)
props.put(COMPRESSION_TYPE_CONFIG, compressionType);
props.put(ACKS_CONFIG, Integer.toString(requiredNumAcks));
props.put(RETRIES_CONFIG, retries);
props.put(DELIVERY_TIMEOUT_MS_CONFIG, deliveryTimeoutMs);
if (securityProtocol != null) {
props.put(SECURITY_PROTOCOL_CONFIG, securityProtocol);
}
if (securityProtocol != null && securityProtocol.contains("SSL") && sslTruststoreLocation != null &&
sslTruststorePassword != null) {
props.put(SSL_TRUSTSTORE_LOCATION_CONFIG, sslTruststoreLocation);
props.put(SSL_TRUSTSTORE_PASSWORD_CONFIG, sslTruststorePassword);
if (sslKeystoreType != null && sslKeystoreLocation != null &&
sslKeystorePassword != null) {
props.put(SSL_KEYSTORE_TYPE_CONFIG, sslKeystoreType);
props.put(SSL_KEYSTORE_LOCATION_CONFIG, sslKeystoreLocation);
props.put(SSL_KEYSTORE_PASSWORD_CONFIG, sslKeystorePassword);
}
}
if (securityProtocol != null && securityProtocol.contains("SASL") && saslKerberosServiceName != null && clientJaasConfPath != null) {
props.put(SASL_KERBEROS_SERVICE_NAME, saslKerberosServiceName);
System.setProperty("java.security.auth.login.config", clientJaasConfPath);
}
if (kerb5ConfPath != null) {
System.setProperty("java.security.krb5.conf", kerb5ConfPath);
}
if (saslMechanism != null) {
props.put(SASL_MECHANISM, saslMechanism);
}
if (clientJaasConf != null) {
props.put(SASL_JAAS_CONFIG, clientJaasConf);
}
if (maxBlockMs != null) {
props.put(MAX_BLOCK_MS_CONFIG, maxBlockMs);
}
props.put(KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
props.put(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
this.producer = getKafkaProducer(props);
LogLog.debug("Kafka producer connected to " + brokerList);
LogLog.debug("Logging for topic: " + topic);
}
protected Producer<byte[], byte[]> getKafkaProducer(Properties props) {
return new KafkaProducer<>(props);
}
@Override
protected void append(LoggingEvent event) {
String message = subAppend(event);
LogLog.debug("[" + new Date(event.getTimeStamp()) + "]" + message);
Future<RecordMetadata> response = producer.send(
new ProducerRecord<>(topic, message.getBytes(StandardCharsets.UTF_8)));
if (syncSend) {
try {
response.get();
} catch (InterruptedException | ExecutionException ex) {
if (!ignoreExceptions)
throw new RuntimeException(ex);
LogLog.debug("Exception while getting response", ex);
}
}
}
private String subAppend(LoggingEvent event) {
return (this.layout == null) ? event.getRenderedMessage() : this.layout.format(event);
}
@Override
public void close() {
if (!this.closed) {
this.closed = true;
producer.close();
}
}
@Override
public boolean requiresLayout() {
return true;
}
}

View File

@@ -0,0 +1,237 @@
/*
* 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.log4jappender;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.hasItem;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.MatcherAssert.assertThat;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.log4j.Logger;
import org.apache.log4j.PropertyConfigurator;
import org.apache.log4j.helpers.LogLog;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.nio.charset.StandardCharsets;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
public class KafkaLog4jAppenderTest {
private Logger logger = Logger.getLogger(KafkaLog4jAppenderTest.class);
@Before
public void setup() {
LogLog.setInternalDebugging(true);
}
@Test
public void testKafkaLog4jConfigs() {
// host missing
Properties props = new Properties();
props.put("log4j.rootLogger", "INFO");
props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender");
props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout");
props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n");
props.put("log4j.appender.KAFKA.Topic", "test-topic");
props.put("log4j.logger.kafka.log4j", "INFO, KAFKA");
try {
PropertyConfigurator.configure(props);
Assert.fail("Missing properties exception was expected !");
} catch (ConfigException ex) {
// It's OK!
}
// topic missing
props = new Properties();
props.put("log4j.rootLogger", "INFO");
props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender");
props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout");
props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n");
props.put("log4j.appender.KAFKA.brokerList", "127.0.0.1:9093");
props.put("log4j.logger.kafka.log4j", "INFO, KAFKA");
try {
PropertyConfigurator.configure(props);
Assert.fail("Missing properties exception was expected !");
} catch (ConfigException ex) {
// It's OK!
}
}
@Test
public void testSetSaslMechanism() {
Properties props = getLog4jConfig(false);
props.put("log4j.appender.KAFKA.SaslMechanism", "PLAIN");
PropertyConfigurator.configure(props);
MockKafkaLog4jAppender mockKafkaLog4jAppender = getMockKafkaLog4jAppender();
assertThat(
mockKafkaLog4jAppender.getProducerProperties().getProperty(SaslConfigs.SASL_MECHANISM),
equalTo("PLAIN"));
}
@Test
public void testSaslMechanismNotSet() {
testProducerPropertyNotSet(SaslConfigs.SASL_MECHANISM);
}
@Test
public void testSetJaasConfig() {
Properties props = getLog4jConfig(false);
props.put("log4j.appender.KAFKA.ClientJaasConf", "jaas-config");
PropertyConfigurator.configure(props);
MockKafkaLog4jAppender mockKafkaLog4jAppender = getMockKafkaLog4jAppender();
assertThat(
mockKafkaLog4jAppender.getProducerProperties().getProperty(SaslConfigs.SASL_JAAS_CONFIG),
equalTo("jaas-config"));
}
@Test
public void testJaasConfigNotSet() {
testProducerPropertyNotSet(SaslConfigs.SASL_JAAS_CONFIG);
}
private void testProducerPropertyNotSet(String name) {
PropertyConfigurator.configure(getLog4jConfig(false));
MockKafkaLog4jAppender mockKafkaLog4jAppender = getMockKafkaLog4jAppender();
assertThat(mockKafkaLog4jAppender.getProducerProperties().stringPropertyNames(), not(hasItem(name)));
}
@Test
public void testLog4jAppends() {
PropertyConfigurator.configure(getLog4jConfig(false));
for (int i = 1; i <= 5; ++i) {
logger.error(getMessage(i));
}
Assert.assertEquals(
5, (getMockKafkaLog4jAppender()).getHistory().size());
}
@Test(expected = RuntimeException.class)
public void testLog4jAppendsWithSyncSendAndSimulateProducerFailShouldThrowException() {
Properties props = getLog4jConfig(true);
props.put("log4j.appender.KAFKA.IgnoreExceptions", "false");
PropertyConfigurator.configure(props);
MockKafkaLog4jAppender mockKafkaLog4jAppender = getMockKafkaLog4jAppender();
replaceProducerWithMocked(mockKafkaLog4jAppender, false);
logger.error(getMessage(0));
}
@Test
public void testLog4jAppendsWithSyncSendWithoutIgnoringExceptionsShouldNotThrowException() {
Properties props = getLog4jConfig(true);
props.put("log4j.appender.KAFKA.IgnoreExceptions", "false");
PropertyConfigurator.configure(props);
MockKafkaLog4jAppender mockKafkaLog4jAppender = getMockKafkaLog4jAppender();
replaceProducerWithMocked(mockKafkaLog4jAppender, true);
logger.error(getMessage(0));
}
@Test
public void testLog4jAppendsWithRealProducerConfigWithSyncSendShouldNotThrowException() {
Properties props = getLog4jConfigWithRealProducer(true);
PropertyConfigurator.configure(props);
logger.error(getMessage(0));
}
@Test(expected = RuntimeException.class)
public void testLog4jAppendsWithRealProducerConfigWithSyncSendAndNotIgnoringExceptionsShouldThrowException() {
Properties props = getLog4jConfigWithRealProducer(false);
PropertyConfigurator.configure(props);
logger.error(getMessage(0));
}
private void replaceProducerWithMocked(MockKafkaLog4jAppender mockKafkaLog4jAppender, boolean success) {
@SuppressWarnings("unchecked")
MockProducer<byte[], byte[]> producer = EasyMock.niceMock(MockProducer.class);
@SuppressWarnings("unchecked")
Future<RecordMetadata> futureMock = EasyMock.niceMock(Future.class);
try {
if (!success)
EasyMock.expect(futureMock.get())
.andThrow(new ExecutionException("simulated timeout", new TimeoutException()));
} catch (InterruptedException | ExecutionException e) {
// just mocking
}
EasyMock.expect(producer.send(EasyMock.anyObject())).andReturn(futureMock);
EasyMock.replay(producer, futureMock);
// reconfiguring mock appender
mockKafkaLog4jAppender.setKafkaProducer(producer);
mockKafkaLog4jAppender.activateOptions();
}
private MockKafkaLog4jAppender getMockKafkaLog4jAppender() {
return (MockKafkaLog4jAppender) Logger.getRootLogger().getAppender("KAFKA");
}
private byte[] getMessage(int i) {
return ("test_" + i).getBytes(StandardCharsets.UTF_8);
}
private Properties getLog4jConfigWithRealProducer(boolean ignoreExceptions) {
Properties props = new Properties();
props.put("log4j.rootLogger", "INFO, KAFKA");
props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender");
props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout");
props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n");
props.put("log4j.appender.KAFKA.BrokerList", "127.0.0.2:9093");
props.put("log4j.appender.KAFKA.Topic", "test-topic");
props.put("log4j.appender.KAFKA.RequiredNumAcks", "1");
props.put("log4j.appender.KAFKA.SyncSend", "true");
// setting producer timeout (max.block.ms) to be low
props.put("log4j.appender.KAFKA.maxBlockMs", "10");
// ignoring exceptions
props.put("log4j.appender.KAFKA.IgnoreExceptions", Boolean.toString(ignoreExceptions));
props.put("log4j.logger.kafka.log4j", "INFO, KAFKA");
return props;
}
private Properties getLog4jConfig(boolean syncSend) {
Properties props = new Properties();
props.put("log4j.rootLogger", "INFO, KAFKA");
props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.MockKafkaLog4jAppender");
props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout");
props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n");
props.put("log4j.appender.KAFKA.BrokerList", "127.0.0.1:9093");
props.put("log4j.appender.KAFKA.Topic", "test-topic");
props.put("log4j.appender.KAFKA.RequiredNumAcks", "1");
props.put("log4j.appender.KAFKA.SyncSend", Boolean.toString(syncSend));
props.put("log4j.logger.kafka.log4j", "INFO, KAFKA");
return props;
}
}

View File

@@ -0,0 +1,59 @@
/*
* 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.log4jappender;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.test.MockSerializer;
import org.apache.log4j.spi.LoggingEvent;
import java.util.List;
import java.util.Properties;
public class MockKafkaLog4jAppender extends KafkaLog4jAppender {
private MockProducer<byte[], byte[]> mockProducer =
new MockProducer<>(false, new MockSerializer(), new MockSerializer());
private Properties producerProperties;
@Override
protected Producer<byte[], byte[]> getKafkaProducer(Properties props) {
producerProperties = props;
return mockProducer;
}
void setKafkaProducer(MockProducer<byte[], byte[]> producer) {
this.mockProducer = producer;
}
@Override
protected void append(LoggingEvent event) {
if (super.getProducer() == null) {
activateOptions();
}
super.append(event);
}
List<ProducerRecord<byte[], byte[]>> getHistory() {
return mockProducer.history();
}
public Properties getProducerProperties() {
return producerProperties;
}
}