diff --git a/src/main/java/org/opensearch/security/OpenSearchSecurityPlugin.java b/src/main/java/org/opensearch/security/OpenSearchSecurityPlugin.java index 95f7d5d33b..136e40014c 100644 --- a/src/main/java/org/opensearch/security/OpenSearchSecurityPlugin.java +++ b/src/main/java/org/opensearch/security/OpenSearchSecurityPlugin.java @@ -110,6 +110,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.cache.query.QueryCache; import org.opensearch.index.shard.SearchOperationListener; +import org.opensearch.indices.IndicesService; import org.opensearch.indices.SystemIndexDescriptor; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.plugins.ClusterPlugin; @@ -779,7 +780,7 @@ public Collection createComponents(Client localClient, ClusterService cl auditLog = new NullAuditLog(); privilegesInterceptor = new PrivilegesInterceptor(resolver, clusterService, localClient, threadPool); } else { - dlsFlsValve = new DlsFlsValveImpl(); + dlsFlsValve = new DlsFlsValveImpl(settings, localClient, clusterService, resolver, xContentRegistry, threadPool.getThreadContext()); auditLog = new AuditLogImpl(settings, configPath, localClient, threadPool, resolver, clusterService, environment); privilegesInterceptor = new PrivilegesInterceptorImpl(resolver, clusterService, localClient, threadPool); } @@ -798,10 +799,10 @@ public Collection createComponents(Client localClient, ClusterService cl // DLS-FLS is enabled if not client and not disabled and not SSL only. final boolean dlsFlsEnabled = !SSLConfig.isSslOnlyMode(); evaluator = new PrivilegesEvaluator(clusterService, threadPool, cr, resolver, auditLog, - settings, privilegesInterceptor, cih, irr, dlsFlsEnabled); - - sf = new SecurityFilter(localClient, settings, evaluator, adminDns, dlsFlsValve, auditLog, threadPool, cs, compatConfig, irr, backendRegistry); + settings, privilegesInterceptor, cih, irr, dlsFlsEnabled, namedXContentRegistry); + sf = new SecurityFilter(localClient, settings, evaluator, adminDns, dlsFlsValve, auditLog, threadPool, cs, compatConfig, irr, backendRegistry, namedXContentRegistry); + final String principalExtractorClass = settings.get(SSLConfigConstants.SECURITY_SSL_TRANSPORT_PRINCIPAL_EXTRACTOR_CLASS, null); if(principalExtractorClass == null) { @@ -1125,12 +1126,14 @@ public static class GuiceHolder implements LifecycleComponent { private static RepositoriesService repositoriesService; private static RemoteClusterService remoteClusterService; + private static IndicesService indicesService; @Inject public GuiceHolder(final RepositoriesService repositoriesService, - final TransportService remoteClusterService) { + final TransportService remoteClusterService, IndicesService indicesService) { GuiceHolder.repositoriesService = repositoriesService; GuiceHolder.remoteClusterService = remoteClusterService.getRemoteClusterService(); + GuiceHolder.indicesService = indicesService; } public static RepositoriesService getRepositoriesService() { @@ -1141,6 +1144,10 @@ public static RemoteClusterService getRemoteClusterService() { return remoteClusterService; } + public static IndicesService getIndicesService() { + return indicesService; + } + @Override public void close() { } diff --git a/src/main/java/org/opensearch/security/configuration/DlsFilterLevelActionHandler.java b/src/main/java/org/opensearch/security/configuration/DlsFilterLevelActionHandler.java new file mode 100644 index 0000000000..09364238e1 --- /dev/null +++ b/src/main/java/org/opensearch/security/configuration/DlsFilterLevelActionHandler.java @@ -0,0 +1,475 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.configuration; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.slf4j.LoggerFactory; +import org.slf4j.Logger; +import org.opensearch.OpenSearchSecurityException; +import org.opensearch.action.ActionListener; +import org.opensearch.action.ActionRequest; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsRequest; +import org.opensearch.action.get.GetRequest; +import org.opensearch.action.get.GetResponse; +import org.opensearch.action.get.MultiGetItemResponse; +import org.opensearch.action.get.MultiGetRequest; +import org.opensearch.action.get.MultiGetResponse; +import org.opensearch.action.search.MultiSearchRequest; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.search.SearchScrollAction; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.document.DocumentField; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.util.concurrent.ThreadContext.StoredContext; +import org.opensearch.index.IndexService; +import org.opensearch.index.get.GetResult; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.query.BoolQueryBuilder; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.index.query.TermsQueryBuilder; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.indices.IndicesService; +import org.opensearch.script.mustache.MultiSearchTemplateAction; +import org.opensearch.script.mustache.SearchTemplateAction; +import org.opensearch.search.SearchHit; +import org.opensearch.search.builder.SearchSourceBuilder; +import org.opensearch.security.privileges.DocumentAllowList; +import org.opensearch.security.queries.QueryBuilderTraverser; +import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; +import org.opensearch.security.securityconf.EvaluatedDlsFlsConfig; +import org.opensearch.security.support.ConfigConstants; +import org.opensearch.security.support.ReflectiveAttributeAccessors; +import org.opensearch.security.support.SecurityUtils; + +public class DlsFilterLevelActionHandler { + private static final Logger log = LoggerFactory.getLogger(DlsFilterLevelActionHandler.class); + + private static final Function LOCAL_CLUSTER_ALIAS_GETTER = ReflectiveAttributeAccessors + .protectedObjectAttr("localClusterAlias", String.class); + + public static boolean handle(String action, ActionRequest request, ActionListener listener, EvaluatedDlsFlsConfig evaluatedDlsFlsConfig, + Resolved resolved, Client nodeClient, ClusterService clusterService, IndicesService indicesService, + IndexNameExpressionResolver resolver, DlsQueryParser dlsQueryParser, ThreadContext threadContext) { + + if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE) != null) { + return true; + } + + if (action.startsWith("cluster:") || action.startsWith("indices:admin/template/") + || action.startsWith("indices:admin/index_template/")) { + return true; + } + + if (action.startsWith(SearchScrollAction.NAME)) { + return true; + } + + if (action.equals(SearchTemplateAction.NAME) || action.equals(MultiSearchTemplateAction.NAME)) { + // Let it pass; DLS will be handled on a lower level + return true; + } + + if (request instanceof MultiSearchRequest) { + // Let it pass; DLS will be handled on a lower level + return true; + } + + return new DlsFilterLevelActionHandler(action, request, listener, evaluatedDlsFlsConfig, resolved, nodeClient, clusterService, indicesService, + resolver, dlsQueryParser, threadContext).handle(); + } + + private final String action; + private final ActionRequest request; + private final ActionListener listener; + private final EvaluatedDlsFlsConfig evaluatedDlsFlsConfig; + private final Resolved resolved; + private final boolean requiresIndexScoping; + private final Client nodeClient; + private final DlsQueryParser dlsQueryParser; + private final ClusterService clusterService; + private final IndicesService indicesService; + private final ThreadContext threadContext; + private final IndexNameExpressionResolver resolver; + private BoolQueryBuilder filterLevelQueryBuilder; + private DocumentAllowList documentWhitelist; + + DlsFilterLevelActionHandler(String action, ActionRequest request, ActionListener listener, EvaluatedDlsFlsConfig evaluatedDlsFlsConfig, + Resolved resolved, Client nodeClient, ClusterService clusterService, IndicesService indicesService, + IndexNameExpressionResolver resolver, DlsQueryParser dlsQueryParser, ThreadContext threadContext) { + this.action = action; + this.request = request; + this.listener = listener; + this.evaluatedDlsFlsConfig = evaluatedDlsFlsConfig; + this.resolved = resolved; + this.nodeClient = nodeClient; + this.clusterService = clusterService; + this.indicesService = indicesService; + this.dlsQueryParser = dlsQueryParser; + this.threadContext = threadContext; + this.resolver = resolver; + + this.requiresIndexScoping = resolved.isLocalAll() || resolved.getAllIndicesResolved(clusterService, resolver).size() != 1; + } + + private boolean handle() { + + try (StoredContext ctx = threadContext.newStoredContext(true)) { + + threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE, request.toString()); + + try { + if (!createQueryExtension()) { + return true; + } + + if (log.isDebugEnabled()) { + log.debug("Created filterLevelQuery for " + request + ":\n" + filterLevelQueryBuilder); + } + + } catch (Exception e) { + log.error("Unable to handle filter level DLS", e); + listener.onFailure(new OpenSearchSecurityException("Unable to handle filter level DLS", e)); + return false; + } + + if (filterLevelQueryBuilder == null) { + return true; + } + + if (request instanceof SearchRequest) { + return handle((SearchRequest) request, ctx); + } else if (request instanceof GetRequest) { + return handle((GetRequest) request, ctx); + } else if (request instanceof MultiGetRequest) { + return handle((MultiGetRequest) request, ctx); + } else if (request instanceof ClusterSearchShardsRequest) { + return handle((ClusterSearchShardsRequest) request, ctx); + } else { + log.error("Unsupported request type for filter level DLS: " + request); + listener.onFailure(new OpenSearchSecurityException( + "Unsupported request type for filter level DLS: " + action + "; " + request.getClass().getName())); + return false; + } + } + } + + private boolean handle(SearchRequest searchRequest, StoredContext ctx) { + if (documentWhitelist != null) { + documentWhitelist.applyTo(threadContext); + } + + String localClusterAlias = LOCAL_CLUSTER_ALIAS_GETTER.apply(searchRequest); + + if (localClusterAlias != null) { + try { + createQueryExtension(localClusterAlias); + } catch (Exception e) { + log.error("Unable to handle filter level DLS", e); + listener.onFailure(new OpenSearchSecurityException("Unable to handle filter level DLS", e)); + return false; + } + } + + if (searchRequest.source().query() != null) { + filterLevelQueryBuilder.must(searchRequest.source().query()); + } + + searchRequest.source().query(filterLevelQueryBuilder); + + nodeClient.search(searchRequest, new ActionListener() { + @Override + public void onResponse(SearchResponse response) { + try { + ctx.restore(); + + @SuppressWarnings("unchecked") + ActionListener searchListener = (ActionListener) listener; + + searchListener.onResponse(response); + } catch (Exception e) { + listener.onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + + return false; + } + + private boolean handle(GetRequest getRequest, StoredContext ctx) { + if (documentWhitelist != null) { + documentWhitelist.applyTo(threadContext); + } + + SearchRequest searchRequest = new SearchRequest(getRequest.indices()); + BoolQueryBuilder query = QueryBuilders.boolQuery().must(QueryBuilders.idsQuery().addIds(getRequest.id())).must(filterLevelQueryBuilder); + searchRequest.source(SearchSourceBuilder.searchSource().query(query)); + + nodeClient.search(searchRequest, new ActionListener() { + @Override + public void onResponse(SearchResponse response) { + try { + + ctx.restore(); + + long hits = response.getHits().getTotalHits().value; + + @SuppressWarnings("unchecked") + ActionListener getListener = (ActionListener) listener; + if (hits == 1) { + getListener.onResponse(new GetResponse(searchHitToGetResult(response.getHits().getAt(0)))); + } else if (hits == 0) { + getListener.onResponse(new GetResponse(new GetResult(searchRequest.indices()[0], "_doc", getRequest.id(), + SequenceNumbers.UNASSIGNED_SEQ_NO, SequenceNumbers.UNASSIGNED_PRIMARY_TERM, -1, false, null, null, null))); + } else { + log.error("Unexpected hit count " + hits + " in " + response); + listener.onFailure(new OpenSearchSecurityException("Internal error when performing DLS")); + } + + } catch (Exception e) { + listener.onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + + return false; + + } + + private boolean handle(MultiGetRequest multiGetRequest, StoredContext ctx) { + if (documentWhitelist != null) { + documentWhitelist.applyTo(threadContext); + } + + Map> idsGroupedByIndex = multiGetRequest.getItems().stream() + .collect(Collectors.groupingBy((item) -> item.index(), Collectors.mapping((item) -> item.id(), Collectors.toSet()))); + Set indices = idsGroupedByIndex.keySet(); + SearchRequest searchRequest = new SearchRequest(indices.toArray(new String[indices.size()])); + + BoolQueryBuilder query; + + if (indices.size() == 1) { + Set ids = idsGroupedByIndex.get(indices.iterator().next()); + query = QueryBuilders.boolQuery().must(QueryBuilders.idsQuery().addIds(ids.toArray(new String[ids.size()]))) + .must(filterLevelQueryBuilder); + } else { + BoolQueryBuilder mgetQuery = QueryBuilders.boolQuery().minimumShouldMatch(1); + + for (Map.Entry> entry : idsGroupedByIndex.entrySet()) { + BoolQueryBuilder indexQuery = QueryBuilders.boolQuery().must(QueryBuilders.termQuery("_index", entry.getKey())) + .must(QueryBuilders.idsQuery().addIds(entry.getValue().toArray(new String[entry.getValue().size()]))); + + mgetQuery.should(indexQuery); + } + + query = QueryBuilders.boolQuery().must(mgetQuery).must(filterLevelQueryBuilder); + } + + searchRequest.source(SearchSourceBuilder.searchSource().query(query)); + + nodeClient.search(searchRequest, new ActionListener() { + @Override + public void onResponse(SearchResponse response) { + try { + + ctx.restore(); + + List itemResponses = new ArrayList<>(response.getHits().getHits().length); + + for (SearchHit hit : response.getHits().getHits()) { + itemResponses.add(new MultiGetItemResponse(new GetResponse(searchHitToGetResult(hit)), null)); + } + + @SuppressWarnings("unchecked") + ActionListener multiGetListener = (ActionListener) listener; + multiGetListener.onResponse(new MultiGetResponse(itemResponses.toArray(new MultiGetItemResponse[itemResponses.size()]))); + } catch (Exception e) { + listener.onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + + return false; + + } + + private boolean handle(ClusterSearchShardsRequest request, StoredContext ctx) { + listener.onFailure(new OpenSearchSecurityException( + "Filter-level DLS via cross cluster search is not available for scrolling and minimize_roundtrips=true")); + return false; + } + + private GetResult searchHitToGetResult(SearchHit hit) { + + if (log.isDebugEnabled()) { + log.debug("Converting to GetResult:\n" + hit); + } + + Map fields = hit.getFields(); + Map documentFields; + Map metadataFields; + + if (fields.isEmpty()) { + documentFields = Collections.emptyMap(); + metadataFields = Collections.emptyMap(); + } else { + IndexMetadata indexMetadata = clusterService.state().getMetadata().indices().get(hit.getIndex()); + IndexService indexService = indexMetadata != null ? indicesService.indexService(indexMetadata.getIndex()) : null; + + if (indexService != null) { + documentFields = new HashMap<>(fields.size()); + metadataFields = new HashMap<>(); + MapperService mapperService = indexService.mapperService(); + + for (Map.Entry entry : fields.entrySet()) { + if (mapperService.isMetadataField(entry.getKey())) { + metadataFields.put(entry.getKey(), entry.getValue()); + } else { + documentFields.put(entry.getKey(), entry.getValue()); + } + } + + if (log.isDebugEnabled()) { + log.debug("Partitioned fields: " + metadataFields + "; " + documentFields); + } + + } else { + if (log.isWarnEnabled()) { + log.warn("Could not find IndexService for " + hit.getIndex() + "; assuming all fields as document fields." + + "This should not happen, however this should also not pose a big problem as ES mixes the fields again anyway.\n" + + "IndexMetadata: " + indexMetadata); + } + + documentFields = fields; + metadataFields = Collections.emptyMap(); + } + } + + @SuppressWarnings("deprecation") + String type = hit.getType(); + + return new GetResult(hit.getIndex(), type, hit.getId(), hit.getSeqNo(), hit.getPrimaryTerm(), hit.getVersion(), true, hit.getSourceRef(), + documentFields, metadataFields); + } + + private boolean createQueryExtension() throws IOException { + return createQueryExtension(null); + } + + private boolean createQueryExtension(String localClusterAlias) throws IOException { + Map> filterLevelQueries = evaluatedDlsFlsConfig.getDlsQueriesByIndex(); + + BoolQueryBuilder dlsQueryBuilder = QueryBuilders.boolQuery().minimumShouldMatch(1); + DocumentAllowList documentWhitelist = new DocumentAllowList(); + + int queryCount = 0; + + Set indices = resolved.getAllIndicesResolved(clusterService, resolver); + + for (String index : indices) { + String dlsEval = SecurityUtils.evalMap(filterLevelQueries, index); + + String prefixedIndex; + + if (localClusterAlias != null) { + prefixedIndex = localClusterAlias + ":" + index; + } else { + prefixedIndex = index; + } + + if (dlsEval == null) { + if (requiresIndexScoping) { + // This index has no DLS configured, thus it is unrestricted. + // To allow the index in a complex query, we need to add the query below to let the index pass. + dlsQueryBuilder.should(QueryBuilders.termQuery("_index", prefixedIndex)); + } + continue; + } + + Set unparsedDlsQueries = filterLevelQueries.get(dlsEval); + + if (unparsedDlsQueries == null || unparsedDlsQueries.isEmpty()) { + if (requiresIndexScoping) { + // This index has no DLS configured, thus it is unrestricted. + // To allow the index in a complex query, we need to add the query below to let the index pass. + dlsQueryBuilder.should(QueryBuilders.termQuery("_index", prefixedIndex)); + } + continue; + } + + for (String unparsedDlsQuery : unparsedDlsQueries) { + queryCount++; + + QueryBuilder parsedDlsQuery = dlsQueryParser.parse(unparsedDlsQuery); + + if (!requiresIndexScoping) { + dlsQueryBuilder.should(parsedDlsQuery); + } else { + // The original request referred to several indices. That's why we have to scope each query to the index it is meant for + dlsQueryBuilder.should(QueryBuilders.boolQuery().must(QueryBuilders.termQuery("_index", prefixedIndex)).must(parsedDlsQuery)); + } + + Set queryBuilders = QueryBuilderTraverser.findAll(parsedDlsQuery, + (q) -> (q instanceof TermsQueryBuilder) && ((TermsQueryBuilder) q).termsLookup() != null); + + for (QueryBuilder queryBuilder : queryBuilders) { + TermsQueryBuilder termsQueryBuilder = (TermsQueryBuilder) queryBuilder; + + documentWhitelist.add(termsQueryBuilder.termsLookup().index(), termsQueryBuilder.termsLookup().id()); + } + } + + } + + if (queryCount == 0) { + // Return false to indicate that no query manipulation is necessary + return false; + } else { + this.filterLevelQueryBuilder = dlsQueryBuilder; + this.documentWhitelist = documentWhitelist; + return true; + } + } + +} diff --git a/src/main/java/org/opensearch/security/configuration/DlsFlsRequestValve.java b/src/main/java/org/opensearch/security/configuration/DlsFlsRequestValve.java index 3eb86196ca..5034e1315c 100644 --- a/src/main/java/org/opensearch/security/configuration/DlsFlsRequestValve.java +++ b/src/main/java/org/opensearch/security/configuration/DlsFlsRequestValve.java @@ -30,35 +30,28 @@ package org.opensearch.security.configuration; -import java.util.Map; -import java.util.Set; - import org.opensearch.action.ActionListener; import org.opensearch.action.ActionRequest; import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.search.internal.SearchContext; import org.opensearch.search.query.QuerySearchResult; +import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; +import org.opensearch.security.securityconf.EvaluatedDlsFlsConfig; import org.opensearch.threadpool.ThreadPool; public interface DlsFlsRequestValve { - /** - * SSLConfigConstants.java - * @param action - * @param request - * @param listener - * @return false to stop - */ - boolean invoke(String action, ActionRequest request, ActionListener listener, Map> allowedFlsFields, final Map> maskedFields, Map> queries); + boolean invoke(String action, ActionRequest request, ActionListener listener, EvaluatedDlsFlsConfig evaluatedDlsFlsConfig, Resolved resolved); void handleSearchContext(SearchContext context, ThreadPool threadPool, NamedXContentRegistry namedXContentRegistry); void onQueryPhase(QuerySearchResult queryResult); - + public static class NoopDlsFlsRequestValve implements DlsFlsRequestValve { - @Override - public boolean invoke(String action, ActionRequest request, ActionListener listener, Map> allowedFlsFields, final Map> maskedFields, Map> queries) { + @Override + public boolean invoke(String action, ActionRequest request, ActionListener listener, EvaluatedDlsFlsConfig evaluatedDlsFlsConfig, + Resolved resolved) { return true; } diff --git a/src/main/java/org/opensearch/security/configuration/DlsFlsValveImpl.java b/src/main/java/org/opensearch/security/configuration/DlsFlsValveImpl.java index f1fe1f36ed..4e04e50a19 100644 --- a/src/main/java/org/opensearch/security/configuration/DlsFlsValveImpl.java +++ b/src/main/java/org/opensearch/security/configuration/DlsFlsValveImpl.java @@ -15,56 +15,88 @@ package org.opensearch.security.configuration; -import org.opensearch.rest.RestStatus; -import org.opensearch.security.support.SecurityUtils; -import org.slf4j.LoggerFactory; -import org.slf4j.Logger; -import org.apache.lucene.util.BytesRef; - +import java.io.Serializable; import java.lang.reflect.Field; import java.security.AccessController; import java.security.PrivilegedAction; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; +import java.util.*; import java.util.function.Consumer; import java.util.stream.StreamSupport; +import com.google.common.collect.ImmutableList; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.util.BytesRef; import org.opensearch.OpenSearchSecurityException; import org.opensearch.SpecialPermission; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionRequest; import org.opensearch.action.DocWriteRequest; import org.opensearch.action.RealtimeRequest; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.opensearch.action.admin.indices.shrink.ResizeRequest; import org.opensearch.action.bulk.BulkItemRequest; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.BulkShardRequest; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.update.UpdateRequest; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Strings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.index.query.ParsedQuery; import org.opensearch.search.DocValueFormat; +import org.opensearch.search.aggregations.AggregationBuilder; import org.opensearch.search.aggregations.BucketOrder; import org.opensearch.search.aggregations.InternalAggregation; import org.opensearch.search.aggregations.InternalAggregations; -import org.opensearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.opensearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.opensearch.search.aggregations.bucket.sampler.DiversifiedAggregationBuilder; import org.opensearch.search.aggregations.bucket.terms.InternalTerms; +import org.opensearch.search.aggregations.bucket.terms.SignificantTermsAggregationBuilder; import org.opensearch.search.aggregations.bucket.terms.StringTerms; +import org.opensearch.search.aggregations.bucket.terms.StringTerms.Bucket; +import org.opensearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.opensearch.search.builder.SearchSourceBuilder; import org.opensearch.search.internal.SearchContext; import org.opensearch.search.query.QuerySearchResult; -import org.opensearch.threadpool.ThreadPool; - +import org.opensearch.security.OpenSearchSecurityPlugin; +import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; +import org.opensearch.security.securityconf.EvaluatedDlsFlsConfig; +import org.opensearch.security.support.Base64Helper; import org.opensearch.security.support.ConfigConstants; import org.opensearch.security.support.HeaderHelper; - -import com.google.common.collect.ImmutableList; +import org.opensearch.security.support.SecurityUtils; +import org.opensearch.threadpool.ThreadPool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DlsFlsValveImpl implements DlsFlsRequestValve { - private static final Logger log = LoggerFactory.getLogger(DlsFlsValveImpl.class); + + private static final String MAP_EXECUTION_HINT = "map"; + private static final Logger log = LoggerFactory.getLogger(DlsFlsValveImpl.class); + + private final Client nodeClient; + private final ClusterService clusterService; + private final ThreadContext threadContext; + private final Mode mode; + private final DlsQueryParser dlsQueryParser; + private final IndexNameExpressionResolver resolver; + + public DlsFlsValveImpl(Settings settings, Client nodeClient, ClusterService clusterService, IndexNameExpressionResolver resolver, + NamedXContentRegistry namedXContentRegistry, ThreadContext threadContext) { + super(); + this.nodeClient = nodeClient; + this.clusterService = clusterService; + this.resolver = resolver; + this.threadContext = threadContext; + this.mode = Mode.get(settings); + this.dlsQueryParser = new DlsQueryParser(namedXContentRegistry); + } /** * @@ -72,78 +104,188 @@ public class DlsFlsValveImpl implements DlsFlsRequestValve { * @param listener * @return false on error */ - public boolean invoke(final String action, final ActionRequest request, final ActionListener listener, - final Map> allowedFlsFields, - final Map> maskedFields, - final Map> queries) { + public boolean invoke(String action, ActionRequest request, final ActionListener listener, EvaluatedDlsFlsConfig evaluatedDlsFlsConfig, + final Resolved resolved) { - final boolean fls = allowedFlsFields != null && !allowedFlsFields.isEmpty(); - final boolean masked = maskedFields != null && !maskedFields.isEmpty(); - final boolean dls = queries != null && !queries.isEmpty(); + if (log.isDebugEnabled()) { + log.debug("DlsFlsValveImpl.invoke()\nrequest: " + request + "\nevaluatedDlsFlsConfig: " + evaluatedDlsFlsConfig + "\nresolved: " + + resolved + "\nmode: " + mode); + } - if(fls || masked || dls) { + if (evaluatedDlsFlsConfig == null || evaluatedDlsFlsConfig.isEmpty()) { + return true; + } - if(request instanceof RealtimeRequest) { - ((RealtimeRequest) request).realtime(Boolean.FALSE); + if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE) != null) { + if (log.isDebugEnabled()) { + log.debug("DLS is already done for: " + threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE)); } - if(request instanceof SearchRequest) { - ((SearchRequest)request).requestCache(Boolean.FALSE); - } + return true; + } + + EvaluatedDlsFlsConfig filteredDlsFlsConfig = evaluatedDlsFlsConfig.filter(resolved); + + boolean doFilterLevelDls; - if(request instanceof UpdateRequest) { - listener.onFailure(new OpenSearchSecurityException("Update is not supported when FLS or DLS or Fieldmasking is activated")); - return false; + if (mode == Mode.FILTER_LEVEL) { + doFilterLevelDls = true; + } else if (mode == Mode.LUCENE_LEVEL) { + doFilterLevelDls = false; + } else { // mode == Mode.ADAPTIVE + Mode modeByHeader = getDlsModeHeader(); + + if (modeByHeader == Mode.FILTER_LEVEL) { + doFilterLevelDls = true; + log.debug("Doing filter-level DLS due to header"); + } else { + doFilterLevelDls = dlsQueryParser.containsTermLookupQuery(filteredDlsFlsConfig.getAllQueries()); + + if (doFilterLevelDls) { + setDlsModeHeader(Mode.FILTER_LEVEL); + log.debug("Doing filter-level DLS because the query contains a TLQ"); + } else { + log.debug("Doing lucene-level DLS because the query does not contain a TLQ"); + } } + } - if(request instanceof BulkRequest) { - for(DocWriteRequest inner:((BulkRequest) request).requests()) { - if(inner instanceof UpdateRequest) { - listener.onFailure(new OpenSearchSecurityException("Update is not supported when FLS or DLS or Fieldmasking is activated")); - return false; + if (!doFilterLevelDls) { + setDlsHeaders(evaluatedDlsFlsConfig, request); + } + + setFlsHeaders(evaluatedDlsFlsConfig, request); + + if (filteredDlsFlsConfig.isEmpty()) { + return true; + } + + if (request instanceof RealtimeRequest) { + ((RealtimeRequest) request).realtime(Boolean.FALSE); + } + + if (request instanceof SearchRequest) { + + SearchRequest searchRequest = ((SearchRequest) request); + + //When we encounter a terms or sampler aggregation with masked fields activated we forcibly + //need to switch off global ordinals because field masking can break ordering + //https://www.elastic.co/guide/en/elasticsearch/reference/master/eager-global-ordinals.html#_avoiding_global_ordinal_loading + if (evaluatedDlsFlsConfig.hasFieldMasking()) { + + if (searchRequest.source() != null && searchRequest.source().aggregations() != null) { + for (AggregationBuilder aggregationBuilder : searchRequest.source().aggregations().getAggregatorFactories()) { + if (aggregationBuilder instanceof TermsAggregationBuilder) { + ((TermsAggregationBuilder) aggregationBuilder).executionHint(MAP_EXECUTION_HINT); + } + + if (aggregationBuilder instanceof SignificantTermsAggregationBuilder) { + ((SignificantTermsAggregationBuilder) aggregationBuilder).executionHint(MAP_EXECUTION_HINT); + } + + if (aggregationBuilder instanceof DiversifiedAggregationBuilder) { + ((DiversifiedAggregationBuilder) aggregationBuilder).executionHint(MAP_EXECUTION_HINT); + } } } } - if(request instanceof BulkShardRequest) { - for(BulkItemRequest inner:((BulkShardRequest) request).items()) { - if(inner.request() instanceof UpdateRequest) { - listener.onFailure(new OpenSearchSecurityException("Update is not supported when FLS or DLS or Fieldmasking is activated")); - return false; + if (!evaluatedDlsFlsConfig.hasFls() && !evaluatedDlsFlsConfig.hasDls() + && searchRequest.source().aggregations() != null) { + + boolean cacheable = true; + + for (AggregationBuilder af : searchRequest.source().aggregations().getAggregatorFactories()) { + + if (!af.getType().equals("cardinality") && !af.getType().equals("count")) { + cacheable = false; + continue; } + + StringBuilder sb = new StringBuilder(); + + if (searchRequest.source() != null) { + sb.append(Strings.toString(searchRequest.source()) + System.lineSeparator()); + } + + sb.append(Strings.toString(af) + System.lineSeparator()); + + LoggerFactory.getLogger("debuglogger").error(sb.toString()); + + } + + if (!cacheable) { + searchRequest.requestCache(Boolean.FALSE); + } else { + LoggerFactory.getLogger("debuglogger").error("Shard requestcache enabled for " + + (searchRequest.source() == null ? "" : Strings.toString(searchRequest.source()))); } + + } else { + searchRequest.requestCache(Boolean.FALSE); } + } - if(request instanceof ResizeRequest) { - listener.onFailure(new OpenSearchSecurityException("Resize is not supported when FLS or DLS or Fieldmasking is activated")); - return false; + if (request instanceof UpdateRequest) { + listener.onFailure(new OpenSearchSecurityException("Update is not supported when FLS or DLS or Fieldmasking is activated")); + return false; + } + + if (request instanceof BulkRequest) { + for (DocWriteRequest inner : ((BulkRequest) request).requests()) { + if (inner instanceof UpdateRequest) { + listener.onFailure(new OpenSearchSecurityException("Update is not supported when FLS or DLS or Fieldmasking is activated")); + return false; + } } + } - if(action.contains("plugins/replication")) { - listener.onFailure(new OpenSearchSecurityException("Cross Cluster Replication is not supported when FLS or DLS or Fieldmasking is activated", RestStatus.FORBIDDEN)); - return false; + if (request instanceof BulkShardRequest) { + for (BulkItemRequest inner : ((BulkShardRequest) request).items()) { + if (inner.request() instanceof UpdateRequest) { + listener.onFailure(new OpenSearchSecurityException("Update is not supported when FLS or DLS or Fieldmasking is activated")); + return false; + } } } - if(dls) { - if(request instanceof SearchRequest) { - final SearchSourceBuilder source = ((SearchRequest)request).source(); - if(source != null) { + if (request instanceof ResizeRequest) { + listener.onFailure(new OpenSearchSecurityException("Resize is not supported when FLS or DLS or Fieldmasking is activated")); + return false; + } + + if(action.contains("plugins/replication")) { + listener.onFailure(new OpenSearchSecurityException("Cross Cluster Replication is not supported when FLS or DLS or Fieldmasking is activated")); + return false; + } + + if (evaluatedDlsFlsConfig.hasDls()) { + if (request instanceof SearchRequest) { + + final SearchSourceBuilder source = ((SearchRequest) request).source(); + if (source != null) { - if(source.profile()) { + if (source.profile()) { listener.onFailure(new OpenSearchSecurityException("Profiling is not supported when DLS is activated")); return false; } + } } } - return true; + if (doFilterLevelDls && filteredDlsFlsConfig.hasDls()) { + return DlsFilterLevelActionHandler.handle(action, request, listener, evaluatedDlsFlsConfig, resolved, nodeClient, clusterService, + OpenSearchSecurityPlugin.GuiceHolder.getIndicesService(), resolver, dlsQueryParser, threadContext); + } else { + return true; + } } @Override public void handleSearchContext(SearchContext context, ThreadPool threadPool, NamedXContentRegistry namedXContentRegistry) { try { + @SuppressWarnings("unchecked") final Map> queries = (Map>) HeaderHelper.deserializeSafeFromHeader(threadPool.getThreadContext(), ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER); @@ -151,23 +293,31 @@ public void handleSearchContext(SearchContext context, ThreadPool threadPool, Na if (dlsEval != null) { - if(context.suggest() != null) { + if (context.suggest() != null) { return; } assert context.parsedQuery() != null; final Set unparsedDlsQueries = queries.get(dlsEval); + if (unparsedDlsQueries != null && !unparsedDlsQueries.isEmpty()) { - final ParsedQuery dlsQuery = DlsQueryParser.parse(unparsedDlsQueries, context.parsedQuery(), context.getQueryShardContext(), namedXContentRegistry); - context.parsedQuery(dlsQuery); - context.preProcess(true); + BooleanQuery.Builder queryBuilder = dlsQueryParser.parse(unparsedDlsQueries, context.getQueryShardContext(), + (q) -> new ConstantScoreQuery(q)); + + queryBuilder.add(context.parsedQuery().query(), Occur.MUST); + + ParsedQuery dlsQuery = new ParsedQuery(queryBuilder.build()); + + if (dlsQuery != null) { + context.parsedQuery(dlsQuery); + context.preProcess(true); + } } } } catch (Exception e) { throw new RuntimeException("Error evaluating dls for a search query: " + e, e); } - } @Override @@ -178,8 +328,8 @@ public void onQueryPhase(QuerySearchResult queryResult) { queryResult.aggregations( InternalAggregations.from( StreamSupport.stream(aggregations.spliterator(), false) - .map(aggregation -> aggregateBuckets((InternalAggregation)aggregation)) - .collect(ImmutableList.toImmutableList()) + .map(aggregation -> aggregateBuckets((InternalAggregation)aggregation)) + .collect(ImmutableList.toImmutableList()) ) ); } @@ -187,7 +337,7 @@ public void onQueryPhase(QuerySearchResult queryResult) { private static InternalAggregation aggregateBuckets(InternalAggregation aggregation) { if (aggregation instanceof StringTerms) { StringTerms stringTerms = (StringTerms) aggregation; - List buckets = stringTerms.getBuckets(); + List buckets = stringTerms.getBuckets(); if (buckets.size() > 1) { buckets = mergeBuckets(buckets, StringTermsGetter.getReduceOrder(stringTerms).comparator()); aggregation = stringTerms.create(buckets); @@ -196,7 +346,7 @@ private static InternalAggregation aggregateBuckets(InternalAggregation aggregat return aggregation; } - private static List mergeBuckets(List buckets, Comparator comparator) { + private static List mergeBuckets(List buckets, Comparator comparator) { if (log.isDebugEnabled()) { log.debug("Merging buckets: {}", buckets.stream().map(b -> b.getKeyAsString()).collect(ImmutableList.toImmutableList())); } @@ -211,8 +361,112 @@ private static List mergeBuckets(List bu return buckets; } - private static class BucketMerger implements Consumer { - private Comparator comparator; + private void setDlsHeaders(EvaluatedDlsFlsConfig dlsFls, ActionRequest request) { + if (!dlsFls.getDlsQueriesByIndex().isEmpty()) { + Map> dlsQueries = dlsFls.getDlsQueriesByIndex(); + + if (request instanceof ClusterSearchShardsRequest && HeaderHelper.isTrustedClusterRequest(threadContext)) { + threadContext.addResponseHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER, Base64Helper.serializeObject((Serializable) dlsQueries)); + if (log.isDebugEnabled()) { + log.debug("added response header for DLS info: {}", dlsQueries); + } + } else { + if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER) != null) { + Object deserializedDlsQueries = Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER)); + if (!dlsQueries.equals(deserializedDlsQueries)) { + throw new OpenSearchSecurityException(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER + " does not match (SG 900D)"); + } + } else { + threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER, Base64Helper.serializeObject((Serializable) dlsQueries)); + if (log.isDebugEnabled()) { + log.debug("attach DLS info: {}", dlsQueries); + } + } + } + } + } + + private void setDlsModeHeader(Mode mode) { + String modeString = mode.name(); + + if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER) != null) { + if (!modeString.equals(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER))) { + log.warn("Cannot update DLS mode to " + mode + "; current: " + threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER)); + } + } else { + threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER, modeString); + } + } + + private Mode getDlsModeHeader() { + String modeString = threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER); + + if (modeString != null) { + return Mode.valueOf(modeString); + } else { + return null; + } + } + + private void setFlsHeaders(EvaluatedDlsFlsConfig dlsFls, ActionRequest request) { + if (!dlsFls.getFieldMaskingByIndex().isEmpty()) { + Map> maskedFieldsMap = dlsFls.getFieldMaskingByIndex(); + + if (request instanceof ClusterSearchShardsRequest && HeaderHelper.isTrustedClusterRequest(threadContext)) { + threadContext.addResponseHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER, Base64Helper.serializeObject((Serializable) maskedFieldsMap)); + if (log.isDebugEnabled()) { + log.debug("added response header for masked fields info: {}", maskedFieldsMap); + } + } else { + + if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER) != null) { + if (!maskedFieldsMap.equals(Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER)))) { + throw new OpenSearchSecurityException(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER + " does not match (SG 901D)"); + } else { + if (log.isDebugEnabled()) { + log.debug(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER + " already set"); + } + } + } else { + threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER, Base64Helper.serializeObject((Serializable) maskedFieldsMap)); + if (log.isDebugEnabled()) { + log.debug("attach masked fields info: {}", maskedFieldsMap); + } + } + } + } + + if (!dlsFls.getFlsByIndex().isEmpty()) { + Map> flsFields = dlsFls.getFlsByIndex(); + + if (request instanceof ClusterSearchShardsRequest && HeaderHelper.isTrustedClusterRequest(threadContext)) { + threadContext.addResponseHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER, Base64Helper.serializeObject((Serializable) flsFields)); + if (log.isDebugEnabled()) { + log.debug("added response header for FLS info: {}", flsFields); + } + } else { + if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER) != null) { + if (!flsFields.equals(Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER)))) { + throw new OpenSearchSecurityException(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER + " does not match (SG 901D) " + flsFields + + "---" + Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER))); + } else { + if (log.isDebugEnabled()) { + log.debug(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER + " already set"); + } + } + } else { + threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER, Base64Helper.serializeObject((Serializable) flsFields)); + if (log.isDebugEnabled()) { + log.debug("attach FLS info: {}", flsFields); + } + } + } + + } + } + + private static class BucketMerger implements Consumer { + private Comparator comparator; private StringTerms.Bucket bucket = null; private int mergeCount; private long mergedDocCount; @@ -220,7 +474,7 @@ private static class BucketMerger implements Consumer { private boolean showDocCountError = true; private final ImmutableList.Builder builder; - BucketMerger(Comparator comparator, int size) { + BucketMerger(Comparator comparator, int size) { this.comparator = Objects.requireNonNull(comparator); builder = ImmutableList.builderWithExpectedSize(size); } @@ -230,8 +484,8 @@ private void finalizeBucket() { builder.add(this.bucket); } else { builder.add(new StringTerms.Bucket(StringTermsGetter.getTerm(bucket), mergedDocCount, - (InternalAggregations) bucket.getAggregations(), showDocCountError, mergedDocCountError, - StringTermsGetter.getDocValueFormat(bucket))); + (InternalAggregations) bucket.getAggregations(), showDocCountError, mergedDocCountError, + StringTermsGetter.getDocValueFormat(bucket))); } } @@ -320,4 +574,23 @@ public static DocValueFormat getDocValueFormat(StringTerms.Bucket bucket) { return getFieldValue(FORMAT, bucket); } } + + public static enum Mode { + ADAPTIVE, LUCENE_LEVEL, FILTER_LEVEL; + + static Mode get(Settings settings) { + String modeString = settings.get(ConfigConstants.SECURITY_DLS_MODE); + + if ("adaptive".equalsIgnoreCase(modeString)) { + return Mode.ADAPTIVE; + } else if ("lucene_level".equalsIgnoreCase(modeString)) { + return Mode.LUCENE_LEVEL; + } else if ("filter_level".equalsIgnoreCase(modeString)) { + return Mode.FILTER_LEVEL; + } else { + return Mode.ADAPTIVE; + } + } + } } + diff --git a/src/main/java/org/opensearch/security/configuration/DlsQueryParser.java b/src/main/java/org/opensearch/security/configuration/DlsQueryParser.java index 32ef21251d..7881038a83 100644 --- a/src/main/java/org/opensearch/security/configuration/DlsQueryParser.java +++ b/src/main/java/org/opensearch/security/configuration/DlsQueryParser.java @@ -15,21 +15,23 @@ package org.opensearch.security.configuration; -import java.io.IOException; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import org.slf4j.LoggerFactory; +import org.slf4j.Logger; import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.search.join.ToChildBlockJoinQuery; +import org.opensearch.common.xcontent.DeprecationHandler; import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.common.xcontent.XContentParser; import org.opensearch.common.xcontent.json.JsonXContent; @@ -37,129 +39,127 @@ import org.opensearch.index.query.ParsedQuery; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.query.TermsQueryBuilder; +import org.opensearch.security.queries.QueryBuilderTraverser; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; -import static org.opensearch.common.xcontent.DeprecationHandler.THROW_UNSUPPORTED_OPERATION; +public final class DlsQueryParser { -final class DlsQueryParser { - + private static final Logger log = LoggerFactory.getLogger(DlsQueryParser.class); private static final Query NON_NESTED_QUERY; static { //Match all documents but not the nested ones //Nested document types start with __ //https://discuss.elastic.co/t/whats-nested-documents-layout-inside-the-lucene/59944/9 - NON_NESTED_QUERY = new BooleanQuery.Builder() - .add(new MatchAllDocsQuery(), Occur.FILTER) - .add(new PrefixQuery(new Term("_type", "__")), Occur.MUST_NOT) - .build(); + NON_NESTED_QUERY = new BooleanQuery.Builder().add(new MatchAllDocsQuery(), Occur.FILTER) + .add(new PrefixQuery(new Term("_type", "__")), Occur.MUST_NOT).build(); } - - private static Cache queries = CacheBuilder.newBuilder().maximumSize(10000).expireAfterWrite(4, TimeUnit.HOURS) + private static Cache parsedQueryCache = CacheBuilder.newBuilder().maximumSize(10000).expireAfterWrite(4, TimeUnit.HOURS) + .build(); + private static Cache queryContainsTlqCache = CacheBuilder.newBuilder().maximumSize(10000).expireAfterWrite(4, TimeUnit.HOURS) .build(); - private DlsQueryParser() { + private final NamedXContentRegistry namedXContentRegistry; + + public DlsQueryParser(NamedXContentRegistry namedXContentRegistry) { + this.namedXContentRegistry = namedXContentRegistry; + } + public BooleanQuery.Builder parse(Set unparsedDlsQueries, QueryShardContext queryShardContext) { + return parse(unparsedDlsQueries, queryShardContext, null); } - static Query parse(final Set unparsedDlsQueries, final QueryShardContext queryShardContext, - final NamedXContentRegistry namedXContentRegistry) throws IOException { + public BooleanQuery.Builder parse(Set unparsedDlsQueries, QueryShardContext queryShardContext, + Function queryMapFunction) { if (unparsedDlsQueries == null || unparsedDlsQueries.isEmpty()) { return null; } - final boolean hasNestedMapping = queryShardContext.getMapperService().hasNested(); + boolean hasNestedMapping = queryShardContext.getMapperService().hasNested(); BooleanQuery.Builder dlsQueryBuilder = new BooleanQuery.Builder(); dlsQueryBuilder.setMinimumNumberShouldMatch(1); - for (final String unparsedDlsQuery : unparsedDlsQueries) { - try { + for (String unparsedDlsQuery : unparsedDlsQueries) { + ParsedQuery parsedQuery = queryShardContext.toQuery(parse(unparsedDlsQuery)); + Query dlsQuery = parsedQuery.query(); - final QueryBuilder qb = queries.get(unparsedDlsQuery, new Callable() { - - @Override - public QueryBuilder call() throws Exception { - final XContentParser parser = JsonXContent.jsonXContent.createParser(namedXContentRegistry, THROW_UNSUPPORTED_OPERATION, unparsedDlsQuery); - final QueryBuilder qb = AbstractQueryBuilder.parseInnerQueryBuilder(parser); - return qb; - } - - }); - final ParsedQuery parsedQuery = queryShardContext.toQuery(qb); - final Query dlsQuery = parsedQuery.query(); - dlsQueryBuilder.add(dlsQuery, Occur.SHOULD); + if (queryMapFunction != null) { + dlsQuery = queryMapFunction.apply(dlsQuery); + } - if (hasNestedMapping) { - handleNested(queryShardContext, dlsQueryBuilder, dlsQuery); - } + dlsQueryBuilder.add(dlsQuery, Occur.SHOULD); - } catch (ExecutionException e) { - throw new IOException(e); + if (hasNestedMapping) { + handleNested(queryShardContext, dlsQueryBuilder, dlsQuery); } } - // no need for scoring here, so its possible to wrap this in a - // ConstantScoreQuery - return new ConstantScoreQuery(dlsQueryBuilder.build()); - + return dlsQueryBuilder; + } + + private static void handleNested(final QueryShardContext queryShardContext, final BooleanQuery.Builder dlsQueryBuilder, + final Query parentQuery) { + final BitSetProducer parentDocumentsFilter = queryShardContext.bitsetFilter(NON_NESTED_QUERY); + dlsQueryBuilder.add(new ToChildBlockJoinQuery(parentQuery, parentDocumentsFilter), Occur.SHOULD); } - static ParsedQuery parse(final Set unparsedDlsQueries, ParsedQuery originalQuery, final QueryShardContext queryShardContext, - final NamedXContentRegistry namedXContentRegistry) throws IOException { - if (unparsedDlsQueries == null || unparsedDlsQueries.isEmpty()) { - return null; - } - - final boolean hasNestedMapping = queryShardContext.getMapperService().hasNested(); - - BooleanQuery.Builder dlsQueryBuilder = new BooleanQuery.Builder(); - dlsQueryBuilder.setMinimumNumberShouldMatch(1); - - for (final String unparsedDlsQuery : unparsedDlsQueries) { - try { - - final QueryBuilder qb = queries.get(unparsedDlsQuery, new Callable() { + public QueryBuilder parse(String unparsedDlsQuery) { + try { + final QueryBuilder qb = parsedQueryCache.get(unparsedDlsQuery, new Callable() { - @Override - public QueryBuilder call() throws Exception { - final XContentParser parser = JsonXContent.jsonXContent.createParser(namedXContentRegistry, THROW_UNSUPPORTED_OPERATION, unparsedDlsQuery); - final QueryBuilder qb = AbstractQueryBuilder.parseInnerQueryBuilder(parser); - return qb; - } + @Override + public QueryBuilder call() throws Exception { + final XContentParser parser = JsonXContent.jsonXContent.createParser(namedXContentRegistry, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, unparsedDlsQuery); + return AbstractQueryBuilder.parseInnerQueryBuilder(parser); + } - }); - final ParsedQuery parsedQuery = queryShardContext.toQuery(qb); + }); - // no need for scoring here, so its possible to wrap this in a - // ConstantScoreQuery - final Query dlsQuery = new ConstantScoreQuery(parsedQuery.query()); - dlsQueryBuilder.add(dlsQuery, Occur.SHOULD); + return qb; + } catch (ExecutionException e) { + throw new RuntimeException("Error while parsing " + unparsedDlsQuery, e.getCause()); + } + } - if (hasNestedMapping) { - handleNested(queryShardContext, dlsQueryBuilder, dlsQuery); + boolean containsTermLookupQuery(Set unparsedQueries) { + for (String query : unparsedQueries) { + if (containsTermLookupQuery(query)) { + if (log.isDebugEnabled()) { + log.debug("containsTermLookupQuery() returns true due to " + query + "\nqueries: " + unparsedQueries); } - - } catch (ExecutionException e) { - throw new IOException(e); + + return true; } } - dlsQueryBuilder.add(originalQuery.query(), Occur.MUST); - return new ParsedQuery(dlsQueryBuilder.build()); + if (log.isDebugEnabled()) { + log.debug("containsTermLookupQuery() returns false\nqueries: " + unparsedQueries); + } + + return false; } - private static void handleNested(final QueryShardContext queryShardContext, - final BooleanQuery.Builder dlsQueryBuilder, - final Query parentQuery) { - final BitSetProducer parentDocumentsFilter = queryShardContext.bitsetFilter(NON_NESTED_QUERY); - dlsQueryBuilder.add(new ToChildBlockJoinQuery(parentQuery, parentDocumentsFilter), Occur.SHOULD); - } + boolean containsTermLookupQuery(String query) { + try { + return queryContainsTlqCache.get(query, () -> { + QueryBuilder queryBuilder = parse(query); + return QueryBuilderTraverser.exists(queryBuilder, + (q) -> (q instanceof TermsQueryBuilder) && ((TermsQueryBuilder) q).termsLookup() != null); + }); + } catch (ExecutionException e) { + throw new RuntimeException("Error handling parsing " + query, e.getCause()); + } + } + } + diff --git a/src/main/java/org/opensearch/security/configuration/SecurityFlsDlsIndexSearcherWrapper.java b/src/main/java/org/opensearch/security/configuration/SecurityFlsDlsIndexSearcherWrapper.java index c92c552d60..0bf91c2984 100644 --- a/src/main/java/org/opensearch/security/configuration/SecurityFlsDlsIndexSearcherWrapper.java +++ b/src/main/java/org/opensearch/security/configuration/SecurityFlsDlsIndexSearcherWrapper.java @@ -24,11 +24,13 @@ import org.opensearch.security.privileges.PrivilegesEvaluator; import org.opensearch.security.support.SecurityUtils; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.Query; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexService; import org.opensearch.index.mapper.IgnoredFieldMapper; +import org.opensearch.index.query.QueryShardContext; import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardUtils; @@ -51,6 +53,7 @@ public class SecurityFlsDlsIndexSearcherWrapper extends SecurityIndexSearcherWra private final IndexService indexService; private final AuditLog auditlog; private final LongSupplier nowInMillis; + private final DlsQueryParser dlsQueryParser; private final Salt salt; public SecurityFlsDlsIndexSearcherWrapper(final IndexService indexService, final Settings settings, @@ -61,6 +64,7 @@ public SecurityFlsDlsIndexSearcherWrapper(final IndexService indexService, final this.clusterService = clusterService; this.indexService = indexService; this.auditlog = auditlog; + this.dlsQueryParser = new DlsQueryParser(indexService.xContentRegistry()); final boolean allowNowinDlsQueries = settings.getAsBoolean(ConfigConstants.SECURITY_UNSUPPORTED_ALLOW_NOW_IN_DLS, false); if (allowNowinDlsQueries) { nowInMillis = () -> System.currentTimeMillis(); @@ -98,14 +102,14 @@ protected DirectoryReader dlsFlsWrap(final DirectoryReader reader, boolean isAdm flsFields = Sets.union(metaFields, allowedFlsFields.get(flsEval)); } - - if (dlsEval != null) { - final Set unparsedDlsQueries = queries.get(dlsEval); - if(unparsedDlsQueries != null && !unparsedDlsQueries.isEmpty()) { - //disable reader optimizations - dlsQuery = DlsQueryParser.parse(unparsedDlsQueries, this.indexService.newQueryShardContext(shardId.getId(), null, nowInMillis, null) - , this.indexService.xContentRegistry()); + Set unparsedDlsQueries = queries.get(dlsEval); + + if (unparsedDlsQueries != null && !unparsedDlsQueries.isEmpty()) { + QueryShardContext queryShardContext = this.indexService.newQueryShardContext(shardId.getId(), null, nowInMillis, null); + // no need for scoring here, so its possible to wrap this in a + // ConstantScoreQuery + dlsQuery = new ConstantScoreQuery(dlsQueryParser.parse(unparsedDlsQueries, queryShardContext).build()); } } diff --git a/src/main/java/org/opensearch/security/filter/SecurityFilter.java b/src/main/java/org/opensearch/security/filter/SecurityFilter.java index e2fd83bfc5..09c683b1cb 100644 --- a/src/main/java/org/opensearch/security/filter/SecurityFilter.java +++ b/src/main/java/org/opensearch/security/filter/SecurityFilter.java @@ -78,6 +78,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.concurrent.ThreadContext.StoredContext; +import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.index.reindex.DeleteByQueryRequest; import org.opensearch.index.reindex.UpdateByQueryRequest; import org.opensearch.rest.RestStatus; @@ -117,10 +118,12 @@ public class SecurityFilter implements ActionFilter { private final RolesInjector rolesInjector; private final Client client; private final BackendRegistry backendRegistry; + private final NamedXContentRegistry namedXContentRegistry; public SecurityFilter(final Client client, final Settings settings, final PrivilegesEvaluator evalp, final AdminDNs adminDns, DlsFlsRequestValve dlsFlsValve, AuditLog auditLog, ThreadPool threadPool, ClusterService cs, - final CompatConfig compatConfig, final IndexResolverReplacer indexResolverReplacer, BackendRegistry backendRegistry) { + final CompatConfig compatConfig, final IndexResolverReplacer indexResolverReplacer, BackendRegistry backendRegistry, + NamedXContentRegistry namedXContentRegistry) { this.client = client; this.evalp = evalp; this.adminDns = adminDns; @@ -133,6 +136,7 @@ public SecurityFilter(final Client client, final Settings settings, final Privil this.immutableIndicesMatcher = WildcardMatcher.from(settings.getAsList(ConfigConstants.SECURITY_COMPLIANCE_IMMUTABLE_INDICES, Collections.emptyList())); this.rolesInjector = new RolesInjector(auditLog); this.backendRegistry = backendRegistry; + this.namedXContentRegistry = namedXContentRegistry; log.info("{} indices are made immutable.", immutableIndicesMatcher); } @@ -311,7 +315,7 @@ private void ap if (pres.isAllowed()) { auditLog.logGrantedPrivileges(action, request, task); auditLog.logIndexEvent(action, request, task); - if(!dlsFlsValve.invoke(action, request, listener, pres.getAllowedFlsFields(), pres.getMaskedFields(), pres.getQueries())) { + if (!dlsFlsValve.invoke(action, request, listener, pres.getEvaluatedDlsFlsConfig(), pres.getResolved())) { return; } final CreateIndexRequestBuilder createIndexRequestBuilder = pres.getCreateIndexRequestBuilder(); diff --git a/src/main/java/org/opensearch/security/filter/SecurityRestFilter.java b/src/main/java/org/opensearch/security/filter/SecurityRestFilter.java index be3bf44129..2d96e22bcd 100644 --- a/src/main/java/org/opensearch/security/filter/SecurityRestFilter.java +++ b/src/main/java/org/opensearch/security/filter/SecurityRestFilter.java @@ -59,7 +59,7 @@ import org.opensearch.security.support.HTTPHelper; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.security.ssl.util.SSLRequestHelper.SSLInfo;; +import org.opensearch.security.ssl.util.SSLRequestHelper.SSLInfo; import org.opensearch.security.auth.BackendRegistry; import org.opensearch.security.user.User; import org.greenrobot.eventbus.Subscribe; diff --git a/src/main/java/org/opensearch/security/privileges/DlsFlsEvaluator.java b/src/main/java/org/opensearch/security/privileges/DlsFlsEvaluator.java deleted file mode 100644 index 6347a572dc..0000000000 --- a/src/main/java/org/opensearch/security/privileges/DlsFlsEvaluator.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Copyright 2015-2018 _floragunn_ GmbH - * Licensed 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. - */ - -/* - * Portions Copyright OpenSearch Contributors - * - * Licensed under the Apache License, Version 2.0 (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * or in the "license" file accompanying this file. This file 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.opensearch.security.privileges; - -import java.io.Serializable; -import java.util.Map; -import java.util.Set; - -import org.slf4j.LoggerFactory; -import org.slf4j.Logger; -import org.opensearch.OpenSearchSecurityException; -import org.opensearch.action.ActionRequest; -import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsRequest; -import org.opensearch.cluster.metadata.IndexNameExpressionResolver; -import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.collect.Tuple; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.security.securityconf.SecurityRoles; -import org.opensearch.threadpool.ThreadPool; - -import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; -import org.opensearch.security.support.Base64Helper; -import org.opensearch.security.support.ConfigConstants; -import org.opensearch.security.support.WildcardMatcher; -import org.opensearch.security.support.HeaderHelper; -import org.opensearch.security.user.User; - -import com.google.common.collect.ImmutableMap; - -public class DlsFlsEvaluator { - - protected final Logger log = LoggerFactory.getLogger(this.getClass()); - - private final ThreadPool threadPool; - - public DlsFlsEvaluator(Settings settings, ThreadPool threadPool) { - this.threadPool = threadPool; - } - - public PrivilegesEvaluatorResponse evaluate(final ActionRequest request, final ClusterService clusterService, final IndexNameExpressionResolver resolver, final Resolved requestedResolved, final User user, - final SecurityRoles securityRoles, final PrivilegesEvaluatorResponse presponse) { - - ThreadContext threadContext = threadPool.getThreadContext(); - - // maskedFields - final Map> maskedFieldsMap = securityRoles.getMaskedFields(user, resolver, clusterService); - final boolean isDebugEnabled = log.isDebugEnabled(); - - if (maskedFieldsMap != null && !maskedFieldsMap.isEmpty()) { - - if(request instanceof ClusterSearchShardsRequest && HeaderHelper.isTrustedClusterRequest(threadContext)) { - threadContext.addResponseHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER, Base64Helper.serializeObject((Serializable) maskedFieldsMap)); - if (isDebugEnabled) { - log.debug("Added response header for masked fields info: {}", maskedFieldsMap); - } - } else { - if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER) != null) { - if (!maskedFieldsMap.equals(Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER)))) { - throw new OpenSearchSecurityException(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER + " does not match "); - } else { - if (isDebugEnabled) { - log.debug("Header {} already set", ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER); - } - } - } else { - threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER, Base64Helper.serializeObject((Serializable) maskedFieldsMap)); - if (isDebugEnabled) { - log.debug("Attach masked fields info: {}", maskedFieldsMap); - } - } - } - - presponse.maskedFields = maskedFieldsMap.entrySet().stream() - .filter(requestedResolved.isLocalAll() || requestedResolved.getAllIndices().isEmpty() ? - entry -> true : entry -> WildcardMatcher.from(entry.getKey()).matchAny(requestedResolved.getAllIndices())) - .collect(ImmutableMap.toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); - - } - - - - // attach dls/fls map if not already done - final Tuple>, Map>> dlsFls = securityRoles.getDlsFls(user, resolver, clusterService); - final Map> dlsQueries = dlsFls.v1(); - final Map> flsFields = dlsFls.v2(); - - if (!dlsQueries.isEmpty()) { - - if(request instanceof ClusterSearchShardsRequest && HeaderHelper.isTrustedClusterRequest(threadContext)) { - threadContext.addResponseHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER, Base64Helper.serializeObject((Serializable) dlsQueries)); - if (isDebugEnabled) { - log.debug("Added response header for DLS info: {}", dlsQueries); - } - } else { - if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER) != null) { - if (!dlsQueries.equals(Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER)))) { - throw new OpenSearchSecurityException(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER + " does not match (SG 900D)"); - } - } else { - threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER, Base64Helper.serializeObject((Serializable) dlsQueries)); - if (isDebugEnabled) { - log.debug("Attach DLS info: {}", dlsQueries); - } - } - } - - presponse.queries = dlsQueries.entrySet().stream() - .filter(requestedResolved.isLocalAll() || requestedResolved.getAllIndices().isEmpty() ? - entry -> true : entry -> WildcardMatcher.from(entry.getKey()).matchAny(requestedResolved.getAllIndices())) - .collect(ImmutableMap.toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); - - } - - if (!flsFields.isEmpty()) { - - if(request instanceof ClusterSearchShardsRequest && HeaderHelper.isTrustedClusterRequest(threadContext)) { - threadContext.addResponseHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER, Base64Helper.serializeObject((Serializable) flsFields)); - if (isDebugEnabled) { - log.debug("Added response header for FLS info: {}", flsFields); - } - } else { - if (threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER) != null) { - if (!flsFields.equals(Base64Helper.deserializeObject(threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER)))) { - throw new OpenSearchSecurityException(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER + " does not match "); - } else { - if (isDebugEnabled) { - log.debug("Header {} already set", ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER); - } - } - } else { - threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER, Base64Helper.serializeObject((Serializable) flsFields)); - if (isDebugEnabled) { - log.debug("Attach FLS info: {}", flsFields); - } - } - } - - presponse.allowedFlsFields = flsFields.entrySet().stream() - .filter(requestedResolved.isLocalAll() || requestedResolved.getAllIndices().isEmpty() ? - entry -> true : entry -> WildcardMatcher.from(entry.getKey()).matchAny(requestedResolved.getAllIndices())) - .collect(ImmutableMap.toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); - - } - - - return presponse; - } -} diff --git a/src/main/java/org/opensearch/security/privileges/DocumentAllowList.java b/src/main/java/org/opensearch/security/privileges/DocumentAllowList.java new file mode 100644 index 0000000000..7ea22ba1ba --- /dev/null +++ b/src/main/java/org/opensearch/security/privileges/DocumentAllowList.java @@ -0,0 +1,278 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.privileges; + +import java.util.HashSet; +import java.util.Set; + +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.security.support.ConfigConstants; + +/** + * Defines which indices and documents are implicitly accessible although a user does not have + * explicit permissions for it. This is required for executing TLQ in DLS queries. In this case + * the user does not have direct access to the index for the term lookup. However, we need to allow + * access for executing the actual TLQ. The document allow list is scoped to individual requests. + */ +public class DocumentAllowList { + + private final Set entries = new HashSet<>(); + + public DocumentAllowList() { + + } + + public void add(String index, String id) { + this.add(new Entry(index, id)); + } + + public void add(Entry entry) { + this.entries.add(entry); + } + + public boolean isEmpty() { + return this.entries.isEmpty(); + } + + public void applyTo(ThreadContext threadContext) { + if (!isEmpty()) { + threadContext.putHeader(ConfigConstants.OPENDISTRO_SECURITY_DOC_ALLOWLIST_HEADER, toString()); + } + } + + public boolean isAllowed(String index, String id) { + for (Entry entry : entries) { + if (entry.index.equals(index) && entry.id.equals(id)) { + return true; + } + } + + return false; + } + + public String toString() { + if (this.entries.isEmpty()) { + return ""; + } + + StringBuilder stringBuilder = new StringBuilder(); + + for (Entry entry : entries) { + if (stringBuilder.length() != 0) { + stringBuilder.append('|'); + } + stringBuilder.append(entry.index).append("/").append(escapeId(entry.id)); + } + + return stringBuilder.toString(); + } + + public static DocumentAllowList parse(String string) { + DocumentAllowList result = new DocumentAllowList(); + + int length = string.length(); + + if (length == 0) { + return result; + } + + int entryStart = 0; + String index = null; + + for (int i = 0;; i++) { + char c; + + if (i < length) { + c = string.charAt(i); + } else { + c = '|'; + } + + while (c == '\\') { + i += 2; + c = string.charAt(i); + } + + if (c == '/') { + index = string.substring(entryStart, i); + entryStart = i + 1; + } else if (c == '|') { + if (index == null) { + throw new IllegalArgumentException("Malformed DocumentAllowList string: " + string); + } + + String id = unescapeId(string.substring(entryStart, i)); + + result.add(index, id); + index = null; + entryStart = i + 1; + } + + if (i >= length) { + break; + } + } + + return result; + } + + private static String escapeId(String id) { + int length = id.length(); + boolean needsEscaping = false; + + for (int i = 0; i < length; i++) { + char c = id.charAt(i); + if (c == '/' || c == '|' || c == '\\') { + needsEscaping = true; + break; + } + } + + if (!needsEscaping) { + return id; + } + + StringBuilder result = new StringBuilder(id.length() + 10); + + for (int i = 0; i < length; i++) { + char c = id.charAt(i); + if (c == '/' || c == '|' || c == '\\') { + result.append('\\'); + } + result.append(c); + } + + return result.toString(); + } + + private static String unescapeId(String id) { + int length = id.length(); + boolean needsEscaping = false; + + for (int i = 0; i < length; i++) { + char c = id.charAt(i); + if (c == '\\') { + needsEscaping = true; + break; + } + } + + if (!needsEscaping) { + return id; + } + + StringBuilder result = new StringBuilder(id.length()); + + for (int i = 0; i < length; i++) { + char c = id.charAt(i); + if (c == '\\') { + i++; + c = id.charAt(i); + } + + result.append(c); + } + + return result.toString(); + } + + public static class Entry { + private final String index; + private final String id; + + Entry(String index, String id) { + if (index.indexOf('/') != -1 || index.indexOf('|') != -1) { + throw new IllegalArgumentException("Invalid index name: " + index); + } + + this.index = index; + this.id = id; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((id == null) ? 0 : id.hashCode()); + result = prime * result + ((index == null) ? 0 : index.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + Entry other = (Entry) obj; + if (id == null) { + if (other.id != null) { + return false; + } + } else if (!id.equals(other.id)) { + return false; + } + if (index == null) { + if (other.index != null) { + return false; + } + } else if (!index.equals(other.index)) { + return false; + } + return true; + } + + @Override + public String toString() { + return "DocumentAllowList.Entry [index=" + index + ", id=" + id + "]"; + } + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((entries == null) ? 0 : entries.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + DocumentAllowList other = (DocumentAllowList) obj; + if (entries == null) { + if (other.entries != null) { + return false; + } + } else if (!entries.equals(other.entries)) { + return false; + } + return true; + } +} diff --git a/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluator.java b/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluator.java index 4d8038b876..8ac47f3fa4 100644 --- a/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluator.java +++ b/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluator.java @@ -64,6 +64,7 @@ import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.BulkShardRequest; import org.opensearch.action.delete.DeleteAction; +import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.MultiGetAction; import org.opensearch.action.index.IndexAction; import org.opensearch.action.search.MultiSearchAction; @@ -82,6 +83,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.index.reindex.ReindexAction; import org.opensearch.security.configuration.ClusterInfoHolder; import org.opensearch.security.configuration.ConfigurationRepository; @@ -133,16 +135,14 @@ public class PrivilegesEvaluator { private final SecurityIndexAccessEvaluator securityIndexAccessEvaluator; private final ProtectedIndexAccessEvaluator protectedIndexAccessEvaluator; private final TermsAggregationEvaluator termsAggregationEvaluator; - - private final DlsFlsEvaluator dlsFlsEvaluator; - private final boolean dlsFlsEnabled; private DynamicConfigModel dcm; - + private final NamedXContentRegistry namedXContentRegistry; + public PrivilegesEvaluator(final ClusterService clusterService, final ThreadPool threadPool, final ConfigurationRepository configurationRepository, final IndexNameExpressionResolver resolver, AuditLog auditLog, final Settings settings, final PrivilegesInterceptor privilegesInterceptor, final ClusterInfoHolder clusterInfoHolder, - final IndexResolverReplacer irr, boolean dlsFlsEnabled) { + final IndexResolverReplacer irr, boolean dlsFlsEnabled, NamedXContentRegistry namedXContentRegistry) { super(); this.clusterService = clusterService; @@ -161,8 +161,8 @@ public PrivilegesEvaluator(final ClusterService clusterService, final ThreadPool snapshotRestoreEvaluator = new SnapshotRestoreEvaluator(settings, auditLog); securityIndexAccessEvaluator = new SecurityIndexAccessEvaluator(settings, auditLog, irr); protectedIndexAccessEvaluator = new ProtectedIndexAccessEvaluator(settings, auditLog); - dlsFlsEvaluator = new DlsFlsEvaluator(settings, threadPool); termsAggregationEvaluator = new TermsAggregationEvaluator(); + this.namedXContentRegistry = namedXContentRegistry; this.dlsFlsEnabled = dlsFlsEnabled; } @@ -261,19 +261,13 @@ public PrivilegesEvaluatorResponse evaluate(final User user, String action0, fin } final Resolved requestedResolved = irr.resolveRequest(request); + presponse.resolved = requestedResolved; + if (isDebugEnabled) { log.debug("RequestedResolved : {}", requestedResolved); } - - // check dlsfls - if (dlsFlsEnabled - //&& (action0.startsWith("indices:data/read") || action0.equals(ClusterSearchShardsAction.NAME)) - && dlsFlsEvaluator.evaluate(request, clusterService, resolver, requestedResolved, user, securityRoles, presponse).isComplete()) { - return presponse; - } - // check snapshot/restore requests if (snapshotRestoreEvaluator.evaluate(request, task, action0, clusterInfoHolder, presponse).isComplete()) { return presponse; @@ -296,6 +290,8 @@ public PrivilegesEvaluatorResponse evaluate(final User user, String action0, fin log.trace("dnfof enabled? {}", dnfofEnabled); } + presponse.evaluatedDlsFlsConfig = getSecurityRoles(mappedRoles).getDlsFls(user, resolver, clusterService, namedXContentRegistry); + if (isClusterPerm(action0)) { if(!securityRoles.impliesClusterPermissionPermission(action0)) { @@ -368,6 +364,11 @@ public PrivilegesEvaluatorResponse evaluate(final User user, String action0, fin } } + if (checkDocAllowListHeader(user, action0, request)) { + presponse.allowed = true; + return presponse; + } + // term aggregations if (termsAggregationEvaluator.evaluate(requestedResolved, request, clusterService, user, securityRoles, resolver, presponse) .isComplete()) { return presponse; @@ -589,7 +590,7 @@ private Set evaluateAdditionalIndexPermissions(final ActionRequest reque return Collections.unmodifiableSet(additionalPermissionsRequired); } - private static boolean isClusterPerm(String action0) { + public static boolean isClusterPerm(String action0) { return ( action0.startsWith("cluster:") || action0.startsWith("indices:admin/template/") @@ -681,6 +682,38 @@ public Iterator iterator() { return false; } + + private boolean checkDocAllowListHeader(User user, String action, ActionRequest request) { + String docAllowListHeader = threadContext.getHeader(ConfigConstants.OPENDISTRO_SECURITY_DOC_ALLOWLIST_HEADER); + + if (docAllowListHeader == null) { + return false; + } + + if (!(request instanceof GetRequest)) { + return false; + } + + try { + DocumentAllowList documentAllowList = DocumentAllowList.parse(docAllowListHeader); + GetRequest getRequest = (GetRequest) request; + + if (documentAllowList.isAllowed(getRequest.index(), getRequest.id())) { + if (log.isDebugEnabled()) { + log.debug("Request " + request + " is allowed by " + documentAllowList); + } + + return true; + } else { + return false; + } + + } catch (Exception e) { + log.error("Error while handling document allow list: " + docAllowListHeader, e); + return false; + } + } + private List toString(List aliases) { if(aliases == null || aliases.size() == 0) { return Collections.emptyList(); diff --git a/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluatorResponse.java b/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluatorResponse.java index 234ad5686c..e8bb362493 100644 --- a/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluatorResponse.java +++ b/src/main/java/org/opensearch/security/privileges/PrivilegesEvaluatorResponse.java @@ -31,9 +31,10 @@ package org.opensearch.security.privileges; import org.opensearch.action.admin.indices.create.CreateIndexRequestBuilder; +import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; +import org.opensearch.security.securityconf.EvaluatedDlsFlsConfig; import java.util.HashSet; -import java.util.Map; import java.util.Set; public class PrivilegesEvaluatorResponse { @@ -41,12 +42,15 @@ public class PrivilegesEvaluatorResponse { Set missingPrivileges = new HashSet(); Set missingSecurityRoles = new HashSet<>(); Set resolvedSecurityRoles = new HashSet<>(); - Map> allowedFlsFields; - Map> maskedFields; - Map> queries; + EvaluatedDlsFlsConfig evaluatedDlsFlsConfig; PrivilegesEvaluatorResponseState state = PrivilegesEvaluatorResponseState.PENDING; + Resolved resolved; CreateIndexRequestBuilder createIndexRequestBuilder; + public Resolved getResolved() { + return resolved; + } + public boolean isAllowed() { return allowed; } @@ -58,18 +62,10 @@ public Set getMissingPrivileges() { public Set getResolvedSecurityRoles() {return new HashSet<>(resolvedSecurityRoles); } - public Map> getAllowedFlsFields() { - return allowedFlsFields; + public EvaluatedDlsFlsConfig getEvaluatedDlsFlsConfig() { + return evaluatedDlsFlsConfig; } - public Map> getMaskedFields() { - return maskedFields; - } - - public Map> getQueries() { - return queries; - } - public CreateIndexRequestBuilder getCreateIndexRequestBuilder() { return createIndexRequestBuilder; } @@ -94,8 +90,8 @@ public boolean isPending() { @Override public String toString() { - return "PrivEvalResponse [allowed=" + allowed + ", missingPrivileges=" + missingPrivileges - + ", allowedFlsFields=" + allowedFlsFields + ", maskedFields=" + maskedFields + ", queries=" + queries + "]"; + return "PrivEvalResponse [allowed=" + allowed + ", missingPrivileges=" + missingPrivileges + ", evaluatedDlsFlsConfig=" + + evaluatedDlsFlsConfig + "]"; } public static enum PrivilegesEvaluatorResponseState { diff --git a/src/main/java/org/opensearch/security/queries/QueryBuilderTraverser.java b/src/main/java/org/opensearch/security/queries/QueryBuilderTraverser.java new file mode 100644 index 0000000000..a10b4bf882 --- /dev/null +++ b/src/main/java/org/opensearch/security/queries/QueryBuilderTraverser.java @@ -0,0 +1,259 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.queries; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Predicate; + +import org.opensearch.common.xcontent.DeprecationHandler; +import org.opensearch.common.xcontent.NamedXContentRegistry; +import org.opensearch.common.xcontent.XContentParser; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.index.query.AbstractQueryBuilder; +import org.opensearch.index.query.BoolQueryBuilder; +import org.opensearch.index.query.BoostingQueryBuilder; +import org.opensearch.index.query.ConstantScoreQueryBuilder; +import org.opensearch.index.query.DisMaxQueryBuilder; +import org.opensearch.index.query.FieldMaskingSpanQueryBuilder; +import org.opensearch.index.query.NestedQueryBuilder; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.SpanContainingQueryBuilder; +import org.opensearch.index.query.SpanFirstQueryBuilder; +import org.opensearch.index.query.SpanMultiTermQueryBuilder; +import org.opensearch.index.query.SpanNearQueryBuilder; +import org.opensearch.index.query.SpanNotQueryBuilder; +import org.opensearch.index.query.SpanOrQueryBuilder; +import org.opensearch.index.query.SpanWithinQueryBuilder; +import org.opensearch.index.query.functionscore.FunctionScoreQueryBuilder; + +public abstract class QueryBuilderTraverser { + + public static boolean exists(QueryBuilder queryBuilder, Predicate predicate) { + Exists traverser = new Exists(predicate); + + return traverser.check(queryBuilder); + } + + public static QueryBuilder find(QueryBuilder queryBuilder, Predicate predicate) { + Exists traverser = new Exists(predicate); + + if (traverser.check(queryBuilder)) { + return traverser.matched; + } else { + return null; + } + } + + public static Set findAll(QueryBuilder queryBuilder, Predicate predicate) { + Complete traverser = new Complete(predicate); + + traverser.check(queryBuilder); + + return traverser.matched; + } + + public boolean check(String query, NamedXContentRegistry namedXContentRegistry) throws IOException { + XContentParser parser = JsonXContent.jsonXContent.createParser(namedXContentRegistry, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, query); + QueryBuilder queryBuilder = AbstractQueryBuilder.parseInnerQueryBuilder(parser); + + return check(queryBuilder); + } + + public abstract boolean check(QueryBuilder queryBuilder); + + public abstract boolean check(List queryBuilders); + + public static class Exists extends QueryBuilderTraverser { + private final Predicate predicate; + private QueryBuilder matched; + + public Exists(Predicate predicate) { + this.predicate = predicate; + } + + public boolean check(QueryBuilder queryBuilder) { + if (queryBuilder == null) { + return false; + } + + if (predicate.test(queryBuilder)) { + this.matched = queryBuilder; + return true; + } + + if (queryBuilder instanceof BoolQueryBuilder) { + BoolQueryBuilder boolQueryBuilder = (BoolQueryBuilder) queryBuilder; + return check(boolQueryBuilder.must()) || check(boolQueryBuilder.mustNot()) || check(boolQueryBuilder.should()) + || check(boolQueryBuilder.filter()); + } else if (queryBuilder instanceof BoostingQueryBuilder) { + BoostingQueryBuilder boostingQueryBuilder = (BoostingQueryBuilder) queryBuilder; + return check(boostingQueryBuilder.positiveQuery()) || check(boostingQueryBuilder.negativeQuery()); + } else if (queryBuilder instanceof ConstantScoreQueryBuilder) { + ConstantScoreQueryBuilder constantScoreQueryBuilder = (ConstantScoreQueryBuilder) queryBuilder; + return check(constantScoreQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof DisMaxQueryBuilder) { + DisMaxQueryBuilder disMaxQueryBuilder = (DisMaxQueryBuilder) queryBuilder; + return check(disMaxQueryBuilder.innerQueries()); + } else if (queryBuilder instanceof FieldMaskingSpanQueryBuilder) { + FieldMaskingSpanQueryBuilder fieldMaskingSpanQueryBuilder = (FieldMaskingSpanQueryBuilder) queryBuilder; + return check(fieldMaskingSpanQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof FunctionScoreQueryBuilder) { + FunctionScoreQueryBuilder functionScoreQueryBuilder = (FunctionScoreQueryBuilder) queryBuilder; + return check(functionScoreQueryBuilder.query()); + } else if (queryBuilder instanceof NestedQueryBuilder) { + NestedQueryBuilder nestedQueryBuilder = (NestedQueryBuilder) queryBuilder; + return check(nestedQueryBuilder.query()); + } else if (queryBuilder instanceof SpanContainingQueryBuilder) { + SpanContainingQueryBuilder spanContainingQueryBuilder = (SpanContainingQueryBuilder) queryBuilder; + return check(spanContainingQueryBuilder.bigQuery()) || check(spanContainingQueryBuilder.littleQuery()); + } else if (queryBuilder instanceof SpanFirstQueryBuilder) { + SpanFirstQueryBuilder spanFirstQueryBuilder = (SpanFirstQueryBuilder) queryBuilder; + return check(spanFirstQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof SpanMultiTermQueryBuilder) { + SpanMultiTermQueryBuilder spanMultiTermQueryBuilder = (SpanMultiTermQueryBuilder) queryBuilder; + return check(spanMultiTermQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof SpanNearQueryBuilder) { + SpanNearQueryBuilder spanNearQueryBuilder = (SpanNearQueryBuilder) queryBuilder; + return check(spanNearQueryBuilder.clauses()); + } else if (queryBuilder instanceof SpanNotQueryBuilder) { + SpanNotQueryBuilder spanNotQueryBuilder = (SpanNotQueryBuilder) queryBuilder; + return check(spanNotQueryBuilder.excludeQuery()) || check(spanNotQueryBuilder.includeQuery()); + } else if (queryBuilder instanceof SpanOrQueryBuilder) { + SpanOrQueryBuilder spanOrQueryBuilder = (SpanOrQueryBuilder) queryBuilder; + return check(spanOrQueryBuilder.clauses()); + } else if (queryBuilder instanceof SpanWithinQueryBuilder) { + SpanWithinQueryBuilder spanWithinQueryBuilder = (SpanWithinQueryBuilder) queryBuilder; + return check(spanWithinQueryBuilder.bigQuery()) || check(spanWithinQueryBuilder.littleQuery()); + } else { + return false; + } + } + + public boolean check(List queryBuilders) { + for (QueryBuilder queryBuilder : queryBuilders) { + if (check(queryBuilder)) { + return true; + } + } + + return false; + } + + public QueryBuilder getMatched() { + return matched; + } + } + + public static class Complete extends QueryBuilderTraverser { + private final Predicate predicate; + private Set matched = new HashSet<>(); + + public Complete(Predicate predicate) { + this.predicate = predicate; + } + + public boolean check(QueryBuilder queryBuilder) { + if (queryBuilder == null) { + return true; + } + + boolean matched = true; + + if (queryBuilder instanceof BoolQueryBuilder) { + BoolQueryBuilder boolQueryBuilder = (BoolQueryBuilder) queryBuilder; + + matched = check(boolQueryBuilder.must()) & check(boolQueryBuilder.mustNot()) & check(boolQueryBuilder.should()) + & check(boolQueryBuilder.filter()); + } else if (queryBuilder instanceof BoostingQueryBuilder) { + BoostingQueryBuilder boostingQueryBuilder = (BoostingQueryBuilder) queryBuilder; + + matched = check(boostingQueryBuilder.positiveQuery()) & check(boostingQueryBuilder.negativeQuery()); + } else if (queryBuilder instanceof ConstantScoreQueryBuilder) { + ConstantScoreQueryBuilder constantScoreQueryBuilder = (ConstantScoreQueryBuilder) queryBuilder; + + matched = check(constantScoreQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof DisMaxQueryBuilder) { + DisMaxQueryBuilder disMaxQueryBuilder = (DisMaxQueryBuilder) queryBuilder; + + matched = check(disMaxQueryBuilder.innerQueries()); + } else if (queryBuilder instanceof FieldMaskingSpanQueryBuilder) { + FieldMaskingSpanQueryBuilder fieldMaskingSpanQueryBuilder = (FieldMaskingSpanQueryBuilder) queryBuilder; + + matched = check(fieldMaskingSpanQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof FunctionScoreQueryBuilder) { + FunctionScoreQueryBuilder functionScoreQueryBuilder = (FunctionScoreQueryBuilder) queryBuilder; + + matched = check(functionScoreQueryBuilder.query()); + } else if (queryBuilder instanceof NestedQueryBuilder) { + NestedQueryBuilder nestedQueryBuilder = (NestedQueryBuilder) queryBuilder; + + matched = check(nestedQueryBuilder.query()); + } else if (queryBuilder instanceof SpanContainingQueryBuilder) { + SpanContainingQueryBuilder spanContainingQueryBuilder = (SpanContainingQueryBuilder) queryBuilder; + + matched = check(spanContainingQueryBuilder.bigQuery()) & check(spanContainingQueryBuilder.littleQuery()); + } else if (queryBuilder instanceof SpanFirstQueryBuilder) { + SpanFirstQueryBuilder spanFirstQueryBuilder = (SpanFirstQueryBuilder) queryBuilder; + + matched = check(spanFirstQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof SpanMultiTermQueryBuilder) { + SpanMultiTermQueryBuilder spanMultiTermQueryBuilder = (SpanMultiTermQueryBuilder) queryBuilder; + + matched = check(spanMultiTermQueryBuilder.innerQuery()); + } else if (queryBuilder instanceof SpanNearQueryBuilder) { + SpanNearQueryBuilder spanNearQueryBuilder = (SpanNearQueryBuilder) queryBuilder; + + matched = check(spanNearQueryBuilder.clauses()); + } else if (queryBuilder instanceof SpanNotQueryBuilder) { + SpanNotQueryBuilder spanNotQueryBuilder = (SpanNotQueryBuilder) queryBuilder; + + matched = check(spanNotQueryBuilder.excludeQuery()) & check(spanNotQueryBuilder.includeQuery()); + } else if (queryBuilder instanceof SpanOrQueryBuilder) { + SpanOrQueryBuilder spanOrQueryBuilder = (SpanOrQueryBuilder) queryBuilder; + + matched = check(spanOrQueryBuilder.clauses()); + } else if (queryBuilder instanceof SpanWithinQueryBuilder) { + SpanWithinQueryBuilder spanWithinQueryBuilder = (SpanWithinQueryBuilder) queryBuilder; + + matched = check(spanWithinQueryBuilder.bigQuery()) & check(spanWithinQueryBuilder.littleQuery()); + } + + if (predicate.test(queryBuilder)) { + this.matched.add(queryBuilder); + return matched; + } else { + return false; + } + } + + public boolean check(List queryBuilders) { + boolean matched = true; + + for (QueryBuilder queryBuilder : queryBuilders) { + matched &= check(queryBuilder); + } + + return matched; + } + + public Set getMatched() { + return matched; + } + } +} diff --git a/src/main/java/org/opensearch/security/resolver/IndexResolverReplacer.java b/src/main/java/org/opensearch/security/resolver/IndexResolverReplacer.java index bb8fdd2ab0..0ce4040987 100644 --- a/src/main/java/org/opensearch/security/resolver/IndexResolverReplacer.java +++ b/src/main/java/org/opensearch/security/resolver/IndexResolverReplacer.java @@ -307,9 +307,9 @@ public String[] provide(String[] original, Object localRequest, boolean supports return IndicesProvider.NOOP; } - Resolved resolved() { + Resolved resolved(IndicesOptions indicesOptions) { final Resolved resolved = alreadyResolved.isEmpty() ? Resolved._LOCAL_ALL : - new Resolved(aliases.build(), allIndices.build(), originalRequested.build(), remoteIndices.build()); + new Resolved(aliases.build(), allIndices.build(), originalRequested.build(), remoteIndices.build(), indicesOptions); if(log.isTraceEnabled()) { log.trace("Finally resolved for {}: {}", name, resolved); @@ -349,30 +349,33 @@ public Resolved resolveRequest(final Object request) { getOrReplaceAllIndices(request, resolvedIndicesProvider, false); - return resolvedIndicesProvider.resolved(); + return resolvedIndicesProvider.resolved(indicesOptionsFrom(request)); } public final static class Resolved { private static final String ANY = "*"; private static final ImmutableSet All_SET = ImmutableSet.of(ANY); private static final Set types = All_SET; - public static final Resolved _LOCAL_ALL = new Resolved(All_SET, All_SET, All_SET, ImmutableSet.of()); + public static final Resolved _LOCAL_ALL = new Resolved(All_SET, All_SET, All_SET, ImmutableSet.of(), SearchRequest.DEFAULT_INDICES_OPTIONS); private final Set aliases; private final Set allIndices; private final Set originalRequested; private final Set remoteIndices; private final boolean isLocalAll; - + private final IndicesOptions indicesOptions; + private Resolved(final ImmutableSet aliases, final ImmutableSet allIndices, final ImmutableSet originalRequested, - final ImmutableSet remoteIndices) { + final ImmutableSet remoteIndices, + IndicesOptions indicesOptions) { this.aliases = aliases; this.allIndices = allIndices; this.originalRequested = originalRequested; this.remoteIndices = remoteIndices; this.isLocalAll = IndexResolverReplacer.isLocalAll(originalRequested.toArray(new String[0])) || (aliases.contains("*") && allIndices.contains("*")); + this.indicesOptions = indicesOptions; } public boolean isLocalAll() { @@ -386,6 +389,18 @@ public Set getAliases() { public Set getAllIndices() { return allIndices; } + + public Set getAllIndicesResolved(ClusterService clusterService, IndexNameExpressionResolver resolver) { + if (isLocalAll) { + return new HashSet<>(Arrays.asList(resolver.concreteIndexNames(clusterService.state(), indicesOptions, "*"))); + } else { + return allIndices; + } + } + + public boolean isAllIndicesEmpty() { + return allIndices.isEmpty(); + } public Set getTypes() { return types; diff --git a/src/main/java/org/opensearch/security/securityconf/ConfigModelV6.java b/src/main/java/org/opensearch/security/securityconf/ConfigModelV6.java index f975ed5891..cd30e83f32 100644 --- a/src/main/java/org/opensearch/security/securityconf/ConfigModelV6.java +++ b/src/main/java/org/opensearch/security/securityconf/ConfigModelV6.java @@ -44,7 +44,8 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; - +import org.opensearch.common.util.set.Sets; +import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; import org.opensearch.security.securityconf.impl.SecurityDynamicConfiguration; import org.opensearch.security.securityconf.impl.v6.ActionGroupsV6; @@ -361,53 +362,25 @@ public SecurityRoles filter(Set keep) { return retVal; } - @Override - public Map> getMaskedFields(User user, IndexNameExpressionResolver resolver, ClusterService cs) { - final Map> maskedFieldsMap = new HashMap<>(); - - for (SecurityRole sr : roles) { - for (IndexPattern ip : sr.getIpatterns()) { - final Set maskedFields = ip.getMaskedFields(); - if (!maskedFields.isEmpty()) { - final String indexPattern = ip.getUnresolvedIndexPattern(user); - Set concreteIndices = ip.getResolvedIndexPattern(user, resolver, cs); - - Set currentMaskedFields = maskedFieldsMap.get(indexPattern); - if (currentMaskedFields != null) { - currentMaskedFields.addAll(maskedFields); - } else { - maskedFieldsMap.put(indexPattern, new HashSet<>(maskedFields)); - } - - for (String concreteIndex : concreteIndices) { - currentMaskedFields = maskedFieldsMap.get(concreteIndex); - if (currentMaskedFields != null) { - currentMaskedFields.addAll(maskedFields); - } else { - maskedFieldsMap.put(concreteIndex, new HashSet<>(maskedFields)); - } - } - } - } - } - return maskedFieldsMap; - } @Override - public Tuple>, Map>> getDlsFls(User user, IndexNameExpressionResolver resolver, - ClusterService cs) { + public EvaluatedDlsFlsConfig getDlsFls(User user, IndexNameExpressionResolver resolver, ClusterService cs, + NamedXContentRegistry namedXContentRegistry) { final Map> dlsQueries = new HashMap>(); final Map> flsFields = new HashMap>(); - + final Map> maskedFieldsMap = new HashMap>(); + for (SecurityRole sr : roles) { for (IndexPattern ip : sr.getIpatterns()) { final Set fls = ip.getFls(); final String dls = ip.getDlsQuery(user); final String indexPattern = ip.getUnresolvedIndexPattern(user); + final Set maskedFields = ip.getMaskedFields(); Set concreteIndices = new HashSet<>(); - if ((dls != null && dls.length() > 0) || (fls != null && fls.size() > 0)) { + + if ((dls != null && dls.length() > 0) || (fls != null && fls.size() > 0) || (maskedFields != null && maskedFields.size() > 0)) { concreteIndices = ip.getResolvedIndexPattern(user, resolver, cs); } @@ -449,12 +422,32 @@ public Tuple>, Map>> getDlsFls(User } } } + + if (maskedFields != null && maskedFields.size() > 0) { + + if (maskedFieldsMap.containsKey(indexPattern)) { + maskedFieldsMap.get(indexPattern).addAll(Sets.newHashSet(maskedFields)); + } else { + maskedFieldsMap.put(indexPattern, new HashSet()); + maskedFieldsMap.get(indexPattern).addAll(Sets.newHashSet(maskedFields)); + } + + for (String concreteIndex : concreteIndices) { + if (maskedFieldsMap.containsKey(concreteIndex)) { + maskedFieldsMap.get(concreteIndex).addAll(Sets.newHashSet(maskedFields)); + } else { + maskedFieldsMap.put(concreteIndex, new HashSet()); + maskedFieldsMap.get(concreteIndex).addAll(Sets.newHashSet(maskedFields)); + } + } + } } } + + return new EvaluatedDlsFlsConfig(dlsQueries, flsFields, maskedFieldsMap); + } - return new Tuple<>(dlsQueries, flsFields); - } //opensearchDashboards special only, terms eval public Set getAllPermittedIndicesForDashboards(Resolved resolved, User user, String[] actions, IndexNameExpressionResolver resolver, ClusterService cs) { diff --git a/src/main/java/org/opensearch/security/securityconf/ConfigModelV7.java b/src/main/java/org/opensearch/security/securityconf/ConfigModelV7.java index 612a8a2992..aec7d881e3 100644 --- a/src/main/java/org/opensearch/security/securityconf/ConfigModelV7.java +++ b/src/main/java/org/opensearch/security/securityconf/ConfigModelV7.java @@ -44,7 +44,8 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; - +import org.opensearch.common.util.set.Sets; +import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; import org.opensearch.security.securityconf.impl.SecurityDynamicConfiguration; import org.opensearch.security.securityconf.impl.v7.ActionGroupsV7; @@ -348,101 +349,71 @@ public SecurityRoles filter(Set keep) { return retVal; } - @Override - public Map> getMaskedFields(User user, IndexNameExpressionResolver resolver, ClusterService cs) { - final Map> maskedFieldsMap = new HashMap<>(); - - for (SecurityRole sr : roles) { - for (IndexPattern ip : sr.getIpatterns()) { - final Set maskedFields = ip.getMaskedFields(); - if (!maskedFields.isEmpty()) { - final String indexPattern = ip.getUnresolvedIndexPattern(user); - Set concreteIndices = ip.getResolvedIndexPattern(user, resolver, cs); - - Set currentMaskedFields = maskedFieldsMap.get(indexPattern); - if (currentMaskedFields != null) { - currentMaskedFields.addAll(maskedFields); - } else { - maskedFieldsMap.put(indexPattern, new HashSet<>(maskedFields)); - } - - for (String concreteIndex : concreteIndices) { - currentMaskedFields = maskedFieldsMap.get(concreteIndex); - if (currentMaskedFields != null) { - currentMaskedFields.addAll(maskedFields); - } else { - maskedFieldsMap.put(concreteIndex, new HashSet<>(maskedFields)); - } - } - } - } - } - return maskedFieldsMap; - } @Override - public Tuple>, Map>> getDlsFls(User user, IndexNameExpressionResolver resolver, - ClusterService cs) { + public EvaluatedDlsFlsConfig getDlsFls(User user, IndexNameExpressionResolver resolver, ClusterService cs, + NamedXContentRegistry namedXContentRegistry) { - final Map> dlsQueries = new HashMap>(); - final Map> flsFields = new HashMap>(); - for (SecurityRole sr : roles) { - for (IndexPattern ip : sr.getIpatterns()) { - final Set fls = ip.getFls(); - final String dls = ip.getDlsQuery(user); - final String indexPattern = ip.getUnresolvedIndexPattern(user); - Set concreteIndices = new HashSet<>(); - - if ((dls != null && dls.length() > 0) || (fls != null && fls.size() > 0)) { - concreteIndices = ip.getResolvedIndexPattern(user, resolver, cs); - } + if (!containsDlsFlsConfig()) { + if(log.isDebugEnabled()) { + log.debug("No fls or dls found for {} in {} security roles", user, roles.size()); + } - if (dls != null && dls.length() > 0) { + return EvaluatedDlsFlsConfig.EMPTY; + } + + Map> dlsQueriesByIndex = new HashMap>(); + Map> flsFields = new HashMap>(); + Map> maskedFieldsMap = new HashMap>(); + + for (SecurityRole role : roles) { + for (IndexPattern ip : role.getIpatterns()) { + Set concreteIndices; + concreteIndices = ip.getResolvedIndexPattern(user, resolver, cs); + String dls = ip.getDlsQuery(user); + + if (dls != null && dls.length() > 0) { + + for (String concreteIndex : concreteIndices) { + dlsQueriesByIndex.computeIfAbsent(concreteIndex, (key) -> new HashSet()).add(dls); + } + } + + Set fls = ip.getFls(); - Set dlsQuery = dlsQueries.get(indexPattern); - if (dlsQuery != null) { - dlsQuery.add(dls); - } else { - dlsQueries.put(indexPattern, new HashSet<>(Arrays.asList(dls))); - } + if (fls != null && fls.size() > 0) { for (String concreteIndex : concreteIndices) { - dlsQuery = dlsQueries.get(concreteIndex); - if (dlsQuery != null) { - dlsQuery.add(dls); + if (flsFields.containsKey(concreteIndex)) { + flsFields.get(concreteIndex).addAll(Sets.newHashSet(fls)); } else { - dlsQueries.put(concreteIndex, new HashSet<>(Arrays.asList(dls))); + flsFields.put(concreteIndex, new HashSet()); + flsFields.get(concreteIndex).addAll(Sets.newHashSet(fls)); } } - } + + Set maskedFields = ip.getMaskedFields(); - if (fls != null && fls.size() > 0) { - - Set flsField = flsFields.get(indexPattern); - if (flsField != null) { - flsField.addAll(fls); - } else { - flsFields.put(indexPattern, new HashSet<>(fls)); - } + if (maskedFields != null && maskedFields.size() > 0) { - for (String concreteIndex : concreteIndices) { - flsField = flsFields.get(concreteIndex); - if (flsField != null) { - flsField.addAll(fls); + for (String concreteIndex : concreteIndices) { + if (maskedFieldsMap.containsKey(concreteIndex)) { + maskedFieldsMap.get(concreteIndex).addAll(Sets.newHashSet(maskedFields)); } else { - flsFields.put(concreteIndex, new HashSet<>(fls)); + maskedFieldsMap.put(concreteIndex, new HashSet()); + maskedFieldsMap.get(concreteIndex).addAll(Sets.newHashSet(maskedFields)); } } - } + } } } - return new Tuple<>(dlsQueries, flsFields); - + return new EvaluatedDlsFlsConfig(dlsQueriesByIndex, flsFields, maskedFieldsMap); } + //opensearchDashboards special only, terms eval public Set getAllPermittedIndicesForDashboards(Resolved resolved, User user, String[] actions, IndexNameExpressionResolver resolver, ClusterService cs) { Set retVal = new HashSet<>(); @@ -486,6 +457,18 @@ public boolean impliesTypePermGlobal(Resolved resolved, User user, String[] acti roles.stream().forEach(p -> ipatterns.addAll(p.getIpatterns())); return ConfigModelV7.impliesTypePerm(ipatterns, resolved, user, actions, resolver, cs); } + + private boolean containsDlsFlsConfig() { + for (SecurityRole role : roles) { + for (IndexPattern ip : role.getIpatterns()) { + if (ip.hasDlsQuery() || ip.hasFlsFields() || ip.hasMaskedFields()) { + return true; + } + } + } + + return false; + } } public static class SecurityRole { @@ -739,7 +722,7 @@ public String getUnresolvedIndexPattern(User user) { return replaceProperties(indexPattern, user); } - private Set getResolvedIndexPattern(User user, IndexNameExpressionResolver resolver, ClusterService cs) { + public Set getResolvedIndexPattern(User user, IndexNameExpressionResolver resolver, ClusterService cs) { String unresolved = getUnresolvedIndexPattern(user); WildcardMatcher matcher = WildcardMatcher.from(unresolved); String[] resolved = null; @@ -772,18 +755,31 @@ public String getDlsQuery(User user) { return replaceProperties(dlsQuery, user); } + public boolean hasDlsQuery() { + return dlsQuery != null && !dlsQuery.isEmpty(); + } + public Set getFls() { return Collections.unmodifiableSet(fls); } + public boolean hasFlsFields() { + return fls != null && !fls.isEmpty(); + } + public Set getMaskedFields() { return Collections.unmodifiableSet(maskedFields); } + public boolean hasMaskedFields() { + return maskedFields != null && !maskedFields.isEmpty(); + } + public WildcardMatcher getPerms() { return WildcardMatcher.from(perms); } + } /*public static class TypePerm { diff --git a/src/main/java/org/opensearch/security/securityconf/EvaluatedDlsFlsConfig.java b/src/main/java/org/opensearch/security/securityconf/EvaluatedDlsFlsConfig.java new file mode 100644 index 0000000000..404ec7be3d --- /dev/null +++ b/src/main/java/org/opensearch/security/securityconf/EvaluatedDlsFlsConfig.java @@ -0,0 +1,130 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.securityconf; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; +import org.opensearch.security.support.WildcardMatcher; + +public class EvaluatedDlsFlsConfig { + public static EvaluatedDlsFlsConfig EMPTY = new EvaluatedDlsFlsConfig(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); + + private final Map> dlsQueriesByIndex; + private final Map> flsByIndex; + private final Map> fieldMaskingByIndex; + + public EvaluatedDlsFlsConfig(Map> dlsQueriesByIndex, Map> flsByIndex, + Map> fieldMaskingByIndex) { + this.dlsQueriesByIndex = Collections.unmodifiableMap(dlsQueriesByIndex); + this.flsByIndex = Collections.unmodifiableMap(flsByIndex); + this.fieldMaskingByIndex = Collections.unmodifiableMap(fieldMaskingByIndex); + } + + public Map> getDlsQueriesByIndex() { + return dlsQueriesByIndex; + } + + public Map> getFlsByIndex() { + return flsByIndex; + } + + public Map> getFieldMaskingByIndex() { + return fieldMaskingByIndex; + } + + public Set getAllQueries() { + int mapSize = dlsQueriesByIndex.size(); + + if (mapSize == 0) { + return Collections.emptySet(); + } else if (mapSize == 1) { + return dlsQueriesByIndex.values().iterator().next(); + } else { + Set result = new HashSet<>(); + + for (Set queries : dlsQueriesByIndex.values()) { + result.addAll(queries); + } + + return result; + } + } + + public boolean hasFls() { + return !flsByIndex.isEmpty(); + } + + public boolean hasFieldMasking() { + return !fieldMaskingByIndex.isEmpty(); + } + + public boolean hasDls() { + return !dlsQueriesByIndex.isEmpty(); + } + + public boolean isEmpty() { + return fieldMaskingByIndex.isEmpty() && flsByIndex.isEmpty() && dlsQueriesByIndex.isEmpty(); + } + + public EvaluatedDlsFlsConfig filter(Resolved indices) { + if (indices.isAllIndicesEmpty()) { + return EMPTY; + } else if (this.isEmpty() || indices.isLocalAll()) { + return this; + } else { + Set allIndices = indices.getAllIndices(); + + return new EvaluatedDlsFlsConfig(filter(dlsQueriesByIndex, allIndices), filter(flsByIndex, allIndices), + filter(fieldMaskingByIndex, allIndices)); + } + } + + public EvaluatedDlsFlsConfig withoutDls() { + if (!hasDls()) { + return this; + } else { + return new EvaluatedDlsFlsConfig(Collections.emptyMap(), flsByIndex, fieldMaskingByIndex); + } + } + + private Map> filter(Map> map, Set allIndices) { + if (allIndices.isEmpty() || map.isEmpty()) { + return map; + } + + HashMap> result = new HashMap<>(map.size()); + + for (Map.Entry> entry : map.entrySet()) { + if (WildcardMatcher.from(entry.getKey(), false).matchAny(allIndices)) { + result.put(entry.getKey(), entry.getValue()); + } + } + + return result; + } + + @Override + public String toString() { + return "EvaluatedDlsFlsConfig [dlsQueriesByIndex=" + dlsQueriesByIndex + ", flsByIndex=" + flsByIndex + ", fieldMaskingByIndex=" + + fieldMaskingByIndex + "]"; + } + +} diff --git a/src/main/java/org/opensearch/security/securityconf/SecurityRoles.java b/src/main/java/org/opensearch/security/securityconf/SecurityRoles.java index d1674260bd..0ad849d51f 100644 --- a/src/main/java/org/opensearch/security/securityconf/SecurityRoles.java +++ b/src/main/java/org/opensearch/security/securityconf/SecurityRoles.java @@ -31,13 +31,11 @@ package org.opensearch.security.securityconf; -import java.util.Map; import java.util.Set; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.collect.Tuple; - +import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.security.resolver.IndexResolverReplacer.Resolved; import org.opensearch.security.user.User; @@ -53,9 +51,7 @@ public interface SecurityRoles { boolean get(Resolved requestedResolved, User user, String[] allIndexPermsRequiredA, IndexNameExpressionResolver resolver, ClusterService clusterService); - Map> getMaskedFields(User user, IndexNameExpressionResolver resolver, ClusterService clusterService); - - Tuple>, Map>> getDlsFls(User user, IndexNameExpressionResolver resolver, ClusterService clusterService); + EvaluatedDlsFlsConfig getDlsFls(User user, IndexNameExpressionResolver resolver, ClusterService clusterService, NamedXContentRegistry namedXContentRegistry); Set getAllPermittedIndicesForDashboards(Resolved resolved, User user, String[] actions, IndexNameExpressionResolver resolver, ClusterService cs); diff --git a/src/main/java/org/opensearch/security/support/ConfigConstants.java b/src/main/java/org/opensearch/security/support/ConfigConstants.java index a0324d1a2b..93e0f328be 100644 --- a/src/main/java/org/opensearch/security/support/ConfigConstants.java +++ b/src/main/java/org/opensearch/security/support/ConfigConstants.java @@ -54,11 +54,23 @@ public class ConfigConstants { public static final String OPENDISTRO_SECURITY_ORIGIN_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"origin_header"; public static final String OPENDISTRO_SECURITY_DLS_QUERY_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"dls_query"; + + public static final String OPENDISTRO_SECURITY_DLS_FILTER_LEVEL_QUERY_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"dls_filter_level_query"; + public static final String OPENDISTRO_SECURITY_DLS_FILTER_LEVEL_QUERY_TRANSIENT = OPENDISTRO_SECURITY_CONFIG_PREFIX+"dls_filter_level_query_t"; + + public static final String OPENDISTRO_SECURITY_DLS_MODE_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"dls_mode"; + public static final String OPENDISTRO_SECURITY_DLS_MODE_TRANSIENT = OPENDISTRO_SECURITY_CONFIG_PREFIX+"dls_mode_t"; public static final String OPENDISTRO_SECURITY_FLS_FIELDS_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"fls_fields"; public static final String OPENDISTRO_SECURITY_MASKED_FIELD_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"masked_fields"; + + public static final String OPENDISTRO_SECURITY_DOC_ALLOWLIST_HEADER = OPENDISTRO_SECURITY_CONFIG_PREFIX+"doc_allowlist"; + public static final String OPENDISTRO_SECURITY_DOC_ALLOWLIST_TRANSIENT = OPENDISTRO_SECURITY_CONFIG_PREFIX+"doc_allowlist_t"; + + public static final String OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE = OPENDISTRO_SECURITY_CONFIG_PREFIX+"filter_level_dls_done"; + public static final String OPENDISTRO_SECURITY_DLS_QUERY_CCS = OPENDISTRO_SECURITY_CONFIG_PREFIX+"dls_query_ccs"; public static final String OPENDISTRO_SECURITY_FLS_FIELDS_CCS = OPENDISTRO_SECURITY_CONFIG_PREFIX+"fls_fields_ccs"; @@ -236,7 +248,7 @@ public enum RolesMappingResolution { } public static final String SECURITY_FILTER_SECURITYINDEX_FROM_ALL_REQUESTS = "plugins.security.filter_securityindex_from_all_requests"; - + public static final String SECURITY_DLS_MODE = "plugins.security.dls.mode"; // REST API public static final String SECURITY_RESTAPI_ROLES_ENABLED = "plugins.security.restapi.roles_enabled"; public static final String SECURITY_RESTAPI_ENDPOINTS_DISABLED = "plugins.security.restapi.endpoints_disabled"; diff --git a/src/main/java/org/opensearch/security/support/ReflectiveAttributeAccessors.java b/src/main/java/org/opensearch/security/support/ReflectiveAttributeAccessors.java new file mode 100644 index 0000000000..e35b789c9c --- /dev/null +++ b/src/main/java/org/opensearch/security/support/ReflectiveAttributeAccessors.java @@ -0,0 +1,157 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.support; + +import java.lang.reflect.Method; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.function.BiFunction; +import java.util.function.Function; + +import org.opensearch.SpecialPermission; + + +public class ReflectiveAttributeAccessors { + public static Function objectAttr(String name) { + return new ReflectiveAttributeGetter(name, Object.class); + } + + public static Function objectAttr(String name, Class type) { + return new ReflectiveAttributeGetter(name, type); + } + + public static Function protectedObjectAttr(String name, Class type) { + return new ProtectedReflectiveAttributeGetter(name, type); + } + + public static BiFunction setObjectAttr(String name, Class type) { + return new ReflectiveAttributeSetter(name, type); + } + + static class ReflectiveAttributeGetter implements Function { + private final String attribute; + private final String methodName; + private final Class type; + + ReflectiveAttributeGetter(String attribute, Class type) { + this.attribute = attribute; + this.methodName = "get" + attribute.substring(0, 1).toUpperCase() + attribute.substring(1); + this.type = type; + } + + @Override + public R apply(O object) { + final SecurityManager sm = System.getSecurityManager(); + + if (sm != null) { + sm.checkPermission(new SpecialPermission()); + } + + return AccessController.doPrivileged((PrivilegedAction) () -> { + if (object == null) { + return null; + } + + try { + Method method = object.getClass().getMethod(methodName); + Object value = method.invoke(object); + + return type.cast(value); + + } catch (Exception e) { + throw new RuntimeException("Error while accessing " + attribute + " in " + object, e); + } + }); + } + } + + static class ProtectedReflectiveAttributeGetter implements Function { + private final String attribute; + private final String methodName; + private final Class type; + + ProtectedReflectiveAttributeGetter(String attribute, Class type) { + this.attribute = attribute; + this.methodName = "get" + attribute.substring(0, 1).toUpperCase() + attribute.substring(1); + this.type = type; + } + + @Override + public R apply(O object) { + final SecurityManager sm = System.getSecurityManager(); + + if (sm != null) { + sm.checkPermission(new SpecialPermission()); + } + + return AccessController.doPrivileged((PrivilegedAction) () -> { + if (object == null) { + return null; + } + + try { + Method method = object.getClass().getDeclaredMethod(methodName); + method.setAccessible(true); + Object value = method.invoke(object); + + return type.cast(value); + + } catch (Exception e) { + throw new RuntimeException("Error while accessing " + attribute + " in " + object, e); + } + }); + } + } + + + static class ReflectiveAttributeSetter implements BiFunction { + private final String attribute; + private final String methodName; + private final Class type; + + ReflectiveAttributeSetter(String attribute, Class type) { + this.attribute = attribute; + this.methodName = "set" + attribute.substring(0, 1).toUpperCase() + attribute.substring(1); + this.type = type; + } + + @Override + public Void apply(O object, R value) { + final SecurityManager sm = System.getSecurityManager(); + + if (sm != null) { + sm.checkPermission(new SpecialPermission()); + } + + return AccessController.doPrivileged((PrivilegedAction) () -> { + if (object == null) { + throw new NullPointerException("Cannot set " + attribute + " because object is null"); + } + + try { + Method method = object.getClass().getMethod(methodName, type); + method.invoke(object, value); + + return null; + + } catch (Exception e) { + throw new RuntimeException("Error while set " + attribute + " in " + object + " to " + value, e); + } + }); + } + } + +} diff --git a/src/main/java/org/opensearch/security/transport/SecurityInterceptor.java b/src/main/java/org/opensearch/security/transport/SecurityInterceptor.java index fd6d309d9f..3e2608e7f3 100644 --- a/src/main/java/org/opensearch/security/transport/SecurityInterceptor.java +++ b/src/main/java/org/opensearch/security/transport/SecurityInterceptor.java @@ -71,10 +71,7 @@ import org.opensearch.security.support.ConfigConstants; import org.opensearch.security.user.User; -import org.opensearch.security.OpenSearchSecurityPlugin; -import org.opensearch.security.configuration.ClusterInfoHolder; import org.opensearch.security.ssl.transport.SSLConfig; -import org.opensearch.security.ssl.transport.PrincipalExtractor; import com.google.common.collect.Maps; @@ -148,6 +145,10 @@ public void sendRequestDecorate(AsyncSender sender || k.equals(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER) || k.equals(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER) || k.equals(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER) + || k.equals(ConfigConstants.OPENDISTRO_SECURITY_DOC_ALLOWLIST_HEADER) + || k.equals(ConfigConstants.OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE) + || k.equals(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER) + || k.equals(ConfigConstants.OPENDISTRO_SECURITY_DLS_FILTER_LEVEL_QUERY_HEADER) || (k.equals("_opendistro_security_source_field_context") && ! (request instanceof SearchRequest) && !(request instanceof GetRequest)) || k.startsWith("_opendistro_security_trace") || k.startsWith(ConfigConstants.OPENDISTRO_SECURITY_INITIAL_ACTION_CLASS_HEADER) @@ -163,8 +164,12 @@ public void sendRequestDecorate(AsyncSender sender log.debug("remove dls/fls/mf because we sent a ccs request to a remote cluster"); } headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER); + headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_DLS_MODE_HEADER); headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER); headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER); + headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_FILTER_LEVEL_DLS_DONE); + headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_DLS_FILTER_LEVEL_QUERY_HEADER); + headerMap.remove(ConfigConstants.OPENDISTRO_SECURITY_DOC_ALLOWLIST_HEADER); } if (OpenSearchSecurityPlugin.GuiceHolder.getRemoteClusterService().isCrossClusterSearchEnabled() @@ -270,33 +275,38 @@ public T read(StreamInput in) throws IOException { @Override public void handleResponse(T response) { - final List flsResponseHeader = getThreadContext().getResponseHeaders().get(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER); - final List dlsResponseHeader = getThreadContext().getResponseHeaders().get(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER); - final List maskedFieldsResponseHeader = getThreadContext().getResponseHeaders().get(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER); - + + ThreadContext threadContext = getThreadContext(); + Map> responseHeaders = threadContext.getResponseHeaders(); + + List flsResponseHeader = responseHeaders.get(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_HEADER); + List dlsResponseHeader = responseHeaders.get(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_HEADER); + List maskedFieldsResponseHeader = responseHeaders.get(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_HEADER); + contextToRestore.restore(); final boolean isDebugEnabled = log.isDebugEnabled(); - if (response instanceof ClusterSearchShardsResponse && flsResponseHeader != null && !flsResponseHeader.isEmpty()) { - if (isDebugEnabled) { - log.debug("add flsResponseHeader as transient"); + if (response instanceof ClusterSearchShardsResponse) { + if (flsResponseHeader != null && !flsResponseHeader.isEmpty()) { + if (isDebugEnabled) { + log.debug("add flsResponseHeader as transient"); + } + threadContext.putTransient(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_CCS, flsResponseHeader.get(0)); } - getThreadContext().putTransient(ConfigConstants.OPENDISTRO_SECURITY_FLS_FIELDS_CCS, flsResponseHeader.get(0)); - } - if (response instanceof ClusterSearchShardsResponse && dlsResponseHeader != null && !dlsResponseHeader.isEmpty()) { - if (isDebugEnabled) { - log.debug("add dlsResponseHeader as transient"); + if (dlsResponseHeader != null && !dlsResponseHeader.isEmpty()) { + if (isDebugEnabled) { + log.debug("add dlsResponseHeader as transient"); + } + threadContext.putTransient(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_CCS, dlsResponseHeader.get(0)); } - getThreadContext().putTransient(ConfigConstants.OPENDISTRO_SECURITY_DLS_QUERY_CCS, dlsResponseHeader.get(0)); - } - - if (response instanceof ClusterSearchShardsResponse && maskedFieldsResponseHeader != null && !maskedFieldsResponseHeader.isEmpty()) { - if (isDebugEnabled) { - log.debug("add maskedFieldsResponseHeader as transient"); + if (maskedFieldsResponseHeader != null && !maskedFieldsResponseHeader.isEmpty()) { + if (isDebugEnabled) { + log.debug("add maskedFieldsResponseHeader as transient"); + } + threadContext.putTransient(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_CCS, maskedFieldsResponseHeader.get(0)); } - getThreadContext().putTransient(ConfigConstants.OPENDISTRO_SECURITY_MASKED_FIELD_CCS, maskedFieldsResponseHeader.get(0)); } innerHandler.handleResponse(response); diff --git a/src/main/java/org/opensearch/security/transport/SecurityRequestHandler.java b/src/main/java/org/opensearch/security/transport/SecurityRequestHandler.java index 6786e4fed6..114f6002d3 100644 --- a/src/main/java/org/opensearch/security/transport/SecurityRequestHandler.java +++ b/src/main/java/org/opensearch/security/transport/SecurityRequestHandler.java @@ -109,7 +109,7 @@ protected void messageReceivedDecorate(final T request, final TransportRequestHa } if(request instanceof ConcreteShardRequest) { - resolvedActionClass = ((ConcreteShardRequest) request).getRequest().getClass().getSimpleName(); + resolvedActionClass = ((ConcreteShardRequest) request).getRequest().getClass().getSimpleName(); } String initialActionClassValue = getThreadContext().getHeader(ConfigConstants.OPENDISTRO_SECURITY_INITIAL_ACTION_CLASS_HEADER); diff --git a/src/test/java/org/opensearch/security/dlic/dlsfls/AbstractDlsFlsTest.java b/src/test/java/org/opensearch/security/dlic/dlsfls/AbstractDlsFlsTest.java index eb7d48f67d..2ea9bf4462 100644 --- a/src/test/java/org/opensearch/security/dlic/dlsfls/AbstractDlsFlsTest.java +++ b/src/test/java/org/opensearch/security/dlic/dlsfls/AbstractDlsFlsTest.java @@ -16,12 +16,25 @@ package org.opensearch.security.dlic.dlsfls; import org.opensearch.client.Client; -import org.opensearch.common.settings.Settings; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.junit.Assert; +import org.opensearch.action.get.GetResponse; +import org.opensearch.action.get.MultiGetResponse; +import org.opensearch.action.search.MultiSearchResponse; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.common.xcontent.NamedXContentRegistry; +import org.opensearch.common.xcontent.XContentParser; +import org.opensearch.common.xcontent.XContentType; import org.opensearch.security.support.ConfigConstants; import org.opensearch.security.test.DynamicSecurityConfig; import org.opensearch.security.test.SingleClusterTest; import org.opensearch.security.test.helper.rest.RestHelper; +import org.opensearch.security.test.helper.rest.RestHelper.HttpResponse; public abstract class AbstractDlsFlsTest extends SingleClusterTest { @@ -54,6 +67,54 @@ protected final void setup(Settings override, DynamicSecurityConfig dynamicSecur rh = nonSslRestHelper(); } + + protected SearchResponse executeSearch(String indexName, String user, String password) throws Exception { + HttpResponse response = rh.executeGetRequest("/"+indexName+"/_search?from=0&size=50&pretty", + encodeBasicHeader(user, password)); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + return SearchResponse.fromXContent(xcp); + } + + protected GetResponse executeGet(String indexName, String id, String user, String password) throws Exception { + HttpResponse response = rh.executeGetRequest("/"+indexName+"/_doc/"+id, encodeBasicHeader(user, password)); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + return GetResponse.fromXContent(xcp); + } + + protected MultiSearchResponse executeMSearchMatchAll(String user, String password, String ... indexName) throws Exception { + StringBuilder body = new StringBuilder(); + + for (String index : indexName) { + body.append("{\"index\": \"").append(index).append("\"}\n"); + body.append("{\"query\" : {\"match_all\" : {}}}\n"); + } + + HttpResponse response = rh.executePostRequest("/_msearch?pretty", body.toString(), + encodeBasicHeader(user, password)); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + return MultiSearchResponse.fromXContext(xcp); + } + + protected MultiGetResponse executeMGet(String user, String password, Map indicesAndIds) throws Exception { + + Set indexAndIdJson = new HashSet<>(); + for (Map.Entry indexAndId : indicesAndIds.entrySet()) { + indexAndIdJson.add("{ \"_index\": \""+indexAndId.getKey()+"\", \"_id\": \""+indexAndId.getValue()+"\" }"); + } + String body = "{ \"docs\": ["+ String.join(",", indexAndIdJson) +"] }"; + + HttpResponse response = rh.executePostRequest("/_mget?pretty", body,encodeBasicHeader(user, password)); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + return MultiGetResponse.fromXContent(xcp); + } abstract void populateData(Client tc); -} \ No newline at end of file + +} diff --git a/src/test/java/org/opensearch/security/dlic/dlsfls/DlsTermLookupQueryTest.java b/src/test/java/org/opensearch/security/dlic/dlsfls/DlsTermLookupQueryTest.java new file mode 100644 index 0000000000..1b28b963cf --- /dev/null +++ b/src/test/java/org/opensearch/security/dlic/dlsfls/DlsTermLookupQueryTest.java @@ -0,0 +1,600 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.dlic.dlsfls; + +import static org.opensearch.security.dlic.dlsfls.DlsTermsLookupAsserts.assertAccessCodesMatch; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.junit.Assert; +import org.junit.Test; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.opensearch.action.get.GetResponse; +import org.opensearch.action.get.MultiGetItemResponse; +import org.opensearch.action.get.MultiGetResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.search.MultiSearchResponse; +import org.opensearch.action.search.MultiSearchResponse.Item; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.support.WriteRequest.RefreshPolicy; +import org.opensearch.client.Client; +import org.opensearch.client.transport.TransportClient; +import org.opensearch.common.ParseField; +import org.opensearch.common.xcontent.ContextParser; +import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.common.xcontent.NamedXContentRegistry; +import org.opensearch.common.xcontent.XContentParser; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.search.SearchHit; +import org.opensearch.search.aggregations.Aggregation; +import org.opensearch.search.aggregations.Aggregations; +import org.opensearch.search.aggregations.bucket.terms.ParsedStringTerms; +import org.opensearch.search.aggregations.bucket.terms.StringTerms; +import org.opensearch.search.aggregations.bucket.terms.Terms; +import org.opensearch.search.aggregations.bucket.terms.Terms.Bucket; +import org.opensearch.search.aggregations.metrics.ParsedTopHits; +import org.opensearch.search.aggregations.metrics.TopHitsAggregationBuilder; +import org.opensearch.security.test.DynamicSecurityConfig; +import org.opensearch.security.test.helper.rest.RestHelper.HttpResponse; + +public class DlsTermLookupQueryTest extends AbstractDlsFlsTest { + + protected void populateData(Client client) { + // user access codes, basis for TLQ query + client.index(new IndexRequest("user_access_codes").id("tlq_1337").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"access_codes\": [1337] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("user_access_codes").id("tlq_42").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"access_codes\": [42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("user_access_codes").id("tlq_1337_42").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"access_codes\": [1337, 42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("user_access_codes").id("tlq_999").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"access_codes\": [999] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("user_access_codes").id("tlq_empty_access_codes") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).source("{ \"access_codes\": [] }", XContentType.JSON)) + .actionGet(); + client.index(new IndexRequest("user_access_codes").id("tlq_no_codes").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bla\": \"blub\" }", XContentType.JSON)).actionGet(); + + // need to have keyword for bu field since we're testing aggregations + client.admin().indices().create(new CreateIndexRequest("tlqdocuments")).actionGet(); + client.admin().indices() + .putMapping(new PutMappingRequest("tlqdocuments").type("_doc").source("bu", "type=keyword")) + .actionGet(); + + // tlqdocuments, protected by TLQ + client.index(new IndexRequest("tlqdocuments").id("1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"AAA\", \"access_codes\": [1337] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("2").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"AAA\", \"access_codes\": [42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("3").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"AAA\", \"access_codes\": [1337, 42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("4").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"BBB\", \"access_codes\": [1337] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("5").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"BBB\", \"access_codes\": [42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("6").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"BBB\", \"access_codes\": [1337, 42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("7").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"CCC\", \"access_codes\": [1337] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("8").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"CCC\", \"access_codes\": [42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("9").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"CCC\", \"access_codes\": [1337, 42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("10").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"DDD\", \"access_codes\": [1337] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("11").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"DDD\", \"access_codes\": [42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("12").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"DDD\", \"access_codes\": [1337, 42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("13").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"EEE\", \"access_codes\": [1337] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("14").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"EEE\", \"access_codes\": [42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("15").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"EEE\", \"access_codes\": [1337, 42] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("16").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"FFF\" }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("17").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"FFF\", \"access_codes\": [12345] }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdocuments").id("18").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"bu\": \"FFF\", \"access_codes\": [12345, 6789] }", XContentType.JSON)).actionGet(); + + // we use a "bu" field here as well to test aggregations over multiple indices + client.admin().indices().create(new CreateIndexRequest("tlqdummy")).actionGet(); + client.admin().indices().putMapping(new PutMappingRequest("tlqdummy").type("_doc").source("bu", "type=keyword")) + .actionGet(); + + // tlqdummy, not protected by TLQ + client.index(new IndexRequest("tlqdummy").id("101").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"mykey\": \"101\", \"bu\": \"GGG\" }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdummy").id("102").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"mykey\": \"102\", \"bu\": \"GGG\" }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdummy").id("103").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"mykey\": \"103\", \"bu\": \"GGG\" }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdummy").id("104").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"mykey\": \"104\", \"bu\": \"GGG\" }", XContentType.JSON)).actionGet(); + client.index(new IndexRequest("tlqdummy").id("105").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .source("{ \"mykey\": \"105\", \"bu\": \"GGG\" }", XContentType.JSON)).actionGet(); + + } + + // ------------------------ + // Test search and msearch + // ------------------------ + + @Test + public void testSimpleSearch_AccessCode_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + HttpResponse response = rh.executeGetRequest("/tlqdocuments/_search?pretty", + encodeBasicHeader("tlq_1337", "password")); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + SearchResponse searchResponse = SearchResponse.fromXContent(xcp); + // 10 docs, all need to have access code 1337 + Assert.assertEquals(searchResponse.toString(), 10, searchResponse.getHits().getTotalHits().value); + // fields need to have 1337 access code + assertAccessCodesMatch(searchResponse.getHits().getHits(), new Integer[] { 1337 }); + } + + @Test + public void testSimpleSearch_AccessCode_42() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + HttpResponse response = rh.executeGetRequest("/tlqdocuments/_search?pretty", + encodeBasicHeader("tlq_42", "password")); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + SearchResponse searchResponse = SearchResponse.fromXContent(xcp); + + // 10 docs, all need to have access code 42 + Assert.assertEquals(searchResponse.toString(), 10, searchResponse.getHits().getTotalHits().value); + // fields need to have 42 access code + assertAccessCodesMatch(searchResponse.getHits().getHits(), new Integer[] { 42 }); + + } + + @Test + public void testSimpleSearch_AccessCodes_1337_42() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + HttpResponse response = rh.executeGetRequest("/tlqdocuments/_search?pretty", + encodeBasicHeader("tlq_1337_42", "password")); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + SearchResponse searchResponse = SearchResponse.fromXContent(xcp); + + // 15 docs, all need to have either access code 1337 or 42 + Assert.assertEquals(searchResponse.toString(), 15, searchResponse.getHits().getTotalHits().value); + // fields need to have 42 or 1337 access code + assertAccessCodesMatch(searchResponse.getHits().getHits(), new Integer[] { 42, 1337 }); + + } + + @Test + public void testSimpleSearch_AccessCodes_999() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + HttpResponse response = rh.executeGetRequest("/tlqdocuments/_search?pretty", + encodeBasicHeader("tlq_999", "password")); + Assert.assertEquals(200, response.getStatusCode()); + XContentParser xcp = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, response.getBody()); + SearchResponse searchResponse = SearchResponse.fromXContent(xcp); + + Assert.assertEquals(searchResponse.toString(), 0, searchResponse.getHits().getTotalHits().value); + } + + @Test + public void testSimpleSearch_AccessCodes_emptyAccessCodes() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + SearchResponse searchResponse = executeSearch("tlqdocuments", "tlq_empty_access_codes", "password"); + Assert.assertEquals(searchResponse.toString(), 0, searchResponse.getHits().getTotalHits().value); + } + + @Test + public void testSimpleSearch_AccessCodes_noAccessCodes() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + SearchResponse searchResponse = executeSearch("tlqdocuments", "tlq_no_codes", "password"); + + Assert.assertEquals(searchResponse.toString(), 0, searchResponse.getHits().getTotalHits().value); + } + + @Test + public void testSimpleSearch_AllIndices_All_AccessCodes_1337() throws Exception { + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + SearchResponse searchResponse = executeSearch("_all", "tlq_1337", "password"); + + // assume hits from 2 indices: + // - tlqdocuments, must contain only docs with access code 1337 + // - tlqdummy, contains all documents + // no access to user_access_codes must be granted + + // check all 5 tlqdummy entries present, index is not protected by DLS + Set tlqdummyHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdummy")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 5, tlqdummyHits.size()); + + // check 10 hits with code 1337 from tlqdocuments index. All other documents + // must be filtered + Set tlqdocumentHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdocuments")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 10, tlqdocumentHits.size()); + assertAccessCodesMatch(tlqdocumentHits, new Integer[] { 1337 }); + + // check no access to user_access_codes index + Set userAccessCodesHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("user_access_codes")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 0, userAccessCodesHits.size()); + } + + @Test + public void testSimpleSearch_AllIndicesWildcard_AccessCodes_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + SearchResponse searchResponse = executeSearch("*", "tlq_1337", "password"); + + // assume hits from 2 indices: + // - tlqdocuments, must contain only docs with access code 1337 + // - tlqdummy, contains all documents + // no access to user_access_codes must be granted + + // check all 5 tlqdummy entries present, index is not protected by DLS + Set tlqdummyHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdummy")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 5, tlqdummyHits.size()); + + // check 10 hits with code 1337 from tlqdocuments index. All other documents + // must be filtered + Set tlqdocumentHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdocuments")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 10, tlqdocumentHits.size()); + assertAccessCodesMatch(tlqdocumentHits, new Integer[] { 1337 }); + + // check no access to user_access_codes index + Set userAccessCodesHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("user_access_codes")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 0, userAccessCodesHits.size()); + } + + @Test + public void testSimpleSearch_ThreeIndicesWildcard_AccessCodes_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + SearchResponse searchResponse = executeSearch("tlq*,user*", "tlq_1337", "password"); + + // assume hits from 2 indices: + // - tlqdocuments, must contain only docs with access code 1337 + // - tlqdummy, contains all documents + // no access to user_access_codes must be granted + + // check all 5 tlqdummy entries present, index is not protected by DLS + Set tlqdummyHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdummy")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 5, tlqdummyHits.size()); + + // check 10 hits with code 1337 from tlqdocuments index. All other documents + // must be filtered + Set tlqdocumentHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdocuments")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 10, tlqdocumentHits.size()); + assertAccessCodesMatch(tlqdocumentHits, new Integer[] { 1337 }); + + // check no access to user_access_codes index + Set userAccessCodesHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("user_access_codes")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 0, userAccessCodesHits.size()); + + } + + @Test + public void testSimpleSearch_TwoIndicesConcreteNames_AccessCodes_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + SearchResponse searchResponse = executeSearch("tlqdocuments,tlqdummy", "tlq_1337", "password"); + + // assume hits from 2 indices: + // - tlqdocuments, must contains only 10 docs with access code 1337 + // - tlqdummy, must contains all 5 documents + + // check all 5 tlqdummy entries present, index is not protected by DLS + Set tlqdummyHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdummy")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 5, tlqdummyHits.size()); + + // ccheck 10 hits with code 1337 from tlqdocuments index. All other documents + // must be filtered + Set tlqdocumentHits = Arrays.asList(searchResponse.getHits().getHits()).stream() + .filter((h) -> h.getIndex().equals("tlqdocuments")).collect(Collectors.toSet()); + Assert.assertEquals(searchResponse.toString(), 10, tlqdocumentHits.size()); + assertAccessCodesMatch(tlqdocumentHits, new Integer[] { 1337 }); + } + + @Test + public void testMSearch_ThreeIndices_AccessCodes_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + MultiSearchResponse searchResponse = executeMSearchMatchAll("tlq_1337", "password", "tlqdummy", "tlqdocuments", + "user_access_codes"); + + Item[] responseItems = searchResponse.getResponses(); + + // as per API order in response is the same as in the msearch request + + // check all 5 tlqdummy entries present + List tlqdummyHits = Arrays.asList(responseItems[0].getResponse().getHits().getHits()); + Assert.assertEquals(searchResponse.toString(), 5, tlqdummyHits.size()); + + // check 10 hits with code 1337 from tlqdocuments index. All other documents + // must be filtered + List tlqdocumentHits = Arrays.asList(responseItems[1].getResponse().getHits().getHits()); + Assert.assertEquals(searchResponse.toString(), 10, tlqdocumentHits.size()); + assertAccessCodesMatch(tlqdocumentHits, new Integer[] { 1337 }); + + // check no access to user_access_codes index, just two indices in the response + Assert.assertTrue(responseItems[2].getResponse() == null); + Assert.assertTrue(responseItems[2].getFailure() != null); + + } + + // ------------------------ + // Test get and mget + // ------------------------ + + @Test + public void testGet_TlqDocumentsIndex_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + // user has 1337, document has 1337 + GetResponse searchResponse = executeGet("tlqdocuments", "1", "tlq_1337", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + assertAccessCodesMatch(searchResponse.getSourceAsMap(), "access_codes", new Integer[] { 1337 }); + + // user has 1337, document has 42, not visible + searchResponse = executeGet("tlqdocuments", "2", "tlq_1337", "password"); + Assert.assertFalse(searchResponse.isExists()); + + // user has 1337, document has 42 and 1337 + searchResponse = executeGet("tlqdocuments", "3", "tlq_1337", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + assertAccessCodesMatch(searchResponse.getSourceAsMap(), "access_codes", new Integer[] { 1337 }); + + // user has 1337, document has no access codes, not visible + searchResponse = executeGet("tlqdocuments", "16", "tlq_1337", "password"); + Assert.assertFalse(searchResponse.isExists()); + + // user has 1337, document has 12345, not visible + searchResponse = executeGet("tlqdocuments", "17", "tlq_1337", "password"); + Assert.assertFalse(searchResponse.isExists()); + + // user has 1337, document has 12345 and 6789, not visible + searchResponse = executeGet("tlqdocuments", "18", "tlq_1337", "password"); + Assert.assertFalse(searchResponse.isExists()); + + } + + @Test + public void testGet_TlqDocumentsIndex_1337_42() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + // user has 1337 and 42, document has 1337 + GetResponse searchResponse = executeGet("tlqdocuments", "1", "tlq_1337_42", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + assertAccessCodesMatch(searchResponse.getSourceAsMap(), "access_codes", new Integer[] { 1337, 42 }); + + // user has 1337 and 42, document has 42 + searchResponse = executeGet("tlqdocuments", "2", "tlq_1337_42", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + assertAccessCodesMatch(searchResponse.getSourceAsMap(), "access_codes", new Integer[] { 1337, 42 }); + + // user has 1337 and 42, document has 42 and 1337 + searchResponse = executeGet("tlqdocuments", "3", "tlq_1337_42", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + assertAccessCodesMatch(searchResponse.getSourceAsMap(), "access_codes", new Integer[] { 1337, 42 }); + + // user has 1337 and 42, document has no access codes, not visible + searchResponse = executeGet("tlqdocuments", "16", "tlq_1337_42", "password"); + Assert.assertFalse(searchResponse.isExists()); + + // user has 1337 and 42, document has 12345, not visible + searchResponse = executeGet("tlqdocuments", "17", "tlq_1337_42", "password"); + Assert.assertFalse(searchResponse.isExists()); + + // user has 1337 and 42, document has 12345 and 6789, not visible + searchResponse = executeGet("tlqdocuments", "18", "tlq_1337_42", "password"); + Assert.assertFalse(searchResponse.isExists()); + + } + + @Test + public void testGet_TlqDummyIndex_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + // no restrictions on this index + GetResponse searchResponse = executeGet("tlqdummy", "101", "tlq_1337", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + + searchResponse = executeGet("tlqdummy", "102", "tlq_1337", "password"); + Assert.assertTrue(searchResponse != null); + Assert.assertTrue(searchResponse.isExists()); + + } + + @Test + public void testGet_UserAccessCodesIndex_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + // we expect a security exception here, user has no direct access to + // user_access_codes index + HttpResponse response = rh.executeGetRequest("/user_access_codes/_doc/tlq_1337", + encodeBasicHeader("tlq_1337", "password")); + Assert.assertEquals(403, response.getStatusCode()); + } + + @Test + public void testMGet_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + Map indicesAndIds = new HashMap<>(); + indicesAndIds.put("tlqdocuments", "1"); + indicesAndIds.put("tlqdocuments", "2"); + indicesAndIds.put("tlqdocuments", "3"); + indicesAndIds.put("tlqdocuments", "16"); + indicesAndIds.put("tlqdocuments", "17"); + indicesAndIds.put("tlqdocuments", "18"); + indicesAndIds.put("tlqdummy", "101"); + indicesAndIds.put("user_access_codes", "tlq_1337"); + + MultiGetResponse searchResponse = executeMGet("tlq_1337", "password", indicesAndIds); + + for (MultiGetItemResponse response : searchResponse.getResponses()) { + // no response from index "user_access_codes" + Assert.assertFalse(response.getIndex().equals("user_access_codes")); + switch (response.getIndex()) { + case "tlqdocuments": + Assert.assertTrue(response.getId(), response.getId().equals("1") | response.getId().equals("3")); + break; + case "tlqdummy": + Assert.assertTrue(response.getId(), response.getId().equals("101")); + break; + default: + Assert.fail("Index " + response.getIndex() + " present in mget response, but should not"); + } + } + } + +// ------------------------ +// Test aggregations +// ------------------------ + + @Test + public void testSimpleAggregation_tlqdocuments_AccessCode_1337() throws Exception { + + setup(new DynamicSecurityConfig().setConfig("securityconfig_tlq.yml") + .setSecurityInternalUsers("internal_users_tlq.yml").setSecurityRoles("roles_tlq.yml") + .setSecurityRolesMapping("roles_mapping_tlq.yml")); + + String body = "" + + " {\n" + + " \"aggs\": {\n" + + " \"buaggregation\": {\n" + + " \"terms\": {\n" + + " \"field\": \"bu\"\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + ""; + + // need to add typed_keys so aggregations can be parsed + // see for example: + // https://stackoverflow.com/questions/49798654/how-do-you-convert-an-elasticsearch-json-string-response-with-an-aggregation-t + HttpResponse response = rh.executePostRequest("/tlqdocuments/_search?pretty&typed_keys", body, + encodeBasicHeader("tlq_1337", "password")); + Assert.assertTrue(response.getStatusCode() == 200); + NamedXContentRegistry registry = new NamedXContentRegistry(getDefaultNamedXContents()); + XContentParser xcp = XContentType.JSON.xContent().createParser(registry, LoggingDeprecationHandler.INSTANCE, + response.getBody()); + SearchResponse searchResponse = SearchResponse.fromXContent(xcp); + + Aggregations aggs = searchResponse.getAggregations(); + Assert.assertNotNull(searchResponse.toString(), aggs); + Terms agg = aggs.get("buaggregation"); + Assert.assertTrue("Expected aggregation with name 'buaggregation'", agg != null); + // expect AAA - EEE (FFF does not match) with 2 docs each + for (String bucketName : new String[] { "AAA", "BBB", "CCC", "DDD", "EEE" }) { + Bucket bucket = agg.getBucketByKey(bucketName); + Assert.assertNotNull("Expected bucket " + bucketName + " to be present in agregations", bucket); + Assert.assertTrue("Expected doc count in bucket " + bucketName + " to be 2", bucket.getDocCount() == 2); + } + // expect FFF to be absent + Assert.assertNull("Expected bucket FFF to be absent", agg.getBucketByKey("FFF")); + } + + + public static List getDefaultNamedXContents() { + Map> map = new HashMap<>(); + map.put(TopHitsAggregationBuilder.NAME, (p, c) -> ParsedTopHits.fromXContent(p, (String) c)); + map.put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c)); + List entries = map.entrySet().stream() + .map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, new ParseField(entry.getKey()), entry.getValue())) + .collect(Collectors.toList()); + return entries; + } +} diff --git a/src/test/java/org/opensearch/security/dlic/dlsfls/DlsTermsLookupAsserts.java b/src/test/java/org/opensearch/security/dlic/dlsfls/DlsTermsLookupAsserts.java new file mode 100644 index 0000000000..c429f18471 --- /dev/null +++ b/src/test/java/org/opensearch/security/dlic/dlsfls/DlsTermsLookupAsserts.java @@ -0,0 +1,131 @@ +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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.opensearch.security.dlic.dlsfls; + +import static org.junit.Assert.assertTrue; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; + +import org.junit.Assert; +import org.opensearch.search.SearchHit; + +/* + * Copyright OpenSearch Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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. + */ +public class DlsTermsLookupAsserts { + + /** + * Asserts that the source map of a search hit contains an field that contains access codes + * and asserts those access codes contain at least on of the access codes for the user + * @param sourceMap + * @param accessCodesKey + * @param userCodes + */ + public static void assertAccessCodesMatch(Map sourceMap, String accessCodesKey, Integer[] userCodes) { + Object field = sourceMap.get(accessCodesKey); + Assert.assertTrue(sourceMap.toString(), field instanceof Collection); + Collection documentAccessCodes = (Collection) field; + // make sure the access codes in the document contain at least one code of the user access codes + Assert.assertTrue(sourceMap.toString(), documentAccessCodes.removeAll(Arrays.asList(userCodes))); + + } + + /** + * Extracts the access codes from search hits and compares them with a given collection of + * access codes. The documents access codes are retrieved for each document from the 'access_codes' field. + * This method asserts that those access codes contain at least one of the access codes for the user + * @param searchHits the search hits from the tlqdocuments index + * @param userCodes the access coded of the user + */ + public static void assertAccessCodesMatch(Collection searchHits, Integer[] userCodes) { + for (SearchHit hit : searchHits) { + assertAccessCodesMatch(hit.getSourceAsMap(), "access_codes", userCodes); + } + } + + /** + * See above + * @param searchHits + * @param userCodes + */ + public static void assertAccessCodesMatch(SearchHit[] searchHits, Integer[] userCodes) { + assertAccessCodesMatch(Arrays.asList(searchHits), userCodes); + } + + /** + * Checks whether a document from the tlqdocuments index contains a certain value in the bu field + * @param searchHit + * @param buCode + */ + public static void assertBuMatches(SearchHit searchHit, String buCode) { + Object field = searchHit.getSourceAsMap().get("bu"); + Assert.assertTrue(searchHit.toString(), field instanceof String); + Assert.assertTrue(searchHit.toString(), ((String) field).equals(buCode)); + } + + /** + * Checks whether all document from the tlqdocuments index contains a certain value in the bu field + * @param searchHit + * @param buCode + */ + public static void assertBuMatches(SearchHit[] searchHits, String buCode) { + for (SearchHit searchHit : searchHits) { + assertBuMatches(searchHit, buCode); + } + } + + /** + * Compares the cluster alias field in search hits with a given alias and fails + * if the alias name is different + * @param searchHits + * @param clusterAlias + */ + public static void assertAllHitsComeFromCluster(Collection searchHits, String clusterAlias) { + assertTrue("Expected cluster alias name to not be null", clusterAlias != null); + for (SearchHit hit : searchHits) { + assertTrue("Expected cluster alias in search hit to not be null\n" + hit, hit.getClusterAlias() != null); + assertTrue(hit.toString(), hit.getClusterAlias().equals(clusterAlias)); + } + } + + public static void assertAllHitsComeFromCluster(SearchHit[] searchHits, String clusterAlias) { + assertAllHitsComeFromCluster(Arrays.asList(searchHits), clusterAlias); + } + + public static void assertAllHitsComeFromLocalCluster(SearchHit[] searchHits) { + assertAllHitsComeFromLocalCluster(Arrays.asList(searchHits)); + } + + public static void assertAllHitsComeFromLocalCluster(Collection searchHits) { + for (SearchHit hit : searchHits) { + assertTrue(hit.toString(), hit.getClusterAlias() == null); + } + } + +} diff --git a/src/test/java/org/opensearch/security/filter/SecurityFilterTest.java b/src/test/java/org/opensearch/security/filter/SecurityFilterTest.java index c28281fc26..3454a6d2e9 100644 --- a/src/test/java/org/opensearch/security/filter/SecurityFilterTest.java +++ b/src/test/java/org/opensearch/security/filter/SecurityFilterTest.java @@ -32,6 +32,7 @@ import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.NamedXContentRegistry; import org.opensearch.threadpool.ThreadPool; import org.junit.Test; import org.junit.runner.RunWith; @@ -88,7 +89,8 @@ public void testImmutableIndicesWildcardMatcher() { mock(ClusterService.class), mock(CompatConfig.class), mock(IndexResolverReplacer.class), - mock(BackendRegistry.class) + mock(BackendRegistry.class), + mock(NamedXContentRegistry.class) ); assertEquals(expected, filter.getImmutableIndicesMatcher()); } @@ -112,7 +114,8 @@ public void testUnexepectedCausesAreNotSendToCallers() { mock(ClusterService.class), mock(CompatConfig.class), mock(IndexResolverReplacer.class), - mock(BackendRegistry.class) + mock(BackendRegistry.class), + mock(NamedXContentRegistry.class) ); // Act diff --git a/src/test/resources/dlsfls/internal_users_tlq.yml b/src/test/resources/dlsfls/internal_users_tlq.yml new file mode 100644 index 0000000000..5bbec586f0 --- /dev/null +++ b/src/test/resources/dlsfls/internal_users_tlq.yml @@ -0,0 +1,28 @@ +--- +_meta: + type: "internalusers" + config_version: 2 + +tlq_1337: + hash: "$2y$12$SP9z.rBgEHTlueKkiqSK/OxqB2PLJN/eRoNJ8WOPoHWIpirvbFAAy" # "password" + backend_roles: ["os_dls_tlq_lookup"] + +tlq_42: + hash: "$2y$12$SP9z.rBgEHTlueKkiqSK/OxqB2PLJN/eRoNJ8WOPoHWIpirvbFAAy" # "password" + backend_roles: ["os_dls_tlq_lookup"] + +tlq_1337_42: + hash: "$2y$12$SP9z.rBgEHTlueKkiqSK/OxqB2PLJN/eRoNJ8WOPoHWIpirvbFAAy" # "password" + backend_roles: ["os_dls_tlq_lookup"] + +tlq_999: + hash: "$2y$12$SP9z.rBgEHTlueKkiqSK/OxqB2PLJN/eRoNJ8WOPoHWIpirvbFAAy" # "password" + backend_roles: ["os_dls_tlq_lookup"] + +tlq_empty_access_codes: + hash: "$2y$12$SP9z.rBgEHTlueKkiqSK/OxqB2PLJN/eRoNJ8WOPoHWIpirvbFAAy" # "password" + backend_roles: ["os_dls_tlq_lookup"] + +tlq_no_codes: + hash: "$2y$12$SP9z.rBgEHTlueKkiqSK/OxqB2PLJN/eRoNJ8WOPoHWIpirvbFAAy" # "password" + backend_roles: ["os_dls_tlq_lookup"] diff --git a/src/test/resources/dlsfls/roles_mapping_tlq.yml b/src/test/resources/dlsfls/roles_mapping_tlq.yml new file mode 100644 index 0000000000..9c146360f7 --- /dev/null +++ b/src/test/resources/dlsfls/roles_mapping_tlq.yml @@ -0,0 +1,7 @@ +--- +_meta: + type: "rolesmapping" + config_version: 2 + +os_dls_tlq_lookup: + backend_roles: ["os_dls_tlq_lookup"] diff --git a/src/test/resources/dlsfls/roles_tlq.yml b/src/test/resources/dlsfls/roles_tlq.yml new file mode 100644 index 0000000000..c2d08ca948 --- /dev/null +++ b/src/test/resources/dlsfls/roles_tlq.yml @@ -0,0 +1,18 @@ +--- +_meta: + type: "roles" + config_version: 2 + +os_dls_tlq_lookup: + cluster_permissions: + - "*" + index_permissions: + - index_patterns: + - "tlqdocuments" + dls: "{ \"terms\": { \"access_codes\": { \"index\": \"user_access_codes\", \"id\": \"${user.name}\", \"path\": \"access_codes\" } } }" + allowed_actions: + - "*" + - index_patterns: + - "tlqdummy" + allowed_actions: + - "*" diff --git a/src/test/resources/dlsfls/securityconfig_tlq.yml b/src/test/resources/dlsfls/securityconfig_tlq.yml new file mode 100644 index 0000000000..02c78087ce --- /dev/null +++ b/src/test/resources/dlsfls/securityconfig_tlq.yml @@ -0,0 +1,16 @@ +--- +_meta: + type: "config" + config_version: 2 +config: + dynamic: + do_not_fail_on_forbidden: true + authc: + authentication_domain_basic_internal: + http_enabled: true + transport_enabled: true + order: 0 + http_authenticator: + type: "basic" + authentication_backend: + type: "intern"