diff --git a/.vscode/settings.json b/.vscode/settings.json new file mode 100644 index 0000000000..10196539ef --- /dev/null +++ b/.vscode/settings.json @@ -0,0 +1,3 @@ +{ + "java.jdt.ls.vmargs": "-XX:+UseParallelGC -XX:GCTimeRatio=4 -XX:AdaptiveSizePolicyWeight=90 -Dsun.zip.disableMemoryMapping=true -Xmx8G -Xms100m -Xlog:disable" +} \ No newline at end of file diff --git a/audit-server/audit-common/pom.xml b/audit-server/audit-common/pom.xml new file mode 100644 index 0000000000..73bef80bea --- /dev/null +++ b/audit-server/audit-common/pom.xml @@ -0,0 +1,54 @@ + + + + 4.0.0 + + + org.apache.ranger + ranger + 3.0.0-SNAPSHOT + ../.. + + + ranger-audit-server-common + jar + Ranger Audit Server Common + Shared classes between audit ingestor and dispatcher + + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + org.apache.ranger + ranger-plugins-common + ${project.version} + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + + ranger-audit-server-common-${project.version} + + \ No newline at end of file diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/server/AuditConfig.java b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/server/AuditConfig.java similarity index 50% rename from audit-server/common/src/main/java/org/apache/ranger/audit/server/AuditConfig.java rename to audit-server/audit-common/src/main/java/org/apache/ranger/audit/server/AuditConfig.java index 68b8eda6cb..a6c122d924 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/server/AuditConfig.java +++ b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/server/AuditConfig.java @@ -19,45 +19,27 @@ package org.apache.ranger.audit.server; -import org.apache.ranger.authorization.hadoop.config.RangerConfiguration; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; import java.util.Properties; /** * Base configuration class for Ranger Audit Server services. * Can be extended by specific services to load their custom configuration files. */ -public class AuditConfig extends RangerConfiguration { - private static final Logger LOG = LoggerFactory.getLogger(AuditConfig.class); - private static volatile AuditConfig sInstance; +public class AuditConfig extends Configuration { + private static final Logger LOG = LoggerFactory.getLogger(AuditConfig.class); - protected AuditConfig() { + public AuditConfig() { super(); } - /** - * Get the singleton instance of AuditConfig. - * Subclasses should override this method to return their specific instance. - */ - public static AuditConfig getInstance() { - AuditConfig ret = AuditConfig.sInstance; - - if (ret == null) { - synchronized (AuditConfig.class) { - ret = AuditConfig.sInstance; - - if (ret == null) { - ret = new AuditConfig(); - AuditConfig.sInstance = ret; - } - } - } - - return ret; - } - public Properties getProperties() { return this.getProps(); } @@ -66,11 +48,11 @@ public Properties getProperties() { * Add a resource file to the configuration. * Subclasses can override to load their specific config files. * - * @param resourcePath Path to the resource file (e.g., "conf/ranger-audit-server-site.xml") + * @param resourcePath Path to the resource file (e.g., "conf/ranger-audit-ingestor-site.xml") * @param required Whether this resource is required * @return true if resource was loaded successfully or is optional, false otherwise */ - protected boolean addAuditResource(String resourcePath, boolean required) { + public boolean addAuditResource(String resourcePath, boolean required) { LOG.debug("==> addAuditResource(path={}, required={})", resourcePath, required); boolean success = addResourceIfReadable(resourcePath); @@ -87,4 +69,58 @@ protected boolean addAuditResource(String resourcePath, boolean required) { return success || !required; } + + public boolean addResourceIfReadable(String aResourceName) { + LOG.debug("==> addResourceIfReadable({})", aResourceName); + + boolean ret = false; + URL fUrl = getFileLocation(aResourceName); + + if (fUrl != null) { + LOG.debug("addResourceIfReadable({}): resource file is {}", aResourceName, fUrl); + + try { + addResource(fUrl); + + ret = true; + } catch (Exception e) { + LOG.error("Unable to load the resource name [{}]. Ignoring the resource:{}", aResourceName, fUrl); + + LOG.debug("Resource loading failed for {}", fUrl, e); + } + } else { + LOG.debug("addResourceIfReadable({}): couldn't find resource file location", aResourceName); + } + + LOG.debug("<== addResourceIfReadable({}), result={}", aResourceName, ret); + + return ret; + } + + private URL getFileLocation(String fileName) { + URL lurl = null; + + if (!StringUtils.isEmpty(fileName)) { + lurl = AuditConfig.class.getClassLoader().getResource(fileName); + + if (lurl == null) { + lurl = AuditConfig.class.getClassLoader().getResource("/" + fileName); + } + + if (lurl == null) { + File f = new File(fileName); + if (f.exists()) { + try { + lurl = f.toURI().toURL(); + } catch (MalformedURLException e) { + LOG.error("Unable to load the resource name [{}]. Ignoring the resource:{}", fileName, f.getPath()); + } + } else { + LOG.debug("Conf file path {} does not exists", fileName); + } + } + } + + return lurl; + } } diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/server/AuditServerConstants.java b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/server/AuditServerConstants.java similarity index 61% rename from audit-server/common/src/main/java/org/apache/ranger/audit/server/AuditServerConstants.java rename to audit-server/audit-common/src/main/java/org/apache/ranger/audit/server/AuditServerConstants.java index a3c2e34208..68100003ba 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/server/AuditServerConstants.java +++ b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/server/AuditServerConstants.java @@ -23,41 +23,41 @@ public class AuditServerConstants { private AuditServerConstants() {} public static final String AUDIT_SERVER_APPNAME = "ranger-audit"; - public static final String PROP_PREFIX_AUDIT_SERVER = "ranger.audit.server."; + public static final String PROP_PREFIX_AUDIT_SERVER = "ranger.audit.ingestor."; public static final String PROP_PREFIX_AUDIT_SERVER_SERVICE = PROP_PREFIX_AUDIT_SERVER + "service."; public static final String PROP_AUTH_TO_LOCAL = PROP_PREFIX_AUDIT_SERVER + "auth.to.local"; public static final String PROP_SUFFIX_ALLOWED_USERS = ".allowed.users"; - public static final String JAAS_KRB5_MODULE = "com.sun.security.auth.module.Krb5LoginModule required"; - public static final String JAAS_USE_KEYTAB = "useKeyTab=true"; - public static final String JAAS_KEYTAB = "keyTab=\""; - public static final String JAAS_STOKE_KEY = "storeKey=true"; - public static final String JAAS_SERVICE_NAME = "serviceName=kafka"; - public static final String JAAS_USER_TICKET_CACHE = "useTicketCache=false"; - public static final String JAAS_PRINCIPAL = "principal=\""; - public static final String PROP_BOOTSTRAP_SERVERS = "bootstrap.servers"; - public static final String PROP_TOPIC_NAME = "topic.name"; - public static final String PROP_SECURITY_PROTOCOL = "security.protocol"; - public static final String PROP_SASL_MECHANISM = "sasl.mechanism"; - public static final String PROP_SASL_JAAS_CONFIG = "sasl.jaas.config"; - public static final String PROP_SASL_KERBEROS_SERVICE_NAME = "sasl.kerberos.service.name"; - public static final String PROP_REQ_TIMEOUT_MS = "request.timeout.ms"; - public static final String PROP_CONN_MAX_IDEAL_MS = "connections.max.idle.ms"; - public static final String PROP_SOLR_DEST_PREFIX = "solr"; - public static final String PROP_HDFS_DEST_PREFIX = "hdfs"; - public static final String PROP_CONSUMER_THREAD_COUNT = "consumer.thread.count"; - public static final String PROP_CONSUMER_OFFSET_COMMIT_STRATEGY = "consumer.offset.commit.strategy"; - public static final String PROP_CONSUMER_OFFSET_COMMIT_INTERVAL = "consumer.offset.commit.interval.ms"; - public static final String PROP_CONSUMER_MAX_POLL_RECORDS = "consumer.max.poll.records"; - public static final String PROP_CONSUMER_SESSION_TIMEOUT_MS = "consumer.session.timeout.ms"; - public static final String PROP_CONSUMER_MAX_POLL_INTERVAL_MS = "consumer.max.poll.interval.ms"; - public static final String PROP_CONSUMER_HEARTBEAT_INTERVAL_MS = "consumer.heartbeat.interval.ms"; - public static final String PROP_CONSUMER_PARTITION_ASSIGNMENT_STRATEGY = "consumer.partition.assignment.strategy"; - public static final String PROP_AUDIT_SERVICE_PRINCIPAL = "kerberos.principal"; - public static final String PROP_AUDIT_SERVICE_KEYTAB = "kerberos.keytab"; - public static final String PROP_KAFKA_PROP_PREFIX = "xasecure.audit.destination.kafka"; - public static final String PROP_KAFKA_STARTUP_MAX_RETRIES = "kafka.startup.max.retries"; - public static final String PROP_KAFKA_STARTUP_RETRY_DELAY_MS = "kafka.startup.retry.delay.ms"; + public static final String JAAS_KRB5_MODULE = "com.sun.security.auth.module.Krb5LoginModule required"; + public static final String JAAS_USE_KEYTAB = "useKeyTab=true"; + public static final String JAAS_KEYTAB = "keyTab=\""; + public static final String JAAS_STOKE_KEY = "storeKey=true"; + public static final String JAAS_SERVICE_NAME = "serviceName=kafka"; + public static final String JAAS_USER_TICKET_CACHE = "useTicketCache=false"; + public static final String JAAS_PRINCIPAL = "principal=\""; + public static final String PROP_BOOTSTRAP_SERVERS = "bootstrap.servers"; + public static final String PROP_TOPIC_NAME = "topic.name"; + public static final String PROP_SECURITY_PROTOCOL = "security.protocol"; + public static final String PROP_SASL_MECHANISM = "sasl.mechanism"; + public static final String PROP_SASL_JAAS_CONFIG = "sasl.jaas.config"; + public static final String PROP_SASL_KERBEROS_SERVICE_NAME = "sasl.kerberos.service.name"; + public static final String PROP_REQ_TIMEOUT_MS = "request.timeout.ms"; + public static final String PROP_CONN_MAX_IDEAL_MS = "connections.max.idle.ms"; + public static final String PROP_SOLR_DEST_PREFIX = "solr"; + public static final String PROP_HDFS_DEST_PREFIX = "hdfs"; + public static final String PROP_DISPATCHER_THREAD_COUNT = "dispatcher.thread.count"; + public static final String PROP_DISPATCHER_OFFSET_COMMIT_STRATEGY = "dispatcher.offset.commit.strategy"; + public static final String PROP_DISPATCHER_OFFSET_COMMIT_INTERVAL = "dispatcher.offset.commit.interval.ms"; + public static final String PROP_DISPATCHER_MAX_POLL_RECORDS = "dispatcher.max.poll.records"; + public static final String PROP_DISPATCHER_SESSION_TIMEOUT_MS = "dispatcher.session.timeout.ms"; + public static final String PROP_DISPATCHER_MAX_POLL_INTERVAL_MS = "dispatcher.max.poll.interval.ms"; + public static final String PROP_DISPATCHER_HEARTBEAT_INTERVAL_MS = "dispatcher.heartbeat.interval.ms"; + public static final String PROP_DISPATCHER_PARTITION_ASSIGNMENT_STRATEGY = "dispatcher.partition.assignment.strategy"; + public static final String PROP_AUDIT_SERVICE_PRINCIPAL = "kerberos.principal"; + public static final String PROP_AUDIT_SERVICE_KEYTAB = "kerberos.keytab"; + public static final String PROP_KAFKA_PROP_PREFIX = "xasecure.audit.destination.kafka"; + public static final String PROP_KAFKA_TOPIC_INIT_MAX_RETRIES = "kafka.topic.init.max.retries"; + public static final String PROP_KAFKA_TOPIC_INIT_RETRY_DELAY_MS = "kafka.topic.init.retry.delay.ms"; // ranger_audits topic configuration public static final String PROP_TOPIC_PARTITIONS = "topic.partitions"; @@ -77,18 +77,18 @@ private AuditServerConstants() {} public static final int DEFAULT_BUFFER_PARTITIONS = 9; // For plugin-based partitioning // Hadoop security configuration for UGI - public static final String PROP_HADOOP_AUTHENTICATION_TYPE = "hadoop.security.authentication"; - public static final String PROP_HADOOP_AUTH_TYPE_KERBEROS = "kerberos"; - public static final String PROP_HADOOP_KERBEROS_NAME_RULES = "hadoop.security.auth_to_local"; + public static final String PROP_HADOOP_AUTHENTICATION_TYPE = "hadoop.security.authentication"; + public static final String PROP_HADOOP_AUTH_TYPE_KERBEROS = "kerberos"; + public static final String PROP_HADOOP_KERBEROS_NAME_RULES = "hadoop.security.auth_to_local"; // Kafka Topic and defaults - public static final String DEFAULT_TOPIC = "ranger_audits"; - public static final String DEFAULT_SASL_MECHANISM = "PLAIN"; - public static final String DEFAULT_SECURITY_PROTOCOL = "PLAINTEXT"; - public static final String DEFAULT_SERVICE_NAME = "kafka"; - public static final String DEFAULT_RANGER_AUDIT_HDFS_CONSUMER_GROUP = "ranger_audit_hdfs_consumer_group"; - public static final String DEFAULT_RANGER_AUDIT_SOLR_CONSUMER_GROUP = "ranger_audit_solr_consumer_group"; - public static final String PROP_SECURITY_PROTOCOL_VALUE = "SASL"; + public static final String DEFAULT_TOPIC = "ranger_audits"; + public static final String DEFAULT_SASL_MECHANISM = "PLAIN"; + public static final String DEFAULT_SECURITY_PROTOCOL = "PLAINTEXT"; + public static final String DEFAULT_SERVICE_NAME = "kafka"; + public static final String DEFAULT_RANGER_AUDIT_HDFS_DISPATCHER_GROUP = "ranger_audit_hdfs_dispatcher_group"; + public static final String DEFAULT_RANGER_AUDIT_SOLR_DISPATCHER_GROUP = "ranger_audit_solr_dispatcher_group"; + public static final String PROP_SECURITY_PROTOCOL_VALUE = "SASL"; // Offset commit strategies public static final String PROP_OFFSET_COMMIT_STRATEGY_MANUAL = "manual"; @@ -97,18 +97,18 @@ private AuditServerConstants() {} public static final long DEFAULT_OFFSET_COMMIT_INTERVAL_MS = 30000; // 30 seconds public static final int DEFAULT_MAX_POLL_RECORDS = 500; // Kafka default batch size - // Kafka consumer rebalancing timeouts (for subscribe mode) - public static final int DEFAULT_SESSION_TIMEOUT_MS = 60000; // 60 seconds - failure detection - public static final int DEFAULT_MAX_POLL_INTERVAL_MS = 300000; // 5 minutes - max processing time - public static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 10000; // 10 seconds - heartbeat frequency + // Kafka dispatcher rebalancing timeouts (for subscribe mode) + public static final int DEFAULT_SESSION_TIMEOUT_MS = 60000; // 60 seconds - failure detection + public static final int DEFAULT_MAX_POLL_INTERVAL_MS = 300000; // 5 minutes - max processing time + public static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 10000; // 10 seconds - heartbeat frequency - // Kafka consumer partition assignment strategy - public static final String DEFAULT_PARTITION_ASSIGNMENT_STRATEGY = "org.apache.kafka.clients.consumer.RangeAssignor"; + // Kafka dispatcher partition assignment strategy + public static final String DEFAULT_PARTITION_ASSIGNMENT_STRATEGY = "org.apache.kafka.clients.consumer.RangeAssignor"; // Destination - public static final String DESTINATION_HDFS = "HDFS"; - public static final String DESTINATION_SOLR = "SOLR"; + public static final String DESTINATION_HDFS = "HDFS"; + public static final String DESTINATION_SOLR = "SOLR"; - // Consumer Registry Configuration - public static final String PROP_CONSUMER_CLASSES = "consumer.classes"; + // Dispatcher Registry Configuration + public static final String PROP_DISPATCHER_CLASSES = "dispatcher.classes"; } diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditMessageQueueUtils.java b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditMessageQueueUtils.java similarity index 68% rename from audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditMessageQueueUtils.java rename to audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditMessageQueueUtils.java index 43d62a4050..bbdc610e86 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditMessageQueueUtils.java +++ b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditMessageQueueUtils.java @@ -27,12 +27,12 @@ import org.apache.kafka.clients.admin.NewPartitions; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.ranger.audit.provider.AuditProviderFactory; import org.apache.ranger.audit.provider.MiscUtil; import org.apache.ranger.audit.server.AuditServerConstants; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.time.Duration; import java.util.Collections; import java.util.HashMap; @@ -43,35 +43,26 @@ public class AuditMessageQueueUtils { private static final Logger LOG = LoggerFactory.getLogger(AuditMessageQueueUtils.class); - AuditServerUtils auditServerUtils = new AuditServerUtils(); - boolean isTopicReady; - boolean isSolrConsumerEnabled; - boolean isHDFSConsumerEnabled; - - public AuditMessageQueueUtils(Properties props) { - isHDFSConsumerEnabled = MiscUtil.getBooleanProperty(props, AuditProviderFactory.AUDIT_DEST_BASE + "." + AuditServerConstants.PROP_HDFS_DEST_PREFIX, false); - isSolrConsumerEnabled = MiscUtil.getBooleanProperty(props, AuditProviderFactory.AUDIT_DEST_BASE + "." + AuditServerConstants.PROP_SOLR_DEST_PREFIX, false); + public AuditMessageQueueUtils() { } public String createAuditsTopicIfNotExists(Properties props, String propPrefix) { - LOG.info("==> AuditMessageQueueUtils:createAuditsTopicIfNotExists()"); - - String ret = null; - String topicName = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_TOPIC_NAME, AuditServerConstants.DEFAULT_TOPIC); - String bootstrapServers = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_BOOTSTRAP_SERVERS); - String securityProtocol = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SECURITY_PROTOCOL, AuditServerConstants.DEFAULT_SECURITY_PROTOCOL); - String saslMechanism = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SASL_MECHANISM, AuditServerConstants.DEFAULT_SASL_MECHANISM); - int connMaxIdleTimeoutMS = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONN_MAX_IDEAL_MS, 10000); - int partitions = getPartitions(props, propPrefix); - short replicationFactor = (short) MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_REPLICATION_FACTOR, AuditServerConstants.DEFAULT_REPLICATION_FACTOR); - int reqTimeoutMS = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_REQ_TIMEOUT_MS, 5000); - - // Retry configuration for Kafka connection during startup - int maxRetries = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_KAFKA_STARTUP_MAX_RETRIES, 10); - int retryDelayMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_KAFKA_STARTUP_RETRY_DELAY_MS, 3000); - int currentAttempt = 0; + LOG.info("==> AuditMessageQueueUtils:createAuditsTopicIfNotExists(propPrefix={})", propPrefix); + + String ret = null; + String topicName = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_TOPIC_NAME, AuditServerConstants.DEFAULT_TOPIC); + String bootstrapServers = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_BOOTSTRAP_SERVERS); + String securityProtocol = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SECURITY_PROTOCOL, AuditServerConstants.DEFAULT_SECURITY_PROTOCOL); + String saslMechanism = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SASL_MECHANISM, AuditServerConstants.DEFAULT_SASL_MECHANISM); + int connMaxIdleTimeoutMS = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONN_MAX_IDEAL_MS, 10000); + int partitions = getPartitions(props, propPrefix); + short replicationFactor = (short) MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_REPLICATION_FACTOR, AuditServerConstants.DEFAULT_REPLICATION_FACTOR); + int reqTimeoutMS = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_REQ_TIMEOUT_MS, 5000); + int maxAttempts = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_KAFKA_TOPIC_INIT_MAX_RETRIES, 10) + 1; + int retryDelayMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_KAFKA_TOPIC_INIT_RETRY_DELAY_MS, 3000); Map kafkaProp = new HashMap<>(); + kafkaProp.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); kafkaProp.put(AuditServerConstants.PROP_SASL_MECHANISM, saslMechanism); kafkaProp.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, securityProtocol); @@ -83,30 +74,32 @@ public String createAuditsTopicIfNotExists(Properties props, String propPrefix) kafkaProp.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, reqTimeoutMS); kafkaProp.put(AdminClientConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG, connMaxIdleTimeoutMS); - while (currentAttempt <= maxRetries && ret == null) { - currentAttempt++; - + for (int currentAttempt = 1; currentAttempt <= maxAttempts && ret == null; currentAttempt++) { try (AdminClient admin = AdminClient.create(kafkaProp)) { - if (currentAttempt > 1) { - LOG.info("Attempting to connect to Kafka (attempt {}/{})", currentAttempt, maxRetries + 1); - } + LOG.info("Attempting to connect to Kafka (attempt {}/{})", currentAttempt, maxAttempts); Set names = admin.listTopics().names().get(); + if (!names.contains(topicName)) { + LOG.info("Creating topic '{}' with {} partitions and replication factor {}", topicName, partitions, replicationFactor); + NewTopic topic = new NewTopic(topicName, partitions, replicationFactor); + admin.createTopics(Collections.singletonList(topic)).all().get(); + ret = topic.name(); - LOG.info("Creating topic '{}' with {} partitions and replication factor {}", topicName, partitions, replicationFactor); + // Wait for metadata to propagate across the cluster - boolean isTopicReady = auditServerUtils.waitUntilTopicReady(admin, topicName, Duration.ofSeconds(60)); + boolean isTopicReady = AuditServerUtils.waitUntilTopicReady(admin, topicName, Duration.ofSeconds(60)); + if (isTopicReady) { try { DescribeTopicsResult result = admin.describeTopics(Collections.singletonList(topicName)); TopicDescription topicDescription = result.values().get(topicName).get(); + ret = topicDescription.name(); int partitionCount = topicDescription.partitions().size(); - setTopicReady(isTopicReady); LOG.info("Topic: {} successfully created with {} partitions", ret, partitionCount); } catch (Exception e) { @@ -121,9 +114,10 @@ public String createAuditsTopicIfNotExists(Properties props, String propPrefix) ret = updateExistingTopicPartitions(admin, topicName, partitions, replicationFactor); } } catch (Exception ex) { - if (currentAttempt <= maxRetries) { + if (currentAttempt < maxAttempts) { LOG.warn("AuditMessageQueueUtils:createAuditsTopicIfNotExists(): Failed to connect to Kafka on attempt {}/{}. Retrying in {} ms. Error: {}", - currentAttempt, maxRetries + 1, retryDelayMs, ex.getMessage()); + currentAttempt, maxAttempts, retryDelayMs, ex.getMessage()); + try { Thread.sleep(retryDelayMs); } catch (InterruptedException ie) { @@ -137,27 +131,11 @@ public String createAuditsTopicIfNotExists(Properties props, String propPrefix) } } - LOG.info("<== AuditMessageQueueUtils:createAuditsTopicIfNotExists() ret: {}", ret); + LOG.info("<== AuditMessageQueueUtils:createAuditsTopicIfNotExists(propPrefix={}) ret: {}", propPrefix, ret); return ret; } - public boolean isSolrConsumerEnabled() { - return isSolrConsumerEnabled; - } - - public boolean isHDFSConsumerEnabled() { - return isHDFSConsumerEnabled; - } - - public boolean isTopicReady() { - return isTopicReady; - } - - public void setTopicReady(boolean topicReady) { - isTopicReady = topicReady; - } - public String getJAASConfig(Properties props, String propPrefix) { // Use ranger service principal and keytab for Kafka authentication // This ensures consistent identity across all Ranger services and destination writes @@ -173,13 +151,15 @@ public String getJAASConfig(Properties props, String propPrefix) { // Validate keytab file exists and is readable if (keytab != null) { - java.io.File keytabFile = new java.io.File(keytab); + File keytabFile = new File(keytab); + if (!keytabFile.exists()) { LOG.error("ERROR: Keytab file does not exist: {}", keytab); + throw new IllegalStateException("Keytab file not found: " + keytab); - } - if (!keytabFile.canRead()) { + } else if (!keytabFile.canRead()) { LOG.error("ERROR: Keytab file is not readable: {}", keytab); + throw new IllegalStateException("Keytab file not readable: " + keytab); } @@ -191,18 +171,21 @@ public String getJAASConfig(Properties props, String propPrefix) { } try { - principal = SecureClientLogin.getPrincipal(props.getProperty(propPrefix + "." + AuditServerConstants.PROP_AUDIT_SERVICE_PRINCIPAL), hostName); + principal = SecureClientLogin.getPrincipal(principal, hostName); + LOG.info("Principal (resolved): {}", principal); } catch (Exception e) { principal = null; + LOG.error("ERROR: Failed to resolve principal from _HOST pattern!", e); } if (keytab == null || principal == null) { - AuditServerLogFormatter.builder("Please configure the following properties in ranger-audit-server-site.xml:") + AuditServerLogFormatter.builder("Please configure the following properties in ranger-audit-ingestor-site.xml:") .add(propPrefix + "." + AuditServerConstants.PROP_AUDIT_SERVICE_PRINCIPAL, "ranger/_HOST@YOUR-REALM") .add(propPrefix + "." + AuditServerConstants.PROP_AUDIT_SERVICE_KEYTAB, "/path/to/ranger.keytab") .logError(LOG); + throw new IllegalStateException("Ranger service principal and keytab must be configured for Kafka authentication. "); } @@ -228,9 +211,9 @@ public String getJAASConfig(Properties props, String propPrefix) { public String updateExistingTopicPartitions(AdminClient admin, String topicName, int partitions, short replicationFactor) { LOG.info("==> AuditMessageQueueUtils:updateExistingTopicPartitions() topic: {}, desired partitions: {}", topicName, partitions); - String ret = null; - int maxRetries = 3; - int retryDelayMs = 1000; // Start with 1 second + String ret; + int maxAttempts = 3; + int retryDelayMs = 1000; // Start with 1 second try { // Describe the existing topic to get current partition count @@ -239,6 +222,7 @@ public String updateExistingTopicPartitions(AdminClient admin, String topicName, int currentPartitions = topicDescription.partitions().size(); ret = topicDescription.name(); + LOG.info("Topic '{}' already exists with {} partitions", ret, currentPartitions); // Check if we need to increase partitions @@ -249,22 +233,24 @@ public String updateExistingTopicPartitions(AdminClient admin, String topicName, Exception lastException = null; // Retry logic while updating partitions - for (int attempt = 1; attempt <= maxRetries; attempt++) { + for (int attempt = 1; attempt <= maxAttempts; attempt++) { try { - LOG.info("Partition update attempt {}/{} for topic '{}'", attempt, maxRetries, topicName); + LOG.info("Partition update attempt {}/{} for topic '{}'", attempt, maxAttempts, topicName); // Create partition increase request Map newPartitionsMap = new HashMap<>(); + newPartitionsMap.put(topicName, NewPartitions.increaseTo(partitions)); // Execute partition increase CreatePartitionsResult createPartitionsResult = admin.createPartitions(newPartitionsMap); + createPartitionsResult.all().get(); // Wait for operation to complete LOG.info("Successfully initiated partition increase for topic '{}' on attempt {}", topicName, attempt); // Wait for metadata to propagate across the cluster - boolean isTopicReady = auditServerUtils.waitUntilTopicReady(admin, topicName, Duration.ofSeconds(60)); + boolean isTopicReady = AuditServerUtils.waitUntilTopicReady(admin, topicName, Duration.ofSeconds(60)); if (isTopicReady) { // Verify the partition count after update @@ -273,47 +259,55 @@ public String updateExistingTopicPartitions(AdminClient admin, String topicName, int updatedPartitions = updatedDescription.partitions().size(); if (updatedPartitions >= partitions) { - setTopicReady(true); LOG.info("Topic '{}' successfully upgraded from {} to {} partitions on attempt {}", topicName, currentPartitions, updatedPartitions, attempt); + updateSuccess = true; break; } else { LOG.warn("Topic '{}' partition update completed but verification shows only {} partitions (expected {})", topicName, updatedPartitions, partitions); + throw new IllegalStateException("Partition count verification failed"); } } else { LOG.warn("Topic '{}' partition update completed but topic not ready within timeout on attempt {}", topicName, attempt); + throw new IllegalStateException("Topic not ready after partition update"); } } catch (Exception e) { lastException = e; - LOG.warn("Partition update attempt {}/{} failed for topic '{}': {}", attempt, maxRetries, topicName, e.getMessage()); - if (attempt < maxRetries) { + + LOG.warn("Partition update attempt {}/{} failed for topic '{}': {}", attempt, maxAttempts, topicName, e.getMessage()); + + if (attempt < maxAttempts) { int currentDelay = retryDelayMs * (1 << (attempt - 1)); + LOG.info("Retrying partition update in {} ms...", currentDelay); + Thread.sleep(currentDelay); } } } + if (!updateSuccess) { - String errorMsg = String.format( - "FATAL: Failed to update partitions for topic '%s' after %d attempts. " + - "Required: %d partitions, Current: %d partitions. " + - "Cannot proceed without sufficient partitions.", topicName, maxRetries, partitions, currentPartitions); + String errorMsg = String.format("Failed to update partitions for topic '%s' after %d attempts. " + + "Required: %d partitions, Current: %d partitions. Cannot proceed without sufficient partitions.", + topicName, maxAttempts, partitions, currentPartitions); + LOG.error(errorMsg, lastException); + throw new RuntimeException(errorMsg, lastException); } } else if (partitions < currentPartitions) { LOG.warn("Topic '{}' has {} partitions, which is more than the configured {} partitions. " + "Kafka does not support reducing partition count. Using existing partition count.", topicName, currentPartitions, partitions); - setTopicReady(true); } else { LOG.info("Topic '{}' already has the correct number of partitions: {}", topicName, currentPartitions); - setTopicReady(true); } } catch (Exception e) { - String errorMsg = String.format("FATAL: Error updating partitions for topic '%s'", topicName); + String errorMsg = String.format("Error updating partitions for topic '%s'", topicName); + LOG.error(errorMsg, e); + throw new RuntimeException(errorMsg, e); } @@ -325,44 +319,47 @@ public String updateExistingTopicPartitions(AdminClient admin, String topicName, /** * Get the number of partitions for the Kafka topic. * - * If configured.plugins is NOT set, uses topic.partitions property (default: 10). - * If configured.plugins is set, auto-calculates: (plugin partitions + overrides) + buffer partitions. + * If configured.plugins is NOT set + * topic.partitions property (default: 10) + * Else + * sum(per-plugin partitions: default 3) + buffer partitions * * @return Number of partitions for the topic */ private int getPartitions(Properties prop, String propPrefix) { // Check if configured.plugins is set (use empty string as default to detect when not configured) + int totalPartitions = 0; String configuredPlugins = MiscUtil.getStringProperty(prop, propPrefix + "." + AuditServerConstants.PROP_CONFIGURED_PLUGINS, AuditServerConstants.DEFAULT_CONFIGURED_PLUGINS); - // If no configured plugins, use simple hash-based partitioning with topic.partitions if (configuredPlugins == null || configuredPlugins.trim().isEmpty()) { - int partitions = MiscUtil.getIntProperty(prop, propPrefix + "." + AuditServerConstants.PROP_TOPIC_PARTITIONS, AuditServerConstants.DEFAULT_TOPIC_PARTITIONS); - LOG.info("No configured plugins - using hash-based partitioning with {} partitions", partitions); - return partitions; - } + totalPartitions = MiscUtil.getIntProperty(prop, propPrefix + "." + AuditServerConstants.PROP_TOPIC_PARTITIONS, AuditServerConstants.DEFAULT_TOPIC_PARTITIONS); - // Auto-calculate based on plugin configuration - String[] plugins = configuredPlugins.split(","); - int defaultPartitionsPerPlugin = MiscUtil.getIntProperty(prop, propPrefix + "." + AuditServerConstants.PROP_TOPIC_PARTITIONS_PER_CONFIGURED_PLUGIN, AuditServerConstants.DEFAULT_PARTITIONS_PER_CONFIGURED_PLUGIN); - - // Calculate total partitions needed - AuditServerLogFormatter.LogBuilder logBuilder = AuditServerLogFormatter.builder("Kafka Topic Partition Allocation (Plugin-based)"); - int totalPartitions = 0; - for (String plugin : plugins) { - String pluginTrimmed = plugin.trim(); - String overrideKey = propPrefix + "." + AuditServerConstants.PROP_PLUGIN_PARTITION_OVERRIDE_PREFIX + pluginTrimmed; - int partitionCount = MiscUtil.getIntProperty(prop, overrideKey, defaultPartitionsPerPlugin); - totalPartitions += partitionCount; - logBuilder.add("Plugin '" + pluginTrimmed + "'", partitionCount + " partitions"); - } + LOG.info("No configured plugins - using hash-based partitioning with {} partitions", totalPartitions); + } else { + // Auto-calculate based on plugin configuration + int defaultPartitionsPerPlugin = MiscUtil.getIntProperty(prop, propPrefix + "." + AuditServerConstants.PROP_TOPIC_PARTITIONS_PER_CONFIGURED_PLUGIN, AuditServerConstants.DEFAULT_PARTITIONS_PER_CONFIGURED_PLUGIN); + + // Calculate total partitions needed + AuditServerLogFormatter.LogBuilder logBuilder = AuditServerLogFormatter.builder("Kafka Topic Partition Allocation (Plugin-based)"); - // Add buffer partitions for unconfigured plugins - int bufferPartitions = MiscUtil.getIntProperty(prop, propPrefix + "." + AuditServerConstants.PROP_BUFFER_PARTITIONS, AuditServerConstants.DEFAULT_BUFFER_PARTITIONS); - totalPartitions += bufferPartitions; + for (String plugin : configuredPlugins.split(",")) { + String overrideKey = propPrefix + "." + AuditServerConstants.PROP_PLUGIN_PARTITION_OVERRIDE_PREFIX + plugin.trim(); + int partitionCount = MiscUtil.getIntProperty(prop, overrideKey, defaultPartitionsPerPlugin); - logBuilder.add("Buffer partitions", bufferPartitions + " partitions"); - logBuilder.add("Total topic partitions (calculated)", totalPartitions); - logBuilder.logInfo(LOG); + totalPartitions += partitionCount; + + logBuilder.add("Plugin '" + plugin + "'", partitionCount + " partitions"); + } + + // Add buffer partitions for unconfigured plugins + int bufferPartitions = MiscUtil.getIntProperty(prop, propPrefix + "." + AuditServerConstants.PROP_BUFFER_PARTITIONS, AuditServerConstants.DEFAULT_BUFFER_PARTITIONS); + + totalPartitions += bufferPartitions; + + logBuilder.add("Buffer partitions", bufferPartitions + " partitions"); + logBuilder.add("Total topic partitions (calculated)", totalPartitions); + logBuilder.logInfo(LOG); + } return totalPartitions; } diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditServerLogFormatter.java b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditServerLogFormatter.java similarity index 57% rename from audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditServerLogFormatter.java rename to audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditServerLogFormatter.java index 1fca5554b3..2f03d70db7 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditServerLogFormatter.java +++ b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditServerLogFormatter.java @@ -30,45 +30,6 @@ public class AuditServerLogFormatter { private AuditServerLogFormatter() {} - /** - * Log a map of logDetails at INFO level with a title - * @param logger The logger to use - * @param title The title/header for this log section - * @param logDetails Map of key-value pairs to log - */ - public static void logInfo(Logger logger, String title, Map logDetails) { - if (logDetails != null && !logDetails.isEmpty()) { - logger.info("{}:", title); - logDetails.forEach((key, value) -> logger.info(" {} = [{}]", key, value)); - } - } - - /** - * Log a map of logDetails at DEBUG level with a title - * @param logger The logger to use - * @param title The title/header for this log section - * @param logDetails Map of key-value pairs to log - */ - public static void logDebug(Logger logger, String title, Map logDetails) { - if (logDetails != null && !logDetails.isEmpty()) { - logger.debug("{}:", title); - logDetails.forEach((key, value) -> logger.debug(" {} = [{}]", key, value)); - } - } - - /** - * Log a map of logDetails at DEBUG level with a title - * @param logger The logger to use - * @param title The title/header for this log section - * @param logDetails Map of key-value pairs to log - */ - public static void logError(Logger logger, String title, Map logDetails) { - if (logDetails != null && !logDetails.isEmpty()) { - logger.error("{}:", title); - logDetails.forEach((key, value) -> logger.error(" {} = [{}]", key, value)); - } - } - /** * Create a builder for constructing LogDetails maps to log * @param title The title for this log section @@ -82,7 +43,7 @@ public static LogBuilder builder(String title) { * Builder class for constructing structured log messages */ public static class LogBuilder { - private final String title; + private final String title; private final Map logDetails = new LinkedHashMap<>(); private LogBuilder(String title) { @@ -118,7 +79,7 @@ public LogBuilder addIfNotNull(String key, Object value) { * @param logger The logger to use */ public void logInfo(Logger logger) { - AuditServerLogFormatter.logInfo(logger, title, logDetails); + logger.info("{} {}", title, logDetails); } /** @@ -126,7 +87,7 @@ public void logInfo(Logger logger) { * @param logger The logger to use */ public void logDebug(Logger logger) { - AuditServerLogFormatter.logDebug(logger, title, logDetails); + logger.debug("{} {}", title, logDetails); } /** @@ -134,15 +95,7 @@ public void logDebug(Logger logger) { * @param logger The logger to use */ public void logError(Logger logger) { - AuditServerLogFormatter.logError(logger, title, logDetails); - } - - /** - * Get the LogDetails map - * @return The LogDetails map - */ - public Map getLogDetails() { - return new LinkedHashMap<>(logDetails); + logger.error("{} {}", title, logDetails); } } } diff --git a/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditServerUtils.java b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditServerUtils.java new file mode 100644 index 0000000000..9f5d877a4a --- /dev/null +++ b/audit-server/audit-common/src/main/java/org/apache/ranger/audit/utils/AuditServerUtils.java @@ -0,0 +1,78 @@ +/* + * 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.ranger.audit.utils; + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; + +import java.time.Duration; +import java.util.Collections; +import java.util.concurrent.ThreadLocalRandom; + +public class AuditServerUtils { + private AuditServerUtils() { + } + + public static boolean waitUntilTopicReady(Admin admin, String topic, Duration totalWait) throws Exception { + long endTime = System.nanoTime() + totalWait.toNanos(); + long baseSleepMs = 100L; + long maxSleepMs = 2000L; + + while (System.nanoTime() < endTime) { + try { + DescribeTopicsResult describeTopicsResult = admin.describeTopics(Collections.singleton(topic)); + TopicDescription topicDescription = describeTopicsResult.values().get(topic).get(); + boolean allHaveLeader = topicDescription.partitions().stream().allMatch(partitionInfo -> partitionInfo.leader() != null); + boolean allHaveISR = topicDescription.partitions().stream().allMatch(partitionInfo -> !partitionInfo.isr().isEmpty()); + + if (allHaveLeader && allHaveISR) { + return true; + } + } catch (Exception e) { + // If topic hasn't propagated yet, you'll see UnknownTopicOrPartitionException + // continue to wait for topic availability + if (!(rootCause(e) instanceof UnknownTopicOrPartitionException)) { + throw e; + } + } + + // Sleep until the created topic is available for metadata fetch + baseSleepMs = Math.min(maxSleepMs, baseSleepMs * 2); + + long sleep = baseSleepMs + ThreadLocalRandom.current().nextLong(0, baseSleepMs / 2 + 1); + + Thread.sleep(sleep); + } + + return false; + } + + private static Throwable rootCause(Throwable t) { + Throwable throwable = t; + + while (throwable.getCause() != null) { + throwable = throwable.getCause(); + } + + return throwable; + } +} diff --git a/audit-server/audit-dispatcher/dispatcher-app/pom.xml b/audit-server/audit-dispatcher/dispatcher-app/pom.xml new file mode 100644 index 0000000000..70bd54b4b3 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-app/pom.xml @@ -0,0 +1,91 @@ + + + + 4.0.0 + + + org.apache.ranger + ranger + 3.0.0-SNAPSHOT + ../../.. + + + ranger-audit-dispatcher-app + war + Ranger Audit Dispatcher App + Unified Ranger Kafka service for writing audits to HDFS/S3/Azure/Solr + + + + com.sun.jersey.contribs + jersey-spring + ${jersey-spring.version} + + + com.github.pjfanning + jersey-json + + + com.sun.jersey + jersey-server + + + org.springframework + * + + + + + org.apache.ranger + ranger-audit-dispatcher-common + ${project.version} + + + org.springframework + spring-context + ${springframework.version} + + + org.springframework + spring-web + ${springframework.version} + + + org.apache.ranger + audit-dispatcher-hdfs + ${project.version} + provided + + + org.apache.ranger + audit-dispatcher-solr + ${project.version} + provided + + + + + ranger-audit-dispatcher-${project.version} + + + org.apache.maven.plugins + maven-war-plugin + + + + diff --git a/audit-server/audit-dispatcher/dispatcher-app/src/main/java/org/apache/ranger/audit/dispatcher/AuditDispatcherApplication.java b/audit-server/audit-dispatcher/dispatcher-app/src/main/java/org/apache/ranger/audit/dispatcher/AuditDispatcherApplication.java new file mode 100644 index 0000000000..36f5aef06c --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-app/src/main/java/org/apache/ranger/audit/dispatcher/AuditDispatcherApplication.java @@ -0,0 +1,95 @@ +/* + * 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.ranger.audit.dispatcher; + +import org.apache.ranger.audit.server.AuditConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +public class AuditDispatcherApplication { + private static final Logger LOG = LoggerFactory.getLogger(AuditDispatcherApplication.class); + private static final String APP_NAME = "audit-dispatcher"; + private static final String CONFIG_PREFIX = "ranger.audit.dispatcher."; + private static final String COMMON_CONFIG_FILE = "ranger-audit-dispatcher-site.xml"; + + private AuditDispatcherApplication() { + } + + public static void main(String[] args) { + AuditConfig config = new AuditConfig(); + config.addResourceIfReadable(COMMON_CONFIG_FILE); + LOG.info("Loaded common configuration from classpath: {}", COMMON_CONFIG_FILE); + + String dispatcherType = System.getProperty(CONFIG_PREFIX + "type"); + if (dispatcherType == null) { + dispatcherType = config.get(CONFIG_PREFIX + "type"); + } + + // Load dispatcher-specific configuration from classpath + if (dispatcherType != null) { + String specificConfig = "ranger-audit-dispatcher-" + dispatcherType + "-site.xml"; + config.addResourceIfReadable(specificConfig); + LOG.info("Loaded dispatcher-specific configuration from classpath: {}", specificConfig); + } else { + LOG.warn("No dispatcher type specified. Service might fail to start correctly."); + } + + LOG.info("=========================================================================="); + LOG.info("==> Starting Ranger Audit Dispatcher Service (Type: {})", dispatcherType); + LOG.info("=========================================================================="); + + // Initialization dispatcher manager based on dispatcher type before starting EmbeddedServer + boolean initSuccess = false; + try { + String dispatcherMgrClass = config.get(CONFIG_PREFIX + dispatcherType + ".class"); + if (dispatcherMgrClass != null && !dispatcherMgrClass.trim().isEmpty()) { + initSuccess = initializeDispatcherManager(dispatcherMgrClass, config.getProperties()); + } else { + LOG.error("Unknown dispatcher type: {}. Cannot initialize dispatcher manager.", dispatcherType); + } + } catch (Exception e) { + LOG.error("Failed to initialize DispatcherManager", e); + } + + if (!initSuccess) { + LOG.error("Dispatcher initialization failed."); + System.exit(1); + } + + try { + EmbeddedServer server = new EmbeddedServer(config, APP_NAME, CONFIG_PREFIX); + server.start(); + + LOG.info("<== Ranger Audit Dispatcher Service Started Successfully"); + } catch (Exception e) { + LOG.error("<== Failed to start Ranger Audit Dispatcher Service", e); + System.exit(1); + } + } + + private static boolean initializeDispatcherManager(String dispatcherMgrClass, Properties props) throws Exception { + Object manager = Class.forName(dispatcherMgrClass, true, Thread.currentThread().getContextClassLoader()).newInstance(); + manager.getClass().getMethod("init", Properties.class).invoke(manager, props); + LOG.info("{} initialized successfully", dispatcherMgrClass); + return true; + } +} diff --git a/audit-server/audit-dispatcher/dispatcher-app/src/main/java/org/apache/ranger/audit/rest/AuditDispatcherHealthREST.java b/audit-server/audit-dispatcher/dispatcher-app/src/main/java/org/apache/ranger/audit/rest/AuditDispatcherHealthREST.java new file mode 100644 index 0000000000..4a61b20fe5 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-app/src/main/java/org/apache/ranger/audit/rest/AuditDispatcherHealthREST.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.ranger.audit.rest; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.ranger.audit.dispatcher.kafka.AuditDispatcherTracker; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.context.annotation.Scope; +import org.springframework.stereotype.Component; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Response; + +import java.util.HashMap; +import java.util.Map; + +/** + * Health check REST endpoint for Audit Dispatcher Service + */ +@Path("/health") +@Component +@Scope("request") +public class AuditDispatcherHealthREST { + private static final Logger LOG = LoggerFactory.getLogger(AuditDispatcherHealthREST.class); + + /** + * Simple ping endpoint to verify the service is running + */ + @GET + @Path("/ping") + @Produces("application/json") + public Response ping() { + Map resp = new HashMap<>(); + resp.put("status", "UP"); + resp.put("service", "audit-dispatcher"); + + try { + ObjectMapper mapper = new ObjectMapper(); + return Response.ok(mapper.writeValueAsString(resp)).build(); + } catch (Exception e) { + LOG.error("Error creating ping response", e); + return Response.serverError().build(); + } + } + + /** + * Detailed health check endpoint that verifies internal components + */ + @GET + @Path("/status") + @Produces("application/json") + public Response status() { + Map resp = new HashMap<>(); + try { + String dispatcherType = System.getProperty("ranger.audit.dispatcher.type"); + resp.put("service", "audit-dispatcher-" + (dispatcherType != null ? dispatcherType : "unknown")); + + if (dispatcherType != null && !dispatcherType.trim().isEmpty()) { + String auditDispatcherType = dispatcherType.toLowerCase(); + boolean isActive = AuditDispatcherTracker.getInstance().getActiveDispatchers().stream() + .filter(d -> d != null) + .anyMatch(d -> d.getClass().getName().toLowerCase().contains(auditDispatcherType)); + + if (isActive) { + resp.put("status", "UP"); + } else { + resp.put("status", "DOWN"); + resp.put("reason", dispatcherType + " Dispatcher is not active"); + } + } else { + resp.put("status", "DOWN"); + resp.put("reason", "Unknown dispatcher type: " + dispatcherType); + } + + ObjectMapper mapper = new ObjectMapper(); + return Response.ok(mapper.writeValueAsString(resp)).build(); + } catch (Exception e) { + LOG.error("Error checking status", e); + resp.put("status", "ERROR"); + resp.put("error", e.getMessage()); + return Response.serverError().entity(resp).build(); + } + } +} diff --git a/audit-server/consumer-solr/src/main/webapp/WEB-INF/applicationContext.xml b/audit-server/audit-dispatcher/dispatcher-app/src/main/webapp/WEB-INF/applicationContext.xml similarity index 92% rename from audit-server/consumer-solr/src/main/webapp/WEB-INF/applicationContext.xml rename to audit-server/audit-dispatcher/dispatcher-app/src/main/webapp/WEB-INF/applicationContext.xml index b6913c970e..b580d710fb 100644 --- a/audit-server/consumer-solr/src/main/webapp/WEB-INF/applicationContext.xml +++ b/audit-server/audit-dispatcher/dispatcher-app/src/main/webapp/WEB-INF/applicationContext.xml @@ -23,9 +23,7 @@ http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-4.3.xsd"> - - - + diff --git a/audit-server/consumer-hdfs/src/main/webapp/WEB-INF/web.xml b/audit-server/audit-dispatcher/dispatcher-app/src/main/webapp/WEB-INF/web.xml similarity index 93% rename from audit-server/consumer-hdfs/src/main/webapp/WEB-INF/web.xml rename to audit-server/audit-dispatcher/dispatcher-app/src/main/webapp/WEB-INF/web.xml index 995d91281a..49598c8928 100644 --- a/audit-server/consumer-hdfs/src/main/webapp/WEB-INF/web.xml +++ b/audit-server/audit-dispatcher/dispatcher-app/src/main/webapp/WEB-INF/web.xml @@ -17,8 +17,8 @@ --> - Apache Ranger - Audit Consumer HDFS Service - Apache Ranger - Audit Consumer HDFS Service + Apache Ranger - Audit Dispatcher Service + Apache Ranger - Audit Dispatcher Service contextConfigLocation diff --git a/audit-server/common/pom.xml b/audit-server/audit-dispatcher/dispatcher-common/pom.xml similarity index 65% rename from audit-server/common/pom.xml rename to audit-server/audit-dispatcher/dispatcher-common/pom.xml index 86fef743b8..7425dbcd32 100644 --- a/audit-server/common/pom.xml +++ b/audit-server/audit-dispatcher/dispatcher-common/pom.xml @@ -1,21 +1,20 @@ + 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. +--> 4.0.0 @@ -23,15 +22,20 @@ org.apache.ranger ranger 3.0.0-SNAPSHOT - ../.. + ../../.. - ranger-audit-common + ranger-audit-dispatcher-common jar - Ranger Audit Common - Shared utilities and base classes for Ranger Audit Server services + Ranger Audit Dispatcher Common + Shared dispatcher framework for Kafka-based audit dispatchers + + ch.qos.logback + logback-classic + ${logback.version} + com.google.guava guava @@ -43,7 +47,6 @@ commons-io ${commons.io.version} - org.apache.commons commons-lang3 @@ -81,13 +84,18 @@ - org.apache.ranger ranger-audit-core ${project.version} + + + org.apache.ranger + ranger-audit-server-common + ${project.version} + org.apache.ranger ranger-plugins-common @@ -100,28 +108,39 @@ + org.apache.tomcat tomcat-annotations-api ${tomcat.embed.version} - org.apache.tomcat.embed tomcat-embed-core ${tomcat.embed.version} + + org.apache.tomcat.embed + tomcat-embed-jasper + ${tomcat.embed.version} + + org.slf4j slf4j-api ${slf4j.version} + org.springframework spring-context ${springframework.version} + + + ranger-audit-dispatcher-common-${project.version} + diff --git a/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/AuditDispatcherLauncher.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/AuditDispatcherLauncher.java new file mode 100644 index 0000000000..f18da136bc --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/AuditDispatcherLauncher.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.ranger.audit.dispatcher; + +import org.apache.ranger.audit.server.AuditConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.lang.reflect.Method; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.List; + +public class AuditDispatcherLauncher { + private static final Logger LOG = LoggerFactory.getLogger(AuditDispatcherLauncher.class); + + private static final String COMMON_CONFIG_FILE = "ranger-audit-dispatcher-site.xml"; + private static final String CONFIG_DISPATCHER_TYPE = "ranger.audit.dispatcher.type"; + + private AuditDispatcherLauncher() { + } + + public static void main(String[] args) { + try { + AuditConfig config = new AuditConfig(); + config.addResourceIfReadable(COMMON_CONFIG_FILE); + + String dispatcherType = System.getProperty(CONFIG_DISPATCHER_TYPE); + if (dispatcherType == null) { + dispatcherType = config.get(CONFIG_DISPATCHER_TYPE); + if (dispatcherType != null) { + System.setProperty(CONFIG_DISPATCHER_TYPE, dispatcherType); + } + } + + if (dispatcherType == null) { + LOG.error("Dispatcher type is not specified. Cannot determine which dispatcher jars to load."); + System.exit(1); + } + + LOG.info("Initializing classloader for dispatcher type: {}", dispatcherType); + + // Build dynamic classpath + String homeDir = System.getenv("AUDIT_DISPATCHER_HOME_DIR"); + if (homeDir == null) { + homeDir = System.getProperty("user.dir"); + } + + File dispatcherLibDir = new File(homeDir, "lib/dispatchers/" + dispatcherType); + List urls = new ArrayList<>(); + + if (dispatcherLibDir.exists() && dispatcherLibDir.isDirectory()) { + File[] jars = dispatcherLibDir.listFiles((dir, name) -> name.endsWith(".jar")); + if (jars != null) { + for (File jar : jars) { + urls.add(jar.toURI().toURL()); + } + } + } else { + LOG.warn("Dispatcher lib directory does not exist: {}", dispatcherLibDir.getAbsolutePath()); + } + + ClassLoader parentClassLoader = AuditDispatcherLauncher.class.getClassLoader(); + URLClassLoader dispatcherClassLoader = new URLClassLoader(urls.toArray(new URL[0]), parentClassLoader); + + Thread.currentThread().setContextClassLoader(dispatcherClassLoader); + + String mainClassName = config.get("ranger.audit.dispatcher.main.class", "org.apache.ranger.audit.dispatcher.AuditDispatcherApplication"); + + LOG.info("Launching main class: {}", mainClassName); + + Class mainClass = Class.forName(mainClassName, true, dispatcherClassLoader); + Method mainMethod = mainClass.getMethod("main", String[].class); + mainMethod.invoke(null, (Object) args); + } catch (Exception e) { + LOG.error("Failed to launch audit dispatcher", e); + System.exit(1); + } + } +} diff --git a/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/EmbeddedServer.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/EmbeddedServer.java new file mode 100644 index 0000000000..e169381ab6 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/EmbeddedServer.java @@ -0,0 +1,784 @@ +/* + * 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.ranger.audit.dispatcher; + +import org.apache.catalina.Context; +import org.apache.catalina.LifecycleException; +import org.apache.catalina.WebResourceRoot; +import org.apache.catalina.connector.Connector; +import org.apache.catalina.core.StandardContext; +import org.apache.catalina.startup.Tomcat; +import org.apache.catalina.valves.AccessLogValve; +import org.apache.catalina.webresources.StandardRoot; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.SecureClientLogin; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.alias.BouncyCastleFipsKeyStoreProvider; +import org.apache.hadoop.security.alias.CredentialProvider; +import org.apache.hadoop.security.alias.CredentialProviderFactory; +import org.apache.hadoop.security.alias.JavaKeyStoreProvider; +import org.apache.hadoop.security.alias.LocalBouncyCastleFipsKeyStoreProvider; +import org.apache.ranger.audit.provider.MiscUtil; +import org.apache.ranger.audit.server.AuditConfig; +import org.apache.ranger.audit.server.AuditServerConstants; +import org.apache.ranger.audit.utils.AuditServerLogFormatter; +import org.apache.ranger.authorization.hadoop.utils.RangerCredentialProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.util.Date; +import java.util.List; +import java.util.Map; + +public class EmbeddedServer { + private static final Logger LOG = LoggerFactory.getLogger(EmbeddedServer.class); + + private static final String ACCESS_LOG_PREFIX = "accesslog.prefix"; + private static final String ACCESS_LOG_DATE_FORMAT = "accesslog.dateformat"; + private static final String ACCESS_LOG_PATTERN = "accesslog.pattern"; + private static final String ACCESS_LOG_ROTATE_MAX_DAYS = "accesslog.rotate.max.days"; + private static final String DEFAULT_LOG_DIR = "/tmp"; + private static final int DEFAULT_HTTP_PORT = 6081; + private static final int DEFAULT_HTTPS_PORT = -1; + private static final int DEFAULT_SHUTDOWN_PORT = 6185; + private static final String DEFAULT_SHUTDOWN_COMMAND = "SHUTDOWN"; + private static final String DEFAULT_WEBAPPS_ROOT_FOLDER = "webapps"; + private static final String DEFAULT_ENABLED_PROTOCOLS = "TLSv1.2"; + public static final String DEFAULT_NAME_RULE = "DEFAULT"; + private static final String RANGER_KEYSTORE_FILE_TYPE_DEFAULT = "jks"; + private static final String RANGER_TRUSTSTORE_FILE_TYPE_DEFAULT = "jks"; + private static final String RANGER_SSL_CONTEXT_ALGO_TYPE = "TLSv1.2"; + private static final String RANGER_SSL_KEYMANAGER_ALGO_TYPE = KeyManagerFactory.getDefaultAlgorithm(); + private static final String RANGER_SSL_TRUSTMANAGER_ALGO_TYPE = TrustManagerFactory.getDefaultAlgorithm(); + public static final String KEYSTORE_FILE_TYPE_DEFAULT = KeyStore.getDefaultType(); + public static final String TRUSTSTORE_FILE_TYPE_DEFAULT = KeyStore.getDefaultType(); + + private final Configuration configuration; + private final String appName; + private final String configPrefix; + private volatile Tomcat server; + private volatile Context webappContext; + + public EmbeddedServer(Configuration configuration, String appName, String configPrefix) { + LOG.info("==> EmbeddedServer(appName={}, configPrefix={})", appName, configPrefix); + + this.configuration = new Configuration(configuration); + this.appName = appName; + this.configPrefix = configPrefix; + + LOG.info("<== EmbeddedServer(appName={}, configPrefix={}", appName, configPrefix); + } + + public static void main(String[] args) { + Configuration config = new AuditConfig(); + String appName = AuditServerConstants.AUDIT_SERVER_APPNAME; + String configPrefix = AuditServerConstants.PROP_PREFIX_AUDIT_SERVER; + + new EmbeddedServer(config, appName, configPrefix).start(); + } + + @SuppressWarnings("deprecation") + public void start() { + LOG.info("==> EmbeddedServer.start(appName={})", appName); + + SSLContext sslContext = getSSLContext(); + + if (sslContext != null) { + SSLContext.setDefault(sslContext); + } + + this.server = new Tomcat(); + + // Add shutdown hook for graceful cleanup + addShutdownHook(); + + String logDir = getConfig("log.dir", DEFAULT_LOG_DIR); + String hostName = getConfig("host"); + int serverPort = getIntConfig("http.port", DEFAULT_HTTP_PORT); + int sslPort = getIntConfig("https.port", DEFAULT_HTTPS_PORT); + int shutdownPort = getIntConfig("shutdown.port", DEFAULT_SHUTDOWN_PORT); + String shutdownCommand = getConfig("shutdown.command", DEFAULT_SHUTDOWN_COMMAND); + boolean isHttpsEnabled = getBooleanConfig("https.attrib.ssl.enabled", false); + boolean ajpEnabled = getBooleanConfig("ajp.enabled", false); + + server.setHostname(hostName); + server.setPort(serverPort); + server.getServer().setPort(shutdownPort); + server.getServer().setShutdown(shutdownCommand); + + if (ajpEnabled) { + Connector ajpConnector = new Connector("org.apache.coyote.ajp.AjpNioProtocol"); + + ajpConnector.setPort(serverPort); + ajpConnector.setProperty("protocol", "AJP/1.3"); + + server.getService().addConnector(ajpConnector); + + // Making this as a default connector + server.setConnector(ajpConnector); + + LOG.info("Created AJP Connector"); + } else if (isHttpsEnabled && sslPort > 0) { + String clientAuth = getConfig("https.attrib.clientAuth", "false"); + String providerPath = getConfig("credential.provider.path"); + String keyAlias = getConfig("https.attrib.keystore.credential.alias", "keyStoreCredentialAlias"); + String keystorePass = null; + String enabledProtocols = getConfig("https.attrib.ssl.enabled.protocols", DEFAULT_ENABLED_PROTOCOLS); + String ciphers = getConfig("tomcat.ciphers"); + + if (StringUtils.equalsIgnoreCase(clientAuth, "false")) { + clientAuth = getConfig("https.attrib.client.auth", "want"); + } + + if (providerPath != null && keyAlias != null) { + keystorePass = getDecryptedString(providerPath.trim(), keyAlias.trim(), getConfig("keystore.file.type", KEYSTORE_FILE_TYPE_DEFAULT)); + + if (StringUtils.isBlank(keystorePass) || StringUtils.equalsIgnoreCase(keystorePass.trim(), "none")) { + keystorePass = getConfig("https.attrib.keystore.pass"); + } + } + + Connector ssl = new Connector(); + String sslKeystoreKeyAlias = getConfig("https.attrib.keystore.keyalias", ""); + ssl.setPort(sslPort); + ssl.setSecure(true); + ssl.setScheme("https"); + ssl.setAttribute("SSLEnabled", "true"); + ssl.setAttribute("sslProtocol", getConfig("https.attrib.ssl.protocol", "TLSv1.2")); + ssl.setAttribute("clientAuth", clientAuth); + if (StringUtils.isNotBlank(sslKeystoreKeyAlias)) { + ssl.setAttribute("keyAlias", sslKeystoreKeyAlias); + } + ssl.setAttribute("keystorePass", keystorePass); + ssl.setAttribute("keystoreFile", getKeystoreFile()); + ssl.setAttribute("sslEnabledProtocols", enabledProtocols); + ssl.setAttribute("keystoreType", getConfig("audit.keystore.file.type", KEYSTORE_FILE_TYPE_DEFAULT)); + ssl.setAttribute("truststoreType", getConfig("audit.truststore.file.type", TRUSTSTORE_FILE_TYPE_DEFAULT)); + + if (StringUtils.isNotBlank(ciphers)) { + ssl.setAttribute("ciphers", ciphers); + } + + server.getService().addConnector(ssl); + + // + // Making this as a default connector + // + server.setConnector(ssl); + } + + updateHttpConnectorAttribConfig(server); + + File logDirectory = new File(logDir); + + if (!logDirectory.exists()) { + boolean created = logDirectory.mkdirs(); + if (!created) { + LOG.error("Failed to create log directory: {}", logDir); + throw new RuntimeException("Failed to create log directory: " + logDir); + } + } + + String logPattern = getConfig(ACCESS_LOG_PATTERN, "%h %l %u %t \"%r\" %s %b"); + + AccessLogValve valve = new AccessLogValve(); + + valve.setRotatable(true); + valve.setAsyncSupported(true); + valve.setBuffered(false); + valve.setEnabled(true); + valve.setPrefix(getConfig(ACCESS_LOG_PREFIX, "access_log-" + hostName + "-")); + valve.setFileDateFormat(getConfig(ACCESS_LOG_DATE_FORMAT, "yyyy-MM-dd.HH")); + valve.setDirectory(logDirectory.getAbsolutePath()); + valve.setSuffix(".log"); + valve.setPattern(logPattern); + valve.setMaxDays(getIntConfig(ACCESS_LOG_ROTATE_MAX_DAYS, 15)); + + server.getHost().getPipeline().addValve(valve); + + try { + String webappDir = getConfig("webapp.dir"); + + if (StringUtils.isBlank(webappDir)) { + LOG.error("Tomcat Server failed to start: {}.webapp.dir is not set", configPrefix); + + System.exit(1); + } + + String webContextName = getConfig("contextName", ""); + + if (StringUtils.isBlank(webContextName)) { + webContextName = ""; + } else if (!webContextName.startsWith("/")) { + LOG.info("Context Name [{}] is being loaded as [ /{}]", webContextName, webContextName); + + webContextName = "/" + webContextName; + } + + File wad = new File(webappDir); + + if (wad.isDirectory()) { + LOG.info("Webapp dir={}, webAppName={}", webappDir, webContextName); + } else if (wad.isFile()) { + File webAppDir = new File(DEFAULT_WEBAPPS_ROOT_FOLDER); + + if (!webAppDir.exists()) { + boolean created = webAppDir.mkdirs(); + if (!created) { + LOG.error("Failed to create webapp directory: {}", DEFAULT_WEBAPPS_ROOT_FOLDER); + throw new RuntimeException("Failed to create webapp directory: " + DEFAULT_WEBAPPS_ROOT_FOLDER); + } + } + + LOG.info("Webapp file={}, webAppName={}", webappDir, webContextName); + } + + LOG.info("Adding webapp [{}] = path [{}] .....", webContextName, webappDir); + + this.webappContext = server.addWebapp(webContextName, new File(webappDir).getAbsolutePath()); + + // Set the parent classloader to the current thread context classloader + // This is crucial for isolated classloading so Tomcat can find the dispatcher classes + this.webappContext.setParentClassLoader(Thread.currentThread().getContextClassLoader()); + + if (webappContext instanceof StandardContext) { + boolean allowLinking = getBooleanConfig("allow.linking", true); + StandardContext standardContext = (StandardContext) webappContext; + standardContext.setDelegate(true); + String workDirPath = getConfig("tomcat.work.dir", ""); + if (!workDirPath.isEmpty() && new File(workDirPath).exists()) { + standardContext.setWorkDir(workDirPath); + } else { + LOG.debug("Skipping to set tomcat server work directory {} as it is blank or directory does not exist.", workDirPath); + } + WebResourceRoot resRoot = new StandardRoot(standardContext); + webappContext.setResources(resRoot); + webappContext.getResources().setAllowLinking(allowLinking); + LOG.debug("Tomcat Configuration - allowLinking=[{}]", allowLinking); + } else { + LOG.error("Tomcat Context [{}] is either NULL OR it's NOT instanceof StandardContext", webappContext); + } + + webappContext.init(); + + LOG.info("Finished init of webapp [{}] = path [{}].", webContextName, webappDir); + } catch (LifecycleException lce) { + LOG.error("Tomcat Server failed to start webapp", lce); + } + + String authenticationType = getConfig(AuditServerConstants.PROP_HADOOP_AUTHENTICATION_TYPE); + + if (StringUtils.equalsIgnoreCase(authenticationType, AuditServerConstants.PROP_HADOOP_AUTH_TYPE_KERBEROS)) { + String keyTab = getConfig(AuditServerConstants.PROP_AUDIT_SERVICE_KEYTAB); + String principal = null; + String nameRules = getConfig(AuditServerConstants.PROP_HADOOP_KERBEROS_NAME_RULES, DEFAULT_NAME_RULE); + + try { + principal = SecureClientLogin.getPrincipal(getConfig(AuditServerConstants.PROP_AUDIT_SERVICE_PRINCIPAL), hostName); + } catch (IOException excp) { + LOG.warn("Failed to get principal" + excp); + } + + if (SecureClientLogin.isKerberosCredentialExists(principal, keyTab)) { + try { + AuditServerLogFormatter.builder("Kerberos Login Attempt") + .add("Principal", principal) + .add("Keytab", keyTab) + .add("Name rules", nameRules) + .logInfo(LOG); + + Configuration hadoopConf = new Configuration(); + hadoopConf.set(AuditServerConstants.PROP_HADOOP_AUTHENTICATION_TYPE, AuditServerConstants.PROP_HADOOP_AUTH_TYPE_KERBEROS); + hadoopConf.set(AuditServerConstants.PROP_HADOOP_KERBEROS_NAME_RULES, nameRules); + UserGroupInformation.setConfiguration(hadoopConf); + UserGroupInformation.loginUserFromKeytab(principal, keyTab); + UserGroupInformation currentUGI = UserGroupInformation.getLoginUser(); + + AuditServerLogFormatter.builder("Kerberos Login Successful") + .add("UGI Username", currentUGI.getUserName()) + .add("UGI Real User", currentUGI.getUserName()) + .add("Authentication Method", currentUGI.getAuthenticationMethod().toString()) + .add("Has Kerberos Credentials", currentUGI.hasKerberosCredentials()) + .logInfo(LOG); + + MiscUtil.setUGILoginUser(currentUGI, null); + + LOG.info("Starting Server using kerberos credential"); + startServer(server); + } catch (Exception excp) { + LOG.error("Tomcat Server failed to start", excp); + } + } else { + LOG.warn("Kerberos principal={} not found in keytab={}. Starting server in non-kerberos mode", principal, keyTab); + + startServer(server); + } + } else { + LOG.info("Starting server in non-kerberos mode"); + + startServer(server); + } + + LOG.info("<== EmbeddedServer.start(appName={})", appName); + } + + public String getDecryptedString(String credentialProviderPath, String alias, String storeType) { + String ret = null; + + if (StringUtils.isNotBlank(credentialProviderPath) && StringUtils.isNotBlank(alias)) { + try { + Configuration conf = new Configuration(); + String prefixJceks = JavaKeyStoreProvider.SCHEME_NAME + "://file"; + String prefixLocalJceks = "localjceks://file"; + String prefixBcfks = BouncyCastleFipsKeyStoreProvider.SCHEME_NAME + "://file"; + String prefixLocalBcfks = LocalBouncyCastleFipsKeyStoreProvider.SCHEME_NAME + "://file"; + + String lowerPath = credentialProviderPath.toLowerCase(); + if (lowerPath.startsWith(prefixJceks.toLowerCase()) || lowerPath.startsWith(prefixLocalJceks.toLowerCase()) || lowerPath.startsWith(prefixBcfks.toLowerCase()) || lowerPath.startsWith(prefixLocalBcfks.toLowerCase())) { + conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, credentialProviderPath); + } else { + if (credentialProviderPath.startsWith("/")) { + if ("bcfks".equalsIgnoreCase(storeType)) { + conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, credentialProviderPath); + } else { + conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, prefixJceks + credentialProviderPath); + } + } else { + conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, prefixJceks + "/" + credentialProviderPath); + } + } + + List providers = CredentialProviderFactory.getProviders(conf); + + LOG.debug("CredentialProviderPath={} alias={} storeType={}", credentialProviderPath, alias, storeType); + LOG.debug("List of CredentialProvider = {}", providers.toString()); + + for (CredentialProvider provider : providers) { + List aliasesList = provider.getAliases(); + + if (CollectionUtils.isNotEmpty(aliasesList) && aliasesList.contains(alias)) { + CredentialProvider.CredentialEntry credEntry = provider.getCredentialEntry(alias); + char[] pass = credEntry.getCredential(); + + if (pass != null && pass.length > 0) { + ret = String.valueOf(pass); + + break; + } + } + } + } catch (Exception ex) { + LOG.error("CredentialReader failed while decrypting provided string. Reason: {}", ex.toString()); + + ret = null; + } + } + + LOG.debug("getDecryptedString() : ret = {}", ret); + + return ret; + } + + protected long getLongConfig(String key, long defaultValue) { + long ret = defaultValue; + String retStr = getConfig(key); + + try { + if (retStr != null) { + ret = Long.parseLong(retStr); + } + } catch (Exception err) { + LOG.warn(retStr + " can't be parsed to long. Reason: {}", err.toString()); + } + + return ret; + } + + private void startServer(final Tomcat server) { + LOG.info("==> EmbeddedServer.startServer(appName={})", appName); + + try { + server.start(); + + server.getServer().await(); + + shutdownServer(); + } catch (LifecycleException e) { + LOG.error("Tomcat Server failed to start", e); + + e.printStackTrace(); + } catch (Exception e) { + LOG.error("Tomcat Server failed to start", e); + + e.printStackTrace(); + } + + LOG.info("<== EmbeddedServer.startServer(appName={})", appName); + } + + private void addShutdownHook() { + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + LOG.info("==> EmbeddedServer shutdown hook triggered"); + try { + gracefulShutdown(); + } catch (Exception e) { + LOG.error("Error during graceful shutdown", e); + } + LOG.info("<== EmbeddedServer shutdown hook completed"); + }, "EmbeddedServer-ShutdownHook")); + } + + private void gracefulShutdown() { + LOG.info("==> EmbeddedServer.gracefulShutdown()"); + + // Instead of trying to manually shutdown Spring context, let's rely on Tomcat's + // normal webapp lifecycle which will trigger ContextLoaderListener.contextDestroyed() + if (server != null) { + try { + LOG.info("Initiating Tomcat server stop to trigger webapp lifecycle shutdown"); + server.stop(); + LOG.info("Tomcat server stop completed"); + } catch (Exception e) { + LOG.error("Error stopping Tomcat server during graceful shutdown", e); + + // Fallback: try to stop the webapp context directly + if (webappContext != null) { + try { + LOG.info("Fallback: stopping webapp context directly"); + webappContext.stop(); + LOG.info("Webapp context stopped"); + } catch (Exception contextE) { + LOG.error("Error stopping webapp context", contextE); + } + } + } + } + + // Give some time for Spring components to shutdown through normal lifecycle + try { + Thread.sleep(3000); + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for Spring shutdown", e); + Thread.currentThread().interrupt(); + } + + LOG.info("<== EmbeddedServer.gracefulShutdown()"); + } + + private void shutdownServer() { + LOG.info("==> EmbeddedServer.shutdownServer(appName={})", appName); + + int timeWaitForShutdownInSeconds = getIntConfig("waitTimeForForceShutdownInSeconds", 0); + + if (timeWaitForShutdownInSeconds > 0) { + long endTime = System.currentTimeMillis() + (timeWaitForShutdownInSeconds * 1000L); + + LOG.info("Will wait for all threads to shutdown gracefully. Final shutdown Time: {}", new Date(endTime)); + + while (System.currentTimeMillis() < endTime) { + int activeCount = Thread.activeCount(); + + if (activeCount == 0) { + LOG.info("Number of active threads = {}", activeCount); + + break; + } else { + LOG.info("Number of active threads = {}. Waiting for all threads to shutdown ...", activeCount); + + try { + Thread.sleep(5000L); + } catch (InterruptedException e) { + LOG.warn("shutdownServer process is interrupted with exception", e); + + break; + } + } + } + } + + LOG.info("Shutting down the Server."); + + LOG.info("<== EmbeddedServer.shutdownServer(appName={})", appName); + + System.exit(0); + } + + @SuppressWarnings("deprecation") + private void updateHttpConnectorAttribConfig(Tomcat server) { + server.getConnector().setAllowTrace(getBooleanConfig("http.connector.attrib.allowTrace", false)); + server.getConnector().setAsyncTimeout(getLongConfig("http.connector.attrib.asyncTimeout", 10000)); + server.getConnector().setEnableLookups(getBooleanConfig("http.connector.attrib.enableLookups", false)); + server.getConnector().setMaxParameterCount(getIntConfig("http.connector.attrib.maxParameterCount", 10000)); + server.getConnector().setMaxPostSize(getIntConfig("http.connector.attrib.maxPostSize", 2097152)); + server.getConnector().setMaxSavePostSize(getIntConfig("http.connector.attrib.maxSavePostSize", 4096)); + server.getConnector().setParseBodyMethods(getConfig("http.connector.attrib.methods", "POST")); + server.getConnector().setURIEncoding(getConfig("http.connector.attrib.URIEncoding", "UTF-8")); + server.getConnector().setProperty("acceptCount", getConfig("http.connector.attrib.acceptCount", "1024")); + server.getConnector().setXpoweredBy(false); + server.getConnector().setAttribute("server", getConfig("servername", "Audit Server")); + + int maxThreads = getIntConfig("http.connector.attrib.maxThreads", 2000); + int maxKeepAliveRequests = getIntConfig("http.connector.attrib.maxKeepAliveRequests", 1000); + int minSpareThreads = getIntConfig("http.connector.attrib.minSpareThreads", (int) (maxThreads * 0.8)); // (default 80% of maxThreads) + boolean prestartminSpareThreads = getBooleanConfig("http.connector.attrib.prestartminSpareThreads", true); + + server.getConnector().setAttribute("maxThreads", maxThreads); + server.getConnector().setAttribute("maxKeepAliveRequests", maxKeepAliveRequests); + server.getConnector().setAttribute("minSpareThreads", minSpareThreads); + server.getConnector().setAttribute("prestartminSpareThreads", prestartminSpareThreads); + server.getConnector().setProperty("sendReasonPhrase", getConfig(configPrefix + "http.connector.property.sendReasonPhrase", "true")); + + for (Map.Entry entry : configuration) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (key != null && key.startsWith(configPrefix + "http.connector.property.")) { + String property = key.replace(configPrefix + "http.connector.property.", ""); + + server.getConnector().setProperty(property, value); + + LOG.info("{}:{}", property, server.getConnector().getProperty(property)); + } + } + } + + private SSLContext getSSLContext() { + SSLContext ret = null; + KeyManager[] kmList = getKeyManagers(); + TrustManager[] tmList = getTrustManagers(); + + if (tmList != null) { + try { + ret = SSLContext.getInstance(RANGER_SSL_CONTEXT_ALGO_TYPE); + + ret.init(kmList, tmList, new SecureRandom()); + } catch (NoSuchAlgorithmException e) { + LOG.error("SSL algorithm is not available in the environment", e); + } catch (KeyManagementException e) { + LOG.error("Unable to initials the SSLContext", e); + } + } + + return ret; + } + + private KeyManager[] getKeyManagers() { + KeyManager[] ret = null; + String keyStoreFile = getConfig("keystore.file"); + String keyStoreAlias = getConfig("keystore.alias"); + String credentialProviderPath = getConfig("credential.provider.path"); + String keyStoreFilepwd = getCredential(credentialProviderPath, keyStoreAlias); + + if (StringUtils.isNotBlank(keyStoreFile) && StringUtils.isNotBlank(keyStoreFilepwd)) { + InputStream in = null; + + try { + in = getFileInputStream(keyStoreFile); + + if (in != null) { + KeyStore keyStore = KeyStore.getInstance(RANGER_KEYSTORE_FILE_TYPE_DEFAULT); + + keyStore.load(in, keyStoreFilepwd.toCharArray()); + + KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(RANGER_SSL_KEYMANAGER_ALGO_TYPE); + + keyManagerFactory.init(keyStore, keyStoreFilepwd.toCharArray()); + + ret = keyManagerFactory.getKeyManagers(); + } else { + LOG.error("Unable to obtain keystore from file [{}]", keyStoreFile); + } + } catch (KeyStoreException e) { + LOG.error("Unable to obtain from KeyStore", e); + } catch (NoSuchAlgorithmException e) { + LOG.error("SSL algorithm is NOT available in the environment", e); + } catch (CertificateException e) { + LOG.error("Unable to obtain the requested certification", e); + } catch (FileNotFoundException e) { + LOG.error("Unable to find the necessary SSL Keystore Files", e); + } catch (IOException e) { + LOG.error("Unable to read the necessary SSL Keystore Files", e); + } catch (UnrecoverableKeyException e) { + LOG.error("Unable to recover the key from keystore", e); + } finally { + close(in, keyStoreFile); + } + } + + return ret; + } + + private TrustManager[] getTrustManagers() { + TrustManager[] ret = null; + String truststoreFile = getConfig("truststore.file"); + String truststoreAlias = getConfig("truststore.alias"); + String credentialProviderPath = getConfig("credential.provider.path"); + String trustStoreFilepwd = getCredential(credentialProviderPath, truststoreAlias); + + if (StringUtils.isNotBlank(truststoreFile) && StringUtils.isNotBlank(trustStoreFilepwd)) { + InputStream in = null; + + try { + in = getFileInputStream(truststoreFile); + + if (in != null) { + KeyStore trustStore = KeyStore.getInstance(RANGER_TRUSTSTORE_FILE_TYPE_DEFAULT); + + trustStore.load(in, trustStoreFilepwd.toCharArray()); + + TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(RANGER_SSL_TRUSTMANAGER_ALGO_TYPE); + + trustManagerFactory.init(trustStore); + + ret = trustManagerFactory.getTrustManagers(); + } else { + LOG.error("Unable to obtain truststore from file [{}]", truststoreFile); + } + } catch (KeyStoreException e) { + LOG.error("Unable to obtain from KeyStore", e); + } catch (NoSuchAlgorithmException e) { + LOG.error("SSL algorithm is NOT available in the environment", e); + } catch (CertificateException e) { + LOG.error("Unable to obtain the requested certification", e); + } catch (FileNotFoundException e) { + LOG.error("Unable to find the necessary SSL TrustStore File:{}", truststoreFile, e); + } catch (IOException e) { + LOG.error("Unable to read the necessary SSL TrustStore Files:{}", truststoreFile, e); + } finally { + close(in, truststoreFile); + } + } + + return ret; + } + + private String getKeystoreFile() { + String ret = getConfig("https.attrib.keystore.file"); + + if (StringUtils.isBlank(ret)) { + // new property not configured, lets use the old property + ret = getConfig("https.attrib.keystore.file"); + } + + return ret; + } + + private String getCredential(String url, String alias) { + return RangerCredentialProvider.getInstance().getCredentialString(url, alias); + } + + private String getConfig(String key) { + String propertyWithPrefix = configPrefix + key; + String value = configuration.get(propertyWithPrefix); + if (value == null) { // config-with-prefix not found; look at System properties + value = System.getProperty(propertyWithPrefix); + if (value == null) { // config-with-prefix not found in System properties; look for config-without-prefix + value = configuration.get(key); + if (value == null) { // config-without-prefix not found; look at System properties + value = System.getProperty(key); + } + } + } + + return value; + } + + private String getConfig(String key, String defaultValue) { + String ret = getConfig(key); + + if (ret == null) { + ret = defaultValue; + } + + return ret; + } + + private int getIntConfig(String key, int defaultValue) { + int ret = defaultValue; + String strValue = getConfig(key); + + try { + if (strValue != null) { + ret = Integer.parseInt(strValue); + } + } catch (Exception err) { + LOG.warn(strValue + " can't be parsed to int. Reason: {}", err.toString()); + } + + return ret; + } + + private boolean getBooleanConfig(String key, boolean defaultValue) { + boolean ret = defaultValue; + String strValue = getConfig(key); + + if (StringUtils.isNotBlank(strValue)) { + ret = Boolean.valueOf(strValue); + } + + return ret; + } + + private InputStream getFileInputStream(String fileName) throws IOException { + InputStream ret = null; + + if (StringUtils.isNotEmpty(fileName)) { + File f = new File(fileName); + + if (f.exists()) { + ret = new FileInputStream(f); + } else { + ret = ClassLoader.getSystemResourceAsStream(fileName); + } + } + + return ret; + } + + private void close(InputStream str, String filename) { + if (str != null) { + try { + str.close(); + } catch (IOException excp) { + LOG.error("Error while closing file: [{}]", filename, excp); + } + } + } +} diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumer.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcher.java similarity index 75% rename from audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumer.java rename to audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcher.java index 028da41f51..26c3bcb75d 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumer.java +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcher.java @@ -17,15 +17,15 @@ * under the License. */ -package org.apache.ranger.audit.consumer.kafka; +package org.apache.ranger.audit.dispatcher.kafka; import org.apache.kafka.clients.consumer.KafkaConsumer; /** - * Interface for Ranger Kafka consumers that consume audit events from Kafka + * Interface for Ranger Kafka dispatchers that consume audit events from Kafka * and forward them to various destinations like Solr, HDFS, etc. */ -public interface AuditConsumer extends Runnable { +public interface AuditDispatcher extends Runnable { /** * Start consuming messages from Kafka topic and process them. * This method should implement the main consumption loop. @@ -35,20 +35,20 @@ public interface AuditConsumer extends Runnable { void run(); /** - * Shutdown the consumer gracefully. + * Shutdown the dispatcher gracefully. * This method should clean up resources and close connections. */ void shutdown(); /** - * Get the underlying Kafka consumer instance. + * Get the underlying Kafka dispatcher instance. * - * @return KafkaConsumer instance used by this consumer + * @return KafkaConsumer instance used by this dispatcher */ - KafkaConsumer getConsumer(); + KafkaConsumer getDispatcher(); /** - * Get the topic name this consumer is subscribed to. + * Get the topic name this dispatcher is subscribed to. * * @return Kafka topic name */ diff --git a/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherBase.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherBase.java new file mode 100644 index 0000000000..0ef77cd049 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherBase.java @@ -0,0 +1,106 @@ +/* + * 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.ranger.audit.dispatcher.kafka; + +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.ranger.audit.provider.MiscUtil; +import org.apache.ranger.audit.server.AuditServerConstants; +import org.apache.ranger.audit.utils.AuditMessageQueueUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +public abstract class AuditDispatcherBase implements AuditDispatcher { + private static final Logger LOG = LoggerFactory.getLogger(AuditDispatcherBase.class); + + public final Properties dispatcherProps = new Properties(); + public final KafkaConsumer dispatcher; + public final String topicName; + public final String dispatcherGroupId; + + public AuditDispatcherBase(Properties props, String propPrefix, String dispatcherGroupId) throws Exception { + AuditMessageQueueUtils auditMessageQueueUtils = new AuditMessageQueueUtils(); + + this.dispatcherGroupId = getDispatcherGroupId(props, propPrefix, dispatcherGroupId); + + dispatcherProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_BOOTSTRAP_SERVERS)); + dispatcherProps.put(ConsumerConfig.GROUP_ID_CONFIG, this.dispatcherGroupId); + dispatcherProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Class.forName("org.apache.kafka.common.serialization.StringDeserializer")); + dispatcherProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Class.forName("org.apache.kafka.common.serialization.StringDeserializer")); + + String securityProtocol = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SECURITY_PROTOCOL, AuditServerConstants.DEFAULT_SECURITY_PROTOCOL); + dispatcherProps.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, securityProtocol); + + dispatcherProps.put(AuditServerConstants.PROP_SASL_MECHANISM, MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SASL_MECHANISM, AuditServerConstants.DEFAULT_SASL_MECHANISM)); + dispatcherProps.put(AuditServerConstants.PROP_SASL_KERBEROS_SERVICE_NAME, AuditServerConstants.DEFAULT_SERVICE_NAME); + + if (securityProtocol.toUpperCase().contains(AuditServerConstants.PROP_SECURITY_PROTOCOL_VALUE)) { + dispatcherProps.put(AuditServerConstants.PROP_SASL_JAAS_CONFIG, auditMessageQueueUtils.getJAASConfig(props, propPrefix)); + } + + dispatcherProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_MAX_POLL_RECORDS, AuditServerConstants.DEFAULT_MAX_POLL_RECORDS)); + dispatcherProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + // Configure re-balancing parameters for subscribe mode + // These ensure stable dispatcher group behavior during horizontal scaling + int sessionTimeoutMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_SESSION_TIMEOUT_MS, AuditServerConstants.DEFAULT_SESSION_TIMEOUT_MS); + int maxPollIntervalMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_MAX_POLL_INTERVAL_MS, AuditServerConstants.DEFAULT_MAX_POLL_INTERVAL_MS); + int heartbeatIntervalMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_HEARTBEAT_INTERVAL_MS, AuditServerConstants.DEFAULT_HEARTBEAT_INTERVAL_MS); + + dispatcherProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, sessionTimeoutMs); + dispatcherProps.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, maxPollIntervalMs); + dispatcherProps.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, heartbeatIntervalMs); + + // Configure partition assignment strategy + String partitionAssignmentStrategy = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_PARTITION_ASSIGNMENT_STRATEGY, AuditServerConstants.DEFAULT_PARTITION_ASSIGNMENT_STRATEGY); + dispatcherProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, partitionAssignmentStrategy); + + LOG.info("Dispatcher '{}' configured for subscription-based partition assignment with re-balancing support", this.dispatcherGroupId); + LOG.info("Re-balancing config - session.timeout.ms: {}, max.poll.interval.ms: {}, heartbeat.interval.ms: {}", sessionTimeoutMs, maxPollIntervalMs, heartbeatIntervalMs); + LOG.info("Partition assignment strategy: {}", partitionAssignmentStrategy); + + dispatcher = new KafkaConsumer<>(dispatcherProps); + topicName = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_TOPIC_NAME, AuditServerConstants.DEFAULT_TOPIC); + } + + @Override + public KafkaConsumer getDispatcher() { + return dispatcher; + } + + @Override + public String getTopicName() { + return topicName; + } + + public String getDispatcherGroupId() { + return dispatcherGroupId; + } + + private String getDispatcherGroupId(Properties props, String propPrefix, String defaultDispatcherGroupId) { + String configuredGroupId = MiscUtil.getStringProperty(props, propPrefix + ".dispatcher.group.id"); + if (configuredGroupId != null && !configuredGroupId.trim().isEmpty()) { + return configuredGroupId.trim(); + } + return defaultDispatcherGroupId; + } +} diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerFactory.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherFactory.java similarity index 64% rename from audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerFactory.java rename to audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherFactory.java index e986756127..d330da0ff8 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerFactory.java +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherFactory.java @@ -17,22 +17,22 @@ * under the License. */ -package org.apache.ranger.audit.consumer.kafka; +package org.apache.ranger.audit.dispatcher.kafka; import java.util.Properties; /** - * Factory interface for creating audit consumer instances. + * Factory interface for creating audit dispatcher instances. */ @FunctionalInterface -public interface AuditConsumerFactory { +public interface AuditDispatcherFactory { /** - * Create a consumer instance with the given configuration. + * Create a dispatcher instance with the given configuration. * * @param props Configuration properties - * @param propPrefix Property prefix for consumer configuration - * @return Initialized consumer instance ready to run - * @throws Exception if consumer creation or initialization fails + * @param propPrefix Property prefix for dispatcher configuration + * @return Initialized dispatcher instance ready to run + * @throws Exception if dispatcher creation or initialization fails */ - AuditConsumer createConsumer(Properties props, String propPrefix) throws Exception; + AuditDispatcher createDispatcher(Properties props, String propPrefix) throws Exception; } diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerRebalanceListener.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherRebalanceListener.java similarity index 81% rename from audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerRebalanceListener.java rename to audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherRebalanceListener.java index 1b002d629c..b412541e85 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerRebalanceListener.java +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherRebalanceListener.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.ranger.audit.consumer.kafka; +package org.apache.ranger.audit.dispatcher.kafka; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -33,37 +33,37 @@ import java.util.concurrent.atomic.AtomicLong; /** - * Reusable ConsumerRebalanceListener for Kafka consumer group re-balancing. + * Reusable ConsumerRebalanceListener for Kafka dispatcher group re-balancing. * * This listener handles graceful partition re-balancing by: * - Committing pending offsets before partitions are revoked * - Updating partition assignments when partitions are assigned * - Logging re-balancing events with customizable log prefixes * - * Used by both AuditSolrConsumer, AuditHDFSConsumer and any new consumers added to the audit serve, ensuring + * Used by both AuditSolrDispatcher, AuditHDFSDispatcher and any new dispatchers added to the audit serve, ensuring * no message duplication during scaling operations. */ -public class AuditConsumerRebalanceListener implements ConsumerRebalanceListener { - private static final Logger LOG = LoggerFactory.getLogger(AuditConsumerRebalanceListener.class); +public class AuditDispatcherRebalanceListener implements ConsumerRebalanceListener { + private static final Logger LOG = LoggerFactory.getLogger(AuditDispatcherRebalanceListener.class); private final String workerId; private final String destinationType; private final String topicName; private final String offsetCommitStrategy; - private final String consumerGroupId; - private final KafkaConsumer workerConsumer; + private final String dispatcherGroupId; + private final KafkaConsumer workerDispatcher; private final Map pendingOffsets; private final AtomicInteger messagesProcessedSinceLastCommit; private final AtomicLong lastCommitTime; private final List assignedPartitions; - public AuditConsumerRebalanceListener( + public AuditDispatcherRebalanceListener( String workerId, String destinationType, String topicName, String offsetCommitStrategy, - String consumerGroupId, - KafkaConsumer workerConsumer, + String dispatcherGroupId, + KafkaConsumer workerDispatcher, Map pendingOffsets, AtomicInteger messagesProcessedSinceLastCommit, AtomicLong lastCommitTime, @@ -72,8 +72,8 @@ public AuditConsumerRebalanceListener( this.destinationType = destinationType; this.topicName = topicName; this.offsetCommitStrategy = offsetCommitStrategy; - this.consumerGroupId = consumerGroupId; - this.workerConsumer = workerConsumer; + this.dispatcherGroupId = dispatcherGroupId; + this.workerDispatcher = workerDispatcher; this.pendingOffsets = pendingOffsets; this.messagesProcessedSinceLastCommit = messagesProcessedSinceLastCommit; this.lastCommitTime = lastCommitTime; @@ -87,7 +87,7 @@ public void onPartitionsRevoked(Collection partitions) { // Commit pending offsets before partitions are revoked if (!pendingOffsets.isEmpty()) { try { - workerConsumer.commitSync(pendingOffsets); + workerDispatcher.commitSync(pendingOffsets); LOG.info("[{}-REBALANCE] Worker '{}': Successfully committed {} pending offsets before rebalance", destinationType, workerId, pendingOffsets.size()); pendingOffsets.clear(); @@ -113,7 +113,7 @@ public void onPartitionsAssigned(Collection partitions) { } // Log assignment details - LOG.info("[{}-CONSUMER-ASSIGNED] Worker '{}' | Topic: '{}' | Partitions: {} | Offset-Strategy: {} | Consumer-Group: {}", - destinationType, workerId, topicName, assignedPartitions, offsetCommitStrategy, consumerGroupId); + LOG.info("[{}-DISPATCHER-ASSIGNED] Worker '{}' | Topic: '{}' | Partitions: {} | Offset-Strategy: {} | Dispatcher-Group: {}", + destinationType, workerId, topicName, assignedPartitions, offsetCommitStrategy, dispatcherGroupId); } } diff --git a/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherTracker.java b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherTracker.java new file mode 100644 index 0000000000..dd317e8773 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditDispatcherTracker.java @@ -0,0 +1,73 @@ +/* + * 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.ranger.audit.dispatcher.kafka; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** +* Tracks active audit dispatcher instances. +*/ +public class AuditDispatcherTracker { + private static final Logger LOG = LoggerFactory.getLogger(AuditDispatcherTracker.class); + private static final AuditDispatcherTracker tracker = new AuditDispatcherTracker(); + private final Map activeDispatchers = new ConcurrentHashMap<>(); + + private AuditDispatcherTracker() { + LOG.debug("AuditDispatcherTracker instance created by classloader: {}", this.getClass().getClassLoader()); + } + + public static AuditDispatcherTracker getInstance() { + return tracker; + } + + /** + * Register an active dispatcher instance for a specific destination type for healthcheck. + * + * @param destinationType The destination type identifier (e.g., "solr", "hdfs", "elasticsearch") + * @param dispatcher The dispatcher instance + */ + public void addActiveDispatcher(String destinationType, AuditDispatcher dispatcher) { + if (dispatcher == null) { + LOG.warn("Attempted to register null dispatcher for destination type: {}", destinationType); + return; + } + activeDispatchers.put(destinationType, dispatcher); + LOG.info("Registered active dispatcher for destination type: {}", destinationType); + } + + public Collection getActiveDispatchers() { + return activeDispatchers.values(); + } + + /** + * Clear all active dispatcher references. + * Called during shutdown after dispatchers have been stopped. + */ + + public void clearActiveDispatchers() { + LOG.debug("Clearing {} active dispatcher references", activeDispatchers.size()); + activeDispatchers.clear(); + } +} diff --git a/audit-server/audit-dispatcher/dispatcher-common/src/main/resources/conf/ranger-audit-dispatcher-site.xml b/audit-server/audit-dispatcher/dispatcher-common/src/main/resources/conf/ranger-audit-dispatcher-site.xml new file mode 100644 index 0000000000..0b7c3791a2 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-common/src/main/resources/conf/ranger-audit-dispatcher-site.xml @@ -0,0 +1,119 @@ + + + + + + + ranger.audit.dispatcher.type + solr + + The type of dispatcher to run. + Supported values: solr, hdfs + + + + + ranger.audit.dispatcher.war.file + ranger-audit-dispatcher.war + + + + ranger.audit.dispatcher.launcher.class + org.apache.ranger.audit.dispatcher.AuditDispatcherLauncher + + + + ranger.audit.dispatcher.main.class + org.apache.ranger.audit.dispatcher.AuditDispatcherApplication + + + + + ranger.audit.dispatcher.host + 0.0.0.0 + Service hostname + + + + ranger.audit.dispatcher.http.port + 7090 + HTTP port for the dispatcher service + + + + ranger.audit.dispatcher.contextName + / + + + + + + ranger.audit.dispatcher.kafka.bootstrap.servers + ranger-kafka:9092 + Kafka broker hosts + + + + ranger.audit.dispatcher.kafka.topic.name + ranger_audits + + + + ranger.audit.dispatcher.kafka.group.id + ranger_audit_dispatcher + + + + ranger.audit.dispatcher.kafka.security.protocol + SASL_PLAINTEXT + + + + ranger.audit.dispatcher.kafka.sasl.mechanism + GSSAPI + + + + ranger.audit.dispatcher.kafka.sasl.kerberos.service.name + kafka + + + + + ranger.audit.dispatcher.hadoop.security.authentication + KERBEROS + + + + ranger.audit.dispatcher.kerberos.principal + rangerauditserver/_HOST@EXAMPLE.COM + Principal for Kafka dispatcher and destination authentication + + + + ranger.audit.dispatcher.kerberos.keytab + /etc/keytabs/rangerauditserver.keytab + + + + ranger.audit.dispatcher.bind.address + 0.0.0.0 + Hostname for resolving _HOST in Kerberos principal + + + diff --git a/audit-server/consumer-hdfs/pom.xml b/audit-server/audit-dispatcher/dispatcher-hdfs/pom.xml similarity index 72% rename from audit-server/consumer-hdfs/pom.xml rename to audit-server/audit-dispatcher/dispatcher-hdfs/pom.xml index 5424f8168c..5a6d53b0ee 100644 --- a/audit-server/consumer-hdfs/pom.xml +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/pom.xml @@ -1,21 +1,20 @@ + 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. +--> 4.0.0 @@ -23,13 +22,13 @@ org.apache.ranger ranger 3.0.0-SNAPSHOT - ../.. + ../../.. - audit-consumer-hdfs - war - Ranger Audit Consumer HDFS - Kafka consumer service for writing audits to HDFS/S3/Azure + audit-dispatcher-hdfs + jar + Ranger Audit Dispatcher HDFS + Kafka dispatcher service for writing audits to HDFS/S3/Azure UTF-8 @@ -37,61 +36,6 @@ - - - ch.qos.logback - logback-classic - ${logback.version} - - - com.fasterxml.jackson.core - jackson-databind - ${fasterxml.jackson.version} - - - - com.google.guava - guava - ${google.guava.version} - - - - com.sun.jersey - jersey-bundle - ${jersey-bundle.version} - - - com.sun.jersey.contribs - jersey-spring - ${jersey-spring.version} - - - com.sun.jersey - jersey-server - - - org.springframework - * - - - - - commons-io - commons-io - ${commons.io.version} - - - javax.servlet - javax.servlet-api - ${javax.servlet.version} - - - - org.apache.commons - commons-lang3 - ${commons.lang3.version} - - org.apache.hadoop hadoop-aws @@ -189,23 +133,6 @@ - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - log4j - * - - - org.slf4j - * - - - - org.apache.orc orc-core @@ -228,10 +155,9 @@ - org.apache.ranger - ranger-audit-common + ranger-audit-dest-hdfs ${project.version} @@ -241,10 +167,118 @@ + + ch.qos.logback + logback-classic + ${logback.version} + provided + + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.jackson.version} + provided + + + + com.google.guava + guava + ${google.guava.version} + provided + + + + com.sun.jersey + jersey-bundle + ${jersey-bundle.version} + provided + + + javax.ws.rs + jsr311-api + + + + + com.sun.jersey + jersey-core + ${jersey-bundle.version} + provided + + + com.sun.jersey + jersey-server + ${jersey-bundle.version} + provided + + + com.sun.jersey + jersey-servlet + ${jersey-bundle.version} + provided + + + com.sun.jersey.contribs + jersey-spring + ${jersey-spring.version} + provided + + + com.github.pjfanning + jersey-json + + + com.sun.jersey + jersey-server + + + org.springframework + * + + + + + commons-io + commons-io + ${commons.io.version} + provided + + + javax.servlet + javax.servlet-api + ${javax.servlet.version} + provided + + + + org.apache.commons + commons-lang3 + ${commons.lang3.version} + provided + + + + org.apache.kafka + kafka-clients + ${kafka.version} + provided + + + log4j + * + + + org.slf4j + * + + + + org.apache.ranger ranger-audit-core ${project.version} + provided org.apache.hadoop @@ -252,10 +286,12 @@ + org.apache.ranger - ranger-audit-dest-hdfs + ranger-audit-dispatcher-common ${project.version} + provided org.apache.hadoop @@ -263,10 +299,18 @@ + + + org.apache.ranger + ranger-audit-server-common + ${project.version} + provided + org.apache.ranger ranger-plugins-common ${project.version} + provided javax.servlet @@ -282,53 +326,63 @@ org.apache.tomcat tomcat-annotations-api ${tomcat.embed.version} + provided org.apache.tomcat.embed tomcat-embed-core ${tomcat.embed.version} + provided org.apache.tomcat.embed tomcat-embed-el ${tomcat.embed.version} + provided org.apache.tomcat.embed tomcat-embed-jasper ${tomcat.embed.version} + provided org.slf4j log4j-over-slf4j ${slf4j.version} + provided org.slf4j slf4j-api ${slf4j.version} + provided org.springframework spring-beans ${springframework.version} + provided org.springframework spring-context ${springframework.version} + provided org.springframework spring-web ${springframework.version} + provided + audit-dispatcher-hdfs-${project.version} true @@ -338,7 +392,29 @@ org.apache.maven.plugins - maven-war-plugin + maven-pmd-plugin + + + ${project.parent.basedir}/dev-support/ranger-pmd-ruleset.xml + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + + copy-dependencies + + package + + ${project.build.directory}/lib + runtime + + + diff --git a/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/HdfsDispatcherManager.java b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/HdfsDispatcherManager.java new file mode 100644 index 0000000000..941dc86db6 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/HdfsDispatcherManager.java @@ -0,0 +1,195 @@ +/* + * 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.ranger.audit.dispatcher; + +import org.apache.ranger.audit.dispatcher.kafka.AuditDispatcher; +import org.apache.ranger.audit.dispatcher.kafka.AuditDispatcherTracker; +import org.apache.ranger.audit.provider.MiscUtil; +import org.apache.ranger.audit.server.AuditServerConstants; +import org.apache.ranger.audit.utils.AuditServerLogFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +/** + * Spring component that manages the lifecycle of HDFS dispatcher threads. + * Manager that handles the lifecycle of HDFS dispatcher threads. + * - Initializes the dispatcher tracker + * - Creates HDFS dispatcher instances + * - Starts dispatcher threads + * - Handles graceful shutdown + */ +public class HdfsDispatcherManager { + private static final Logger LOG = LoggerFactory.getLogger(HdfsDispatcherManager.class); + private static final String CONFIG_DISPATCHER_TYPE = "ranger.audit.dispatcher.type"; + + private final AuditDispatcherTracker tracker = AuditDispatcherTracker.getInstance(); + private AuditDispatcher dispatcher; + private Thread dispatcherThread; + + public void init(Properties props) { + LOG.info("==> HdfsDispatcherManager.init()"); + + String dispatcherType = System.getProperty(CONFIG_DISPATCHER_TYPE); + if (dispatcherType != null && !dispatcherType.equalsIgnoreCase("hdfs")) { + LOG.info("Skipping HdfsDispatcherManager initialization since dispatcher type is {}", dispatcherType); + return; + } + + try { + if (props == null) { + LOG.error("Configuration properties are null"); + throw new RuntimeException("Failed to load configuration"); + } + + boolean isEnabled = MiscUtil.getBooleanProperty(props, "xasecure.audit.destination.hdfs", false); + if (!isEnabled) { + LOG.warn("HDFS destination is disabled (xasecure.audit.destination.hdfs=false). No dispatchers will be created."); + return; + } + + // Initialize and register HDFS Dispatcher + initializeDispatcher(props, AuditServerConstants.PROP_KAFKA_PROP_PREFIX); + + if (dispatcher == null) { + LOG.warn("No dispatcher was created! Verify that xasecure.audit.destination.hdfs=true and classes are configured correctly."); + } else { + LOG.info("Created HDFS dispatcher"); + + // Register shutdown hook + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + LOG.info("JVM shutdown detected, stopping HdfsDispatcherManager..."); + shutdown(); + }, "HdfsDispatcherManager-ShutdownHook")); + + // Start dispatcher thread + startDispatcher(); + } + } catch (Exception e) { + LOG.error("Failed to initialize HdfsDispatcherManager", e); + throw new RuntimeException("Failed to initialize HdfsDispatcherManager", e); + } + LOG.info("<== HdfsDispatcherManager.init()"); + } + + private void initializeDispatcher(Properties props, String propPrefix) { + LOG.info("==> HdfsDispatcherManager.initializeDispatcher()"); + + String clsStr = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_CLASSES, "org.apache.ranger.audit.dispatcher.kafka.AuditHDFSDispatcher"); + String[] hdfsDispatcherClasses = clsStr.split(","); + + LOG.info("Initializing {} dispatcher class(es)", hdfsDispatcherClasses.length); + + String hdfsDispatcherClassName = clsStr.split(",")[0].trim(); + if (hdfsDispatcherClassName.isEmpty()) { + LOG.error("Dispatcher class name is empty"); + return; + } + + try { + Class dispatcherClass = Class.forName(hdfsDispatcherClassName); + dispatcher = (AuditDispatcher) dispatcherClass + .getConstructor(Properties.class, String.class) + .newInstance(props, propPrefix); + tracker.addActiveDispatcher("hdfs", dispatcher); + LOG.info("Successfully initialized dispatcher class: {}", dispatcherClass.getName()); + } catch (ClassNotFoundException e) { + LOG.error("Dispatcher class not found: {}. Ensure the class is on the classpath.", hdfsDispatcherClassName, e); + } catch (Exception e) { + LOG.error("Error initializing dispatcher class: {}", hdfsDispatcherClassName, e); + } + + LOG.info("<== HdfsDispatcherManager.initializeDispatcher()"); + } + + /** + * Start dispatcher thread + */ + private void startDispatcher() { + LOG.info("==> HdfsDispatcherManager.startDispatcher()"); + + logDispatcherStartup(); + try { + String dispatcherName = dispatcher.getClass().getSimpleName(); + Thread dispatcherThread = new Thread(dispatcher, dispatcherName); + dispatcherThread.setDaemon(true); + dispatcherThread.start(); + LOG.info("Started {} thread [Thread-ID: {}, Thread-Name: '{}']", dispatcherName, dispatcherThread.getId(), dispatcherThread.getName()); + } catch (Exception e) { + LOG.error("Error starting dispatcher: {}", dispatcher.getClass().getSimpleName(), e); + } + + LOG.info("<== HdfsDispatcherManager.startDispatcher()"); + } + + private void logDispatcherStartup() { + LOG.info("################## HDFS DISPATCHER SERVICE STARTUP ######################"); + + if (dispatcher == null) { + LOG.warn("WARNING: No HDFS dispatchers are enabled!"); + LOG.warn("Verify: xasecure.audit.destination.hdfs=true in configuration"); + } else { + AuditServerLogFormatter.LogBuilder builder = AuditServerLogFormatter.builder("HDFS Dispatcher Status"); + String dispatcherType = dispatcher.getClass().getSimpleName(); + builder.add(dispatcherType, "ENABLED"); + builder.add("Topic", dispatcher.getTopicName()); + builder.logInfo(LOG); + LOG.info("Starting HDFS dispatcher thread..."); + } + + LOG.info("########################################################################"); + } + + public void shutdown() { + LOG.info("==> HdfsDispatcherManager.shutdown()"); + + // Shutdown dispatcher + if (dispatcher != null) { + try { + LOG.info("Shutting down dispatcher: {}", dispatcher.getClass().getSimpleName()); + dispatcher.shutdown(); + LOG.info("Dispatcher shutdown completed: {}", dispatcher.getClass().getSimpleName()); + } catch (Exception e) { + LOG.error("Error shutting down dispatcher: {}", dispatcher.getClass().getSimpleName(), e); + } + } + + // Wait for thread to terminate + if (dispatcherThread != null && dispatcherThread.isAlive()) { + try { + LOG.info("Waiting for thread to terminate: {}", dispatcherThread.getName()); + dispatcherThread.join(10000); // Wait up to 10 seconds + if (dispatcherThread.isAlive()) { + LOG.warn("Thread did not terminate within 10 seconds: {}", dispatcherThread.getName()); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for thread to terminate: {}", dispatcherThread.getName(), e); + Thread.currentThread().interrupt(); + } + } + + dispatcher = null; + dispatcherThread = null; + tracker.clearActiveDispatchers(); + + LOG.info("<== HdfsDispatcherManager.shutdown() - HDFS dispatcher stopped"); + } +} diff --git a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditHDFSConsumer.java b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditHDFSDispatcher.java similarity index 55% rename from audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditHDFSConsumer.java rename to audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditHDFSDispatcher.java index 926bcdfb9e..680fffc216 100644 --- a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditHDFSConsumer.java +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditHDFSDispatcher.java @@ -16,8 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.ranger.audit.consumer.kafka; +package org.apache.ranger.audit.dispatcher.kafka; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.SecureClientLogin; import org.apache.hadoop.security.UserGroupInformation; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -28,7 +29,6 @@ import org.apache.ranger.audit.provider.AuditProviderFactory; import org.apache.ranger.audit.provider.MiscUtil; import org.apache.ranger.audit.server.AuditServerConstants; -import org.apache.ranger.audit.server.HdfsConsumerConfig; import org.apache.ranger.audit.utils.AuditServerLogFormatter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,34 +48,26 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -public class AuditHDFSConsumer extends AuditConsumerBase { - private static final Logger LOG = LoggerFactory.getLogger(AuditHDFSConsumer.class); +public class AuditHDFSDispatcher extends AuditDispatcherBase { + private static final Logger LOG = LoggerFactory.getLogger(AuditHDFSDispatcher.class); - private static final String RANGER_AUDIT_HDFS_CONSUMER_GROUP = AuditServerConstants.DEFAULT_RANGER_AUDIT_HDFS_CONSUMER_GROUP; + private static final String RANGER_AUDIT_HDFS_DISPATCHER_GROUP = AuditServerConstants.DEFAULT_RANGER_AUDIT_HDFS_DISPATCHER_GROUP; + private static final String CONFIG_CORE_SITE = "core-site.xml"; + private static final String CONFIG_HDFS_SITE = "hdfs-site.xml"; - // Register AuditHDFSConsumer factory in the audit consumer registry - static { - try { - AuditConsumerRegistry.getInstance().registerFactory(AuditServerConstants.PROP_HDFS_DEST_PREFIX, AuditHDFSConsumer::new); - LOG.info("Registered HDFS consumer with AuditConsumerRegistry"); - } catch (Exception e) { - LOG.error("Failed to register HDFS consumer factory", e); - } - } + private final AtomicBoolean running = new AtomicBoolean(false); + private final Map dispatcherWorkers = new ConcurrentHashMap<>(); + private final AuditRouterHDFS auditRouterHDFS; - private final AtomicBoolean running = new AtomicBoolean(false); - private final Map consumerWorkers = new ConcurrentHashMap<>(); - private final AuditRouterHDFS auditRouterHDFS; - - private ExecutorService consumerThreadPool; - private int consumerThreadCount = 1; + private ExecutorService dispatcherThreadPool; + private int dispatcherThreadCount = 1; // Offset management configuration (batch or manual only supported) private String offsetCommitStrategy = AuditServerConstants.DEFAULT_OFFSET_COMMIT_STRATEGY; private long offsetCommitInterval = AuditServerConstants.DEFAULT_OFFSET_COMMIT_INTERVAL_MS; - public AuditHDFSConsumer(Properties props, String propPrefix) throws Exception { - super(props, propPrefix, RANGER_AUDIT_HDFS_CONSUMER_GROUP); + public AuditHDFSDispatcher(Properties props, String propPrefix) throws Exception { + super(props, propPrefix, RANGER_AUDIT_HDFS_DISPATCHER_GROUP); auditRouterHDFS = new AuditRouterHDFS(); @@ -85,21 +77,21 @@ public AuditHDFSConsumer(Properties props, String propPrefix) throws Exception { @Override public void run() { try { - LOG.info("Starting AuditHDFSConsumer with appId-based thread"); + LOG.info("Starting AuditHDFSDispatcher with appId-based thread"); startMultithreadedConsumption(); - // Keep main thread alive while consumer threads are running + // Keep main thread alive while dispatcher threads are running while (running.get()) { try { Thread.sleep(1000); } catch (InterruptedException e) { - LOG.info("HDFS consumer main thread interrupted"); + LOG.info("HDFS dispatcher main thread interrupted"); Thread.currentThread().interrupt(); break; } } } catch (Throwable e) { - LOG.error("Error in AuditHDFSConsumer", e); + LOG.error("Error in AuditHDFSDispatcher", e); } finally { shutdown(); } @@ -107,25 +99,25 @@ public void run() { @Override public void shutdown() { - LOG.info("==> AuditHDFSConsumer.shutdown()"); + LOG.info("==> AuditHDFSDispatcher.shutdown()"); - // Stop consumer threads + // Stop dispatcher threads running.set(false); - // Shutdown consumer workers - if (consumerThreadPool != null) { - consumerThreadPool.shutdownNow(); + // Shutdown dispatcher workers + if (dispatcherThreadPool != null) { + dispatcherThreadPool.shutdownNow(); try { - if (!consumerThreadPool.awaitTermination(30, java.util.concurrent.TimeUnit.SECONDS)) { - LOG.warn("HDFS consumer thread pool did not terminate within 30 seconds"); + if (!dispatcherThreadPool.awaitTermination(30, java.util.concurrent.TimeUnit.SECONDS)) { + LOG.warn("HDFS dispatcher thread pool did not terminate within 30 seconds"); } } catch (InterruptedException e) { - LOG.warn("Interrupted while waiting for HDFS consumer thread pool to terminate", e); + LOG.warn("Interrupted while waiting for HDFS dispatcher thread pool to terminate", e); Thread.currentThread().interrupt(); } } - consumerWorkers.clear(); + dispatcherWorkers.clear(); // Shutdown destination handler if (auditRouterHDFS != null) { @@ -136,34 +128,34 @@ public void shutdown() { } } - // Close main Kafka consumer - if (consumer != null) { + // Close main Kafka dispatcher + if (dispatcher != null) { try { - consumer.close(); + dispatcher.close(); } catch (Exception e) { - LOG.error("Error closing main Kafka consumer", e); + LOG.error("Error closing main Kafka dispatcher", e); } } - LOG.info("<== AuditHDFSConsumer.shutdown() complete"); + LOG.info("<== AuditHDFSDispatcher.shutdown() complete"); } private void init(Properties props, String propPrefix) throws Exception { - LOG.info("==> AuditHDFSConsumer.init(): AuditHDFSConsumer initializing with appId-based threading and offset management"); + LOG.info("==> AuditHDFSDispatcher.init(): AuditHDFSDispatcher initializing with appId-based threading and offset management"); // Initialize Ranger UGI for HDFS operations if Kerberos is enabled initializeRangerUGI(props, propPrefix); - // Add Hadoop configuration properties from HdfsConsumerConfig to props + // Add Hadoop configuration properties from HdfsDispatcherConfig to props addHadoopConfigToProps(props); - // Initialize consumer configuration - initConsumerConfig(props, propPrefix); + // Initialize dispatcher configuration + initDispatcherConfig(props, propPrefix); // Initialize destination handler for message processing auditRouterHDFS.init(props, AuditProviderFactory.AUDIT_DEST_BASE + "." + AuditServerConstants.PROP_HDFS_DEST_PREFIX); - LOG.info("<== AuditHDFSConsumer.init(): AuditHDFSConsumer initialized successfully"); + LOG.info("<== AuditHDFSDispatcher.init(): AuditHDFSDispatcher initialized successfully"); } private void processMessage(String message, String partitionKey) throws Exception { @@ -171,11 +163,11 @@ private void processMessage(String message, String partitionKey) throws Exceptio } private void initializeRangerUGI(Properties props, String propPrefix) throws Exception { - LOG.info("==> AuditHDFSConsumer.initializeRangerUGI()"); + LOG.info("==> AuditHDFSDispatcher.initializeRangerUGI()"); try { - HdfsConsumerConfig auditConfig = HdfsConsumerConfig.getInstance(); - String authType = auditConfig.get(AuditServerConstants.PROP_HADOOP_AUTHENTICATION_TYPE, "simple"); + Configuration coreSite = getCoreSiteConfiguration(); + String authType = coreSite.get(AuditServerConstants.PROP_HADOOP_AUTHENTICATION_TYPE, "simple"); if (!AuditServerConstants.PROP_HADOOP_AUTH_TYPE_KERBEROS.equalsIgnoreCase(authType)) { LOG.info("Hadoop authentication is not Kerberos ({}), skipping Ranger UGI initialization", authType); @@ -203,19 +195,18 @@ private void initializeRangerUGI(Properties props, String propPrefix) throws Exc } // Set Hadoop security configuration from core-site.xml - org.apache.hadoop.conf.Configuration coreSite = auditConfig.getCoreSiteConfiguration(); UserGroupInformation.setConfiguration(coreSite); - LOG.info("Initializing Ranger UGI for HDFS writes: principal={}, keytab={}", principal, keytab); + LOG.info("Initializing Ranger AuditServer UGI for HDFS writes: principal={}, keytab={}", principal, keytab); - UserGroupInformation rangerUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab); + UserGroupInformation rangerAuditServerUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab); - MiscUtil.setUGILoginUser(rangerUGI, null); + MiscUtil.setUGILoginUser(rangerAuditServerUGI, null); - LOG.info("<== AuditHDFSConsumer.initializeRangerUGI(): Ranger UGI initialized successfully: user={}, auth={}, hasKerberos={}", - rangerUGI.getUserName(), rangerUGI.getAuthenticationMethod(), rangerUGI.hasKerberosCredentials()); + LOG.info("<== AuditHDFSDispatcher.initializeRangerUGI(): Ranger UGI initialized successfully: user={}, auth={}, hasKerberos={}", + rangerAuditServerUGI.getUserName(), rangerAuditServerUGI.getAuthenticationMethod(), rangerAuditServerUGI.hasKerberosCredentials()); } catch (IOException e) { - LOG.error("Failed to initialize Ranger UGI for HDFS writes", e); + LOG.error("Failed to initialize Ranger AuditServer UGI for HDFS writes", e); throw e; } } @@ -224,97 +215,161 @@ private void initializeRangerUGI(Properties props, String propPrefix) throws Exc * Add Hadoop configuration properties from core-site.xml and hdfs-site.xml to props. */ private void addHadoopConfigToProps(Properties props) { - LOG.info("==> AuditHDFSConsumer.addHadoopConfigToProps()"); + LOG.info("==> AuditHDFSDispatcher.addHadoopConfigToProps()"); try { - HdfsConsumerConfig hdfsConfig = HdfsConsumerConfig.getInstance(); - String configPrefix = "xasecure.audit.destination.hdfs.config."; - - Properties hadoopProps = hdfsConfig.getHadoopPropertiesWithPrefix(configPrefix); + String configPrefix = "xasecure.audit.destination.hdfs.config."; + Properties hadoopProps = getHadoopPropertiesWithPrefix(configPrefix); props.putAll(hadoopProps); - LOG.info("<== AuditHDFSConsumer.addHadoopConfigToProps(): Added {} Hadoop configuration properties from HdfsConsumerConfig", hadoopProps.size()); + LOG.info("<== AuditHDFSDispatcher.addHadoopConfigToProps(): Added {} Hadoop configuration properties", hadoopProps.size()); } catch (Exception e) { LOG.error("Failed to add Hadoop configuration properties to props", e); } } - private void initConsumerConfig(Properties props, String propPrefix) { - LOG.info("==> AuditHDFSConsumer.initConsumerConfig()"); + private void initDispatcherConfig(Properties props, String propPrefix) { + LOG.info("==> AuditHDFSDispatcher.initDispatcherConfig()"); - // Get consumer thread count - this.consumerThreadCount = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_THREAD_COUNT, 1); - LOG.info("HDFS consumer thread count: {}", consumerThreadCount); + // Get dispatcher thread count + this.dispatcherThreadCount = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_THREAD_COUNT, 1); + LOG.info("HDFS dispatcher thread count: {}", dispatcherThreadCount); // Initialize offset management configuration initializeOffsetManagement(props, propPrefix); - LOG.info("<== AuditHDFSConsumer.initConsumerConfig()"); + LOG.info("<== AuditHDFSDispatcher.initDispatcherConfig()"); + } + + /** + * Get Hadoop configuration properties (from core-site.xml and hdfs-site.xml) with a specific prefix. + * This is need for the HdfsAuditDestination and its parent classes for routing the audits to hdfs location. + * @param prefix The prefix to add to each property name ("xasecure.audit.destination.hdfs.config.") + * @return Properties from core-site.xml and hdfs-site.xml with the specified prefix + */ + private Properties getHadoopPropertiesWithPrefix(String prefix) { + LOG.debug("==> AuditHDFSDispatcher.getHadoopPropertiesWithPrefix(prefix={})", prefix); + + Properties prefixedProps = new java.util.Properties(); + int propsAdded = 0; + + try { + // Load core-site.xml separately to get pure Hadoop security properties + Configuration coreSite = new Configuration(false); + coreSite.addResource(CONFIG_CORE_SITE); + + for (java.util.Map.Entry entry : coreSite) { + String propName = entry.getKey(); + String propValue = entry.getValue(); + + if (propValue != null && !propValue.trim().isEmpty()) { + prefixedProps.setProperty(prefix + propName, propValue); + LOG.trace("Added from core-site.xml: {} = {}", propName, propValue); + propsAdded++; + } + } + + // Load hdfs-site.xml separately to get pure HDFS client properties + Configuration hdfsSite = new Configuration(false); + hdfsSite.addResource(CONFIG_HDFS_SITE); + + for (java.util.Map.Entry entry : hdfsSite) { + String propName = entry.getKey(); + String propValue = entry.getValue(); + + if (propValue != null && !propValue.trim().isEmpty()) { + prefixedProps.setProperty(prefix + propName, propValue); + LOG.trace("Added from hdfs-site.xml: {} = {}", propName, propValue); + propsAdded++; + } + } + + LOG.debug("<== AuditHDFSDispatcher.getHadoopPropertiesWithPrefix(): Added {} Hadoop properties with prefix '{}'", propsAdded, prefix); + } catch (Exception e) { + LOG.error("Failed to load Hadoop properties from core-site.xml and hdfs-site.xml", e); + } + + return prefixedProps; + } + + /** + * Get core-site.xml Configuration for UGI initialization. + * @return Configuration loaded from core-site.xml + */ + private Configuration getCoreSiteConfiguration() { + LOG.debug("==> AuditHDFSDispatcher.getCoreSiteConfiguration()"); + + Configuration coreSite = new Configuration(false); + coreSite.addResource(CONFIG_CORE_SITE); + + LOG.debug("<== AuditHDFSDispatcher.getCoreSiteConfiguration(): authentication={}", coreSite.get("hadoop.security.authentication")); + + return coreSite; } private void initializeOffsetManagement(Properties props, String propPrefix) { - LOG.info("==> AuditHDFSConsumer.initializeOffsetManagement()"); + LOG.info("==> AuditHDFSDispatcher.initializeOffsetManagement()"); // Get offset commit strategy this.offsetCommitStrategy = MiscUtil.getStringProperty(props, - propPrefix + "." + AuditServerConstants.PROP_CONSUMER_OFFSET_COMMIT_STRATEGY, + propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_OFFSET_COMMIT_STRATEGY, AuditServerConstants.DEFAULT_OFFSET_COMMIT_STRATEGY); // Get offset commit interval (only used for manual strategy) this.offsetCommitInterval = MiscUtil.getLongProperty(props, - propPrefix + "." + AuditServerConstants.PROP_CONSUMER_OFFSET_COMMIT_INTERVAL, + propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_OFFSET_COMMIT_INTERVAL, AuditServerConstants.DEFAULT_OFFSET_COMMIT_INTERVAL_MS); - AuditServerLogFormatter.builder("HDFS Consumer Offset Management Configuration") + AuditServerLogFormatter.builder("HDFS Dispatcher Offset Management Configuration") .add("Commit Strategy", offsetCommitStrategy) .add("Commit Interval (ms)", offsetCommitInterval + " (used in manual mode only)") .logInfo(LOG); - LOG.info("<== AuditHDFSConsumer.initializeOffsetManagement()"); + LOG.info("<== AuditHDFSDispatcher.initializeOffsetManagement()"); } private void startMultithreadedConsumption() { - LOG.debug("==> AuditHDFSConsumer.startMultithreadedConsumption()"); + LOG.debug("==> AuditHDFSDispatcher.startMultithreadedConsumption()"); if (running.compareAndSet(false, true)) { - startConsumerWorkers(); + startDispatcherWorkers(); } - LOG.debug("<== AuditHDFSConsumer.startMultithreadedConsumption()"); + LOG.debug("<== AuditHDFSDispatcher.startMultithreadedConsumption()"); } - private void startConsumerWorkers() { - LOG.info("==> AuditHDFSConsumer.startConsumerWorkers(): Creating {} consumer workers for horizontal scaling", consumerThreadCount); + private void startDispatcherWorkers() { + LOG.info("==> AuditHDFSDispatcher.startDispatcherWorkers(): Creating {} dispatcher workers for horizontal scaling", dispatcherThreadCount); LOG.info("Each worker will subscribe to topic '{}' and process partitions assigned by Kafka", topicName); - // Create thread pool sized for consumer workers - consumerThreadPool = Executors.newFixedThreadPool(consumerThreadCount); - LOG.info("Created thread pool with {} threads for scalable HDFS consumption", consumerThreadCount); + // Create thread pool sized for dispatcher workers + dispatcherThreadPool = Executors.newFixedThreadPool(dispatcherThreadCount); + LOG.info("Created thread pool with {} threads for scalable HDFS consumption", dispatcherThreadCount); - // Create HDFS consumer workers - for (int i = 0; i < consumerThreadCount; i++) { + // Create HDFS dispatcher workers + for (int i = 0; i < dispatcherThreadCount; i++) { String workerId = "hdfs-worker-" + i; - ConsumerWorker worker = new ConsumerWorker(workerId, new ArrayList<>()); - consumerWorkers.put(workerId, worker); - consumerThreadPool.submit(worker); + DispatcherWorker worker = new DispatcherWorker(workerId, new ArrayList<>()); + dispatcherWorkers.put(workerId, worker); + dispatcherThreadPool.submit(worker); - LOG.info("Started HDFS consumer worker '{}' - will process ANY appId assigned by Kafka", workerId); + LOG.info("Started HDFS dispatcher worker '{}' - will process ANY appId assigned by Kafka", workerId); } - LOG.info("<== AuditHDFSConsumer.startConsumerWorkers(): All {} workers started in SUBSCRIBE mode", consumerThreadCount); + LOG.info("<== AuditHDFSDispatcher.startDispatcherWorkers(): All {} workers started in SUBSCRIBE mode", dispatcherThreadCount); } - private class ConsumerWorker implements Runnable { + private class DispatcherWorker implements Runnable { private final String workerId; private final List assignedPartitions; - private KafkaConsumer workerConsumer; + private KafkaConsumer workerDispatcher; // Offset management private final Map pendingOffsets = new HashMap<>(); private final AtomicLong lastCommitTime = new AtomicLong(System.currentTimeMillis()); private final AtomicInteger messagesProcessedSinceLastCommit = new AtomicInteger(0); - public ConsumerWorker(String workerId, List assignedPartitions) { + public DispatcherWorker(String workerId, List assignedPartitions) { this.workerId = workerId; this.assignedPartitions = assignedPartitions; } @@ -322,40 +377,40 @@ public ConsumerWorker(String workerId, List assignedPartitions) { @Override public void run() { try { - // Create consumer for this worker with offset management configuration - Properties workerConsumerProps = new Properties(); - workerConsumerProps.putAll(consumerProps); + // Create dispatcher for this worker with offset management configuration + Properties workerDispatcherProps = new Properties(); + workerDispatcherProps.putAll(dispatcherProps); // Configure offset management based on strategy - configureOffsetManagement(workerConsumerProps); + configureOffsetManagement(workerDispatcherProps); - workerConsumer = new KafkaConsumer<>(workerConsumerProps); + workerDispatcher = new KafkaConsumer<>(workerDispatcherProps); // Create re-balance listener - AuditConsumerRebalanceListener rebalanceListener = new AuditConsumerRebalanceListener( + AuditDispatcherRebalanceListener rebalanceListener = new AuditDispatcherRebalanceListener( workerId, AuditServerConstants.DESTINATION_HDFS, topicName, offsetCommitStrategy, - consumerGroupId, - workerConsumer, + dispatcherGroupId, + workerDispatcher, pendingOffsets, messagesProcessedSinceLastCommit, lastCommitTime, assignedPartitions); // Subscribe to topic with re-balance listener - workerConsumer.subscribe(Collections.singletonList(topicName), rebalanceListener); + workerDispatcher.subscribe(Collections.singletonList(topicName), rebalanceListener); - LOG.info("[HDFS-CONSUMER] Worker '{}' subscribed successfully, waiting for partition assignment from Kafka", workerId); + LOG.info("[HDFS-DISPATCHER] Worker '{}' subscribed successfully, waiting for partition assignment from Kafka", workerId); long threadId = Thread.currentThread().getId(); String threadName = Thread.currentThread().getName(); - LOG.info("[HDFS-CONSUMER-STARTUP] Worker '{}' [Thread-ID: {}, Thread-Name: '{}'] started | Topic: '{}' | Consumer-Group: {} | Mode: SUBSCRIBE", - workerId, threadId, threadName, topicName, consumerGroupId); + LOG.info("[HDFS-DISPATCHER-STARTUP] Worker '{}' [Thread-ID: {}, Thread-Name: '{}'] started | Topic: '{}' | Dispatcher-Group: {} | Mode: SUBSCRIBE", + workerId, threadId, threadName, topicName, dispatcherGroupId); // Consume messages while (running.get()) { - ConsumerRecords records = workerConsumer.poll(Duration.ofMillis(100)); + ConsumerRecords records = workerDispatcher.poll(Duration.ofMillis(100)); if (!records.isEmpty()) { processRecordBatch(records); @@ -363,34 +418,34 @@ public void run() { handleOffsetCommitting(); } } - } catch (Exception e) { - LOG.error("Error in HDFS consumer worker '{}'", workerId, e); + } catch (Throwable e) { + LOG.error("Error in HDFS dispatcher worker '{}'", workerId, e); } finally { // Final offset commit before shutdown commitPendingOffsets(true); - if (workerConsumer != null) { + if (workerDispatcher != null) { try { LOG.info("HDFS Worker '{}': Unsubscribing from topic", workerId); - workerConsumer.unsubscribe(); + workerDispatcher.unsubscribe(); } catch (Exception e) { LOG.warn("HDFS Worker '{}': Error during unsubscribe", workerId, e); } try { - LOG.info("HDFS Worker '{}': Closing consumer", workerId); - workerConsumer.close(); + LOG.info("HDFS Worker '{}': Closing dispatcher", workerId); + workerDispatcher.close(); } catch (Exception e) { - LOG.error("Error closing consumer for HDFS worker '{}'", workerId, e); + LOG.error("Error closing dispatcher for HDFS worker '{}'", workerId, e); } } - LOG.info("HDFS consumer worker '{}' stopped", workerId); + LOG.info("HDFS dispatcher worker '{}' stopped", workerId); } } - private void configureOffsetManagement(Properties consumerProps) { + private void configureOffsetManagement(Properties dispatcherProps) { // Always disable auto commit - only batch or manual strategies supported - consumerProps.put("enable.auto.commit", "false"); + dispatcherProps.put("enable.auto.commit", "false"); LOG.debug("HDFS worker '{}' configured for manual offset commit with strategy: {}", workerId, offsetCommitStrategy); } @@ -442,7 +497,7 @@ private void commitPendingOffsets(boolean isShutdown) { } try { - workerConsumer.commitSync(pendingOffsets); + workerDispatcher.commitSync(pendingOffsets); LOG.debug("HDFS worker '{}' committed {} offsets, processed {} messages", workerId, pendingOffsets.size(), messagesProcessedSinceLastCommit.get()); @@ -458,7 +513,7 @@ private void commitPendingOffsets(boolean isShutdown) { // During shutdown, retry to avoid loss of any offsets try { Thread.sleep(1000); - workerConsumer.commitSync(pendingOffsets); + workerDispatcher.commitSync(pendingOffsets); LOG.info("Successfully committed offsets on retry during shutdown for HDFS worker '{}'", workerId); } catch (Exception retryException) { LOG.error("Failed to commit offsets even on retry during shutdown for HDFS worker '{}'", workerId, retryException); diff --git a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditRouterHDFS.java b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditRouterHDFS.java similarity index 99% rename from audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditRouterHDFS.java rename to audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditRouterHDFS.java index b4b9612a7f..22728393d4 100644 --- a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditRouterHDFS.java +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditRouterHDFS.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.ranger.audit.consumer.kafka; +package org.apache.ranger.audit.dispatcher.kafka; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/audit-server/consumer-hdfs/src/main/resources/conf/core-site.xml b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/core-site.xml similarity index 86% rename from audit-server/consumer-hdfs/src/main/resources/conf/core-site.xml rename to audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/core-site.xml index 3e8bc00a16..30a1f5e832 100644 --- a/audit-server/consumer-hdfs/src/main/resources/conf/core-site.xml +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/core-site.xml @@ -14,7 +14,7 @@ --> @@ -28,11 +28,11 @@ hadoop.security.auth_to_local RULE:[1:$1@$0](rangerauditserver@EXAMPLE\.COM)s/.*/rangerauditserver/ - RULE:[1:$1/$2@$0](rangerauditserver/ranger-audit-consumer-hdfs.rangernw@EXAMPLE\.COM)s/.*/rangerauditserver/ + RULE:[1:$1/$2@$0](rangerauditserver/ranger-audit-dispatcher-hdfs.rangernw@EXAMPLE\.COM)s/.*/rangerauditserver/ DEFAULT - Kerberos principal to local user name mapping rules for Ranger Audit HDFS Consumer. + Kerberos principal to local user name mapping rules for Ranger Audit HDFS Dispatcher. Note: rangerauditserver/hostname@REALM and rangerauditserver@REALM both map to 'ranger' OS user diff --git a/audit-server/consumer-hdfs/src/main/resources/conf/hdfs-site.xml b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/hdfs-site.xml similarity index 96% rename from audit-server/consumer-hdfs/src/main/resources/conf/hdfs-site.xml rename to audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/hdfs-site.xml index 26e604dcfe..da30318127 100644 --- a/audit-server/consumer-hdfs/src/main/resources/conf/hdfs-site.xml +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/hdfs-site.xml @@ -14,7 +14,7 @@ --> diff --git a/audit-server/consumer-hdfs/src/main/resources/conf/logback.xml b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/logback.xml similarity index 93% rename from audit-server/consumer-hdfs/src/main/resources/conf/logback.xml rename to audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/logback.xml index e8a08e8ea8..5c8f205c8b 100644 --- a/audit-server/consumer-hdfs/src/main/resources/conf/logback.xml +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/logback.xml @@ -14,8 +14,8 @@ --> - - + + diff --git a/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/ranger-audit-dispatcher-hdfs-site.xml b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/ranger-audit-dispatcher-hdfs-site.xml new file mode 100644 index 0000000000..174e3a3f70 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/ranger-audit-dispatcher-hdfs-site.xml @@ -0,0 +1,140 @@ + + + + + ranger.audit.dispatcher.hdfs.class + org.apache.ranger.audit.dispatcher.HdfsDispatcherManager + + + + + + log.dir + ${audit.dispatcher.hdfs.log.dir} + Log directory for HDFS dispatcher service + + + + + xasecure.audit.destination.kafka.host + ranger-audit-dispatcher-hdfs.rangernw + + - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In VM: Use the actual FQDN (e.g., ranger.example.com) + + + + + xasecure.audit.destination.kafka.kerberos.principal + rangerauditserver/_HOST@EXAMPLE.COM + + rangerauditserver user kerberos principal for authentication into kafka + + + + + xasecure.audit.destination.kafka.kerberos.keytab + /etc/keytabs/rangerauditserver.keytab + + keytab of the rangerauditserver principal + + + + xasecure.audit.destination.kafka.bootstrap.servers + ranger-kafka:9092 + + + + xasecure.audit.destination.kafka.topic.name + ranger_audits + + + + xasecure.audit.destination.kafka.security.protocol + SASL_PLAINTEXT + + + + xasecure.audit.destination.kafka.sasl.mechanism + GSSAPI + + + + + xasecure.audit.destination.kafka.dispatcher.thread.count + 3 + Number of HDFS dispatcher worker threads + + + + + xasecure.audit.destination.kafka.dispatcher.offset.commit.strategy + batch + batch or manual + + + + xasecure.audit.destination.kafka.dispatcher.offset.commit.interval.ms + 30000 + Used only if strategy is 'manual' + + + + xasecure.audit.destination.kafka.dispatcher.max.poll.records + 500 + Maximum records per poll + + + + + xasecure.audit.destination.kafka.dispatcher.classes + org.apache.ranger.audit.dispatcher.kafka.AuditHDFSDispatcher + + + + + xasecure.audit.destination.hdfs + true + MUST be true for HDFS dispatcher to work + + + + xasecure.audit.destination.hdfs.dir + hdfs://ranger-hadoop:9000/ranger/audit + + Namenode host for HDFS audit destination. + example: hdfs://hdfs-namenode:8020/ranger/audit + In Docker, use the service name of the namenode: hdfs://ranger-hadoop:9000/ranger/audit + + + + + xasecure.audit.destination.hdfs.batch.filespool.dir + /var/log/audit-dispatcher-hdfs/spool + + + + + xasecure.audit.destination.hdfs.batch.filequeue.filetype + json + File format: json or orc + + + + xasecure.audit.destination.hdfs.file.rollover.sec + 86400 + File rollover time in seconds (default: 24 hours) + + diff --git a/audit-server/consumer-solr/pom.xml b/audit-server/audit-dispatcher/dispatcher-solr/pom.xml similarity index 72% rename from audit-server/consumer-solr/pom.xml rename to audit-server/audit-dispatcher/dispatcher-solr/pom.xml index f690ee8f50..52defd99df 100644 --- a/audit-server/consumer-solr/pom.xml +++ b/audit-server/audit-dispatcher/dispatcher-solr/pom.xml @@ -1,21 +1,20 @@ + 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. +--> 4.0.0 @@ -23,13 +22,13 @@ org.apache.ranger ranger 3.0.0-SNAPSHOT - ../.. + ../../.. - audit-consumer-solr - war - Ranger Audit Consumer Solr - Kafka consumer service for indexing audits into Solr/OpenSearch + audit-dispatcher-solr + jar + Ranger Audit Dispatcher Solr + Kafka dispatcher service for indexing audits into Solr/OpenSearch UTF-8 @@ -37,64 +36,6 @@ - - ch.qos.logback - logback-classic - ${logback.version} - - - - - com.fasterxml.jackson.core - jackson-databind - ${fasterxml.jackson.version} - - - com.google.guava - guava - ${google.guava.version} - - - - - com.sun.jersey - jersey-bundle - ${jersey-bundle.version} - - - com.sun.jersey.contribs - jersey-spring - ${jersey-spring.version} - - - com.sun.jersey - jersey-server - - - org.springframework - * - - - - - commons-io - commons-io - ${commons.io.version} - - - javax.servlet - javax.servlet-api - ${javax.servlet.version} - - - - - org.apache.commons - commons-lang3 - ${commons.lang3.version} - - - org.apache.httpcomponents httpasyncclient @@ -127,23 +68,6 @@ httpcore-nio ${httpcomponents.httpcore.version} - - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - log4j - * - - - org.slf4j - * - - - org.apache.lucene lucene-analyzers-common @@ -155,7 +79,6 @@ ${lucene.version} - org.apache.lucene lucene-core @@ -216,10 +139,9 @@ lucene-suggest ${lucene.version} - org.apache.ranger - ranger-audit-common + ranger-audit-dest-solr ${project.version} @@ -229,11 +151,118 @@ - + + ch.qos.logback + logback-classic + ${logback.version} + provided + + + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.jackson.version} + provided + + + com.google.guava + guava + ${google.guava.version} + provided + + + + com.sun.jersey + jersey-bundle + ${jersey-bundle.version} + provided + + + javax.ws.rs + jsr311-api + + + + + com.sun.jersey + jersey-core + ${jersey-bundle.version} + provided + + + com.sun.jersey + jersey-server + ${jersey-bundle.version} + provided + + + com.sun.jersey + jersey-servlet + ${jersey-bundle.version} + provided + + + com.sun.jersey.contribs + jersey-spring + ${jersey-spring.version} + provided + + + com.github.pjfanning + jersey-json + + + com.sun.jersey + jersey-server + + + org.springframework + * + + + + + commons-io + commons-io + ${commons.io.version} + provided + + + javax.servlet + javax.servlet-api + ${javax.servlet.version} + provided + + + + org.apache.commons + commons-lang3 + ${commons.lang3.version} + provided + + + + org.apache.kafka + kafka-clients + ${kafka.version} + provided + + + log4j + * + + + org.slf4j + * + + + + org.apache.ranger ranger-audit-core ${project.version} + provided org.apache.hadoop @@ -241,10 +270,12 @@ + org.apache.ranger - ranger-audit-dest-solr + ranger-audit-dispatcher-common ${project.version} + provided org.apache.hadoop @@ -252,10 +283,18 @@ + + + org.apache.ranger + ranger-audit-server-common + ${project.version} + provided + org.apache.ranger ranger-plugins-common ${project.version} + provided javax.servlet @@ -271,55 +310,63 @@ org.apache.tomcat tomcat-annotations-api ${tomcat.embed.version} + provided - org.apache.tomcat.embed tomcat-embed-core ${tomcat.embed.version} + provided org.apache.tomcat.embed tomcat-embed-el ${tomcat.embed.version} + provided org.apache.tomcat.embed tomcat-embed-jasper ${tomcat.embed.version} + provided org.slf4j log4j-over-slf4j ${slf4j.version} + provided org.slf4j slf4j-api ${slf4j.version} + provided org.springframework spring-beans ${springframework.version} + provided - org.springframework spring-context ${springframework.version} + provided org.springframework spring-web ${springframework.version} + provided + audit-dispatcher-solr-${project.version} true @@ -329,7 +376,29 @@ org.apache.maven.plugins - maven-war-plugin + maven-pmd-plugin + + + ${project.parent.basedir}/dev-support/ranger-pmd-ruleset.xml + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + + copy-dependencies + + package + + ${project.build.directory}/lib + runtime + + + diff --git a/audit-server/audit-dispatcher/dispatcher-solr/src/main/java/org/apache/ranger/audit/dispatcher/SolrDispatcherManager.java b/audit-server/audit-dispatcher/dispatcher-solr/src/main/java/org/apache/ranger/audit/dispatcher/SolrDispatcherManager.java new file mode 100644 index 0000000000..fee93ac85b --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-solr/src/main/java/org/apache/ranger/audit/dispatcher/SolrDispatcherManager.java @@ -0,0 +1,205 @@ +/* + * 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.ranger.audit.dispatcher; + +import org.apache.ranger.audit.dispatcher.kafka.AuditDispatcher; +import org.apache.ranger.audit.dispatcher.kafka.AuditDispatcherTracker; +import org.apache.ranger.audit.provider.MiscUtil; +import org.apache.ranger.audit.server.AuditServerConstants; +import org.apache.ranger.audit.utils.AuditServerLogFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.PostConstruct; + +import java.util.Properties; + +/** + * Spring component that manages the lifecycle of Solr dispatcher threads. + * Manager that manages the lifecycle of Solr dispatcher threads. + * - Initializes the dispatcher tracker + * - Creates Solr dispatcher instances + * - Starts dispatcher threads + * - Handles graceful shutdown + */ +public class SolrDispatcherManager { + private static final Logger LOG = LoggerFactory.getLogger(SolrDispatcherManager.class); + private static final String CONFIG_DISPATCHER_TYPE = "ranger.audit.dispatcher.type"; + + private final AuditDispatcherTracker tracker = AuditDispatcherTracker.getInstance(); + private AuditDispatcher dispatcher; + private Thread dispatcherThread; + + @PostConstruct + public void init(Properties props) { + LOG.info("==> SolrDispatcherManager.init()"); + + String dispatcherType = System.getProperty(CONFIG_DISPATCHER_TYPE); + if (dispatcherType != null && !dispatcherType.equalsIgnoreCase("solr")) { + LOG.info("Skipping SolrDispatcherManager initialization since dispatcher type is {}", dispatcherType); + return; + } + + try { + if (props == null) { + LOG.error("Configuration properties are null"); + throw new RuntimeException("Failed to load configuration"); + } + + boolean isEnabled = MiscUtil.getBooleanProperty(props, "xasecure.audit.destination.solr", false); + if (!isEnabled) { + LOG.warn("Solr destination is disabled (xasecure.audit.destination.solr=false). No dispatchers will be created."); + return; + } + + // Initialize and register Solr Dispatcher + initializeDispatcher(props, AuditServerConstants.PROP_KAFKA_PROP_PREFIX); + + if (dispatcher == null) { + LOG.warn("No dispatcher was created! Verify that xasecure.audit.destination.solr=true and classes are configured correctly."); + } else { + LOG.info("Created Solr dispatcher"); + + // Register shutdown hook + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + LOG.info("JVM shutdown detected, stopping SolrDispatcherManager..."); + shutdown(); + }, "SolrDispatcherManager-ShutdownHook")); + + // Start dispatcher thread + startDispatcher(); + } + } catch (Exception e) { + LOG.error("Failed to initialize SolrDispatcherManager", e); + throw new RuntimeException("Failed to initialize SolrDispatcherManager", e); + } + + LOG.info("<== SolrDispatcherManager.init()"); + } + + private void initializeDispatcher(Properties props, String propPrefix) { + LOG.info("==> SolrDispatcherManager.initializeDispatcher()"); + + // Get dispatcher classes from configuration + String clsStr = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_CLASSES, + "org.apache.ranger.audit.dispatcher.kafka.AuditSolrDispatcher"); + + String solrDispatcherClassName = clsStr.split(",")[0].trim(); + if (solrDispatcherClassName.isEmpty()) { + LOG.error("Dispatcher class name is empty"); + return; + } + + try { + Class dispatcherClass = Class.forName(solrDispatcherClassName); + dispatcher = (AuditDispatcher) dispatcherClass + .getConstructor(Properties.class, String.class) + .newInstance(props, propPrefix); + tracker.addActiveDispatcher("solr", dispatcher); + LOG.info("Successfully initialized dispatcher class: {}", dispatcherClass.getName()); + } catch (ClassNotFoundException e) { + LOG.error("Dispatcher class not found: {}. Ensure the class is on the classpath.", solrDispatcherClassName, e); + } catch (Exception e) { + LOG.error("Error initializing dispatcher class: {}", solrDispatcherClassName, e); + } + + LOG.info("<== SolrDispatcherManager.initializeDispatcher()"); + } + + /** + * Start dispatcher thread + */ + private void startDispatcher() { + LOG.info("==> SolrDispatcherManager.startDispatcher()"); + + logSolrDispatcherStartup(); + + if (dispatcher != null) { + try { + String dispatcherName = dispatcher.getClass().getSimpleName(); + dispatcherThread = new Thread(dispatcher, dispatcherName); + dispatcherThread.setDaemon(true); + dispatcherThread.start(); + + LOG.info("Started {} thread [Thread-ID: {}, Thread-Name: '{}']", + dispatcherName, dispatcherThread.getId(), dispatcherThread.getName()); + } catch (Exception e) { + LOG.error("Error starting dispatcher: {}", dispatcher.getClass().getSimpleName(), e); + } + } + + LOG.info("<== SolrDispatcherManager.startDispatcher()"); + } + + /** + * Log startup banner with dispatcher information + */ + private void logSolrDispatcherStartup() { + LOG.info("################## SOLR DISPATCHER SERVICE STARTUP ######################"); + + if (dispatcher == null) { + LOG.warn("WARNING: No Solr dispatchers are enabled!"); + LOG.warn("Verify: xasecure.audit.destination.solr=true in configuration"); + } else { + AuditServerLogFormatter.LogBuilder builder = AuditServerLogFormatter.builder("Solr Dispatcher Status"); + String dispatcherType = dispatcher.getClass().getSimpleName(); + builder.add(dispatcherType, "ENABLED"); + builder.add("Topic", dispatcher.getTopicName()); + builder.logInfo(LOG); + + LOG.info("Starting Solr dispatcher thread..."); + } + LOG.info("########################################################################"); + } + + public void shutdown() { + LOG.info("==> SolrDispatcherManager.shutdown()"); + + if (dispatcher != null) { + try { + LOG.info("Shutting down dispatcher: {}", dispatcher.getClass().getSimpleName()); + dispatcher.shutdown(); + LOG.info("Dispatcher shutdown completed: {}", dispatcher.getClass().getSimpleName()); + } catch (Exception e) { + LOG.error("Error shutting down dispatcher: {}", dispatcher.getClass().getSimpleName(), e); + } + } + + // Wait for thread to terminate + if (dispatcherThread != null && dispatcherThread.isAlive()) { + try { + LOG.info("Waiting for thread to terminate: {}", dispatcherThread.getName()); + dispatcherThread.join(10000); // Wait up to 10 seconds + if (dispatcherThread.isAlive()) { + LOG.warn("Thread did not terminate within 10 seconds: {}", dispatcherThread.getName()); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for thread to terminate: {}", dispatcherThread.getName(), e); + Thread.currentThread().interrupt(); + } + } + + dispatcher = null; + dispatcherThread = null; + tracker.clearActiveDispatchers(); + + LOG.info("<== SolrDispatcherManager.shutdown() - Solr dispatcher stopped"); + } +} diff --git a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditSolrConsumer.java b/audit-server/audit-dispatcher/dispatcher-solr/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditSolrDispatcher.java similarity index 68% rename from audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditSolrConsumer.java rename to audit-server/audit-dispatcher/dispatcher-solr/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditSolrDispatcher.java index d6c03c9d37..b6d98d4b17 100644 --- a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditSolrConsumer.java +++ b/audit-server/audit-dispatcher/dispatcher-solr/src/main/java/org/apache/ranger/audit/dispatcher/kafka/AuditSolrDispatcher.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.ranger.audit.consumer.kafka; +package org.apache.ranger.audit.dispatcher.kafka; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -49,37 +49,26 @@ import java.util.concurrent.atomic.AtomicLong; /** - * Solr consumer that writes audits into Solr index using rangerauditserver user + * Solr dispatcher that writes audits into Solr index using rangerauditserver user */ -public class AuditSolrConsumer extends AuditConsumerBase { - private static final Logger LOG = LoggerFactory.getLogger(AuditSolrConsumer.class); +public class AuditSolrDispatcher extends AuditDispatcherBase { + private static final Logger LOG = LoggerFactory.getLogger(AuditSolrDispatcher.class); - private static final String RANGER_AUDIT_SOLR_CONSUMER_GROUP = AuditServerConstants.DEFAULT_RANGER_AUDIT_SOLR_CONSUMER_GROUP; + private static final String RANGER_AUDIT_SOLR_DISPATCHER_GROUP = AuditServerConstants.DEFAULT_RANGER_AUDIT_SOLR_DISPATCHER_GROUP; - // Register AuditSolrConsumer factory in the audit consumer registry - static { - try { - AuditConsumerRegistry.getInstance().registerFactory(AuditServerConstants.PROP_SOLR_DEST_PREFIX, AuditSolrConsumer::new); - - LOG.info("Registered Solr consumer factory with AuditConsumerRegistry"); - } catch (Exception e) { - LOG.error("Failed to register Solr consumer factory", e); - } - } - - private final AtomicBoolean running = new AtomicBoolean(false); - private final Map consumerWorkers = new ConcurrentHashMap<>(); - private final SolrAuditDestination solrAuditDestination; + private final AtomicBoolean running = new AtomicBoolean(false); + private final Map dispatcherWorkers = new ConcurrentHashMap<>(); + private final SolrAuditDestination solrAuditDestination; - private ExecutorService consumerThreadPool; - private int consumerThreadCount = 1; + private ExecutorService dispatcherThreadPool; + private int dispatcherThreadCount = 1; // Offset management configuration (batch or manual only supported) private String offsetCommitStrategy = AuditServerConstants.DEFAULT_OFFSET_COMMIT_STRATEGY; private long offsetCommitInterval = AuditServerConstants.DEFAULT_OFFSET_COMMIT_INTERVAL_MS; - public AuditSolrConsumer(Properties props, String propPrefix) throws Exception { - super(props, propPrefix, RANGER_AUDIT_SOLR_CONSUMER_GROUP); + public AuditSolrDispatcher(Properties props, String propPrefix) throws Exception { + super(props, propPrefix, RANGER_AUDIT_SOLR_DISPATCHER_GROUP); this.solrAuditDestination = new SolrAuditDestination(); @@ -90,15 +79,15 @@ public AuditSolrConsumer(Properties props, String propPrefix) throws Exception { public void run() { try { if (running.compareAndSet(false, true)) { - startConsumerWorkers(); + startDispatcherWorkers(); } - // Keep main thread alive while consumer threads are running + // Keep main thread alive while dispatcher threads are running while (running.get()) { Thread.sleep(1000); } } catch (Throwable e) { - LOG.error("Error in AuditSolrConsumer", e); + LOG.error("Error in AuditSolrDispatcher", e); } finally { shutdown(); } @@ -106,58 +95,58 @@ public void run() { @Override public void shutdown() { - LOG.info("Shutting down AuditSolrConsumer..."); + LOG.info("Shutting down AuditSolrDispatcher..."); running.set(false); - // Shutdown consumer workers - if (consumerThreadPool != null) { - consumerThreadPool.shutdownNow(); + // Shutdown dispatcher workers + if (dispatcherThreadPool != null) { + dispatcherThreadPool.shutdownNow(); try { - if (!consumerThreadPool.awaitTermination(30, TimeUnit.SECONDS)) { - LOG.warn("Consumer thread pool did not terminate within 30 seconds"); + if (!dispatcherThreadPool.awaitTermination(30, TimeUnit.SECONDS)) { + LOG.warn("Dispatcher thread pool did not terminate within 30 seconds"); } } catch (InterruptedException e) { - LOG.warn("Interrupted while waiting for consumer thread pool to terminate", e); + LOG.warn("Interrupted while waiting for dispatcher thread pool to terminate", e); Thread.currentThread().interrupt(); } } - // Close main consumer - if (consumer != null) { + // Close main dispatcher + if (dispatcher != null) { try { - consumer.close(); + dispatcher.close(); } catch (Exception e) { - LOG.error("Error closing main consumer", e); + LOG.error("Error closing main dispatcher", e); } } - consumerWorkers.clear(); + dispatcherWorkers.clear(); - LOG.info("AuditSolrConsumer shutdown complete"); + LOG.info("AuditSolrDispatcher shutdown complete"); } private void init(Properties props, String propPrefix) throws Exception { - LOG.info("==> AuditSolrConsumer.init()"); + LOG.info("==> AuditSolrDispatcher.init()"); solrAuditDestination.init(props, AuditProviderFactory.AUDIT_DEST_BASE + "." + AuditServerConstants.PROP_SOLR_DEST_PREFIX); - this.consumerThreadCount = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_THREAD_COUNT, 1); + this.dispatcherThreadCount = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_THREAD_COUNT, 1); - LOG.info("Consumer thread count: {}", consumerThreadCount); + LOG.info("Dispatcher thread count: {}", dispatcherThreadCount); - this.offsetCommitStrategy = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_OFFSET_COMMIT_STRATEGY, AuditServerConstants.DEFAULT_OFFSET_COMMIT_STRATEGY); + this.offsetCommitStrategy = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_OFFSET_COMMIT_STRATEGY, AuditServerConstants.DEFAULT_OFFSET_COMMIT_STRATEGY); // Get offset commit interval (only used for manual strategy) - this.offsetCommitInterval = MiscUtil.getLongProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_OFFSET_COMMIT_INTERVAL, AuditServerConstants.DEFAULT_OFFSET_COMMIT_INTERVAL_MS); + this.offsetCommitInterval = MiscUtil.getLongProperty(props, propPrefix + "." + AuditServerConstants.PROP_DISPATCHER_OFFSET_COMMIT_INTERVAL, AuditServerConstants.DEFAULT_OFFSET_COMMIT_INTERVAL_MS); - AuditServerLogFormatter.builder("AuditSolrConsumer Offset Management Configuration") + AuditServerLogFormatter.builder("AuditSolrDispatcher Offset Management Configuration") .add("Commit Strategy", offsetCommitStrategy) .add("Commit Interval (ms)", offsetCommitInterval + " (used in manual mode only)") .logInfo(LOG); - LOG.info("<== AuditSolrConsumer.init()"); + LOG.info("<== AuditSolrDispatcher.init()"); } /** @@ -177,45 +166,45 @@ public void processMessageBatch(Collection audits) throws Exception { } /** - * Start consumer workers for horizontal scaling. + * Start dispatcher workers for horizontal scaling. * Each worker subscribes to the topic and Kafka automatically assigns partitions. * Workers can process messages from ANY appId. */ - private void startConsumerWorkers() { - int workerCount = consumerThreadCount; + private void startDispatcherWorkers() { + int workerCount = dispatcherThreadCount; - LOG.info("==> AuditSolrConsumer.startConsumerWorkers(): Creating {} generic workers for horizontal scaling", workerCount); + LOG.info("==> AuditSolrDispatcher.startDispatcherWorkers(): Creating {} generic workers for horizontal scaling", workerCount); LOG.info("Each worker will subscribe to topic '{}' and process partitions assigned by Kafka", topicName); // Create thread pool sized for generic workers - consumerThreadPool = Executors.newFixedThreadPool(workerCount); + dispatcherThreadPool = Executors.newFixedThreadPool(workerCount); LOG.info("Created thread pool with {} threads for scalable SOLR consumption", workerCount); // Create generic workers (no appId pre-assignment) for (int i = 0; i < workerCount; i++) { String workerId = "solr-worker-" + i; // Pass empty list for partitions (Kafka will assign dynamically) - ConsumerWorker worker = new ConsumerWorker(workerId, new ArrayList<>()); - consumerWorkers.put(workerId, worker); - consumerThreadPool.submit(worker); + DispatcherWorker worker = new DispatcherWorker(workerId, new ArrayList<>()); + dispatcherWorkers.put(workerId, worker); + dispatcherThreadPool.submit(worker); - LOG.info("Started SOLR consumer worker '{}' - will process ANY appId assigned by Kafka", workerId); + LOG.info("Started SOLR dispatcher worker '{}' - will process ANY appId assigned by Kafka", workerId); } - LOG.info("<== AuditSolrConsumer.startConsumerWorkers(): All {} workers started in SUBSCRIBE mode", workerCount); + LOG.info("<== AuditSolrDispatcher.startDispatcherWorkers(): All {} workers started in SUBSCRIBE mode", workerCount); } - private class ConsumerWorker implements Runnable { + private class DispatcherWorker implements Runnable { private final String workerId; private final List assignedPartitions; - private KafkaConsumer workerConsumer; + private KafkaConsumer workerDispatcher; // Offset management private final Map pendingOffsets = new HashMap<>(); private final AtomicLong lastCommitTime = new AtomicLong(System.currentTimeMillis()); private final AtomicInteger messagesProcessedSinceLastCommit = new AtomicInteger(0); - public ConsumerWorker(String workerId, List assignedPartitions) { + public DispatcherWorker(String workerId, List assignedPartitions) { this.workerId = workerId; this.assignedPartitions = assignedPartitions; } @@ -223,40 +212,40 @@ public ConsumerWorker(String workerId, List assignedPartitions) { @Override public void run() { try { - // Create consumer for this worker with offset management configuration - Properties workerConsumerProps = new Properties(); - workerConsumerProps.putAll(consumerProps); + // Create dispatcher for this worker with offset management configuration + Properties workerDispatcherProps = new Properties(); + workerDispatcherProps.putAll(dispatcherProps); // Configure offset management based on strategy - configureOffsetManagement(workerConsumerProps); + configureOffsetManagement(workerDispatcherProps); - workerConsumer = new KafkaConsumer<>(workerConsumerProps); + workerDispatcher = new KafkaConsumer<>(workerDispatcherProps); // Create re-balance listener - AuditConsumerRebalanceListener reBalanceListener = new AuditConsumerRebalanceListener( + AuditDispatcherRebalanceListener reBalanceListener = new AuditDispatcherRebalanceListener( workerId, AuditServerConstants.DESTINATION_SOLR, topicName, offsetCommitStrategy, - consumerGroupId, - workerConsumer, + dispatcherGroupId, + workerDispatcher, pendingOffsets, messagesProcessedSinceLastCommit, lastCommitTime, assignedPartitions); // Subscribe to topic with re-balance listener and let kafka automatically assign partitions - workerConsumer.subscribe(Collections.singletonList(topicName), reBalanceListener); + workerDispatcher.subscribe(Collections.singletonList(topicName), reBalanceListener); - LOG.info("[SOLR-CONSUMER] Worker '{}' subscribed successfully, waiting for partition assignment from Kafka", workerId); + LOG.info("[SOLR-DISPATCHER] Worker '{}' subscribed successfully, waiting for partition assignment from Kafka", workerId); long threadId = Thread.currentThread().getId(); String threadName = Thread.currentThread().getName(); - LOG.info("[SOLR-CONSUMER-STARTUP] Worker '{}' [Thread-ID: {}, Thread-Name: '{}'] started | Topic: '{}' | Consumer-Group: {} | Mode: SUBSCRIBE", - workerId, threadId, threadName, topicName, consumerGroupId); + LOG.info("[SOLR-DISPATCHER-STARTUP] Worker '{}' [Thread-ID: {}, Thread-Name: '{}'] started | Topic: '{}' | Dispatcher-Group: {} | Mode: SUBSCRIBE", + workerId, threadId, threadName, topicName, dispatcherGroupId); // Consume messages while (running.get()) { - ConsumerRecords records = workerConsumer.poll(Duration.ofMillis(100)); + ConsumerRecords records = workerDispatcher.poll(Duration.ofMillis(100)); if (!records.isEmpty()) { processRecordBatch(records); @@ -265,38 +254,38 @@ public void run() { handleOffsetCommitting(); } } - } catch (Exception e) { - LOG.error("Error in consumer worker '{}'", workerId, e); + } catch (Throwable e) { + LOG.error("Error in dispatcher worker '{}'", workerId, e); } finally { // Final offset commit before shutdown commitPendingOffsets(true); - if (workerConsumer != null) { + if (workerDispatcher != null) { try { // Unsubscribe before closing LOG.info("Worker '{}': Unsubscribing from topic", workerId); - workerConsumer.unsubscribe(); + workerDispatcher.unsubscribe(); } catch (Exception e) { LOG.warn("Worker '{}': Error during unsubscribe", workerId, e); } try { - LOG.info("Worker '{}': Closing consumer", workerId); - workerConsumer.close(); + LOG.info("Worker '{}': Closing dispatcher", workerId); + workerDispatcher.close(); } catch (Exception e) { - LOG.error("Error closing consumer for worker '{}'", workerId, e); + LOG.error("Error closing dispatcher for worker '{}'", workerId, e); } } - LOG.info("Consumer worker '{}' stopped", workerId); + LOG.info("Dispatcher worker '{}' stopped", workerId); } } /** - * Configure offset management for this worker's consumer (batch or manual) + * Configure offset management for this worker's dispatcher (batch or manual) */ - private void configureOffsetManagement(Properties consumerProps) { + private void configureOffsetManagement(Properties dispatcherProps) { // Always disable auto commit - only batch or manual strategies supported - consumerProps.put("enable.auto.commit", "false"); + dispatcherProps.put("enable.auto.commit", "false"); LOG.debug("Worker '{}' configured for manual offset commit with strategy: {}", workerId, offsetCommitStrategy); } @@ -373,7 +362,7 @@ private void commitPendingOffsets(boolean isShutdown) { } try { - workerConsumer.commitSync(pendingOffsets); + workerDispatcher.commitSync(pendingOffsets); LOG.debug("Worker '{}' committed {} offsets, processed {} messages", workerId, pendingOffsets.size(), messagesProcessedSinceLastCommit.get()); @@ -389,7 +378,7 @@ private void commitPendingOffsets(boolean isShutdown) { // During shutdown, retry to avoid loss of any offsets try { Thread.sleep(1000); - workerConsumer.commitSync(pendingOffsets); + workerDispatcher.commitSync(pendingOffsets); LOG.info("Successfully committed offsets on retry during shutdown for worker '{}'", workerId); } catch (Exception retryException) { LOG.error("Failed to commit offsets even on retry during shutdown for worker '{}'", workerId, retryException); diff --git a/audit-server/consumer-solr/src/main/resources/conf/logback.xml b/audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf/logback.xml similarity index 94% rename from audit-server/consumer-solr/src/main/resources/conf/logback.xml rename to audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf/logback.xml index fb7eced612..3a1eb4e304 100644 --- a/audit-server/consumer-solr/src/main/resources/conf/logback.xml +++ b/audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf/logback.xml @@ -14,8 +14,8 @@ --> - - + + diff --git a/audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf/ranger-audit-dispatcher-solr-site.xml b/audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf/ranger-audit-dispatcher-solr-site.xml new file mode 100644 index 0000000000..010d7bbe86 --- /dev/null +++ b/audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf/ranger-audit-dispatcher-solr-site.xml @@ -0,0 +1,190 @@ + + + + + ranger.audit.dispatcher.solr.class + org.apache.ranger.audit.dispatcher.SolrDispatcherManager + + + + + + log.dir + ${audit.dispatcher.solr.log.dir} + Log directory for Solr dispatcher service + + + + + xasecure.audit.destination.kafka.host + ranger-audit-dispatcher-solr.rangernw + + - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In VM: Use the actual FQDN (e.g., ranger.example.com) + + + + + xasecure.audit.destination.kafka.kerberos.principal + rangerauditserver/_HOST@EXAMPLE.COM + + rangerauditserver user kerberos principal for authentication into kafka + + + + + xasecure.audit.destination.kafka.kerberos.keytab + /etc/keytabs/rangerauditserver.keytab + + keytab of the rangerauditserver principal + + + + + + xasecure.audit.destination.kafka.bootstrap.servers + ranger-kafka:9092 + + + + xasecure.audit.destination.kafka.topic.name + ranger_audits + + + + xasecure.audit.destination.kafka.security.protocol + SASL_PLAINTEXT + + + + xasecure.audit.destination.kafka.sasl.mechanism + GSSAPI + + + + + xasecure.audit.destination.kafka.dispatcher.thread.count + 5 + Number of Solr dispatcher worker threads (higher for indexing throughput) + + + + + xasecure.audit.destination.kafka.dispatcher.offset.commit.strategy + batch + batch or manual + + + + xasecure.audit.destination.kafka.dispatcher.offset.commit.interval.ms + 30000 + Used only if strategy is 'manual' + + + + xasecure.audit.destination.kafka.dispatcher.max.poll.records + 500 + Maximum records per poll for batch processing + + + + + xasecure.audit.destination.kafka.dispatcher.classes + org.apache.ranger.audit.dispatcher.kafka.AuditSolrDispatcher + + + + + + xasecure.audit.destination.solr + true + MUST be true for Solr dispatcher to work + + + + xasecure.audit.destination.solr.urls + http://ranger-solr:8983/solr/ranger_audits + + Solr URLs for audits when SolrCloud is not enabled. + Docker supports only standalone mode for now, + configure http://ranger-solr:8983/solr/ranger_audits + + + + + xasecure.audit.destination.solr.zookeepers + + + Zookeepers for Solr audit destination (SolrCloud mode). + example: zookeeper-host1:2181,zookeeper-host2:2181,zookeeper-host3:2181 + In Docker, use: ranger-zk:2181/ranger_audits + + + + + xasecure.audit.destination.solr.batch.filespool.dir + /var/log/audit-dispatcher-solr/spool + + + + + xasecure.audit.destination.solr.force.use.inmemory.jaas.config + true + + + + xasecure.audit.jaas.Client.loginModuleName + com.sun.security.auth.module.Krb5LoginModule + + + + xasecure.audit.jaas.Client.loginModuleControlFlag + required + + + + xasecure.audit.jaas.Client.option.useKeyTab + true + + + + xasecure.audit.jaas.Client.option.storeKey + true + + + + xasecure.audit.jaas.Client.option.useTicketCache + true + Allow use of cached Kerberos tickets for Solr authentication + + + + xasecure.audit.jaas.Client.option.serviceName + solr + Kerberos service name for Solr + + + + xasecure.audit.jaas.Client.option.principal + rangerauditserver/_HOST@EXAMPLE.COM + Principal for Solr authentication + + + + xasecure.audit.jaas.Client.option.keyTab + /etc/keytabs/rangerauditserver.keytab + Keytab for Solr authentication + + diff --git a/audit-server/audit-dispatcher/pom.xml b/audit-server/audit-dispatcher/pom.xml new file mode 100644 index 0000000000..a306cc1674 --- /dev/null +++ b/audit-server/audit-dispatcher/pom.xml @@ -0,0 +1,49 @@ + + + + 4.0.0 + + + org.apache.ranger + ranger + 3.0.0-SNAPSHOT + ../.. + + + ranger-audit-dispatcher + pom + Ranger Audit Dispatcher Parent + Parent module for audit dispatcher services + + + dispatcher-app + dispatcher-common + dispatcher-hdfs + dispatcher-solr + + + + + + org.apache.ranger + ranger-audit-dispatcher-common + ${project.version} + + + + diff --git a/audit-server/audit-dispatcher/scripts/start-audit-dispatcher.sh b/audit-server/audit-dispatcher/scripts/start-audit-dispatcher.sh new file mode 100755 index 0000000000..e521d6d694 --- /dev/null +++ b/audit-server/audit-dispatcher/scripts/start-audit-dispatcher.sh @@ -0,0 +1,154 @@ +#!/bin/bash + +# 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. + +# Resolve real path +realScriptPath=$(readlink -f "$0" 2>/dev/null) +if [ $? -eq 0 ]; then + scriptDir=$(dirname "$realScriptPath") +else + scriptDir=$(cd "$(dirname "$0")"; pwd) +fi + +AUDIT_DISPATCHER_HOME_DIR=${AUDIT_DISPATCHER_HOME_DIR:-$(cd "${scriptDir}/.."; pwd)} +AUDIT_DISPATCHER_CONF_DIR=${AUDIT_DISPATCHER_CONF_DIR:-"${AUDIT_DISPATCHER_HOME_DIR}/conf"} +AUDIT_DISPATCHER_LOG_DIR=${AUDIT_DISPATCHER_LOG_DIR:-"${AUDIT_DISPATCHER_HOME_DIR}/logs"} + +COMMON_CONF_FILE="${AUDIT_DISPATCHER_CONF_DIR}/ranger-audit-dispatcher-site.xml" + +# Determine the dispatcher type for starting the dispatcher of that type +DISPATCHER_TYPE=$1 + +CONF_FILE="${COMMON_CONF_FILE}" + +# Set default heap size if not set +if [ -z "${AUDIT_DISPATCHER_HEAP}" ]; then + AUDIT_DISPATCHER_HEAP="-Xms512m -Xmx2g" +fi + +# Set default Java options if not set +if [ -z "${AUDIT_DISPATCHER_OPTS}" ]; then + AUDIT_DISPATCHER_OPTS="-Dlogback.configurationFile=${AUDIT_DISPATCHER_CONF_DIR}/logback.xml \ + -Daudit.dispatcher.log.dir=${AUDIT_DISPATCHER_LOG_DIR} \ + -Daudit.dispatcher.log.file=ranger-audit-dispatcher.log \ + -Djava.net.preferIPv4Stack=true \ + -server -XX:+UseG1GC -XX:MaxGCPauseMillis=200 \ + -XX:InitiatingHeapOccupancyPercent=35 -XX:ConcGCThreads=4 -XX:ParallelGCThreads=8" +fi + +export AUDIT_DISPATCHER_OPTS + +echo "[INFO] JAVA_HOME: ${JAVA_HOME}" +echo "[INFO] AUDIT_DISPATCHER_HEAP: ${AUDIT_DISPATCHER_HEAP}" +echo "[INFO] AUDIT_DISPATCHER_OPTS: ${AUDIT_DISPATCHER_OPTS}" + +if [ ! -f "${CONF_FILE}" ]; then + echo "[ERROR] Configuration file not found: ${CONF_FILE}" + exit 1 +fi + +if [ -n "${DISPATCHER_TYPE}" ]; then + echo "[INFO] Starting dispatcher of type: ${DISPATCHER_TYPE}" +else + echo "[INFO] Starting dispatcher (type will be determined from configuration)" +fi + +# Helper function to read property from XML +get_property_value() { + local prop_name=$1 + local file_path=$2 + if [ -f "$file_path" ]; then + local val=$(grep -A 1 "${prop_name}" "$file_path" | grep "" | sed -e 's/.*\(.*\)<\/value>.*/\1/') + # Trim whitespace + echo "${val}" | sed -e 's/^[[:space:]]*//' -e 's/[[:space:]]*$//' + fi +} + +WAR_FILE_NAME=$(get_property_value "ranger.audit.dispatcher.war.file" "${COMMON_CONF_FILE}") +if [ -z "${WAR_FILE_NAME}" ]; then + WAR_FILE_NAME="ranger-audit-dispatcher.war" +fi + +MAIN_CLASS=$(get_property_value "ranger.audit.dispatcher.launcher.class" "${COMMON_CONF_FILE}") +if [ -z "${MAIN_CLASS}" ]; then + MAIN_CLASS="org.apache.ranger.audit.dispatcher.AuditDispatcherLauncher" +fi + +WEBAPP_ROOT="${AUDIT_DISPATCHER_HOME_DIR}/webapp" +WAR_FILE="${WEBAPP_ROOT}/${WAR_FILE_NAME}" + +if [ -n "${DISPATCHER_TYPE}" ]; then + WEBAPP_DIR="${WEBAPP_ROOT}/audit-dispatcher-${DISPATCHER_TYPE}" +else + WEBAPP_DIR="${WEBAPP_ROOT}/audit-dispatcher" +fi + +if [ ! -f "${WAR_FILE}" ]; then + echo "[ERROR] WAR file not found: ${WAR_FILE}" + exit 1 +fi + +# Extract the specific WAR +if [ -f "${WAR_FILE}" ] && [ ! -d "${WEBAPP_DIR}/WEB-INF" ]; then + echo "[INFO] Extracting ${WAR_FILE}..." + mkdir -p "${WEBAPP_DIR}" + cd "${WEBAPP_DIR}" + jar xf "${WAR_FILE}" + cd - > /dev/null +fi + +# Build minimal classpath for the Launcher +RANGER_CLASSPATH="${AUDIT_DISPATCHER_CONF_DIR}:${WEBAPP_DIR}/WEB-INF/classes" +for jar in "${WEBAPP_DIR}"/WEB-INF/lib/*.jar; do + if [ -f "${jar}" ]; then + RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" + fi +done +for jar in "${AUDIT_DISPATCHER_HOME_DIR}"/lib/common/*.jar; do + if [ -f "${jar}" ]; then + RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" + fi +done + +# Add libext directory if it exists +if [ -d "${AUDIT_DISPATCHER_HOME_DIR}/libext" ]; then + for jar in "${AUDIT_DISPATCHER_HOME_DIR}"/libext/*.jar; do + if [ -f "${jar}" ]; then + RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" + fi + done +fi + +echo "[INFO] Webapp dir: ${WEBAPP_DIR}" +echo "[INFO] Main class: ${MAIN_CLASS}" + +JAVA_CMD="java ${AUDIT_DISPATCHER_HEAP} ${AUDIT_DISPATCHER_OPTS}" + +if [ -n "${DISPATCHER_TYPE}" ]; then + JAVA_CMD="${JAVA_CMD} -Dranger.audit.dispatcher.type=${DISPATCHER_TYPE}" +fi + +JAVA_CMD="${JAVA_CMD} -Dranger.audit.dispatcher.webapp.dir=${WEBAPP_DIR} -cp ${RANGER_CLASSPATH} ${MAIN_CLASS}" + +# Start the Java application +if [ -n "${HADOOP_HOME}" ]; then + export HADOOP_HOME=${HADOOP_HOME} +else + # Set a default HADOOP_HOME if not set, typical for Ranger docker + export HADOOP_HOME=/opt/ranger/audit-dispatcher +fi + +exec ${JAVA_CMD} \ No newline at end of file diff --git a/audit-server/audit-dispatcher/scripts/stop-audit-dispatcher.sh b/audit-server/audit-dispatcher/scripts/stop-audit-dispatcher.sh new file mode 100755 index 0000000000..689d4202dc --- /dev/null +++ b/audit-server/audit-dispatcher/scripts/stop-audit-dispatcher.sh @@ -0,0 +1,62 @@ +#!/bin/bash + +# 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. + +realScriptPath=$(readlink -f "$0" 2>/dev/null) +if [ $? -eq 0 ]; then + scriptDir=$(dirname "$realScriptPath") +else + scriptDir=$(cd "$(dirname "$0")"; pwd) +fi + +AUDIT_DISPATCHER_HOME_DIR=${AUDIT_DISPATCHER_HOME_DIR:-$(cd "${scriptDir}/.."; pwd)} +AUDIT_DISPATCHER_LOG_DIR=${AUDIT_DISPATCHER_LOG_DIR:-"${AUDIT_DISPATCHER_HOME_DIR}/logs"} + +DISPATCHER_TYPE=$1 + +if [ -n "${DISPATCHER_TYPE}" ]; then + PID_FILE="${AUDIT_DISPATCHER_LOG_DIR}/ranger-audit-dispatcher-${DISPATCHER_TYPE}.pid" +else + PID_FILE="${AUDIT_DISPATCHER_LOG_DIR}/ranger-audit-dispatcher.pid" +fi + +if [ -f "$PID_FILE" ]; then + PID=$(cat "$PID_FILE") + if ps -p $PID > /dev/null; then + echo "Stopping Ranger Audit Dispatcher (Type: ${DISPATCHER_TYPE:-unknown}) with PID: $PID" + kill "$PID" + + # Wait for process to exit + for i in {1..30}; do + if ! ps -p $PID > /dev/null; then + echo "Service stopped." + rm -f "$PID_FILE" + exit 0 + fi + sleep 1 + done + + echo "Service did not stop gracefully, forcing kill..." + kill -9 "$PID" + rm -f "$PID_FILE" + else + echo "Process with PID $PID is not running. Removing stale PID file." + rm -f "$PID_FILE" + fi +else + echo "PID file not found at $PID_FILE. Service may not be running." +fi diff --git a/audit-server/server/pom.xml b/audit-server/audit-ingestor/pom.xml similarity index 88% rename from audit-server/server/pom.xml rename to audit-server/audit-ingestor/pom.xml index e786246700..99206a82d5 100644 --- a/audit-server/server/pom.xml +++ b/audit-server/audit-ingestor/pom.xml @@ -1,21 +1,20 @@ + 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. +--> 4.0.0 @@ -26,14 +25,13 @@ ../.. - audit-server + ranger-audit-ingestor war - Ranger Audit Server Service - Core audit service: REST API for receiving audit events and producing to Kafka + Ranger Audit Ingestor + Audit ingestion service: REST API for receiving audit events from plugins and producing to Kafka UTF-8 - false @@ -45,6 +43,7 @@ ${logback.version} + com.fasterxml.jackson.core jackson-annotations @@ -66,6 +65,7 @@ ${fasterxml.jackson.version} + com.google.guava guava @@ -83,7 +83,7 @@ - + com.sun.jersey jersey-bundle @@ -99,25 +99,21 @@ - com.sun.jersey jersey-core ${jersey-bundle.version} - com.sun.jersey jersey-server ${jersey-bundle.version} - com.sun.jersey jersey-servlet ${jersey-bundle.version} - com.sun.jersey.contribs jersey-spring @@ -138,42 +134,43 @@ + com.sun.xml.bind jaxb-impl ${jaxb-impl.version} + commons-codec commons-codec ${commons.codec.version} - commons-io commons-io ${commons.io.version} + javax.servlet javax.servlet-api ${javax.servlet.version} - javax.xml.bind jaxb-api ${jaxb.api.version} - org.apache.commons commons-lang3 ${commons.lang3.version} + org.apache.hadoop hadoop-common @@ -244,11 +241,9 @@ - - org.apache.ranger - ranger-audit-common + ranger-audit-core ${project.version} @@ -257,16 +252,12 @@ + + org.apache.ranger - ranger-audit-core + ranger-audit-server-common ${project.version} - - - org.apache.hadoop - hadoop-client-api - - org.apache.ranger @@ -319,7 +310,7 @@ - + org.apache.tomcat tomcat-annotations-api @@ -341,7 +332,7 @@ ${tomcat.embed.version} - + org.slf4j log4j-over-slf4j @@ -385,6 +376,8 @@ spring-web ${springframework.version} + + org.springframework.security spring-security-config @@ -410,6 +403,7 @@ + audit-ingestor-${project.version} true @@ -421,6 +415,15 @@ org.apache.maven.plugins maven-war-plugin + + org.apache.maven.plugins + maven-pmd-plugin + + + ${project.parent.basedir}/dev-support/ranger-pmd-ruleset.xml + + + diff --git a/audit-server/server/scripts/start-audit-server.sh b/audit-server/audit-ingestor/scripts/start-audit-ingestor.sh similarity index 84% rename from audit-server/server/scripts/start-audit-server.sh rename to audit-server/audit-ingestor/scripts/start-audit-ingestor.sh index 52b9a00f26..5aff0d963b 100755 --- a/audit-server/server/scripts/start-audit-server.sh +++ b/audit-server/audit-ingestor/scripts/start-audit-ingestor.sh @@ -17,9 +17,9 @@ # limitations under the License. # ======================================== -# Ranger Audit Server Service Start Script +# Ranger Audit Ingestor Service Start Script # ======================================== -# This script starts the Ranger Audit Server service +# This script starts the Ranger Audit Ingestor service # The service receives audit events from Ranger plugins and produces them to Kafka set -e @@ -37,7 +37,7 @@ AUDIT_SERVER_LOG_DIR="${AUDIT_SERVER_LOG_DIR:-${SERVICE_DIR}/logs}" mkdir -p "${AUDIT_SERVER_LOG_DIR}" echo "==========================================" -echo "Starting Ranger Audit Server Service" +echo "Starting Ranger Audit Ingestor Service" echo "==========================================" echo "Service Directory: ${SERVICE_DIR}" echo "Home Directory: ${AUDIT_SERVER_HOME_DIR}" @@ -69,7 +69,7 @@ AUDIT_SERVER_HEAP="${AUDIT_SERVER_HEAP:--Xms512m -Xmx2g}" if [ -z "$AUDIT_SERVER_OPTS" ]; then AUDIT_SERVER_OPTS="-Dlogback.configurationFile=${AUDIT_SERVER_CONF_DIR}/logback.xml" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Daudit.server.log.dir=${AUDIT_SERVER_LOG_DIR}" - AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Daudit.server.log.file=ranger-audit-server.log" + AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Daudit.server.log.file=ranger-audit-ingestor.log" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Djava.net.preferIPv4Stack=true -server" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -XX:+UseG1GC -XX:MaxGCPauseMillis=200" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -XX:InitiatingHeapOccupancyPercent=35" @@ -90,7 +90,7 @@ echo "[INFO] HEAP: ${AUDIT_SERVER_HEAP}" echo "[INFO] JVM_OPTS: ${AUDIT_SERVER_OPTS}" # Find the WAR file -WAR_FILE=$(find "${AUDIT_SERVER_HOME_DIR}" -name "ranger-audit-server-service*.war" | head -1) +WAR_FILE=$(find "${AUDIT_SERVER_HOME_DIR}" -name "ranger-audit-ingestor*.war" | head -1) if [ -z "$WAR_FILE" ] || [ ! -f "$WAR_FILE" ]; then echo "[ERROR] WAR file not found in ${AUDIT_SERVER_HOME_DIR}" @@ -101,7 +101,7 @@ fi echo "[INFO] Using WAR file: ${WAR_FILE}" # Extract WAR if not already extracted -WEBAPP_DIR="${AUDIT_SERVER_HOME_DIR}/webapp/ranger-audit-server-service" +WEBAPP_DIR="${AUDIT_SERVER_HOME_DIR}/webapp/ranger-audit-ingestor-service" if [ ! -d "${WEBAPP_DIR}" ]; then echo "[INFO] Extracting WAR file..." mkdir -p "${WEBAPP_DIR}" @@ -128,12 +128,12 @@ fi export RANGER_CLASSPATH # Check if already running -PID_FILE="${AUDIT_SERVER_LOG_DIR}/ranger-audit-server.pid" +PID_FILE="${AUDIT_SERVER_LOG_DIR}/ranger-audit-ingestor.pid" if [ -f "${PID_FILE}" ]; then OLD_PID=$(cat "${PID_FILE}") if kill -0 "$OLD_PID" 2>/dev/null; then - echo "[WARNING] Ranger Audit Server is already running (PID: ${OLD_PID})" - echo "[INFO] Use stop-audit-server.sh to stop it first" + echo "[WARNING] Ranger Audit Ingestor is already running (PID: ${OLD_PID})" + echo "[INFO] Use stop-audit-ingestor.sh to stop it first" exit 1 else echo "[INFO] Removing stale PID file" @@ -142,9 +142,9 @@ if [ -f "${PID_FILE}" ]; then fi # Start the service -echo "[INFO] Starting Ranger Audit Server Service..." +echo "[INFO] Starting Ranger Audit Ingestor Service..." nohup java ${AUDIT_SERVER_HEAP} ${AUDIT_SERVER_OPTS} \ - -Daudit.config=${AUDIT_SERVER_CONF_DIR}/ranger-audit-server-site.xml \ + -Daudit.config=${AUDIT_SERVER_CONF_DIR}/ranger-audit-ingestor-site.xml \ -cp "${RANGER_CLASSPATH}" \ org.apache.ranger.audit.server.AuditServerApplication \ >> "${AUDIT_SERVER_LOG_DIR}/catalina.out" 2>&1 & @@ -152,11 +152,11 @@ nohup java ${AUDIT_SERVER_HEAP} ${AUDIT_SERVER_OPTS} \ PID=$! echo $PID > "${PID_FILE}" -echo "[INFO] ✓ Ranger Audit Server started successfully" +echo "[INFO] Ranger Audit Ingestor started successfully" echo "[INFO] PID: ${PID}" -echo "[INFO] Log file: ${AUDIT_SERVER_LOG_DIR}/ranger-audit-server.log" +echo "[INFO] Log file: ${AUDIT_SERVER_LOG_DIR}/ranger-audit-ingestor.log" echo "[INFO] Catalina out: ${AUDIT_SERVER_LOG_DIR}/catalina.out" echo "[INFO] Health check: http://localhost:7081/api/audit/health" echo "" -echo "To monitor logs: tail -f ${AUDIT_SERVER_LOG_DIR}/ranger-audit-server.log" -echo "To stop service: ${SCRIPT_DIR}/stop-audit-server.sh" +echo "To monitor logs: tail -f ${AUDIT_SERVER_LOG_DIR}/ranger-audit-ingestor.log" +echo "To stop service: ${SCRIPT_DIR}/stop-audit-ingestor.sh" diff --git a/audit-server/server/scripts/stop-audit-server.sh b/audit-server/audit-ingestor/scripts/stop-audit-ingestor.sh similarity index 90% rename from audit-server/server/scripts/stop-audit-server.sh rename to audit-server/audit-ingestor/scripts/stop-audit-ingestor.sh index 87ff4af9ad..023732d274 100755 --- a/audit-server/server/scripts/stop-audit-server.sh +++ b/audit-server/audit-ingestor/scripts/stop-audit-ingestor.sh @@ -17,7 +17,7 @@ # limitations under the License. # ======================================== -# Ranger Audit Server Service Stop Script +# Ranger Audit Ingestor Service Stop Script # ======================================== set -e @@ -28,10 +28,10 @@ SERVICE_DIR="$(dirname "$SCRIPT_DIR")" # Default log directory AUDIT_SERVER_LOG_DIR="${AUDIT_SERVER_LOG_DIR:-${SERVICE_DIR}/logs}" -PID_FILE="${AUDIT_SERVER_LOG_DIR}/ranger-audit-server.pid" +PID_FILE="${AUDIT_SERVER_LOG_DIR}/ranger-audit-ingestor.pid" echo "==========================================" -echo "Stopping Ranger Audit Server Service" +echo "Stopping Ranger Audit Ingestor Service" echo "==========================================" if [ ! -f "${PID_FILE}" ]; then @@ -69,4 +69,4 @@ if kill -0 "$PID" 2>/dev/null; then fi rm -f "${PID_FILE}" -echo "[INFO] ✓ Ranger Audit Server stopped successfully" +echo "[INFO] Ranger Audit Ingestor stopped successfully" diff --git a/audit-server/server/src/main/java/javax/ws/rs/core/NoContentException.java b/audit-server/audit-ingestor/src/main/java/javax/ws/rs/core/NoContentException.java similarity index 100% rename from audit-server/server/src/main/java/javax/ws/rs/core/NoContentException.java rename to audit-server/audit-ingestor/src/main/java/javax/ws/rs/core/NoContentException.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/AuditDestinationMgr.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/AuditDestinationMgr.java similarity index 83% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/AuditDestinationMgr.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/AuditDestinationMgr.java index e1d0dbd0b9..056b66ebd3 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/AuditDestinationMgr.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/AuditDestinationMgr.java @@ -26,7 +26,6 @@ import org.apache.ranger.audit.provider.MiscUtil; import org.apache.ranger.audit.server.AuditServerConfig; import org.apache.ranger.audit.server.AuditServerConstants; -import org.apache.ranger.audit.utils.AuditServerUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; @@ -41,12 +40,29 @@ public class AuditDestinationMgr { private static final Logger LOG = LoggerFactory.getLogger(AuditDestinationMgr.class); - public AuditHandler auditHandler; - public AuditServerUtils auditServerUtils; + private AuditHandler auditHandler; @PostConstruct - public void configure() { - init(); + public void init() { + LOG.info("==> AuditDestinationMgr.init()"); + + AuditServerConfig auditConfig = AuditServerConfig.getInstance(); + Properties properties = auditConfig.getProperties(); + String kafkaDestPrefix = AuditProviderFactory.AUDIT_DEST_BASE + "." + AuditServerConstants.DEFAULT_SERVICE_NAME; + boolean isAuditToKafkaDestinationEnabled = MiscUtil.getBooleanProperty(properties, kafkaDestPrefix, false); + + if (isAuditToKafkaDestinationEnabled) { + auditHandler = new AuditMessageQueue(); + + auditHandler.init(properties, kafkaDestPrefix); + auditHandler.start(); + + LOG.info("Kafka producer initialized and started"); + } else { + LOG.warn("Kafka audit destination is not enabled. Producer service will not function."); + } + + LOG.info("<== AuditDestinationMgr.init(): auditDestination: {} ", kafkaDestPrefix); } @PreDestroy @@ -94,33 +110,4 @@ public boolean logBatch(Collection events, String appI return ret; } - - private void init() { - LOG.info("==> AuditDestinationMgr.init()"); - - auditServerUtils = new AuditServerUtils(); - - AuditServerConfig auditConfig = AuditServerConfig.getInstance(); - Properties properties = auditConfig.getProperties(); - - if (properties != null) { - auditServerUtils.setAuditConfig(properties); - } - - String kafkaDestPrefix = AuditProviderFactory.AUDIT_DEST_BASE + "." + AuditServerConstants.DEFAULT_SERVICE_NAME; - boolean isAuditToKafkaDestinationEnabled = MiscUtil.getBooleanProperty(properties, kafkaDestPrefix, false); - - if (isAuditToKafkaDestinationEnabled) { - auditHandler = new AuditMessageQueue(); - - auditHandler.init(properties, kafkaDestPrefix); - auditHandler.start(); - - LOG.info("Kafka producer initialized and started"); - } else { - LOG.warn("Kafka audit destination is not enabled. Producer service will not function."); - } - - LOG.info("<== AuditDestinationMgr.init(): auditDestination: {} ", kafkaDestPrefix); - } } diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.java similarity index 92% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.java index cbc0df7fda..f494de5c88 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.java @@ -39,23 +39,22 @@ * It creates the necessary audit topics, producer threads and recovery threads. */ public class AuditMessageQueue extends AuditDestination { - public KafkaProducer kafkaProducer; - public AuditProducer auditProducerRunnable; - public AuditMessageQueueUtils auditMessageQueueUtils; - public String topicName; - - private Thread producerThread; - private AuditRecoveryManager recoveryManager; - private static final Logger LOG = LoggerFactory.getLogger(AuditMessageQueue.class); + private KafkaProducer kafkaProducer; + private AuditProducer auditProducerRunnable; + private AuditMessageQueueUtils auditMessageQueueUtils; + private String topicName; + private Thread producerThread; + private AuditRecoveryManager recoveryManager; + @Override public void init(Properties props, String propPrefix) { LOG.info("==> AuditMessageQueue.init() [CORE AUDIT SERVER]"); super.init(props, propPrefix); - auditMessageQueueUtils = new AuditMessageQueueUtils(props); + auditMessageQueueUtils = new AuditMessageQueueUtils(); createAuditsTopic(props, propPrefix); createKafkaProducer(props, propPrefix); @@ -135,18 +134,6 @@ public synchronized boolean log(final AuditEventBase event) { if (event instanceof AuthzAuditEvent) { AuthzAuditEvent authzEvent = (AuthzAuditEvent) event; - if (authzEvent.getAgentHostname() == null) { - authzEvent.setAgentHostname(MiscUtil.getHostname()); - } - - if (authzEvent.getLogType() == null) { - authzEvent.setLogType("RangerAudit"); - } - - if (authzEvent.getEventId() == null) { - authzEvent.setEventId(MiscUtil.generateUniqueId()); - } - /** Partition key is agentId (aka plugin ID). AuditPartitioner allocates configured plugins (hdfs, hiveServer2, etc.) to fixed partition sets of partitions. @@ -156,9 +143,6 @@ Partition key is agentId (aka plugin ID). AuditPartitioner allocates configured final String message = MiscUtil.stringify(event); try { - if (topicName == null || kafkaProducer == null) { - init(props, propPrefix); - } if (kafkaProducer != null) { MiscUtil.executePrivilegedAction((PrivilegedExceptionAction) () -> { AuditProducer.send(kafkaProducer, topicName, key, message); @@ -255,7 +239,7 @@ public synchronized boolean log(final Collection events, String }); successCount = authzEvents.size(); allSuccess = true; - LOG.debug("ranger-audit-server/ranger-audit-server-service/src/main/java/org/apache/ranger/audit/producer/kafka/AuditMessageQueue.javaSuccessfully sent batch of {} events to Kafka topic: {} with key: {}", successCount, topicName, finalBatchKey); + LOG.debug("Successfully sent batch of {} events to Kafka topic: {} with key: {}", successCount, topicName, finalBatchKey); } else { LOG.warn("Kafka producer not available, spooling batch of {} messages to recovery", authzEvents.size()); for (String message : messages) { @@ -294,6 +278,14 @@ public synchronized boolean log(final Collection events, String return allSuccess; } + public KafkaProducer getKafkaProducer() { + return kafkaProducer; + } + + private void setKafkaProducer(KafkaProducer kafkaProducer) { + this.kafkaProducer = kafkaProducer; + } + private void startRangerAuditRecoveryThread() { LOG.info("==> AuditMessageQueue.startRangerAuditRecoveryThread()"); @@ -335,7 +327,7 @@ private void createKafkaProducer(final Properties props, final String propPrefix try { auditProducerRunnable = new AuditProducer(props, propPrefix); if (auditProducerRunnable != null) { - kafkaProducer = auditProducerRunnable.getKafkaProducer(); + setKafkaProducer(auditProducerRunnable.getKafkaProducer()); } } catch (Exception e) { LOG.error("Error creating Kafka producer", e); diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditPartitioner.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditPartitioner.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditPartitioner.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditPartitioner.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditProducer.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditProducer.java similarity index 99% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditProducer.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditProducer.java index 25abbb9b27..1441e6fe51 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditProducer.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditProducer.java @@ -49,7 +49,7 @@ public class AuditProducer implements Runnable { public AuditProducer(Properties props, String propPrefix) throws Exception { LOG.debug("==> AuditProducer()"); - AuditMessageQueueUtils auditMessageQueueUtils = new AuditMessageQueueUtils(props); + AuditMessageQueueUtils auditMessageQueueUtils = new AuditMessageQueueUtils(); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_BOOTSTRAP_SERVERS)); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, Class.forName("org.apache.kafka.common.serialization.StringSerializer")); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, Class.forName("org.apache.kafka.common.serialization.StringSerializer")); diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryManager.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryManager.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryManager.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryManager.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryRetry.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryRetry.java similarity index 99% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryRetry.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryRetry.java index 35ee4bcbc9..7c6d192c2a 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryRetry.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryRetry.java @@ -252,7 +252,7 @@ private void processFailedFile(File file) { private boolean retrySendToKafka(String key, String message) { // Get current producer from parent - may have been recreated after Kafka came back online - KafkaProducer currentProducer = (messageQueue != null) ? messageQueue.kafkaProducer : null; + KafkaProducer currentProducer = (messageQueue != null) ? messageQueue.getKafkaProducer() : null; if (currentProducer == null) { LOG.warn("Kafka producer is null, cannot retry message - will retry on next cycle"); diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryWriter.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryWriter.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryWriter.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/producer/kafka/AuditRecoveryWriter.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/rest/AuditREST.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/rest/AuditREST.java similarity index 98% rename from audit-server/server/src/main/java/org/apache/ranger/audit/rest/AuditREST.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/rest/AuditREST.java index 378e847bd2..3a93fc1f96 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/rest/AuditREST.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/rest/AuditREST.java @@ -303,7 +303,7 @@ private String applyAuthToLocal(String principal) { } /** - * Rules are loaded from ranger.audit.service.auth.to.local property in ranger-audit-server-site.xml. + * Rules are loaded from ranger.audit.service.auth.to.local property in ranger-audit-ingestor-site.xml. */ private static void initializeAuthToLocal() { AuditServerConfig config = AuditServerConfig.getInstance(); @@ -317,7 +317,7 @@ private static void initializeAuthToLocal() { } } else { LOG.warn("No auth_to_local rules configured. Kerberos principal mapping may not work correctly."); - LOG.warn("Set property '{}' in ranger-audit-server-site.xml", AuditServerConstants.PROP_AUTH_TO_LOCAL); + LOG.warn("Set property '{}' in ranger-audit-ingestor-site.xml", AuditServerConstants.PROP_AUTH_TO_LOCAL); } } diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/rest/RangerJsonProvider.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/rest/RangerJsonProvider.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/rest/RangerJsonProvider.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/rest/RangerJsonProvider.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditAuthEntryPoint.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditAuthEntryPoint.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditAuthEntryPoint.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditAuthEntryPoint.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditDelegationTokenFilter.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditDelegationTokenFilter.java similarity index 98% rename from audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditDelegationTokenFilter.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditDelegationTokenFilter.java index db8b8c28d9..daa9ceb551 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditDelegationTokenFilter.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditDelegationTokenFilter.java @@ -63,10 +63,8 @@ public class AuditDelegationTokenFilter extends DelegationTokenAuthenticationFil private String tokenKindStr; private static final String AUDIT_DELEGATION_TOKEN_KIND_DEFAULT = "AUDIT_DELEGATION_TOKEN"; - - // Uses "ranger.audit" prefix for KerberosAuthenticationHandler - private static final String DEFAULT_CONFIG_PREFIX = "ranger.audit"; - private static final String PROP_CONFIG_PREFIX = "config.prefix"; + private static final String DEFAULT_CONFIG_PREFIX = "ranger.audit.ingestor"; + private static final String PROP_CONFIG_PREFIX = "config.prefix"; /** * Initialized this filter. Only PROP_CONFIG_PREFIX needs to be set. @@ -185,7 +183,7 @@ protected Properties getConfiguration(String configPrefix, FilterConfig filterCo props.setProperty(DelegationTokenAuthenticationHandler.TOKEN_KIND, tokenKindStr); // Resolve _HOST into bind address - String bindAddress = auditConfig.get("ranger.audit.server.bind.address"); + String bindAddress = auditConfig.get("ranger.audit.ingestor.bind.address"); LOG.debug("Bind address from config: [{}]", bindAddress); if (Objects.isNull(bindAddress)) { diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditJwtAuthFilter.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditJwtAuthFilter.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/security/AuditJwtAuthFilter.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/AuditJwtAuthFilter.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/security/FilterChainWrapper.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/FilterChainWrapper.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/security/FilterChainWrapper.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/FilterChainWrapper.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/security/NullServletContext.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/NullServletContext.java similarity index 100% rename from audit-server/server/src/main/java/org/apache/ranger/audit/security/NullServletContext.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/security/NullServletContext.java diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/server/AuditServerApplication.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/AuditServerApplication.java similarity index 93% rename from audit-server/server/src/main/java/org/apache/ranger/audit/server/AuditServerApplication.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/AuditServerApplication.java index 15fc7a30d9..4c2b5d1114 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/server/AuditServerApplication.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/AuditServerApplication.java @@ -33,8 +33,8 @@ public class AuditServerApplication { private static final Logger LOG = LoggerFactory.getLogger(AuditServerApplication.class); - private static final String APP_NAME = "ranger-audit-server"; - private static final String CONFIG_PREFIX = "ranger.audit.server."; + private static final String APP_NAME = "audit-ingestor"; + private static final String CONFIG_PREFIX = "ranger.audit.ingestor."; private AuditServerApplication() { } @@ -50,6 +50,7 @@ public static void main(String[] args) { LOG.info("Configuration loaded successfully"); EmbeddedServer server = new EmbeddedServer(config, APP_NAME, CONFIG_PREFIX); + server.start(); LOG.info("==> Ranger Audit Server Service Started Successfully"); diff --git a/audit-server/server/src/main/java/org/apache/ranger/audit/server/AuditServerConfig.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/AuditServerConfig.java similarity index 88% rename from audit-server/server/src/main/java/org/apache/ranger/audit/server/AuditServerConfig.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/AuditServerConfig.java index afc7bbb844..185ca10000 100644 --- a/audit-server/server/src/main/java/org/apache/ranger/audit/server/AuditServerConfig.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/AuditServerConfig.java @@ -27,8 +27,10 @@ * Loads audit server configuration files. */ public class AuditServerConfig extends AuditConfig { - private static final Logger LOG = LoggerFactory.getLogger(AuditServerConfig.class); - private static final String CONFIG_FILE_PATH = "conf/ranger-audit-server-site.xml"; + private static final Logger LOG = LoggerFactory.getLogger(AuditServerConfig.class); + + private static final String CONFIG_FILE_PATH = "conf/ranger-audit-ingestor-site.xml"; + private static volatile AuditServerConfig sInstance; private AuditServerConfig() { @@ -58,7 +60,7 @@ private boolean addAuditServerResources() { boolean ret = true; - // Load ranger-audit-server-site.xml + // Load ranger-audit-ingestor-site.xml if (!addAuditResource(CONFIG_FILE_PATH, true)) { LOG.error("Could not load required configuration: {}", CONFIG_FILE_PATH); ret = false; diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/server/EmbeddedServer.java b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/EmbeddedServer.java similarity index 99% rename from audit-server/common/src/main/java/org/apache/ranger/audit/server/EmbeddedServer.java rename to audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/EmbeddedServer.java index bbedc66ef7..2bda7e2190 100644 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/server/EmbeddedServer.java +++ b/audit-server/audit-ingestor/src/main/java/org/apache/ranger/audit/server/EmbeddedServer.java @@ -66,7 +66,7 @@ import java.util.Map; public class EmbeddedServer { - private static final Logger LOG = LoggerFactory.getLogger(EmbeddedServer.class); + private static final Logger LOG = LoggerFactory.getLogger(EmbeddedServer.class); private static final String ACCESS_LOG_PREFIX = "accesslog.prefix"; private static final String ACCESS_LOG_DATE_FORMAT = "accesslog.dateformat"; @@ -91,27 +91,28 @@ public class EmbeddedServer { private final Configuration configuration; private final String appName; private final String configPrefix; - private volatile Tomcat server; - private volatile Context webappContext; + + private volatile Tomcat server; + private volatile Context webappContext; + + public static void main(String[] args) { + Configuration config = new AuditConfig(); + String appName = AuditServerConstants.AUDIT_SERVER_APPNAME; + String configPrefix = AuditServerConstants.PROP_PREFIX_AUDIT_SERVER; + + new EmbeddedServer(config, appName, configPrefix).start(); + } public EmbeddedServer(Configuration configuration, String appName, String configPrefix) { LOG.info("==> EmbeddedServer(appName={}, configPrefix={})", appName, configPrefix); - this.configuration = new Configuration(configuration); + this.configuration = configuration; this.appName = appName; this.configPrefix = configPrefix; LOG.info("<== EmbeddedServer(appName={}, configPrefix={}", appName, configPrefix); } - public static void main(String[] args) { - Configuration config = AuditConfig.getInstance(); - String appName = AuditServerConstants.AUDIT_SERVER_APPNAME; - String configPrefix = AuditServerConstants.PROP_PREFIX_AUDIT_SERVER; - - new EmbeddedServer(config, appName, configPrefix).start(); - } - @SuppressWarnings("deprecation") public void start() { LOG.info("==> EmbeddedServer.start(appName={})", appName); diff --git a/audit-server/server/src/main/resources/conf/logback.xml b/audit-server/audit-ingestor/src/main/resources/conf/logback.xml similarity index 96% rename from audit-server/server/src/main/resources/conf/logback.xml rename to audit-server/audit-ingestor/src/main/resources/conf/logback.xml index 4f1aa0500a..29d60f307e 100644 --- a/audit-server/server/src/main/resources/conf/logback.xml +++ b/audit-server/audit-ingestor/src/main/resources/conf/logback.xml @@ -15,7 +15,7 @@ - + diff --git a/audit-server/server/src/main/resources/conf/ranger-audit-server-site.xml b/audit-server/audit-ingestor/src/main/resources/conf/ranger-audit-ingestor-site.xml similarity index 88% rename from audit-server/server/src/main/resources/conf/ranger-audit-server-site.xml rename to audit-server/audit-ingestor/src/main/resources/conf/ranger-audit-ingestor-site.xml index 6cd462d78c..98155d3293 100644 --- a/audit-server/server/src/main/resources/conf/ranger-audit-server-site.xml +++ b/audit-server/audit-ingestor/src/main/resources/conf/ranger-audit-ingestor-site.xml @@ -12,7 +12,7 @@ limitations under the License. See accompanying LICENSE file. --> - + log.dir @@ -21,73 +21,73 @@ - ranger.audit.server.webapp.dir - webapp/ranger-audit-server + ranger.audit.ingestor.webapp.dir + webapp/audit-ingestor Path to the extracted webapp directory (relative to AUDIT_SERVER_HOME_DIR). Must be extracted directory, not WAR file, because Jersey cannot scan packages in unexpanded WAR files. - ranger.audit.server.contextName + ranger.audit.ingestor.contextName / - ranger.audit.server.host - ranger-audit-server.rangernw + ranger.audit.ingestor.host + ranger-audit-ingestor.rangernw - - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In Docker: Use full service name with domain (e.g., ranger-audit-ingestor.rangernw) - In VM: Use the actual FQDN (e.g., ranger.example.com) - ranger.audit.server.http.port + ranger.audit.ingestor.http.port 7081 - ranger.audit.server.https.port + ranger.audit.ingestor.https.port 7182 - ranger.audit.server.http.enabled + ranger.audit.ingestor.http.enabled true - ranger.audit.server.enableTLS + ranger.audit.ingestor.enableTLS false - ranger.audit.server.https.attrib.ssl.enabled + ranger.audit.ingestor.https.attrib.ssl.enabled false - ranger.audit.server.https.attrib.keystore.keyalias + ranger.audit.ingestor.https.attrib.keystore.keyalias myKey - ranger.audit.server.https.attrib.keystore.pass + ranger.audit.ingestor.https.attrib.keystore.pass _ - ranger.audit.server.https.attrib.keystore.file - /etc/ranger/ranger-audit-server/keys/server.jks + ranger.audit.ingestor.https.attrib.keystore.file + /etc/ranger/ranger-audit-ingestor/keys/server.jks - ranger.audit.server.https.attrib.keystore.credential.alias + ranger.audit.ingestor.https.attrib.keystore.credential.alias keyStoreCredentialAlias - ranger.audit.server.tomcat.ciphers + ranger.audit.ingestor.tomcat.ciphers @@ -102,19 +102,19 @@ - ranger.audit.kerberos.enabled + ranger.audit.ingestor.kerberos.enabled true Enable Kerberos authentication for incoming REST API requests - ranger.audit.kerberos.type + ranger.audit.ingestor.kerberos.type kerberos Authentication type: kerberos or simple - ranger.audit.kerberos.principal + ranger.audit.ingestor.kerberos.principal HTTP/_HOST@EXAMPLE.COM INCOMING: SPNEGO HTTP principal for authenticating REST API requests FROM plugins. @@ -123,7 +123,7 @@ - ranger.audit.kerberos.keytab + ranger.audit.ingestor.kerberos.keytab /etc/keytabs/HTTP.keytab Keytab file containing HTTP principal for SPNEGO authentication @@ -131,26 +131,26 @@ - ranger.audit.kerberos.name.rules + ranger.audit.ingestor.kerberos.name.rules DEFAULT Kerberos name rules for principal mapping - ranger.audit.server.bind.address - ranger-audit-server.rangernw + ranger.audit.ingestor.bind.address + ranger-audit-ingestor.rangernw Hostname for Kerberos principal _HOST resolution in HTTP principal - Example: If set to "myhost.example.com" and principal is "HTTP/_HOST@REALM", it becomes "HTTP/myhost.example.com@REALM" - - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In Docker: Use full service name with domain (e.g., ranger-audit-ingestor.rangernw) - In VM: Use the actual FQDN (e.g., ranger.example.com) - MUST match the hostname in the keytab principal - ranger.audit.kerberos.cookie.path + ranger.audit.ingestor.kerberos.cookie.path / Cookie path for delegation token @@ -193,26 +193,26 @@ - ranger.audit.server.service.dev_hdfs.allowed.users + ranger.audit.ingestor.service.dev_hdfs.allowed.users hdfs Comma-separated list of allowed users that can send audits for dev_hdfs service - ranger.audit.server.service.dev_hive.allowed.users + ranger.audit.ingestor.service.dev_hive.allowed.users hive Comma-separated list of allowed users that can send audits for dev_hive service. - ranger.audit.server.service.dev_hbase.allowed.users + ranger.audit.ingestor.service.dev_hbase.allowed.users hbase Comma-separated list of allowed users that can send audits for dev_hbase service. - ranger.audit.server.auth.to.local + ranger.audit.ingestor.auth.to.local RULE:[2:$1/$2@$0]([ndj]n/.*@.*|hdfs/.*@.*)s/.*/hdfs/ RULE:[2:$1/$2@$0]([rn]m/.*@.*|yarn/.*@.*)s/.*/yarn/ @@ -238,15 +238,15 @@ - ranger.audit.server.authentication.method + ranger.audit.ingestor.authentication.method KERBEROS xasecure.audit.destination.kafka.host - ranger-audit-server.rangernw + ranger-audit-ingestor.rangernw - - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In Docker: Use full service name with domain (e.g., ranger-audit-ingestor.rangernw) - In VM: Use the actual FQDN (e.g., ranger.example.com) @@ -416,7 +416,7 @@ ranger.audit.kafka.recovery.spool.dir - /var/log/ranger/ranger-audit-server/audit/spool + /var/log/ranger/ranger-audit-ingestor/audit/spool Directory for storing .failed files with audit messages that failed to send to Kafka. Files remain here until successfully processed, then moved to archive as .processed files. @@ -425,7 +425,7 @@ ranger.audit.kafka.recovery.archive.dir - /var/log/ranger/ranger-audit-server/audit/archive + /var/log/ranger/ranger-audit-ingestor/audit/archive Directory for successfully processed files. Files are moved here with .processed extension after all messages are successfully retried to Kafka. Old .processed files are automatically diff --git a/audit-server/server/src/main/webapp/WEB-INF/applicationContext.xml b/audit-server/audit-ingestor/src/main/webapp/WEB-INF/applicationContext.xml similarity index 100% rename from audit-server/server/src/main/webapp/WEB-INF/applicationContext.xml rename to audit-server/audit-ingestor/src/main/webapp/WEB-INF/applicationContext.xml diff --git a/audit-server/server/src/main/webapp/WEB-INF/security-applicationContext.xml b/audit-server/audit-ingestor/src/main/webapp/WEB-INF/security-applicationContext.xml similarity index 100% rename from audit-server/server/src/main/webapp/WEB-INF/security-applicationContext.xml rename to audit-server/audit-ingestor/src/main/webapp/WEB-INF/security-applicationContext.xml diff --git a/audit-server/server/src/main/webapp/WEB-INF/web.xml b/audit-server/audit-ingestor/src/main/webapp/WEB-INF/web.xml similarity index 100% rename from audit-server/server/src/main/webapp/WEB-INF/web.xml rename to audit-server/audit-ingestor/src/main/webapp/WEB-INF/web.xml diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerBase.java b/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerBase.java deleted file mode 100644 index 8d46dd9785..0000000000 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerBase.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.ranger.audit.consumer.kafka; - -import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.ranger.audit.provider.MiscUtil; -import org.apache.ranger.audit.server.AuditServerConstants; -import org.apache.ranger.audit.utils.AuditMessageQueueUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Properties; - -public abstract class AuditConsumerBase implements AuditConsumer { - private static final Logger LOG = LoggerFactory.getLogger(AuditConsumerBase.class); - - protected final Properties consumerProps = new Properties(); - protected final KafkaConsumer consumer; - protected final String topicName; - protected final String consumerGroupId; - - public AuditConsumerBase(Properties props, String propPrefix, String consumerGroupId) throws Exception { - AuditMessageQueueUtils auditMessageQueueUtils = new AuditMessageQueueUtils(props); - - this.consumerGroupId = getConsumerGroupId(props, propPrefix, consumerGroupId); - - consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_BOOTSTRAP_SERVERS)); - consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, this.consumerGroupId); - consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Class.forName("org.apache.kafka.common.serialization.StringDeserializer")); - consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Class.forName("org.apache.kafka.common.serialization.StringDeserializer")); - - String securityProtocol = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SECURITY_PROTOCOL, AuditServerConstants.DEFAULT_SECURITY_PROTOCOL); - consumerProps.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, securityProtocol); - - consumerProps.put(AuditServerConstants.PROP_SASL_MECHANISM, MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_SASL_MECHANISM, AuditServerConstants.DEFAULT_SASL_MECHANISM)); - consumerProps.put(AuditServerConstants.PROP_SASL_KERBEROS_SERVICE_NAME, AuditServerConstants.DEFAULT_SERVICE_NAME); - - if (securityProtocol.toUpperCase().contains(AuditServerConstants.PROP_SECURITY_PROTOCOL_VALUE)) { - consumerProps.put(AuditServerConstants.PROP_SASL_JAAS_CONFIG, auditMessageQueueUtils.getJAASConfig(props, propPrefix)); - } - - consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_MAX_POLL_RECORDS, AuditServerConstants.DEFAULT_MAX_POLL_RECORDS)); - consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - - // Configure re-balancing parameters for subscribe mode - // These ensure stable consumer group behavior during horizontal scaling - int sessionTimeoutMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_SESSION_TIMEOUT_MS, AuditServerConstants.DEFAULT_SESSION_TIMEOUT_MS); - int maxPollIntervalMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_MAX_POLL_INTERVAL_MS, AuditServerConstants.DEFAULT_MAX_POLL_INTERVAL_MS); - int heartbeatIntervalMs = MiscUtil.getIntProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_HEARTBEAT_INTERVAL_MS, AuditServerConstants.DEFAULT_HEARTBEAT_INTERVAL_MS); - - consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, sessionTimeoutMs); - consumerProps.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, maxPollIntervalMs); - consumerProps.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, heartbeatIntervalMs); - - // Configure partition assignment strategy - String partitionAssignmentStrategy = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_PARTITION_ASSIGNMENT_STRATEGY, AuditServerConstants.DEFAULT_PARTITION_ASSIGNMENT_STRATEGY); - consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, partitionAssignmentStrategy); - - LOG.info("Consumer '{}' configured for subscription-based partition assignment with re-balancing support", this.consumerGroupId); - LOG.info("Re-balancing config - session.timeout.ms: {}, max.poll.interval.ms: {}, heartbeat.interval.ms: {}", sessionTimeoutMs, maxPollIntervalMs, heartbeatIntervalMs); - LOG.info("Partition assignment strategy: {}", partitionAssignmentStrategy); - - consumer = new KafkaConsumer<>(consumerProps); - - topicName = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_TOPIC_NAME, AuditServerConstants.DEFAULT_TOPIC); - } - - @Override - public KafkaConsumer getConsumer() { - return consumer; - } - - @Override - public String getTopicName() { - return topicName; - } - - public String getConsumerGroupId() { - return consumerGroupId; - } - - private String getConsumerGroupId(Properties props, String propPrefix, String defaultConsumerGroupId) { - String configuredGroupId = MiscUtil.getStringProperty(props, propPrefix + ".consumer.group.id"); - if (configuredGroupId != null && !configuredGroupId.trim().isEmpty()) { - return configuredGroupId.trim(); - } - return defaultConsumerGroupId; - } -} diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerRegistry.java b/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerRegistry.java deleted file mode 100644 index d1cf560741..0000000000 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/consumer/kafka/AuditConsumerRegistry.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.ranger.audit.consumer.kafka; - -import org.apache.ranger.audit.provider.AuditProviderFactory; -import org.apache.ranger.audit.provider.MiscUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Registry for managing audit consumer factories and instances. - * Supports dynamic consumer registration and creation based on configuration. - */ -public class AuditConsumerRegistry { - private static final Logger LOG = LoggerFactory.getLogger(AuditConsumerRegistry.class); - - private static final AuditConsumerRegistry auditConsumerRegistry = new AuditConsumerRegistry(); - private final Map consumerFactories = new ConcurrentHashMap<>(); - private final Map activeConsumers = new ConcurrentHashMap<>(); - - private AuditConsumerRegistry() { - LOG.debug("AuditConsumerRegistry instance created"); - } - - public static AuditConsumerRegistry getInstance() { - return auditConsumerRegistry; - } - - /** - * Register a consumer factory for a specific destination type. - * This method can be called early in the application lifecycle (static initializers) - * to register consumers before the AuditMessageQueue is initialized. - * - * @param destinationType The destination type identifier (e.g., "solr", "hdfs", "elasticsearch") - * @param factory The factory that creates consumer instances - */ - public void registerFactory(String destinationType, AuditConsumerFactory factory) { - if (destinationType == null || destinationType.trim().isEmpty()) { - LOG.warn("Attempted to register factory with null or empty destination type"); - return; - } - - if (factory == null) { - LOG.warn("Attempted to register null factory for destination type: {}", destinationType); - return; - } - - AuditConsumerFactory existing = consumerFactories.put(destinationType, factory); - if (existing != null) { - LOG.warn("Replaced existing factory for destination type: {}", destinationType); - } else { - LOG.info("Registered consumer factory for destination type: {}", destinationType); - } - } - - /** - * Create consumer instances for all enabled destinations based on configuration. - * - * @param props Configuration properties - * @param propPrefix Property prefix for Kafka configuration - * @return List of created consumer instances - */ - public List createConsumers(Properties props, String propPrefix) { - LOG.info("==> AuditConsumerRegistry.createConsumers()"); - - List consumers = new ArrayList<>(); - - for (Map.Entry entry : consumerFactories.entrySet()) { - String destinationType = entry.getKey(); - AuditConsumerFactory factory = entry.getValue(); - String destPropPrefix = AuditProviderFactory.AUDIT_DEST_BASE + "." + destinationType; - boolean isEnabled = MiscUtil.getBooleanProperty(props, destPropPrefix, false); - - if (isEnabled) { - try { - LOG.info("Creating consumer for enabled destination: {}", destinationType); - AuditConsumer consumer = factory.createConsumer(props, propPrefix); - - if (consumer != null) { - consumers.add(consumer); - activeConsumers.put(destinationType, consumer); - LOG.info("Successfully created consumer for destination: {}", destinationType); - } else { - LOG.warn("Factory returned null consumer for destination: {}", destinationType); - } - } catch (Exception e) { - LOG.error("Failed to create consumer for destination: {}", destinationType, e); - } - } else { - LOG.debug("Destination '{}' is disabled (property: {} = false)", destinationType, destPropPrefix); - } - } - - LOG.info("<== AuditConsumerRegistry.createConsumers(): Created {} consumers out of {} registered factories", consumers.size(), consumerFactories.size()); - - return consumers; - } - - public Collection getActiveConsumers() { - return activeConsumers.values(); - } - - public Collection getRegisteredDestinationTypes() { - return consumerFactories.keySet(); - } - - /** - * Clear all active consumer references. - * Called during shutdown after consumers have been stopped. - */ - public void clearActiveConsumers() { - LOG.debug("Clearing {} active consumer references", activeConsumers.size()); - activeConsumers.clear(); - } - - public int getFactoryCount() { - return consumerFactories.size(); - } -} diff --git a/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditServerUtils.java b/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditServerUtils.java deleted file mode 100644 index 1536134a30..0000000000 --- a/audit-server/common/src/main/java/org/apache/ranger/audit/utils/AuditServerUtils.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * 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.ranger.audit.utils; - -import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.DescribeTopicsResult; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; -import org.apache.ranger.audit.model.AuthzAuditEvent; -import org.apache.ranger.audit.provider.AuditHandler; -import org.apache.ranger.audit.provider.AuditProviderFactory; -import org.apache.ranger.authorization.utils.JsonUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.ranger.audit.provider.MiscUtil.TOKEN_APP_TYPE; -import static org.apache.ranger.audit.provider.MiscUtil.TOKEN_END; -import static org.apache.ranger.audit.provider.MiscUtil.TOKEN_START; -import static org.apache.ranger.audit.provider.MiscUtil.toArray; - -public class AuditServerUtils { - private static final Logger LOG = LoggerFactory.getLogger(AuditServerUtils.class); - - private Properties auditConfig; - private ConcurrentHashMap> auditProviderMap = new ConcurrentHashMap<>(); - - public Properties getAuditConfig() { - if (auditConfig == null) { - return null; - } - Properties copy = new Properties(); - copy.putAll(auditConfig); - return copy; - } - - public void setAuditConfig(Properties auditConfig) { - if (auditConfig != null) { - this.auditConfig = new Properties(); - this.auditConfig.putAll(auditConfig); - } else { - this.auditConfig = null; - } - } - - public AuditHandler getAuditProvider(AuthzAuditEvent auditEvent) throws Exception { - AuditHandler ret = null; - - AuditProviderFactory auditProviderFactory = getAuditProvideFactory(auditEvent); - if (auditProviderFactory != null) { - ret = auditProviderFactory.getAuditProvider(); - } - - return ret; - } - - private AuditProviderFactory getAuditProvideFactory(AuthzAuditEvent auditEvent) throws Exception { - LOG.debug("==> AuditServerUtils.getAuditProviderFactory()"); - - AuditProviderFactory ret = null; - String hostName = auditEvent.getAgentHostname(); - String appId = auditEvent.getAgentId(); - String serviceType = getServiceType(auditEvent); - - Map auditProviderFactoryMap = auditProviderMap.get(hostName); - if (MapUtils.isNotEmpty(auditProviderFactoryMap)) { - ret = auditProviderFactoryMap.get(appId); - } - - if (ret == null) { - ret = createAndCacheAuditProvider(hostName, serviceType, appId); - if (MapUtils.isEmpty(auditProviderFactoryMap)) { - auditProviderFactoryMap = new HashMap<>(); - } - auditProviderFactoryMap.put(appId, ret); - auditProviderMap.put(hostName, auditProviderFactoryMap); - } - - if (LOG.isDebugEnabled()) { - logAuditProviderCreated(); - LOG.debug("<== AuditServerUtils.getAuditProviderFactory(): {}", ret); - } - - return ret; - } - - private AuditProviderFactory createAndCacheAuditProvider(String hostName, String serviceType, String appId) throws Exception { - LOG.debug("==> AuditServerUtils.createAndCacheAuditProvider(hostname: {}, serviceType: {}, appId: {})", hostName, serviceType, appId); - - AuditProviderFactory ret = initAuditProvider(serviceType, appId); - if (!ret.isInitDone()) { - ret = initAuditProvider(serviceType, appId); - } - if (!ret.isInitDone()) { - String msg = String.format("AuditHandler for appId={%s}, hostname={%s} not initialized correctly. Please check audit configuration...", appId, hostName); - LOG.error(msg); - throw new Exception(msg); - } - - LOG.debug("<== AuditServerUtils.createAndCacheAuditProvider(hostname: {}, serviceType: {}, appId: {})", hostName, serviceType, appId); - return ret; - } - - private AuditProviderFactory initAuditProvider(String serviceType, String appId) { - Properties properties = getConfigurationForAuditService(serviceType); - AuditProviderFactory ret = AuditProviderFactory.getInstance(); - ret.init(properties, appId); - return ret; - } - - private Properties getConfigurationForAuditService(String serviceType) { - Properties ret = getAuditConfig(); - setCloudStorageLocationProperty(ret, serviceType); - setSpoolFolderForDestination(ret, serviceType); - return ret; - } - - private String getServiceType(AuthzAuditEvent auditEvent) { - String ret = null; - String additionalInfo = auditEvent.getAdditionalInfo(); - Map addInfoMap = JsonUtils.jsonToMapStringString(additionalInfo); - if (MapUtils.isNotEmpty(addInfoMap)) { - ret = addInfoMap.get("serviceType"); - } - return ret; - } - - private void setCloudStorageLocationProperty(Properties prop, String serviceType) { - String hdfsDir = prop.getProperty("xasecure.audit.destination.hdfs.dir"); - if (StringUtils.isNotBlank(hdfsDir)) { - StringBuilder sb = new StringBuilder(hdfsDir).append("/").append(serviceType); - setProperty(prop, "xasecure.audit.destination.hdfs.dir", sb.toString()); - } - } - - private void setSpoolFolderForDestination(Properties ret, String serviceType) { - ArrayList enabledDestinations = getAuditDestinationList(ret); - for (String dest : enabledDestinations) { - String spoolDirProp = new StringBuilder("xasecure.audit.destination.").append(dest).append(".batch.filespool.dir").toString(); - String spoolDir = ret.getProperty(spoolDirProp); - if (StringUtils.isNotBlank(spoolDir)) { - spoolDir = replaceToken(spoolDir, serviceType); - setProperty(ret, spoolDirProp, spoolDir); - } - } - } - - private void setProperty(Properties properties, String key, String val) { - properties.setProperty(key, val); - } - - private ArrayList getAuditDestinationList(Properties props) { - ArrayList ret = new ArrayList<>(); - - for (Object propNameObj : props.keySet()) { - String propName = propNameObj.toString(); - if (!propName.startsWith(AuditProviderFactory.AUDIT_DEST_BASE)) { - continue; - } - String destName = propName.substring(AuditProviderFactory.AUDIT_DEST_BASE.length() + 1); - List splits = toArray(destName, "."); - if (splits.size() > 1) { - continue; - } - String value = props.getProperty(propName); - if ("enable".equalsIgnoreCase(value) || "enabled".equalsIgnoreCase(value) || "true".equalsIgnoreCase(value)) { - ret.add(destName); - LOG.info("Audit destination {} is set to {}", propName, value); - } - } - return ret; - } - - private String replaceToken(String str, String appType) { - String ret = str; - for (int startPos = 0; startPos < str.length(); ) { - int tagStartPos = str.indexOf(TOKEN_START, startPos); - - if (tagStartPos == -1) { - break; - } - - int tagEndPos = str.indexOf(TOKEN_END, tagStartPos + TOKEN_START.length()); - - if (tagEndPos == -1) { - break; - } - - String tag = str.substring(tagStartPos, tagEndPos + TOKEN_END.length()); - String token = tag.substring(TOKEN_START.length(), tag.lastIndexOf(TOKEN_END)); - String val = ""; - if (token != null) { - if (token.equals(TOKEN_APP_TYPE)) { - val = appType; - } - } - ret = str.substring(0, tagStartPos) + val + str.substring(tagEndPos + TOKEN_END.length()); - startPos = tagStartPos + val.length(); - } - - return ret; - } - - private void logAuditProviderCreated() { - if (MapUtils.isNotEmpty(auditProviderMap)) { - for (Map.Entry> entry : auditProviderMap.entrySet()) { - String hostname = entry.getKey(); - Map providerFactoryMap = entry.getValue(); - if (MapUtils.isNotEmpty(providerFactoryMap)) { - for (Map.Entry ap : providerFactoryMap.entrySet()) { - String serviceAppId = ap.getKey(); - AuditProviderFactory auditProviderFactoryVal = ap.getValue(); - String apVal = auditProviderFactoryVal.getAuditProvider().getName(); - LOG.debug("AuditProvider created for HostName: {} ServiceAppId: {} AuditProvider: {}", hostname, serviceAppId, apVal); - } - } - } - } - } - - public boolean waitUntilTopicReady(Admin admin, String topic, Duration totalWait) throws Exception { - long endTime = System.nanoTime() + totalWait.toNanos(); - long baseSleepMs = 100L; - long maxSleepMs = 2000L; - - while (System.nanoTime() < endTime) { - try { - DescribeTopicsResult describeTopicsResult = admin.describeTopics(Collections.singleton(topic)); - TopicDescription topicDescription = describeTopicsResult.values().get(topic).get(); - boolean allHaveLeader = topicDescription.partitions().stream().allMatch(partitionInfo -> partitionInfo.leader() != null); - boolean allHaveISR = topicDescription.partitions().stream().allMatch(partitionInfo -> !partitionInfo.isr().isEmpty()); - if (allHaveLeader && allHaveISR) { - return true; - } - } catch (Exception e) { - // If topic hasn't propagated yet, you'll see UnknownTopicOrPartitionException - // continue to wait for topic availability - if (!(rootCause(e) instanceof UnknownTopicOrPartitionException)) { - throw e; - } - } - - // Sleep until the created topic is available for metadata fetch - baseSleepMs = Math.min(maxSleepMs, baseSleepMs * 2); - long sleep = baseSleepMs + ThreadLocalRandom.current().nextLong(0, baseSleepMs / 2 + 1); - Thread.sleep(sleep); - } - return false; - } - - private static Throwable rootCause(Throwable t) { - Throwable throwable = t; - while (throwable.getCause() != null) { - throwable = throwable.getCause(); - } - return throwable; - } -} diff --git a/audit-server/consumer-hdfs/scripts/start-consumer-hdfs.sh b/audit-server/consumer-hdfs/scripts/start-consumer-hdfs.sh deleted file mode 100755 index a86f7a67dc..0000000000 --- a/audit-server/consumer-hdfs/scripts/start-consumer-hdfs.sh +++ /dev/null @@ -1,164 +0,0 @@ -#!/bin/bash - -# 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. - -# ======================================== -# Ranger Audit Consumer HDFS Start Script -# ======================================== -# This script starts the HDFS consumer service -# The service consumes audit events from Kafka and writes them to HDFS/S3/Azure - -set -e - -# Script directory -SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" -SERVICE_DIR="$(dirname "$SCRIPT_DIR")" - -# Default directories - can be overridden by environment variables -AUDIT_CONSUMER_HOME_DIR="${AUDIT_CONSUMER_HOME_DIR:-${SERVICE_DIR}/target}" -AUDIT_CONSUMER_CONF_DIR="${AUDIT_CONSUMER_CONF_DIR:-${SERVICE_DIR}/src/main/resources/conf}" -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-${SERVICE_DIR}/logs}" - -# Create log directory if it doesn't exist -mkdir -p "${AUDIT_CONSUMER_LOG_DIR}" - -echo "==========================================" -echo "Starting Ranger Audit Consumer - HDFS" -echo "==========================================" -echo "Service Directory: ${SERVICE_DIR}" -echo "Home Directory: ${AUDIT_CONSUMER_HOME_DIR}" -echo "Config Directory: ${AUDIT_CONSUMER_CONF_DIR}" -echo "Log Directory: ${AUDIT_CONSUMER_LOG_DIR}" -echo "==========================================" - -# Check if Java is available -if [ -z "$JAVA_HOME" ]; then - JAVA_CMD=$(which java 2>/dev/null || true) - if [ -z "$JAVA_CMD" ]; then - echo "[ERROR] JAVA_HOME is not set and java is not in PATH" - exit 1 - fi - JAVA_HOME=$(dirname $(dirname $(readlink -f "$JAVA_CMD"))) - echo "[INFO] JAVA_HOME not set, detected: ${JAVA_HOME}" -fi - -export JAVA_HOME -export PATH=$JAVA_HOME/bin:$PATH - -echo "[INFO] Java version:" -java -version - -# Set heap size (default: 512MB to 2GB) -AUDIT_CONSUMER_HEAP="${AUDIT_CONSUMER_HEAP:--Xms512m -Xmx2g}" - -# Set JVM options -if [ -z "$AUDIT_CONSUMER_OPTS" ]; then - AUDIT_CONSUMER_OPTS="-Dlogback.configurationFile=${AUDIT_CONSUMER_CONF_DIR}/logback.xml" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.hdfs.log.dir=${AUDIT_CONSUMER_LOG_DIR}" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.hdfs.log.file=ranger-audit-consumer-hdfs.log" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.net.preferIPv4Stack=true -server" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:+UseG1GC -XX:MaxGCPauseMillis=200" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:InitiatingHeapOccupancyPercent=35" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:ConcGCThreads=4 -XX:ParallelGCThreads=8" -fi - -# Add Kerberos configuration if needed -if [ "${KERBEROS_ENABLED}" == "true" ]; then - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.security.krb5.conf=/etc/krb5.conf" - echo "[INFO] Kerberos is enabled" -fi - -export AUDIT_CONSUMER_OPTS -export AUDIT_CONSUMER_LOG_DIR - -echo "[INFO] JAVA_HOME: ${JAVA_HOME}" -echo "[INFO] HEAP: ${AUDIT_CONSUMER_HEAP}" -echo "[INFO] JVM_OPTS: ${AUDIT_CONSUMER_OPTS}" - -# Find the WAR file -WAR_FILE=$(find "${AUDIT_CONSUMER_HOME_DIR}" -name "ranger-audit-consumer-hdfs*.war" | head -1) - -if [ -z "$WAR_FILE" ] || [ ! -f "$WAR_FILE" ]; then - echo "[ERROR] WAR file not found in ${AUDIT_CONSUMER_HOME_DIR}" - echo "[ERROR] Please build the project first using: mvn clean package" - exit 1 -fi - -echo "[INFO] Using WAR file: ${WAR_FILE}" - -# Extract WAR if not already extracted -WEBAPP_DIR="${AUDIT_CONSUMER_HOME_DIR}/webapp/ranger-audit-consumer-hdfs" -if [ ! -d "${WEBAPP_DIR}" ]; then - echo "[INFO] Extracting WAR file..." - mkdir -p "${WEBAPP_DIR}" - cd "${WEBAPP_DIR}" - jar xf "${WAR_FILE}" - cd - > /dev/null -fi - -# Build classpath -RANGER_CLASSPATH="${WEBAPP_DIR}/WEB-INF/classes" -for jar in "${WEBAPP_DIR}"/WEB-INF/lib/*.jar; do - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" -done - -# Add libext directory if it exists -if [ -d "${AUDIT_CONSUMER_HOME_DIR}/libext" ]; then - for jar in "${AUDIT_CONSUMER_HOME_DIR}"/libext/*.jar; do - if [ -f "${jar}" ]; then - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" - fi - done -fi - -export RANGER_CLASSPATH - -# Check if already running -PID_FILE="${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-hdfs.pid" -if [ -f "${PID_FILE}" ]; then - OLD_PID=$(cat "${PID_FILE}") - if kill -0 "$OLD_PID" 2>/dev/null; then - echo "[WARNING] Ranger Audit Consumer - HDFS is already running (PID: ${OLD_PID})" - echo "[INFO] Use stop-consumer-hdfs.sh to stop it first" - exit 1 - else - echo "[INFO] Removing stale PID file" - rm -f "${PID_FILE}" - fi -fi - -# Start the service -echo "[INFO] Starting Ranger Audit Consumer - HDFS..." -nohup java ${AUDIT_CONSUMER_HEAP} ${AUDIT_CONSUMER_OPTS} \ - -Daudit.config=${AUDIT_CONSUMER_CONF_DIR}/ranger-audit-consumer-hdfs-site.xml \ - -Dhadoop.config.dir=${AUDIT_CONSUMER_CONF_DIR} \ - -Dranger.audit.consumer.webapp.dir="${WAR_FILE}" \ - -cp "${RANGER_CLASSPATH}" \ - org.apache.ranger.audit.consumer.HdfsConsumerApplication \ - >> "${AUDIT_CONSUMER_LOG_DIR}/catalina.out" 2>&1 & - -PID=$! -echo $PID > "${PID_FILE}" - -echo "[INFO] ✓ Ranger Audit Consumer - HDFS started successfully" -echo "[INFO] PID: ${PID}" -echo "[INFO] Log file: ${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-hdfs.log" -echo "[INFO] Catalina out: ${AUDIT_CONSUMER_LOG_DIR}/catalina.out" -echo "[INFO] Health check: http://localhost:7092/api/health" -echo "" -echo "To monitor logs: tail -f ${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-hdfs.log" -echo "To stop service: ${SCRIPT_DIR}/stop-consumer-hdfs.sh" diff --git a/audit-server/consumer-hdfs/scripts/stop-consumer-hdfs.sh b/audit-server/consumer-hdfs/scripts/stop-consumer-hdfs.sh deleted file mode 100755 index 7e98af9b14..0000000000 --- a/audit-server/consumer-hdfs/scripts/stop-consumer-hdfs.sh +++ /dev/null @@ -1,72 +0,0 @@ -#!/bin/bash - -# 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. - -# ======================================== -# Ranger Audit Consumer HDFS Stop Script -# ======================================== - -set -e - -# Script directory -SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" -SERVICE_DIR="$(dirname "$SCRIPT_DIR")" - -# Default log directory -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-${SERVICE_DIR}/logs}" -PID_FILE="${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-hdfs.pid" - -echo "==========================================" -echo "Stopping Ranger Audit Consumer - HDFS" -echo "==========================================" - -if [ ! -f "${PID_FILE}" ]; then - echo "[WARNING] PID file not found: ${PID_FILE}" - echo "[INFO] Service may not be running" - exit 0 -fi - -PID=$(cat "${PID_FILE}") - -if ! kill -0 "$PID" 2>/dev/null; then - echo "[WARNING] Process ${PID} is not running" - echo "[INFO] Removing stale PID file" - rm -f "${PID_FILE}" - exit 0 -fi - -echo "[INFO] Stopping process ${PID}..." -kill "$PID" - -# Wait for process to stop (max 30 seconds) -TIMEOUT=30 -COUNT=0 -while kill -0 "$PID" 2>/dev/null && [ $COUNT -lt $TIMEOUT ]; do - sleep 1 - COUNT=$((COUNT + 1)) - echo -n "." -done -echo "" - -if kill -0 "$PID" 2>/dev/null; then - echo "[WARNING] Process did not stop gracefully, forcing shutdown..." - kill -9 "$PID" - sleep 2 -fi - -rm -f "${PID_FILE}" -echo "[INFO] ✓ Ranger Audit Consumer - HDFS stopped successfully" diff --git a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/HdfsConsumerApplication.java b/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/HdfsConsumerApplication.java deleted file mode 100644 index 268b47420a..0000000000 --- a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/HdfsConsumerApplication.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.ranger.audit.consumer; - -import org.apache.hadoop.conf.Configuration; -import org.apache.ranger.audit.server.EmbeddedServer; -import org.apache.ranger.audit.server.HdfsConsumerConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Main application class for Audit Consumer HDFS Service. - * This service consumes audit events from Kafka and writes them to HDFS. - * It uses embedded Tomcat server for lifecycle management and health check endpoints. - */ -public class HdfsConsumerApplication { - private static final Logger LOG = LoggerFactory.getLogger(HdfsConsumerApplication.class); - private static final String APP_NAME = "ranger-audit-consumer-hdfs"; - private static final String CONFIG_PREFIX = "ranger.audit.consumer.hdfs."; - - private HdfsConsumerApplication() { - } - - public static void main(String[] args) { - LOG.info("=========================================================================="); - LOG.info("==> Starting Ranger Audit Consumer HDFS Service"); - LOG.info("=========================================================================="); - - try { - // Load configuration (includes core-site.xml, hdfs-site.xml, and ranger-audit-consumer-hdfs-site.xml) - Configuration config = HdfsConsumerConfig.getInstance(); - - EmbeddedServer server = new EmbeddedServer(config, APP_NAME, CONFIG_PREFIX); - server.start(); - - LOG.info("<== Ranger Audit Consumer HDFS Service Started Successfully"); - } catch (Exception e) { - LOG.error("<== Failed to start Ranger Audit Consumer HDFS Service", e); - System.exit(1); - } - } -} diff --git a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/HdfsConsumerManager.java b/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/HdfsConsumerManager.java deleted file mode 100644 index 643de22c7a..0000000000 --- a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/consumer/HdfsConsumerManager.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.ranger.audit.consumer; - -import org.apache.ranger.audit.consumer.kafka.AuditConsumer; -import org.apache.ranger.audit.consumer.kafka.AuditConsumerRegistry; -import org.apache.ranger.audit.provider.MiscUtil; -import org.apache.ranger.audit.server.AuditServerConstants; -import org.apache.ranger.audit.server.HdfsConsumerConfig; -import org.apache.ranger.audit.utils.AuditServerLogFormatter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.stereotype.Component; - -import javax.annotation.PostConstruct; -import javax.annotation.PreDestroy; - -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -/** - * Spring component that manages the lifecycle of HDFS consumer threads. - * This manager: - * - Initializes the consumer registry - * - Creates HDFS consumer instances - * - Starts consumer threads - * - Handles graceful shutdown - */ -@Component -public class HdfsConsumerManager { - private static final Logger LOG = LoggerFactory.getLogger(HdfsConsumerManager.class); - - private final AuditConsumerRegistry consumerRegistry = AuditConsumerRegistry.getInstance(); - private final List consumers = new ArrayList<>(); - private final List consumerThreads = new ArrayList<>(); - - @PostConstruct - public void init() { - LOG.info("==> HdfsConsumerManager.init()"); - - try { - HdfsConsumerConfig config = HdfsConsumerConfig.getInstance(); - Properties props = config.getProperties(); - - if (props == null) { - LOG.error("Configuration properties are null"); - throw new RuntimeException("Failed to load configuration"); - } - - // Initialize and register HDFS Consumer - initializeConsumerClasses(props, AuditServerConstants.PROP_KAFKA_PROP_PREFIX); - - // Create consumers from registry - List createdConsumers = consumerRegistry.createConsumers(props, AuditServerConstants.PROP_KAFKA_PROP_PREFIX); - consumers.addAll(createdConsumers); - - if (consumers.isEmpty()) { - LOG.warn("No consumers were created! Verify that xasecure.audit.destination.hdfs=true"); - } else { - LOG.info("Created {} HDFS consumer(s)", consumers.size()); - - // Start consumer threads - startConsumers(); - } - } catch (Exception e) { - LOG.error("Failed to initialize HdfsConsumerManager", e); - throw new RuntimeException("Failed to initialize HdfsConsumerManager", e); - } - - LOG.info("<== HdfsConsumerManager.init() - {} consumer thread(s) started", consumerThreads.size()); - } - - private void initializeConsumerClasses(Properties props, String propPrefix) { - LOG.info("==> HdfsConsumerManager.initializeConsumerClasses()"); - - String clsStr = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_CLASSES, "org.apache.ranger.audit.consumer.kafka.AuditHDFSConsumer"); - - String[] hdfsConsumerClasses = clsStr.split(","); - - LOG.info("Initializing {} consumer class(es)", hdfsConsumerClasses.length); - - for (String hdfsConsumerClassName : hdfsConsumerClasses) { - hdfsConsumerClassName = hdfsConsumerClassName.trim(); - - if (hdfsConsumerClassName.isEmpty()) { - continue; - } - - try { - Class consumerClass = Class.forName(hdfsConsumerClassName); - LOG.info("Successfully initialized consumer class: {}", consumerClass.getName()); - } catch (ClassNotFoundException e) { - LOG.error("Consumer class not found: {}. Ensure the class is on the classpath.", hdfsConsumerClassName, e); - } catch (Exception e) { - LOG.error("Error initializing consumer class: {}", hdfsConsumerClassName, e); - } - } - - LOG.info("Registered consumer factories: {}", consumerRegistry.getRegisteredDestinationTypes()); - LOG.info("<== HdfsConsumerManager.initializeConsumerClasses()"); - } - - /** - * Start all consumer threads - */ - private void startConsumers() { - LOG.info("==> HdfsConsumerManager.startConsumers()"); - - logConsumerStartup(); - - for (AuditConsumer consumer : consumers) { - try { - String consumerName = consumer.getClass().getSimpleName(); - Thread consumerThread = new Thread(consumer, consumerName); - consumerThread.setDaemon(true); - consumerThread.start(); - consumerThreads.add(consumerThread); - - LOG.info("Started {} thread [Thread-ID: {}, Thread-Name: '{}']", - consumerName, consumerThread.getId(), consumerThread.getName()); - } catch (Exception e) { - LOG.error("Error starting consumer: {}", consumer.getClass().getSimpleName(), e); - } - } - - LOG.info("<== HdfsConsumerManager.startConsumers() - {} thread(s) started", consumerThreads.size()); - } - - private void logConsumerStartup() { - LOG.info("################## HDFS CONSUMER SERVICE STARTUP ######################"); - - if (consumers.isEmpty()) { - LOG.warn("WARNING: No HDFS consumers are enabled!"); - LOG.warn("Verify: xasecure.audit.destination.hdfs=true in configuration"); - } else { - AuditServerLogFormatter.LogBuilder builder = AuditServerLogFormatter.builder("HDFS Consumer Status"); - - for (AuditConsumer consumer : consumers) { - String consumerType = consumer.getClass().getSimpleName(); - builder.add(consumerType, "ENABLED"); - builder.add("Topic", consumer.getTopicName()); - } - - builder.logInfo(LOG); - LOG.info("Starting {} HDFS consumer thread(s)...", consumers.size()); - } - LOG.info("########################################################################"); - } - - @PreDestroy - public void shutdown() { - LOG.info("==> HdfsConsumerManager.shutdown()"); - - // Shutdown all consumers - for (AuditConsumer consumer : consumers) { - try { - LOG.info("Shutting down consumer: {}", consumer.getClass().getSimpleName()); - consumer.shutdown(); - LOG.info("Consumer shutdown completed: {}", consumer.getClass().getSimpleName()); - } catch (Exception e) { - LOG.error("Error shutting down consumer: {}", consumer.getClass().getSimpleName(), e); - } - } - - // Wait for threads to terminate - for (Thread thread : consumerThreads) { - if (thread.isAlive()) { - try { - LOG.info("Waiting for thread to terminate: {}", thread.getName()); - thread.join(10000); // Wait up to 10 seconds - if (thread.isAlive()) { - LOG.warn("Thread did not terminate within 10 seconds: {}", thread.getName()); - } - } catch (InterruptedException e) { - LOG.warn("Interrupted while waiting for thread to terminate: {}", thread.getName(), e); - Thread.currentThread().interrupt(); - } - } - } - - consumers.clear(); - consumerThreads.clear(); - consumerRegistry.clearActiveConsumers(); - - LOG.info("<== HdfsConsumerManager.shutdown() - All HDFS consumers stopped"); - } -} diff --git a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/rest/HealthCheckREST.java b/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/rest/HealthCheckREST.java deleted file mode 100644 index ba1303582c..0000000000 --- a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/rest/HealthCheckREST.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.ranger.audit.rest; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.ranger.audit.consumer.HdfsConsumerManager; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.context.annotation.Scope; -import org.springframework.stereotype.Component; - -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Response; - -import java.util.HashMap; -import java.util.Map; - -/** - * Health check REST endpoint for HDFS Consumer Service - */ -@Path("/health") -@Component -@Scope("request") -public class HealthCheckREST { - private static final Logger LOG = LoggerFactory.getLogger(HealthCheckREST.class); - - @Autowired(required = false) - HdfsConsumerManager hdfsConsumerManager; - - /** - * Health check endpoint - */ - @GET - @Produces("application/json") - public Response healthCheck() { - LOG.debug("==> HealthCheckREST.healthCheck()"); - Response ret; - String jsonString; - - try { - // Check if consumer manager is available and healthy - if (hdfsConsumerManager != null) { - Map resp = new HashMap<>(); - resp.put("status", "UP"); - resp.put("service", "audit-consumer-hdfs"); - resp.put("timestamp", System.currentTimeMillis()); - jsonString = buildResponse(resp); - ret = Response.ok() - .entity(jsonString) - .build(); - } else { - Map resp = new HashMap<>(); - resp.put("status", "DOWN"); - resp.put("service", "audit-consumer-hdfs"); - resp.put("reason", "HdfsConsumerManager not available"); - resp.put("timestamp", System.currentTimeMillis()); - jsonString = buildResponse(resp); - ret = Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity(jsonString) - .build(); - } - } catch (Exception e) { - LOG.error("Health check failed", e); - Map resp = new HashMap<>(); - resp.put("status", "DOWN"); - resp.put("service", "audit-consumer-hdfs"); - resp.put("reason", e.getMessage()); - resp.put("timestamp", System.currentTimeMillis()); - jsonString = buildResponse(resp); - ret = Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity(jsonString) - .build(); - } - - LOG.debug("<== HealthCheckREST.healthCheck(): {}", ret); - - return ret; - } - - private String buildResponse(Map respMap) { - String ret; - - try { - ObjectMapper objectMapper = new ObjectMapper(); - ret = objectMapper.writeValueAsString(respMap); - } catch (Exception e) { - ret = "Error: " + e.getMessage(); - } - - return ret; - } -} diff --git a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/server/HdfsConsumerConfig.java b/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/server/HdfsConsumerConfig.java deleted file mode 100644 index 6c38a3ac95..0000000000 --- a/audit-server/consumer-hdfs/src/main/java/org/apache/ranger/audit/server/HdfsConsumerConfig.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * 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.ranger.audit.server; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Loads HDFS consumer-specific configuration files including Hadoop configs. - */ -public class HdfsConsumerConfig extends AuditConfig { - private static final Logger LOG = LoggerFactory.getLogger(HdfsConsumerConfig.class); - private static final String CONFIG_FILE_PATH = "conf/ranger-audit-consumer-hdfs-site.xml"; - private static final String CORE_SITE_FILE_PATH = "conf/core-site.xml"; - private static final String HDFS_SITE_FILE_PATH = "conf/hdfs-site.xml"; - private static volatile HdfsConsumerConfig sInstance; - - private HdfsConsumerConfig() { - super(); - addHdfsConsumerResources(); - } - - public static HdfsConsumerConfig getInstance() { - HdfsConsumerConfig ret = HdfsConsumerConfig.sInstance; - - if (ret == null) { - synchronized (HdfsConsumerConfig.class) { - ret = HdfsConsumerConfig.sInstance; - - if (ret == null) { - ret = new HdfsConsumerConfig(); - HdfsConsumerConfig.sInstance = ret; - } - } - } - - return ret; - } - - private boolean addHdfsConsumerResources() { - LOG.debug("==> HdfsConsumerConfig.addHdfsConsumerResources()"); - - boolean ret = true; - - if (!addAuditResource(CORE_SITE_FILE_PATH, false)) { - LOG.warn("Could not load required configuration: {}", CORE_SITE_FILE_PATH); - ret = false; - } - - if (!addAuditResource(HDFS_SITE_FILE_PATH, true)) { - LOG.error("Could not load required configuration: {}", HDFS_SITE_FILE_PATH); - ret = false; - } - - if (!addAuditResource(CONFIG_FILE_PATH, true)) { - LOG.error("Could not load required configuration: {}", CONFIG_FILE_PATH); - ret = false; - } - - LOG.debug("<== HdfsConsumerConfig.addHdfsConsumerResources(), result={}", ret); - - return ret; - } - - /** - * Get Hadoop configuration properties (from core-site.xml and hdfs-site.xml) with a specific prefix. - * @param prefix The prefix to add to each property name ("xasecure.audit.destination.hdfs.config.") - * @return Properties from core-site.xml and hdfs-site.xml with the specified prefix - */ - public java.util.Properties getHadoopPropertiesWithPrefix(String prefix) { - LOG.debug("==> HdfsConsumerConfig.getHadoopPropertiesWithPrefix(prefix={})", prefix); - - java.util.Properties prefixedProps = new java.util.Properties(); - int propsAdded = 0; - - try { - // Load core-site.xml separately to get pure Hadoop security properties - org.apache.hadoop.conf.Configuration coreSite = new org.apache.hadoop.conf.Configuration(false); - coreSite.addResource(CORE_SITE_FILE_PATH); - - for (java.util.Map.Entry entry : coreSite) { - String propName = entry.getKey(); - String propValue = entry.getValue(); - - if (propValue != null && !propValue.trim().isEmpty()) { - prefixedProps.setProperty(prefix + propName, propValue); - LOG.trace("Added from core-site.xml: {} = {}", propName, propValue); - propsAdded++; - } - } - - // Load hdfs-site.xml separately to get pure HDFS client properties - org.apache.hadoop.conf.Configuration hdfsSite = new org.apache.hadoop.conf.Configuration(false); - hdfsSite.addResource(HDFS_SITE_FILE_PATH); - - for (java.util.Map.Entry entry : hdfsSite) { - String propName = entry.getKey(); - String propValue = entry.getValue(); - - if (propValue != null && !propValue.trim().isEmpty()) { - prefixedProps.setProperty(prefix + propName, propValue); - LOG.trace("Added from hdfs-site.xml: {} = {}", propName, propValue); - propsAdded++; - } - } - - LOG.debug("<== HdfsConsumerConfig.getHadoopPropertiesWithPrefix(): Added {} Hadoop properties with prefix '{}'", propsAdded, prefix); - } catch (Exception e) { - LOG.error("Failed to load Hadoop properties from {} and {}", CORE_SITE_FILE_PATH, HDFS_SITE_FILE_PATH, e); - } - - return prefixedProps; - } - - /** - * Get core-site.xml Configuration for UGI initialization. - * @return Configuration loaded from core-site.xml - */ - public org.apache.hadoop.conf.Configuration getCoreSiteConfiguration() { - LOG.debug("==> HdfsConsumerConfig.getCoreSiteConfiguration()"); - - org.apache.hadoop.conf.Configuration coreSite = new org.apache.hadoop.conf.Configuration(false); - coreSite.addResource(CORE_SITE_FILE_PATH); - - LOG.debug("<== HdfsConsumerConfig.getCoreSiteConfiguration(): authentication={}", coreSite.get("hadoop.security.authentication")); - - return coreSite; - } -} diff --git a/audit-server/consumer-hdfs/src/main/webapp/WEB-INF/applicationContext.xml b/audit-server/consumer-hdfs/src/main/webapp/WEB-INF/applicationContext.xml deleted file mode 100644 index 5a7696647f..0000000000 --- a/audit-server/consumer-hdfs/src/main/webapp/WEB-INF/applicationContext.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - - - - - - - - - - - - - - diff --git a/audit-server/consumer-solr/scripts/start-consumer-solr.sh b/audit-server/consumer-solr/scripts/start-consumer-solr.sh deleted file mode 100755 index 88f64e4231..0000000000 --- a/audit-server/consumer-solr/scripts/start-consumer-solr.sh +++ /dev/null @@ -1,163 +0,0 @@ -#!/bin/bash - -# 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. - -# ======================================== -# Ranger Audit Consumer Solr Start Script -# ======================================== -# This script starts the Solr consumer service -# The service consumes audit events from Kafka and indexes them to Solr - -set -e - -# Script directory -SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" -SERVICE_DIR="$(dirname "$SCRIPT_DIR")" - -# Default directories - can be overridden by environment variables -AUDIT_CONSUMER_HOME_DIR="${AUDIT_CONSUMER_HOME_DIR:-${SERVICE_DIR}/target}" -AUDIT_CONSUMER_CONF_DIR="${AUDIT_CONSUMER_CONF_DIR:-${SERVICE_DIR}/src/main/resources/conf}" -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-${SERVICE_DIR}/logs}" - -# Create log directory if it doesn't exist -mkdir -p "${AUDIT_CONSUMER_LOG_DIR}" - -echo "==========================================" -echo "Starting Ranger Audit Consumer - Solr" -echo "==========================================" -echo "Service Directory: ${SERVICE_DIR}" -echo "Home Directory: ${AUDIT_CONSUMER_HOME_DIR}" -echo "Config Directory: ${AUDIT_CONSUMER_CONF_DIR}" -echo "Log Directory: ${AUDIT_CONSUMER_LOG_DIR}" -echo "==========================================" - -# Check if Java is available -if [ -z "$JAVA_HOME" ]; then - JAVA_CMD=$(which java 2>/dev/null || true) - if [ -z "$JAVA_CMD" ]; then - echo "[ERROR] JAVA_HOME is not set and java is not in PATH" - exit 1 - fi - JAVA_HOME=$(dirname $(dirname $(readlink -f "$JAVA_CMD"))) - echo "[INFO] JAVA_HOME not set, detected: ${JAVA_HOME}" -fi - -export JAVA_HOME -export PATH=$JAVA_HOME/bin:$PATH - -echo "[INFO] Java version:" -java -version - -# Set heap size (default: 512MB to 2GB) -AUDIT_CONSUMER_HEAP="${AUDIT_CONSUMER_HEAP:--Xms512m -Xmx2g}" - -# Set JVM options -if [ -z "$AUDIT_CONSUMER_OPTS" ]; then - AUDIT_CONSUMER_OPTS="-Dlogback.configurationFile=${AUDIT_CONSUMER_CONF_DIR}/logback.xml" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.solr.log.dir=${AUDIT_CONSUMER_LOG_DIR}" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.solr.log.file=ranger-audit-consumer-solr.log" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.net.preferIPv4Stack=true -server" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:+UseG1GC -XX:MaxGCPauseMillis=200" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:InitiatingHeapOccupancyPercent=35" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:ConcGCThreads=4 -XX:ParallelGCThreads=8" -fi - -# Add Kerberos configuration if needed -if [ "${KERBEROS_ENABLED}" == "true" ]; then - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.security.krb5.conf=/etc/krb5.conf" - echo "[INFO] Kerberos is enabled" -fi - -export AUDIT_CONSUMER_OPTS -export AUDIT_CONSUMER_LOG_DIR - -echo "[INFO] JAVA_HOME: ${JAVA_HOME}" -echo "[INFO] HEAP: ${AUDIT_CONSUMER_HEAP}" -echo "[INFO] JVM_OPTS: ${AUDIT_CONSUMER_OPTS}" - -# Find the WAR file -WAR_FILE=$(find "${AUDIT_CONSUMER_HOME_DIR}" -name "ranger-audit-consumer-solr*.war" | head -1) - -if [ -z "$WAR_FILE" ] || [ ! -f "$WAR_FILE" ]; then - echo "[ERROR] WAR file not found in ${AUDIT_CONSUMER_HOME_DIR}" - echo "[ERROR] Please build the project first using: mvn clean package" - exit 1 -fi - -echo "[INFO] Using WAR file: ${WAR_FILE}" - -# Extract WAR if not already extracted -WEBAPP_DIR="${AUDIT_CONSUMER_HOME_DIR}/webapp/ranger-audit-consumer-solr" -if [ ! -d "${WEBAPP_DIR}" ]; then - echo "[INFO] Extracting WAR file..." - mkdir -p "${WEBAPP_DIR}" - cd "${WEBAPP_DIR}" - jar xf "${WAR_FILE}" - cd - > /dev/null -fi - -# Build classpath -RANGER_CLASSPATH="${WEBAPP_DIR}/WEB-INF/classes" -for jar in "${WEBAPP_DIR}"/WEB-INF/lib/*.jar; do - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" -done - -# Add libext directory if it exists -if [ -d "${AUDIT_CONSUMER_HOME_DIR}/libext" ]; then - for jar in "${AUDIT_CONSUMER_HOME_DIR}"/libext/*.jar; do - if [ -f "${jar}" ]; then - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" - fi - done -fi - -export RANGER_CLASSPATH - -# Check if already running -PID_FILE="${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-solr.pid" -if [ -f "${PID_FILE}" ]; then - OLD_PID=$(cat "${PID_FILE}") - if kill -0 "$OLD_PID" 2>/dev/null; then - echo "[WARNING] Ranger Audit Consumer - Solr is already running (PID: ${OLD_PID})" - echo "[INFO] Use stop-consumer-solr.sh to stop it first" - exit 1 - else - echo "[INFO] Removing stale PID file" - rm -f "${PID_FILE}" - fi -fi - -# Start the service -echo "[INFO] Starting Ranger Audit Consumer - Solr..." -nohup java ${AUDIT_CONSUMER_HEAP} ${AUDIT_CONSUMER_OPTS} \ - -Daudit.config=${AUDIT_CONSUMER_CONF_DIR}/ranger-audit-consumer-solr-site.xml \ - -Dranger.audit.consumer.webapp.dir="${WAR_FILE}" \ - -cp "${RANGER_CLASSPATH}" \ - org.apache.ranger.audit.consumer.SolrConsumerApplication \ - >> "${AUDIT_CONSUMER_LOG_DIR}/catalina.out" 2>&1 & - -PID=$! -echo $PID > "${PID_FILE}" - -echo "[INFO] ✓ Ranger Audit Consumer - Solr started successfully" -echo "[INFO] PID: ${PID}" -echo "[INFO] Log file: ${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-solr.log" -echo "[INFO] Catalina out: ${AUDIT_CONSUMER_LOG_DIR}/catalina.out" -echo "[INFO] Health check: http://localhost:7091/api/health" -echo "" -echo "To monitor logs: tail -f ${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-solr.log" -echo "To stop service: ${SCRIPT_DIR}/stop-consumer-solr.sh" diff --git a/audit-server/consumer-solr/scripts/stop-consumer-solr.sh b/audit-server/consumer-solr/scripts/stop-consumer-solr.sh deleted file mode 100755 index 8e2049ac4b..0000000000 --- a/audit-server/consumer-solr/scripts/stop-consumer-solr.sh +++ /dev/null @@ -1,72 +0,0 @@ -#!/bin/bash - -# 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. - -# ======================================== -# Ranger Audit Consumer Solr Stop Script -# ======================================== - -set -e - -# Script directory -SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" -SERVICE_DIR="$(dirname "$SCRIPT_DIR")" - -# Default log directory -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-${SERVICE_DIR}/logs}" -PID_FILE="${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-solr.pid" - -echo "==========================================" -echo "Stopping Ranger Audit Consumer - Solr" -echo "==========================================" - -if [ ! -f "${PID_FILE}" ]; then - echo "[WARNING] PID file not found: ${PID_FILE}" - echo "[INFO] Service may not be running" - exit 0 -fi - -PID=$(cat "${PID_FILE}") - -if ! kill -0 "$PID" 2>/dev/null; then - echo "[WARNING] Process ${PID} is not running" - echo "[INFO] Removing stale PID file" - rm -f "${PID_FILE}" - exit 0 -fi - -echo "[INFO] Stopping process ${PID}..." -kill "$PID" - -# Wait for process to stop (max 30 seconds) -TIMEOUT=30 -COUNT=0 -while kill -0 "$PID" 2>/dev/null && [ $COUNT -lt $TIMEOUT ]; do - sleep 1 - COUNT=$((COUNT + 1)) - echo -n "." -done -echo "" - -if kill -0 "$PID" 2>/dev/null; then - echo "[WARNING] Process did not stop gracefully, forcing shutdown..." - kill -9 "$PID" - sleep 2 -fi - -rm -f "${PID_FILE}" -echo "[INFO] ✓ Ranger Audit Consumer - Solr stopped successfully" diff --git a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/SolrConsumerApplication.java b/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/SolrConsumerApplication.java deleted file mode 100644 index 16236abbf9..0000000000 --- a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/SolrConsumerApplication.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.ranger.audit.consumer; - -import org.apache.hadoop.conf.Configuration; -import org.apache.ranger.audit.server.EmbeddedServer; -import org.apache.ranger.audit.server.SolrConsumerConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Main application class for Audit Consumer Solr Service. - * This service consumes audit events from Kafka and indexes them into Solr - * It uses embedded Tomcat server for lifecycle management and health check endpoints. - */ -public class SolrConsumerApplication { - private static final Logger LOG = LoggerFactory.getLogger(SolrConsumerApplication.class); - - private static final String APP_NAME = "ranger-audit-consumer-solr"; - private static final String CONFIG_PREFIX = "ranger.audit.consumer.solr."; - - private SolrConsumerApplication() { - } - - public static void main(String[] args) { - LOG.info("=========================================================================="); - LOG.info("==> Starting Ranger Audit Consumer Solr Service"); - LOG.info("=========================================================================="); - - try { - // Load configuration - Configuration config = SolrConsumerConfig.getInstance(); - - LOG.info("Configuration loaded successfully"); - - // Create and start embedded server - // The server will load Spring context which initializes SolrConsumerManager - // which then creates and starts the Solr consumer threads - EmbeddedServer server = new EmbeddedServer(config, APP_NAME, CONFIG_PREFIX); - server.start(); - - LOG.info("==> Ranger Audit Consumer Solr Service Started Successfully"); - } catch (Exception e) { - LOG.error("Failed to start Ranger Audit Consumer Solr Service", e); - System.exit(1); - } - } -} diff --git a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/SolrConsumerManager.java b/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/SolrConsumerManager.java deleted file mode 100644 index 2024f2ddcf..0000000000 --- a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/consumer/SolrConsumerManager.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * 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.ranger.audit.consumer; - -import org.apache.ranger.audit.consumer.kafka.AuditConsumer; -import org.apache.ranger.audit.consumer.kafka.AuditConsumerRegistry; -import org.apache.ranger.audit.provider.MiscUtil; -import org.apache.ranger.audit.server.AuditServerConstants; -import org.apache.ranger.audit.server.SolrConsumerConfig; -import org.apache.ranger.audit.utils.AuditServerLogFormatter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.stereotype.Component; - -import javax.annotation.PostConstruct; -import javax.annotation.PreDestroy; - -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -/** - * Spring component that manages the lifecycle of Solr consumer threads. - * This manager: - * - Initializes the consumer registry - * - Creates Solr consumer instances - * - Starts consumer threads - * - Handles graceful shutdown - */ -@Component -public class SolrConsumerManager { - private static final Logger LOG = LoggerFactory.getLogger(SolrConsumerManager.class); - - private final AuditConsumerRegistry consumerRegistry = AuditConsumerRegistry.getInstance(); - private final List consumers = new ArrayList<>(); - private final List consumerThreads = new ArrayList<>(); - - @PostConstruct - public void init() { - LOG.info("==> SolrConsumerManager.init()"); - - try { - SolrConsumerConfig config = SolrConsumerConfig.getInstance(); - Properties props = config.getProperties(); - - if (props == null) { - LOG.error("Configuration properties are null"); - throw new RuntimeException("Failed to load configuration"); - } - - // Initialize and register Solr Consumer - initializeConsumerClasses(props, AuditServerConstants.PROP_KAFKA_PROP_PREFIX); - - // Create consumers from registry - List createdConsumers = consumerRegistry.createConsumers(props, AuditServerConstants.PROP_KAFKA_PROP_PREFIX); - consumers.addAll(createdConsumers); - - if (consumers.isEmpty()) { - LOG.warn("No consumers were created! Verify that xasecure.audit.destination.solr=true"); - } else { - LOG.info("Created {} Solr consumer(s)", consumers.size()); - - // Start consumer threads - startConsumers(); - } - } catch (Exception e) { - LOG.error("Failed to initialize SolrConsumerManager", e); - throw new RuntimeException("Failed to initialize SolrConsumerManager", e); - } - - LOG.info("<== SolrConsumerManager.init() - {} consumer thread(s) started", consumerThreads.size()); - } - - private void initializeConsumerClasses(Properties props, String propPrefix) { - LOG.info("==> SolrConsumerManager.initializeConsumerClasses()"); - - // Get consumer classes from configuration - String clsStr = MiscUtil.getStringProperty(props, propPrefix + "." + AuditServerConstants.PROP_CONSUMER_CLASSES, - "org.apache.ranger.audit.consumer.kafka.AuditSolrConsumer"); - - String[] solrConsumerClasses = clsStr.split(","); - - LOG.info("Initializing {} consumer class(es)", solrConsumerClasses.length); - - for (String solrConsumerClassName : solrConsumerClasses) { - solrConsumerClassName = solrConsumerClassName.trim(); - - if (solrConsumerClassName.isEmpty()) { - continue; - } - - try { - Class consumerClass = Class.forName(solrConsumerClassName); - LOG.info("Successfully initialized consumer class: {}", consumerClass.getName()); - } catch (ClassNotFoundException e) { - LOG.error("Consumer class not found: {}. Ensure the class is on the classpath.", solrConsumerClassName, e); - } catch (Exception e) { - LOG.error("Error initializing consumer class: {}", solrConsumerClassName, e); - } - } - - LOG.info("Registered consumer factories: {}", consumerRegistry.getRegisteredDestinationTypes()); - LOG.info("<== SolrConsumerManager.initializeConsumerClasses()"); - } - - /** - * Start all consumer threads - */ - private void startConsumers() { - LOG.info("==> SolrConsumerManager.startConsumers()"); - - logSolrConsumerStartup(); - - for (AuditConsumer consumer : consumers) { - try { - String consumerName = consumer.getClass().getSimpleName(); - Thread consumerThread = new Thread(consumer, consumerName); - consumerThread.setDaemon(true); - consumerThread.start(); - consumerThreads.add(consumerThread); - - LOG.info("Started {} thread [Thread-ID: {}, Thread-Name: '{}']", - consumerName, consumerThread.getId(), consumerThread.getName()); - } catch (Exception e) { - LOG.error("Error starting consumer: {}", consumer.getClass().getSimpleName(), e); - } - } - - LOG.info("<== SolrConsumerManager.startConsumers() - {} thread(s) started", consumerThreads.size()); - } - - /** - * Log startup banner with consumer information - */ - private void logSolrConsumerStartup() { - LOG.info("################## SOLR CONSUMER SERVICE STARTUP ######################"); - - if (consumers.isEmpty()) { - LOG.warn("WARNING: No Solr consumers are enabled!"); - LOG.warn("Verify: xasecure.audit.destination.solr=true in configuration"); - } else { - AuditServerLogFormatter.LogBuilder builder = AuditServerLogFormatter.builder("Solr Consumer Status"); - - for (AuditConsumer consumer : consumers) { - String consumerType = consumer.getClass().getSimpleName(); - builder.add(consumerType, "ENABLED"); - builder.add("Topic", consumer.getTopicName()); - } - - builder.logInfo(LOG); - LOG.info("Starting {} Solr consumer thread(s)...", consumers.size()); - } - LOG.info("########################################################################"); - } - - @PreDestroy - public void shutdown() { - LOG.info("==> SolrConsumerManager.shutdown()"); - - // Shutdown all consumers - for (AuditConsumer consumer : consumers) { - try { - LOG.info("Shutting down consumer: {}", consumer.getClass().getSimpleName()); - consumer.shutdown(); - LOG.info("Consumer shutdown completed: {}", consumer.getClass().getSimpleName()); - } catch (Exception e) { - LOG.error("Error shutting down consumer: {}", consumer.getClass().getSimpleName(), e); - } - } - - // Wait for threads to terminate - for (Thread thread : consumerThreads) { - if (thread.isAlive()) { - try { - LOG.info("Waiting for thread to terminate: {}", thread.getName()); - thread.join(10000); // Wait up to 10 seconds - if (thread.isAlive()) { - LOG.warn("Thread did not terminate within 10 seconds: {}", thread.getName()); - } - } catch (InterruptedException e) { - LOG.warn("Interrupted while waiting for thread to terminate: {}", thread.getName(), e); - Thread.currentThread().interrupt(); - } - } - } - - consumers.clear(); - consumerThreads.clear(); - consumerRegistry.clearActiveConsumers(); - - LOG.info("<== SolrConsumerManager.shutdown() - All Solr consumers stopped"); - } -} diff --git a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/rest/HealthCheckREST.java b/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/rest/HealthCheckREST.java deleted file mode 100644 index 3245c14471..0000000000 --- a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/rest/HealthCheckREST.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.ranger.audit.rest; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.ranger.audit.consumer.SolrConsumerManager; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.context.annotation.Scope; -import org.springframework.stereotype.Component; - -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Response; - -import java.util.HashMap; -import java.util.Map; - -/** - * Health check REST endpoint for Solr Consumer Service - */ -@Path("/health") -@Component -@Scope("request") -public class HealthCheckREST { - private static final Logger LOG = LoggerFactory.getLogger(HealthCheckREST.class); - - @Autowired(required = false) - SolrConsumerManager solrConsumerManager; - - /** - * Health check endpoint - */ - @GET - @Produces("application/json") - public Response healthCheck() { - LOG.debug("==> HealthCheckREST.healthCheck()"); - Response ret; - String jsonString; - - try { - // Check if consumer manager is available and healthy - if (solrConsumerManager != null) { - Map resp = new HashMap<>(); - resp.put("status", "UP"); - resp.put("service", "audit-consumer-solr"); - resp.put("timestamp", System.currentTimeMillis()); - jsonString = buildResponse(resp); - ret = Response.ok() - .entity(jsonString) - .build(); - } else { - Map resp = new HashMap<>(); - resp.put("status", "DOWN"); - resp.put("service", "audit-consumer-solr"); - resp.put("reason", "SolrConsumerManager not available"); - resp.put("timestamp", System.currentTimeMillis()); - jsonString = buildResponse(resp); - ret = Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity(jsonString) - .build(); - } - } catch (Exception e) { - LOG.error("Health check failed", e); - Map resp = new HashMap<>(); - resp.put("status", "DOWN"); - resp.put("service", "audit-consumer-solr"); - resp.put("reason", e.getMessage()); - resp.put("timestamp", System.currentTimeMillis()); - jsonString = buildResponse(resp); - ret = Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity(jsonString) - .build(); - } - - LOG.debug("<== HealthCheckREST.healthCheck(): {}", ret); - - return ret; - } - - private String buildResponse(Map respMap) { - String ret; - - try { - ObjectMapper objectMapper = new ObjectMapper(); - ret = objectMapper.writeValueAsString(respMap); - } catch (Exception e) { - ret = "Error: " + e.getMessage(); - } - - return ret; - } -} diff --git a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/server/SolrConsumerConfig.java b/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/server/SolrConsumerConfig.java deleted file mode 100644 index 837a6fc734..0000000000 --- a/audit-server/consumer-solr/src/main/java/org/apache/ranger/audit/server/SolrConsumerConfig.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.ranger.audit.server; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Configuration class for Solr Consumer Service. - * Loads Solr consumer-specific configuration files. - */ -public class SolrConsumerConfig extends AuditConfig { - private static final Logger LOG = LoggerFactory.getLogger(SolrConsumerConfig.class); - private static final String CONFIG_FILE_PATH = "conf/ranger-audit-consumer-solr-site.xml"; - private static volatile SolrConsumerConfig sInstance; - - private SolrConsumerConfig() { - super(); - addSolrConsumerResources(); - } - - public static SolrConsumerConfig getInstance() { - SolrConsumerConfig ret = SolrConsumerConfig.sInstance; - - if (ret == null) { - synchronized (SolrConsumerConfig.class) { - ret = SolrConsumerConfig.sInstance; - - if (ret == null) { - ret = new SolrConsumerConfig(); - SolrConsumerConfig.sInstance = ret; - } - } - } - - return ret; - } - - private boolean addSolrConsumerResources() { - LOG.debug("==> SolrConsumerConfig.addSolrConsumerResources()"); - - boolean ret = true; - - // Load ranger-audit-consumer-solr-site.xml - if (!addAuditResource(CONFIG_FILE_PATH, true)) { - LOG.error("Could not load required configuration: {}", CONFIG_FILE_PATH); - ret = false; - } - - LOG.debug("<== SolrConsumerConfig.addSolrConsumerResources(), result={}", ret); - - return ret; - } -} diff --git a/audit-server/consumer-solr/src/main/webapp/WEB-INF/web.xml b/audit-server/consumer-solr/src/main/webapp/WEB-INF/web.xml deleted file mode 100644 index db19d1326b..0000000000 --- a/audit-server/consumer-solr/src/main/webapp/WEB-INF/web.xml +++ /dev/null @@ -1,60 +0,0 @@ - - - - - Apache Ranger - Audit Consumer Solr Service - Apache Ranger - Audit Consumer Solr Service (Kafka to Solr) - - - contextConfigLocation - - WEB-INF/applicationContext.xml - - - - - org.springframework.web.context.ContextLoaderListener - - - - org.springframework.web.context.request.RequestContextListener - - - - Health Check Service - com.sun.jersey.spi.spring.container.servlet.SpringServlet - - - com.sun.jersey.config.property.packages - org.apache.ranger.audit.rest - - - - com.sun.jersey.api.json.POJOMappingFeature - true - - - 1 - - - - Health Check Service - /api/* - - - diff --git a/audit-server/pom.xml b/audit-server/pom.xml index edd966aa5a..33ffe4253d 100644 --- a/audit-server/pom.xml +++ b/audit-server/pom.xml @@ -1,21 +1,20 @@ + 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. +--> 4.0.0 @@ -29,14 +28,13 @@ ranger-audit-server pom Ranger Audit Server Parent - Parent project for Ranger Audit Server microservices - Producer and Consumer services + Parent project for Ranger Audit Server microservices - Producer and Dispatcher services - common - consumer-hdfs - consumer-solr - server + audit-common + audit-dispatcher + audit-ingestor @@ -275,25 +273,31 @@ org.apache.ranger - ranger-audit-common + ranger-audit-core ${project.version} org.apache.ranger - ranger-audit-core + ranger-audit-dest-hdfs ${project.version} org.apache.ranger - ranger-audit-dest-hdfs + ranger-audit-dest-solr ${project.version} org.apache.ranger - ranger-audit-dest-solr + ranger-audit-dispatcher-common + ${project.version} + + + + org.apache.ranger + ranger-audit-server-common ${project.version} @@ -429,7 +433,7 @@ ${maven.pmd.plugin.version} - ${project.parent.parent.basedir}/dev-support/ranger-pmd-ruleset.xml + ${project.parent.basedir}/dev-support/ranger-pmd-ruleset.xml UTF-8 true diff --git a/audit-server/scripts/README.md b/audit-server/scripts/README.md index 509b778048..4dde235f7d 100644 --- a/audit-server/scripts/README.md +++ b/audit-server/scripts/README.md @@ -22,11 +22,13 @@ This directory contains shell scripts to start and stop the Ranger Audit Server The Ranger Audit Server consists of three microservices: -1. **ranger-audit-server-service** - Core audit server that receives audit events via REST API and produces them to Kafka -2. **ranger-audit-consumer-solr** - Consumer service that reads from Kafka and indexes audits to Solr -3. **ranger-audit-consumer-hdfs** - Consumer service that reads from Kafka and writes audits to HDFS/S3/Azure +1. **ranger-audit-ingestor** - Core audit server that receives audit events via REST API and produces them to Kafka +2. **ranger-audit-dispatcher-solr** - Dispatcher service that reads from Kafka and indexes audits to Solr +3. **ranger-audit-dispatcher-hdfs** - Dispatcher service that reads from Kafka and writes audits to HDFS/S3/Azure -Each service has its own `scripts` folder with start/stop scripts in its main directory. +The codebase is organized into two main directories: +- `audit-ingestor/` +- `audit-dispatcher/` (contains a unified dispatcher application that dynamically loads specific dispatcher types) ## Prerequisites @@ -39,8 +41,8 @@ Before running these scripts, ensure you have: mvn clean package -DskipTests ``` 3. **Kafka running** (required for all services) -4. **Solr running** (required for Solr consumer) -5. **HDFS/Hadoop running** (required for HDFS consumer) +4. **Solr running** (required for Solr dispatcher) +5. **HDFS/Hadoop running** (required for HDFS dispatcher) ## Quick Start - All Services @@ -52,8 +54,8 @@ Before running these scripts, ensure you have: This script will start all three services in the correct order: 1. Audit Server (waits 10 seconds) -2. Solr Consumer (waits 5 seconds) -3. HDFS Consumer +2. Solr Dispatcher (waits 5 seconds) +3. HDFS Dispatcher ### Stop All Services @@ -71,40 +73,40 @@ Each service can also be started/stopped individually: ```bash # Start -./ranger-audit-server-service/scripts/start-audit-server.sh +./audit-ingestor/scripts/start-audit-ingestor.sh # Stop -./ranger-audit-server-service/scripts/stop-audit-server.sh +./audit-ingestor/scripts/stop-audit-ingestor.sh ``` **Default Ports:** 7081 (HTTP), 7182 (HTTPS) **Health Check:** http://localhost:7081/api/audit/health -### Solr Consumer +### Solr Dispatcher ```bash # Start -./ranger-audit-consumer-solr/scripts/start-consumer-solr.sh +./audit-dispatcher/scripts/start-audit-dispatcher.sh solr # Stop -./ranger-audit-consumer-solr/scripts/stop-consumer-solr.sh +./audit-dispatcher/scripts/stop-audit-dispatcher.sh solr ``` **Default Port:** 7091 -**Health Check:** http://localhost:7091/api/health +**Health Check:** http://localhost:7091/api/health/ping -### HDFS Consumer +### HDFS Dispatcher ```bash # Start -./ranger-audit-consumer-hdfs/scripts/start-consumer-hdfs.sh +./audit-dispatcher/scripts/start-audit-dispatcher.sh hdfs # Stop -./ranger-audit-consumer-hdfs/scripts/stop-consumer-hdfs.sh +./audit-dispatcher/scripts/stop-audit-dispatcher.sh hdfs ``` **Default Port:** 7092 -**Health Check:** http://localhost:7092/api/health +**Health Check:** http://localhost:7092/api/health/ping ## Configuration @@ -120,12 +122,12 @@ Each script supports the following environment variables: - `AUDIT_SERVER_OPTS` - Additional JVM options - `KERBEROS_ENABLED` - Enable Kerberos authentication (default: `false`) -#### Consumers (HDFS and Solr) -- `AUDIT_CONSUMER_HOME_DIR` - Home directory (default: `target/`) -- `AUDIT_CONSUMER_CONF_DIR` - Configuration directory (default: `src/main/resources/conf/`) -- `AUDIT_CONSUMER_LOG_DIR` - Log directory (default: `logs/`) -- `AUDIT_CONSUMER_HEAP` - JVM heap settings (default: `-Xms512m -Xmx2g`) -- `AUDIT_CONSUMER_OPTS` - Additional JVM options +#### Dispatchers (HDFS and Solr) +- `AUDIT_DISPATCHER_HOME_DIR` - Home directory (default: `target/`) +- `AUDIT_DISPATCHER_CONF_DIR` - Configuration directory (default: `src/main/resources/conf/`) +- `AUDIT_DISPATCHER_LOG_DIR` - Log directory (default: `logs/`) +- `AUDIT_DISPATCHER_HEAP` - JVM heap settings (default: `-Xms512m -Xmx2g`) +- `AUDIT_DISPATCHER_OPTS` - Additional JVM options - `KERBEROS_ENABLED` - Enable Kerberos authentication (default: `false`) ### Example with Custom Settings @@ -135,7 +137,7 @@ Each script supports the following environment variables: export AUDIT_SERVER_HEAP="-Xms1g -Xmx4g" export AUDIT_SERVER_LOG_DIR="/var/log/ranger/range-audit-server" -./ranger-audit-server-service/scripts/start-audit-server.sh +./audit-ingestor/scripts/start-audit-ingestor.sh ``` ## Log Files @@ -143,31 +145,31 @@ export AUDIT_SERVER_LOG_DIR="/var/log/ranger/range-audit-server" Each service creates logs in its respective `logs/` directory (or custom location if set): - **Audit Server:** - - Application logs: `logs/ranger-audit-server.log` + - Application logs: `logs/ranger-audit-ingestor.log` - Catalina output: `logs/catalina.out` - - PID file: `logs/ranger-audit-server.pid` + - PID file: `logs/ranger-audit-ingestor.pid` -- **Solr Consumer:** - - Application logs: `logs/ranger-audit-consumer-solr.log` +- **Solr Dispatcher:** + - Application logs: `logs/ranger-audit-dispatcher.log` - Catalina output: `logs/catalina.out` - - PID file: `logs/ranger-audit-consumer-solr.pid` + - PID file: `logs/ranger-audit-dispatcher-solr.pid` -- **HDFS Consumer:** - - Application logs: `logs/ranger-audit-consumer-hdfs.log` +- **HDFS Dispatcher:** + - Application logs: `logs/ranger-audit-dispatcher.log` - Catalina output: `logs/catalina.out` - - PID file: `logs/ranger-audit-consumer-hdfs.pid` + - PID file: `logs/ranger-audit-dispatcher-hdfs.pid` ### Monitoring Logs ```bash # Tail audit server logs -tail -f ranger-audit-server-service/logs/ranger-audit-server.log +tail -f audit-ingestor/logs/ranger-audit-ingestor.log -# Tail Solr consumer logs -tail -f ranger-audit-consumer-solr/logs/ranger-audit-consumer-solr.log +# Tail Solr dispatcher logs +tail -f audit-dispatcher/logs/ranger-audit-dispatcher.log -# Tail HDFS consumer logs -tail -f ranger-audit-consumer-hdfs/logs/ranger-audit-consumer-hdfs.log +# Tail HDFS dispatcher logs +tail -f audit-dispatcher/logs/ranger-audit-dispatcher.log ``` ### Enabling Debug Logging @@ -175,7 +177,7 @@ tail -f ranger-audit-consumer-hdfs/logs/ranger-audit-consumer-hdfs.log To enable debug logging for troubleshooting, modify the `logback.xml` configuration file in the service's `conf/` directory: **For Audit Server:** -Edit `ranger-audit-server-service/src/main/resources/conf/logback.xml` (or `/opt/ranger-audit-server/conf/logback.xml` in Docker): +Edit `audit-ingestor/src/main/resources/conf/logback.xml` (or `/opt/ranger/audit-ingestor/conf/logback.xml` in Docker): ```xml @@ -191,7 +193,7 @@ Edit `ranger-audit-server-service/src/main/resources/conf/logback.xml` (or `/opt ``` -**For Consumers (HDFS/Solr):** +**For Dispatchers (HDFS/Solr):** Similarly, edit the `logback.xml` in their respective `conf/` directories. **Available log levels:** `TRACE`, `DEBUG`, `INFO`, `WARN`, `ERROR` @@ -210,8 +212,8 @@ After modifying the logback configuration, restart the service for changes to ta 2. **Check for port conflicts:** ```bash lsof -i :7081 # Audit Server - lsof -i :7091 # Solr Consumer - lsof -i :7092 # HDFS Consumer + lsof -i :7091 # Solr Dispatcher + lsof -i :7092 # HDFS Dispatcher ``` 3. **Verify WAR file exists:** @@ -237,7 +239,7 @@ kill kill -9 # Remove stale PID file -rm -f logs/ranger-audit-server.pid +rm -f logs/ranger-audit-ingestor.pid ``` ### Java Not Found @@ -256,9 +258,9 @@ java -version ### Kafka Connection Issues Check Kafka bootstrap servers configuration in: -- `ranger-audit-server-service/src/main/resources/conf/ranger-audit-server-site.xml` -- `ranger-audit-consumer-solr/src/main/resources/conf/ranger-audit-consumer-solr-site.xml` -- `ranger-audit-consumer-hdfs/src/main/resources/conf/ranger-audit-consumer-hdfs-site.xml` +- `audit-ingestor/src/main/resources/conf/ranger-audit-ingestor-site.xml` +- `audit-dispatcher/dispatcher-solr/src/main/resources/conf/ranger-audit-dispatcher-solr-site.xml` +- `audit-dispatcher/dispatcher-hdfs/src/main/resources/conf/ranger-audit-dispatcher-hdfs-site.xml` ## Architecture @@ -285,7 +287,7 @@ Check Kafka bootstrap servers configuration in: ▼ ▼ ▼ ▼ ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ │ Solr │ │ HDFS │ │ New │ ... │ Nth │ -│ Consumer │ │ Consumer │ │ Consumer │ │ Consumer │ +│ Dispatcher │ │ Dispatcher │ │ Dispatcher │ │ Dispatcher │ │ (7091) │ │ (7092) │ │ (709N) │ │ (709N+1) │ └────┬─────┘ └────┬─────┘ └────┬─────┘ └────┬─────┘ │ │ │ │ @@ -300,221 +302,101 @@ Check Kafka bootstrap servers configuration in: To add a new audit destination (e.g., Elasticsearch, MongoDB, Cloud Storage, etc.), follow these steps: -### 1. Create a New Consumer Module +### 1. Create a New Dispatcher Module -Create a new Maven module in the `ranger-audit-server` directory: +Create a new Maven module in the `audit-server/audit-dispatcher` directory: ```bash -cd ranger-audit-server -mkdir ranger-audit-consumer- -cd ranger-audit-consumer- +cd audit-server/audit-dispatcher +mkdir dispatcher- +cd dispatcher- ``` -Create a `pom.xml` based on the existing consumers (Solr or HDFS). Key dependencies: -- Spring Boot Starter -- Spring Kafka +Create a `pom.xml` based on the existing dispatchers (Solr or HDFS). Key dependencies: +- `ranger-audit-dispatcher-common` (provided scope) - Your destination-specific client library (e.g., Elasticsearch client, MongoDB driver) -### 2. Implement the Consumer Application +### 2. Implement the Dispatcher Manager and Kafka Dispatcher -Create the main Spring Boot application class: +Create a `DispatcherManager` class that implements the singleton pattern and a `KafkaDispatcher` class that extends `AuditDispatcherBase`: ```java -package org.apache.ranger.audit.consumer; +package org.apache.ranger.audit.dispatcher; -import org.springframework.boot.SpringApplication; -import org.springframework.boot.autoconfigure.SpringBootApplication; - -@SpringBootApplication -public class YourDestinationConsumerApplication { - public static void main(String[] args) { - SpringApplication.run(YourDestinationConsumerApplication.class, args); - } +public class YourDestinationDispatcherManager { + // Implement singleton and initialization logic + // See SolrDispatcherManager for reference } ``` -### 3. Create the Kafka Consumer - -Implement a Kafka consumer to read audit events: - ```java -package org.apache.ranger.audit.consumer; - -import org.springframework.kafka.annotation.KafkaListener; -import org.springframework.stereotype.Service; - -@Service -public class YourDestinationConsumer { - @KafkaListener(topics = "${ranger.audit.kafka.topic:ranger_audits}", groupId = "${ranger.audit.kafka.consumer.group:audit-consumer-your-destination}") - public void consumeAudit(String auditEvent) { - // Parse audit event - // Transform if needed - // Write to your destination - } +package org.apache.ranger.audit.dispatcher.kafka; + +public class AuditYourDestinationDispatcher extends AuditDispatcherBase { + // Implement consume logic + // See AuditSolrDispatcher for reference } ``` -### 4. Add Configuration Files +### 3. Add Configuration Files Create configuration files in `src/main/resources/conf/`: -**ranger-audit-consumer--site.xml:** +**ranger-audit-dispatcher--site.xml:** ```xml - ranger.audit.kafka.bootstrap.servers - localhost:9092 + ranger.audit.dispatcher.type + {your-destination} - ranger.audit.kafka.topic - ranger_audits - - - ranger.audit.your-destination.url - http://localhost:PORT + ranger.audit.dispatcher.{your-destination}.class + org.apache.ranger.audit.dispatcher.{your-destination}DispatcherManager ``` -**application.yml:** -```yaml -server: - port: 709X # Choose next available port (e.g., 7093, 7094...) - -spring: - kafka: - bootstrap-servers: ${ranger.audit.kafka.bootstrap.servers:localhost:9092} - consumer: - group-id: audit-consumer-your-destination - auto-offset-reset: earliest - key-deserializer: org.apache.kafka.common.serialization.StringDeserializer - value-deserializer: org.apache.kafka.common.serialization.StringDeserializer - -# Add destination-specific Spring configurations -``` - -### 5. Create Start/Stop Scripts +### 4. Update Parent POM and Assembly -Create a `scripts` directory with start/stop scripts: - -**scripts/start-consumer-.sh:** -```bash -#!/bin/bash - -SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -SERVICE_DIR="$(dirname "$SCRIPT_DIR")" - -# Environment variables -AUDIT_CONSUMER_HOME_DIR="${AUDIT_CONSUMER_HOME_DIR:-$SERVICE_DIR/target}" -AUDIT_CONSUMER_CONF_DIR="${AUDIT_CONSUMER_CONF_DIR:-$SERVICE_DIR/src/main/resources/conf}" -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-$SERVICE_DIR/logs}" -AUDIT_CONSUMER_HEAP="${AUDIT_CONSUMER_HEAP:--Xms512m -Xmx2g}" -AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS:-}" -KERBEROS_ENABLED="${KERBEROS_ENABLED:-false}" - -# Find WAR file -WAR_FILE=$(find "$AUDIT_CONSUMER_HOME_DIR" -name "ranger-audit-consumer-*.war" | head -1) - -if [ -z "$WAR_FILE" ]; then - echo "Error: WAR file not found in $AUDIT_CONSUMER_HOME_DIR" - exit 1 -fi - -# Start service -java $AUDIT_CONSUMER_HEAP $AUDIT_CONSUMER_OPTS \ - -Dlog.dir="$AUDIT_CONSUMER_LOG_DIR" \ - -Dconf.dir="$AUDIT_CONSUMER_CONF_DIR" \ - -jar "$WAR_FILE" > "$AUDIT_CONSUMER_LOG_DIR/catalina.out" 2>&1 & - -echo $! > "$AUDIT_CONSUMER_LOG_DIR/ranger-audit-consumer-.pid" -echo "Started Ranger Audit Consumer () with PID: $(cat $AUDIT_CONSUMER_LOG_DIR/ranger-audit-consumer-.pid)" -``` - -**scripts/stop-consumer-.sh:** -```bash -#!/bin/bash - -SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -SERVICE_DIR="$(dirname "$SCRIPT_DIR")" -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-$SERVICE_DIR/logs}" -PID_FILE="$AUDIT_CONSUMER_LOG_DIR/ranger-audit-consumer-.pid" - -if [ -f "$PID_FILE" ]; then - PID=$(cat "$PID_FILE") - kill "$PID" - echo "Stopped Ranger Audit Consumer () with PID: $PID" - rm -f "$PID_FILE" -else - echo "PID file not found. Service may not be running." -fi -``` - -Make scripts executable: -```bash -chmod +x scripts/*.sh -``` - -### 6. Update Parent POM - -Add the new module to the parent `ranger-audit-server/pom.xml`: +Add the new module to the parent `audit-server/audit-dispatcher/pom.xml`: ```xml - ranger-audit-server-service - ranger-audit-consumer-solr - ranger-audit-consumer-hdfs - ranger-audit-consumer- + dispatcher-common + dispatcher-solr + dispatcher-hdfs + dispatcher- + dispatcher-app ``` -### 7. Update Start/Stop All Scripts - -Add your consumer to `scripts/start-all-services.sh`: - -```bash -# Start Your Destination Consumer -echo "Starting Ranger Audit Consumer ()..." -cd "$BASE_DIR/ranger-audit-consumer-" -./scripts/start-consumer-.sh -echo "Waiting 5 seconds for consumer to initialize..." -sleep 5 -``` - -Add to `scripts/stop-all-services.sh`: +Update `distro/src/main/assembly/audit-dispatcher.xml` to include the new module's JARs in `lib/dispatchers/`. -```bash -# Stop Your Destination Consumer -echo "Stopping Ranger Audit Consumer ()..." -cd "$BASE_DIR/ranger-audit-consumer-" -./scripts/stop-consumer-.sh -``` - -### 8. Build and Test +### 5. Build and Test ```bash -# Build the new consumer -cd ranger-audit-consumer- +# Build the project mvn clean package -DskipTests # Test individually -./scripts/start-consumer-.sh +./audit-dispatcher/scripts/start-audit-dispatcher.sh -# Check health (implement a health endpoint) -curl http://localhost:709X/api/health +# Check health +curl http://localhost:709X/api/health/ping # View logs -tail -f logs/ranger-audit-consumer-.log +tail -f audit-dispatcher/logs/ranger-audit-dispatcher.log # Stop when done -./scripts/stop-consumer-.sh +./audit-dispatcher/scripts/stop-audit-dispatcher.sh ``` -### 9. Add Documentation +### 6. Add Documentation Update this README to include: -- The new consumer in the "Overview" section +- The new dispatcher in the "Overview" section - Individual start/stop commands - Default port and health check endpoint - Configuration details specific to the destination @@ -527,13 +409,10 @@ Update this README to include: ```bash # Build specific service -cd ranger-audit-server-service -mvn clean package - -cd ../ranger-audit-consumer-solr +cd audit-ingestor mvn clean package -cd ../ranger-audit-consumer-hdfs +cd ../audit-dispatcher mvn clean package ``` @@ -543,7 +422,7 @@ Add debug options to the `OPTS` environment variable: ```bash export AUDIT_SERVER_OPTS="-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:5005" -./ranger-audit-server-service/scripts/start-audit-server.sh +./audit-ingestor/scripts/start-audit-ingestor.sh ``` Then attach your IDE debugger to port 5005. diff --git a/audit-server/scripts/start-all-services.sh b/audit-server/scripts/start-all-services.sh index cd172f8f18..af9e20d8fa 100755 --- a/audit-server/scripts/start-all-services.sh +++ b/audit-server/scripts/start-all-services.sh @@ -20,9 +20,9 @@ # Start All Ranger Audit Services # ======================================== # This script starts all three audit services in the correct order: -# 1. Audit Server (receives audits and produces to Kafka) -# 2. Solr Consumer (consumes from Kafka and indexes to Solr) -# 3. HDFS Consumer (consumes from Kafka and writes to HDFS) +# 1. Audit Ingestor (receives audits and produces to Kafka) +# 2. Solr Dispatcher (consumes from Kafka and indexes to Solr) +# 3. HDFS Dispatcher (consumes from Kafka and writes to HDFS) set -e @@ -34,45 +34,51 @@ echo "Starting All Ranger Audit Services" echo "==========================================" echo "" -# Start Audit Server -echo "[1/3] Starting Ranger Audit Server Service..." -if [ -f "${PARENT_DIR}/ranger-audit-server-service/scripts/start-audit-server.sh" ]; then - bash "${PARENT_DIR}/ranger-audit-server-service/scripts/start-audit-server.sh" +# Start Audit Ingestor +echo "[1/3] Starting Ranger Audit Ingestor Service..." +if [ -f "${PARENT_DIR}/audit-ingestor/scripts/start-audit-ingestor.sh" ]; then + bash "${PARENT_DIR}/audit-ingestor/scripts/start-audit-ingestor.sh" echo "" - echo "Waiting 10 seconds for Audit Server to initialize..." + echo "Waiting 10 seconds for Audit Ingestor to initialize..." sleep 10 else - echo "[ERROR] Audit Server start script not found" + echo "[ERROR] Audit Ingestor start script not found" exit 1 fi -# Start Solr Consumer -echo "[2/3] Starting Ranger Audit Consumer - Solr..." -if [ -f "${PARENT_DIR}/ranger-audit-consumer-solr/scripts/start-consumer-solr.sh" ]; then - bash "${PARENT_DIR}/ranger-audit-consumer-solr/scripts/start-consumer-solr.sh" +# Start Solr Dispatcher +echo "[2/3] Starting Ranger Audit Dispatcher - Solr..." +if [ -f "${PARENT_DIR}/audit-dispatcher/scripts/start-audit-dispatcher.sh" ]; then + nohup bash "${PARENT_DIR}/audit-dispatcher/scripts/start-audit-dispatcher.sh" solr > "${PARENT_DIR}/audit-dispatcher/logs/start-solr.log" 2>&1 & + PID=$! + echo $PID > "${PARENT_DIR}/audit-dispatcher/logs/ranger-audit-dispatcher-solr.pid" + echo "Started Solr Dispatcher with PID: $PID" echo "" - echo "Waiting 5 seconds for Solr Consumer to initialize..." + echo "Waiting 5 seconds for Solr Dispatcher to initialize..." sleep 5 else - echo "[WARNING] Solr Consumer start script not found, skipping..." + echo "[WARNING] Solr Dispatcher start script not found, skipping..." fi -# Start HDFS Consumer -echo "[3/3] Starting Ranger Audit Consumer - HDFS..." -if [ -f "${PARENT_DIR}/ranger-audit-consumer-hdfs/scripts/start-consumer-hdfs.sh" ]; then - bash "${PARENT_DIR}/ranger-audit-consumer-hdfs/scripts/start-consumer-hdfs.sh" +# Start HDFS Dispatcher +echo "[3/3] Starting Ranger Audit Dispatcher - HDFS..." +if [ -f "${PARENT_DIR}/audit-dispatcher/scripts/start-audit-dispatcher.sh" ]; then + nohup bash "${PARENT_DIR}/audit-dispatcher/scripts/start-audit-dispatcher.sh" hdfs > "${PARENT_DIR}/audit-dispatcher/logs/start-hdfs.log" 2>&1 & + PID=$! + echo $PID > "${PARENT_DIR}/audit-dispatcher/logs/ranger-audit-dispatcher-hdfs.pid" + echo "Started HDFS Dispatcher with PID: $PID" echo "" else - echo "[WARNING] HDFS Consumer start script not found, skipping..." + echo "[WARNING] HDFS Dispatcher start script not found, skipping..." fi echo "==========================================" -echo "✓ All Ranger Audit Services Started" +echo "All Ranger Audit Services Started" echo "==========================================" echo "" echo "Service Endpoints:" -echo " - Audit Server: http://localhost:7081/api/audit/health" -echo " - Solr Consumer: http://localhost:7091/api/health" -echo " - HDFS Consumer: http://localhost:7092/api/health" +echo " - Audit Ingestor: http://localhost:7081/api/audit/health" +echo " - Solr Dispatcher: http://localhost:7091/api/health" +echo " - HDFS Dispatcher: http://localhost:7092/api/health" echo "" echo "To stop all services: ${SCRIPT_DIR}/stop-all-services.sh" diff --git a/audit-server/scripts/stop-all-services.sh b/audit-server/scripts/stop-all-services.sh index 6bd0124c19..7c9ca5a047 100755 --- a/audit-server/scripts/stop-all-services.sh +++ b/audit-server/scripts/stop-all-services.sh @@ -20,9 +20,9 @@ # Stop All Ranger Audit Services # ======================================== # This script stops all three audit services in the reverse order: -# 1. HDFS Consumer -# 2. Solr Consumer -# 3. Audit Server +# 1. HDFS Dispatcher +# 2. Solr Dispatcher +# 3. Audit Ingestor set -e @@ -34,33 +34,33 @@ echo "Stopping All Ranger Audit Services" echo "==========================================" echo "" -# Stop HDFS Consumer -echo "[1/3] Stopping Ranger Audit Consumer - HDFS..." -if [ -f "${PARENT_DIR}/ranger-audit-consumer-hdfs/scripts/stop-consumer-hdfs.sh" ]; then - bash "${PARENT_DIR}/ranger-audit-consumer-hdfs/scripts/stop-consumer-hdfs.sh" || true +# Stop HDFS Dispatcher +echo "[1/3] Stopping Ranger Audit Dispatcher - HDFS..." +if [ -f "${PARENT_DIR}/audit-dispatcher/scripts/stop-audit-dispatcher.sh" ]; then + bash "${PARENT_DIR}/audit-dispatcher/scripts/stop-audit-dispatcher.sh" hdfs || true echo "" else - echo "[WARNING] HDFS Consumer stop script not found, skipping..." + echo "[WARNING] HDFS Dispatcher stop script not found, skipping..." fi -# Stop Solr Consumer -echo "[2/3] Stopping Ranger Audit Consumer - Solr..." -if [ -f "${PARENT_DIR}/ranger-audit-consumer-solr/scripts/stop-consumer-solr.sh" ]; then - bash "${PARENT_DIR}/ranger-audit-consumer-solr/scripts/stop-consumer-solr.sh" || true +# Stop Solr Dispatcher +echo "[2/3] Stopping Ranger Audit Dispatcher - Solr..." +if [ -f "${PARENT_DIR}/audit-dispatcher/scripts/stop-audit-dispatcher.sh" ]; then + bash "${PARENT_DIR}/audit-dispatcher/scripts/stop-audit-dispatcher.sh" solr || true echo "" else - echo "[WARNING] Solr Consumer stop script not found, skipping..." + echo "[WARNING] Solr Dispatcher stop script not found, skipping..." fi -# Stop Audit Server -echo "[3/3] Stopping Ranger Audit Server Service..." -if [ -f "${PARENT_DIR}/ranger-audit-server-service/scripts/stop-audit-server.sh" ]; then - bash "${PARENT_DIR}/ranger-audit-server-service/scripts/stop-audit-server.sh" || true +# Stop Audit Ingestor +echo "[3/3] Stopping Ranger Audit Ingestor Service..." +if [ -f "${PARENT_DIR}/audit-ingestor/scripts/stop-audit-ingestor.sh" ]; then + bash "${PARENT_DIR}/audit-ingestor/scripts/stop-audit-ingestor.sh" || true echo "" else - echo "[WARNING] Audit Server stop script not found, skipping..." + echo "[WARNING] Audit Ingestor stop script not found, skipping..." fi echo "==========================================" -echo "✓ All Ranger Audit Services Stopped" +echo "All Ranger Audit Services Stopped" echo "==========================================" diff --git a/dev-support/ranger-docker/.dockerignore b/dev-support/ranger-docker/.dockerignore index 6e66a678ee..05026e91e9 100644 --- a/dev-support/ranger-docker/.dockerignore +++ b/dev-support/ranger-docker/.dockerignore @@ -5,10 +5,9 @@ !dist/ranger-*-kms.tar.gz !dist/ranger-*-usersync.tar.gz !dist/ranger-*-tagsync.tar.gz +!dist/ranger-*-audit-ingestor.tar.gz +!dist/ranger-*-audit-dispatcher.tar.gz !dist/ranger-*-pdp.tar.gz -!dist/ranger-*-audit-server.tar.gz -!dist/ranger-*-audit-consumer-solr.tar.gz -!dist/ranger-*-audit-consumer-hdfs.tar.gz !dist/ranger-*-hdfs-plugin.tar.gz !dist/ranger-*-yarn-plugin.tar.gz !dist/ranger-*-hive-plugin.tar.gz diff --git a/dev-support/ranger-docker/Dockerfile.ranger-audit-consumer-hdfs b/dev-support/ranger-docker/Dockerfile.ranger-audit-consumer-hdfs deleted file mode 100644 index be88e169c1..0000000000 --- a/dev-support/ranger-docker/Dockerfile.ranger-audit-consumer-hdfs +++ /dev/null @@ -1,85 +0,0 @@ -# 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. - -ARG RANGER_BASE_IMAGE -ARG RANGER_BASE_VERSION -FROM ${RANGER_BASE_IMAGE}:${RANGER_BASE_VERSION} - -# Declare ARG for use in this build stage -ARG RANGER_VERSION -ARG KERBEROS_ENABLED - -# Install required packages including Kerberos client -RUN apt-get update && apt-get install -y \ - python3 \ - python3-pip \ - curl \ - vim \ - net-tools \ - krb5-user \ - && rm -rf /var/lib/apt/lists/* - -# Volume for keytabs -VOLUME /etc/keytabs - -# Set up working directory -WORKDIR /opt/ranger/audit-consumer-hdfs - -# Copy ranger-audit-consumer-hdfs distribution -COPY ./dist/ranger-${RANGER_VERSION}-audit-consumer-hdfs.tar.gz /tmp/ - -# Copy scripts, Hadoop configs, and Kerberos configuration -COPY ./scripts/audit-server/service-check-functions.sh ${RANGER_SCRIPTS}/ -COPY ./scripts/audit-server/ranger-audit-consumer-hdfs.sh ${RANGER_SCRIPTS}/ -COPY ./scripts/kdc/krb5.conf /etc/krb5.conf - -# Extract and setup -RUN tar xzf /tmp/ranger-${RANGER_VERSION}-audit-consumer-hdfs.tar.gz -C /opt/ranger/audit-consumer-hdfs --strip-components=1 && \ - rm -f /tmp/ranger-${RANGER_VERSION}-audit-consumer-hdfs.tar.gz && \ - mkdir -p /var/log/ranger/audit-consumer-hdfs && \ - mkdir -p /home/ranger/scripts && \ - rm -rf /opt/ranger/audit-consumer-hdfs/logs && \ - ln -sf /var/log/ranger/audit-consumer-hdfs /opt/ranger/audit-consumer-hdfs/logs && \ - chown -R ranger:ranger /opt/ranger/audit-consumer-hdfs/ /var/log/ranger/ ${RANGER_SCRIPTS} && \ - chmod -R 755 /opt/ranger/audit-consumer-hdfs && \ - chmod 755 ${RANGER_SCRIPTS}/ranger-audit-consumer-hdfs.sh - -# Set Java home -ENV JAVA_HOME=/opt/java/openjdk -ENV PATH=$JAVA_HOME/bin:$PATH - -# Environment variables for HDFS consumer -ENV AUDIT_CONSUMER_HOME_DIR=/opt/ranger/audit-consumer-hdfs -ENV AUDIT_CONSUMER_CONF_DIR=/opt/ranger/audit-consumer-hdfs/conf -ENV AUDIT_CONSUMER_LOG_DIR=/var/log/ranger/audit-consumer-hdfs -ENV RANGER_USER=ranger - -# Expose health check port -EXPOSE 7092 - -# Health check - test the health endpoint -HEALTHCHECK --interval=30s --timeout=10s --start-period=60s --retries=3 \ - CMD curl -f -o /dev/null -s -w "%{http_code}" http://localhost:7092/api/health | grep -q "200" || exit 1 - -# Environment variable for Kerberos support -ENV KERBEROS_ENABLED=${KERBEROS_ENABLED:-false} - -# Switch to ranger user -USER ranger - -# Start the HDFS consumer using the custom startup script -WORKDIR /opt/ranger/audit-consumer-hdfs -ENTRYPOINT ["/home/ranger/scripts/ranger-audit-consumer-hdfs.sh"] diff --git a/dev-support/ranger-docker/Dockerfile.ranger-audit-consumer-solr b/dev-support/ranger-docker/Dockerfile.ranger-audit-consumer-solr deleted file mode 100644 index bdf210cf10..0000000000 --- a/dev-support/ranger-docker/Dockerfile.ranger-audit-consumer-solr +++ /dev/null @@ -1,86 +0,0 @@ -# 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. - -ARG RANGER_BASE_IMAGE -ARG RANGER_BASE_VERSION -FROM ${RANGER_BASE_IMAGE}:${RANGER_BASE_VERSION} - -# Declare ARG for use in this build stage -ARG RANGER_VERSION -ARG KERBEROS_ENABLED - -# Install required packages including Kerberos client -RUN apt-get update && apt-get install -y \ - python3 \ - python3-pip \ - curl \ - vim \ - net-tools \ - krb5-user \ - && rm -rf /var/lib/apt/lists/* - -# Volume for keytabs -VOLUME /etc/keytabs - -# Set up working directory -WORKDIR /opt/ranger/audit-consumer-solr - -# Copy ranger-audit-consumer-solr distribution -COPY ./dist/ranger-${RANGER_VERSION}-audit-consumer-solr.tar.gz /tmp/ - -# Copy scripts and Kerberos configuration -COPY ./scripts/audit-server/service-check-functions.sh ${RANGER_SCRIPTS}/ -COPY ./scripts/audit-server/ranger-audit-consumer-solr.sh ${RANGER_SCRIPTS}/ -COPY ./scripts/kdc/krb5.conf /etc/krb5.conf - -# Extract and setup -RUN tar xzf /tmp/ranger-${RANGER_VERSION}-audit-consumer-solr.tar.gz -C /opt/ranger/audit-consumer-solr --strip-components=1 && \ - rm -f /tmp/ranger-${RANGER_VERSION}-audit-consumer-solr.tar.gz && \ - mkdir -p /var/log/ranger/audit-consumer-solr && \ - mkdir -p /home/ranger/scripts && \ - rm -rf /opt/ranger/audit-consumer-solr/logs && \ - ln -sf /var/log/ranger/audit-consumer-solr /opt/ranger/audit-consumer-solr/logs && \ - chown -R ranger:ranger /opt/ranger/audit-consumer-solr/ /var/log/ranger/ ${RANGER_SCRIPTS} && \ - chmod -R 755 /opt/ranger/audit-consumer-solr && \ - chmod 755 ${RANGER_SCRIPTS}/ranger-audit-consumer-solr.sh && \ - chmod 755 ${RANGER_SCRIPTS}/wait_for_keytab.sh - -# Set Java home -ENV JAVA_HOME=/opt/java/openjdk -ENV PATH=$JAVA_HOME/bin:$PATH - -# Environment variables for Solr consumer -ENV AUDIT_CONSUMER_HOME_DIR=/opt/ranger/audit-consumer-solr -ENV AUDIT_CONSUMER_CONF_DIR=/opt/ranger/audit-consumer-solr/conf -ENV AUDIT_CONSUMER_LOG_DIR=/var/log/ranger/audit-consumer-solr -ENV RANGER_USER=ranger - -# Expose health check port -EXPOSE 7091 - -# Health check - test the health endpoint -HEALTHCHECK --interval=30s --timeout=10s --start-period=60s --retries=3 \ - CMD curl -f -o /dev/null -s -w "%{http_code}" http://localhost:7091/api/health | grep -q "200" || exit 1 - -# Environment variable for Kerberos support -ENV KERBEROS_ENABLED=${KERBEROS_ENABLED:-false} - -# Switch to ranger user -USER ranger - -# Start the Solr consumer using the custom startup script -WORKDIR /opt/ranger/audit-consumer-solr -ENTRYPOINT ["/home/ranger/scripts/ranger-audit-consumer-solr.sh"] diff --git a/dev-support/ranger-docker/Dockerfile.ranger-audit-dispatcher b/dev-support/ranger-docker/Dockerfile.ranger-audit-dispatcher new file mode 100644 index 0000000000..5dee7cb98d --- /dev/null +++ b/dev-support/ranger-docker/Dockerfile.ranger-audit-dispatcher @@ -0,0 +1,79 @@ +# 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. + +ARG RANGER_BASE_IMAGE +ARG RANGER_BASE_VERSION +FROM ${RANGER_BASE_IMAGE}:${RANGER_BASE_VERSION} + +# Declare ARG for use in this build stage +ARG RANGER_VERSION +ARG KERBEROS_ENABLED + +# Install required packages including Kerberos client +RUN apt-get update && apt-get install -y \ + python3 \ + python3-pip \ + curl \ + vim \ + net-tools \ + krb5-user \ + && rm -rf /var/lib/apt/lists/* + +# Volume for keytabs +VOLUME /etc/keytabs + +# Set up working directory +WORKDIR /opt/ranger/audit-dispatcher + +# Copy audit-dispatcher distribution +COPY ./dist/ranger-${RANGER_VERSION}-audit-dispatcher.tar.gz /tmp/ + +# Copy scripts and Kerberos configuration +COPY ./scripts/audit-dispatcher/service-check-functions.sh ${RANGER_SCRIPTS}/ +COPY ./scripts/kdc/krb5.conf /etc/krb5.conf + +# Extract and setup +RUN tar xzf /tmp/ranger-${RANGER_VERSION}-audit-dispatcher.tar.gz -C /opt/ranger/audit-dispatcher --strip-components=1 && \ + rm -f /tmp/ranger-${RANGER_VERSION}-audit-dispatcher.tar.gz && \ + mkdir -p /var/log/ranger/audit-dispatcher && \ + mkdir -p /home/ranger/scripts && \ + rm -rf /opt/ranger/audit-dispatcher/logs && \ + ln -sf /var/log/ranger/audit-dispatcher /opt/ranger/audit-dispatcher/logs && \ + cp /opt/ranger/audit-dispatcher/scripts/start-audit-dispatcher.sh ${RANGER_SCRIPTS}/ && \ + chown -R ranger:ranger /opt/ranger/audit-dispatcher/ /var/log/ranger/ ${RANGER_SCRIPTS} && \ + chmod -R 755 /opt/ranger/audit-dispatcher && \ + chmod 755 ${RANGER_SCRIPTS}/start-audit-dispatcher.sh + +# Set Java home +ENV JAVA_HOME=/opt/java/openjdk +ENV PATH=$JAVA_HOME/bin:$PATH + +# Environment variables for dispatcher +ENV AUDIT_DISPATCHER_HOME_DIR=/opt/ranger/audit-dispatcher +ENV AUDIT_DISPATCHER_CONF_DIR=/opt/ranger/audit-dispatcher/conf +ENV AUDIT_DISPATCHER_LOG_DIR=/var/log/ranger/audit-dispatcher +ENV RANGER_USER=ranger +ENV HADOOP_HOME=/opt/ranger/audit-dispatcher + +# Environment variable for Kerberos support +ENV KERBEROS_ENABLED=${KERBEROS_ENABLED:-false} + +# Switch to ranger user +USER ranger + +# Start the dispatcher using the custom startup script +WORKDIR /opt/ranger/audit-dispatcher +ENTRYPOINT ["/home/ranger/scripts/start-audit-dispatcher.sh"] diff --git a/dev-support/ranger-docker/Dockerfile.ranger-audit-server b/dev-support/ranger-docker/Dockerfile.ranger-audit-ingestor similarity index 59% rename from dev-support/ranger-docker/Dockerfile.ranger-audit-server rename to dev-support/ranger-docker/Dockerfile.ranger-audit-ingestor index 8cdec0d6ba..4db9239106 100644 --- a/dev-support/ranger-docker/Dockerfile.ranger-audit-server +++ b/dev-support/ranger-docker/Dockerfile.ranger-audit-ingestor @@ -36,36 +36,36 @@ RUN apt-get update && apt-get install -y \ VOLUME /etc/keytabs # Set up working directory -WORKDIR /opt/ranger/audit-server +WORKDIR /opt/ranger/audit-ingestor -# Copy ranger-audit-server distribution -COPY ./dist/ranger-${RANGER_VERSION}-audit-server.tar.gz /tmp/ +# Copy ranger-audit-ingestor distribution +COPY ./dist/ranger-${RANGER_VERSION}-audit-ingestor.tar.gz /tmp/ # Copy scripts and Kerberos configuration -COPY ./scripts/audit-server/service-check-functions.sh ${RANGER_SCRIPTS}/ -COPY ./scripts/audit-server/ranger-audit-server.sh ${RANGER_SCRIPTS}/ +COPY ./scripts/audit-ingestor/service-check-functions.sh ${RANGER_SCRIPTS}/ +COPY ./scripts/audit-ingestor/ranger-audit-ingestor.sh ${RANGER_SCRIPTS}/ COPY ./scripts/kdc/krb5.conf /etc/krb5.conf # Extract and setup -RUN tar xzf /tmp/ranger-${RANGER_VERSION}-audit-server.tar.gz -C /opt/ranger/audit-server --strip-components=1 && \ - rm -f /tmp/ranger-${RANGER_VERSION}-audit-server.tar.gz && \ - mkdir -p /var/log/ranger/audit-server/audit/spool && \ - mkdir -p /var/log/ranger/audit-server/audit/archive && \ +RUN tar xzf /tmp/ranger-${RANGER_VERSION}-audit-ingestor.tar.gz -C /opt/ranger/audit-ingestor --strip-components=1 && \ + rm -f /tmp/ranger-${RANGER_VERSION}-audit-ingestor.tar.gz && \ + mkdir -p /var/log/ranger/audit-ingestor/audit/spool && \ + mkdir -p /var/log/ranger/audit-ingestor/audit/archive && \ mkdir -p /home/ranger/scripts && \ - rm -rf /opt/ranger/audit-server/logs && \ - ln -sf /var/log/ranger/audit-server /opt/ranger/audit-server/logs && \ - chown -R ranger:ranger /opt/ranger/audit-server/ /var/log/ranger/ ${RANGER_SCRIPTS} && \ - chmod -R 755 /opt/ranger/audit-server && \ - chmod 755 ${RANGER_SCRIPTS}/ranger-audit-server.sh + rm -rf /opt/ranger/audit-ingestor/logs && \ + ln -sf /var/log/ranger/audit-ingestor /opt/ranger/audit-ingestor/logs && \ + chown -R ranger:ranger /opt/ranger/audit-ingestor/ /var/log/ranger/ ${RANGER_SCRIPTS} && \ + chmod -R 755 /opt/ranger/audit-ingestor && \ + chmod 755 ${RANGER_SCRIPTS}/ranger-audit-ingestor.sh # Set Java home ENV JAVA_HOME=/opt/java/openjdk ENV PATH=$JAVA_HOME/bin:$PATH -# Environment variables for audit server -ENV AUDIT_SERVER_HOME_DIR=/opt/ranger/audit-server -ENV AUDIT_SERVER_CONF_DIR=/opt/ranger/audit-server/conf -ENV AUDIT_SERVER_LOG_DIR=/var/log/ranger/audit-server +# Environment variables for audit ingestor +ENV AUDIT_INGESTOR_HOME_DIR=/opt/ranger/audit-ingestor +ENV AUDIT_INGESTOR_CONF_DIR=/opt/ranger/audit-ingestor/conf +ENV AUDIT_INGESTOR_LOG_DIR=/var/log/ranger/audit-ingestor ENV RANGER_USER=ranger # Expose ports (HTTP and HTTPS) @@ -81,6 +81,6 @@ ENV KERBEROS_ENABLED=${KERBEROS_ENABLED:-false} # Switch to ranger user USER ranger -# Start the audit server using the custom startup script -WORKDIR /opt/ranger/audit-server -ENTRYPOINT ${RANGER_SCRIPTS}/ranger-audit-server.sh +# Start the audit ingestor using the custom startup script +WORKDIR /opt/ranger/audit-ingestor +ENTRYPOINT ${RANGER_SCRIPTS}/ranger-audit-ingestor.sh diff --git a/dev-support/ranger-docker/README.md b/dev-support/ranger-docker/README.md index b0156d61c7..8015c32c98 100644 --- a/dev-support/ranger-docker/README.md +++ b/dev-support/ranger-docker/README.md @@ -121,5 +121,5 @@ docker compose -f docker-compose.ranger.yml -f docker-compose.ranger-usersync.ym #### To bring up audit server, solr and hdfs consumer. Make sure kafka,solr and hdfs containers are running before bring up audit server. ~~~ -docker compose -f docker-compose.ranger.yml -f docker-compose.ranger-hadoop.yml -f docker-compose.ranger-kafka.yml -f docker-compose.ranger-audit-server.yml up -d +docker compose -f docker-compose.ranger.yml -f docker-compose.ranger-hadoop.yml -f docker-compose.ranger-kafka.yml -f docker-compose.ranger-audit-ingestor.yml up -d ~~~ diff --git a/dev-support/ranger-docker/docker-compose.ranger-audit-server.yml b/dev-support/ranger-docker/docker-compose.ranger-audit-ingestor.yml similarity index 54% rename from dev-support/ranger-docker/docker-compose.ranger-audit-server.yml rename to dev-support/ranger-docker/docker-compose.ranger-audit-ingestor.yml index 924f125cfa..158c5d60d6 100644 --- a/dev-support/ranger-docker/docker-compose.ranger-audit-server.yml +++ b/dev-support/ranger-docker/docker-compose.ranger-audit-ingestor.yml @@ -15,19 +15,23 @@ # limitations under the License. services: - # Core Audit Server Service - Receives audits from plugins and produces to Kafka - ranger-audit-server: + # Audit Ingestor Service - Receives audits from plugins and produces to Kafka (REFACTORED) + ranger-audit-ingestor: build: context: . - dockerfile: Dockerfile.ranger-audit-server + dockerfile: Dockerfile.ranger-audit-ingestor args: - RANGER_BASE_IMAGE=${RANGER_BASE_IMAGE} - RANGER_BASE_VERSION=${RANGER_BASE_VERSION} - RANGER_VERSION=${RANGER_VERSION} - KERBEROS_ENABLED=${KERBEROS_ENABLED} - image: ranger-audit-server:latest - container_name: ranger-audit-server - hostname: ranger-audit-server.rangernw + image: ranger-audit-ingestor:latest + container_name: ranger-audit-ingestor + hostname: ranger-audit-ingestor.rangernw + networks: + ranger: + aliases: + - ranger-audit-ingestor.rangernw stdin_open: true tty: true depends_on: @@ -40,19 +44,17 @@ services: - "7182:7182" environment: JAVA_HOME: /opt/java/openjdk - AUDIT_SERVER_HOME_DIR: /opt/ranger/audit-server - AUDIT_SERVER_CONF_DIR: /opt/ranger/audit-server/conf - AUDIT_SERVER_LOG_DIR: /var/log/ranger/audit-server - AUDIT_SERVER_HEAP: "-Xms512m -Xmx2g" + AUDIT_INGESTOR_HOME_DIR: /opt/ranger/audit-ingestor + AUDIT_INGESTOR_CONF_DIR: /opt/ranger/audit-ingestor/conf + AUDIT_INGESTOR_LOG_DIR: /var/log/ranger/audit-ingestor + AUDIT_INGESTOR_HEAP: "-Xms512m -Xmx2g" RANGER_VERSION: ${RANGER_VERSION} KERBEROS_ENABLED: ${KERBEROS_ENABLED} KAFKA_BOOTSTRAP_SERVERS: ranger-kafka.rangernw:9092 - networks: - - ranger volumes: - - ./dist/keytabs/ranger-audit-server:/etc/keytabs - - ranger-audit-server-logs:/var/log/ranger/audit-server - - ranger-audit-server-spool:/var/log/ranger/audit-server/audit + - ./dist/keytabs/ranger-audit-ingestor:/etc/keytabs + - ranger-audit-ingestor-logs:/var/log/ranger/audit-ingestor + - ranger-audit-ingestor-spool:/var/log/ranger/audit-ingestor/audit healthcheck: test: ["CMD", "curl", "-f", "http://localhost:7081/api/audit/health"] interval: 30s @@ -60,19 +62,20 @@ services: retries: 3 start_period: 60s - # Solr Consumer Service - Consumes from Kafka and indexes to Solr - ranger-audit-consumer-solr: + # Solr Dispatcher Service (Unified Dispatcher) + ranger-audit-dispatcher-solr: build: context: . - dockerfile: Dockerfile.ranger-audit-consumer-solr + dockerfile: Dockerfile.ranger-audit-dispatcher args: - RANGER_BASE_IMAGE=${RANGER_BASE_IMAGE} - RANGER_BASE_VERSION=${RANGER_BASE_VERSION} - RANGER_VERSION=${RANGER_VERSION} - KERBEROS_ENABLED=${KERBEROS_ENABLED} - image: ranger-audit-consumer-solr:latest - container_name: ranger-audit-consumer-solr - hostname: ranger-audit-consumer-solr.rangernw + image: ranger-audit-dispatcher:latest + container_name: ranger-audit-dispatcher-solr + command: ["solr"] + hostname: ranger-audit-dispatcher-solr.rangernw stdin_open: true tty: true depends_on: @@ -82,16 +85,16 @@ services: condition: service_started ranger-solr: condition: service_started - ranger-audit-server: + ranger-audit-ingestor: condition: service_healthy ports: - - "7091:7091" + - "7091:7090" environment: JAVA_HOME: /opt/java/openjdk - AUDIT_CONSUMER_HOME_DIR: /opt/ranger/audit-consumer-solr - AUDIT_CONSUMER_CONF_DIR: /opt/ranger/audit-consumer-solr/conf - AUDIT_CONSUMER_LOG_DIR: /var/log/ranger/audit-consumer-solr - AUDIT_CONSUMER_HEAP: "-Xms512m -Xmx2g" + AUDIT_DISPATCHER_HOME_DIR: /opt/ranger/audit-dispatcher + AUDIT_DISPATCHER_CONF_DIR: /opt/ranger/audit-dispatcher/conf + AUDIT_DISPATCHER_LOG_DIR: /var/log/ranger/audit-dispatcher/solr + AUDIT_DISPATCHER_HEAP: "-Xms512m -Xmx2g" RANGER_VERSION: ${RANGER_VERSION} KERBEROS_ENABLED: ${KERBEROS_ENABLED} KAFKA_BOOTSTRAP_SERVERS: ranger-kafka.rangernw:9092 @@ -99,28 +102,30 @@ services: networks: - ranger volumes: - - ./dist/keytabs/ranger-audit-consumer-solr:/etc/keytabs - - ranger-audit-consumer-solr-logs:/var/log/ranger/audit-consumer-solr + - ./dist/keytabs/ranger-audit-dispatcher-solr:/etc/keytabs + - ranger-audit-dispatcher-solr-logs:/var/log/ranger/audit-dispatcher/solr + - ./scripts/audit-dispatcher/ranger-audit-dispatcher-solr-site.xml:/opt/ranger/audit-dispatcher/conf/ranger-audit-dispatcher-solr-site.xml healthcheck: - test: ["CMD", "curl", "-f", "http://localhost:7091/api/health"] + test: ["CMD", "curl", "-f", "http://localhost:7090/api/health/ping"] interval: 30s timeout: 10s retries: 3 start_period: 60s - # HDFS Consumer Service - Consumes from Kafka and writes to HDFS - ranger-audit-consumer-hdfs: + # HDFS Dispatcher Service (Unified Dispatcher) + ranger-audit-dispatcher-hdfs: build: context: . - dockerfile: Dockerfile.ranger-audit-consumer-hdfs + dockerfile: Dockerfile.ranger-audit-dispatcher args: - RANGER_BASE_IMAGE=${RANGER_BASE_IMAGE} - RANGER_BASE_VERSION=${RANGER_BASE_VERSION} - RANGER_VERSION=${RANGER_VERSION} - KERBEROS_ENABLED=${KERBEROS_ENABLED} - image: ranger-audit-consumer-hdfs:latest - container_name: ranger-audit-consumer-hdfs - hostname: ranger-audit-consumer-hdfs.rangernw + image: ranger-audit-dispatcher:latest + container_name: ranger-audit-dispatcher-hdfs + command: ["hdfs"] + hostname: ranger-audit-dispatcher-hdfs.rangernw stdin_open: true tty: true depends_on: @@ -130,16 +135,16 @@ services: condition: service_started ranger-hadoop: condition: service_healthy - ranger-audit-server: + ranger-audit-ingestor: condition: service_healthy ports: - - "7092:7092" + - "7092:7090" environment: JAVA_HOME: /opt/java/openjdk - AUDIT_CONSUMER_HOME_DIR: /opt/ranger/audit-consumer-hdfs - AUDIT_CONSUMER_CONF_DIR: /opt/ranger/audit-consumer-hdfs/conf - AUDIT_CONSUMER_LOG_DIR: /var/log/ranger/audit-consumer-hdfs - AUDIT_CONSUMER_HEAP: "-Xms512m -Xmx2g" + AUDIT_DISPATCHER_HOME_DIR: /opt/ranger/audit-dispatcher + AUDIT_DISPATCHER_CONF_DIR: /opt/ranger/audit-dispatcher/conf + AUDIT_DISPATCHER_LOG_DIR: /var/log/ranger/audit-dispatcher/hdfs + AUDIT_DISPATCHER_HEAP: "-Xms512m -Xmx2g" RANGER_VERSION: ${RANGER_VERSION} KERBEROS_ENABLED: ${KERBEROS_ENABLED} KAFKA_BOOTSTRAP_SERVERS: ranger-kafka.rangernw:9092 @@ -147,20 +152,21 @@ services: networks: - ranger volumes: - - ./dist/keytabs/ranger-audit-consumer-hdfs:/etc/keytabs - - ranger-audit-consumer-hdfs-logs:/var/log/ranger/audit-consumer-hdfs + - ./dist/keytabs/ranger-audit-dispatcher-hdfs:/etc/keytabs + - ranger-audit-dispatcher-hdfs-logs:/var/log/ranger/audit-dispatcher/hdfs + - ./scripts/audit-dispatcher/ranger-audit-dispatcher-hdfs-site.xml:/opt/ranger/audit-dispatcher/conf/ranger-audit-dispatcher-hdfs-site.xml healthcheck: - test: ["CMD", "curl", "-f", "http://localhost:7092/api/health"] + test: ["CMD", "curl", "-f", "http://localhost:7090/api/health/ping"] interval: 30s timeout: 10s retries: 3 start_period: 60s volumes: - ranger-audit-server-logs: - ranger-audit-server-spool: - ranger-audit-consumer-solr-logs: - ranger-audit-consumer-hdfs-logs: + ranger-audit-ingestor-logs: + ranger-audit-ingestor-spool: + ranger-audit-dispatcher-solr-logs: + ranger-audit-dispatcher-hdfs-logs: networks: ranger: diff --git a/audit-server/consumer-hdfs/src/main/resources/conf/ranger-audit-consumer-hdfs-site.xml b/dev-support/ranger-docker/scripts/audit-dispatcher/ranger-audit-dispatcher-hdfs-site.xml similarity index 59% rename from audit-server/consumer-hdfs/src/main/resources/conf/ranger-audit-consumer-hdfs-site.xml rename to dev-support/ranger-docker/scripts/audit-dispatcher/ranger-audit-dispatcher-hdfs-site.xml index 6de0e5601a..e6f2c92779 100644 --- a/audit-server/consumer-hdfs/src/main/resources/conf/ranger-audit-consumer-hdfs-site.xml +++ b/dev-support/ranger-docker/scripts/audit-dispatcher/ranger-audit-dispatcher-hdfs-site.xml @@ -12,66 +12,30 @@ limitations under the License. See accompanying LICENSE file. --> - - + - log.dir - ${audit.consumer.hdfs.log.dir} - Log directory for HDFS consumer service - - - - ranger.audit.consumer.hdfs.webapp.dir - webapp/ranger-audit-consumer-hdfs - Path to the extracted webapp directory - - - - ranger.audit.consumer.hdfs.contextName - / + ranger.audit.dispatcher.hdfs.class + org.apache.ranger.audit.dispatcher.HdfsDispatcherManager + + + + - ranger.audit.consumer.hdfs.host - ranger-audit-consumer-hdfs.rangernw - Service hostname + log.dir + ${audit.dispatcher.hdfs.log.dir} + Log directory for HDFS dispatcher service - - ranger.audit.consumer.hdfs.http.port - 7092 - Health check endpoint port (different from Solr: 7091) - - - ranger.audit.consumer.hdfs.authentication.method - KERBEROS - - - - ranger.audit.consumer.hdfs.host - ranger-audit-consumer-hdfs.rangernw - Hostname for resolving _HOST in Kerberos principal - - - - ranger.audit.consumer.hdfs.kerberos.principal - rangerauditserver/_HOST@EXAMPLE.COM - Principal for Kafka consumer and HDFS writes - - - - ranger.audit.consumer.hdfs.kerberos.keytab - /etc/keytabs/rangerauditserver.keytab - - - + xasecure.audit.destination.kafka.host - ranger-audit-consumer-hdfs.rangernw + ranger-audit-dispatcher-hdfs.rangernw - - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In Docker: Use full service name with domain (e.g., ranger-audit-ingestor.rangernw) - In VM: Use the actual FQDN (e.g., ranger.example.com) @@ -111,43 +75,43 @@ GSSAPI - + - xasecure.audit.destination.kafka.consumer.thread.count + xasecure.audit.destination.kafka.dispatcher.thread.count 3 - Number of HDFS consumer worker threads + Number of HDFS dispatcher worker threads - xasecure.audit.destination.kafka.consumer.offset.commit.strategy + xasecure.audit.destination.kafka.dispatcher.offset.commit.strategy batch batch or manual - xasecure.audit.destination.kafka.consumer.offset.commit.interval.ms + xasecure.audit.destination.kafka.dispatcher.offset.commit.interval.ms 30000 Used only if strategy is 'manual' - xasecure.audit.destination.kafka.consumer.max.poll.records + xasecure.audit.destination.kafka.dispatcher.max.poll.records 500 Maximum records per poll - + - xasecure.audit.destination.kafka.consumer.classes - org.apache.ranger.audit.consumer.kafka.AuditHDFSConsumer + xasecure.audit.destination.kafka.dispatcher.classes + org.apache.ranger.audit.dispatcher.kafka.AuditHDFSDispatcher xasecure.audit.destination.hdfs true - MUST be true for HDFS consumer to work + MUST be true for HDFS dispatcher to work @@ -162,7 +126,7 @@ xasecure.audit.destination.hdfs.batch.filespool.dir - /var/log/audit-consumer-hdfs/spool + /var/log/audit-dispatcher-hdfs/spool diff --git a/audit-server/consumer-solr/src/main/resources/conf/ranger-audit-consumer-solr-site.xml b/dev-support/ranger-docker/scripts/audit-dispatcher/ranger-audit-dispatcher-solr-site.xml similarity index 69% rename from audit-server/consumer-solr/src/main/resources/conf/ranger-audit-consumer-solr-site.xml rename to dev-support/ranger-docker/scripts/audit-dispatcher/ranger-audit-dispatcher-solr-site.xml index 2cd7873771..137ba9609e 100644 --- a/audit-server/consumer-solr/src/main/resources/conf/ranger-audit-consumer-solr-site.xml +++ b/dev-support/ranger-docker/scripts/audit-dispatcher/ranger-audit-dispatcher-solr-site.xml @@ -12,59 +12,29 @@ limitations under the License. See accompanying LICENSE file. --> - - - - log.dir - ${audit.consumer.solr.log.dir} - Log directory for Solr consumer service - - - - ranger.audit.consumer.solr.webapp.dir - webapp/ranger-audit-consumer-solr - Path to the extracted webapp directory - - + - ranger.audit.consumer.solr.contextName - / + ranger.audit.dispatcher.solr.class + org.apache.ranger.audit.dispatcher.SolrDispatcherManager + + + + - ranger.audit.consumer.solr.host - ranger-audit-consumer-solr.rangernw - Service hostname + log.dir + ${audit.dispatcher.solr.log.dir} + Log directory for Solr dispatcher service - - ranger.audit.consumer.solr.http.port - 7091 - Health check endpoint port - - - ranger.audit.consumer.solr.authentication.method - KERBEROS - - - - ranger.audit.consumer.solr.kerberos.principal - rangerauditserver/_HOST@EXAMPLE.COM - Principal for Kafka consumer and Solr authentication - - - - ranger.audit.consumer.solr.kerberos.keytab - /etc/keytabs/rangerauditserver.keytab - - xasecure.audit.destination.kafka.host - ranger-audit-consumer-solr.rangernw + ranger-audit-dispatcher-solr.rangernw - - In Docker: Use full service name with domain (e.g., ranger-audit-server.rangernw) + - In Docker: Use full service name with domain (e.g., ranger-audit-ingestor.rangernw) - In VM: Use the actual FQDN (e.g., ranger.example.com) @@ -85,7 +55,7 @@ - + xasecure.audit.destination.kafka.bootstrap.servers ranger-kafka:9092 @@ -106,36 +76,36 @@ GSSAPI - + - xasecure.audit.destination.kafka.consumer.thread.count + xasecure.audit.destination.kafka.dispatcher.thread.count 5 - Number of Solr consumer worker threads (higher for indexing throughput) + Number of Solr dispatcher worker threads (higher for indexing throughput) - xasecure.audit.destination.kafka.consumer.offset.commit.strategy + xasecure.audit.destination.kafka.dispatcher.offset.commit.strategy batch batch or manual - xasecure.audit.destination.kafka.consumer.offset.commit.interval.ms + xasecure.audit.destination.kafka.dispatcher.offset.commit.interval.ms 30000 Used only if strategy is 'manual' - xasecure.audit.destination.kafka.consumer.max.poll.records + xasecure.audit.destination.kafka.dispatcher.max.poll.records 500 Maximum records per poll for batch processing - + - xasecure.audit.destination.kafka.consumer.classes - org.apache.ranger.audit.consumer.kafka.AuditSolrConsumer + xasecure.audit.destination.kafka.dispatcher.classes + org.apache.ranger.audit.dispatcher.kafka.AuditSolrDispatcher @@ -143,7 +113,7 @@ xasecure.audit.destination.solr true - MUST be true for Solr consumer to work + MUST be true for Solr dispatcher to work @@ -168,7 +138,7 @@ xasecure.audit.destination.solr.batch.filespool.dir - /var/log/audit-consumer-solr/spool + /var/log/audit-dispatcher-solr/spool diff --git a/dev-support/ranger-docker/scripts/audit-server/service-check-functions.sh b/dev-support/ranger-docker/scripts/audit-dispatcher/service-check-functions.sh similarity index 100% rename from dev-support/ranger-docker/scripts/audit-server/service-check-functions.sh rename to dev-support/ranger-docker/scripts/audit-dispatcher/service-check-functions.sh diff --git a/dev-support/ranger-docker/scripts/audit-server/ranger-audit-server.sh b/dev-support/ranger-docker/scripts/audit-ingestor/ranger-audit-ingestor.sh similarity index 72% rename from dev-support/ranger-docker/scripts/audit-server/ranger-audit-server.sh rename to dev-support/ranger-docker/scripts/audit-ingestor/ranger-audit-ingestor.sh index bc029d98c2..f883f9e3f3 100755 --- a/dev-support/ranger-docker/scripts/audit-server/ranger-audit-server.sh +++ b/dev-support/ranger-docker/scripts/audit-ingestor/ranger-audit-ingestor.sh @@ -18,16 +18,17 @@ set -e -AUDIT_SERVER_HOME_DIR="${AUDIT_SERVER_HOME_DIR:-/opt/ranger/audit-server}" -AUDIT_SERVER_CONF_DIR="${AUDIT_SERVER_CONF_DIR:-/opt/ranger/audit-server/conf}" -AUDIT_SERVER_LOG_DIR="${AUDIT_SERVER_LOG_DIR:-/var/log/ranger/audit-server}" +# Support both old and new environment variable names for backward compatibility +AUDIT_SERVER_HOME_DIR="${AUDIT_INGESTOR_HOME_DIR:-${AUDIT_SERVER_HOME_DIR:-/opt/ranger/audit-ingestor}}" +AUDIT_SERVER_CONF_DIR="${AUDIT_INGESTOR_CONF_DIR:-${AUDIT_SERVER_CONF_DIR:-/opt/ranger/audit-ingestor/conf}}" +AUDIT_SERVER_LOG_DIR="${AUDIT_INGESTOR_LOG_DIR:-${AUDIT_SERVER_LOG_DIR:-/var/log/ranger/audit-ingestor}}" # Create log directory if it doesn't exist mkdir -p ${AUDIT_SERVER_LOG_DIR} chown -R ranger:ranger /var/log/ranger 2>/dev/null || true echo "==========================================" -echo "Starting Ranger Audit Server Service..." +echo "Starting Ranger Audit Ingestor Service..." echo "==========================================" echo "AUDIT_SERVER_HOME_DIR: ${AUDIT_SERVER_HOME_DIR}" echo "AUDIT_SERVER_CONF_DIR: ${AUDIT_SERVER_CONF_DIR}" @@ -39,35 +40,37 @@ source /home/ranger/scripts/service-check-functions.sh # Quick check for Kafka availability # The audit server has a built-in recovery/spool mechanism for when Kafka is unavailable -KAFKA_BOOTSTRAP_SERVERS="${KAFKA_BOOTSTRAP_SERVERS:-ranger-kafka.rangernw:9092}" +KAFKA_BOOTSTRAP_SERVERS="${KAFKA_BOOTSTRAP_SERVERS:-ranger-kafka:9092}" if check_tcp_port "Kafka" "${KAFKA_BOOTSTRAP_SERVERS}" 30; then echo "[INFO] Kafka is available at startup" else echo "[INFO] Kafka not immediately available - audit server will use recovery/spool mechanism" fi -# Start the audit server -echo "[INFO] Starting Ranger Audit Server Service..." +# Start the audit ingestor +echo "[INFO] Starting Ranger Audit Ingestor Service (refactored module)..." cd ${AUDIT_SERVER_HOME_DIR} -# Export environment variables for Java and audit server +# Export environment variables for Java and audit ingestor export JAVA_HOME=${JAVA_HOME:-/opt/java/openjdk} export PATH=$JAVA_HOME/bin:$PATH export AUDIT_SERVER_LOG_DIR=${AUDIT_SERVER_LOG_DIR} -# Set heap size -AUDIT_SERVER_HEAP="${AUDIT_SERVER_HEAP:--Xms512m -Xmx2g}" +# Set heap size (support both old and new env var names) +AUDIT_SERVER_HEAP="${AUDIT_INGESTOR_HEAP:-${AUDIT_SERVER_HEAP:--Xms512m -Xmx2g}}" export AUDIT_SERVER_HEAP -# Set JVM options including logback configuration -if [ -z "$AUDIT_SERVER_OPTS" ]; then +# Set JVM options including logback configuration (support both old and new env var names) +if [ -z "$AUDIT_SERVER_OPTS" ] && [ -z "$AUDIT_INGESTOR_OPTS" ]; then AUDIT_SERVER_OPTS="-Dlogback.configurationFile=${AUDIT_SERVER_CONF_DIR}/logback.xml" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Daudit.server.log.dir=${AUDIT_SERVER_LOG_DIR}" - AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Daudit.server.log.file=ranger-audit-server.log" + AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Daudit.server.log.file=ranger-audit-ingestor.log" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -Djava.net.preferIPv4Stack=true -server" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -XX:+UseG1GC -XX:MaxGCPauseMillis=200" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -XX:InitiatingHeapOccupancyPercent=35" AUDIT_SERVER_OPTS="${AUDIT_SERVER_OPTS} -XX:ConcGCThreads=4 -XX:ParallelGCThreads=8" +else + AUDIT_SERVER_OPTS="${AUDIT_INGESTOR_OPTS:-${AUDIT_SERVER_OPTS}}" fi # Point to krb5.conf for Kerberos @@ -89,18 +92,18 @@ echo "[INFO] JAVA_HOME: ${JAVA_HOME}" echo "[INFO] AUDIT_SERVER_HEAP: ${AUDIT_SERVER_HEAP}" echo "[INFO] AUDIT_SERVER_OPTS: ${AUDIT_SERVER_OPTS}" -# Build classpath from WAR file +# Build classpath from WAR file (refactored artifact name: ranger-audit-ingestor.war) WEBAPP_ROOT="${AUDIT_SERVER_HOME_DIR}/webapp" -WAR_FILE="${WEBAPP_ROOT}/ranger-audit-server.war" -WEBAPP_DIR="${WEBAPP_ROOT}/ranger-audit-server" +WAR_FILE="${WEBAPP_ROOT}/ranger-audit-ingestor.war" +WEBAPP_DIR="${WEBAPP_ROOT}/audit-ingestor" # Extract WAR if not already extracted -if [ -f "${WAR_FILE}" ] && [ ! -d "${WEBAPP_DIR}" ]; then +if [ -f "${WAR_FILE}" ] && [ ! -d "${WEBAPP_DIR}/WEB-INF" ]; then echo "[INFO] Extracting WAR file..." mkdir -p "${WEBAPP_DIR}" cd "${WEBAPP_DIR}" jar xf "${WAR_FILE}" - cd - + cd - > /dev/null fi # Build classpath @@ -120,18 +123,18 @@ fi export RANGER_CLASSPATH -echo "[INFO] Starting Ranger Audit Server Service..." +echo "[INFO] Starting Ranger Audit Ingestor Service (refactored module)..." echo "[INFO] Webapp dir: ${WEBAPP_DIR}" java ${AUDIT_SERVER_HEAP} ${AUDIT_SERVER_OPTS} \ - -Daudit.config=${AUDIT_SERVER_CONF_DIR}/ranger-audit-server-site.xml \ + -Daudit.config=${AUDIT_SERVER_CONF_DIR}/ranger-audit-ingestor-site.xml \ -cp "${RANGER_CLASSPATH}" \ org.apache.ranger.audit.server.AuditServerApplication \ >> ${AUDIT_SERVER_LOG_DIR}/catalina.out 2>&1 & PID=$! -echo $PID > ${AUDIT_SERVER_LOG_DIR}/ranger-audit-server.pid +echo $PID > ${AUDIT_SERVER_LOG_DIR}/ranger-audit-ingestor.pid -echo "[INFO] Ranger Audit Server started with PID: $PID" +echo "[INFO] Ranger Audit Ingestor started with PID: $PID" # Keep the container running by tailing logs tail -f ${AUDIT_SERVER_LOG_DIR}/catalina.out 2>/dev/null diff --git a/dev-support/ranger-docker/scripts/audit-ingestor/service-check-functions.sh b/dev-support/ranger-docker/scripts/audit-ingestor/service-check-functions.sh new file mode 100755 index 0000000000..6a106d164a --- /dev/null +++ b/dev-support/ranger-docker/scripts/audit-ingestor/service-check-functions.sh @@ -0,0 +1,76 @@ +#!/bin/bash + +# +# 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. +# + +# +# Reusable service availability check functions +# + +# Check if a TCP port is available +# Usage: check_tcp_port "ServiceName" "host:port" [max_wait_seconds] +check_tcp_port() { + local SERVICE_NAME=$1 + local HOST_PORT=$2 + local MAX_WAIT=${3:-60} + + # Parse host and port + local HOST="${HOST_PORT%%:*}" + local PORT="${HOST_PORT##*:}" + + echo "[INFO] Waiting for ${SERVICE_NAME}..." + local WAIT_COUNT=0 + + while ! timeout 1 bash -c "echo > /dev/tcp/${HOST}/${PORT}" 2>/dev/null; do + WAIT_COUNT=$((WAIT_COUNT+1)) + if [ $WAIT_COUNT -ge $MAX_WAIT ]; then + echo "[WARN] ${SERVICE_NAME} not available after ${MAX_WAIT} seconds, continuing anyway..." + return 1 + fi + echo "[INFO] Waiting for ${SERVICE_NAME}... ($WAIT_COUNT/$MAX_WAIT)" + sleep 2 + done + + echo "[INFO] ✓ ${SERVICE_NAME} is available at ${HOST}:${PORT}" + return 0 +} + +# Check if an HTTP service is available +# Usage: check_http_service "ServiceName" "http://host:port/path" [max_wait_seconds] +check_http_service() { + local SERVICE_NAME=$1 + local URL=$2 + local MAX_WAIT=${3:-60} + + echo "[INFO] Waiting for ${SERVICE_NAME}..." + local WAIT_COUNT=0 + + while ! curl -f -s -m 2 "${URL}" > /dev/null 2>&1; do + WAIT_COUNT=$((WAIT_COUNT+1)) + if [ $WAIT_COUNT -ge $MAX_WAIT ]; then + echo "[WARN] ${SERVICE_NAME} not available after ${MAX_WAIT} seconds, continuing anyway..." + return 1 + fi + echo "[INFO] Waiting for ${SERVICE_NAME}... ($WAIT_COUNT/$MAX_WAIT)" + sleep 2 + done + + echo "[INFO] ✓ ${SERVICE_NAME} is available at ${URL}" + return 0 +} diff --git a/dev-support/ranger-docker/scripts/audit-server/ranger-audit-consumer-hdfs.sh b/dev-support/ranger-docker/scripts/audit-server/ranger-audit-consumer-hdfs.sh deleted file mode 100755 index 60e5d2bc87..0000000000 --- a/dev-support/ranger-docker/scripts/audit-server/ranger-audit-consumer-hdfs.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/bin/bash - -# 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. - -set -e - -AUDIT_CONSUMER_HOME_DIR="${AUDIT_CONSUMER_HOME_DIR:-/opt/ranger/audit-consumer-hdfs}" -AUDIT_CONSUMER_CONF_DIR="${AUDIT_CONSUMER_CONF_DIR:-/opt/ranger/audit-consumer-hdfs/conf}" -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-/var/log/ranger/audit-consumer-hdfs}" - -# Create log directory if it doesn't exist -mkdir -p ${AUDIT_CONSUMER_LOG_DIR} -chown -R ranger:ranger /var/log/ranger 2>/dev/null || true - -echo "==========================================" -echo "Starting Ranger Audit Consumer - HDFS" -echo "==========================================" -echo "AUDIT_CONSUMER_HOME_DIR: ${AUDIT_CONSUMER_HOME_DIR}" -echo "AUDIT_CONSUMER_CONF_DIR: ${AUDIT_CONSUMER_CONF_DIR}" -echo "AUDIT_CONSUMER_LOG_DIR: ${AUDIT_CONSUMER_LOG_DIR}" -echo "==========================================" - -# Source service check functions -source /home/ranger/scripts/service-check-functions.sh - -# Wait for Kafka to be available -KAFKA_BOOTSTRAP_SERVERS="${KAFKA_BOOTSTRAP_SERVERS:-ranger-kafka.rangernw:9092}" -check_tcp_port "Kafka" "${KAFKA_BOOTSTRAP_SERVERS}" 60 - -# Wait for HDFS to be available -HDFS_NAMENODE="${HDFS_NAMENODE:-ranger-hadoop.rangernw:9000}" -check_tcp_port "HDFS NameNode" "${HDFS_NAMENODE}" 60 - -# Start the HDFS consumer -echo "[INFO] Starting Ranger Audit Consumer - HDFS..." -cd ${AUDIT_CONSUMER_HOME_DIR} - -# Export environment variables for Java -export JAVA_HOME=${JAVA_HOME:-/opt/java/openjdk} -export PATH=$JAVA_HOME/bin:$PATH -export AUDIT_CONSUMER_LOG_DIR=${AUDIT_CONSUMER_LOG_DIR} - -# Set heap size -AUDIT_CONSUMER_HEAP="${AUDIT_CONSUMER_HEAP:--Xms512m -Xmx2g}" -export AUDIT_CONSUMER_HEAP - -# Set JVM options including logback configuration -if [ -z "$AUDIT_CONSUMER_OPTS" ]; then - AUDIT_CONSUMER_OPTS="-Dlogback.configurationFile=${AUDIT_CONSUMER_CONF_DIR}/logback.xml" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.hdfs.log.dir=${AUDIT_CONSUMER_LOG_DIR}" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.hdfs.log.file=ranger-audit-consumer-hdfs.log" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.net.preferIPv4Stack=true -server" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:+UseG1GC -XX:MaxGCPauseMillis=200" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:InitiatingHeapOccupancyPercent=35" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:ConcGCThreads=4 -XX:ParallelGCThreads=8" -fi - -# Point to krb5.conf and Hadoop configs for Kerberos -if [ "${KERBEROS_ENABLED}" == "true" ]; then - export AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.security.krb5.conf=/etc/krb5.conf" - - # Wait for keytabs if Kerberos is enabled - if [ -f /home/ranger/scripts/wait_for_keytab.sh ]; then - echo "[INFO] Waiting for Kerberos keytabs..." - bash /home/ranger/scripts/wait_for_keytab.sh HTTP.keytab - bash /home/ranger/scripts/wait_for_keytab.sh rangerauditserver.keytab - echo "[INFO] ✓ Keytabs are available" - fi -fi - -export AUDIT_CONSUMER_OPTS - -echo "[INFO] JAVA_HOME: ${JAVA_HOME}" -echo "[INFO] AUDIT_CONSUMER_HEAP: ${AUDIT_CONSUMER_HEAP}" -echo "[INFO] AUDIT_CONSUMER_OPTS: ${AUDIT_CONSUMER_OPTS}" - -# Build classpath from WAR file -WEBAPP_ROOT="${AUDIT_CONSUMER_HOME_DIR}/webapp" -WAR_FILE="${WEBAPP_ROOT}/ranger-audit-consumer-hdfs.war" -WEBAPP_DIR="${WEBAPP_ROOT}/ranger-audit-consumer-hdfs" - -# Extract WAR if not already extracted -if [ -f "${WAR_FILE}" ] && [ ! -d "${WEBAPP_DIR}" ]; then - echo "[INFO] Extracting WAR file..." - mkdir -p "${WEBAPP_DIR}" - cd "${WEBAPP_DIR}" - jar xf "${WAR_FILE}" - cd - -fi - -# Build classpath -RANGER_CLASSPATH="${WEBAPP_DIR}/WEB-INF/classes" -for jar in "${WEBAPP_DIR}"/WEB-INF/lib/*.jar; do - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" -done - -# Add libext directory if it exists -if [ -d "${AUDIT_CONSUMER_HOME_DIR}/libext" ]; then - for jar in "${AUDIT_CONSUMER_HOME_DIR}"/libext/*.jar; do - if [ -f "${jar}" ]; then - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" - fi - done -fi - -export RANGER_CLASSPATH - -echo "[INFO] Starting Ranger Audit Consumer - HDFS..." -echo "[INFO] Webapp dir: ${WEBAPP_DIR}" -java ${AUDIT_CONSUMER_HEAP} ${AUDIT_CONSUMER_OPTS} \ - -Daudit.config=${AUDIT_CONSUMER_CONF_DIR}/ranger-audit-consumer-hdfs-site.xml \ - -Dhadoop.config.dir=${AUDIT_CONSUMER_CONF_DIR} \ - -Dranger.audit.consumer.webapp.dir="${WAR_FILE}" \ - -cp "${RANGER_CLASSPATH}" \ - org.apache.ranger.audit.consumer.HdfsConsumerApplication \ - >> ${AUDIT_CONSUMER_LOG_DIR}/catalina.out 2>&1 & - -PID=$! -echo $PID > ${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-hdfs.pid - -echo "[INFO] Ranger Audit Consumer - HDFS started with PID: $PID" - -# Keep the container running by tailing logs -tail -f ${AUDIT_CONSUMER_LOG_DIR}/catalina.out 2>/dev/null diff --git a/dev-support/ranger-docker/scripts/audit-server/ranger-audit-consumer-solr.sh b/dev-support/ranger-docker/scripts/audit-server/ranger-audit-consumer-solr.sh deleted file mode 100755 index 92123e2f29..0000000000 --- a/dev-support/ranger-docker/scripts/audit-server/ranger-audit-consumer-solr.sh +++ /dev/null @@ -1,137 +0,0 @@ -#!/bin/bash - -# 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. - -set -e - -AUDIT_CONSUMER_HOME_DIR="${AUDIT_CONSUMER_HOME_DIR:-/opt/ranger/audit-consumer-solr}" -AUDIT_CONSUMER_CONF_DIR="${AUDIT_CONSUMER_CONF_DIR:-/opt/ranger/audit-consumer-solr/conf}" -AUDIT_CONSUMER_LOG_DIR="${AUDIT_CONSUMER_LOG_DIR:-/var/log/ranger/audit-consumer-solr}" - -# Create log directory if it doesn't exist -mkdir -p ${AUDIT_CONSUMER_LOG_DIR} -chown -R ranger:ranger /var/log/ranger 2>/dev/null || true - -echo "==========================================" -echo "Starting Ranger Audit Consumer - Solr" -echo "==========================================" -echo "AUDIT_CONSUMER_HOME_DIR: ${AUDIT_CONSUMER_HOME_DIR}" -echo "AUDIT_CONSUMER_CONF_DIR: ${AUDIT_CONSUMER_CONF_DIR}" -echo "AUDIT_CONSUMER_LOG_DIR: ${AUDIT_CONSUMER_LOG_DIR}" -echo "==========================================" - -# Source service check functions -source /home/ranger/scripts/service-check-functions.sh - -# Wait for Kafka to be available -KAFKA_BOOTSTRAP_SERVERS="${KAFKA_BOOTSTRAP_SERVERS:-ranger-kafka.rangernw:9092}" -check_tcp_port "Kafka" "${KAFKA_BOOTSTRAP_SERVERS}" 60 - -# Wait for Solr to be available -SOLR_URL="${SOLR_URL:-http://ranger-solr.rangernw:8983/solr}" -check_http_service "Solr" "${SOLR_URL}/" 60 - -# Start the Solr consumer -echo "[INFO] Starting Ranger Audit Consumer - Solr..." -cd ${AUDIT_CONSUMER_HOME_DIR} - -# Export environment variables for Java -export JAVA_HOME=${JAVA_HOME:-/opt/java/openjdk} -export PATH=$JAVA_HOME/bin:$PATH -export AUDIT_CONSUMER_LOG_DIR=${AUDIT_CONSUMER_LOG_DIR} - -# Set heap size -AUDIT_CONSUMER_HEAP="${AUDIT_CONSUMER_HEAP:--Xms512m -Xmx2g}" -export AUDIT_CONSUMER_HEAP - -# Set JVM options including logback configuration -if [ -z "$AUDIT_CONSUMER_OPTS" ]; then - AUDIT_CONSUMER_OPTS="-Dlogback.configurationFile=${AUDIT_CONSUMER_CONF_DIR}/logback.xml" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.solr.log.dir=${AUDIT_CONSUMER_LOG_DIR}" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Daudit.consumer.solr.log.file=ranger-audit-consumer-solr.log" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.net.preferIPv4Stack=true -server" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:+UseG1GC -XX:MaxGCPauseMillis=200" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:InitiatingHeapOccupancyPercent=35" - AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -XX:ConcGCThreads=4 -XX:ParallelGCThreads=8" -fi - -# Point to krb5.conf for Kerberos -if [ "${KERBEROS_ENABLED}" == "true" ]; then - export AUDIT_CONSUMER_OPTS="${AUDIT_CONSUMER_OPTS} -Djava.security.krb5.conf=/etc/krb5.conf" - - # Wait for keytabs if Kerberos is enabled - if [ -f /home/ranger/scripts/wait_for_keytab.sh ]; then - echo "[INFO] Waiting for Kerberos keytabs..." - bash /home/ranger/scripts/wait_for_keytab.sh HTTP.keytab - bash /home/ranger/scripts/wait_for_keytab.sh rangerauditserver.keytab - echo "[INFO] ✓ Keytabs are available" - fi -fi - -export AUDIT_CONSUMER_OPTS - -echo "[INFO] JAVA_HOME: ${JAVA_HOME}" -echo "[INFO] AUDIT_CONSUMER_HEAP: ${AUDIT_CONSUMER_HEAP}" -echo "[INFO] AUDIT_CONSUMER_OPTS: ${AUDIT_CONSUMER_OPTS}" - -# Build classpath from WAR file -WEBAPP_ROOT="${AUDIT_CONSUMER_HOME_DIR}/webapp" -WAR_FILE="${WEBAPP_ROOT}/ranger-audit-consumer-solr.war" -WEBAPP_DIR="${WEBAPP_ROOT}/ranger-audit-consumer-solr" - -# Extract WAR if not already extracted -if [ -f "${WAR_FILE}" ] && [ ! -d "${WEBAPP_DIR}" ]; then - echo "[INFO] Extracting WAR file..." - mkdir -p "${WEBAPP_DIR}" - cd "${WEBAPP_DIR}" - jar xf "${WAR_FILE}" - cd - -fi - -# Build classpath -RANGER_CLASSPATH="${WEBAPP_DIR}/WEB-INF/classes" -for jar in "${WEBAPP_DIR}"/WEB-INF/lib/*.jar; do - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" -done - -# Add libext directory if it exists -if [ -d "${AUDIT_CONSUMER_HOME_DIR}/libext" ]; then - for jar in "${AUDIT_CONSUMER_HOME_DIR}"/libext/*.jar; do - if [ -f "${jar}" ]; then - RANGER_CLASSPATH="${RANGER_CLASSPATH}:${jar}" - fi - done -fi - -export RANGER_CLASSPATH - -echo "[INFO] Starting Ranger Audit Consumer - Solr..." -echo "[INFO] Webapp dir: ${WEBAPP_DIR}" -java ${AUDIT_CONSUMER_HEAP} ${AUDIT_CONSUMER_OPTS} \ - -Daudit.config=${AUDIT_CONSUMER_CONF_DIR}/ranger-audit-consumer-solr-site.xml \ - -Dranger.audit.consumer.webapp.dir="${WAR_FILE}" \ - -cp "${RANGER_CLASSPATH}" \ - org.apache.ranger.audit.consumer.SolrConsumerApplication \ - >> ${AUDIT_CONSUMER_LOG_DIR}/catalina.out 2>&1 & - -PID=$! -echo $PID > ${AUDIT_CONSUMER_LOG_DIR}/ranger-audit-consumer-solr.pid - -echo "[INFO] Ranger Audit Consumer - Solr started with PID: $PID" - -# Keep the container running by tailing logs -tail -f ${AUDIT_CONSUMER_LOG_DIR}/catalina.out 2>/dev/null diff --git a/dev-support/ranger-docker/scripts/hadoop/ranger-hdfs-plugin-install.properties b/dev-support/ranger-docker/scripts/hadoop/ranger-hdfs-plugin-install.properties index 9e1554ea4c..4b83a15389 100644 --- a/dev-support/ranger-docker/scripts/hadoop/ranger-hdfs-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/hadoop/ranger-hdfs-plugin-install.properties @@ -21,8 +21,8 @@ CUSTOM_USER=hdfs CUSTOM_GROUP=hadoop XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hadoop/hdfs/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hadoop/hdfs/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/hadoop/ranger-yarn-plugin-install.properties b/dev-support/ranger-docker/scripts/hadoop/ranger-yarn-plugin-install.properties index c88054a8ab..a3f74b957d 100644 --- a/dev-support/ranger-docker/scripts/hadoop/ranger-yarn-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/hadoop/ranger-yarn-plugin-install.properties @@ -21,8 +21,8 @@ CUSTOM_USER=yarn CUSTOM_GROUP=hadoop XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hadoop/yarn/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hadoop/yarn/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/hbase/ranger-hbase-plugin-install.properties b/dev-support/ranger-docker/scripts/hbase/ranger-hbase-plugin-install.properties index 3122614e7a..f1a1377d18 100644 --- a/dev-support/ranger-docker/scripts/hbase/ranger-hbase-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/hbase/ranger-hbase-plugin-install.properties @@ -21,8 +21,8 @@ CUSTOM_USER=hbase CUSTOM_GROUP=hadoop XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hadoop/hbase/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hadoop/hbase/audit/audit-ingestor/spool XAAUDIT.SUMMARY.ENABLE=true UPDATE_XAPOLICIES_ON_GRANT_REVOKE=true diff --git a/dev-support/ranger-docker/scripts/hive/ranger-hive-plugin-install.properties b/dev-support/ranger-docker/scripts/hive/ranger-hive-plugin-install.properties index adaeda252c..ae58f44040 100644 --- a/dev-support/ranger-docker/scripts/hive/ranger-hive-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/hive/ranger-hive-plugin-install.properties @@ -22,8 +22,8 @@ CUSTOM_USER=hive CUSTOM_GROUP=hadoop XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hive/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/hive/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/kafka/ranger-kafka-plugin-install.properties b/dev-support/ranger-docker/scripts/kafka/ranger-kafka-plugin-install.properties index 49a8016552..2e0c1350b0 100644 --- a/dev-support/ranger-docker/scripts/kafka/ranger-kafka-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/kafka/ranger-kafka-plugin-install.properties @@ -24,8 +24,8 @@ XAAUDIT.SUMMARY.ENABLE=true UPDATE_XAPOLICIES_ON_GRANT_REVOKE=true XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kafka/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kafka/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/kdc/entrypoint.sh b/dev-support/ranger-docker/scripts/kdc/entrypoint.sh index 998bad4410..4e35f54572 100644 --- a/dev-support/ranger-docker/scripts/kdc/entrypoint.sh +++ b/dev-support/ranger-docker/scripts/kdc/entrypoint.sh @@ -75,14 +75,14 @@ function create_keytabs() { create_principal_and_keytab rangerlookup ranger # Ranger Audit Server Microservices - create_principal_and_keytab HTTP ranger-audit-server - create_principal_and_keytab rangerauditserver ranger-audit-server + create_principal_and_keytab HTTP ranger-audit-ingestor + create_principal_and_keytab rangerauditserver ranger-audit-ingestor - create_principal_and_keytab HTTP ranger-audit-consumer-solr - create_principal_and_keytab rangerauditserver ranger-audit-consumer-solr + create_principal_and_keytab HTTP ranger-audit-dispatcher-solr + create_principal_and_keytab rangerauditserver ranger-audit-dispatcher-solr - create_principal_and_keytab HTTP ranger-audit-consumer-hdfs - create_principal_and_keytab rangerauditserver ranger-audit-consumer-hdfs + create_principal_and_keytab HTTP ranger-audit-dispatcher-hdfs + create_principal_and_keytab rangerauditserver ranger-audit-dispatcher-hdfs create_principal_and_keytab rangertagsync ranger-tagsync @@ -145,7 +145,7 @@ if [ ! -f $DB_DIR/principal ]; then echo "Database initialized" create_keytabs - create_testusers ranger ranger-usersync ranger-tagsync ranger-pdp ranger-audit-server ranger-audit-consumer-solr ranger-audit-consumer-hdfs ranger-hadoop ranger-hive ranger-hbase ranger-kafka ranger-solr ranger-knox ranger-kms ranger-ozone ranger-trino ranger-opensearch + create_testusers ranger ranger-usersync ranger-tagsync ranger-pdp ranger-audit-ingestor ranger-audit-dispatcher-solr ranger-audit-dispatcher-hdfs ranger-hadoop ranger-hive ranger-hbase ranger-kafka ranger-solr ranger-knox ranger-kms ranger-ozone ranger-trino ranger-opensearch else echo "KDC DB already exists; skipping create" fi diff --git a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-mysql.properties b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-mysql.properties index 6cc8668bc2..978a806586 100755 --- a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-mysql.properties +++ b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-mysql.properties @@ -138,8 +138,8 @@ REPOSITORY_NAME=dev_kms # AUDIT configuration with V3 properties XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-oracle.properties b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-oracle.properties index e2775fb7fe..3db3b1729c 100755 --- a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-oracle.properties +++ b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-oracle.properties @@ -138,8 +138,8 @@ REPOSITORY_NAME=dev_kms # AUDIT configuration with V3 properties XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-postgres.properties b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-postgres.properties index 4e22b5f579..51226a55be 100755 --- a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-postgres.properties +++ b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-postgres.properties @@ -138,8 +138,8 @@ REPOSITORY_NAME=dev_kms # AUDIT configuration with V3 properties XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-sqlserver.properties b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-sqlserver.properties index 3c3eebe95b..559a392324 100644 --- a/dev-support/ranger-docker/scripts/kms/ranger-kms-install-sqlserver.properties +++ b/dev-support/ranger-docker/scripts/kms/ranger-kms-install-sqlserver.properties @@ -137,8 +137,8 @@ REPOSITORY_NAME=dev_kms # AUDIT configuration with V3 properties XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/kms/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/knox/ranger-knox-plugin-install.properties b/dev-support/ranger-docker/scripts/knox/ranger-knox-plugin-install.properties index e1eee44820..a471ec3fa4 100644 --- a/dev-support/ranger-docker/scripts/knox/ranger-knox-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/knox/ranger-knox-plugin-install.properties @@ -33,8 +33,8 @@ XAAUDIT.SOLR.SOLR_URL=http://ranger-solr:8983/solr/ranger_audits # Following properties are needed to get past installation script! Please don't remove XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/knox/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/knox/audit/audit-ingestor/spool XAAUDIT.HDFS.IS_ENABLED=false XAAUDIT.HDFS.DESTINATION_DIRECTORY=/ranger/audit diff --git a/dev-support/ranger-docker/scripts/ozone/ranger-ozone-plugin-install.properties b/dev-support/ranger-docker/scripts/ozone/ranger-ozone-plugin-install.properties index 71af427ddf..90e97a330d 100644 --- a/dev-support/ranger-docker/scripts/ozone/ranger-ozone-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/ozone/ranger-ozone-plugin-install.properties @@ -21,8 +21,8 @@ CUSTOM_USER=ozone CUSTOM_GROUP=hadoop XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/ozone/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/ozone/audit/audit-ingestor/spool XAAUDIT.SOLR.IS_ENABLED=false XAAUDIT.SOLR.MAX_QUEUE_SIZE=1 diff --git a/dev-support/ranger-docker/scripts/solr/ranger-solr-plugin-install.properties b/dev-support/ranger-docker/scripts/solr/ranger-solr-plugin-install.properties index b5e2a67dc7..4a008eccab 100644 --- a/dev-support/ranger-docker/scripts/solr/ranger-solr-plugin-install.properties +++ b/dev-support/ranger-docker/scripts/solr/ranger-solr-plugin-install.properties @@ -27,8 +27,8 @@ XAAUDIT.SOLR.SOLR_URL=http://ranger-solr:8983/solr/ranger_audits # Following properties are needed to get past installation script! Please don't remove XAAUDIT.AUDITSERVER.ENABLE=true -XAAUDIT.AUDITSERVER.URL=http://ranger-audit-server.rangernw:7081 -XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/solr/audit/audit-server/spool +XAAUDIT.AUDITSERVER.URL=http://ranger-audit-ingestor.rangernw:7081 +XAAUDIT.AUDITSERVER.FILE_SPOOL_DIR=/var/log/solr/audit/audit-ingestor/spool XAAUDIT.HDFS.IS_ENABLED=false XAAUDIT.HDFS.DESTINATION_DIRECTORY=/ranger/audit diff --git a/distro/pom.xml b/distro/pom.xml index be243be0cc..01b5fd54f1 100644 --- a/distro/pom.xml +++ b/distro/pom.xml @@ -28,28 +28,36 @@ Apache Ranger Distribution Apache Ranger Distribution - org.apache.ranger - audit-consumer-hdfs + audit-dispatcher-hdfs ${project.version} - war - provided org.apache.ranger - audit-consumer-solr + audit-dispatcher-solr ${project.version} - war - provided org.apache.ranger - audit-server + ranger-audit-dest-hdfs + ${project.version} + + + org.apache.ranger + ranger-audit-dest-solr + ${project.version} + + + org.apache.ranger + ranger-audit-dispatcher-common + ${project.version} + + + org.apache.ranger + ranger-audit-server-common ${project.version} - war - provided org.apache.ranger @@ -87,12 +95,6 @@ ${project.version} provided - - org.apache.ranger - ranger-audit-common - ${project.version} - provided - org.apache.ranger ranger-audit-dest-cloudwatch @@ -107,26 +109,28 @@ org.apache.ranger - ranger-audit-dest-hdfs + ranger-audit-dest-kafka ${project.version} provided org.apache.ranger - ranger-audit-dest-kafka + ranger-audit-dest-log4j ${project.version} provided org.apache.ranger - ranger-audit-dest-log4j + ranger-audit-dispatcher-app ${project.version} + war provided org.apache.ranger - ranger-audit-dest-solr + ranger-audit-ingestor ${project.version} + war provided @@ -502,9 +506,8 @@ src/main/assembly/plugin-sqoop.xml src/main/assembly/plugin-kylin.xml src/main/assembly/plugin-elasticsearch.xml - src/main/assembly/audit-server.xml - src/main/assembly/audit-consumer-solr.xml - src/main/assembly/audit-consumer-hdfs.xml + src/main/assembly/audit-ingestor.xml + src/main/assembly/audit-dispatcher.xml src/main/assembly/plugin-schema-registry.xml src/main/assembly/plugin-presto.xml src/main/assembly/plugin-trino.xml @@ -1091,9 +1094,8 @@ src/main/assembly/plugin-sqoop.xml src/main/assembly/plugin-kylin.xml src/main/assembly/plugin-elasticsearch.xml - src/main/assembly/audit-server.xml - src/main/assembly/audit-consumer-solr.xml - src/main/assembly/audit-consumer-hdfs.xml + src/main/assembly/audit-ingestor.xml + src/main/assembly/audit-dispatcher.xml src/main/assembly/plugin-schema-registry.xml src/main/assembly/plugin-presto.xml src/main/assembly/plugin-trino.xml diff --git a/distro/src/main/assembly/audit-consumer-hdfs.xml b/distro/src/main/assembly/audit-consumer-hdfs.xml deleted file mode 100644 index 44e3916884..0000000000 --- a/distro/src/main/assembly/audit-consumer-hdfs.xml +++ /dev/null @@ -1,85 +0,0 @@ - - - - audit-consumer-hdfs - - tar.gz - - ${project.parent.name}-${project.version}-audit-consumer-hdfs - true - - - - bin - ${project.parent.basedir}/audit-server/consumer-hdfs/scripts - - **/*.sh - **/*.py - - 755 - - - - - conf - ${project.parent.basedir}/audit-server/consumer-hdfs/src/main/resources/conf - - ranger-audit-consumer-hdfs-site.xml - core-site.xml - hdfs-site.xml - logback.xml - - 644 - - - - - - ${project.build.directory} - - version - - 444 - - - - - libext - - **/* - - - - - - logs - - **/* - - - - - - - - webapp - ${project.parent.basedir}/audit-server/consumer-hdfs/target/audit-consumer-hdfs-${project.version}.war - ranger-audit-consumer-hdfs.war - - - diff --git a/distro/src/main/assembly/audit-consumer-solr.xml b/distro/src/main/assembly/audit-consumer-solr.xml deleted file mode 100644 index 4c65f2a71f..0000000000 --- a/distro/src/main/assembly/audit-consumer-solr.xml +++ /dev/null @@ -1,83 +0,0 @@ - - - - audit-consumer-solr - - tar.gz - - ${project.parent.name}-${project.version}-audit-consumer-solr - true - - - - bin - ${project.parent.basedir}/audit-server/consumer-solr/scripts - - **/*.sh - **/*.py - - 755 - - - - - conf - ${project.parent.basedir}/audit-server/consumer-solr/src/main/resources/conf - - ranger-audit-consumer-solr-site.xml - logback.xml - - 644 - - - - - - ${project.build.directory} - - version - - 444 - - - - - libext - - **/* - - - - - - logs - - **/* - - - - - - - - webapp - ${project.parent.basedir}/audit-server/consumer-solr/target/audit-consumer-solr-${project.version}.war - ranger-audit-consumer-solr.war - - - diff --git a/distro/src/main/assembly/audit-dispatcher.xml b/distro/src/main/assembly/audit-dispatcher.xml new file mode 100644 index 0000000000..268237d06d --- /dev/null +++ b/distro/src/main/assembly/audit-dispatcher.xml @@ -0,0 +1,135 @@ + + + + audit-dispatcher + + tar.gz + + ${project.parent.name}-${project.version}-audit-dispatcher + true + + + + scripts + ${project.parent.basedir}/audit-server/audit-dispatcher/scripts + + **/*.sh + **/*.py + + 755 + + + + + conf + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-common/src/main/resources/conf + + ranger-audit-dispatcher-site.xml + + 644 + + + + conf + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-solr/src/main/resources/conf + + ranger-audit-dispatcher-solr-site.xml + logback.xml + + 644 + + + + conf + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-hdfs/src/main/resources/conf + + ranger-audit-dispatcher-hdfs-site.xml + core-site.xml + hdfs-site.xml + + 644 + + + + + + ${project.build.directory} + + version + + 444 + + + + + libext + + **/* + + + + + + logs + + **/* + + + + + + lib/dispatchers/hdfs + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-hdfs/target/lib + + *.jar + + + + lib/dispatchers/hdfs + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-hdfs/target + + audit-dispatcher-hdfs-${project.version}.jar + + + + + + lib/dispatchers/solr + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-solr/target/lib + + *.jar + + + + lib/dispatchers/solr + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-solr/target + + audit-dispatcher-solr-${project.version}.jar + + + + + + + + webapp + ${project.parent.basedir}/audit-server/audit-dispatcher/dispatcher-app/target/ranger-audit-dispatcher-${project.version}.war + ranger-audit-dispatcher.war + + + diff --git a/distro/src/main/assembly/audit-server.xml b/distro/src/main/assembly/audit-ingestor.xml similarity index 82% rename from distro/src/main/assembly/audit-server.xml rename to distro/src/main/assembly/audit-ingestor.xml index 8a7f6d8e46..80d6829a6a 100644 --- a/distro/src/main/assembly/audit-server.xml +++ b/distro/src/main/assembly/audit-ingestor.xml @@ -16,17 +16,17 @@ limitations under the License. --> - audit-server + audit-ingestor tar.gz - ${project.parent.name}-${project.version}-audit-server + ${project.parent.name}-${project.version}-audit-ingestor true bin - ${project.parent.basedir}/audit-server/server/scripts + ${project.parent.basedir}/audit-server/audit-ingestor/scripts **/*.sh **/*.py @@ -37,9 +37,9 @@ conf - ${project.parent.basedir}/audit-server/server/src/main/resources/conf + ${project.parent.basedir}/audit-server/audit-ingestor/src/main/resources/conf - ranger-audit-server-site.xml + ranger-audit-ingestor-site.xml logback.xml 644 @@ -76,8 +76,8 @@ webapp - ${project.parent.basedir}/audit-server/server/target/audit-server-${project.version}.war - ranger-audit-server.war + ${project.parent.basedir}/audit-server/audit-ingestor/target/audit-ingestor-${project.version}.war + ranger-audit-ingestor.war