SOLR-12120: New AuditLoggerPlugin type allowing custom Audit logger plugins

This commit is contained in:
Jan Høydahl 2019-04-04 15:48:51 +02:00
parent 9b44fba0d4
commit 361502da8a
19 changed files with 2262 additions and 16 deletions

View File

@ -88,6 +88,8 @@ New Features
* SOLR-12121: JWT Token authentication plugin with OpenID Connect implicit flow login through Admin UI (janhoy)
* SOLR-12120: New AuditLoggerPlugin type allowing custom Audit logger plugins (janhoy)
* SOLR-10436: Add hashRollup Streaming Expression (Joel Bernstein)
* SOLR-13276: Adding Http2 equivalent classes of CloudSolrClient and HttpClusterStateProvider (Cao Manh Dat)

View File

@ -100,6 +100,7 @@ import org.apache.solr.metrics.SolrMetricProducer;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.search.SolrFieldCacheBean;
import org.apache.solr.security.AuditLoggerPlugin;
import org.apache.solr.security.AuthenticationPlugin;
import org.apache.solr.security.AuthorizationPlugin;
import org.apache.solr.security.HttpClientBuilderPlugin;
@ -203,6 +204,8 @@ public class CoreContainer {
private volatile SecurityPluginHolder<AuthenticationPlugin> authenticationPlugin;
private SecurityPluginHolder<AuditLoggerPlugin> auditloggerPlugin;
private volatile BackupRepositoryFactory backupRepoFactory;
protected volatile SolrMetricManager metricManager;
@ -357,10 +360,43 @@ public class CoreContainer {
try {
old.plugin.close();
} catch (Exception e) {
log.error("Exception while attempting to close old authorization plugin", e);
}
}
}
private void initializeAuditloggerPlugin(Map<String, Object> auditConf) {
auditConf = Utils.getDeepCopy(auditConf, 4);
//Initialize the Auditlog module
SecurityPluginHolder<AuditLoggerPlugin> old = auditloggerPlugin;
SecurityPluginHolder<AuditLoggerPlugin> newAuditloggerPlugin = null;
if (auditConf != null) {
String klas = (String) auditConf.get("class");
if (klas == null) {
throw new SolrException(ErrorCode.SERVER_ERROR, "class is required for auditlogger plugin");
}
if (old != null && old.getZnodeVersion() == readVersion(auditConf)) {
return;
}
log.info("Initializing auditlogger plugin: " + klas);
newAuditloggerPlugin = new SecurityPluginHolder<>(readVersion(auditConf),
getResourceLoader().newInstance(klas, AuditLoggerPlugin.class));
newAuditloggerPlugin.plugin.init(auditConf);
} else {
log.debug("Security conf doesn't exist. Skipping setup for audit logging module.");
}
this.auditloggerPlugin = newAuditloggerPlugin;
if (old != null) {
try {
old.plugin.close();
} catch (Exception e) {
log.error("Exception while attempting to close old auditlogger plugin", e);
}
}
}
private synchronized void initializeAuthenticationPlugin(Map<String, Object> authenticationConfig) {
authenticationConfig = Utils.getDeepCopy(authenticationConfig, 4);
String pluginClassName = null;
@ -401,7 +437,9 @@ public class CoreContainer {
this.authenticationPlugin = authenticationPlugin;
try {
if (old != null) old.plugin.close();
} catch (Exception e) {/*do nothing*/ }
} catch (Exception e) {
log.error("Exception while attempting to close old authentication plugin", e);
}
}
@ -815,6 +853,10 @@ public class CoreContainer {
if (pkiAuthenticationPlugin != null && pkiAuthenticationPlugin.getMetricRegistry() == null) {
pkiAuthenticationPlugin.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, "/authentication/pki");
}
initializeAuditloggerPlugin((Map<String, Object>) securityConfig.getData().get("auditlogging"));
if (auditloggerPlugin != null) {
auditloggerPlugin.plugin.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, "/auditlogging");
}
}
private static void checkForDuplicateCoreNames(List<CoreDescriptor> cds) {
@ -983,6 +1025,16 @@ public class CoreContainer {
log.warn("Exception while closing authentication plugin.", e);
}
// It should be safe to close the auditlogger plugin at this point.
try {
if (auditloggerPlugin != null) {
auditloggerPlugin.plugin.close();
auditloggerPlugin = null;
}
} catch (Exception e) {
log.warn("Exception while closing auditlogger plugin.", e);
}
org.apache.lucene.util.IOUtils.closeWhileHandlingException(loader); // best effort
}
@ -1845,6 +1897,10 @@ public class CoreContainer {
return authenticationPlugin == null ? null : authenticationPlugin.plugin;
}
public AuditLoggerPlugin getAuditLoggerPlugin() {
return auditloggerPlugin == null ? null : auditloggerPlugin.plugin;
}
public NodeConfig getNodeConfig() {
return cfg;
}

View File

@ -0,0 +1,476 @@
/*
* 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.solr.security;
import javax.servlet.http.HttpServletRequest;
import java.lang.invoke.MethodHandles;
import java.security.Principal;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
import static org.apache.solr.security.AuditEvent.EventType.ANONYMOUS;
import static org.apache.solr.security.AuditEvent.EventType.ERROR;
/**
* Audit event that takes request and auth context as input to be able to audit log custom things.
* This interface may change in next release and is marked experimental
* @since 8.1.0
* @lucene.experimental
*/
public class AuditEvent {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private StringBuffer requestUrl;
private String nodeName;
private String message;
private Level level;
private Date date;
private String username;
private String session;
private String clientIp;
private List<String> collections;
private Map<String, Object> context;
private Map<String, String> headers;
private Map<String, List<String>> solrParams = new HashMap<>();
private String solrHost;
private int solrPort;
private String solrIp;
private String resource;
private String httpMethod;
private String httpQueryString;
private EventType eventType;
private AuthorizationResponse autResponse;
private RequestType requestType;
private double QTime = -1;
private int status = -1;
private Throwable exception;
/* Predefined event types. Custom types can be made through constructor */
public enum EventType {
AUTHENTICATED("Authenticated", "User successfully authenticated", Level.INFO, -1),
REJECTED("Rejected", "Authentication request rejected", Level.WARN, 401),
ANONYMOUS("Anonymous", "Request proceeds with unknown user", Level.INFO, -1),
ANONYMOUS_REJECTED("AnonymousRejected", "Request from unknown user rejected", Level.WARN, 401),
AUTHORIZED("Authorized", "Authorization succeeded", Level.INFO, -1),
UNAUTHORIZED("Unauthorized", "Authorization failed", Level.WARN, 403),
COMPLETED("Completed", "Request completed", Level.INFO, 200),
ERROR("Error", "Request was not executed due to an error", Level.ERROR, 500);
public final String message;
public String explanation;
public final Level level;
public int status;
EventType(String message, String explanation, Level level, int status) {
this.message = message;
this.explanation = explanation;
this.level = level;
this.status = status;
}
}
/**
* Empty event, must be filled by user using setters.
* Message and Loglevel will be initialized from EventType but can
* be overridden with setters afterwards.
* @param eventType a predefined or custom EventType
*/
public AuditEvent(EventType eventType) {
this.date = new Date();
this.eventType = eventType;
this.status = eventType.status;
this.level = eventType.level;
this.message = eventType.message;
}
public AuditEvent(EventType eventType, HttpServletRequest httpRequest) {
this(eventType, null, httpRequest);
}
// Constructor for testing and deserialization only
protected AuditEvent() { }
/**
* Event based on an HttpServletRequest, typically used during authentication.
* Solr will fill in details such as ip, http method etc from the request, and
* username if Principal exists on the request.
* @param eventType a predefined or custom EventType
* @param httpRequest the request to initialize from
*/
public AuditEvent(EventType eventType, Throwable exception, HttpServletRequest httpRequest) {
this(eventType);
this.solrHost = httpRequest.getLocalName();
this.solrPort = httpRequest.getLocalPort();
this.solrIp = httpRequest.getLocalAddr();
this.clientIp = httpRequest.getRemoteAddr();
this.resource = httpRequest.getContextPath();
this.httpMethod = httpRequest.getMethod();
this.httpQueryString = httpRequest.getQueryString();
this.headers = getHeadersFromRequest(httpRequest);
this.requestUrl = httpRequest.getRequestURL();
this.nodeName = MDC.get(ZkStateReader.NODE_NAME_PROP);
setRequestType(findRequestType());
if (exception != null) setException(exception);
Principal principal = httpRequest.getUserPrincipal();
if (principal != null) {
this.username = httpRequest.getUserPrincipal().getName();
} else if (eventType.equals(EventType.AUTHENTICATED)) {
this.eventType = ANONYMOUS;
this.message = ANONYMOUS.message;
this.level = ANONYMOUS.level;
log.debug("Audit event type changed from AUTHENTICATED to ANONYMOUS since no Principal found on request");
}
}
/**
* Event based on request and AuthorizationContext. Solr will fill in details
* such as collections, ip, http method etc from the context.
* @param eventType a predefined or custom EventType
* @param httpRequest the request to initialize from
* @param authorizationContext the context to initialize from
*/
public AuditEvent(EventType eventType, HttpServletRequest httpRequest, AuthorizationContext authorizationContext) {
this(eventType, httpRequest);
this.collections = authorizationContext.getCollectionRequests()
.stream().map(r -> r.collectionName).collect(Collectors.toList());
this.resource = authorizationContext.getResource();
this.requestType = RequestType.convertType(authorizationContext.getRequestType());
authorizationContext.getParams().forEach(p -> {
this.solrParams.put(p.getKey(), Arrays.asList(p.getValue()));
});
}
/**
* Event to log completed requests. Takes time and status. Solr will fill in details
* such as collections, ip, http method etc from the HTTP request and context.
*
* @param eventType a predefined or custom EventType
* @param httpRequest the request to initialize from
* @param authorizationContext the context to initialize from
* @param qTime query time
* @param exception exception from query response, or null if OK
*/
public AuditEvent(EventType eventType, HttpServletRequest httpRequest, AuthorizationContext authorizationContext, double qTime, Throwable exception) {
this(eventType, httpRequest, authorizationContext);
setQTime(qTime);
setException(exception);
}
private HashMap<String, String> getHeadersFromRequest(HttpServletRequest httpRequest) {
HashMap<String, String> h = new HashMap<>();
Enumeration<String> headersEnum = httpRequest.getHeaderNames();
while (headersEnum != null && headersEnum.hasMoreElements()) {
String name = headersEnum.nextElement();
h.put(name, httpRequest.getHeader(name));
}
return h;
}
public enum Level {
INFO, // Used for normal successful events
WARN, // Used when a user is blocked etc
ERROR // Used when there is an exception or error during auth / authz
}
public enum RequestType {
ADMIN, SEARCH, UPDATE, STREAMING, UNKNOWN;
static RequestType convertType(AuthorizationContext.RequestType ctxReqType) {
switch (ctxReqType) {
case ADMIN:
return RequestType.ADMIN;
case READ:
return RequestType.SEARCH;
case WRITE:
return RequestType.UPDATE;
default:
return RequestType.UNKNOWN;
}
}
}
public String getMessage() {
return message;
}
public Level getLevel() {
return level;
}
public Date getDate() {
return date;
}
public String getUsername() {
return username;
}
public String getSession() {
return session;
}
public String getClientIp() {
return clientIp;
}
public Map<String, Object> getContext() {
return context;
}
public List<String> getCollections() {
return collections;
}
public String getResource() {
return resource;
}
public String getHttpMethod() {
return httpMethod;
}
public String getHttpQueryString() {
return httpQueryString;
}
public EventType getEventType() {
return eventType;
}
public String getSolrHost() {
return solrHost;
}
public String getSolrIp() {
return solrIp;
}
public int getSolrPort() {
return solrPort;
}
public Map<String, String> getHeaders() {
return headers;
}
public Map<String, List<String>> getSolrParams() {
return solrParams;
}
public String getSolrParamAsString(String key) {
List<String> v = getSolrParams().get(key);
if (v != null && v.size() > 0) {
return String.valueOf((v).get(0));
}
return null;
}
public AuthorizationResponse getAutResponse() {
return autResponse;
}
public String getNodeName() {
return nodeName;
}
public RequestType getRequestType() {
return requestType;
}
public int getStatus() {
return status;
}
public double getQTime() {
return QTime;
}
public Throwable getException() {
return exception;
}
public StringBuffer getRequestUrl() {
return requestUrl;
}
// Setters, builder style
public AuditEvent setRequestUrl(StringBuffer requestUrl) {
this.requestUrl = requestUrl;
return this;
}
public AuditEvent setSession(String session) {
this.session = session;
return this;
}
public AuditEvent setClientIp(String clientIp) {
this.clientIp = clientIp;
return this;
}
public AuditEvent setContext(Map<String, Object> context) {
this.context = context;
return this;
}
public AuditEvent setContextEntry(String key, Object value) {
this.context.put(key, value);
return this;
}
public AuditEvent setMessage(String message) {
this.message = message;
return this;
}
public AuditEvent setLevel(Level level) {
this.level = level;
return this;
}
public AuditEvent setDate(Date date) {
this.date = date;
return this;
}
public AuditEvent setUsername(String username) {
this.username = username;
return this;
}
public AuditEvent setCollections(List<String> collections) {
this.collections = collections;
return this;
}
public AuditEvent setResource(String resource) {
this.resource = resource;
return this;
}
public AuditEvent setHttpMethod(String httpMethod) {
this.httpMethod = httpMethod;
return this;
}
public AuditEvent setHttpQueryString(String httpQueryString) {
this.httpQueryString = httpQueryString;
return this;
}
public AuditEvent setSolrHost(String solrHost) {
this.solrHost = solrHost;
return this;
}
public AuditEvent setSolrPort(int solrPort) {
this.solrPort = solrPort;
return this;
}
public AuditEvent setSolrIp(String solrIp) {
this.solrIp = solrIp;
return this;
}
public AuditEvent setHeaders(Map<String, String> headers) {
this.headers = headers;
return this;
}
public AuditEvent setSolrParams(Map<String, List<String>> solrParams) {
this.solrParams = solrParams;
return this;
}
public AuditEvent setAutResponse(AuthorizationResponse autResponse) {
this.autResponse = autResponse;
return this;
}
public AuditEvent setRequestType(RequestType requestType) {
this.requestType = requestType;
return this;
}
public AuditEvent setQTime(double QTime) {
this.QTime = QTime;
return this;
}
public AuditEvent setStatus(int status) {
this.status = status;
return this;
}
public AuditEvent setException(Throwable exception) {
this.exception = exception;
if (exception != null) {
this.eventType = ERROR;
this.level = ERROR.level;
this.message = ERROR.message;
if (exception instanceof SolrException)
status = ((SolrException)exception).code();
}
return this;
}
private RequestType findRequestType() {
if (ADMIN_PATH_REGEXES.stream().map(Pattern::compile)
.anyMatch(p -> p.matcher(resource).matches())) return RequestType.ADMIN;
if (SEARCH_PATH_REGEXES.stream().map(Pattern::compile)
.anyMatch(p -> p.matcher(resource).matches())) return RequestType.SEARCH;
if (INDEXING_PATH_REGEXES.stream().map(Pattern::compile)
.anyMatch(p -> p.matcher(resource).matches())) return RequestType.UPDATE;
if (STREAMING_PATH_REGEXES.stream().map(Pattern::compile)
.anyMatch(p -> p.matcher(resource).matches())) return RequestType.STREAMING;
return RequestType.UNKNOWN;
}
private static final List<String> ADMIN_PATH_REGEXES = Arrays.asList(
"^/solr/admin/.*",
"^/api/(c|collections)/$",
"^/api/(c|collections)/[^/]+/config$",
"^/api/(c|collections)/[^/]+/schema$",
"^/api/(c|collections)/[^/]+/shards.*",
"^/api/cores.*$",
"^/api/node$",
"^/api/cluster$");
private static final List<String> STREAMING_PATH_REGEXES = Collections.singletonList(".*/stream.*");
private static final List<String> INDEXING_PATH_REGEXES = Collections.singletonList(".*/update.*");
private static final List<String> SEARCH_PATH_REGEXES = Arrays.asList(".*/select.*", ".*/query.*");
}

View File

@ -0,0 +1,400 @@
/*
* 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.solr.security;
import java.io.Closeable;
import java.io.IOException;
import java.io.StringWriter;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import com.codahale.metrics.Counter;
import com.codahale.metrics.Meter;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.core.SolrInfoBean;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.metrics.SolrMetricProducer;
import org.apache.solr.security.AuditEvent.EventType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Base class for Audit logger plugins.
* This interface may change in next release and is marked experimental
* @since 8.1.0
* @lucene.experimental
*/
public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfoBean, SolrMetricProducer {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String PARAM_EVENT_TYPES = "eventTypes";
static final String PARAM_ASYNC = "async";
static final String PARAM_BLOCKASYNC = "blockAsync";
static final String PARAM_QUEUE_SIZE = "queueSize";
static final String PARAM_NUM_THREADS = "numThreads";
static final String PARAM_MUTE_RULES = "muteRules";
private static final int DEFAULT_QUEUE_SIZE = 4096;
private static final int DEFAULT_NUM_THREADS = Math.max(2, Runtime.getRuntime().availableProcessors() / 2);
BlockingQueue<AuditEvent> queue;
boolean async;
boolean blockAsync;
int blockingQueueSize;
protected AuditEventFormatter formatter;
private MetricRegistry registry;
private Set<String> metricNames = ConcurrentHashMap.newKeySet();
private ExecutorService executorService;
private boolean closed;
private MuteRules muteRules;
protected String registryName;
protected SolrMetricManager metricManager;
protected Meter numErrors = new Meter();
protected Meter numLost = new Meter();
protected Meter numLogged = new Meter();
protected Timer requestTimes = new Timer();
protected Timer queuedTime = new Timer();
protected Counter totalTime = new Counter();
// Event types to be logged by default
protected List<String> eventTypes = Arrays.asList(
EventType.COMPLETED.name(),
EventType.ERROR.name(),
EventType.REJECTED.name(),
EventType.UNAUTHORIZED.name(),
EventType.ANONYMOUS_REJECTED.name());
/**
* Initialize the plugin from security.json.
* This method removes parameters from config object after consuming, so subclasses can check for config errors.
* @param pluginConfig the config for the plugin
*/
public void init(Map<String, Object> pluginConfig) {
formatter = new JSONAuditEventFormatter();
if (pluginConfig.containsKey(PARAM_EVENT_TYPES)) {
eventTypes = (List<String>) pluginConfig.get(PARAM_EVENT_TYPES);
}
pluginConfig.remove(PARAM_EVENT_TYPES);
async = Boolean.parseBoolean(String.valueOf(pluginConfig.getOrDefault(PARAM_ASYNC, true)));
blockAsync = Boolean.parseBoolean(String.valueOf(pluginConfig.getOrDefault(PARAM_BLOCKASYNC, false)));
blockingQueueSize = async ? Integer.parseInt(String.valueOf(pluginConfig.getOrDefault(PARAM_QUEUE_SIZE, DEFAULT_QUEUE_SIZE))) : 1;
int numThreads = async ? Integer.parseInt(String.valueOf(pluginConfig.getOrDefault(PARAM_NUM_THREADS, DEFAULT_NUM_THREADS))) : 1;
muteRules = new MuteRules(pluginConfig.remove(PARAM_MUTE_RULES));
pluginConfig.remove(PARAM_ASYNC);
pluginConfig.remove(PARAM_BLOCKASYNC);
pluginConfig.remove(PARAM_QUEUE_SIZE);
pluginConfig.remove(PARAM_NUM_THREADS);
if (async) {
queue = new ArrayBlockingQueue<>(blockingQueueSize);
executorService = ExecutorUtil.newMDCAwareFixedThreadPool(numThreads, new SolrjNamedThreadFactory("audit"));
executorService.submit(this);
}
pluginConfig.remove("class");
log.debug("AuditLogger initialized in {} mode with event types {}", async?"async":"syncronous", eventTypes);
}
/**
* This is the method that each Audit plugin has to implement to do the actual logging.
* @param event the audit event
*/
protected abstract void audit(AuditEvent event);
/**
* Called by the framework, and takes care of metrics tracking and to dispatch
* to either synchronous or async logging.
*/
public final void doAudit(AuditEvent event) {
if (shouldMute(event)) {
log.debug("Event muted due to mute rule(s)");
return;
}
if (async) {
auditAsync(event);
} else {
Timer.Context timer = requestTimes.time();
numLogged.mark();
try {
audit(event);
} catch(Exception e) {
numErrors.mark();
throw e;
} finally {
totalTime.inc(timer.stop());
}
}
}
/**
* Returns true if any of the configured mute rules matches. The inner lists are ORed, while rules inside
* inner lists are ANDed
* @param event the audit event
*/
protected boolean shouldMute(AuditEvent event) {
return muteRules.shouldMute(event);
}
/**
* Enqueues an {@link AuditEvent} to a queue and returns immediately.
* A background thread will pull events from this queue and call {@link #audit(AuditEvent)}
* @param event the audit event
*/
protected final void auditAsync(AuditEvent event) {
assert(async);
if (blockAsync) {
try {
queue.put(event);
} catch (InterruptedException e) {
log.warn("Interrupted while waiting to insert AuditEvent into blocking queue");
Thread.currentThread().interrupt();
}
} else {
if (!queue.offer(event)) {
log.warn("Audit log async queue is full (size={}), not blocking since {}", blockingQueueSize, PARAM_BLOCKASYNC + "==false");
numLost.mark();
}
}
}
/**
* Pick next event from async queue and call {@link #audit(AuditEvent)}
*/
@Override
public void run() {
assert(async);
while (!closed && !Thread.currentThread().isInterrupted()) {
try {
AuditEvent event = queue.poll(1000, TimeUnit.MILLISECONDS);
if (event == null) continue;
if (event.getDate() != null) {
queuedTime.update(new Date().getTime() - event.getDate().getTime(), TimeUnit.MILLISECONDS);
}
Timer.Context timer = requestTimes.time();
audit(event);
numLogged.mark();
totalTime.inc(timer.stop());
} catch (InterruptedException e) {
log.warn("Interrupted while waiting for next audit log event");
Thread.currentThread().interrupt();
} catch (Exception ex) {
log.warn("Exception when attempting to audit log asynchronously", ex);
numErrors.mark();
}
}
}
/**
* Checks whether this event type should be logged based on "eventTypes" config parameter.
*
* @param eventType the event type to consider
* @return true if this event type should be logged
*/
public boolean shouldLog(EventType eventType) {
boolean shouldLog = eventTypes.contains(eventType.name());
if (!shouldLog) {
log.debug("Event type {} is not configured for audit logging", eventType.name());
}
return shouldLog;
}
public void setFormatter(AuditEventFormatter formatter) {
this.formatter = formatter;
}
@Override
public void initializeMetrics(SolrMetricManager manager, String registryName, String tag, final String scope) {
String className = this.getClass().getSimpleName();
log.debug("Initializing metrics for {}", className);
this.metricManager = manager;
this.registryName = registryName;
// Metrics
registry = manager.registry(registryName);
numErrors = manager.meter(this, registryName, "errors", getCategory().toString(), scope, className);
numLost = manager.meter(this, registryName, "lost", getCategory().toString(), scope, className);
numLogged = manager.meter(this, registryName, "count", getCategory().toString(), scope, className);
requestTimes = manager.timer(this, registryName, "requestTimes", getCategory().toString(), scope, className);
totalTime = manager.counter(this, registryName, "totalTime", getCategory().toString(), scope, className);
if (async) {
manager.registerGauge(this, registryName, () -> blockingQueueSize, "queueCapacity", true, "queueCapacity", getCategory().toString(), scope, className);
manager.registerGauge(this, registryName, () -> blockingQueueSize - queue.remainingCapacity(), "queueSize", true, "queueSize", getCategory().toString(), scope, className);
queuedTime = manager.timer(this, registryName, "queuedTime", getCategory().toString(), scope, className);
}
manager.registerGauge(this, registryName, () -> async, "async", true, "async", getCategory().toString(), scope, className);
metricNames.addAll(Arrays.asList("errors", "logged", "requestTimes", "totalTime", "queueCapacity", "queueSize", "async"));
}
@Override
public String getName() {
return this.getClass().getName();
}
@Override
public String getDescription() {
return "Auditlogger Plugin " + this.getClass().getName();
}
@Override
public Category getCategory() {
return Category.SECURITY;
}
@Override
public Set<String> getMetricNames() {
return metricNames;
}
@Override
public MetricRegistry getMetricRegistry() {
return registry;
}
/**
* Interface for formatting the event
*/
public interface AuditEventFormatter {
String formatEvent(AuditEvent event);
}
/**
* Event formatter that returns event as JSON string
*/
public static class JSONAuditEventFormatter implements AuditEventFormatter {
/**
* Formats an audit event as a JSON string
*/
@Override
public String formatEvent(AuditEvent event) {
ObjectMapper mapper = new ObjectMapper();
mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
mapper.setSerializationInclusion(Include.NON_NULL);
try {
StringWriter sw = new StringWriter();
mapper.writeValue(sw, event);
return sw.toString();
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error converting Event to JSON", e);
}
}
}
@Override
public void close() throws IOException {
if (async && executorService != null) {
int timeSlept = 0;
while (!queue.isEmpty() && timeSlept < 30) {
try {
log.info("Async auditlogger queue still has {} elements, sleeping to let it drain...", queue.size());
Thread.sleep(1000);
timeSlept ++;
} catch (InterruptedException ignored) {}
}
closed = true;
log.info("Shutting down async Auditlogger background thread(s)");
executorService.shutdownNow();
}
}
/**
* Set of rules for when audit logging should be muted.
*/
private class MuteRules {
private List<List<MuteRule>> rules;
MuteRules(Object o) {
rules = new ArrayList<>();
if (o != null) {
if (o instanceof List) {
((List)o).forEach(l -> {
if (l instanceof String) {
rules.add(Collections.singletonList(parseRule(l)));
} else if (l instanceof List) {
List<MuteRule> rl = new ArrayList<>();
((List) l).forEach(r -> rl.add(parseRule(r)));
rules.add(rl);
}
});
} else {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The " + PARAM_MUTE_RULES + " configuration must be a list");
}
}
}
private MuteRule parseRule(Object ruleObj) {
try {
String rule = (String) ruleObj;
if (rule.startsWith("type:")) {
AuditEvent.RequestType muteType = AuditEvent.RequestType.valueOf(rule.substring("type:".length()));
return event -> event.getRequestType() != null && event.getRequestType().equals(muteType);
}
if (rule.startsWith("collection:")) {
return event -> event.getCollections() != null && event.getCollections().contains(rule.substring("collection:".length()));
}
if (rule.startsWith("user:")) {
return event -> event.getUsername() != null && event.getUsername().equals(rule.substring("user:".length()));
}
if (rule.startsWith("path:")) {
return event -> event.getResource() != null && event.getResource().startsWith(rule.substring("path:".length()));
}
if (rule.startsWith("ip:")) {
return event -> event.getClientIp() != null && event.getClientIp().equals(rule.substring("ip:".length()));
}
if (rule.startsWith("param:")) {
String[] kv = rule.substring("param:".length()).split("=");
if (kv.length == 2) {
return event -> event.getSolrParams() != null && kv[1].equals(event.getSolrParamAsString(kv[0]));
} else {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The 'param' muteRule must be of format 'param:key=value', got " + rule);
}
}
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unkonwn mute rule " + rule);
} catch (ClassCastException | IllegalArgumentException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "There was a problem parsing muteRules. Must be a list of valid rule strings", e);
}
}
/**
* Returns true if any of the configured mute rules matches. The inner lists are ORed, while rules inside
* inner lists are ANDed
*/
boolean shouldMute(AuditEvent event) {
if (rules == null) return false;
return rules.stream().anyMatch(rl -> rl.stream().allMatch(r -> r.shouldMute(event)));
}
}
public interface MuteRule {
boolean shouldMute(AuditEvent event);
}
}

View File

@ -0,0 +1,138 @@
/*
* 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.solr.security;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.lucene.analysis.util.ResourceLoader;
import org.apache.lucene.analysis.util.ResourceLoaderAware;
import org.apache.solr.common.SolrException;
import org.apache.solr.metrics.SolrMetricManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
/**
* Audit logger that chains other loggers. Lets you configure logging to multiple destinations.
* The config is simply a list of configs for the sub plugins:
* <pre>
* "class" : "solr.MultiDestinationAuditLogger",
* "plugins" : [
* { "class" : "solr.SolrLogAuditLoggerPlugin" },
* { "class" : "solr.MyOtherAuditPlugin"}
* ]
* </pre>
*
* This interface may change in next release and is marked experimental
* @since 8.1.0
* @lucene.experimental
*/
public class MultiDestinationAuditLogger extends AuditLoggerPlugin implements ResourceLoaderAware {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String PARAM_PLUGINS = "plugins";
private ResourceLoader loader;
List<AuditLoggerPlugin> plugins = new ArrayList<>();
List<String> pluginNames = new ArrayList<>();
/**
* Passes the AuditEvent to all sub plugins in parallel. The event should be a {@link AuditEvent} to be able to pull context info.
* @param event the audit event
*/
@Override
public void audit(AuditEvent event) {
log.debug("Passing auditEvent to plugins {}", pluginNames);
plugins.parallelStream().forEach(plugin -> {
if (plugin.shouldLog(event.getEventType()))
plugin.doAudit(event);
});
}
/**
* Initialize the plugin from security.json
* @param pluginConfig the config for the plugin
*/
@Override
public void init(Map<String, Object> pluginConfig) {
pluginConfig.put(PARAM_ASYNC, false); // Force the multi plugin to synchronous operation
super.init(pluginConfig);
if (!pluginConfig.containsKey(PARAM_PLUGINS)) {
log.warn("No plugins configured");
} else {
@SuppressWarnings("unchecked")
List<Map<String, Object>> pluginList = (List<Map<String, Object>>) pluginConfig.get(PARAM_PLUGINS);
pluginList.forEach(pluginConf -> plugins.add(createPlugin(pluginConf)));
pluginConfig.remove(PARAM_PLUGINS);
pluginNames = plugins.stream().map(AuditLoggerPlugin::getName).collect(Collectors.toList());
}
if (pluginConfig.size() > 0) {
log.error("Plugin config was not fully consumed. Remaining parameters are {}", pluginConfig);
}
log.info("Initialized {} audit plugins: {}", plugins.size(), pluginNames);
}
@Override
public boolean shouldLog(AuditEvent.EventType eventType) {
return super.shouldLog(eventType) || plugins.stream().anyMatch(p -> p.shouldLog(eventType));
}
private AuditLoggerPlugin createPlugin(Map<String, Object> auditConf) {
if (auditConf != null) {
String klas = (String) auditConf.get("class");
if (klas == null) {
throw new SolrException(SERVER_ERROR, "class is required for auditlogger plugin");
}
log.info("Initializing auditlogger plugin: " + klas);
AuditLoggerPlugin p = loader.newInstance(klas, AuditLoggerPlugin.class);
if (p.getClass().equals(this.getClass())) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Cannot nest MultiDestinationAuditLogger");
}
p.init(auditConf);
return p;
} else {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Empty config when creating audit plugin");
}
}
@Override
public void inform(ResourceLoader loader) {
this.loader = loader;
}
@Override
public void initializeMetrics(SolrMetricManager manager, String registryName, String tag, String scope) {
super.initializeMetrics(manager, registryName, tag, scope);
plugins.forEach(p -> p.initializeMetrics(manager, registryName, tag, scope));
}
@Override
public void close() throws IOException {
plugins.forEach(p -> {
try {
p.close();
} catch (IOException e) {
log.error("Exception trying to close {}", p.getName());
}
});
super.close();
}
}

View File

@ -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.
*/
package org.apache.solr.security;
import java.lang.invoke.MethodHandles;
import java.util.Map;
import org.apache.solr.common.SolrException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Audit logger that writes to the Solr log.
* This interface may change in next release and is marked experimental
* @since 8.1.0
* @lucene.experimental
*/
public class SolrLogAuditLoggerPlugin extends AuditLoggerPlugin {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
/**
* Initialize the plugin from security.json
* @param pluginConfig the config for the plugin
*/
@Override
public void init(Map<String, Object> pluginConfig) {
super.init(pluginConfig);
setFormatter(event ->
new StringBuilder()
.append("type=\"").append(event.getEventType().name()).append("\"")
.append(" message=\"").append(event.getMessage()).append("\"")
.append(" method=\"").append(event.getHttpMethod()).append("\"")
.append(" status=\"").append(event.getStatus()).append("\"")
.append(" requestType=\"").append(event.getRequestType()).append("\"")
.append(" username=\"").append(event.getUsername()).append("\"")
.append(" resource=\"").append(event.getResource()).append("\"")
.append(" queryString=\"").append(event.getHttpQueryString()).append("\"")
.append(" collections=").append(event.getCollections()).toString());
if (pluginConfig.size() > 0) {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Plugin config was not fully consumed. Remaining parameters are " + pluginConfig);
}
log.debug("Initialized SolrLogAuditLoggerPlugin");
}
/**
* Audit logs an event to Solr log. The event should be a {@link AuditEvent} to be able to pull context info
* @param event the event to log
*/
@Override
public void audit(AuditEvent event) {
switch (event.getLevel()) {
case INFO:
log.info(formatter.formatEvent(event));
break;
case WARN:
log.warn(formatter.formatEvent(event));
break;
case ERROR:
log.error(formatter.formatEvent(event));
break;
}
}
}

View File

@ -92,6 +92,8 @@ import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.QueryResponseWriter;
import org.apache.solr.response.QueryResponseWriterUtil;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuditEvent;
import org.apache.solr.security.AuditEvent.EventType;
import org.apache.solr.security.AuthenticationPlugin;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.AuthorizationContext.CollectionRequest;
@ -462,6 +464,9 @@ public class HttpSolrCall {
if (solrDispatchFilter.abortErrorMessage != null) {
sendError(500, solrDispatchFilter.abortErrorMessage);
if (shouldAudit(EventType.ERROR)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.ERROR, getReq()));
}
return RETURN;
}
@ -481,13 +486,22 @@ public class HttpSolrCall {
for (Map.Entry<String, String> e : headers.entrySet()) response.setHeader(e.getKey(), e.getValue());
}
log.debug("USER_REQUIRED "+req.getHeader("Authorization")+" "+ req.getUserPrincipal());
if (shouldAudit(EventType.REJECTED)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.REJECTED, req, context));
}
}
if (!(authResponse.statusCode == HttpStatus.SC_ACCEPTED) && !(authResponse.statusCode == HttpStatus.SC_OK)) {
log.info("USER_REQUIRED auth header {} context : {} ", req.getHeader("Authorization"), context);
sendError(authResponse.statusCode,
"Unauthorized request, Response code: " + authResponse.statusCode);
if (shouldAudit(EventType.UNAUTHORIZED)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.UNAUTHORIZED, req, context));
}
return RETURN;
}
if (shouldAudit(EventType.AUTHORIZED)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.AUTHORIZED, req, context));
}
}
HttpServletResponse resp = response;
@ -514,6 +528,13 @@ public class HttpSolrCall {
*/
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, solrRsp));
execute(solrRsp);
if (shouldAudit()) {
EventType eventType = solrRsp.getException() == null ? EventType.COMPLETED : EventType.ERROR;
if (shouldAudit(eventType)) {
cores.getAuditLoggerPlugin().doAudit(
new AuditEvent(eventType, req, getAuthCtx(), solrReq.getRequestTimer().getTime(), solrRsp.getException()));
}
}
HttpCacheHeaderUtil.checkHttpCachingVeto(solrRsp, resp, reqMethod);
Iterator<Map.Entry<String, String>> headers = solrRsp.httpHeaders();
while (headers.hasNext()) {
@ -528,6 +549,9 @@ public class HttpSolrCall {
default: return action;
}
} catch (Throwable ex) {
if (shouldAudit(EventType.ERROR)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.ERROR, ex, req));
}
sendError(ex);
// walk the the entire cause chain to search for an Error
Throwable t = ex;
@ -547,6 +571,14 @@ public class HttpSolrCall {
}
private boolean shouldAudit() {
return cores.getAuditLoggerPlugin() != null;
}
private boolean shouldAudit(AuditEvent.EventType eventType) {
return shouldAudit() && cores.getAuditLoggerPlugin().shouldLog(eventType);
}
private boolean shouldAuthorize() {
if(PublicKeyHandler.PATH.equals(path)) return false;
//admin/info/key is the path where public key is exposed . it is always unsecured
@ -577,7 +609,7 @@ public class HttpSolrCall {
//TODO using Http2Client
private void remoteQuery(String coreUrl, HttpServletResponse resp) throws IOException {
HttpRequestBase method = null;
HttpRequestBase method;
HttpEntity httpEntity = null;
try {
String urlstr = coreUrl + queryParams.toQueryString();
@ -671,7 +703,7 @@ public class HttpSolrCall {
solrParams = SolrRequestParsers.parseQueryString(req.getQueryString());
} else {
// we have no params at all, use empty ones:
solrParams = new MapSolrParams(Collections.<String, String>emptyMap());
solrParams = new MapSolrParams(Collections.emptyMap());
}
solrReq = new SolrQueryRequestBase(core, solrParams) {
};
@ -722,6 +754,13 @@ public class HttpSolrCall {
QueryResponseWriter respWriter = SolrCore.DEFAULT_RESPONSE_WRITERS.get(solrReq.getParams().get(CommonParams.WT));
if (respWriter == null) respWriter = getResponseWriter();
writeResponse(solrResp, respWriter, Method.getMethod(req.getMethod()));
if (shouldAudit()) {
EventType eventType = solrResp.getException() == null ? EventType.COMPLETED : EventType.ERROR;
if (shouldAudit(eventType)) {
cores.getAuditLoggerPlugin().doAudit(
new AuditEvent(eventType, req, getAuthCtx(), solrReq.getRequestTimer().getTime(), solrResp.getException()));
}
}
}
/**

View File

@ -72,6 +72,7 @@ import org.apache.solr.metrics.AltBufferPoolMetricSet;
import org.apache.solr.metrics.MetricsMap;
import org.apache.solr.metrics.OperatingSystemMetricSet;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.security.AuditEvent;
import org.apache.solr.security.AuthenticationPlugin;
import org.apache.solr.security.PKIAuthenticationPlugin;
import org.apache.solr.security.PublicKeyHandler;
@ -81,6 +82,8 @@ import org.apache.solr.util.configuration.SSLConfigurationsFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.security.AuditEvent.EventType;
/**
* This filter looks at the incoming URL maps them to handlers defined in solrconfig.xml
*
@ -452,6 +455,9 @@ public class SolrDispatchFilter extends BaseSolrFilter {
final AtomicBoolean isAuthenticated = new AtomicBoolean(false);
AuthenticationPlugin authenticationPlugin = cores.getAuthenticationPlugin();
if (authenticationPlugin == null) {
if (shouldAudit(EventType.ANONYMOUS)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.ANONYMOUS, request));
}
return true;
} else {
// /admin/info/key must be always open. see SOLR-9188
@ -494,8 +500,14 @@ public class SolrDispatchFilter extends BaseSolrFilter {
// multiple code paths.
if (!requestContinues || !isAuthenticated.get()) {
response.flushBuffer();
if (shouldAudit(EventType.REJECTED)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.REJECTED, request));
}
return false;
}
if (shouldAudit(EventType.AUTHENTICATED)) {
cores.getAuditLoggerPlugin().doAudit(new AuditEvent(EventType.AUTHENTICATED, request));
}
return true;
}
@ -553,6 +565,15 @@ public class SolrDispatchFilter extends BaseSolrFilter {
+ "Closing the request stream prevents us from gauranteeing ourselves that streams are fully read for proper connection reuse."
+ "Let the container manage the lifecycle of these streams when possible.";
/**
* Check if audit logging is enabled and should happen for given event type
* @param eventType the audit event
*/
private boolean shouldAudit(AuditEvent.EventType eventType) {
return cores.getAuditLoggerPlugin() != null && cores.getAuditLoggerPlugin().shouldLog(eventType);
}
/**
* Wrap the request's input stream with a close shield. If this is a
* retry, we will assume that the stream has already been wrapped and do nothing.

View File

@ -0,0 +1,9 @@
{
"auditlogging": {
"class": "solr.CallbackAuditLoggerPlugin",
"callbackPort": "_PORT_",
"async": _ASYNC_,
"delay": "_DELAY_",
"muteRules": _MUTERULES_
},_AUTH_
}

View File

@ -39,5 +39,6 @@ public class TokenizerChainTest extends SolrTestCaseJ4 {
tff);
assertEquals(new BytesRef("fooba"),
tokenizerChain.normalize(fieldName, "FOOB\u00c4"));
tokenizerChain.close();
}
}

View File

@ -0,0 +1,384 @@
/*
* 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.solr.security;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.lang.invoke.MethodHandles;
import java.net.ServerSocket;
import java.net.Socket;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.commons.io.FileUtils;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.MiniSolrCloudCluster;
import org.apache.solr.cloud.SolrCloudAuthTestCase;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.security.AuditEvent.EventType;
import org.apache.solr.security.AuditEvent.RequestType;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.client.solrj.request.CollectionAdminRequest.getClusterStatus;
import static org.apache.solr.client.solrj.request.CollectionAdminRequest.getOverseerStatus;
import static org.apache.solr.security.AuditEvent.EventType.COMPLETED;
import static org.apache.solr.security.AuditEvent.EventType.ERROR;
import static org.apache.solr.security.AuditEvent.EventType.REJECTED;
import static org.apache.solr.security.AuditEvent.EventType.UNAUTHORIZED;
import static org.apache.solr.security.AuditEvent.RequestType.ADMIN;
import static org.apache.solr.security.AuditEvent.RequestType.SEARCH;
/**
* Validate that audit logging works in a live cluster
*/
@SolrTestCaseJ4.SuppressSSL
public class AuditLoggerIntegrationTest extends SolrCloudAuthTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected static final int NUM_SERVERS = 1;
protected static final int NUM_SHARDS = 1;
protected static final int REPLICATION_FACTOR = 1;
// Use a harness per thread to be able to beast this test
private ThreadLocal<AuditTestHarness> testHarness = new ThreadLocal<>();
@Override
@Before
public void setUp() throws Exception {
super.setUp();
testHarness.set(new AuditTestHarness());
}
@Override
@After
public void tearDown() throws Exception {
testHarness.get().close();
super.tearDown();
}
@Test
public void testSynchronous() throws Exception {
setupCluster(false, 0, false, null);
runAdminCommands();
assertAuditMetricsMinimums(testHarness.get().cluster, CallbackAuditLoggerPlugin.class.getSimpleName(), 3, 0);
testHarness.get().shutdownCluster();
assertThreeAdminEvents();
}
@Test
public void testAsync() throws Exception {
setupCluster(true, 0, false, null);
runAdminCommands();
assertAuditMetricsMinimums(testHarness.get().cluster, CallbackAuditLoggerPlugin.class.getSimpleName(), 3, 0);
testHarness.get().shutdownCluster();
assertThreeAdminEvents();
}
@Test
public void testAsyncWithQueue() throws Exception {
setupCluster(true, 100, false, null);
runAdminCommands();
assertAuditMetricsMinimums(testHarness.get().cluster, CallbackAuditLoggerPlugin.class.getSimpleName(), 3, 0);
ArrayList<MetricRegistry> registries = getMetricsReigstries(testHarness.get().cluster);
Timer timer = ((Timer) registries.get(0).getMetrics().get("SECURITY./auditlogging.CallbackAuditLoggerPlugin.queuedTime"));
double meanTimeOnQueue = timer.getSnapshot().getMean() / 1000000; // Convert to ms
assertTrue(meanTimeOnQueue > 50);
testHarness.get().shutdownCluster();
assertThreeAdminEvents();
}
@Test
public void testMuteAdminListCollections() throws Exception {
setupCluster(false, 0, false, "[ \"type:UNKNOWN\", [ \"path:/admin\", \"param:action=LIST\" ] ]");
runAdminCommands();
testHarness.get().shutdownCluster();
waitForAuditEventCallbacks(2);
assertEquals(2, testHarness.get().receiver.getBuffer().size());
}
@Test
public void searchWithException() throws Exception {
setupCluster(false, 0, false, null);
try {
testHarness.get().cluster.getSolrClient().request(CollectionAdminRequest.createCollection("test", 1, 1));
testHarness.get().cluster.getSolrClient().query("test", new MapSolrParams(Collections.singletonMap("q", "a(bc")));
fail("Query should fail");
} catch (SolrException ex) {
waitForAuditEventCallbacks(3);
CallbackReceiver receiver = testHarness.get().receiver;
assertAuditEvent(receiver.popEvent(), COMPLETED, "/admin/cores");
assertAuditEvent(receiver.popEvent(), COMPLETED, "/admin/collections");
assertAuditEvent(receiver.popEvent(), ERROR,"/select", SEARCH, null, 400);
}
}
@Test
public void auth() throws Exception {
setupCluster(false, 0, true, null);
CloudSolrClient client = testHarness.get().cluster.getSolrClient();
try {
CollectionAdminRequest.List request = new CollectionAdminRequest.List();
client.request(request);
request.setBasicAuthCredentials("solr", "SolrRocks");
client.request(request);
CollectionAdminRequest.Create createRequest = CollectionAdminRequest.createCollection("test", 1, 1);
client.request(createRequest);
fail("Call should fail with 401");
} catch (SolrException ex) {
waitForAuditEventCallbacks(3);
CallbackReceiver receiver = testHarness.get().receiver;
assertAuditEvent(receiver.popEvent(), COMPLETED, "/admin/collections", ADMIN, null, 200, "action", "LIST");
AuditEvent e = receiver.popEvent();
System.out.println(new AuditLoggerPlugin.JSONAuditEventFormatter().formatEvent(e));
assertAuditEvent(e, COMPLETED, "/admin/collections", ADMIN, "solr", 200, "action", "LIST");
assertAuditEvent(receiver.popEvent(), REJECTED, "/admin/collections", ADMIN, null,401);
}
try {
CollectionAdminRequest.Create createRequest = CollectionAdminRequest.createCollection("test", 1, 1);
createRequest.setBasicAuthCredentials("solr", "wrongPW");
client.request(createRequest);
fail("Call should fail with 403");
} catch (SolrException ex) {
waitForAuditEventCallbacks(1);
CallbackReceiver receiver = testHarness.get().receiver;
assertAuditEvent(receiver.popEvent(), UNAUTHORIZED, "/admin/collections", ADMIN, null,403);
}
}
private void assertAuditEvent(AuditEvent e, EventType type, String path, String... params) {
assertAuditEvent(e, type, path, null, null,null, params);
}
private void assertAuditEvent(AuditEvent e, EventType type, String path, RequestType requestType, String username, Integer status, String... params) {
assertEquals(type, e.getEventType());
assertEquals(path, e.getResource());
if (requestType != null) {
assertEquals(requestType, e.getRequestType());
}
if (username != null) {
assertEquals(username, e.getUsername());
}
if (status != null) {
assertEquals(status.intValue(), e.getStatus());
}
if (params != null && params.length > 0) {
List<String> p = new LinkedList<>(Arrays.asList(params));
while (p.size() >= 2) {
String val = e.getSolrParamAsString(p.get(0));
assertEquals(p.get(1), val);
p.remove(0);
p.remove(0);
}
}
}
private void waitForAuditEventCallbacks(int number) throws InterruptedException {
CallbackReceiver receiver = testHarness.get().receiver;
int count = 0;
while(receiver.buffer.size() < number) {
Thread.sleep(100);
if (++count >= 30) fail("Failed waiting for " + number + " callbacks");
}
}
private ArrayList<MetricRegistry> getMetricsReigstries(MiniSolrCloudCluster cluster) {
ArrayList<MetricRegistry> registries = new ArrayList<>();
cluster.getJettySolrRunners().forEach(r -> {
MetricRegistry registry = r.getCoreContainer().getMetricManager().registry("solr.node");
assertNotNull(registry);
registries.add(registry);
});
return registries;
}
private void runAdminCommands() throws IOException, SolrServerException {
SolrClient client = testHarness.get().cluster.getSolrClient();
CollectionAdminRequest.listCollections(client);
client.request(getClusterStatus());
client.request(getOverseerStatus());
}
private void assertThreeAdminEvents() throws Exception {
CallbackReceiver receiver = testHarness.get().receiver;
waitForAuditEventCallbacks(3);
assertEquals(3, receiver.getTotalCount());
assertEquals(3, receiver.getCountForPath("/admin/collections"));
AuditEvent e = receiver.getBuffer().pop();
assertEquals(COMPLETED, e.getEventType());
assertEquals("GET", e.getHttpMethod());
assertEquals("action=LIST&wt=javabin&version=2", e.getHttpQueryString());
assertEquals("LIST", e.getSolrParamAsString("action"));
assertEquals("javabin", e.getSolrParamAsString("wt"));
e = receiver.getBuffer().pop();
assertEquals(COMPLETED, e.getEventType());
assertEquals("GET", e.getHttpMethod());
assertEquals("CLUSTERSTATUS", e.getSolrParamAsString("action"));
e = receiver.getBuffer().pop();
assertEquals(COMPLETED, e.getEventType());
assertEquals("GET", e.getHttpMethod());
assertEquals("OVERSEERSTATUS", e.getSolrParamAsString("action"));
}
private static String AUTH_SECTION = ",\n" +
" \"authentication\":{\n" +
" \"blockUnknown\":\"false\",\n" +
" \"class\":\"solr.BasicAuthPlugin\",\n" +
" \"credentials\":{\"solr\":\"orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y= Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw=\"}},\n" +
" \"authorization\":{\n" +
" \"class\":\"solr.RuleBasedAuthorizationPlugin\",\n" +
" \"user-role\":{\"solr\":\"admin\"},\n" +
" \"permissions\":[{\"name\":\"collection-admin-edit\",\"role\":\"admin\"}]\n" +
" }\n";
private void setupCluster(boolean async, int delay, boolean enableAuth, String muteRulesJson) throws Exception {
String securityJson = FileUtils.readFileToString(TEST_PATH().resolve("security").resolve("auditlog_plugin_security.json").toFile(), StandardCharsets.UTF_8);
securityJson = securityJson.replace("_PORT_", Integer.toString(testHarness.get().callbackPort));
securityJson = securityJson.replace("_ASYNC_", Boolean.toString(async));
securityJson = securityJson.replace("_DELAY_", Integer.toString(delay));
securityJson = securityJson.replace("_AUTH_", enableAuth ? AUTH_SECTION : "");
securityJson = securityJson.replace("_MUTERULES_", muteRulesJson != null ? muteRulesJson : "[]");
MiniSolrCloudCluster myCluster = new Builder(NUM_SERVERS, createTempDir())
.withSecurityJson(securityJson)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
.build();
myCluster.waitForAllNodes(10);
testHarness.get().setCluster(myCluster);
}
/**
* Listening for socket callbacks in background thread from the custom CallbackAuditLoggerPlugin
*/
private class CallbackReceiver implements Runnable, AutoCloseable {
private final ServerSocket serverSocket;
private AtomicInteger count = new AtomicInteger();
private Map<String,AtomicInteger> resourceCounts = new HashMap<>();
private LinkedList<AuditEvent> buffer = new LinkedList<>();
CallbackReceiver() throws IOException {
serverSocket = new ServerSocket(0);
}
int getTotalCount() {
return count.get();
}
int getCountForPath(String path) {
return resourceCounts.getOrDefault(path, new AtomicInteger()).get();
}
public int getPort() {
return serverSocket.getLocalPort();
}
@Override
public void run() {
try {
log.info("Listening for audit callbacks on on port {}", serverSocket.getLocalPort());
Socket socket = serverSocket.accept();
BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8));
while (!Thread.currentThread().isInterrupted()) {
if (!reader.ready()) continue;
ObjectMapper om = new ObjectMapper();
om.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
AuditEvent event = om.readValue(reader.readLine(), AuditEvent.class);
buffer.add(event);
String r = event.getResource();
log.info("Received audit event for path " + r);
count.incrementAndGet();
AtomicInteger resourceCounter = resourceCounts.get(r);
if (resourceCounter == null) {
resourceCounter = new AtomicInteger(1);
resourceCounts.put(r, resourceCounter);
} else {
resourceCounter.incrementAndGet();
}
}
} catch (IOException e) {
log.info("Socket closed", e);
}
}
@Override
public void close() throws Exception {
serverSocket.close();
}
protected LinkedList<AuditEvent> getBuffer() {
return buffer;
}
protected AuditEvent popEvent() {
return buffer.pop();
}
}
private class AuditTestHarness implements AutoCloseable {
CallbackReceiver receiver;
int callbackPort;
Thread receiverThread;
private MiniSolrCloudCluster cluster;
AuditTestHarness() throws IOException {
receiver = new CallbackReceiver();
callbackPort = receiver.getPort();
receiverThread = new DefaultSolrThreadFactory("auditTestCallback").newThread(receiver);
receiverThread.start();
}
@Override
public void close() throws Exception {
shutdownCluster();
receiverThread.interrupt();
receiver.close();
receiverThread = null;
}
public void shutdownCluster() throws Exception {
if (cluster != null) cluster.shutdown();
}
public void setCluster(MiniSolrCloudCluster cluster) {
this.cluster = cluster;
}
}
}

View File

@ -0,0 +1,176 @@
/*
* 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.solr.security;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class AuditLoggerPluginTest extends SolrTestCaseJ4 {
protected static final Date SAMPLE_DATE = new Date(1234567890);
protected static final AuditEvent EVENT_ANONYMOUS = new AuditEvent(AuditEvent.EventType.ANONYMOUS)
.setHttpMethod("GET")
.setMessage("Anonymous")
.setResource("/collection1")
.setDate(SAMPLE_DATE);
protected static final AuditEvent EVENT_ANONYMOUS_REJECTED = new AuditEvent(AuditEvent.EventType.ANONYMOUS_REJECTED)
.setHttpMethod("GET")
.setMessage("Anonymous rejected")
.setResource("/collection1");
protected static final AuditEvent EVENT_AUTHENTICATED = new AuditEvent(AuditEvent.EventType.AUTHENTICATED)
.setUsername("Jan")
.setHttpMethod("GET")
.setMessage("Authenticated")
.setDate(SAMPLE_DATE)
.setResource("/collection1");
protected static final AuditEvent EVENT_REJECTED = new AuditEvent(AuditEvent.EventType.REJECTED)
.setUsername("Jan")
.setHttpMethod("POST")
.setMessage("Wrong password")
.setDate(SAMPLE_DATE)
.setResource("/collection1");
protected static final AuditEvent EVENT_AUTHORIZED = new AuditEvent(AuditEvent.EventType.AUTHORIZED)
.setUsername("Per")
.setClientIp("192.168.0.10")
.setHttpMethod("GET")
.setMessage("Async")
.setDate(SAMPLE_DATE)
.setResource("/collection1");
protected static final AuditEvent EVENT_UNAUTHORIZED = new AuditEvent(AuditEvent.EventType.UNAUTHORIZED)
.setUsername("Jan")
.setHttpMethod("POST")
.setMessage("No access to collection1")
.setDate(SAMPLE_DATE)
.setResource("/collection1");
protected static final AuditEvent EVENT_ERROR = new AuditEvent(AuditEvent.EventType.ERROR)
.setUsername("Jan")
.setHttpMethod("POST")
.setMessage("Error occurred")
.setDate(SAMPLE_DATE)
.setSolrParams(Collections.singletonMap("action", Collections.singletonList("DELETE")))
.setResource("/admin/collections");
protected static final AuditEvent EVENT_UPDATE = new AuditEvent(AuditEvent.EventType.COMPLETED)
.setUsername("updateuser")
.setHttpMethod("POST")
.setRequestType(AuditEvent.RequestType.UPDATE)
.setMessage("Success")
.setDate(SAMPLE_DATE)
.setCollections(Collections.singletonList("updatecoll"))
.setRequestType(AuditEvent.RequestType.UPDATE)
.setResource("/update");
protected static final AuditEvent EVENT_STREAMING = new AuditEvent(AuditEvent.EventType.COMPLETED)
.setUsername("streaminguser")
.setHttpMethod("POST")
.setRequestType(AuditEvent.RequestType.STREAMING)
.setMessage("Success")
.setDate(SAMPLE_DATE)
.setCollections(Collections.singletonList("streamcoll"))
.setResource("/stream");
private MockAuditLoggerPlugin plugin;
private HashMap<String, Object> config;
@Before
public void setUp() throws Exception {
super.setUp();
plugin = new MockAuditLoggerPlugin();
config = new HashMap<>();
config.put("async", false);
plugin.init(config);
}
@Override
@After
public void tearDown() throws Exception {
plugin.close();
super.tearDown();
}
@Test
public void init() {
config = new HashMap<>();
config.put("eventTypes", Collections.singletonList("REJECTED"));
config.put("async", false);
plugin.init(config);
assertTrue(plugin.shouldLog(EVENT_REJECTED.getEventType()));
assertFalse(plugin.shouldLog(EVENT_UNAUTHORIZED.getEventType()));
}
@Test
public void shouldLog() {
// Default types
assertTrue(plugin.shouldLog(EVENT_ANONYMOUS_REJECTED.getEventType()));
assertTrue(plugin.shouldLog(EVENT_REJECTED.getEventType()));
assertTrue(plugin.shouldLog(EVENT_UNAUTHORIZED.getEventType()));
assertTrue(plugin.shouldLog(EVENT_ERROR.getEventType()));
assertFalse(plugin.shouldLog(EVENT_ANONYMOUS.getEventType()));
assertFalse(plugin.shouldLog(EVENT_AUTHENTICATED.getEventType()));
assertFalse(plugin.shouldLog(EVENT_AUTHORIZED.getEventType()));
}
@Test(expected = SolrException.class)
public void invalidMuteRule() {
config.put("muteRules", Collections.singletonList("foo:bar"));
plugin.init(config);
}
@Test
public void shouldMute() {
List<Object> rules = new ArrayList<>();
rules.add("type:STREAMING");
rules.add(Arrays.asList("user:updateuser", "collection:updatecoll"));
rules.add(Arrays.asList("path:/admin/collection", "param:action=DELETE"));
rules.add("ip:192.168.0.10");
config.put("muteRules",rules);
plugin.init(config);
assertFalse(plugin.shouldMute(EVENT_ANONYMOUS));
assertFalse(plugin.shouldMute(EVENT_AUTHENTICATED));
assertTrue(plugin.shouldMute(EVENT_STREAMING)); // type:STREAMING
assertTrue(plugin.shouldMute(EVENT_UPDATE)); // updateuser, updatecoll
assertTrue(plugin.shouldMute(EVENT_ERROR)); // admin/collection action=DELETE
assertTrue(plugin.shouldMute(EVENT_AUTHORIZED)); // ip
}
@Test
public void audit() {
plugin.doAudit(EVENT_ANONYMOUS_REJECTED);
plugin.doAudit(EVENT_REJECTED);
assertEquals(1, plugin.typeCounts.getOrDefault("ANONYMOUS_REJECTED", new AtomicInteger()).get());
assertEquals(1, plugin.typeCounts.getOrDefault("REJECTED", new AtomicInteger()).get());
assertEquals(2, plugin.events.size());
}
@Test
public void jsonEventFormatter() {
assertEquals("{\"message\":\"Anonymous\",\"level\":\"INFO\",\"date\":" + SAMPLE_DATE.getTime() + ",\"solrParams\":{},\"solrPort\":0,\"resource\":\"/collection1\",\"httpMethod\":\"GET\",\"eventType\":\"ANONYMOUS\",\"status\":-1,\"qtime\":-1.0}",
plugin.formatter.formatEvent(EVENT_ANONYMOUS));
assertEquals("{\"message\":\"Authenticated\",\"level\":\"INFO\",\"date\":" + SAMPLE_DATE.getTime() + ",\"username\":\"Jan\",\"solrParams\":{},\"solrPort\":0,\"resource\":\"/collection1\",\"httpMethod\":\"GET\",\"eventType\":\"AUTHENTICATED\",\"status\":-1,\"qtime\":-1.0}",
plugin.formatter.formatEvent(EVENT_AUTHENTICATED));
}
}

View File

@ -0,0 +1,76 @@
/*
* 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.solr.security;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.lang.invoke.MethodHandles;
import java.net.Socket;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Special test-only audit logger which will send the path (e.g. /select) as a callback to the running test
*/
public class CallbackAuditLoggerPlugin extends AuditLoggerPlugin {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private int callbackPort;
private Socket socket;
private PrintWriter out;
private int delay;
/**
* Opens a socket to send a callback, e.g. to a running test client
* @param event the audit event
*/
@Override
public void audit(AuditEvent event) {
if (delay > 0) {
log.info("Sleeping for {}ms before sending callback", delay);
try {
Thread.sleep(delay);
} catch (InterruptedException e) {
}
}
out.write(formatter.formatEvent(event) + "\n");
out.flush();
log.info("Sent audit callback {} to localhost:{}", formatter.formatEvent(event), callbackPort);
}
@Override
public void init(Map<String, Object> pluginConfig) {
super.init(pluginConfig);
callbackPort = Integer.parseInt((String) pluginConfig.get("callbackPort"));
delay = Integer.parseInt((String) pluginConfig.get("delay"));
try {
socket = new Socket("localhost", callbackPort);
out = new PrintWriter(new OutputStreamWriter(socket.getOutputStream(), StandardCharsets.UTF_8), true);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public void close() throws IOException {
if (socket != null) socket.close();
super.close();
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.solr.security;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class MockAuditLoggerPlugin extends AuditLoggerPlugin {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public List<AuditEvent> events = new ArrayList<>();
public Map<String,AtomicInteger> typeCounts = new HashMap<>();
/**
* Audits an event to an internal list that can be inspected later by the test code
* @param event the audit event
*/
@Override
public void audit(AuditEvent event) {
events.add(event);
incrementType(event.getEventType().name());
log.info("#{} - {}", events.size(), typeCounts);
}
private void incrementType(String type) {
if (!typeCounts.containsKey(type))
typeCounts.put(type, new AtomicInteger(0));
typeCounts.get(type).incrementAndGet();
}
public void reset() {
events.clear();
typeCounts.clear();
}
}

View File

@ -0,0 +1,82 @@
/*
* 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.solr.security;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.core.SolrResourceLoader;
import org.junit.Test;
public class MultiDestinationAuditLoggerTest extends SolrTestCaseJ4 {
@Test
public void init() throws IOException {
MultiDestinationAuditLogger al = new MultiDestinationAuditLogger();
Map<String,Object> config = new HashMap<>();
config.put("class", "solr.MultiDestinationAuditLogger");
config.put("async", false);
config.put("eventTypes", Arrays.asList(AuditEvent.EventType.COMPLETED.name()));
ArrayList<Map<String, Object>> plugins = new ArrayList<Map<String, Object>>();
Map<String,Object> conf1 = new HashMap<>();
conf1.put("class", "solr.SolrLogAuditLoggerPlugin");
conf1.put("async", false);
conf1.put("eventTypes", Arrays.asList(AuditEvent.EventType.ANONYMOUS.name()));
plugins.add(conf1);
Map<String,Object> conf2 = new HashMap<>();
conf2.put("class", "solr.MockAuditLoggerPlugin");
conf2.put("async", false);
conf2.put("eventTypes", Arrays.asList(AuditEvent.EventType.AUTHENTICATED.name()));
plugins.add(conf2);
config.put("plugins", plugins);
SolrResourceLoader loader = new SolrResourceLoader();
al.inform(loader);
al.init(config);
al.doAudit(new AuditEvent(AuditEvent.EventType.ANONYMOUS).setUsername("me"));
assertEquals(0, ((MockAuditLoggerPlugin)al.plugins.get(1)).events.size()); // not configured for ANONYMOUS
al.doAudit(new AuditEvent(AuditEvent.EventType.AUTHENTICATED).setUsername("me"));
assertEquals(1, ((MockAuditLoggerPlugin)al.plugins.get(1)).events.size()); // configured for authenticated
assertFalse(al.shouldLog(AuditEvent.EventType.ERROR));
assertFalse(al.shouldLog(AuditEvent.EventType.UNAUTHORIZED));
assertTrue(al.shouldLog(AuditEvent.EventType.COMPLETED));
assertTrue(al.shouldLog(AuditEvent.EventType.ANONYMOUS));
assertTrue(al.shouldLog(AuditEvent.EventType.AUTHENTICATED));
assertEquals(0, config.size());
al.close();
loader.close();
}
@Test
public void wrongConfigParam() throws IOException {
MultiDestinationAuditLogger al = new MultiDestinationAuditLogger();
Map<String,Object> config = new HashMap<>();
config.put("class", "solr.MultiDestinationAuditLogger");
config.put("foo", "Should complain");
al.init(config);
assertEquals(1, config.size());
al.close();
}
}

View File

@ -0,0 +1,72 @@
/*
* 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.solr.security;
import java.io.IOException;
import java.util.HashMap;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.apache.solr.security.AuditLoggerPluginTest.EVENT_ANONYMOUS;
import static org.apache.solr.security.AuditLoggerPluginTest.EVENT_AUTHENTICATED;
public class SolrLogAuditLoggerPluginTest extends SolrTestCaseJ4 {
private SolrLogAuditLoggerPlugin plugin;
private HashMap<String, Object> config;
@Override
@Before
public void setUp() throws Exception {
super.setUp();
plugin = new SolrLogAuditLoggerPlugin();
config = new HashMap<>();
config.put("async", false);
}
@Test(expected = SolrException.class)
public void badConfig() throws IOException {
config.put("invalid", "parameter");
plugin.init(config);
}
@Test
public void audit() {
plugin.init(config);
plugin.doAudit(EVENT_ANONYMOUS);
}
@Test
public void eventFormatter() {
plugin.init(config);
assertEquals("type=\"ANONYMOUS\" message=\"Anonymous\" method=\"GET\" status=\"-1\" requestType=\"null\" username=\"null\" resource=\"/collection1\" queryString=\"null\" collections=null",
plugin.formatter.formatEvent(EVENT_ANONYMOUS));
assertEquals("type=\"AUTHENTICATED\" message=\"Authenticated\" method=\"GET\" status=\"-1\" requestType=\"null\" username=\"Jan\" resource=\"/collection1\" queryString=\"null\" collections=null",
plugin.formatter.formatEvent(EVENT_AUTHENTICATED));
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
plugin.close();
}
}

View File

@ -0,0 +1,154 @@
= Audit Logging
// 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.
Solr has the ability to log an audit trail of all events in the system.
Audit loggers are pluggable to suit any possible format or log destination.
[quote]
An audit trail (also called audit log) is a security-relevant chronological record, set of records, and/or destination and source of records that provide documentary evidence of the sequence of activities that have affected at any time a specific operation, procedure, or event. (https://en.wikipedia.org/wiki/Audit_trail[Wikipedia])
== Configuration in security.json
Audit logging is configured in `security.json` under the `auditlogging` key.
The example `security.json` below configures synchronous audit logging to Solr default log file.
[source,json]
----
{
"auditlogging":{
"class": "solr.SolrLogAuditLoggerPlugin"
}
}
----
By default any AuditLogger plugin configured will log asynchronously in the background to avoid slowing down the requests. To make audit logging happen synchronously, add the parameter `async: false`. For async logging, you may optionally also configure queue size, number of threads and whether it should block when the queue is full or discard events:
[source,json]
----
{
"auditlogging":{
"class": "solr.SolrLogAuditLoggerPlugin",
"async": true,
"blockAsync" : false,
"numThreads" : 2,
"queueSize" : 4096,
"eventTypes": ["REJECTED", "ANONYMOUS_REJECTED", "UNAUTHORIZED", "COMPLETED" "ERROR"]
}
}
----
The defaults are `async: true`, `blockAsync: false`, `queueSize: 4096`. The default for `numThreads` is 2, or if the server has more than 4 CPU-cores then we use CPU-cores/2.
[#audit-event-types]
=== Event types
These are the event types triggered by the framework:
[%header,format=csv,separator=;]
|===
EventType;Usage
AUTHENTICATED;User successfully authenticated
REJECTED;Authentication request rejected
ANONYMOUS;Request proceeds with unknown user
ANONYMOUS_REJECTED;Request from unknown user rejected
AUTHORIZED;Authorization succeeded
UNAUTHORIZED;Authorization failed
COMPLETED;Request completed successfully
ERROR;Request was not executed due to an error
|===
By default only the final event types `REJECTED`, `ANONYMOUS_REJECTED`, `UNAUTHORIZED`, `COMPLETED` and `ERROR` are logged. What eventTypes are logged can be configured with the `eventTypes` configuration parameter.
=== Muting certain events
The configuration parameter `muteRules` lets you mute logging for certain events. You may specify multiple rules and combination of rules that will cause muting. You can mute by request type, username, collection name, path, request parameters or IP address. We'll explain through examples:
The below example will mute logging for all `SEARCH` requests as well as all requests made my user `johndoe` or from IP address `192.168.0.10`:
[source,json]
----
{
"auditlogging":{
"class": "solr.SolrLogAuditLoggerPlugin"
"muteRules": [ "type:SEARCH", "user:johndoe", "ip:192.168.0.10" ]
}
}
----
An mute rule may also be a list, in which case all must be true for muting to happen. The configuration below has three mute rules:
[source,json]
----
{
"auditlogging":{
"class": "solr.SolrLogAuditLoggerPlugin"
"muteRules": [
"ip:192.168.0.10", <1>
[ "path:/admin/collections", "param:action=LIST" ], <2>
[ "path:/admin/collections", "param:collection=test" ] <3>
]
}
}
----
<1> The first will mute all events from client IP `192.168.0.10`
<2> The second rule will mute collection admin requests with action=LIST
<3> The third rule will mute collection admin requests for the collection named `test`
Note how you can mix single string rules with lists of rules that must all match:
*Valid mute rules are:*
* `type:<request-type>` (request-type by name: `ADMIN`, `SEARCH`, `UPDATE`, `STREAMING`, `UNKNOWN`)
* `collection:<collection-name>` (collection by name)
* `user:<userid>` (user by userid)
* `path:</path/to/handler>` (request path relative to `/solr` or for search/update requests relative to collection. Path is prefix matched, i.e. `/admin` will mute any sub path as well.
* `ip:<ip-address>` (IPv4-address)
* `param:<param>=<value>` (request parameter)
=== Chaining multiple loggers
Using the `MultiDestinationAuditLogger` you can configure multiple audit logger plugins in a chain, to log to multiple destinations, as follows:
[source,json]
----
{
"auditlogging":{
"class" : "solr.MultiDestinationAuditLogger",
"plugins" : [
{ "class" : "solr.SolrLogAuditLoggerPlugin" },
{ "class" : "solr.MyOtherAuditPlugin",
"customParam" : "value"
}
]
}
}
----
== Metrics
AuditLoggerPlugins record metrics about count and timing of log requests, as well as queue size for async loggers. The metrics keys are all recorded on the `SECURITY` category, and each metric name are prefixed with a scope of `/auditlogging` and the class name of the logger, e.g. `SolrLogAuditLoggerPlugin`. The individual metrics are:
* `count` (type: meter. Records number and rate of audit logs done)
* `errors` (type: meter. Records number and rate of errors)
* `lost` (type: meter. Records number and rate of events lost due to queue full and `blockAsync=false`)
* `requestTimes` (type: timer. Records latency and perceniles for logging performance)
* `totalTime` (type: counter. Records total time spent)
* `queueCapacity` (type: gauge. Records the max size of the async logging queue)
* `queueSize` (type: gauge. Records the number of events currently waiting in the queue)
* `queuedTime` (type: timer. Records the amount of time events waited in queue. Adding this with requestTimes you get total time from event to logging complete)
* `async` (type: gauge. Tells whether this logger is in async mode)
TIP: If you expect a very high request rate and have a slow audit logger plugin, you may see that the `queueSize` and `queuedTime` metrics increase, and in worst case start dropping events and see an increase in `lost` count. In this case you may want to increas the `numThreads` setting.

View File

@ -1,5 +1,5 @@
= Securing Solr
:page-children: authentication-and-authorization-plugins, enabling-ssl
:page-children: authentication-and-authorization-plugins, enabling-ssl, audit-logging
// 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
@ -26,6 +26,7 @@ When planning how to secure Solr, you should consider which of the available fea
** <<authentication-and-authorization-plugins.adoc#authentication-and-authorization-plugins,Custom authentication or authorization plugin>>
* <<enabling-ssl.adoc#enabling-ssl,Enabling SSL>>
* If using SolrCloud, <<zookeeper-access-control.adoc#zookeeper-access-control,ZooKeeper Access Control>>
* <<audit-logging.adoc#audit-logging,Audit logging>> for recording an audit trail
[WARNING]
====

View File

@ -27,7 +27,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.function.Predicate;
import java.util.stream.Stream;
import com.codahale.metrics.Counter;
import com.codahale.metrics.Meter;
@ -59,12 +58,15 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final List<String> AUTH_METRICS_KEYS = Arrays.asList("errors", "requests", "authenticated",
"passThrough", "failWrongCredentials", "failMissingCredentials", "requestTimes", "totalTime");
private static final List<String> AUTH_METRICS_METER_KEYS = Arrays.asList("errors");
private static final List<String> AUTH_METRICS_METER_KEYS = Arrays.asList("errors", "count");
private static final List<String> AUTH_METRICS_TIMER_KEYS = Collections.singletonList("requestTimes");
private static final String METRICS_PREFIX_PKI = "SECURITY./authentication/pki.";
private static final String METRICS_PREFIX = "SECURITY./authentication.";
public static final Predicate NOT_NULL_PREDICATE = o -> o != null;
private static final List<String> AUDIT_METRICS_KEYS = Arrays.asList("count");
private static final List<String> AUTH_METRICS_TO_COMPARE = Arrays.asList("requests", "authenticated", "passThrough", "failWrongCredentials", "failMissingCredentials", "errors");
private static final List<String> AUDIT_METRICS_TO_COMPARE = Arrays.asList("count");
/**
* Used to check metric counts for PKI auth
*/
@ -83,9 +85,11 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
/**
* Common test method to be able to check security from any authentication plugin
* @param cluster the MiniSolrCloudCluster to fetch metrics from
* @param prefix the metrics key prefix, currently "SECURITY./authentication." for basic auth and "SECURITY./authentication/pki." for PKI
* @param keys what keys to examine
*/
Map<String,Long> countAuthMetrics(String prefix) {
Map<String,Long> countSecurityMetrics(MiniSolrCloudCluster cluster, String prefix, List<String> keys) {
List<Map<String, Metric>> metrics = new ArrayList<>();
cluster.getJettySolrRunners().forEach(r -> {
MetricRegistry registry = r.getCoreContainer().getMetricManager().registry("solr.node");
@ -94,14 +98,14 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
});
Map<String,Long> counts = new HashMap<>();
AUTH_METRICS_KEYS.forEach(k -> {
keys.forEach(k -> {
counts.put(k, sumCount(prefix, k, metrics));
});
return counts;
}
/**
* Common test method to be able to check security from any authentication plugin
* Common test method to be able to check auth metrics from any authentication plugin
* @param prefix the metrics key prefix, currently "SECURITY./authentication." for basic auth and "SECURITY./authentication/pki." for PKI
*/
private void assertAuthMetricsMinimums(String prefix, int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) throws InterruptedException {
@ -113,13 +117,13 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
expectedCounts.put("failMissingCredentials", (long) failMissingCredentials);
expectedCounts.put("errors", (long) errors);
Map<String, Long> counts = countAuthMetrics(prefix);
boolean success = isMetricsEqualOrLarger(expectedCounts, counts);
Map<String, Long> counts = countSecurityMetrics(cluster, prefix, AUTH_METRICS_KEYS);
boolean success = isMetricsEqualOrLarger(AUTH_METRICS_TO_COMPARE, expectedCounts, counts);
if (!success) {
log.info("First metrics count assert failed, pausing 2s before re-attempt");
Thread.sleep(2000);
counts = countAuthMetrics(prefix);
success = isMetricsEqualOrLarger(expectedCounts, counts);
counts = countSecurityMetrics(cluster, prefix, AUTH_METRICS_KEYS);
success = isMetricsEqualOrLarger(AUTH_METRICS_TO_COMPARE, expectedCounts, counts);
}
assertTrue("Expected metric minimums for prefix " + prefix + ": " + expectedCounts + ", but got: " + counts, success);
@ -130,8 +134,29 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
}
}
private boolean isMetricsEqualOrLarger(Map<String, Long> expectedCounts, Map<String, Long> actualCounts) {
return Stream.of("requests", "authenticated", "passThrough", "failWrongCredentials", "failMissingCredentials", "errors")
/**
* Common test method to be able to check audit metrics
* @param className the class name to be used for composing prefix, e.g. "SECURITY./auditlogging/SolrLogAuditLoggerPlugin"
*/
protected void assertAuditMetricsMinimums(MiniSolrCloudCluster cluster, String className, int count, int errors) throws InterruptedException {
String prefix = "SECURITY./auditlogging." + className + ".";
Map<String, Long> expectedCounts = new HashMap<>();
expectedCounts.put("count", (long) count);
Map<String, Long> counts = countSecurityMetrics(cluster, prefix, AUDIT_METRICS_KEYS);
boolean success = isMetricsEqualOrLarger(AUDIT_METRICS_TO_COMPARE, expectedCounts, counts);
if (!success) {
log.info("First metrics count assert failed, pausing 2s before re-attempt");
Thread.sleep(2000);
counts = countSecurityMetrics(cluster, prefix, AUDIT_METRICS_KEYS);
success = isMetricsEqualOrLarger(AUDIT_METRICS_TO_COMPARE, expectedCounts, counts);
}
assertTrue("Expected metric minimums for prefix " + prefix + ": " + expectedCounts + ", but got: " + counts, success);
}
private boolean isMetricsEqualOrLarger(List<String> metricsToCompare, Map<String, Long> expectedCounts, Map<String, Long> actualCounts) {
return metricsToCompare.stream()
.allMatch(k -> actualCounts.get(k).intValue() >= expectedCounts.get(k).intValue());
}